diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 1d21fc00cab454..768588beedc5dd 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -210,9 +210,19 @@ repos: entry: "./scripts/ci/pre_commit_pylint_tests.sh" files: ^tests/.*\.py$ pass_filenames: true + - id: isort + name: Run isort to sort imports + language: python + entry: isort + files: \.py$ + # To keep consistent with the global isort skip config defined in setup.cfg + exclude: ^airflow/_vendor/.*$|^build/.*$|^.tox/.*$|^venv/.*$ + additional_dependencies: ['isort'] - id: flake8 name: Run flake8 language: system entry: "./scripts/ci/pre_commit_flake8.sh" files: \.py$ pass_filenames: true + args: + - --isort-show-traceback diff --git a/airflow/__init__.py b/airflow/__init__.py index 51a0fdc697edff..6dcf68cdcea90f 100644 --- a/airflow/__init__.py +++ b/airflow/__init__.py @@ -23,40 +23,41 @@ implement their own login mechanisms by providing an `airflow_login` module in their PYTHONPATH. airflow_login should be based off the `airflow.www.login` -""" -from typing import Optional, Callable -from airflow import version -from airflow.utils.log.logging_mixin import LoggingMixin - -__version__ = version.version - -import sys +isort:skip_file +""" # flake8: noqa: F401 +from typing import Callable, Optional + from airflow import settings +from airflow import version +from airflow.utils.log.logging_mixin import LoggingMixin from airflow.configuration import conf -from airflow.models import DAG from airflow.exceptions import AirflowException +from airflow.models import DAG + + +__version__ = version.version settings.initialize() login = None # type: Optional[Callable] +from airflow import executors +from airflow import hooks +from airflow import macros +from airflow import operators +from airflow import sensors + class AirflowMacroPlugin: def __init__(self, namespace): self.namespace = namespace -from airflow import operators # noqa: E402 -from airflow import sensors # noqa: E402 -from airflow import hooks # noqa: E402 -from airflow import executors # noqa: E402 -from airflow import macros # noqa: E402 - operators._integrate_plugins() -sensors._integrate_plugins() # noqa: E402 +sensors._integrate_plugins() hooks._integrate_plugins() executors._integrate_plugins() macros._integrate_plugins() diff --git a/airflow/api/__init__.py b/airflow/api/__init__.py index 9162b0709b2a7d..f138caaf0ae0b8 100644 --- a/airflow/api/__init__.py +++ b/airflow/api/__init__.py @@ -20,9 +20,8 @@ from importlib import import_module -from airflow.exceptions import AirflowException, AirflowConfigException from airflow.configuration import conf - +from airflow.exceptions import AirflowConfigException, AirflowException from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/api/auth/backend/default.py b/airflow/api/auth/backend/default.py index bd1ec3643f987d..637a993bdb8d62 100644 --- a/airflow/api/auth/backend/default.py +++ b/airflow/api/auth/backend/default.py @@ -17,8 +17,9 @@ # specific language governing permissions and limitations # under the License. """Default authentication backend - everything is allowed""" -from typing import Optional from functools import wraps +from typing import Optional + from airflow.typing import Protocol diff --git a/airflow/api/auth/backend/deny_all.py b/airflow/api/auth/backend/deny_all.py index 4b0134720d3f6d..bd8ad15cb2cdd9 100644 --- a/airflow/api/auth/backend/deny_all.py +++ b/airflow/api/auth/backend/deny_all.py @@ -17,11 +17,12 @@ # specific language governing permissions and limitations # under the License. """Authentication backend that denies all requests""" -from typing import Optional from functools import wraps +from typing import Optional + from flask import Response -from airflow.api.auth.backend.default import ClientAuthProtocol +from airflow.api.auth.backend.default import ClientAuthProtocol CLIENT_AUTH = None # type: Optional[ClientAuthProtocol] diff --git a/airflow/api/auth/backend/kerberos_auth.py b/airflow/api/auth/backend/kerberos_auth.py index b1bff2eb106d54..9751bda15209b4 100644 --- a/airflow/api/auth/backend/kerberos_auth.py +++ b/airflow/api/auth/backend/kerberos_auth.py @@ -42,25 +42,17 @@ # SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """Kerberos authentication module""" import os - from functools import wraps from socket import getfqdn -from flask import Response -# noinspection PyProtectedMember -from flask import _request_ctx_stack as stack # type: ignore -from flask import make_response -from flask import request -from flask import g - import kerberos - +# noinspection PyProtectedMember +from flask import Response, _request_ctx_stack as stack, g, make_response, request # type: ignore from requests_kerberos import HTTPKerberosAuth from airflow.configuration import conf from airflow.utils.log.logging_mixin import LoggingMixin - # pylint: disable=c-extension-no-member CLIENT_AUTH = HTTPKerberosAuth(service='airflow') diff --git a/airflow/api/client/local_client.py b/airflow/api/client/local_client.py index fc3c1cecea4692..37c9785740bf51 100644 --- a/airflow/api/client/local_client.py +++ b/airflow/api/client/local_client.py @@ -19,9 +19,7 @@ """Local client API""" from airflow.api.client import api_client -from airflow.api.common.experimental import pool -from airflow.api.common.experimental import trigger_dag -from airflow.api.common.experimental import delete_dag +from airflow.api.common.experimental import delete_dag, pool, trigger_dag class Client(api_client.Client): diff --git a/airflow/api/common/experimental/__init__.py b/airflow/api/common/experimental/__init__.py index 9e022e1ef26cdb..8fb319bd6def73 100644 --- a/airflow/api/common/experimental/__init__.py +++ b/airflow/api/common/experimental/__init__.py @@ -20,7 +20,7 @@ from datetime import datetime from typing import Optional -from airflow.exceptions import DagNotFound, TaskNotFound, DagRunNotFound +from airflow.exceptions import DagNotFound, DagRunNotFound, TaskNotFound from airflow.models import DagBag, DagModel, DagRun diff --git a/airflow/api/common/experimental/delete_dag.py b/airflow/api/common/experimental/delete_dag.py index 8169c18fd83ac9..b4e73c01492f04 100644 --- a/airflow/api/common/experimental/delete_dag.py +++ b/airflow/api/common/experimental/delete_dag.py @@ -21,9 +21,9 @@ from sqlalchemy import or_ from airflow import models -from airflow.models import TaskFail, DagModel -from airflow.utils.db import provide_session from airflow.exceptions import DagNotFound +from airflow.models import DagModel, TaskFail +from airflow.utils.db import provide_session @provide_session diff --git a/airflow/api/common/experimental/get_dag_runs.py b/airflow/api/common/experimental/get_dag_runs.py index 79539b34451626..cfda954f3d8bc0 100644 --- a/airflow/api/common/experimental/get_dag_runs.py +++ b/airflow/api/common/experimental/get_dag_runs.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. """DAG runs APIs.""" -from typing import Optional, List, Dict, Any +from typing import Any, Dict, List, Optional from flask import url_for diff --git a/airflow/api/common/experimental/trigger_dag.py b/airflow/api/common/experimental/trigger_dag.py index db41588e9a6683..f875d4a07c093b 100644 --- a/airflow/api/common/experimental/trigger_dag.py +++ b/airflow/api/common/experimental/trigger_dag.py @@ -19,10 +19,10 @@ """Triggering DAG runs APIs.""" import json from datetime import datetime -from typing import Union, Optional, List +from typing import List, Optional, Union -from airflow.exceptions import DagRunAlreadyExists, DagNotFound -from airflow.models import DagRun, DagBag, DagModel +from airflow.exceptions import DagNotFound, DagRunAlreadyExists +from airflow.models import DagBag, DagModel, DagRun from airflow.utils import timezone from airflow.utils.state import State diff --git a/airflow/bin/airflow b/airflow/bin/airflow index dfb60dfb650373..67e41664bb58ff 100755 --- a/airflow/bin/airflow +++ b/airflow/bin/airflow @@ -19,10 +19,11 @@ # specific language governing permissions and limitations # under the License. import os + import argcomplete -from airflow.configuration import conf from airflow.bin.cli import CLIFactory +from airflow.configuration import conf if __name__ == '__main__': diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index cebc2c6b3ae501..f2623cde95b3a3 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -17,57 +17,49 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import argparse import errno +import functools +import getpass import importlib +import json import logging - import os -import subprocess -import textwrap import random -import string -from importlib import import_module -import functools - -import getpass +import re import reprlib -import argparse -from argparse import RawTextHelpFormatter - -from airflow.utils.dot_renderer import render_dag -from airflow.utils.timezone import parse as parsedate -import json -from tabulate import tabulate - -import daemon -from daemon.pidfile import TimeoutPIDLockFile import signal +import string +import subprocess import sys +import textwrap import threading -import traceback import time -import psutil -import re -from urllib.parse import urlunparse +import traceback +from argparse import RawTextHelpFormatter +from importlib import import_module from typing import Any +from urllib.parse import urlunparse + +import daemon +import psutil +from daemon.pidfile import TimeoutPIDLockFile +from sqlalchemy.orm import exc +from tabulate import tabulate import airflow -from airflow import api -from airflow import jobs, settings +from airflow import api, jobs, settings from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowWebServerTimeout from airflow.executors import get_default_executor -from airflow.models import ( - Connection, DagModel, DagBag, DagPickle, TaskInstance, DagRun, Variable, DAG -) -from airflow.ti_deps.dep_context import (DepContext, SCHEDULER_QUEUED_DEPS) +from airflow.models import DAG, Connection, DagBag, DagModel, DagPickle, DagRun, TaskInstance, Variable +from airflow.ti_deps.dep_context import SCHEDULER_QUEUED_DEPS, DepContext from airflow.utils import cli as cli_utils, db +from airflow.utils.dot_renderer import render_dag +from airflow.utils.log.logging_mixin import LoggingMixin, redirect_stderr, redirect_stdout from airflow.utils.net import get_hostname -from airflow.utils.log.logging_mixin import (LoggingMixin, redirect_stderr, - redirect_stdout) -from airflow.www.app import cached_app, create_app, cached_appbuilder - -from sqlalchemy.orm import exc +from airflow.utils.timezone import parse as parsedate +from airflow.www.app import cached_app, cached_appbuilder, create_app api.load_auth() api_module = import_module(conf.get('cli', 'api_client')) # type: Any diff --git a/airflow/config_templates/airflow_local_settings.py b/airflow/config_templates/airflow_local_settings.py index 3e97c03dfd0027..754b2470ada4be 100644 --- a/airflow/config_templates/airflow_local_settings.py +++ b/airflow/config_templates/airflow_local_settings.py @@ -18,7 +18,7 @@ # under the License. import os -from typing import Dict, Any +from typing import Any, Dict from airflow.configuration import conf from airflow.utils.file import mkdirs diff --git a/airflow/config_templates/default_webserver_config.py b/airflow/config_templates/default_webserver_config.py index bc8a6bb1fb6131..65f775842ab807 100644 --- a/airflow/config_templates/default_webserver_config.py +++ b/airflow/config_templates/default_webserver_config.py @@ -18,13 +18,16 @@ # under the License. """Default configuration for the Airflow webserver""" import os + from flask_appbuilder.security.manager import AUTH_DB + +from airflow.configuration import conf + # from flask_appbuilder.security.manager import AUTH_LDAP # from flask_appbuilder.security.manager import AUTH_OAUTH # from flask_appbuilder.security.manager import AUTH_OID # from flask_appbuilder.security.manager import AUTH_REMOTE_USER -from airflow.configuration import conf basedir = os.path.abspath(os.path.dirname(__file__)) diff --git a/airflow/configuration.py b/airflow/configuration.py index b36062d19c5c01..3ed6ad828214de 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -17,8 +17,6 @@ # specific language governing permissions and limitations # under the License. -from base64 import b64encode -from collections import OrderedDict import copy import os import pathlib @@ -26,8 +24,10 @@ import subprocess import sys import warnings +from base64 import b64encode +from collections import OrderedDict # Ignored Mypy on configparser because it thinks the configparser module has no _UNSET attribute -from configparser import ConfigParser, _UNSET, NoOptionError, NoSectionError # type: ignore +from configparser import _UNSET, ConfigParser, NoOptionError, NoSectionError # type: ignore from zope.deprecation import deprecated diff --git a/airflow/contrib/auth/backends/github_enterprise_auth.py b/airflow/contrib/auth/backends/github_enterprise_auth.py index 0426a00f9b50d8..21d9bb8ed8d5b4 100644 --- a/airflow/contrib/auth/backends/github_enterprise_auth.py +++ b/airflow/contrib/auth/backends/github_enterprise_auth.py @@ -17,13 +17,10 @@ # specific language governing permissions and limitations # under the License. import flask_login - +from flask import redirect, request, url_for # Need to expose these downstream # flake8: noqa: F401 -from flask_login import current_user, logout_user, login_required, login_user - -from flask import url_for, redirect, request - +from flask_login import current_user, login_required, login_user, logout_user from flask_oauthlib.client import OAuth from airflow import models diff --git a/airflow/contrib/auth/backends/google_auth.py b/airflow/contrib/auth/backends/google_auth.py index 8179c7448ecb94..896826b985c7c2 100644 --- a/airflow/contrib/auth/backends/google_auth.py +++ b/airflow/contrib/auth/backends/google_auth.py @@ -17,13 +17,10 @@ # specific language governing permissions and limitations # under the License. import flask_login - +from flask import redirect, request, url_for # Need to expose these downstream # flake8: noqa: F401 -from flask_login import current_user, logout_user, login_required, login_user - -from flask import url_for, redirect, request - +from flask_login import current_user, login_required, login_user, logout_user from flask_oauthlib.client import OAuth from airflow import models diff --git a/airflow/contrib/auth/backends/kerberos_auth.py b/airflow/contrib/auth/backends/kerberos_auth.py index 7f67848c096e65..e4c125e8b61a6a 100644 --- a/airflow/contrib/auth/backends/kerberos_auth.py +++ b/airflow/contrib/auth/backends/kerberos_auth.py @@ -18,22 +18,20 @@ # under the License. """Kerberos authentication module""" import logging -import flask_login -from airflow.exceptions import AirflowConfigException -from flask_login import current_user -from flask import flash -from wtforms import Form, PasswordField, StringField -from wtforms.validators import InputRequired +import flask_login # pykerberos should be used as it verifies the KDC, the "kerberos" module does not do so # and make it possible to spoof the KDC import kerberos -from airflow.security import utils - -from flask import url_for, redirect +from flask import flash, redirect, url_for +from flask_login import current_user +from wtforms import Form, PasswordField, StringField +from wtforms.validators import InputRequired from airflow import models from airflow.configuration import conf +from airflow.exceptions import AirflowConfigException +from airflow.security import utils from airflow.utils.db import provide_session from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/contrib/auth/backends/ldap_auth.py b/airflow/contrib/auth/backends/ldap_auth.py index eeadccaffef5b7..fd2ddeb2b2eb9c 100644 --- a/airflow/contrib/auth/backends/ldap_auth.py +++ b/airflow/contrib/auth/backends/ldap_auth.py @@ -17,25 +17,20 @@ # specific language governing permissions and limitations # under the License. +import re +import ssl +import traceback + import flask_login -from flask_login import login_required, current_user, logout_user # noqa: F401 -from flask import flash +from flask import flash, redirect, url_for +from flask_login import current_user, login_required, logout_user # noqa: F401 +from ldap3 import LEVEL, SUBTREE, Connection, Server, Tls, set_config_parameter from wtforms import Form, PasswordField, StringField from wtforms.validators import InputRequired -from ldap3 import Server, Connection, Tls, set_config_parameter, LEVEL, SUBTREE -import ssl - -from flask import url_for, redirect - from airflow import models -from airflow.configuration import conf -from airflow.configuration import AirflowConfigException +from airflow.configuration import AirflowConfigException, conf from airflow.utils.db import provide_session - -import traceback -import re - from airflow.utils.log.logging_mixin import LoggingMixin login_manager = flask_login.LoginManager() diff --git a/airflow/contrib/auth/backends/password_auth.py b/airflow/contrib/auth/backends/password_auth.py index 53deb3583057ee..194e2a0cec6a5c 100644 --- a/airflow/contrib/auth/backends/password_auth.py +++ b/airflow/contrib/auth/backends/password_auth.py @@ -20,24 +20,19 @@ import base64 from functools import wraps -from flask import flash, Response -from flask import url_for, redirect, make_response -from flask_bcrypt import generate_password_hash, check_password_hash - import flask_login +from flask import Response, flash, make_response, redirect, url_for +from flask_bcrypt import check_password_hash, generate_password_hash # noinspection PyUnresolvedReferences # pylint: disable=unused-import -from flask_login import login_required, current_user, logout_user # noqa: F401 -# pylint: enable=unused-import - -from wtforms import Form, PasswordField, StringField -from wtforms.validators import InputRequired - +from flask_login import current_user, login_required, logout_user # noqa: F401 from sqlalchemy import Column, String from sqlalchemy.ext.hybrid import hybrid_property +from wtforms import Form, PasswordField, StringField +from wtforms.validators import InputRequired from airflow import models -from airflow.utils.db import provide_session, create_session +from airflow.utils.db import create_session, provide_session from airflow.utils.log.logging_mixin import LoggingMixin LOGIN_MANAGER = flask_login.LoginManager() diff --git a/airflow/contrib/example_dags/example_azure_cosmosdb_sensor.py b/airflow/contrib/example_dags/example_azure_cosmosdb_sensor.py index c4e7035358ce34..38beca90b97b7b 100644 --- a/airflow/contrib/example_dags/example_azure_cosmosdb_sensor.py +++ b/airflow/contrib/example_dags/example_azure_cosmosdb_sensor.py @@ -28,8 +28,8 @@ """ from airflow import DAG -from airflow.contrib.sensors.azure_cosmos_sensor import AzureCosmosDocumentSensor from airflow.contrib.operators.azure_cosmos_operator import AzureCosmosInsertDocumentOperator +from airflow.contrib.sensors.azure_cosmos_sensor import AzureCosmosDocumentSensor from airflow.utils import dates default_args = { diff --git a/airflow/contrib/example_dags/example_databricks_operator.py b/airflow/contrib/example_dags/example_databricks_operator.py index 262edbdc2c3250..93989f81e6cf60 100644 --- a/airflow/contrib/example_dags/example_databricks_operator.py +++ b/airflow/contrib/example_dags/example_databricks_operator.py @@ -33,11 +33,9 @@ """ import airflow - from airflow import DAG from airflow.contrib.operators.databricks_operator import DatabricksSubmitRunOperator - default_args = { 'owner': 'Airflow', 'email': ['airflow@example.com'], diff --git a/airflow/contrib/example_dags/example_dingding_operator.py b/airflow/contrib/example_dags/example_dingding_operator.py index d67ad6af2c7bbf..a9639f884d9333 100644 --- a/airflow/contrib/example_dags/example_dingding_operator.py +++ b/airflow/contrib/example_dags/example_dingding_operator.py @@ -25,7 +25,6 @@ from airflow import DAG from airflow.contrib.operators.dingding_operator import DingdingOperator - args = { 'owner': 'Airflow', 'retries': 3, diff --git a/airflow/contrib/example_dags/example_emr_job_flow_automatic_steps.py b/airflow/contrib/example_dags/example_emr_job_flow_automatic_steps.py index 6e6a4ee7c9fa31..03f19e0f770767 100644 --- a/airflow/contrib/example_dags/example_emr_job_flow_automatic_steps.py +++ b/airflow/contrib/example_dags/example_emr_job_flow_automatic_steps.py @@ -20,10 +20,10 @@ This is an example dag for a AWS EMR Pipeline with auto steps. """ from datetime import timedelta + import airflow from airflow import DAG -from airflow.contrib.operators.emr_create_job_flow_operator \ - import EmrCreateJobFlowOperator +from airflow.contrib.operators.emr_create_job_flow_operator import EmrCreateJobFlowOperator from airflow.contrib.sensors.emr_job_flow_sensor import EmrJobFlowSensor DEFAULT_ARGS = { diff --git a/airflow/contrib/example_dags/example_emr_job_flow_manual_steps.py b/airflow/contrib/example_dags/example_emr_job_flow_manual_steps.py index 6eaa4e5ec62adc..b03dd92369c6f3 100644 --- a/airflow/contrib/example_dags/example_emr_job_flow_manual_steps.py +++ b/airflow/contrib/example_dags/example_emr_job_flow_manual_steps.py @@ -26,13 +26,10 @@ import airflow from airflow import DAG -from airflow.contrib.operators.emr_create_job_flow_operator \ - import EmrCreateJobFlowOperator -from airflow.contrib.operators.emr_add_steps_operator \ - import EmrAddStepsOperator +from airflow.contrib.operators.emr_add_steps_operator import EmrAddStepsOperator +from airflow.contrib.operators.emr_create_job_flow_operator import EmrCreateJobFlowOperator +from airflow.contrib.operators.emr_terminate_job_flow_operator import EmrTerminateJobFlowOperator from airflow.contrib.sensors.emr_step_sensor import EmrStepSensor -from airflow.contrib.operators.emr_terminate_job_flow_operator \ - import EmrTerminateJobFlowOperator DEFAULT_ARGS = { 'owner': 'Airflow', diff --git a/airflow/contrib/example_dags/example_gcs_to_gdrive.py b/airflow/contrib/example_dags/example_gcs_to_gdrive.py index f6b185953c5dd9..afd04c10ac7a12 100644 --- a/airflow/contrib/example_dags/example_gcs_to_gdrive.py +++ b/airflow/contrib/example_dags/example_gcs_to_gdrive.py @@ -23,7 +23,6 @@ import airflow from airflow import models - from airflow.contrib.operators.gcs_to_gdrive_operator import GcsToGDriveOperator GCS_TO_GDRIVE_BUCKET = os.environ.get("GCS_TO_DRIVE_BUCKET", "example-object") diff --git a/airflow/contrib/example_dags/example_kubernetes_executor_config.py b/airflow/contrib/example_dags/example_kubernetes_executor_config.py index 498cf6e54198f0..e235777a696665 100644 --- a/airflow/contrib/example_dags/example_kubernetes_executor_config.py +++ b/airflow/contrib/example_dags/example_kubernetes_executor_config.py @@ -26,7 +26,6 @@ from airflow.models import DAG from airflow.operators.python_operator import PythonOperator - default_args = { 'owner': 'Airflow', 'start_date': airflow.utils.dates.days_ago(2) diff --git a/airflow/contrib/example_dags/example_kubernetes_operator.py b/airflow/contrib/example_dags/example_kubernetes_operator.py index e9453356bcff0f..ce3c40cf8475e7 100644 --- a/airflow/contrib/example_dags/example_kubernetes_operator.py +++ b/airflow/contrib/example_dags/example_kubernetes_operator.py @@ -19,9 +19,9 @@ """ This is an example dag for using the KubernetesPodOperator. """ +from airflow.models import DAG from airflow.utils.dates import days_ago from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.models import DAG log = LoggingMixin().log diff --git a/airflow/contrib/example_dags/example_papermill_operator.py b/airflow/contrib/example_dags/example_papermill_operator.py index 8710538d4a5f18..6840bfd2f04761 100644 --- a/airflow/contrib/example_dags/example_papermill_operator.py +++ b/airflow/contrib/example_dags/example_papermill_operator.py @@ -25,11 +25,9 @@ from datetime import timedelta import airflow - from airflow.models import DAG from airflow.operators.papermill_operator import PapermillOperator - default_args = { 'owner': 'Airflow', 'start_date': airflow.utils.dates.days_ago(2) diff --git a/airflow/contrib/example_dags/example_qubole_operator.py b/airflow/contrib/example_dags/example_qubole_operator.py index 5d48fb6aca5dae..97ed61d2d3f6ee 100644 --- a/airflow/contrib/example_dags/example_qubole_operator.py +++ b/airflow/contrib/example_dags/example_qubole_operator.py @@ -35,7 +35,7 @@ from airflow import DAG from airflow.contrib.operators.qubole_operator import QuboleOperator from airflow.operators.dummy_operator import DummyOperator -from airflow.operators.python_operator import PythonOperator, BranchPythonOperator +from airflow.operators.python_operator import BranchPythonOperator, PythonOperator default_args = { 'owner': 'Airflow', diff --git a/airflow/contrib/example_dags/example_twitter_dag.py b/airflow/contrib/example_dags/example_twitter_dag.py index 648e2fbe7d402d..f7050d4c1a884b 100644 --- a/airflow/contrib/example_dags/example_twitter_dag.py +++ b/airflow/contrib/example_dags/example_twitter_dag.py @@ -37,7 +37,6 @@ from airflow.operators.hive_operator import HiveOperator from airflow.operators.python_operator import PythonOperator - # -------------------------------------------------------------------------------- # Create a few placeholder scripts. In practice these would be different python # script files, which are imported in this section with absolute or relative imports diff --git a/airflow/contrib/example_dags/example_winrm_operator.py b/airflow/contrib/example_dags/example_winrm_operator.py index df28d9e2e576bd..e3fd907fa3583a 100644 --- a/airflow/contrib/example_dags/example_winrm_operator.py +++ b/airflow/contrib/example_dags/example_winrm_operator.py @@ -37,7 +37,6 @@ from airflow.models import DAG from airflow.operators.dummy_operator import DummyOperator - default_args = { 'owner': 'Airflow', 'start_date': airflow.utils.dates.days_ago(2) diff --git a/airflow/contrib/hooks/aws_athena_hook.py b/airflow/contrib/hooks/aws_athena_hook.py index 4c23624dd9b651..0ebdeee40ad4ae 100644 --- a/airflow/contrib/hooks/aws_athena_hook.py +++ b/airflow/contrib/hooks/aws_athena_hook.py @@ -21,6 +21,7 @@ This module contains AWS Athena hook """ from time import sleep + from airflow.contrib.hooks.aws_hook import AwsHook diff --git a/airflow/contrib/hooks/aws_dynamodb_hook.py b/airflow/contrib/hooks/aws_dynamodb_hook.py index 03c5546d672797..53aff50e7bf2e2 100644 --- a/airflow/contrib/hooks/aws_dynamodb_hook.py +++ b/airflow/contrib/hooks/aws_dynamodb_hook.py @@ -21,8 +21,8 @@ """ This module contains AWS Athena hook """ -from airflow.exceptions import AirflowException from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.exceptions import AirflowException class AwsDynamoDBHook(AwsHook): diff --git a/airflow/contrib/hooks/aws_hook.py b/airflow/contrib/hooks/aws_hook.py index cf07839b28cd61..714c8a883ccc10 100644 --- a/airflow/contrib/hooks/aws_hook.py +++ b/airflow/contrib/hooks/aws_hook.py @@ -21,8 +21,9 @@ This module contains Base AWS Hook """ -import logging import configparser +import logging + import boto3 from airflow.exceptions import AirflowException diff --git a/airflow/contrib/hooks/azure_container_instance_hook.py b/airflow/contrib/hooks/azure_container_instance_hook.py index 81bb4c316e99a6..af329ccb7bd5ec 100644 --- a/airflow/contrib/hooks/azure_container_instance_hook.py +++ b/airflow/contrib/hooks/azure_container_instance_hook.py @@ -20,15 +20,14 @@ import os -from airflow.hooks.base_hook import BaseHook -from airflow.exceptions import AirflowException - from azure.common.client_factory import get_client_from_auth_file from azure.common.credentials import ServicePrincipalCredentials - from azure.mgmt.containerinstance import ContainerInstanceManagementClient from zope.deprecation import deprecation +from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook + class AzureContainerInstanceHook(BaseHook): """ diff --git a/airflow/contrib/hooks/azure_container_volume_hook.py b/airflow/contrib/hooks/azure_container_volume_hook.py index c81c5190776c60..62cd62da83e49f 100644 --- a/airflow/contrib/hooks/azure_container_volume_hook.py +++ b/airflow/contrib/hooks/azure_container_volume_hook.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. +from azure.mgmt.containerinstance.models import AzureFileVolume, Volume + from airflow.hooks.base_hook import BaseHook -from azure.mgmt.containerinstance.models import (Volume, - AzureFileVolume) class AzureContainerVolumeHook(BaseHook): diff --git a/airflow/contrib/hooks/azure_cosmos_hook.py b/airflow/contrib/hooks/azure_cosmos_hook.py index 8a3cc41d51805d..d0eed82d4e7f7b 100644 --- a/airflow/contrib/hooks/azure_cosmos_hook.py +++ b/airflow/contrib/hooks/azure_cosmos_hook.py @@ -24,9 +24,10 @@ login (=Endpoint uri), password (=secret key) and extra fields database_name and collection_name to specify the default database and collection to use (see connection `azure_cosmos_default` for an example). """ +import uuid + import azure.cosmos.cosmos_client as cosmos_client from azure.cosmos.errors import HTTPFailure -import uuid from airflow.exceptions import AirflowBadRequest from airflow.hooks.base_hook import BaseHook diff --git a/airflow/contrib/hooks/azure_data_lake_hook.py b/airflow/contrib/hooks/azure_data_lake_hook.py index 186cae5d1d4858..bbb9c253c9afa1 100644 --- a/airflow/contrib/hooks/azure_data_lake_hook.py +++ b/airflow/contrib/hooks/azure_data_lake_hook.py @@ -25,9 +25,10 @@ login (=Client ID), password (=Client Secret) and extra fields tenant (Tenant) and account_name (Account Name) (see connection `azure_data_lake_default` for an example). """ -from airflow.hooks.base_hook import BaseHook from azure.datalake.store import core, lib, multithread +from airflow.hooks.base_hook import BaseHook + class AzureDataLakeHook(BaseHook): """ diff --git a/airflow/contrib/hooks/azure_fileshare_hook.py b/airflow/contrib/hooks/azure_fileshare_hook.py index 8afa1540d78b41..545a8d3735cc74 100644 --- a/airflow/contrib/hooks/azure_fileshare_hook.py +++ b/airflow/contrib/hooks/azure_fileshare_hook.py @@ -18,9 +18,10 @@ # under the License. # -from airflow.hooks.base_hook import BaseHook from azure.storage.file import FileService +from airflow.hooks.base_hook import BaseHook + class AzureFileShareHook(BaseHook): """ diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 23dc04b9abaaa3..68bf566e3d6753 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -22,11 +22,7 @@ # pylint: disable=unused-import from airflow.gcp.hooks.bigquery import ( # noqa - BigQueryPandasConnector, - BigQueryCursor, - BigQueryConnection, - BigQueryHook, - BigQueryBaseCursor, + BigQueryBaseCursor, BigQueryConnection, BigQueryCursor, BigQueryHook, BigQueryPandasConnector, GbqConnector, ) diff --git a/airflow/contrib/hooks/cassandra_hook.py b/airflow/contrib/hooks/cassandra_hook.py index 08def3a5b775cc..c159b8741e2e6b 100644 --- a/airflow/contrib/hooks/cassandra_hook.py +++ b/airflow/contrib/hooks/cassandra_hook.py @@ -17,10 +17,11 @@ # specific language governing permissions and limitations # under the License. -from cassandra.cluster import Cluster -from cassandra.policies import (RoundRobinPolicy, DCAwareRoundRobinPolicy, - TokenAwarePolicy, WhiteListRoundRobinPolicy) from cassandra.auth import PlainTextAuthProvider +from cassandra.cluster import Cluster +from cassandra.policies import ( + DCAwareRoundRobinPolicy, RoundRobinPolicy, TokenAwarePolicy, WhiteListRoundRobinPolicy, +) from airflow.hooks.base_hook import BaseHook from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py index b805f6aa6bbd29..b284fb171a2e42 100644 --- a/airflow/contrib/hooks/databricks_hook.py +++ b/airflow/contrib/hooks/databricks_hook.py @@ -23,8 +23,8 @@ operators talk to the ``api/2.0/jobs/runs/submit`` `endpoint `_. """ -from urllib.parse import urlparse from time import sleep +from urllib.parse import urlparse import requests from requests import exceptions as requests_exceptions diff --git a/airflow/contrib/hooks/datadog_hook.py b/airflow/contrib/hooks/datadog_hook.py index 50209dd112e066..21fb415c63a41f 100644 --- a/airflow/contrib/hooks/datadog_hook.py +++ b/airflow/contrib/hooks/datadog_hook.py @@ -18,10 +18,11 @@ # under the License. import time -from airflow.hooks.base_hook import BaseHook -from airflow.exceptions import AirflowException -from datadog import initialize, api +from datadog import api, initialize + +from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/contrib/hooks/discord_webhook_hook.py b/airflow/contrib/hooks/discord_webhook_hook.py index 0789553de5bc97..1b3c1aa7d92438 100644 --- a/airflow/contrib/hooks/discord_webhook_hook.py +++ b/airflow/contrib/hooks/discord_webhook_hook.py @@ -20,8 +20,8 @@ import json import re -from airflow.hooks.http_hook import HttpHook from airflow.exceptions import AirflowException +from airflow.hooks.http_hook import HttpHook class DiscordWebhookHook(HttpHook): diff --git a/airflow/contrib/hooks/emr_hook.py b/airflow/contrib/hooks/emr_hook.py index 7bf90ac1b68694..52232f3a294b7e 100644 --- a/airflow/contrib/hooks/emr_hook.py +++ b/airflow/contrib/hooks/emr_hook.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from airflow.exceptions import AirflowException from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.exceptions import AirflowException class EmrHook(AwsHook): diff --git a/airflow/contrib/hooks/ftp_hook.py b/airflow/contrib/hooks/ftp_hook.py index 75192dcc15045d..d02acff59d56a6 100644 --- a/airflow/contrib/hooks/ftp_hook.py +++ b/airflow/contrib/hooks/ftp_hook.py @@ -21,6 +21,7 @@ import datetime import ftplib import os.path + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py index ac3a92ad728e50..e37d71b09a1961 100644 --- a/airflow/contrib/hooks/gcp_dataflow_hook.py +++ b/airflow/contrib/hooks/gcp_dataflow_hook.py @@ -21,11 +21,7 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.hooks.dataflow import ( # noqa - DataFlowHook, - DataflowJobStatus, - DEFAULT_DATAFLOW_LOCATION -) +from airflow.gcp.hooks.dataflow import DEFAULT_DATAFLOW_LOCATION, DataFlowHook, DataflowJobStatus # noqa warnings.warn( "This module is deprecated. Please use `airflow.gcp.hooks.dataflow`.", diff --git a/airflow/contrib/hooks/gcp_dataproc_hook.py b/airflow/contrib/hooks/gcp_dataproc_hook.py index bc993837b4d0d9..4a0092bf25205a 100644 --- a/airflow/contrib/hooks/gcp_dataproc_hook.py +++ b/airflow/contrib/hooks/gcp_dataproc_hook.py @@ -21,7 +21,7 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.hooks.dataproc import DataprocJobStatus, DataProcHook # noqa +from airflow.gcp.hooks.dataproc import DataProcHook, DataprocJobStatus # noqa warnings.warn( "This module is deprecated. Please use `airflow.gcp.hooks.dataproc`.", diff --git a/airflow/contrib/hooks/gcp_dlp_hook.py b/airflow/contrib/hooks/gcp_dlp_hook.py index 74424075fb8f88..f6eccc85c6b820 100644 --- a/airflow/contrib/hooks/gcp_dlp_hook.py +++ b/airflow/contrib/hooks/gcp_dlp_hook.py @@ -21,7 +21,7 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.hooks.dlp import DlpJob, CloudDLPHook # noqa +from airflow.gcp.hooks.dlp import CloudDLPHook, DlpJob # noqa warnings.warn( "This module is deprecated. Please use `airflow.gcp.hooks.dlp`.", diff --git a/airflow/contrib/hooks/gcp_pubsub_hook.py b/airflow/contrib/hooks/gcp_pubsub_hook.py index 2231450c592aa3..ad5ecf5c79a6c1 100644 --- a/airflow/contrib/hooks/gcp_pubsub_hook.py +++ b/airflow/contrib/hooks/gcp_pubsub_hook.py @@ -21,7 +21,7 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.hooks.pubsub import PubSubHook, PubSubException # noqa +from airflow.gcp.hooks.pubsub import PubSubException, PubSubHook # noqa warnings.warn( "This module is deprecated. Please use `airflow.gcp.hooks.pubsub`.", diff --git a/airflow/contrib/hooks/gcp_transfer_hook.py b/airflow/contrib/hooks/gcp_transfer_hook.py index 43e69441af4241..5bdd9315544284 100644 --- a/airflow/contrib/hooks/gcp_transfer_hook.py +++ b/airflow/contrib/hooks/gcp_transfer_hook.py @@ -24,9 +24,7 @@ # pylint: disable=unused-import from airflow.gcp.hooks.cloud_storage_transfer_service import ( # noqa - GCPTransferServiceHook, - GcpTransferJobsStatus, - GcpTransferOperationStatus, + GcpTransferJobsStatus, GcpTransferOperationStatus, GCPTransferServiceHook, ) warnings.warn( diff --git a/airflow/contrib/hooks/grpc_hook.py b/airflow/contrib/hooks/grpc_hook.py index 664ed70b7372af..b008ab8ad3c5bf 100644 --- a/airflow/contrib/hooks/grpc_hook.py +++ b/airflow/contrib/hooks/grpc_hook.py @@ -22,11 +22,12 @@ import grpc from google import auth as google_auth from google.auth import jwt as google_auth_jwt -from google.auth.transport import grpc as google_auth_transport_grpc -from google.auth.transport import requests as google_auth_transport_requests +from google.auth.transport import ( + grpc as google_auth_transport_grpc, requests as google_auth_transport_requests, +) -from airflow.hooks.base_hook import BaseHook from airflow.exceptions import AirflowConfigException +from airflow.hooks.base_hook import BaseHook class GrpcHook(BaseHook): diff --git a/airflow/contrib/hooks/imap_hook.py b/airflow/contrib/hooks/imap_hook.py index 6b531089f2c5e0..2c8a2a29cc8c7e 100644 --- a/airflow/contrib/hooks/imap_hook.py +++ b/airflow/contrib/hooks/imap_hook.py @@ -26,7 +26,7 @@ import os import re -from airflow import LoggingMixin, AirflowException +from airflow import AirflowException, LoggingMixin from airflow.hooks.base_hook import BaseHook diff --git a/airflow/contrib/hooks/jenkins_hook.py b/airflow/contrib/hooks/jenkins_hook.py index 626fd1fe9e3795..a5a6cd5ac0edaf 100644 --- a/airflow/contrib/hooks/jenkins_hook.py +++ b/airflow/contrib/hooks/jenkins_hook.py @@ -18,10 +18,11 @@ # under the License. # -from airflow.hooks.base_hook import BaseHook +from distutils.util import strtobool import jenkins -from distutils.util import strtobool + +from airflow.hooks.base_hook import BaseHook class JenkinsHook(BaseHook): diff --git a/airflow/contrib/hooks/mongo_hook.py b/airflow/contrib/hooks/mongo_hook.py index 6ab96065d9a08e..520feff865f209 100644 --- a/airflow/contrib/hooks/mongo_hook.py +++ b/airflow/contrib/hooks/mongo_hook.py @@ -18,6 +18,7 @@ # under the License. """Hook for Mongo DB""" from ssl import CERT_NONE + from pymongo import MongoClient, ReplaceOne from airflow.hooks.base_hook import BaseHook diff --git a/airflow/contrib/hooks/openfaas_hook.py b/airflow/contrib/hooks/openfaas_hook.py index 9581f4432b82c6..d444ae9c041de4 100644 --- a/airflow/contrib/hooks/openfaas_hook.py +++ b/airflow/contrib/hooks/openfaas_hook.py @@ -17,9 +17,10 @@ # specific language governing permissions and limitations # under the License. -from airflow.hooks.base_hook import BaseHook import requests + from airflow import AirflowException +from airflow.hooks.base_hook import BaseHook OK_STATUS_CODE = 202 diff --git a/airflow/contrib/hooks/opsgenie_alert_hook.py b/airflow/contrib/hooks/opsgenie_alert_hook.py index 12efe1e13f29f5..d66bead6081348 100644 --- a/airflow/contrib/hooks/opsgenie_alert_hook.py +++ b/airflow/contrib/hooks/opsgenie_alert_hook.py @@ -22,8 +22,8 @@ import requests -from airflow.hooks.http_hook import HttpHook from airflow import AirflowException +from airflow.hooks.http_hook import HttpHook class OpsgenieAlertHook(HttpHook): diff --git a/airflow/contrib/hooks/qubole_check_hook.py b/airflow/contrib/hooks/qubole_check_hook.py index 9fe2e27aa776c4..118fbe4a3ab7ba 100644 --- a/airflow/contrib/hooks/qubole_check_hook.py +++ b/airflow/contrib/hooks/qubole_check_hook.py @@ -20,11 +20,11 @@ from io import StringIO -from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.contrib.hooks.qubole_hook import QuboleHook -from airflow.exceptions import AirflowException from qds_sdk.commands import Command +from airflow.contrib.hooks.qubole_hook import QuboleHook +from airflow.exceptions import AirflowException +from airflow.utils.log.logging_mixin import LoggingMixin COL_DELIM = '\t' ROW_DELIM = '\r\n' diff --git a/airflow/contrib/hooks/qubole_hook.py b/airflow/contrib/hooks/qubole_hook.py index 8a87c04aca22a9..b9f9dbf0e22052 100644 --- a/airflow/contrib/hooks/qubole_hook.py +++ b/airflow/contrib/hooks/qubole_hook.py @@ -18,23 +18,24 @@ # under the License. # """Qubole hook""" +import datetime import os import pathlib -import time -import datetime import re +import time +from qds_sdk.commands import ( + Command, DbExportCommand, DbImportCommand, DbTapQueryCommand, HadoopCommand, HiveCommand, PigCommand, + PrestoCommand, ShellCommand, SparkCommand, SqlCommand, +) from qds_sdk.qubole import Qubole -from qds_sdk.commands import Command, HiveCommand, PrestoCommand, HadoopCommand, \ - PigCommand, ShellCommand, SparkCommand, DbTapQueryCommand, DbExportCommand, \ - DbImportCommand, SqlCommand +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.hooks.base_hook import BaseHook -from airflow.configuration import conf +from airflow.models import TaskInstance from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State -from airflow.models import TaskInstance COMMAND_CLASSES = { "hivecmd": HiveCommand, diff --git a/airflow/contrib/hooks/redis_hook.py b/airflow/contrib/hooks/redis_hook.py index 3ef7e87003cbe8..4785e9485fa9c0 100644 --- a/airflow/contrib/hooks/redis_hook.py +++ b/airflow/contrib/hooks/redis_hook.py @@ -21,6 +21,7 @@ RedisHook module """ from redis import Redis + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/contrib/hooks/sagemaker_hook.py b/airflow/contrib/hooks/sagemaker_hook.py index 097e6b4a75cb12..0f2b2d95e73594 100644 --- a/airflow/contrib/hooks/sagemaker_hook.py +++ b/airflow/contrib/hooks/sagemaker_hook.py @@ -16,18 +16,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import collections +import os import tarfile import tempfile import time -import os -import collections import warnings from botocore.exceptions import ClientError -from airflow.exceptions import AirflowException from airflow.contrib.hooks.aws_hook import AwsHook from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook +from airflow.exceptions import AirflowException from airflow.hooks.S3_hook import S3Hook from airflow.utils import timezone diff --git a/airflow/contrib/hooks/segment_hook.py b/airflow/contrib/hooks/segment_hook.py index a072a9f1eaf999..6bd6433c840447 100644 --- a/airflow/contrib/hooks/segment_hook.py +++ b/airflow/contrib/hooks/segment_hook.py @@ -26,8 +26,9 @@ https://github.com/segmentio/analytics-python """ import analytics -from airflow.hooks.base_hook import BaseHook + from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook class SegmentHook(BaseHook): diff --git a/airflow/contrib/hooks/sftp_hook.py b/airflow/contrib/hooks/sftp_hook.py index f765d82a88e804..76cd52bc0523a0 100644 --- a/airflow/contrib/hooks/sftp_hook.py +++ b/airflow/contrib/hooks/sftp_hook.py @@ -17,9 +17,11 @@ # specific language governing permissions and limitations # under the License. +import datetime import stat + import pysftp -import datetime + from airflow.contrib.hooks.ssh_hook import SSHHook diff --git a/airflow/contrib/hooks/slack_webhook_hook.py b/airflow/contrib/hooks/slack_webhook_hook.py index 8e3d4888d96371..eb0378d08a7b4c 100644 --- a/airflow/contrib/hooks/slack_webhook_hook.py +++ b/airflow/contrib/hooks/slack_webhook_hook.py @@ -19,8 +19,8 @@ # import json -from airflow.hooks.http_hook import HttpHook from airflow.exceptions import AirflowException +from airflow.hooks.http_hook import HttpHook class SlackWebhookHook(HttpHook): diff --git a/airflow/contrib/hooks/spark_jdbc_hook.py b/airflow/contrib/hooks/spark_jdbc_hook.py index 7d8d02c060ac89..8e37816298c2ae 100644 --- a/airflow/contrib/hooks/spark_jdbc_hook.py +++ b/airflow/contrib/hooks/spark_jdbc_hook.py @@ -18,6 +18,7 @@ # under the License. # import os + from airflow.contrib.hooks.spark_submit_hook import SparkSubmitHook from airflow.exceptions import AirflowException diff --git a/airflow/contrib/hooks/spark_jdbc_script.py b/airflow/contrib/hooks/spark_jdbc_script.py index 8e27f363f8945c..8d0014529f9624 100644 --- a/airflow/contrib/hooks/spark_jdbc_script.py +++ b/airflow/contrib/hooks/spark_jdbc_script.py @@ -18,6 +18,7 @@ # under the License. # import argparse + from pyspark.sql import SparkSession diff --git a/airflow/contrib/hooks/spark_sql_hook.py b/airflow/contrib/hooks/spark_sql_hook.py index 25c0bc53c55764..a107568a6ac104 100644 --- a/airflow/contrib/hooks/spark_sql_hook.py +++ b/airflow/contrib/hooks/spark_sql_hook.py @@ -19,8 +19,8 @@ # import subprocess -from airflow.hooks.base_hook import BaseHook from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook class SparkSqlHook(BaseHook): diff --git a/airflow/contrib/hooks/spark_submit_hook.py b/airflow/contrib/hooks/spark_submit_hook.py index 449f072b42f924..54ae2e226d7c2f 100644 --- a/airflow/contrib/hooks/spark_submit_hook.py +++ b/airflow/contrib/hooks/spark_submit_hook.py @@ -18,14 +18,14 @@ # under the License. # import os -import subprocess import re +import subprocess import time -from airflow.hooks.base_hook import BaseHook from airflow.exceptions import AirflowException -from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.hooks.base_hook import BaseHook from airflow.kubernetes import kube_client +from airflow.utils.log.logging_mixin import LoggingMixin class SparkSubmitHook(BaseHook, LoggingMixin): diff --git a/airflow/contrib/hooks/sqoop_hook.py b/airflow/contrib/hooks/sqoop_hook.py index aa5ec5222afd0d..e8e2feb3e62e83 100644 --- a/airflow/contrib/hooks/sqoop_hook.py +++ b/airflow/contrib/hooks/sqoop_hook.py @@ -22,10 +22,10 @@ This module contains a sqoop 1.x hook """ import subprocess +from copy import deepcopy from airflow.exceptions import AirflowException from airflow.hooks.base_hook import BaseHook -from copy import deepcopy class SqoopHook(BaseHook): diff --git a/airflow/contrib/hooks/wasb_hook.py b/airflow/contrib/hooks/wasb_hook.py index 077f8d92205e1f..b1eef4aba6ec17 100644 --- a/airflow/contrib/hooks/wasb_hook.py +++ b/airflow/contrib/hooks/wasb_hook.py @@ -25,11 +25,11 @@ login (=Storage account name) and password (=KEY), or login and SAS token in the extra field (see connection `wasb_default` for an example). """ +from azure.storage.blob import BlockBlobService + from airflow.exceptions import AirflowException from airflow.hooks.base_hook import BaseHook -from azure.storage.blob import BlockBlobService - class WasbHook(BaseHook): """ diff --git a/airflow/contrib/operators/aws_athena_operator.py b/airflow/contrib/operators/aws_athena_operator.py index 6d26385bb02154..eb29c0840d6caa 100644 --- a/airflow/contrib/operators/aws_athena_operator.py +++ b/airflow/contrib/operators/aws_athena_operator.py @@ -20,9 +20,9 @@ from uuid import uuid4 +from airflow.contrib.hooks.aws_athena_hook import AWSAthenaHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.aws_athena_hook import AWSAthenaHook class AWSAthenaOperator(BaseOperator): diff --git a/airflow/contrib/operators/aws_sqs_publish_operator.py b/airflow/contrib/operators/aws_sqs_publish_operator.py index 0bf2f7f84c6600..2181c9aa545804 100644 --- a/airflow/contrib/operators/aws_sqs_publish_operator.py +++ b/airflow/contrib/operators/aws_sqs_publish_operator.py @@ -18,9 +18,9 @@ # under the License. """Publish message to SQS queue""" +from airflow.contrib.hooks.aws_sqs_hook import SQSHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.aws_sqs_hook import SQSHook class SQSPublishOperator(BaseOperator): diff --git a/airflow/contrib/operators/awsbatch_operator.py b/airflow/contrib/operators/awsbatch_operator.py index 9d3f27327ba3c5..22508d17c56a2a 100644 --- a/airflow/contrib/operators/awsbatch_operator.py +++ b/airflow/contrib/operators/awsbatch_operator.py @@ -17,20 +17,18 @@ # specific language governing permissions and limitations # under the License. # -from typing import Optional -from airflow.typing import Protocol import sys - from math import pow from random import randint from time import sleep +from typing import Optional +from airflow.contrib.hooks.aws_hook import AwsHook from airflow.exceptions import AirflowException from airflow.models import BaseOperator +from airflow.typing import Protocol from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.aws_hook import AwsHook - class BatchProtocol(Protocol): def submit_job(self, jobName, jobQueue, jobDefinition, containerOverrides): diff --git a/airflow/contrib/operators/azure_container_instances_operator.py b/airflow/contrib/operators/azure_container_instances_operator.py index 4b1d48b0dc7eaf..06b126377f98ed 100644 --- a/airflow/contrib/operators/azure_container_instances_operator.py +++ b/airflow/contrib/operators/azure_container_instances_operator.py @@ -17,28 +17,23 @@ # specific language governing permissions and limitations # under the License. +import re from collections import namedtuple -from typing import Sequence, Dict from time import sleep -import re +from typing import Dict, Sequence + +from azure.mgmt.containerinstance.models import ( + Container, ContainerGroup, EnvironmentVariable, ResourceRequests, ResourceRequirements, VolumeMount, +) +from msrestazure.azure_exceptions import CloudError from airflow.contrib.hooks.azure_container_instance_hook import AzureContainerInstanceHook from airflow.contrib.hooks.azure_container_registry_hook import AzureContainerRegistryHook from airflow.contrib.hooks.azure_container_volume_hook import AzureContainerVolumeHook - from airflow.exceptions import AirflowException, AirflowTaskTimeout from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from azure.mgmt.containerinstance.models import (EnvironmentVariable, - VolumeMount, - ResourceRequests, - ResourceRequirements, - Container, - ContainerGroup) -from msrestazure.azure_exceptions import CloudError - - Volume = namedtuple( 'Volume', ['conn_id', 'account_name', 'share_name', 'mount_path', 'read_only'], diff --git a/airflow/contrib/operators/bigquery_check_operator.py b/airflow/contrib/operators/bigquery_check_operator.py index 2987265253fe22..ee2199a7ed48da 100644 --- a/airflow/contrib/operators/bigquery_check_operator.py +++ b/airflow/contrib/operators/bigquery_check_operator.py @@ -22,9 +22,7 @@ # pylint: disable=unused-import from airflow.gcp.operators.bigquery import ( # noqa - BigQueryCheckOperator, - BigQueryIntervalCheckOperator, - BigQueryValueCheckOperator + BigQueryCheckOperator, BigQueryIntervalCheckOperator, BigQueryValueCheckOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 3ee33bb02f5912..ae07f01410bfb9 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -22,16 +22,9 @@ # pylint: disable=unused-import from airflow.gcp.operators.bigquery import ( # noqa - BigQueryConsoleLink, - BigQueryConsoleIndexableLink, - BigQueryOperator, - BigQueryCreateEmptyTableOperator, - BigQueryCreateExternalTableOperator, - BigQueryDeleteDatasetOperator, - BigQueryCreateEmptyDatasetOperator, - BigQueryGetDatasetOperator, - BigQueryPatchDatasetOperator, - BigQueryUpdateDatasetOperator, + BigQueryConsoleIndexableLink, BigQueryConsoleLink, BigQueryCreateEmptyDatasetOperator, + BigQueryCreateEmptyTableOperator, BigQueryCreateExternalTableOperator, BigQueryDeleteDatasetOperator, + BigQueryGetDatasetOperator, BigQueryOperator, BigQueryPatchDatasetOperator, BigQueryUpdateDatasetOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/databricks_operator.py b/airflow/contrib/operators/databricks_operator.py index 34258ee9204002..34c706b0caf2ec 100644 --- a/airflow/contrib/operators/databricks_operator.py +++ b/airflow/contrib/operators/databricks_operator.py @@ -23,12 +23,11 @@ import time -from airflow.exceptions import AirflowException from airflow.contrib.hooks.databricks_hook import DatabricksHook +from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults - XCOM_RUN_ID_KEY = 'run_id' XCOM_RUN_PAGE_URL_KEY = 'run_page_url' diff --git a/airflow/contrib/operators/dataflow_operator.py b/airflow/contrib/operators/dataflow_operator.py index d4e6c519a14f97..1bc556e1e906db 100644 --- a/airflow/contrib/operators/dataflow_operator.py +++ b/airflow/contrib/operators/dataflow_operator.py @@ -22,9 +22,7 @@ # pylint: disable=unused-import from airflow.gcp.operators.dataflow import ( # noqa - DataFlowJavaOperator, - DataFlowPythonOperator, - DataflowTemplateOperator + DataFlowJavaOperator, DataFlowPythonOperator, DataflowTemplateOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index b68cc08c499243..3c42768bb30efa 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -22,19 +22,10 @@ # pylint: disable=unused-import from airflow.gcp.operators.dataproc import ( # noqa - DataProcHadoopOperator, - DataProcPySparkOperator, - DataProcHiveOperator, - DataProcPigOperator, - DataProcSparkOperator, - DataProcSparkSqlOperator, - DataprocClusterScaleOperator, - DataprocOperationBaseOperator, - DataProcJobBaseOperator, - DataprocClusterDeleteOperator, - DataprocClusterCreateOperator, - DataprocWorkflowTemplateInstantiateOperator, - DataprocWorkflowTemplateInstantiateInlineOperator, + DataprocClusterCreateOperator, DataprocClusterDeleteOperator, DataprocClusterScaleOperator, + DataProcHadoopOperator, DataProcHiveOperator, DataProcJobBaseOperator, DataprocOperationBaseOperator, + DataProcPigOperator, DataProcPySparkOperator, DataProcSparkOperator, DataProcSparkSqlOperator, + DataprocWorkflowTemplateInstantiateInlineOperator, DataprocWorkflowTemplateInstantiateOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/ecs_operator.py b/airflow/contrib/operators/ecs_operator.py index 5d13ef1d3645eb..2476d746ab4b6b 100644 --- a/airflow/contrib/operators/ecs_operator.py +++ b/airflow/contrib/operators/ecs_operator.py @@ -16,19 +16,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Optional -from airflow.typing import Protocol -import sys import re +import sys from datetime import datetime +from typing import Optional +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook from airflow.exceptions import AirflowException from airflow.models import BaseOperator +from airflow.typing import Protocol from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.aws_hook import AwsHook -from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook - class ECSProtocol(Protocol): def run_task(self, **kwargs): diff --git a/airflow/contrib/operators/emr_add_steps_operator.py b/airflow/contrib/operators/emr_add_steps_operator.py index e3625957995555..9ff5488f87ba26 100644 --- a/airflow/contrib/operators/emr_add_steps_operator.py +++ b/airflow/contrib/operators/emr_add_steps_operator.py @@ -16,10 +16,10 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from airflow.contrib.hooks.emr_hook import EmrHook +from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException -from airflow.contrib.hooks.emr_hook import EmrHook class EmrAddStepsOperator(BaseOperator): diff --git a/airflow/contrib/operators/emr_create_job_flow_operator.py b/airflow/contrib/operators/emr_create_job_flow_operator.py index f29f5031a45360..a190d017cc2ecf 100644 --- a/airflow/contrib/operators/emr_create_job_flow_operator.py +++ b/airflow/contrib/operators/emr_create_job_flow_operator.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. from airflow.contrib.hooks.emr_hook import EmrHook +from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException class EmrCreateJobFlowOperator(BaseOperator): diff --git a/airflow/contrib/operators/emr_terminate_job_flow_operator.py b/airflow/contrib/operators/emr_terminate_job_flow_operator.py index f9093ae25fd3f6..6042e5ab80ec06 100644 --- a/airflow/contrib/operators/emr_terminate_job_flow_operator.py +++ b/airflow/contrib/operators/emr_terminate_job_flow_operator.py @@ -16,10 +16,10 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from airflow.contrib.hooks.emr_hook import EmrHook +from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException -from airflow.contrib.hooks.emr_hook import EmrHook class EmrTerminateJobFlowOperator(BaseOperator): diff --git a/airflow/contrib/operators/gcp_bigtable_operator.py b/airflow/contrib/operators/gcp_bigtable_operator.py index 15d935bc338151..519e6dd7bf12b8 100644 --- a/airflow/contrib/operators/gcp_bigtable_operator.py +++ b/airflow/contrib/operators/gcp_bigtable_operator.py @@ -25,12 +25,8 @@ # pylint: disable=unused-import from airflow.gcp.operators.bigtable import ( # noqa - BigtableClusterUpdateOperator, - BigtableInstanceCreateOperator, - BigtableInstanceDeleteOperator, - BigtableTableCreateOperator, - BigtableTableDeleteOperator, - BigtableValidationMixin, + BigtableClusterUpdateOperator, BigtableInstanceCreateOperator, BigtableInstanceDeleteOperator, + BigtableTableCreateOperator, BigtableTableDeleteOperator, BigtableValidationMixin, ) from airflow.gcp.sensors.bigtable import BigtableTableWaitForReplicationSensor # noqa diff --git a/airflow/contrib/operators/gcp_compute_operator.py b/airflow/contrib/operators/gcp_compute_operator.py index 9c430782c89b59..4cbea320cf084b 100644 --- a/airflow/contrib/operators/gcp_compute_operator.py +++ b/airflow/contrib/operators/gcp_compute_operator.py @@ -22,14 +22,9 @@ # pylint: disable=unused-import from airflow.gcp.operators.compute import ( # noqa - GceBaseOperator, - GceInstanceGroupManagerUpdateTemplateOperator, - GceInstanceStartOperator, - GceInstanceStopOperator, - GceInstanceTemplateCopyOperator, - GceSetMachineTypeOperator, - GcpBodyFieldSanitizer, - GcpBodyFieldValidator, + GceBaseOperator, GceInstanceGroupManagerUpdateTemplateOperator, GceInstanceStartOperator, + GceInstanceStopOperator, GceInstanceTemplateCopyOperator, GceSetMachineTypeOperator, + GcpBodyFieldSanitizer, GcpBodyFieldValidator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_container_operator.py b/airflow/contrib/operators/gcp_container_operator.py index d476f87057c4ca..573535fc92f689 100644 --- a/airflow/contrib/operators/gcp_container_operator.py +++ b/airflow/contrib/operators/gcp_container_operator.py @@ -24,9 +24,7 @@ # pylint: disable=unused-import from airflow.gcp.operators.kubernetes_engine import ( # noqa - GKEClusterCreateOperator, - GKEClusterDeleteOperator, - GKEPodOperator + GKEClusterCreateOperator, GKEClusterDeleteOperator, GKEPodOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_dlp_operator.py b/airflow/contrib/operators/gcp_dlp_operator.py index 181c2ed195266b..4f5771f20b0e00 100644 --- a/airflow/contrib/operators/gcp_dlp_operator.py +++ b/airflow/contrib/operators/gcp_dlp_operator.py @@ -22,36 +22,18 @@ # pylint: disable=unused-import from airflow.gcp.operators.dlp import ( # noqa - CloudDLPCancelDLPJobOperator, - CloudDLPCreateDeidentifyTemplateOperator, - CloudDLPCreateDLPJobOperator, - CloudDLPCreateInspectTemplateOperator, - CloudDLPCreateJobTriggerOperator, - CloudDLPCreateStoredInfoTypeOperator, - CloudDLPDeidentifyContentOperator, - CloudDLPDeleteDeidentifyTemplateOperator, - CloudDLPDeleteDlpJobOperator, - CloudDLPDeleteInspectTemplateOperator, - CloudDLPDeleteJobTriggerOperator, - CloudDLPDeleteStoredInfoTypeOperator, - CloudDLPGetDeidentifyTemplateOperator, - CloudDLPGetDlpJobOperator, - CloudDLPGetInspectTemplateOperator, - CloudDLPGetJobTripperOperator, - CloudDLPGetStoredInfoTypeOperator, - CloudDLPInspectContentOperator, - CloudDLPListDeidentifyTemplatesOperator, - CloudDLPListDlpJobsOperator, - CloudDLPListInfoTypesOperator, - CloudDLPListInspectTemplatesOperator, - CloudDLPListJobTriggersOperator, - CloudDLPListStoredInfoTypesOperator, - CloudDLPRedactImageOperator, - CloudDLPReidentifyContentOperator, - CloudDLPUpdateDeidentifyTemplateOperator, - CloudDLPUpdateInspectTemplateOperator, - CloudDLPUpdateJobTriggerOperator, - CloudDLPUpdateStoredInfoTypeOperator, + CloudDLPCancelDLPJobOperator, CloudDLPCreateDeidentifyTemplateOperator, CloudDLPCreateDLPJobOperator, + CloudDLPCreateInspectTemplateOperator, CloudDLPCreateJobTriggerOperator, + CloudDLPCreateStoredInfoTypeOperator, CloudDLPDeidentifyContentOperator, + CloudDLPDeleteDeidentifyTemplateOperator, CloudDLPDeleteDlpJobOperator, + CloudDLPDeleteInspectTemplateOperator, CloudDLPDeleteJobTriggerOperator, + CloudDLPDeleteStoredInfoTypeOperator, CloudDLPGetDeidentifyTemplateOperator, CloudDLPGetDlpJobOperator, + CloudDLPGetInspectTemplateOperator, CloudDLPGetJobTripperOperator, CloudDLPGetStoredInfoTypeOperator, + CloudDLPInspectContentOperator, CloudDLPListDeidentifyTemplatesOperator, CloudDLPListDlpJobsOperator, + CloudDLPListInfoTypesOperator, CloudDLPListInspectTemplatesOperator, CloudDLPListJobTriggersOperator, + CloudDLPListStoredInfoTypesOperator, CloudDLPRedactImageOperator, CloudDLPReidentifyContentOperator, + CloudDLPUpdateDeidentifyTemplateOperator, CloudDLPUpdateInspectTemplateOperator, + CloudDLPUpdateJobTriggerOperator, CloudDLPUpdateStoredInfoTypeOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_function_operator.py b/airflow/contrib/operators/gcp_function_operator.py index e54a42fa3b969c..b4e7ae70d7fcc6 100644 --- a/airflow/contrib/operators/gcp_function_operator.py +++ b/airflow/contrib/operators/gcp_function_operator.py @@ -21,10 +21,8 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.operators.functions import ( # noqa - GcfFunctionDeleteOperator, - GcfFunctionDeployOperator, - ZipPathPreprocessor +from airflow.gcp.operators.functions import ( # # noqa + GcfFunctionDeleteOperator, GcfFunctionDeployOperator, ZipPathPreprocessor, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_natural_language_operator.py b/airflow/contrib/operators/gcp_natural_language_operator.py index d12a6d7d5fd466..7070249a626d9a 100644 --- a/airflow/contrib/operators/gcp_natural_language_operator.py +++ b/airflow/contrib/operators/gcp_natural_language_operator.py @@ -24,10 +24,8 @@ # pylint: disable=unused-import from airflow.gcp.operators.natural_language import ( # noqa - CloudLanguageClassifyTextOperator, - CloudLanguageAnalyzeSentimentOperator, - CloudLanguageAnalyzeEntitySentimentOperator, - CloudLanguageAnalyzeEntitiesOperator, + CloudLanguageAnalyzeEntitiesOperator, CloudLanguageAnalyzeEntitySentimentOperator, + CloudLanguageAnalyzeSentimentOperator, CloudLanguageClassifyTextOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_spanner_operator.py b/airflow/contrib/operators/gcp_spanner_operator.py index afe1ddf4f0859f..dc1a6b3e4ceda9 100644 --- a/airflow/contrib/operators/gcp_spanner_operator.py +++ b/airflow/contrib/operators/gcp_spanner_operator.py @@ -22,12 +22,9 @@ # pylint: disable=unused-import from airflow.gcp.operators.spanner import ( # noqa - CloudSpannerInstanceDatabaseDeleteOperator, - CloudSpannerInstanceDatabaseDeployOperator, - CloudSpannerInstanceDatabaseQueryOperator, - CloudSpannerInstanceDatabaseUpdateOperator, - CloudSpannerInstanceDeleteOperator, - CloudSpannerInstanceDeployOperator, + CloudSpannerInstanceDatabaseDeleteOperator, CloudSpannerInstanceDatabaseDeployOperator, + CloudSpannerInstanceDatabaseQueryOperator, CloudSpannerInstanceDatabaseUpdateOperator, + CloudSpannerInstanceDeleteOperator, CloudSpannerInstanceDeployOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_sql_operator.py b/airflow/contrib/operators/gcp_sql_operator.py index d1082d9e2f10fb..0bc04f9c159aa1 100644 --- a/airflow/contrib/operators/gcp_sql_operator.py +++ b/airflow/contrib/operators/gcp_sql_operator.py @@ -24,16 +24,10 @@ # pylint: disable=unused-import from airflow.gcp.operators.cloud_sql import ( # noqa - CloudSqlBaseOperator, - CloudSqlInstanceCreateOperator, - CloudSqlInstanceDatabaseCreateOperator, - CloudSqlInstanceDatabaseDeleteOperator, - CloudSqlInstanceDatabasePatchOperator, - CloudSqlInstanceDeleteOperator, - CloudSqlInstanceExportOperator, - CloudSqlInstanceImportOperator, - CloudSqlInstancePatchOperator, - CloudSqlQueryOperator, + CloudSqlBaseOperator, CloudSqlInstanceCreateOperator, CloudSqlInstanceDatabaseCreateOperator, + CloudSqlInstanceDatabaseDeleteOperator, CloudSqlInstanceDatabasePatchOperator, + CloudSqlInstanceDeleteOperator, CloudSqlInstanceExportOperator, CloudSqlInstanceImportOperator, + CloudSqlInstancePatchOperator, CloudSqlQueryOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_tasks_operator.py b/airflow/contrib/operators/gcp_tasks_operator.py index d38cf37c13f47e..a780b05486778e 100644 --- a/airflow/contrib/operators/gcp_tasks_operator.py +++ b/airflow/contrib/operators/gcp_tasks_operator.py @@ -24,18 +24,10 @@ # pylint: disable=unused-import from airflow.gcp.operators.tasks import ( # noqa - CloudTasksQueueCreateOperator, - CloudTasksQueueDeleteOperator, - CloudTasksQueueGetOperator, - CloudTasksQueuePauseOperator, - CloudTasksQueuePurgeOperator, - CloudTasksQueueResumeOperator, - CloudTasksQueuesListOperator, - CloudTasksQueueUpdateOperator, - CloudTasksTaskCreateOperator, - CloudTasksTaskDeleteOperator, - CloudTasksTaskGetOperator, - CloudTasksTaskRunOperator, + CloudTasksQueueCreateOperator, CloudTasksQueueDeleteOperator, CloudTasksQueueGetOperator, + CloudTasksQueuePauseOperator, CloudTasksQueuePurgeOperator, CloudTasksQueueResumeOperator, + CloudTasksQueuesListOperator, CloudTasksQueueUpdateOperator, CloudTasksTaskCreateOperator, + CloudTasksTaskDeleteOperator, CloudTasksTaskGetOperator, CloudTasksTaskRunOperator, CloudTasksTasksListOperator, ) diff --git a/airflow/contrib/operators/gcp_transfer_operator.py b/airflow/contrib/operators/gcp_transfer_operator.py index 618a1293d734a8..ab6f9d29c308b0 100644 --- a/airflow/contrib/operators/gcp_transfer_operator.py +++ b/airflow/contrib/operators/gcp_transfer_operator.py @@ -24,17 +24,11 @@ # pylint: disable=unused-import from airflow.gcp.operators.cloud_storage_transfer_service import ( # noqa - TransferJobPreprocessor, - TransferJobValidator, - GcpTransferServiceJobCreateOperator, - GcpTransferServiceJobDeleteOperator, - GcpTransferServiceJobUpdateOperator, - GcpTransferServiceOperationCancelOperator, - GcpTransferServiceOperationGetOperator, - GcpTransferServiceOperationPauseOperator, - GcpTransferServiceOperationResumeOperator, - GcpTransferServiceOperationsListOperator, - GoogleCloudStorageToGoogleCloudStorageTransferOperator, + GcpTransferServiceJobCreateOperator, GcpTransferServiceJobDeleteOperator, + GcpTransferServiceJobUpdateOperator, GcpTransferServiceOperationCancelOperator, + GcpTransferServiceOperationGetOperator, GcpTransferServiceOperationPauseOperator, + GcpTransferServiceOperationResumeOperator, GcpTransferServiceOperationsListOperator, + GoogleCloudStorageToGoogleCloudStorageTransferOperator, TransferJobPreprocessor, TransferJobValidator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcp_video_intelligence_operator.py b/airflow/contrib/operators/gcp_video_intelligence_operator.py index eea72e62329672..02556dcbfbc70c 100644 --- a/airflow/contrib/operators/gcp_video_intelligence_operator.py +++ b/airflow/contrib/operators/gcp_video_intelligence_operator.py @@ -24,8 +24,7 @@ # pylint: disable=unused-import from airflow.gcp.operators.video_intelligence import ( # noqa - CloudVideoIntelligenceDetectVideoExplicitContentOperator, - CloudVideoIntelligenceDetectVideoLabelsOperator, + CloudVideoIntelligenceDetectVideoExplicitContentOperator, CloudVideoIntelligenceDetectVideoLabelsOperator, CloudVideoIntelligenceDetectVideoShotsOperator, ) diff --git a/airflow/contrib/operators/gcp_vision_operator.py b/airflow/contrib/operators/gcp_vision_operator.py index 75c593c2d3ec03..cad87ed52cf671 100644 --- a/airflow/contrib/operators/gcp_vision_operator.py +++ b/airflow/contrib/operators/gcp_vision_operator.py @@ -24,22 +24,13 @@ # pylint: disable=unused-import from airflow.gcp.operators.vision import ( # noqa - CloudVisionAddProductToProductSetOperator, - CloudVisionAnnotateImageOperator, - CloudVisionDetectDocumentTextOperator, - CloudVisionDetectImageLabelsOperator, - CloudVisionDetectImageSafeSearchOperator, - CloudVisionDetectTextOperator, - CloudVisionProductCreateOperator, - CloudVisionProductDeleteOperator, - CloudVisionProductGetOperator, - CloudVisionProductSetCreateOperator, - CloudVisionProductSetDeleteOperator, - CloudVisionProductSetGetOperator, - CloudVisionProductSetUpdateOperator, - CloudVisionProductUpdateOperator, - CloudVisionReferenceImageCreateOperator, - CloudVisionRemoveProductFromProductSetOperator, + CloudVisionAddProductToProductSetOperator, CloudVisionAnnotateImageOperator, + CloudVisionDetectDocumentTextOperator, CloudVisionDetectImageLabelsOperator, + CloudVisionDetectImageSafeSearchOperator, CloudVisionDetectTextOperator, CloudVisionProductCreateOperator, + CloudVisionProductDeleteOperator, CloudVisionProductGetOperator, CloudVisionProductSetCreateOperator, + CloudVisionProductSetDeleteOperator, CloudVisionProductSetGetOperator, + CloudVisionProductSetUpdateOperator, CloudVisionProductUpdateOperator, + CloudVisionReferenceImageCreateOperator, CloudVisionRemoveProductFromProductSetOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcs_acl_operator.py b/airflow/contrib/operators/gcs_acl_operator.py index e8e36b555434b8..e3f6450886d6e2 100644 --- a/airflow/contrib/operators/gcs_acl_operator.py +++ b/airflow/contrib/operators/gcs_acl_operator.py @@ -24,8 +24,7 @@ # pylint: disable=unused-import from airflow.gcp.operators.gcs import ( # noqa - GoogleCloudStorageObjectCreateAclEntryOperator, - GoogleCloudStorageBucketCreateAclEntryOperator + GoogleCloudStorageBucketCreateAclEntryOperator, GoogleCloudStorageObjectCreateAclEntryOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py b/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py index f48adec09da054..3b04a4755e22a9 100644 --- a/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py +++ b/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py @@ -23,8 +23,8 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.operators.cloud_storage_transfer_service import ( # noqa - GoogleCloudStorageToGoogleCloudStorageTransferOperator +from airflow.gcp.operators.cloud_storage_transfer_service import ( # # noqa + GoogleCloudStorageToGoogleCloudStorageTransferOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/gcs_to_gdrive_operator.py b/airflow/contrib/operators/gcs_to_gdrive_operator.py index 9678718e0b79db..2ff7a6a487469b 100644 --- a/airflow/contrib/operators/gcs_to_gdrive_operator.py +++ b/airflow/contrib/operators/gcs_to_gdrive_operator.py @@ -22,11 +22,11 @@ import tempfile from typing import Optional -from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.contrib.hooks.gdrive_hook import GoogleDriveHook +from airflow.exceptions import AirflowException +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException WILDCARD = "*" diff --git a/airflow/contrib/operators/jenkins_job_trigger_operator.py b/airflow/contrib/operators/jenkins_job_trigger_operator.py index 498d0207e6f306..5006bf954622ba 100644 --- a/airflow/contrib/operators/jenkins_job_trigger_operator.py +++ b/airflow/contrib/operators/jenkins_job_trigger_operator.py @@ -17,19 +17,20 @@ # specific language governing permissions and limitations # under the License. -import time -import socket import json +import socket +import time from urllib.error import HTTPError, URLError -from airflow.exceptions import AirflowException -from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.jenkins_hook import JenkinsHook import jenkins from jenkins import JenkinsException from requests import Request +from airflow.contrib.hooks.jenkins_hook import JenkinsHook +from airflow.exceptions import AirflowException +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults + def jenkins_request_with_headers(jenkins_server, req): """ diff --git a/airflow/contrib/operators/jira_operator.py b/airflow/contrib/operators/jira_operator.py index ceac4f793a6178..583146f48d2c1e 100644 --- a/airflow/contrib/operators/jira_operator.py +++ b/airflow/contrib/operators/jira_operator.py @@ -18,8 +18,7 @@ # under the License. -from airflow.contrib.hooks.jira_hook import JIRAError -from airflow.contrib.hooks.jira_hook import JiraHook +from airflow.contrib.hooks.jira_hook import JIRAError, JiraHook from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py index 7616b2ca7f9aaa..b545f073f78cb8 100644 --- a/airflow/contrib/operators/kubernetes_pod_operator.py +++ b/airflow/contrib/operators/kubernetes_pod_operator.py @@ -16,10 +16,10 @@ # under the License. """Executes task in a Kubernetes POD""" from airflow.exceptions import AirflowException +from airflow.kubernetes import kube_client, pod_generator, pod_launcher +from airflow.kubernetes.k8s_model import append_to_pod from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.kubernetes import pod_generator, kube_client, pod_launcher -from airflow.kubernetes.k8s_model import append_to_pod from airflow.utils.state import State diff --git a/airflow/contrib/operators/mlengine_operator.py b/airflow/contrib/operators/mlengine_operator.py index 2d60a0b7bfb42c..4fc6c86a740945 100644 --- a/airflow/contrib/operators/mlengine_operator.py +++ b/airflow/contrib/operators/mlengine_operator.py @@ -22,10 +22,7 @@ # pylint: disable=unused-import from airflow.gcp.operators.mlengine import ( # noqa - MLEngineBatchPredictionOperator, - MLEngineModelOperator, - MLEngineTrainingOperator, - MLEngineVersionOperator, + MLEngineBatchPredictionOperator, MLEngineModelOperator, MLEngineTrainingOperator, MLEngineVersionOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/mongo_to_s3.py b/airflow/contrib/operators/mongo_to_s3.py index 1767732ed2ea8e..72d2a0c5b79e6d 100644 --- a/airflow/contrib/operators/mongo_to_s3.py +++ b/airflow/contrib/operators/mongo_to_s3.py @@ -18,11 +18,12 @@ # under the License. import json +from bson import json_util + from airflow.contrib.hooks.mongo_hook import MongoHook from airflow.hooks.S3_hook import S3Hook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from bson import json_util class MongoToS3Operator(BaseOperator): diff --git a/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py index db7a6eb94ee59e..8f816dda780580 100644 --- a/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py +++ b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py @@ -17,15 +17,16 @@ # specific language governing permissions and limitations # under the License. -from airflow.hooks.oracle_hook import OracleHook +import os + +import unicodecsv as csv + from airflow.contrib.hooks.azure_data_lake_hook import AzureDataLakeHook +from airflow.hooks.oracle_hook import OracleHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults from airflow.utils.file import TemporaryDirectory -import unicodecsv as csv -import os - class OracleToAzureDataLakeTransfer(BaseOperator): """ diff --git a/airflow/contrib/operators/pubsub_operator.py b/airflow/contrib/operators/pubsub_operator.py index 4c7089d3525bc8..3f5245158cb75e 100644 --- a/airflow/contrib/operators/pubsub_operator.py +++ b/airflow/contrib/operators/pubsub_operator.py @@ -22,11 +22,8 @@ # pylint: disable=unused-import from airflow.gcp.operators.pubsub import ( # noqa - PubSubPublishOperator, - PubSubSubscriptionCreateOperator, - PubSubSubscriptionDeleteOperator, - PubSubTopicCreateOperator, - PubSubTopicDeleteOperator, + PubSubPublishOperator, PubSubSubscriptionCreateOperator, PubSubSubscriptionDeleteOperator, + PubSubTopicCreateOperator, PubSubTopicDeleteOperator, ) warnings.warn( diff --git a/airflow/contrib/operators/qubole_check_operator.py b/airflow/contrib/operators/qubole_check_operator.py index a2505527b1ea3f..75c5fd164103da 100644 --- a/airflow/contrib/operators/qubole_check_operator.py +++ b/airflow/contrib/operators/qubole_check_operator.py @@ -17,11 +17,11 @@ # specific language governing permissions and limitations # under the License. # -from airflow.contrib.operators.qubole_operator import QuboleOperator -from airflow.utils.decorators import apply_defaults from airflow.contrib.hooks.qubole_check_hook import QuboleCheckHook -from airflow.operators.check_operator import CheckOperator, ValueCheckOperator +from airflow.contrib.operators.qubole_operator import QuboleOperator from airflow.exceptions import AirflowException +from airflow.operators.check_operator import CheckOperator, ValueCheckOperator +from airflow.utils.decorators import apply_defaults class QuboleCheckOperator(CheckOperator, QuboleOperator): diff --git a/airflow/contrib/operators/qubole_operator.py b/airflow/contrib/operators/qubole_operator.py index ddd7e80542f00b..c39c22adc2f084 100644 --- a/airflow/contrib/operators/qubole_operator.py +++ b/airflow/contrib/operators/qubole_operator.py @@ -19,10 +19,12 @@ """Qubole operator""" from typing import Iterable + +from airflow.contrib.hooks.qubole_hook import ( + COMMAND_ARGS, HYPHEN_ARGS, POSITIONAL_ARGS, QuboleHook, flatten_list, +) from airflow.models.baseoperator import BaseOperator, BaseOperatorLink from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.qubole_hook import QuboleHook, COMMAND_ARGS, HYPHEN_ARGS, \ - flatten_list, POSITIONAL_ARGS class QDSLink(BaseOperatorLink): diff --git a/airflow/contrib/operators/redis_publish_operator.py b/airflow/contrib/operators/redis_publish_operator.py index d753de65884bd8..d7579bd6446271 100644 --- a/airflow/contrib/operators/redis_publish_operator.py +++ b/airflow/contrib/operators/redis_publish_operator.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. -from airflow.utils.decorators import apply_defaults from airflow.contrib.hooks.redis_hook import RedisHook from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults class RedisPublishOperator(BaseOperator): diff --git a/airflow/contrib/operators/s3_to_gcs_operator.py b/airflow/contrib/operators/s3_to_gcs_operator.py index d71986b530b264..0c86e481da95d2 100644 --- a/airflow/contrib/operators/s3_to_gcs_operator.py +++ b/airflow/contrib/operators/s3_to_gcs_operator.py @@ -19,10 +19,9 @@ import warnings from tempfile import NamedTemporaryFile -from airflow.gcp.hooks.gcs import (GoogleCloudStorageHook, - _parse_gcs_url) from airflow.contrib.operators.s3_list_operator import S3ListOperator from airflow.exceptions import AirflowException +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook, _parse_gcs_url from airflow.hooks.S3_hook import S3Hook from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/operators/s3_to_gcs_transfer_operator.py b/airflow/contrib/operators/s3_to_gcs_transfer_operator.py index 1538ecce2d3614..c5886faa1e6590 100644 --- a/airflow/contrib/operators/s3_to_gcs_transfer_operator.py +++ b/airflow/contrib/operators/s3_to_gcs_transfer_operator.py @@ -21,10 +21,8 @@ """ import warnings -# pylint: disable=unused-import -from airflow.gcp.operators.cloud_storage_transfer_service import ( # noqa - S3ToGoogleCloudStorageTransferOperator -) +# pylint: disable=unused-import,line-too-long +from airflow.gcp.operators.cloud_storage_transfer_service import S3ToGoogleCloudStorageTransferOperator # noqa isort:skip warnings.warn( "This module is deprecated. Please use `airflow.gcp.operators.cloud_storage_transfer_service`.", diff --git a/airflow/contrib/operators/s3_to_sftp_operator.py b/airflow/contrib/operators/s3_to_sftp_operator.py index 456875bd832f2c..9fc95788f8091e 100644 --- a/airflow/contrib/operators/s3_to_sftp_operator.py +++ b/airflow/contrib/operators/s3_to_sftp_operator.py @@ -17,11 +17,12 @@ # specific language governing permissions and limitations # under the License. -from airflow.models import BaseOperator -from airflow.hooks.S3_hook import S3Hook -from airflow.contrib.hooks.ssh_hook import SSHHook from tempfile import NamedTemporaryFile from urllib.parse import urlparse + +from airflow.contrib.hooks.ssh_hook import SSHHook +from airflow.hooks.S3_hook import S3Hook +from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/operators/sagemaker_base_operator.py b/airflow/contrib/operators/sagemaker_base_operator.py index c7d743fa702e63..d07390634840c9 100644 --- a/airflow/contrib/operators/sagemaker_base_operator.py +++ b/airflow/contrib/operators/sagemaker_base_operator.py @@ -18,7 +18,6 @@ # under the License. import json - from typing import Iterable from airflow.contrib.hooks.sagemaker_hook import SageMakerHook diff --git a/airflow/contrib/operators/sagemaker_endpoint_config_operator.py b/airflow/contrib/operators/sagemaker_endpoint_config_operator.py index 23c8d053b28a45..f0ddbd1b58c261 100644 --- a/airflow/contrib/operators/sagemaker_endpoint_config_operator.py +++ b/airflow/contrib/operators/sagemaker_endpoint_config_operator.py @@ -18,8 +18,8 @@ # under the License. from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults class SageMakerEndpointConfigOperator(SageMakerBaseOperator): diff --git a/airflow/contrib/operators/sagemaker_endpoint_operator.py b/airflow/contrib/operators/sagemaker_endpoint_operator.py index a1dbf47bf22643..aba16ce615af78 100644 --- a/airflow/contrib/operators/sagemaker_endpoint_operator.py +++ b/airflow/contrib/operators/sagemaker_endpoint_operator.py @@ -19,8 +19,8 @@ from airflow.contrib.hooks.aws_hook import AwsHook from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults class SageMakerEndpointOperator(SageMakerBaseOperator): diff --git a/airflow/contrib/operators/sagemaker_model_operator.py b/airflow/contrib/operators/sagemaker_model_operator.py index df5670b78fc090..c6855232e9d76c 100644 --- a/airflow/contrib/operators/sagemaker_model_operator.py +++ b/airflow/contrib/operators/sagemaker_model_operator.py @@ -19,8 +19,8 @@ from airflow.contrib.hooks.aws_hook import AwsHook from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults class SageMakerModelOperator(SageMakerBaseOperator): diff --git a/airflow/contrib/operators/sagemaker_training_operator.py b/airflow/contrib/operators/sagemaker_training_operator.py index 903fb1f2fa75b8..135fcf57ce6060 100644 --- a/airflow/contrib/operators/sagemaker_training_operator.py +++ b/airflow/contrib/operators/sagemaker_training_operator.py @@ -19,8 +19,8 @@ from airflow.contrib.hooks.aws_hook import AwsHook from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults class SageMakerTrainingOperator(SageMakerBaseOperator): diff --git a/airflow/contrib/operators/sagemaker_transform_operator.py b/airflow/contrib/operators/sagemaker_transform_operator.py index c0fbbc8cebcbd8..71475cf8311959 100644 --- a/airflow/contrib/operators/sagemaker_transform_operator.py +++ b/airflow/contrib/operators/sagemaker_transform_operator.py @@ -19,8 +19,8 @@ from airflow.contrib.hooks.aws_hook import AwsHook from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults class SageMakerTransformOperator(SageMakerBaseOperator): diff --git a/airflow/contrib/operators/sagemaker_tuning_operator.py b/airflow/contrib/operators/sagemaker_tuning_operator.py index 75c1caf684fd49..a5893519adbdee 100644 --- a/airflow/contrib/operators/sagemaker_tuning_operator.py +++ b/airflow/contrib/operators/sagemaker_tuning_operator.py @@ -19,8 +19,8 @@ from airflow.contrib.hooks.aws_hook import AwsHook from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults class SageMakerTuningOperator(SageMakerBaseOperator): diff --git a/airflow/contrib/operators/sftp_to_s3_operator.py b/airflow/contrib/operators/sftp_to_s3_operator.py index 8d1bfb5194ef7b..5dabb3bfc168cc 100644 --- a/airflow/contrib/operators/sftp_to_s3_operator.py +++ b/airflow/contrib/operators/sftp_to_s3_operator.py @@ -17,11 +17,12 @@ # specific language governing permissions and limitations # under the License. -from airflow.models import BaseOperator -from airflow.hooks.S3_hook import S3Hook -from airflow.contrib.hooks.ssh_hook import SSHHook from tempfile import NamedTemporaryFile from urllib.parse import urlparse + +from airflow.contrib.hooks.ssh_hook import SSHHook +from airflow.hooks.S3_hook import S3Hook +from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/operators/slack_webhook_operator.py b/airflow/contrib/operators/slack_webhook_operator.py index 8da3381ab99618..cd2516c1f7323c 100644 --- a/airflow/contrib/operators/slack_webhook_operator.py +++ b/airflow/contrib/operators/slack_webhook_operator.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. # +from airflow.contrib.hooks.slack_webhook_hook import SlackWebhookHook from airflow.operators.http_operator import SimpleHttpOperator from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.slack_webhook_hook import SlackWebhookHook class SlackWebhookOperator(SimpleHttpOperator): diff --git a/airflow/contrib/operators/spark_jdbc_operator.py b/airflow/contrib/operators/spark_jdbc_operator.py index 84debab398ccf5..20aadb00c3b8c9 100644 --- a/airflow/contrib/operators/spark_jdbc_operator.py +++ b/airflow/contrib/operators/spark_jdbc_operator.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. # -from airflow.contrib.operators.spark_submit_operator import SparkSubmitOperator from airflow.contrib.hooks.spark_jdbc_hook import SparkJDBCHook +from airflow.contrib.operators.spark_submit_operator import SparkSubmitOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/operators/spark_sql_operator.py b/airflow/contrib/operators/spark_sql_operator.py index 240ba96312c54a..3ea65054ed8066 100644 --- a/airflow/contrib/operators/spark_sql_operator.py +++ b/airflow/contrib/operators/spark_sql_operator.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. # +from airflow.contrib.hooks.spark_sql_hook import SparkSqlHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.spark_sql_hook import SparkSqlHook class SparkSqlOperator(BaseOperator): diff --git a/airflow/contrib/operators/vertica_to_hive.py b/airflow/contrib/operators/vertica_to_hive.py index 91bfc228fb69a0..23831ba3bfb2c2 100644 --- a/airflow/contrib/operators/vertica_to_hive.py +++ b/airflow/contrib/operators/vertica_to_hive.py @@ -18,11 +18,12 @@ # under the License. from collections import OrderedDict -import unicodecsv as csv from tempfile import NamedTemporaryFile -from airflow.hooks.hive_hooks import HiveCliHook +import unicodecsv as csv + from airflow.contrib.hooks.vertica_hook import VerticaHook +from airflow.hooks.hive_hooks import HiveCliHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/operators/vertica_to_mysql.py b/airflow/contrib/operators/vertica_to_mysql.py index 4d944c51edf54c..b20033768419ef 100644 --- a/airflow/contrib/operators/vertica_to_mysql.py +++ b/airflow/contrib/operators/vertica_to_mysql.py @@ -17,18 +17,17 @@ # specific language governing permissions and limitations # under the License. +from contextlib import closing +from tempfile import NamedTemporaryFile + import MySQLdb +import unicodecsv as csv from airflow.contrib.hooks.vertica_hook import VerticaHook from airflow.hooks.mysql_hook import MySqlHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from contextlib import closing - -import unicodecsv as csv -from tempfile import NamedTemporaryFile - class VerticaToMySqlTransfer(BaseOperator): """ diff --git a/airflow/contrib/operators/winrm_operator.py b/airflow/contrib/operators/winrm_operator.py index 7bf1bab93533aa..25d7831950b952 100644 --- a/airflow/contrib/operators/winrm_operator.py +++ b/airflow/contrib/operators/winrm_operator.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from base64 import b64encode import logging +from base64 import b64encode from winrm.exceptions import WinRMOperationTimeoutError diff --git a/airflow/contrib/plugins/metastore_browser/main.py b/airflow/contrib/plugins/metastore_browser/main.py index c42d1e16a51cbd..5359ecf01464bc 100644 --- a/airflow/contrib/plugins/metastore_browser/main.py +++ b/airflow/contrib/plugins/metastore_browser/main.py @@ -19,15 +19,15 @@ '''Plugins metabrowser''' +import json from datetime import datetime from typing import List -import json +import pandas as pd from flask import Blueprint, request from flask_appbuilder import BaseView, expose -import pandas as pd -from airflow.hooks.hive_hooks import HiveMetastoreHook, HiveCliHook +from airflow.hooks.hive_hooks import HiveCliHook, HiveMetastoreHook from airflow.hooks.mysql_hook import MySqlHook from airflow.hooks.presto_hook import PrestoHook from airflow.plugins_manager import AirflowPlugin diff --git a/airflow/contrib/sensors/aws_athena_sensor.py b/airflow/contrib/sensors/aws_athena_sensor.py index 1f09d32a2f8a95..6626fe03c7960b 100644 --- a/airflow/contrib/sensors/aws_athena_sensor.py +++ b/airflow/contrib/sensors/aws_athena_sensor.py @@ -18,10 +18,10 @@ # under the License. -from airflow.exceptions import AirflowException -from airflow.utils.decorators import apply_defaults from airflow.contrib.hooks.aws_athena_hook import AWSAthenaHook +from airflow.exceptions import AirflowException from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class AthenaSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/aws_sqs_sensor.py b/airflow/contrib/sensors/aws_sqs_sensor.py index 7dd69cc0a20d10..ad692a1cac3d38 100644 --- a/airflow/contrib/sensors/aws_sqs_sensor.py +++ b/airflow/contrib/sensors/aws_sqs_sensor.py @@ -17,10 +17,10 @@ # specific language governing permissions and limitations # under the License. -from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.decorators import apply_defaults from airflow.contrib.hooks.aws_sqs_hook import SQSHook from airflow.exceptions import AirflowException +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class SQSSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/bash_sensor.py b/airflow/contrib/sensors/bash_sensor.py index 5405c302799de5..5aaa2dfb87f8d3 100644 --- a/airflow/contrib/sensors/bash_sensor.py +++ b/airflow/contrib/sensors/bash_sensor.py @@ -18,10 +18,11 @@ # under the License. import os -from subprocess import Popen, STDOUT, PIPE -from tempfile import gettempdir, NamedTemporaryFile -from airflow.utils.decorators import apply_defaults +from subprocess import PIPE, STDOUT, Popen +from tempfile import NamedTemporaryFile, gettempdir + from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults from airflow.utils.file import TemporaryDirectory diff --git a/airflow/contrib/sensors/celery_queue_sensor.py b/airflow/contrib/sensors/celery_queue_sensor.py index a1f5adf44c1716..d0e5831ca0a6e9 100644 --- a/airflow/contrib/sensors/celery_queue_sensor.py +++ b/airflow/contrib/sensors/celery_queue_sensor.py @@ -17,11 +17,11 @@ # specific language governing permissions and limitations # under the License. +from celery.app import control + from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults -from celery.app import control - class CeleryQueueSensor(BaseSensorOperator): """ diff --git a/airflow/contrib/sensors/datadog_sensor.py b/airflow/contrib/sensors/datadog_sensor.py index d396c6adb80c95..622d6fffc936a5 100644 --- a/airflow/contrib/sensors/datadog_sensor.py +++ b/airflow/contrib/sensors/datadog_sensor.py @@ -16,11 +16,12 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from airflow.sensors.base_sensor_operator import BaseSensorOperator +from datadog import api + from airflow.contrib.hooks.datadog_hook import DatadogHook -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException -from datadog import api +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults class DatadogSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/emr_base_sensor.py b/airflow/contrib/sensors/emr_base_sensor.py index cf785e3f5fb785..b8cf5468817ec2 100644 --- a/airflow/contrib/sensors/emr_base_sensor.py +++ b/airflow/contrib/sensors/emr_base_sensor.py @@ -16,9 +16,9 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from airflow.exceptions import AirflowException from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException class EmrBaseSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/gcp_transfer_sensor.py b/airflow/contrib/sensors/gcp_transfer_sensor.py index 7d7ad2cb255b20..8b43edaf9a2c9e 100644 --- a/airflow/contrib/sensors/gcp_transfer_sensor.py +++ b/airflow/contrib/sensors/gcp_transfer_sensor.py @@ -22,7 +22,7 @@ # pylint: disable=unused-import from airflow.gcp.sensors.cloud_storage_transfer_service import ( # noqa - GCPTransferServiceWaitForJobStatusSensor + GCPTransferServiceWaitForJobStatusSensor, ) warnings.warn( diff --git a/airflow/contrib/sensors/gcs_sensor.py b/airflow/contrib/sensors/gcs_sensor.py index 3536e52560749a..6a205e50d9e8ab 100644 --- a/airflow/contrib/sensors/gcs_sensor.py +++ b/airflow/contrib/sensors/gcs_sensor.py @@ -22,10 +22,8 @@ # pylint: disable=unused-import from airflow.gcp.sensors.gcs import ( # noqa - GoogleCloudStorageObjectSensor, - GoogleCloudStorageObjectUpdatedSensor, - GoogleCloudStoragePrefixSensor, - GoogleCloudStorageUploadSessionCompleteSensor + GoogleCloudStorageObjectSensor, GoogleCloudStorageObjectUpdatedSensor, GoogleCloudStoragePrefixSensor, + GoogleCloudStorageUploadSessionCompleteSensor, ) warnings.warn( diff --git a/airflow/contrib/sensors/jira_sensor.py b/airflow/contrib/sensors/jira_sensor.py index e8b8a2fc635421..7716af76447a5d 100644 --- a/airflow/contrib/sensors/jira_sensor.py +++ b/airflow/contrib/sensors/jira_sensor.py @@ -18,8 +18,7 @@ # under the License. from jira.resources import Resource -from airflow.contrib.operators.jira_operator import JIRAError -from airflow.contrib.operators.jira_operator import JiraOperator +from airflow.contrib.operators.jira_operator import JIRAError, JiraOperator from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/sensors/python_sensor.py b/airflow/contrib/sensors/python_sensor.py index a4e5ec77aa520f..a6a09a9adaa824 100644 --- a/airflow/contrib/sensors/python_sensor.py +++ b/airflow/contrib/sensors/python_sensor.py @@ -16,11 +16,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from airflow.operators.python_operator import PythonOperator +from typing import Callable, Dict, List, Optional +from airflow.operators.python_operator import PythonOperator from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults -from typing import Optional, Dict, Callable, List class PythonSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/redis_pub_sub_sensor.py b/airflow/contrib/sensors/redis_pub_sub_sensor.py index f91fbfe5e323b4..5cf1fa3047f996 100644 --- a/airflow/contrib/sensors/redis_pub_sub_sensor.py +++ b/airflow/contrib/sensors/redis_pub_sub_sensor.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. +from airflow.contrib.hooks.redis_hook import RedisHook from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults -from airflow.contrib.hooks.redis_hook import RedisHook class RedisPubSubSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/sagemaker_base_sensor.py b/airflow/contrib/sensors/sagemaker_base_sensor.py index 8209daaf5b4a6a..72183093bfb804 100644 --- a/airflow/contrib/sensors/sagemaker_base_sensor.py +++ b/airflow/contrib/sensors/sagemaker_base_sensor.py @@ -16,9 +16,9 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from airflow.exceptions import AirflowException from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException class SageMakerBaseSensor(BaseSensorOperator): diff --git a/airflow/contrib/sensors/sagemaker_training_sensor.py b/airflow/contrib/sensors/sagemaker_training_sensor.py index 3e8673e8d73428..f54bb9b0d9cfaa 100644 --- a/airflow/contrib/sensors/sagemaker_training_sensor.py +++ b/airflow/contrib/sensors/sagemaker_training_sensor.py @@ -19,7 +19,7 @@ import time -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook, LogState +from airflow.contrib.hooks.sagemaker_hook import LogState, SageMakerHook from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/sensors/sftp_sensor.py b/airflow/contrib/sensors/sftp_sensor.py index 3324d6c202130e..39dc0d953c8418 100644 --- a/airflow/contrib/sensors/sftp_sensor.py +++ b/airflow/contrib/sensors/sftp_sensor.py @@ -18,6 +18,7 @@ # under the License. from paramiko import SFTP_NO_SUCH_FILE + from airflow.contrib.hooks.sftp_hook import SFTPHook from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/contrib/task_runner/cgroup_task_runner.py b/airflow/contrib/task_runner/cgroup_task_runner.py index a54e4350bb8862..6e149ac7581f98 100644 --- a/airflow/contrib/task_runner/cgroup_task_runner.py +++ b/airflow/contrib/task_runner/cgroup_task_runner.py @@ -24,8 +24,8 @@ import os import uuid -from cgroupspy import trees import psutil +from cgroupspy import trees from airflow.task.task_runner.base_task_runner import BaseTaskRunner from airflow.utils.helpers import reap_process_group diff --git a/airflow/contrib/utils/gcp_field_validator.py b/airflow/contrib/utils/gcp_field_validator.py index 74c6fc09a16a21..b8987e1c03b909 100644 --- a/airflow/contrib/utils/gcp_field_validator.py +++ b/airflow/contrib/utils/gcp_field_validator.py @@ -24,9 +24,7 @@ # pylint: disable=unused-import from airflow.gcp.utils.field_validator import ( # noqa - GcpBodyFieldValidator, - GcpFieldValidationException, - GcpValidationSpecificationException + GcpBodyFieldValidator, GcpFieldValidationException, GcpValidationSpecificationException, ) warnings.warn( diff --git a/airflow/contrib/utils/log/task_handler_with_custom_formatter.py b/airflow/contrib/utils/log/task_handler_with_custom_formatter.py index e4d38f24e42f48..f6ca3eb1da568d 100644 --- a/airflow/contrib/utils/log/task_handler_with_custom_formatter.py +++ b/airflow/contrib/utils/log/task_handler_with_custom_formatter.py @@ -21,8 +21,8 @@ """ import logging - from logging import StreamHandler + from airflow.configuration import conf from airflow.utils.helpers import parse_template_string diff --git a/airflow/contrib/utils/mlengine_operator_utils.py b/airflow/contrib/utils/mlengine_operator_utils.py index c1ae9e220882c8..fc8badb3d6f9a7 100644 --- a/airflow/contrib/utils/mlengine_operator_utils.py +++ b/airflow/contrib/utils/mlengine_operator_utils.py @@ -22,6 +22,7 @@ # pylint: disable=unused-import from airflow.gcp.utils.mlengine_operator_utils import create_evaluate_ops # noqa + warnings.warn( "This module is deprecated. Please use `airflow.gcp.utils.mlengine_operator_utils`.", DeprecationWarning, stacklevel=2 diff --git a/airflow/contrib/utils/mlengine_prediction_summary.py b/airflow/contrib/utils/mlengine_prediction_summary.py index 665eccd523fa07..51cc7a7053f0a4 100644 --- a/airflow/contrib/utils/mlengine_prediction_summary.py +++ b/airflow/contrib/utils/mlengine_prediction_summary.py @@ -21,7 +21,7 @@ import warnings # pylint: disable=unused-import -from airflow.gcp.utils.mlengine_prediction_summary import JsonCoder, run, MakeSummary # noqa +from airflow.gcp.utils.mlengine_prediction_summary import JsonCoder, MakeSummary # noqa warnings.warn( "This module is deprecated. Please use `airflow.gcp.utils.mlengine_prediction_summary`.", diff --git a/airflow/contrib/utils/sendgrid.py b/airflow/contrib/utils/sendgrid.py index 243b46f47f465c..d90898c5019b5b 100644 --- a/airflow/contrib/utils/sendgrid.py +++ b/airflow/contrib/utils/sendgrid.py @@ -26,8 +26,8 @@ import sendgrid from sendgrid.helpers.mail import ( - Attachment, Content, Email, Mail, Personalization, CustomArg, Category, - MailSettings, SandBoxMode) + Attachment, Category, Content, CustomArg, Email, Mail, MailSettings, Personalization, SandBoxMode, +) from airflow.utils.email import get_email_address_list from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/dag/serialization/__init__.py b/airflow/dag/serialization/__init__.py index 5c4eae28469ed6..3bce221040f30f 100644 --- a/airflow/dag/serialization/__init__.py +++ b/airflow/dag/serialization/__init__.py @@ -20,5 +20,5 @@ """DAG serialization.""" from airflow.dag.serialization.serialization import Serialization # noqa: F401 -from airflow.dag.serialization.serialized_dag import SerializedDAG # noqa: F401 from airflow.dag.serialization.serialized_baseoperator import SerializedBaseOperator # noqa: F401 +from airflow.dag.serialization.serialized_dag import SerializedDAG # noqa: F401 diff --git a/airflow/dag/serialization/serialization.py b/airflow/dag/serialization/serialization.py index aeba939369c1a7..756fc385220701 100644 --- a/airflow/dag/serialization/serialization.py +++ b/airflow/dag/serialization/serialization.py @@ -22,7 +22,7 @@ import datetime import json import logging -from typing import Dict, Optional, TYPE_CHECKING, Union +from typing import TYPE_CHECKING, Dict, Optional, Union import dateutil.parser import jsonschema @@ -31,12 +31,11 @@ import airflow from airflow.dag.serialization.enum import DagAttributeTypes as DAT, Encoding from airflow.exceptions import AirflowException -from airflow.models import BaseOperator, DAG +from airflow.models import DAG, BaseOperator from airflow.models.connection import Connection from airflow.utils.log.logging_mixin import LoggingMixin from airflow.www.utils import get_python_source - if TYPE_CHECKING: from airflow.dag.serialization.serialized_baseoperator import SerializedBaseOperator # noqa: F401, E501; # pylint: disable=cyclic-import from airflow.dag.serialization.serialized_dag import SerializedDAG # noqa: F401, E501; # pylint: disable=cyclic-import diff --git a/airflow/example_dags/example_gcs_to_bq.py b/airflow/example_dags/example_gcs_to_bq.py index 2a7b7050fc1234..91149457746fb9 100644 --- a/airflow/example_dags/example_gcs_to_bq.py +++ b/airflow/example_dags/example_gcs_to_bq.py @@ -22,10 +22,8 @@ import airflow from airflow import models from airflow.operators import bash_operator - from airflow.operators.gcs_to_bq import GoogleCloudStorageToBigQueryOperator - args = { 'owner': 'Airflow', 'start_date': airflow.utils.dates.days_ago(2) diff --git a/airflow/example_dags/example_gcs_to_gcs.py b/airflow/example_dags/example_gcs_to_gcs.py index ff7cddd1880b28..e757917bb4d246 100644 --- a/airflow/example_dags/example_gcs_to_gcs.py +++ b/airflow/example_dags/example_gcs_to_gcs.py @@ -21,6 +21,7 @@ """ import os + import airflow from airflow import models from airflow.operators.gcs_to_gcs import GoogleCloudStorageSynchronizeBuckets diff --git a/airflow/executors/__init__.py b/airflow/executors/__init__.py index 258682b7903032..1ee664f20f2128 100644 --- a/airflow/executors/__init__.py +++ b/airflow/executors/__init__.py @@ -17,14 +17,15 @@ # specific language governing permissions and limitations # under the License. -from typing import Optional import sys -from airflow.utils.log.logging_mixin import LoggingMixin +from typing import Optional + from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.executors.base_executor import BaseExecutor from airflow.executors.local_executor import LocalExecutor from airflow.executors.sequential_executor import SequentialExecutor +from airflow.utils.log.logging_mixin import LoggingMixin DEFAULT_EXECUTOR = None # type: Optional[BaseExecutor] diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py index 8558918ee12b84..83fc44b59f3f90 100644 --- a/airflow/executors/celery_executor.py +++ b/airflow/executors/celery_executor.py @@ -24,11 +24,10 @@ import traceback from multiprocessing import Pool, cpu_count -from celery import Celery -from celery import states as celery_states +from celery import Celery, states as celery_states -from airflow.configuration import conf from airflow.config_templates.default_celery import DEFAULT_CELERY_CONFIG +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.executors.base_executor import BaseExecutor from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/executors/dask_executor.py b/airflow/executors/dask_executor.py index 2ddf2ec3112ccd..b355c4b6e7e86e 100644 --- a/airflow/executors/dask_executor.py +++ b/airflow/executors/dask_executor.py @@ -17,10 +17,11 @@ # specific language governing permissions and limitations # under the License. -import distributed import subprocess import warnings +import distributed + from airflow.configuration import conf from airflow.executors.base_executor import BaseExecutor diff --git a/airflow/executors/kubernetes_executor.py b/airflow/executors/kubernetes_executor.py index 1d23725abfe4a4..3134455a21fd55 100644 --- a/airflow/executors/kubernetes_executor.py +++ b/airflow/executors/kubernetes_executor.py @@ -17,30 +17,29 @@ """Kubernetes executor""" import base64 import hashlib -from queue import Empty - -import re import json import multiprocessing +import re +from queue import Empty from uuid import uuid4 -from dateutil import parser - import kubernetes -from kubernetes import watch, client +from dateutil import parser +from kubernetes import client, watch from kubernetes.client.rest import ApiException -from airflow.kubernetes.pod_launcher import PodLauncher -from airflow.kubernetes.kube_client import get_kube_client -from airflow.kubernetes.worker_configuration import WorkerConfiguration -from airflow.kubernetes.pod_generator import PodGenerator -from airflow.executors.base_executor import BaseExecutor -from airflow.models import KubeResourceVersion, KubeWorkerIdentifier, TaskInstance -from airflow.utils.state import State -from airflow.utils.db import provide_session, create_session + from airflow import settings from airflow.configuration import conf from airflow.exceptions import AirflowConfigException, AirflowException +from airflow.executors.base_executor import BaseExecutor +from airflow.kubernetes.kube_client import get_kube_client +from airflow.kubernetes.pod_generator import PodGenerator +from airflow.kubernetes.pod_launcher import PodLauncher +from airflow.kubernetes.worker_configuration import WorkerConfiguration +from airflow.models import KubeResourceVersion, KubeWorkerIdentifier, TaskInstance +from airflow.utils.db import create_session, provide_session from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.state import State MAX_POD_ID_LEN = 253 MAX_LABEL_LEN = 63 diff --git a/airflow/executors/local_executor.py b/airflow/executors/local_executor.py index 66af4941bf01cf..bf6209b0e2d9c3 100644 --- a/airflow/executors/local_executor.py +++ b/airflow/executors/local_executor.py @@ -46,7 +46,6 @@ import multiprocessing import subprocess - from queue import Empty from airflow.executors.base_executor import BaseExecutor diff --git a/airflow/gcp/example_dags/example_automl_nl_text_classification.py b/airflow/gcp/example_dags/example_automl_nl_text_classification.py index 3d2eda88782d81..6179af12253f67 100644 --- a/airflow/gcp/example_dags/example_automl_nl_text_classification.py +++ b/airflow/gcp/example_dags/example_automl_nl_text_classification.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_nl_text_extraction.py b/airflow/gcp/example_dags/example_automl_nl_text_extraction.py index ddf24efa8b1e8e..1c62816254b116 100644 --- a/airflow/gcp/example_dags/example_automl_nl_text_extraction.py +++ b/airflow/gcp/example_dags/example_automl_nl_text_extraction.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_nl_text_sentiment.py b/airflow/gcp/example_dags/example_automl_nl_text_sentiment.py index c82ee4b3baa15d..1de0c0e64ad1c6 100644 --- a/airflow/gcp/example_dags/example_automl_nl_text_sentiment.py +++ b/airflow/gcp/example_dags/example_automl_nl_text_sentiment.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_tables.py b/airflow/gcp/example_dags/example_automl_tables.py index 48e049a99fb247..cf3d1438b24da7 100644 --- a/airflow/gcp/example_dags/example_automl_tables.py +++ b/airflow/gcp/example_dags/example_automl_tables.py @@ -20,27 +20,18 @@ """ Example Airflow DAG that uses Google AutoML services. """ -from typing import List, Dict import os from copy import deepcopy +from typing import Dict, List import airflow from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLTablesListColumnSpecsOperator, - AutoMLPredictOperator, - AutoMLBatchPredictOperator, - AutoMLCreateDatasetOperator, - AutoMLTablesUpdateDatasetOperator, - AutoMLTablesListTableSpecsOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLListDatasetOperator, - AutoMLDeployModelOperator, - AutoMLGetModelOperator, - AutoMLDeleteModelOperator, + AutoMLBatchPredictOperator, AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, + AutoMLDeleteModelOperator, AutoMLDeployModelOperator, AutoMLGetModelOperator, AutoMLImportDataOperator, + AutoMLListDatasetOperator, AutoMLPredictOperator, AutoMLTablesListColumnSpecsOperator, + AutoMLTablesListTableSpecsOperator, AutoMLTablesUpdateDatasetOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_translation.py b/airflow/gcp/example_dags/example_automl_translation.py index 55bc53b4ce08b3..e52857a68ba5ec 100644 --- a/airflow/gcp/example_dags/example_automl_translation.py +++ b/airflow/gcp/example_dags/example_automl_translation.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_video_intelligence_classification.py b/airflow/gcp/example_dags/example_automl_video_intelligence_classification.py index 7f004dc7647cf1..61b4e9b5e130f4 100644 --- a/airflow/gcp/example_dags/example_automl_video_intelligence_classification.py +++ b/airflow/gcp/example_dags/example_automl_video_intelligence_classification.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_video_intelligence_tracking.py b/airflow/gcp/example_dags/example_automl_video_intelligence_tracking.py index 8cc0b9ef86937a..1a4bbe04c6b791 100644 --- a/airflow/gcp/example_dags/example_automl_video_intelligence_tracking.py +++ b/airflow/gcp/example_dags/example_automl_video_intelligence_tracking.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_vision_classification.py b/airflow/gcp/example_dags/example_automl_vision_classification.py index 8bdcac5b41766b..533ced6770b273 100644 --- a/airflow/gcp/example_dags/example_automl_vision_classification.py +++ b/airflow/gcp/example_dags/example_automl_vision_classification.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_automl_vision_object_detection.py b/airflow/gcp/example_dags/example_automl_vision_object_detection.py index 5935f513ed2293..a08a6fc4266db7 100644 --- a/airflow/gcp/example_dags/example_automl_vision_object_detection.py +++ b/airflow/gcp/example_dags/example_automl_vision_object_detection.py @@ -26,11 +26,8 @@ from airflow import models from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, AutoMLDeleteModelOperator, + AutoMLImportDataOperator, AutoMLTrainModelOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "your-project-id") diff --git a/airflow/gcp/example_dags/example_bigquery.py b/airflow/gcp/example_dags/example_bigquery.py index d6d8a658a93c87..86856475ee0ece 100644 --- a/airflow/gcp/example_dags/example_bigquery.py +++ b/airflow/gcp/example_dags/example_bigquery.py @@ -26,22 +26,14 @@ import airflow from airflow import models from airflow.gcp.operators.bigquery import ( - BigQueryOperator, - BigQueryCreateEmptyTableOperator, - BigQueryCreateEmptyDatasetOperator, - BigQueryGetDatasetOperator, - BigQueryPatchDatasetOperator, - BigQueryUpdateDatasetOperator, - BigQueryDeleteDatasetOperator, - BigQueryCreateExternalTableOperator, - BigQueryGetDataOperator, - BigQueryTableDeleteOperator, - BigQueryGetDatasetTablesOperator + BigQueryCreateEmptyDatasetOperator, BigQueryCreateEmptyTableOperator, BigQueryCreateExternalTableOperator, + BigQueryDeleteDatasetOperator, BigQueryGetDataOperator, BigQueryGetDatasetOperator, + BigQueryGetDatasetTablesOperator, BigQueryOperator, BigQueryPatchDatasetOperator, + BigQueryTableDeleteOperator, BigQueryUpdateDatasetOperator, ) - +from airflow.operators.bash_operator import BashOperator from airflow.operators.bigquery_to_bigquery import BigQueryToBigQueryOperator from airflow.operators.bigquery_to_gcs import BigQueryToCloudStorageOperator -from airflow.operators.bash_operator import BashOperator # 0x06012c8cf97BEaD5deAe237070F9587f8E7A266d = CryptoKitties contract address WALLET_ADDRESS = os.environ.get("GCP_ETH_WALLET_ADDRESS", "0x06012c8cf97BEaD5deAe237070F9587f8E7A266d") diff --git a/airflow/gcp/example_dags/example_bigquery_dts.py b/airflow/gcp/example_dags/example_bigquery_dts.py index e1a308603dc198..6760890d6e5044 100644 --- a/airflow/gcp/example_dags/example_bigquery_dts.py +++ b/airflow/gcp/example_dags/example_bigquery_dts.py @@ -23,20 +23,16 @@ import os import time -from google.protobuf.json_format import ParseDict from google.cloud.bigquery_datatransfer_v1.types import TransferConfig +from google.protobuf.json_format import ParseDict import airflow from airflow import models from airflow.gcp.operators.bigquery_dts import ( - BigQueryCreateDataTransferOperator, + BigQueryCreateDataTransferOperator, BigQueryDataTransferServiceStartTransferRunsOperator, BigQueryDeleteDataTransferConfigOperator, - BigQueryDataTransferServiceStartTransferRunsOperator, ) -from airflow.gcp.sensors.bigquery_dts import ( - BigQueryDataTransferServiceTransferRunSensor, -) - +from airflow.gcp.sensors.bigquery_dts import BigQueryDataTransferServiceTransferRunSensor GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") BUCKET_URI = os.environ.get( diff --git a/airflow/gcp/example_dags/example_bigtable.py b/airflow/gcp/example_dags/example_bigtable.py index e8bbc8e665fab8..3eaf79ae976a16 100644 --- a/airflow/gcp/example_dags/example_bigtable.py +++ b/airflow/gcp/example_dags/example_bigtable.py @@ -48,17 +48,14 @@ """ import json - from os import getenv import airflow from airflow import models -from airflow.gcp.operators.bigtable import \ - BigtableInstanceCreateOperator, \ - BigtableInstanceDeleteOperator, \ - BigtableClusterUpdateOperator, \ - BigtableTableCreateOperator, \ - BigtableTableDeleteOperator +from airflow.gcp.operators.bigtable import ( + BigtableClusterUpdateOperator, BigtableInstanceCreateOperator, BigtableInstanceDeleteOperator, + BigtableTableCreateOperator, BigtableTableDeleteOperator, +) from airflow.gcp.sensors.bigtable import BigtableTableWaitForReplicationSensor # [START howto_operator_gcp_bigtable_args] diff --git a/airflow/gcp/example_dags/example_cloud_memorystore.py b/airflow/gcp/example_dags/example_cloud_memorystore.py index 45026a6bd26156..dbdab126664a96 100644 --- a/airflow/gcp/example_dags/example_cloud_memorystore.py +++ b/airflow/gcp/example_dags/example_cloud_memorystore.py @@ -22,26 +22,21 @@ import os from urllib.parse import urlparse -from google.cloud.redis_v1.gapic.enums import Instance, FailoverInstanceRequest +from google.cloud.redis_v1.gapic.enums import FailoverInstanceRequest, Instance from airflow import models -from airflow.gcp.operators.gcs import GoogleCloudStorageBucketCreateAclEntryOperator from airflow.gcp.operators.cloud_memorystore import ( - CloudMemorystoreCreateInstanceOperator, - CloudMemorystoreDeleteInstanceOperator, - CloudMemorystoreExportInstanceOperator, - CloudMemorystoreFailoverInstanceOperator, - CloudMemorystoreGetInstanceOperator, - CloudMemorystoreImportOperator, - CloudMemorystoreListInstancesOperator, + CloudMemorystoreCreateInstanceAndImportOperator, CloudMemorystoreCreateInstanceOperator, + CloudMemorystoreDeleteInstanceOperator, CloudMemorystoreExportAndDeleteInstanceOperator, + CloudMemorystoreExportInstanceOperator, CloudMemorystoreFailoverInstanceOperator, + CloudMemorystoreGetInstanceOperator, CloudMemorystoreImportOperator, + CloudMemorystoreListInstancesOperator, CloudMemorystoreScaleInstanceOperator, CloudMemorystoreUpdateInstanceOperator, - CloudMemorystoreCreateInstanceAndImportOperator, - CloudMemorystoreScaleInstanceOperator, - CloudMemorystoreExportAndDeleteInstanceOperator) +) +from airflow.gcp.operators.gcs import GoogleCloudStorageBucketCreateAclEntryOperator from airflow.operators.bash_operator import BashOperator from airflow.utils import dates - GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") INSTANCE_NAME = os.environ.get("GCP_MEMORYSTORE_INSTANCE_NAME", "test-memorystore") diff --git a/airflow/gcp/example_dags/example_cloud_sql.py b/airflow/gcp/example_dags/example_cloud_sql.py index f778bade0b1f5f..53806b0fa0a71c 100644 --- a/airflow/gcp/example_dags/example_cloud_sql.py +++ b/airflow/gcp/example_dags/example_cloud_sql.py @@ -33,14 +33,15 @@ import airflow from airflow import models -from airflow.gcp.operators.cloud_sql import CloudSqlInstanceCreateOperator, \ - CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator, \ - CloudSqlInstanceDatabaseCreateOperator, CloudSqlInstanceDatabasePatchOperator, \ - CloudSqlInstanceDatabaseDeleteOperator, CloudSqlInstanceExportOperator, \ - CloudSqlInstanceImportOperator -from airflow.gcp.operators.gcs import \ - GoogleCloudStorageBucketCreateAclEntryOperator, \ - GoogleCloudStorageObjectCreateAclEntryOperator +from airflow.gcp.operators.cloud_sql import ( + CloudSqlInstanceCreateOperator, CloudSqlInstanceDatabaseCreateOperator, + CloudSqlInstanceDatabaseDeleteOperator, CloudSqlInstanceDatabasePatchOperator, + CloudSqlInstanceDeleteOperator, CloudSqlInstanceExportOperator, CloudSqlInstanceImportOperator, + CloudSqlInstancePatchOperator, +) +from airflow.gcp.operators.gcs import ( + GoogleCloudStorageBucketCreateAclEntryOperator, GoogleCloudStorageObjectCreateAclEntryOperator, +) # [START howto_operator_cloudsql_arguments] GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') diff --git a/airflow/gcp/example_dags/example_cloud_sql_query.py b/airflow/gcp/example_dags/example_cloud_sql_query.py index 685107df7d8fee..ea2c11a1b293e7 100644 --- a/airflow/gcp/example_dags/example_cloud_sql_query.py +++ b/airflow/gcp/example_dags/example_cloud_sql_query.py @@ -37,8 +37,8 @@ * GCSQL_MYSQL_PUBLIC_IP - Public IP of the mysql database * GCSQL_MYSQL_PUBLIC_PORT - Port of the mysql database """ -import subprocess import os +import subprocess from os.path import expanduser from urllib.parse import quote_plus diff --git a/airflow/gcp/example_dags/example_cloud_storage_transfer_service.py b/airflow/gcp/example_dags/example_cloud_storage_transfer_service.py index 7d47d61dea57e4..4313df64c32235 100644 --- a/airflow/gcp/example_dags/example_cloud_storage_transfer_service.py +++ b/airflow/gcp/example_dags/example_cloud_storage_transfer_service.py @@ -45,36 +45,16 @@ from airflow import models from airflow.gcp.hooks.cloud_storage_transfer_service import ( - GcpTransferOperationStatus, - GcpTransferJobsStatus, - TRANSFER_OPTIONS, - PROJECT_ID, - BUCKET_NAME, - GCS_DATA_SINK, - STATUS, - DESCRIPTION, - GCS_DATA_SOURCE, - START_TIME_OF_DAY, - SCHEDULE_END_DATE, - SCHEDULE_START_DATE, - SCHEDULE, - AWS_S3_DATA_SOURCE, - TRANSFER_SPEC, - FILTER_PROJECT_ID, - FILTER_JOB_NAMES, - TRANSFER_JOB, - TRANSFER_JOB_FIELD_MASK, - ALREADY_EXISTING_IN_SINK, + ALREADY_EXISTING_IN_SINK, AWS_S3_DATA_SOURCE, BUCKET_NAME, DESCRIPTION, FILTER_JOB_NAMES, + FILTER_PROJECT_ID, GCS_DATA_SINK, GCS_DATA_SOURCE, PROJECT_ID, SCHEDULE, SCHEDULE_END_DATE, + SCHEDULE_START_DATE, START_TIME_OF_DAY, STATUS, TRANSFER_JOB, TRANSFER_JOB_FIELD_MASK, TRANSFER_OPTIONS, + TRANSFER_SPEC, GcpTransferJobsStatus, GcpTransferOperationStatus, ) from airflow.gcp.operators.cloud_storage_transfer_service import ( - GcpTransferServiceJobCreateOperator, - GcpTransferServiceJobDeleteOperator, - GcpTransferServiceJobUpdateOperator, - GcpTransferServiceOperationsListOperator, - GcpTransferServiceOperationGetOperator, - GcpTransferServiceOperationPauseOperator, - GcpTransferServiceOperationResumeOperator, - GcpTransferServiceOperationCancelOperator, + GcpTransferServiceJobCreateOperator, GcpTransferServiceJobDeleteOperator, + GcpTransferServiceJobUpdateOperator, GcpTransferServiceOperationCancelOperator, + GcpTransferServiceOperationGetOperator, GcpTransferServiceOperationPauseOperator, + GcpTransferServiceOperationResumeOperator, GcpTransferServiceOperationsListOperator, ) from airflow.gcp.sensors.cloud_storage_transfer_service import GCPTransferServiceWaitForJobStatusSensor from airflow.utils.dates import days_ago diff --git a/airflow/gcp/example_dags/example_compute.py b/airflow/gcp/example_dags/example_compute.py index 724ec0fa91ac8f..967a40ca7306b2 100644 --- a/airflow/gcp/example_dags/example_compute.py +++ b/airflow/gcp/example_dags/example_compute.py @@ -34,8 +34,9 @@ import airflow from airflow import models -from airflow.gcp.operators.compute import GceInstanceStartOperator, \ - GceInstanceStopOperator, GceSetMachineTypeOperator +from airflow.gcp.operators.compute import ( + GceInstanceStartOperator, GceInstanceStopOperator, GceSetMachineTypeOperator, +) # [START howto_operator_gce_args_common] GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') diff --git a/airflow/gcp/example_dags/example_compute_igm.py b/airflow/gcp/example_dags/example_compute_igm.py index 7093181b2b2f74..20a4b0ca856cb7 100644 --- a/airflow/gcp/example_dags/example_compute_igm.py +++ b/airflow/gcp/example_dags/example_compute_igm.py @@ -43,8 +43,9 @@ import airflow from airflow import models -from airflow.gcp.operators.compute import \ - GceInstanceTemplateCopyOperator, GceInstanceGroupManagerUpdateTemplateOperator +from airflow.gcp.operators.compute import ( + GceInstanceGroupManagerUpdateTemplateOperator, GceInstanceTemplateCopyOperator, +) # [START howto_operator_compute_igm_common_args] GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') diff --git a/airflow/gcp/example_dags/example_dataflow.py b/airflow/gcp/example_dags/example_dataflow.py index a4e0c5e3c63ebc..2dde25d2b5de17 100644 --- a/airflow/gcp/example_dags/example_dataflow.py +++ b/airflow/gcp/example_dags/example_dataflow.py @@ -24,8 +24,9 @@ import airflow from airflow import models -from airflow.gcp.operators.dataflow import DataFlowJavaOperator, CheckJobRunning, DataFlowPythonOperator, \ - DataflowTemplateOperator +from airflow.gcp.operators.dataflow import ( + CheckJobRunning, DataFlowJavaOperator, DataFlowPythonOperator, DataflowTemplateOperator, +) GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') GCS_TMP = os.environ.get('GCP_DATAFLOW_GCS_TMP', 'gs://test-dataflow-example/temp/') diff --git a/airflow/gcp/example_dags/example_dataproc.py b/airflow/gcp/example_dags/example_dataproc.py index 9a5ee48210fba7..c07492c730e450 100644 --- a/airflow/gcp/example_dags/example_dataproc.py +++ b/airflow/gcp/example_dags/example_dataproc.py @@ -22,18 +22,13 @@ """ import os + import airflow from airflow import models from airflow.gcp.operators.dataproc import ( - DataprocClusterCreateOperator, - DataprocClusterDeleteOperator, - DataprocClusterScaleOperator, - DataProcSparkSqlOperator, - DataProcSparkOperator, - DataProcPySparkOperator, - DataProcPigOperator, - DataProcHiveOperator, - DataProcHadoopOperator, + DataprocClusterCreateOperator, DataprocClusterDeleteOperator, DataprocClusterScaleOperator, + DataProcHadoopOperator, DataProcHiveOperator, DataProcPigOperator, DataProcPySparkOperator, + DataProcSparkOperator, DataProcSparkSqlOperator, ) PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "an-id") diff --git a/airflow/gcp/example_dags/example_datastore.py b/airflow/gcp/example_dags/example_datastore.py index 911bbc5fb01d5f..9bcf9ab1633645 100644 --- a/airflow/gcp/example_dags/example_datastore.py +++ b/airflow/gcp/example_dags/example_datastore.py @@ -26,11 +26,8 @@ import os from airflow import models +from airflow.gcp.operators.datastore import DatastoreExportOperator, DatastoreImportOperator from airflow.utils import dates -from airflow.gcp.operators.datastore import ( - DatastoreImportOperator, - DatastoreExportOperator, -) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") BUCKET = os.environ.get("GCP_DATASTORE_BUCKET", "datastore-system-test") diff --git a/airflow/gcp/example_dags/example_functions.py b/airflow/gcp/example_dags/example_functions.py index 712ad7bdaa2ed5..83928044b2eacc 100644 --- a/airflow/gcp/example_dags/example_functions.py +++ b/airflow/gcp/example_dags/example_functions.py @@ -44,8 +44,9 @@ import os from airflow import models -from airflow.gcp.operators.functions \ - import GcfFunctionDeployOperator, GcfFunctionDeleteOperator, GcfFunctionInvokeOperator +from airflow.gcp.operators.functions import ( + GcfFunctionDeleteOperator, GcfFunctionDeployOperator, GcfFunctionInvokeOperator, +) from airflow.utils import dates # [START howto_operator_gcf_common_variables] diff --git a/airflow/gcp/example_dags/example_gcp_dlp_operator.py b/airflow/gcp/example_dags/example_gcp_dlp_operator.py index 1df6185bed8bfa..2067130cdcadf4 100644 --- a/airflow/gcp/example_dags/example_gcp_dlp_operator.py +++ b/airflow/gcp/example_dags/example_gcp_dlp_operator.py @@ -26,16 +26,15 @@ """ import os + from google.cloud.dlp_v2.types import ContentItem, InspectConfig, InspectTemplate import airflow -from airflow.models import DAG from airflow.gcp.operators.dlp import ( - CloudDLPCreateInspectTemplateOperator, - CloudDLPDeleteInspectTemplateOperator, + CloudDLPCreateInspectTemplateOperator, CloudDLPDeleteInspectTemplateOperator, CloudDLPInspectContentOperator, ) - +from airflow.models import DAG default_args = {"start_date": airflow.utils.dates.days_ago(1)} diff --git a/airflow/gcp/example_dags/example_gcs.py b/airflow/gcp/example_dags/example_gcs.py index cb66723a137f0a..f381e9a2a92b34 100644 --- a/airflow/gcp/example_dags/example_gcs.py +++ b/airflow/gcp/example_dags/example_gcs.py @@ -21,20 +21,18 @@ """ import os + import airflow from airflow import models -from airflow.operators.bash_operator import BashOperator -from airflow.operators.local_to_gcs import FileToGoogleCloudStorageOperator -from airflow.operators.gcs_to_gcs import GoogleCloudStorageToGoogleCloudStorageOperator from airflow.gcp.operators.gcs import ( - GoogleCloudStorageBucketCreateAclEntryOperator, + GcsFileTransformOperator, GoogleCloudStorageBucketCreateAclEntryOperator, + GoogleCloudStorageCreateBucketOperator, GoogleCloudStorageDeleteOperator, + GoogleCloudStorageDownloadOperator, GoogleCloudStorageListOperator, GoogleCloudStorageObjectCreateAclEntryOperator, - GoogleCloudStorageListOperator, - GoogleCloudStorageDeleteOperator, - GoogleCloudStorageDownloadOperator, - GoogleCloudStorageCreateBucketOperator, - GcsFileTransformOperator ) +from airflow.operators.bash_operator import BashOperator +from airflow.operators.gcs_to_gcs import GoogleCloudStorageToGoogleCloudStorageOperator +from airflow.operators.local_to_gcs import FileToGoogleCloudStorageOperator default_args = {"start_date": airflow.utils.dates.days_ago(1)} diff --git a/airflow/gcp/example_dags/example_kubernetes_engine.py b/airflow/gcp/example_dags/example_kubernetes_engine.py index 6a901c7287a7ee..73fdfb18155c19 100644 --- a/airflow/gcp/example_dags/example_kubernetes_engine.py +++ b/airflow/gcp/example_dags/example_kubernetes_engine.py @@ -21,12 +21,11 @@ """ import os + import airflow from airflow import models from airflow.gcp.operators.kubernetes_engine import ( - GKEClusterCreateOperator, - GKEClusterDeleteOperator, - GKEPodOperator, + GKEClusterCreateOperator, GKEClusterDeleteOperator, GKEPodOperator, ) GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") diff --git a/airflow/gcp/example_dags/example_natural_language.py b/airflow/gcp/example_dags/example_natural_language.py index 6539b42d77fc9b..6e16fba024e74d 100644 --- a/airflow/gcp/example_dags/example_natural_language.py +++ b/airflow/gcp/example_dags/example_natural_language.py @@ -26,10 +26,8 @@ import airflow from airflow import models from airflow.gcp.operators.natural_language import ( - CloudLanguageAnalyzeEntitiesOperator, - CloudLanguageAnalyzeEntitySentimentOperator, - CloudLanguageAnalyzeSentimentOperator, - CloudLanguageClassifyTextOperator, + CloudLanguageAnalyzeEntitiesOperator, CloudLanguageAnalyzeEntitySentimentOperator, + CloudLanguageAnalyzeSentimentOperator, CloudLanguageClassifyTextOperator, ) from airflow.operators.bash_operator import BashOperator diff --git a/airflow/gcp/example_dags/example_pubsub.py b/airflow/gcp/example_dags/example_pubsub.py index cd909ec891ebb5..af628d1563e572 100644 --- a/airflow/gcp/example_dags/example_pubsub.py +++ b/airflow/gcp/example_dags/example_pubsub.py @@ -24,13 +24,9 @@ import airflow from airflow import models - from airflow.gcp.operators.pubsub import ( - PubSubTopicCreateOperator, - PubSubSubscriptionDeleteOperator, - PubSubSubscriptionCreateOperator, - PubSubPublishOperator, - PubSubTopicDeleteOperator, + PubSubPublishOperator, PubSubSubscriptionCreateOperator, PubSubSubscriptionDeleteOperator, + PubSubTopicCreateOperator, PubSubTopicDeleteOperator, ) from airflow.gcp.sensors.pubsub import PubSubPullSensor from airflow.operators.bash_operator import BashOperator diff --git a/airflow/gcp/example_dags/example_spanner.py b/airflow/gcp/example_dags/example_spanner.py index fe9a5fc9519283..288cc81dd54989 100644 --- a/airflow/gcp/example_dags/example_spanner.py +++ b/airflow/gcp/example_dags/example_spanner.py @@ -37,13 +37,11 @@ import airflow from airflow import models -from airflow.gcp.operators.spanner import \ - CloudSpannerInstanceDeployOperator, \ - CloudSpannerInstanceDatabaseQueryOperator, \ - CloudSpannerInstanceDeleteOperator, \ - CloudSpannerInstanceDatabaseDeployOperator, \ - CloudSpannerInstanceDatabaseUpdateOperator, \ - CloudSpannerInstanceDatabaseDeleteOperator +from airflow.gcp.operators.spanner import ( + CloudSpannerInstanceDatabaseDeleteOperator, CloudSpannerInstanceDatabaseDeployOperator, + CloudSpannerInstanceDatabaseQueryOperator, CloudSpannerInstanceDatabaseUpdateOperator, + CloudSpannerInstanceDeleteOperator, CloudSpannerInstanceDeployOperator, +) # [START howto_operator_spanner_arguments] GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') diff --git a/airflow/gcp/example_dags/example_speech.py b/airflow/gcp/example_dags/example_speech.py index 50d3e784552d24..9b4c947ae39b6c 100644 --- a/airflow/gcp/example_dags/example_speech.py +++ b/airflow/gcp/example_dags/example_speech.py @@ -27,11 +27,11 @@ import os -from airflow.utils import dates from airflow import models -from airflow.gcp.operators.text_to_speech import GcpTextToSpeechSynthesizeOperator from airflow.gcp.operators.speech_to_text import GcpSpeechToTextRecognizeSpeechOperator +from airflow.gcp.operators.text_to_speech import GcpTextToSpeechSynthesizeOperator from airflow.gcp.operators.translate_speech import GcpTranslateSpeechOperator +from airflow.utils import dates # [START howto_operator_text_to_speech_env_variables] GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") diff --git a/airflow/gcp/example_dags/example_tasks.py b/airflow/gcp/example_dags/example_tasks.py index 30efc5ae35bc46..1e36262c1fd9fe 100644 --- a/airflow/gcp/example_dags/example_tasks.py +++ b/airflow/gcp/example_dags/example_tasks.py @@ -32,9 +32,7 @@ import airflow from airflow.gcp.operators.tasks import ( - CloudTasksQueueCreateOperator, - CloudTasksTaskCreateOperator, - CloudTasksTaskRunOperator, + CloudTasksQueueCreateOperator, CloudTasksTaskCreateOperator, CloudTasksTaskRunOperator, ) from airflow.models import DAG diff --git a/airflow/gcp/example_dags/example_translate.py b/airflow/gcp/example_dags/example_translate.py index 345939d328caa2..d9c0be95ecc33b 100644 --- a/airflow/gcp/example_dags/example_translate.py +++ b/airflow/gcp/example_dags/example_translate.py @@ -25,7 +25,6 @@ import airflow from airflow import models - from airflow.gcp.operators.translate import CloudTranslateTextOperator from airflow.operators.bash_operator import BashOperator diff --git a/airflow/gcp/example_dags/example_video_intelligence.py b/airflow/gcp/example_dags/example_video_intelligence.py index 70b4b5d54b7539..c606c492421d41 100644 --- a/airflow/gcp/example_dags/example_video_intelligence.py +++ b/airflow/gcp/example_dags/example_video_intelligence.py @@ -30,17 +30,17 @@ # [START howto_operator_vision_retry_import] from google.api_core.retry import Retry -# [END howto_operator_vision_retry_import] - import airflow from airflow import models from airflow.gcp.operators.video_intelligence import ( - CloudVideoIntelligenceDetectVideoLabelsOperator, - CloudVideoIntelligenceDetectVideoExplicitContentOperator, + CloudVideoIntelligenceDetectVideoExplicitContentOperator, CloudVideoIntelligenceDetectVideoLabelsOperator, CloudVideoIntelligenceDetectVideoShotsOperator, ) from airflow.operators.bash_operator import BashOperator +# [END howto_operator_vision_retry_import] + + default_args = {"start_date": airflow.utils.dates.days_ago(1)} # [START howto_operator_video_intelligence_os_args] diff --git a/airflow/gcp/example_dags/example_vision.py b/airflow/gcp/example_dags/example_vision.py index 47cd7aa5737558..5631d8939fbeeb 100644 --- a/airflow/gcp/example_dags/example_vision.py +++ b/airflow/gcp/example_dags/example_vision.py @@ -34,44 +34,35 @@ import os +import airflow +from airflow import models +from airflow.gcp.operators.vision import ( + CloudVisionAddProductToProductSetOperator, CloudVisionAnnotateImageOperator, + CloudVisionDetectDocumentTextOperator, CloudVisionDetectImageLabelsOperator, + CloudVisionDetectImageSafeSearchOperator, CloudVisionDetectTextOperator, CloudVisionProductCreateOperator, + CloudVisionProductDeleteOperator, CloudVisionProductGetOperator, CloudVisionProductSetCreateOperator, + CloudVisionProductSetDeleteOperator, CloudVisionProductSetGetOperator, + CloudVisionProductSetUpdateOperator, CloudVisionProductUpdateOperator, + CloudVisionReferenceImageCreateOperator, CloudVisionRemoveProductFromProductSetOperator, +) +from airflow.operators.bash_operator import BashOperator + # [START howto_operator_vision_retry_import] -from google.api_core.retry import Retry +from google.api_core.retry import Retry # isort:skip pylint: disable=wrong-import-order # [END howto_operator_vision_retry_import] # [START howto_operator_vision_product_set_import] -from google.cloud.vision_v1.types import ProductSet +from google.cloud.vision_v1.types import ProductSet # isort:skip pylint: disable=wrong-import-order # [END howto_operator_vision_product_set_import] # [START howto_operator_vision_product_import] -from google.cloud.vision_v1.types import Product +from google.cloud.vision_v1.types import Product # isort:skip pylint: disable=wrong-import-order # [END howto_operator_vision_product_import] # [START howto_operator_vision_reference_image_import] -from google.cloud.vision_v1.types import ReferenceImage +from google.cloud.vision_v1.types import ReferenceImage # isort:skip pylint: disable=wrong-import-order # [END howto_operator_vision_reference_image_import] # [START howto_operator_vision_enums_import] -from google.cloud.vision import enums +from google.cloud.vision import enums # isort:skip pylint: disable=wrong-import-order # [END howto_operator_vision_enums_import] -import airflow -from airflow import models -from airflow.operators.bash_operator import BashOperator - -from airflow.gcp.operators.vision import ( - CloudVisionProductSetCreateOperator, - CloudVisionProductSetGetOperator, - CloudVisionProductSetUpdateOperator, - CloudVisionProductSetDeleteOperator, - CloudVisionProductCreateOperator, - CloudVisionProductGetOperator, - CloudVisionProductUpdateOperator, - CloudVisionProductDeleteOperator, - CloudVisionReferenceImageCreateOperator, - CloudVisionAddProductToProductSetOperator, - CloudVisionRemoveProductFromProductSetOperator, - CloudVisionAnnotateImageOperator, - CloudVisionDetectTextOperator, - CloudVisionDetectDocumentTextOperator, - CloudVisionDetectImageLabelsOperator, - CloudVisionDetectImageSafeSearchOperator, -) default_args = {'start_date': airflow.utils.dates.days_ago(1)} diff --git a/airflow/gcp/hooks/automl.py b/airflow/gcp/hooks/automl.py index f6e8e0471f5bc9..6be721e0b7b703 100644 --- a/airflow/gcp/hooks/automl.py +++ b/airflow/gcp/hooks/automl.py @@ -20,24 +20,14 @@ """ This module contains a Google AutoML hook. """ -from typing import Dict, Sequence, Tuple, Union, List, Optional -from cached_property import cached_property +from typing import Dict, List, Optional, Sequence, Tuple, Union +from cached_property import cached_property from google.api_core.retry import Retry -from google.cloud.automl_v1beta1 import PredictionServiceClient, AutoMlClient +from google.cloud.automl_v1beta1 import AutoMlClient, PredictionServiceClient from google.cloud.automl_v1beta1.types import ( - BatchPredictInputConfig, - BatchPredictOutputConfig, - Model, - ExamplePayload, - Dataset, - FieldMask, - ImageObjectDetectionModelDeploymentMetadata, - PredictResponse, - ColumnSpec, - Operation, - TableSpec, - InputConfig, + BatchPredictInputConfig, BatchPredictOutputConfig, ColumnSpec, Dataset, ExamplePayload, FieldMask, + ImageObjectDetectionModelDeploymentMetadata, InputConfig, Model, Operation, PredictResponse, TableSpec, ) from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/base.py b/airflow/gcp/hooks/base.py index 859b076d8171ce..c450689a24412e 100644 --- a/airflow/gcp/hooks/base.py +++ b/airflow/gcp/hooks/base.py @@ -21,29 +21,26 @@ This module contains a Google Cloud API base hook. """ -import json import functools +import json import os import tempfile from contextlib import contextmanager -from typing import Any, Optional, Dict, Callable, TypeVar, Sequence - -import httplib2 +from typing import Any, Callable, Dict, Optional, Sequence, TypeVar import google.auth import google.oauth2.service_account +import google_auth_httplib2 +import httplib2 +from google.api_core.exceptions import AlreadyExists, GoogleAPICallError, RetryError from google.api_core.gapic_v1.client_info import ClientInfo -from google.api_core.exceptions import GoogleAPICallError, AlreadyExists, RetryError from google.auth.environment_vars import CREDENTIALS - -import google_auth_httplib2 from googleapiclient.errors import HttpError from airflow import version from airflow.exceptions import AirflowException from airflow.hooks.base_hook import BaseHook - _DEFAULT_SCOPES = ('https://www.googleapis.com/auth/cloud-platform',) # type: Sequence[str] diff --git a/airflow/gcp/hooks/bigquery.py b/airflow/gcp/hooks/bigquery.py index fad4dcb9c62f34..be12e22b231f92 100644 --- a/airflow/gcp/hooks/bigquery.py +++ b/airflow/gcp/hooks/bigquery.py @@ -23,19 +23,18 @@ """ import time -from copy import deepcopy -from typing import Any, NoReturn, Mapping, Union, Iterable, Dict, List, Optional, Tuple, Type import warnings +from copy import deepcopy +from typing import Any, Dict, Iterable, List, Mapping, NoReturn, Optional, Tuple, Type, Union from googleapiclient.discovery import build from googleapiclient.errors import HttpError from pandas import DataFrame -from pandas_gbq.gbq import \ - _check_google_client_version as gbq_check_google_client_version from pandas_gbq import read_gbq -from pandas_gbq.gbq import \ - _test_google_api_imports as gbq_test_google_api_imports -from pandas_gbq.gbq import GbqConnector +from pandas_gbq.gbq import ( + GbqConnector, _check_google_client_version as gbq_check_google_client_version, + _test_google_api_imports as gbq_test_google_api_imports, +) from airflow import AirflowException from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/bigquery_dts.py b/airflow/gcp/hooks/bigquery_dts.py index bae2906f0fe4f7..554eeb71fe49d5 100644 --- a/airflow/gcp/hooks/bigquery_dts.py +++ b/airflow/gcp/hooks/bigquery_dts.py @@ -20,17 +20,15 @@ """ This module contains a BigQuery Hook. """ -from typing import Union, Sequence, Tuple, Optional from copy import copy +from typing import Optional, Sequence, Tuple, Union -from google.protobuf.json_format import MessageToDict, ParseDict from google.api_core.retry import Retry from google.cloud.bigquery_datatransfer_v1 import DataTransferServiceClient from google.cloud.bigquery_datatransfer_v1.types import ( - TransferConfig, - StartManualTransferRunsResponse, - TransferRun, + StartManualTransferRunsResponse, TransferConfig, TransferRun, ) +from google.protobuf.json_format import MessageToDict, ParseDict from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/bigtable.py b/airflow/gcp/hooks/bigtable.py index 90fbc8d69ebdf3..a4c757ca97a6c9 100644 --- a/airflow/gcp/hooks/bigtable.py +++ b/airflow/gcp/hooks/bigtable.py @@ -23,10 +23,11 @@ from google.cloud.bigtable import Client from google.cloud.bigtable.cluster import Cluster -from google.cloud.bigtable.column_family import GarbageCollectionRule, ColumnFamily +from google.cloud.bigtable.column_family import ColumnFamily, GarbageCollectionRule from google.cloud.bigtable.instance import Instance -from google.cloud.bigtable.table import Table, ClusterState +from google.cloud.bigtable.table import ClusterState, Table from google.cloud.bigtable_admin_v2 import enums + from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/cloud_build.py b/airflow/gcp/hooks/cloud_build.py index b5737b9307a88b..b7cdb7373a9813 100644 --- a/airflow/gcp/hooks/cloud_build.py +++ b/airflow/gcp/hooks/cloud_build.py @@ -19,7 +19,7 @@ """Hook for Google Cloud Build service""" import time -from typing import Dict, Any, Optional +from typing import Any, Dict, Optional from googleapiclient.discovery import build diff --git a/airflow/gcp/hooks/cloud_memorystore.py b/airflow/gcp/hooks/cloud_memorystore.py index 13baf44e1f06cf..3822cfefeaca62 100644 --- a/airflow/gcp/hooks/cloud_memorystore.py +++ b/airflow/gcp/hooks/cloud_memorystore.py @@ -19,13 +19,13 @@ """ Hooks for Cloud Memorystore service """ -from typing import Dict, Sequence, Tuple, Union, Optional +from typing import Dict, Optional, Sequence, Tuple, Union from google.api_core.exceptions import NotFound from google.api_core.retry import Retry +from google.cloud.redis_v1 import CloudRedisClient from google.cloud.redis_v1.gapic.enums import FailoverInstanceRequest from google.cloud.redis_v1.types import FieldMask, InputConfig, Instance, OutputConfig -from google.cloud.redis_v1 import CloudRedisClient from google.protobuf.json_format import ParseDict from airflow import AirflowException, version diff --git a/airflow/gcp/hooks/cloud_sql.py b/airflow/gcp/hooks/cloud_sql.py index 41937830a1fb75..3a40e32c01e33f 100644 --- a/airflow/gcp/hooks/cloud_sql.py +++ b/airflow/gcp/hooks/cloud_sql.py @@ -25,28 +25,26 @@ import json import os import os.path +import platform import random import re import shutil +import socket import string import subprocess -from subprocess import Popen, PIPE -from typing import Dict, Union, Optional, Any, List -from urllib.parse import quote_plus - -import socket -import platform import time import uuid -import requests +from subprocess import PIPE, Popen +from typing import Any, Dict, List, Optional, Union +from urllib.parse import quote_plus -from googleapiclient.errors import HttpError +import requests from googleapiclient.discovery import build +from googleapiclient.errors import HttpError from sqlalchemy.orm import Session from airflow import AirflowException, LoggingMixin from airflow.gcp.hooks.base import GoogleCloudBaseHook - # Number of retries - used by googleapiclient method calls to perform retries # For requests that are "retriable" from airflow.hooks.base_hook import BaseHook diff --git a/airflow/gcp/hooks/cloud_storage_transfer_service.py b/airflow/gcp/hooks/cloud_storage_transfer_service.py index cb7a39966b4c70..bc15378bbd2253 100644 --- a/airflow/gcp/hooks/cloud_storage_transfer_service.py +++ b/airflow/gcp/hooks/cloud_storage_transfer_service.py @@ -25,7 +25,7 @@ import warnings from copy import deepcopy from datetime import timedelta -from typing import Dict, List, Union, Set, Optional +from typing import Dict, List, Optional, Set, Union from googleapiclient.discovery import build diff --git a/airflow/gcp/hooks/compute.py b/airflow/gcp/hooks/compute.py index 5c5006ddeff20f..b46a1de2c28de3 100644 --- a/airflow/gcp/hooks/compute.py +++ b/airflow/gcp/hooks/compute.py @@ -21,7 +21,7 @@ """ import time -from typing import Dict, Any, Optional +from typing import Any, Dict, Optional from googleapiclient.discovery import build diff --git a/airflow/gcp/hooks/dataflow.py b/airflow/gcp/hooks/dataflow.py index 64446a21174e1a..2647922b028836 100644 --- a/airflow/gcp/hooks/dataflow.py +++ b/airflow/gcp/hooks/dataflow.py @@ -26,7 +26,7 @@ import subprocess import time import uuid -from typing import Dict, List, Callable, Any, Optional, Union +from typing import Any, Callable, Dict, List, Optional, Union from googleapiclient.discovery import build diff --git a/airflow/gcp/hooks/dataproc.py b/airflow/gcp/hooks/dataproc.py index d24fc6339ed3e8..e693f1065defc2 100644 --- a/airflow/gcp/hooks/dataproc.py +++ b/airflow/gcp/hooks/dataproc.py @@ -23,7 +23,7 @@ import time import uuid -from typing import Dict, List, Optional, Any, Iterable +from typing import Any, Dict, Iterable, List, Optional from googleapiclient.discovery import build from zope.deprecation import deprecation diff --git a/airflow/gcp/hooks/datastore.py b/airflow/gcp/hooks/datastore.py index 9b6d8632b69d60..8305f0f15af832 100644 --- a/airflow/gcp/hooks/datastore.py +++ b/airflow/gcp/hooks/datastore.py @@ -22,8 +22,8 @@ """ import time -from typing import Any, List, Dict, Union, Optional import warnings +from typing import Any, Dict, List, Optional, Union from googleapiclient.discovery import build diff --git a/airflow/gcp/hooks/dlp.py b/airflow/gcp/hooks/dlp.py index 929f9bfed9dd4e..1405b70ad83e32 100644 --- a/airflow/gcp/hooks/dlp.py +++ b/airflow/gcp/hooks/dlp.py @@ -28,17 +28,12 @@ from google.api_core.retry import Retry from google.cloud.dlp_v2 import DlpServiceClient -from google.cloud.dlp_v2.types import (ByteContentItem, ContentItem, - DeidentifyConfig, - DeidentifyContentResponse, - DeidentifyTemplate, DlpJob, FieldMask, - InspectConfig, InspectContentResponse, - InspectJobConfig, InspectTemplate, - JobTrigger, ListInfoTypesResponse, - RedactImageRequest, RedactImageResponse, - ReidentifyContentResponse, - RiskAnalysisJobConfig, StoredInfoType, - StoredInfoTypeConfig) +from google.cloud.dlp_v2.types import ( + ByteContentItem, ContentItem, DeidentifyConfig, DeidentifyContentResponse, DeidentifyTemplate, DlpJob, + FieldMask, InspectConfig, InspectContentResponse, InspectJobConfig, InspectTemplate, JobTrigger, + ListInfoTypesResponse, RedactImageRequest, RedactImageResponse, ReidentifyContentResponse, + RiskAnalysisJobConfig, StoredInfoType, StoredInfoTypeConfig, +) from airflow import AirflowException from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/functions.py b/airflow/gcp/hooks/functions.py index 2c1b80b93780ba..9a426320ae266e 100644 --- a/airflow/gcp/hooks/functions.py +++ b/airflow/gcp/hooks/functions.py @@ -20,7 +20,7 @@ This module contains a Google Cloud Functions Hook. """ import time -from typing import Dict, List, Optional, Any +from typing import Any, Dict, List, Optional import requests from googleapiclient.discovery import build diff --git a/airflow/gcp/hooks/gcs.py b/airflow/gcp/hooks/gcs.py index b140df02e776c3..efb206069f58de 100644 --- a/airflow/gcp/hooks/gcs.py +++ b/airflow/gcp/hooks/gcs.py @@ -20,19 +20,19 @@ """ This module contains a Google Cloud Storage hook. """ -import os -from os import path -from typing import Optional, Set, Tuple, Union import gzip as gz +import os import shutil -from io import BytesIO import warnings - +from io import BytesIO +from os import path +from typing import Optional, Set, Tuple, Union from urllib.parse import urlparse + from google.cloud import storage -from airflow.gcp.hooks.base import GoogleCloudBaseHook from airflow.exceptions import AirflowException +from airflow.gcp.hooks.base import GoogleCloudBaseHook from airflow.version import version diff --git a/airflow/gcp/hooks/gsheets.py b/airflow/gcp/hooks/gsheets.py index ee3d21b8a1be9a..b6cab3d87eb462 100644 --- a/airflow/gcp/hooks/gsheets.py +++ b/airflow/gcp/hooks/gsheets.py @@ -22,9 +22,11 @@ """ from typing import Any, Dict, List, Optional + from googleapiclient.discovery import build -from airflow.gcp.hooks.base import GoogleCloudBaseHook + from airflow.exceptions import AirflowException +from airflow.gcp.hooks.base import GoogleCloudBaseHook class GSheetsHook(GoogleCloudBaseHook): diff --git a/airflow/gcp/hooks/kubernetes_engine.py b/airflow/gcp/hooks/kubernetes_engine.py index aaa1555ae05664..624e9b8ed7ebac 100644 --- a/airflow/gcp/hooks/kubernetes_engine.py +++ b/airflow/gcp/hooks/kubernetes_engine.py @@ -23,12 +23,11 @@ import time import warnings -from typing import Dict, Union, Optional +from typing import Dict, Optional, Union from google.api_core.exceptions import AlreadyExists, NotFound from google.api_core.gapic_v1.method import DEFAULT from google.api_core.retry import Retry - from google.cloud import container_v1, exceptions from google.cloud.container_v1.gapic.enums import Operation from google.cloud.container_v1.types import Cluster diff --git a/airflow/gcp/hooks/mlengine.py b/airflow/gcp/hooks/mlengine.py index c44eb6d341795f..61111823bbf3dc 100644 --- a/airflow/gcp/hooks/mlengine.py +++ b/airflow/gcp/hooks/mlengine.py @@ -21,10 +21,10 @@ import random import time -from typing import Dict, Callable, List, Optional +from typing import Callable, Dict, List, Optional -from googleapiclient.errors import HttpError from googleapiclient.discovery import build +from googleapiclient.errors import HttpError from airflow.gcp.hooks.base import GoogleCloudBaseHook from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/gcp/hooks/natural_language.py b/airflow/gcp/hooks/natural_language.py index 12349a4de510ae..a1a445bee9de7c 100644 --- a/airflow/gcp/hooks/natural_language.py +++ b/airflow/gcp/hooks/natural_language.py @@ -19,21 +19,14 @@ """ This module contains a Google Cloud Natural Language Hook. """ -from typing import Sequence, Tuple, Union, Dict, Optional +from typing import Dict, Optional, Sequence, Tuple, Union from google.api_core.retry import Retry -from google.cloud.language_v1 import LanguageServiceClient - +from google.cloud.language_v1 import LanguageServiceClient, enums from google.cloud.language_v1.types import ( - Document, - AnalyzeEntitiesResponse, - ClassifyTextResponse, - AnalyzeSyntaxResponse, - AnnotateTextResponse, - AnalyzeSentimentResponse, - AnalyzeEntitySentimentResponse + AnalyzeEntitiesResponse, AnalyzeEntitySentimentResponse, AnalyzeSentimentResponse, AnalyzeSyntaxResponse, + AnnotateTextResponse, ClassifyTextResponse, Document, ) -from google.cloud.language_v1 import enums from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/pubsub.py b/airflow/gcp/hooks/pubsub.py index d8f612665b64e1..f5c5f62aedd219 100644 --- a/airflow/gcp/hooks/pubsub.py +++ b/airflow/gcp/hooks/pubsub.py @@ -21,19 +21,19 @@ """ import warnings from base64 import b64decode -from typing import List, Dict, Optional, Sequence, Tuple, Union +from typing import Dict, List, Optional, Sequence, Tuple, Union from uuid import uuid4 from cached_property import cached_property -from google.api_core.retry import Retry from google.api_core.exceptions import AlreadyExists, GoogleAPICallError +from google.api_core.retry import Retry from google.cloud.exceptions import NotFound from google.cloud.pubsub_v1 import PublisherClient, SubscriberClient -from google.cloud.pubsub_v1.types import Duration, PushConfig, MessageStoragePolicy +from google.cloud.pubsub_v1.types import Duration, MessageStoragePolicy, PushConfig from googleapiclient.errors import HttpError -from airflow.version import version from airflow.gcp.hooks.base import GoogleCloudBaseHook +from airflow.version import version class PubSubException(Exception): diff --git a/airflow/gcp/hooks/spanner.py b/airflow/gcp/hooks/spanner.py index 5580e9250fcec1..3472d55e70bc9b 100644 --- a/airflow/gcp/hooks/spanner.py +++ b/airflow/gcp/hooks/spanner.py @@ -21,9 +21,9 @@ """ from typing import Callable, List, Optional -from google.api_core.exceptions import GoogleAPICallError, AlreadyExists -from google.cloud.spanner_v1.database import Database +from google.api_core.exceptions import AlreadyExists, GoogleAPICallError from google.cloud.spanner_v1.client import Client +from google.cloud.spanner_v1.database import Database from google.cloud.spanner_v1.instance import Instance from google.cloud.spanner_v1.transaction import Transaction from google.longrunning.operations_grpc_pb2 import Operation # noqa: F401 diff --git a/airflow/gcp/hooks/speech_to_text.py b/airflow/gcp/hooks/speech_to_text.py index 6b1ed5982180c0..2cf157f136d212 100644 --- a/airflow/gcp/hooks/speech_to_text.py +++ b/airflow/gcp/hooks/speech_to_text.py @@ -19,11 +19,11 @@ """ This module contains a Google Cloud Speech Hook. """ -from typing import Union, Dict, Optional +from typing import Dict, Optional, Union from google.api_core.retry import Retry from google.cloud.speech_v1 import SpeechClient -from google.cloud.speech_v1.types import RecognitionConfig, RecognitionAudio +from google.cloud.speech_v1.types import RecognitionAudio, RecognitionConfig from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/text_to_speech.py b/airflow/gcp/hooks/text_to_speech.py index 16cb8816525cdf..e079fe4d03f986 100644 --- a/airflow/gcp/hooks/text_to_speech.py +++ b/airflow/gcp/hooks/text_to_speech.py @@ -19,13 +19,14 @@ """ This module contains a Google Cloud Text to Speech Hook. """ -from typing import Union, Dict, Optional +from typing import Dict, Optional, Union from google.api_core.retry import Retry from google.cloud.texttospeech_v1 import TextToSpeechClient from google.cloud.texttospeech_v1.types import ( - AudioConfig, SynthesisInput, VoiceSelectionParams, SynthesizeSpeechResponse + AudioConfig, SynthesisInput, SynthesizeSpeechResponse, VoiceSelectionParams, ) + from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/translate.py b/airflow/gcp/hooks/translate.py index 4d1e7f8673add5..655e57415f0096 100644 --- a/airflow/gcp/hooks/translate.py +++ b/airflow/gcp/hooks/translate.py @@ -19,9 +19,10 @@ """ This module contains a Google Cloud Translate Hook. """ -from typing import Union, List, Dict, Optional +from typing import Dict, List, Optional, Union from google.cloud.translate_v2 import Client + from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/video_intelligence.py b/airflow/gcp/hooks/video_intelligence.py index 0afa32595b9c8c..613f588dbcb914 100644 --- a/airflow/gcp/hooks/video_intelligence.py +++ b/airflow/gcp/hooks/video_intelligence.py @@ -19,12 +19,12 @@ """ This module contains a Google Cloud Video Intelligence Hook. """ -from typing import Sequence, Tuple, Union, Dict, List, Optional +from typing import Dict, List, Optional, Sequence, Tuple, Union +from google.api_core.operation import Operation from google.api_core.retry import Retry from google.cloud.videointelligence_v1 import VideoIntelligenceServiceClient from google.cloud.videointelligence_v1.types import VideoContext -from google.api_core.operation import Operation from airflow.gcp.hooks.base import GoogleCloudBaseHook diff --git a/airflow/gcp/hooks/vision.py b/airflow/gcp/hooks/vision.py index 77f2f5a0a611b0..e405ae9be4fdfb 100644 --- a/airflow/gcp/hooks/vision.py +++ b/airflow/gcp/hooks/vision.py @@ -21,18 +21,13 @@ """ from copy import deepcopy -from typing import Callable, Dict, Sequence, Tuple, Union, Any, Optional, List +from typing import Any, Callable, Dict, List, Optional, Sequence, Tuple, Union from cached_property import cached_property from google.api_core.retry import Retry -from google.cloud.vision_v1 import ProductSearchClient, ImageAnnotatorClient +from google.cloud.vision_v1 import ImageAnnotatorClient, ProductSearchClient from google.cloud.vision_v1.types import ( - AnnotateImageRequest, - FieldMask, - Image, - Product, - ProductSet, - ReferenceImage, + AnnotateImageRequest, FieldMask, Image, Product, ProductSet, ReferenceImage, ) from google.protobuf.json_format import MessageToDict diff --git a/airflow/gcp/operators/automl.py b/airflow/gcp/operators/automl.py index 9eb5a42b9caa2d..272e3336886690 100644 --- a/airflow/gcp/operators/automl.py +++ b/airflow/gcp/operators/automl.py @@ -22,14 +22,14 @@ This module contains Google AutoML operators. """ import ast -from typing import Sequence, Tuple, Union, List, Dict, Optional +from typing import Dict, List, Optional, Sequence, Tuple, Union from google.api_core.retry import Retry from google.protobuf.json_format import MessageToDict +from airflow.gcp.hooks.automl import CloudAutoMLHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.gcp.hooks.automl import CloudAutoMLHook MetaData = Sequence[Tuple[str, str]] diff --git a/airflow/gcp/operators/bigquery.py b/airflow/gcp/operators/bigquery.py index 67b2dfeb4983f0..b9a46f22f9b5d6 100644 --- a/airflow/gcp/operators/bigquery.py +++ b/airflow/gcp/operators/bigquery.py @@ -23,17 +23,15 @@ import json import warnings -from typing import Iterable, List, Optional, Union, Dict, Any, SupportsAbs +from typing import Any, Dict, Iterable, List, Optional, SupportsAbs, Union from airflow.exceptions import AirflowException +from airflow.gcp.hooks.bigquery import BigQueryHook +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook, _parse_gcs_url from airflow.models.baseoperator import BaseOperator, BaseOperatorLink from airflow.models.taskinstance import TaskInstance +from airflow.operators.check_operator import CheckOperator, IntervalCheckOperator, ValueCheckOperator from airflow.utils.decorators import apply_defaults -from airflow.operators.check_operator import \ - CheckOperator, ValueCheckOperator, IntervalCheckOperator -from airflow.gcp.hooks.bigquery import BigQueryHook -from airflow.gcp.hooks.gcs import GoogleCloudStorageHook, _parse_gcs_url - BIGQUERY_JOB_DETAILS_LINK_FMT = 'https://console.cloud.google.com/bigquery?j={job_id}' diff --git a/airflow/gcp/operators/bigquery_dts.py b/airflow/gcp/operators/bigquery_dts.py index 6a92dc7d03b524..12caccc32c621f 100644 --- a/airflow/gcp/operators/bigquery_dts.py +++ b/airflow/gcp/operators/bigquery_dts.py @@ -19,14 +19,12 @@ """ This module contains Google BigQuery Data Transfer Service operators. """ -from typing import Sequence, Tuple, Optional -from google.protobuf.json_format import MessageToDict +from typing import Optional, Sequence, Tuple + from google.api_core.retry import Retry +from google.protobuf.json_format import MessageToDict -from airflow.gcp.hooks.bigquery_dts import ( - BiqQueryDataTransferServiceHook, - get_object_id, -) +from airflow.gcp.hooks.bigquery_dts import BiqQueryDataTransferServiceHook, get_object_id from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/bigtable.py b/airflow/gcp/operators/bigtable.py index 46ef211e1a84f3..bdeb19830292ee 100644 --- a/airflow/gcp/operators/bigtable.py +++ b/airflow/gcp/operators/bigtable.py @@ -20,14 +20,14 @@ This module contains Google Cloud Bigtable operators. """ from enum import IntEnum -from typing import Iterable, List, Optional, Dict +from typing import Dict, Iterable, List, Optional import google.api_core.exceptions from google.cloud.bigtable.column_family import GarbageCollectionRule from airflow import AirflowException -from airflow.models import BaseOperator from airflow.gcp.hooks.bigtable import BigtableHook +from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/cloud_build.py b/airflow/gcp/operators/cloud_build.py index b63d3e55382060..e1a9d0256c925e 100644 --- a/airflow/gcp/operators/cloud_build.py +++ b/airflow/gcp/operators/cloud_build.py @@ -17,10 +17,10 @@ # specific language governing permissions and limitations # under the License. """Operators that integrat with Google Cloud Build service.""" -from copy import deepcopy import re -from typing import Dict, Iterable, Any, Optional -from urllib.parse import urlparse, unquote +from copy import deepcopy +from typing import Any, Dict, Iterable, Optional +from urllib.parse import unquote, urlparse from airflow import AirflowException from airflow.gcp.hooks.cloud_build import CloudBuildHook diff --git a/airflow/gcp/operators/cloud_memorystore.py b/airflow/gcp/operators/cloud_memorystore.py index ed174dc268f941..da28028bc46d70 100644 --- a/airflow/gcp/operators/cloud_memorystore.py +++ b/airflow/gcp/operators/cloud_memorystore.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. """Operators for Google Cloud Memorystore service""" -from typing import Dict, Sequence, Tuple, Union, Optional +from typing import Dict, Optional, Sequence, Tuple, Union from google.api_core.retry import Retry from google.cloud.redis_v1.gapic.enums import FailoverInstanceRequest diff --git a/airflow/gcp/operators/cloud_sql.py b/airflow/gcp/operators/cloud_sql.py index cae0deef5050eb..fe1725df1d0eb0 100644 --- a/airflow/gcp/operators/cloud_sql.py +++ b/airflow/gcp/operators/cloud_sql.py @@ -19,16 +19,16 @@ """ This module contains Google Cloud SQL operators. """ -from typing import Union, List, Optional, Iterable, Dict +from typing import Dict, Iterable, List, Optional, Union from googleapiclient.errors import HttpError from airflow import AirflowException -from airflow.gcp.hooks.cloud_sql import CloudSqlHook, CloudSqlDatabaseHook +from airflow.gcp.hooks.cloud_sql import CloudSqlDatabaseHook, CloudSqlHook from airflow.gcp.utils.field_validator import GcpBodyFieldValidator +from airflow.hooks.base_hook import BaseHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.hooks.base_hook import BaseHook SETTINGS = 'settings' SETTINGS_VERSION = 'settingsVersion' diff --git a/airflow/gcp/operators/cloud_storage_transfer_service.py b/airflow/gcp/operators/cloud_storage_transfer_service.py index 67268821e862a4..f65783368ba5fa 100644 --- a/airflow/gcp/operators/cloud_storage_transfer_service.py +++ b/airflow/gcp/operators/cloud_storage_transfer_service.py @@ -23,36 +23,14 @@ from copy import deepcopy from datetime import date, time -from typing import Optional, Dict +from typing import Dict, Optional from airflow import AirflowException from airflow.gcp.hooks.cloud_storage_transfer_service import ( - GCPTransferServiceHook, - GcpTransferJobsStatus, - TRANSFER_OPTIONS, - OBJECT_CONDITIONS, - PROJECT_ID, - BUCKET_NAME, - GCS_DATA_SINK, - STATUS, - DESCRIPTION, - GCS_DATA_SOURCE, - HTTP_DATA_SOURCE, - SECONDS, - MINUTES, - HOURS, - YEAR, - MONTH, - DAY, - START_TIME_OF_DAY, - SCHEDULE_END_DATE, - SCHEDULE_START_DATE, - SCHEDULE, - SECRET_ACCESS_KEY, - ACCESS_KEY_ID, - AWS_ACCESS_KEY, - AWS_S3_DATA_SOURCE, - TRANSFER_SPEC, + ACCESS_KEY_ID, AWS_ACCESS_KEY, AWS_S3_DATA_SOURCE, BUCKET_NAME, DAY, DESCRIPTION, GCS_DATA_SINK, + GCS_DATA_SOURCE, HOURS, HTTP_DATA_SOURCE, MINUTES, MONTH, OBJECT_CONDITIONS, PROJECT_ID, SCHEDULE, + SCHEDULE_END_DATE, SCHEDULE_START_DATE, SECONDS, SECRET_ACCESS_KEY, START_TIME_OF_DAY, STATUS, + TRANSFER_OPTIONS, TRANSFER_SPEC, YEAR, GcpTransferJobsStatus, GCPTransferServiceHook, ) from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/compute.py b/airflow/gcp/operators/compute.py index f0c35f2d544115..2356ae9cedb9ad 100644 --- a/airflow/gcp/operators/compute.py +++ b/airflow/gcp/operators/compute.py @@ -21,10 +21,10 @@ """ from copy import deepcopy -from typing import Dict, Optional, List, Any -from json_merge_patch import merge +from typing import Any, Dict, List, Optional from googleapiclient.errors import HttpError +from json_merge_patch import merge from airflow import AirflowException from airflow.gcp.hooks.compute import GceHook diff --git a/airflow/gcp/operators/dataflow.py b/airflow/gcp/operators/dataflow.py index 5f3d75f1a1eae3..534ea503e12461 100644 --- a/airflow/gcp/operators/dataflow.py +++ b/airflow/gcp/operators/dataflow.py @@ -20,19 +20,19 @@ This module contains Google Dataflow operators. """ +import copy import os import re -import uuid -import copy import tempfile +import uuid from enum import Enum from typing import List, Optional -from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.gcp.hooks.dataflow import DataFlowHook +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.models import BaseOperator -from airflow.version import version from airflow.utils.decorators import apply_defaults +from airflow.version import version class CheckJobRunning(Enum): diff --git a/airflow/gcp/operators/dataproc.py b/airflow/gcp/operators/dataproc.py index 1154ec7d991ca8..91b84c95d2b677 100644 --- a/airflow/gcp/operators/dataproc.py +++ b/airflow/gcp/operators/dataproc.py @@ -28,15 +28,15 @@ import time import uuid from datetime import datetime, timedelta -from typing import List, Dict, Set, Optional +from typing import Dict, List, Optional, Set +from airflow.exceptions import AirflowException from airflow.gcp.hooks.dataproc import DataProcHook from airflow.gcp.hooks.gcs import GoogleCloudStorageHook -from airflow.exceptions import AirflowException from airflow.models import BaseOperator +from airflow.utils import timezone from airflow.utils.decorators import apply_defaults from airflow.version import version -from airflow.utils import timezone class DataprocOperationBaseOperator(BaseOperator): diff --git a/airflow/gcp/operators/datastore.py b/airflow/gcp/operators/datastore.py index 568a7da6561694..313e0b087e4f39 100644 --- a/airflow/gcp/operators/datastore.py +++ b/airflow/gcp/operators/datastore.py @@ -22,9 +22,9 @@ """ from typing import Optional +from airflow.exceptions import AirflowException from airflow.gcp.hooks.datastore import DatastoreHook from airflow.gcp.hooks.gcs import GoogleCloudStorageHook -from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/functions.py b/airflow/gcp/operators/functions.py index 3cd62d88d95af3..798e05abada00f 100644 --- a/airflow/gcp/operators/functions.py +++ b/airflow/gcp/operators/functions.py @@ -21,17 +21,16 @@ """ import re -from typing import Optional, List, Dict, Any +from typing import Any, Dict, List, Optional from googleapiclient.errors import HttpError from airflow import AirflowException -from airflow.gcp.utils.field_validator import GcpBodyFieldValidator, \ - GcpFieldValidationException -from airflow.version import version -from airflow.models import BaseOperator from airflow.gcp.hooks.functions import GcfHook +from airflow.gcp.utils.field_validator import GcpBodyFieldValidator, GcpFieldValidationException +from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults +from airflow.version import version def _validate_available_memory_in_mb(value): diff --git a/airflow/gcp/operators/gcs.py b/airflow/gcp/operators/gcs.py index 7dabba1c59c261..efbe264dfe6284 100644 --- a/airflow/gcp/operators/gcs.py +++ b/airflow/gcp/operators/gcs.py @@ -19,18 +19,17 @@ """ This module contains a Google Cloud Storage Bucket operator. """ +import subprocess import sys import warnings -import subprocess -from typing import Dict, Optional, Iterable, Union, List from tempfile import NamedTemporaryFile +from typing import Dict, Iterable, List, Optional, Union - +from airflow import AirflowException from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults from airflow.models.xcom import MAX_XCOM_SIZE -from airflow import AirflowException +from airflow.utils.decorators import apply_defaults class GoogleCloudStorageCreateBucketOperator(BaseOperator): diff --git a/airflow/gcp/operators/kubernetes_engine.py b/airflow/gcp/operators/kubernetes_engine.py index 7a7a8a56d8568d..752a33f01b0df9 100644 --- a/airflow/gcp/operators/kubernetes_engine.py +++ b/airflow/gcp/operators/kubernetes_engine.py @@ -24,14 +24,14 @@ import os import subprocess import tempfile -from typing import Union, Dict, Optional +from typing import Dict, Optional, Union from google.cloud.container_v1.types import Cluster from airflow import AirflowException from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook -from airflow.gcp.hooks.kubernetes_engine import GKEClusterHook from airflow.contrib.operators.kubernetes_pod_operator import KubernetesPodOperator +from airflow.gcp.hooks.kubernetes_engine import GKEClusterHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/mlengine.py b/airflow/gcp/operators/mlengine.py index 5b1ed88516b453..d3070512d8b247 100644 --- a/airflow/gcp/operators/mlengine.py +++ b/airflow/gcp/operators/mlengine.py @@ -21,8 +21,8 @@ import re from typing import List, Optional -from airflow.gcp.hooks.mlengine import MLEngineHook from airflow.exceptions import AirflowException +from airflow.gcp.hooks.mlengine import MLEngineHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/gcp/operators/natural_language.py b/airflow/gcp/operators/natural_language.py index f5f19a870eba61..bf11504e4366cb 100644 --- a/airflow/gcp/operators/natural_language.py +++ b/airflow/gcp/operators/natural_language.py @@ -19,12 +19,12 @@ """ This module contains Google Cloud Language operators. """ -from typing import Union, Tuple, Sequence, Optional +from typing import Optional, Sequence, Tuple, Union -from google.protobuf.json_format import MessageToDict -from google.cloud.language_v1.types import Document -from google.cloud.language_v1 import enums from google.api_core.retry import Retry +from google.cloud.language_v1 import enums +from google.cloud.language_v1.types import Document +from google.protobuf.json_format import MessageToDict from airflow.gcp.hooks.natural_language import CloudNaturalLanguageHook from airflow.models import BaseOperator diff --git a/airflow/gcp/operators/pubsub.py b/airflow/gcp/operators/pubsub.py index 67c9ee72f1d384..dd24055da3ddd9 100644 --- a/airflow/gcp/operators/pubsub.py +++ b/airflow/gcp/operators/pubsub.py @@ -20,10 +20,10 @@ This module contains Google PubSub operators. """ import warnings -from typing import List, Optional, Sequence, Tuple, Dict, Union +from typing import Dict, List, Optional, Sequence, Tuple, Union from google.api_core.retry import Retry -from google.cloud.pubsub_v1.types import Duration, PushConfig, MessageStoragePolicy +from google.cloud.pubsub_v1.types import Duration, MessageStoragePolicy, PushConfig from airflow.gcp.hooks.pubsub import PubSubHook from airflow.models import BaseOperator diff --git a/airflow/gcp/operators/speech_to_text.py b/airflow/gcp/operators/speech_to_text.py index c3dffda4779535..7cb8e8f6d6fd21 100644 --- a/airflow/gcp/operators/speech_to_text.py +++ b/airflow/gcp/operators/speech_to_text.py @@ -25,7 +25,7 @@ from google.cloud.speech_v1.types import RecognitionConfig from airflow import AirflowException -from airflow.gcp.hooks.speech_to_text import RecognitionAudio, GCPSpeechToTextHook +from airflow.gcp.hooks.speech_to_text import GCPSpeechToTextHook, RecognitionAudio from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/tasks.py b/airflow/gcp/operators/tasks.py index 9509b7c60f41fa..2f1e9ce697aca2 100644 --- a/airflow/gcp/operators/tasks.py +++ b/airflow/gcp/operators/tasks.py @@ -22,11 +22,11 @@ which allow you to perform basic operations using Cloud Tasks queues/tasks. """ -from typing import Tuple, Sequence, Union, Dict, Optional +from typing import Dict, Optional, Sequence, Tuple, Union from google.api_core.retry import Retry -from google.cloud.tasks_v2.types import Queue, FieldMask, Task from google.cloud.tasks_v2 import enums +from google.cloud.tasks_v2.types import FieldMask, Queue, Task from airflow.gcp.hooks.tasks import CloudTasksHook from airflow.models import BaseOperator diff --git a/airflow/gcp/operators/text_to_speech.py b/airflow/gcp/operators/text_to_speech.py index 669a8bec3bf362..0a8b93021774f6 100644 --- a/airflow/gcp/operators/text_to_speech.py +++ b/airflow/gcp/operators/text_to_speech.py @@ -21,14 +21,14 @@ """ from tempfile import NamedTemporaryFile -from typing import Dict, Union, Optional +from typing import Dict, Optional, Union from google.api_core.retry import Retry -from google.cloud.texttospeech_v1.types import SynthesisInput, VoiceSelectionParams, AudioConfig +from google.cloud.texttospeech_v1.types import AudioConfig, SynthesisInput, VoiceSelectionParams from airflow import AirflowException -from airflow.gcp.hooks.text_to_speech import GCPTextToSpeechHook from airflow.gcp.hooks.gcs import GoogleCloudStorageHook +from airflow.gcp.hooks.text_to_speech import GCPTextToSpeechHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/operators/translate_speech.py b/airflow/gcp/operators/translate_speech.py index c837b00008e24d..d2227f46d499a2 100644 --- a/airflow/gcp/operators/translate_speech.py +++ b/airflow/gcp/operators/translate_speech.py @@ -21,8 +21,8 @@ """ from typing import Optional -from google.protobuf.json_format import MessageToDict from google.cloud.speech_v1.types import RecognitionAudio, RecognitionConfig +from google.protobuf.json_format import MessageToDict from airflow import AirflowException from airflow.gcp.hooks.speech_to_text import GCPSpeechToTextHook diff --git a/airflow/gcp/operators/video_intelligence.py b/airflow/gcp/operators/video_intelligence.py index 9523d860e836f8..96827330661a9b 100644 --- a/airflow/gcp/operators/video_intelligence.py +++ b/airflow/gcp/operators/video_intelligence.py @@ -19,12 +19,12 @@ """ This module contains Google Cloud Vision operators. """ -from typing import Dict, Union, Optional +from typing import Dict, Optional, Union from google.api_core.retry import Retry -from google.protobuf.json_format import MessageToDict from google.cloud.videointelligence_v1 import enums from google.cloud.videointelligence_v1.types import VideoContext +from google.protobuf.json_format import MessageToDict from airflow.gcp.hooks.video_intelligence import CloudVideoIntelligenceHook from airflow.models import BaseOperator diff --git a/airflow/gcp/operators/vision.py b/airflow/gcp/operators/vision.py index 71b0be3d06ec9a..d7d6901b8af166 100644 --- a/airflow/gcp/operators/vision.py +++ b/airflow/gcp/operators/vision.py @@ -21,17 +21,12 @@ """ from copy import deepcopy -from typing import Union, List, Dict, Any, Sequence, Tuple, Optional +from typing import Any, Dict, List, Optional, Sequence, Tuple, Union from google.api_core.exceptions import AlreadyExists from google.api_core.retry import Retry from google.cloud.vision_v1.types import ( - ProductSet, - FieldMask, - Image, - Product, - AnnotateImageRequest, - ReferenceImage + AnnotateImageRequest, FieldMask, Image, Product, ProductSet, ReferenceImage, ) from airflow.gcp.hooks.vision import CloudVisionHook diff --git a/airflow/gcp/sensors/bigquery.py b/airflow/gcp/sensors/bigquery.py index 72acd3fe0869c4..d91679627c67a7 100644 --- a/airflow/gcp/sensors/bigquery.py +++ b/airflow/gcp/sensors/bigquery.py @@ -21,8 +21,8 @@ """ from typing import Optional -from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.gcp.hooks.bigquery import BigQueryHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/sensors/bigquery_dts.py b/airflow/gcp/sensors/bigquery_dts.py index 2894560f899482..066bb0147fe8fd 100644 --- a/airflow/gcp/sensors/bigquery_dts.py +++ b/airflow/gcp/sensors/bigquery_dts.py @@ -19,7 +19,7 @@ """ This module contains a Google BigQuery Data Transfer Service sensor. """ -from typing import Sequence, Tuple, Union, Set, Optional +from typing import Optional, Sequence, Set, Tuple, Union from google.api_core.retry import Retry from google.protobuf.json_format import MessageToDict diff --git a/airflow/gcp/sensors/bigtable.py b/airflow/gcp/sensors/bigtable.py index 50e7e622509ee7..84c21818b79acf 100644 --- a/airflow/gcp/sensors/bigtable.py +++ b/airflow/gcp/sensors/bigtable.py @@ -21,12 +21,12 @@ """ import google.api_core.exceptions -from google.cloud.bigtable_admin_v2 import enums from google.cloud.bigtable.table import ClusterState +from google.cloud.bigtable_admin_v2 import enums -from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.gcp.hooks.bigtable import BigtableHook from airflow.gcp.operators.bigtable import BigtableValidationMixin +from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/gcp/sensors/cloud_storage_transfer_service.py b/airflow/gcp/sensors/cloud_storage_transfer_service.py index 628a141820cb4f..ae97dc8548dcf0 100644 --- a/airflow/gcp/sensors/cloud_storage_transfer_service.py +++ b/airflow/gcp/sensors/cloud_storage_transfer_service.py @@ -19,7 +19,7 @@ """ This module contains a Google Cloud Transfer sensor. """ -from typing import Set, Union, Optional +from typing import Optional, Set, Union from airflow.gcp.hooks.cloud_storage_transfer_service import GCPTransferServiceHook from airflow.sensors.base_sensor_operator import BaseSensorOperator diff --git a/airflow/gcp/sensors/gcs.py b/airflow/gcp/sensors/gcs.py index a7abbc7a142f98..3d16ee7dec3229 100644 --- a/airflow/gcp/sensors/gcs.py +++ b/airflow/gcp/sensors/gcs.py @@ -24,10 +24,10 @@ from datetime import datetime from typing import Callable, List, Optional +from airflow import AirflowException from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults -from airflow import AirflowException class GoogleCloudStorageObjectSensor(BaseSensorOperator): diff --git a/airflow/gcp/utils/field_sanitizer.py b/airflow/gcp/utils/field_sanitizer.py index d7b155a0cb47ea..b31f49ba738f42 100644 --- a/airflow/gcp/utils/field_sanitizer.py +++ b/airflow/gcp/utils/field_sanitizer.py @@ -100,7 +100,7 @@ from typing import List -from airflow import LoggingMixin, AirflowException +from airflow import AirflowException, LoggingMixin class GcpFieldSanitizerException(AirflowException): diff --git a/airflow/gcp/utils/field_validator.py b/airflow/gcp/utils/field_validator.py index c633ac492a9d31..6bc5fc27b47632 100644 --- a/airflow/gcp/utils/field_validator.py +++ b/airflow/gcp/utils/field_validator.py @@ -132,9 +132,9 @@ """ import re -from typing import Sequence, Dict, Callable +from typing import Callable, Dict, Sequence -from airflow import LoggingMixin, AirflowException +from airflow import AirflowException, LoggingMixin COMPOSITE_FIELD_TYPES = ['union', 'dict', 'list'] diff --git a/airflow/gcp/utils/mlengine_operator_utils.py b/airflow/gcp/utils/mlengine_operator_utils.py index bc68e8deb2f671..6ee9ce206e9213 100644 --- a/airflow/gcp/utils/mlengine_operator_utils.py +++ b/airflow/gcp/utils/mlengine_operator_utils.py @@ -28,10 +28,10 @@ import dill +from airflow.exceptions import AirflowException from airflow.gcp.hooks.gcs import GoogleCloudStorageHook -from airflow.gcp.operators.mlengine import MLEngineBatchPredictionOperator from airflow.gcp.operators.dataflow import DataFlowPythonOperator -from airflow.exceptions import AirflowException +from airflow.gcp.operators.mlengine import MLEngineBatchPredictionOperator from airflow.operators.python_operator import PythonOperator diff --git a/airflow/gcp/utils/mlengine_prediction_summary.py b/airflow/gcp/utils/mlengine_prediction_summary.py index 1a0853160133cd..8a56460710eada 100644 --- a/airflow/gcp/utils/mlengine_prediction_summary.py +++ b/airflow/gcp/utils/mlengine_prediction_summary.py @@ -86,9 +86,9 @@ def metric_fn(inst): import base64 import json import os -import dill import apache_beam as beam +import dill # pylint: disable=wrong-import-order class JsonCoder: diff --git a/airflow/hooks/__init__.py b/airflow/hooks/__init__.py index 44a434a07eef95..bfc33d2b02af37 100644 --- a/airflow/hooks/__init__.py +++ b/airflow/hooks/__init__.py @@ -20,10 +20,10 @@ import sys - # Imports the hooks dynamically while keeping the package API clean, # abstracting the underlying modules + def _integrate_plugins(): """Integrate plugins to the context""" from airflow.plugins_manager import hooks_modules diff --git a/airflow/hooks/base_hook.py b/airflow/hooks/base_hook.py index 433a7aec9398ff..9a7cc3e6716b38 100644 --- a/airflow/hooks/base_hook.py +++ b/airflow/hooks/base_hook.py @@ -21,8 +21,8 @@ import random from typing import Iterable -from airflow.models import Connection from airflow.exceptions import AirflowException +from airflow.models import Connection from airflow.utils.db import provide_session from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/hooks/dbapi_hook.py b/airflow/hooks/dbapi_hook.py index 93835fdf541ac7..a853ed0d6761fe 100644 --- a/airflow/hooks/dbapi_hook.py +++ b/airflow/hooks/dbapi_hook.py @@ -17,15 +17,15 @@ # specific language governing permissions and limitations # under the License. -from typing import Optional -from airflow.typing import Protocol -from datetime import datetime from contextlib import closing +from datetime import datetime +from typing import Optional from sqlalchemy import create_engine -from airflow.hooks.base_hook import BaseHook from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook +from airflow.typing import Protocol class ConnectorProtocol(Protocol): diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py index 6397c9fa9f4c9a..76396646646579 100644 --- a/airflow/hooks/druid_hook.py +++ b/airflow/hooks/druid_hook.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. -import requests import time +import requests from pydruid.db import connect from airflow.exceptions import AirflowException diff --git a/airflow/hooks/hive_hooks.py b/airflow/hooks/hive_hooks.py index 8308d140824c0f..f06f50618d1d46 100644 --- a/airflow/hooks/hive_hooks.py +++ b/airflow/hooks/hive_hooks.py @@ -20,9 +20,9 @@ import contextlib import os import re +import socket import subprocess import time -import socket from collections import OrderedDict from tempfile import NamedTemporaryFile diff --git a/airflow/hooks/http_hook.py b/airflow/hooks/http_hook.py index 7ce7898895ac93..316fe3fa4048f0 100644 --- a/airflow/hooks/http_hook.py +++ b/airflow/hooks/http_hook.py @@ -20,8 +20,8 @@ import requests import tenacity -from airflow.hooks.base_hook import BaseHook from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook class HttpHook(BaseHook): diff --git a/airflow/hooks/mysql_hook.py b/airflow/hooks/mysql_hook.py index 2c8f73bc78859f..6b32e2a75f457a 100644 --- a/airflow/hooks/mysql_hook.py +++ b/airflow/hooks/mysql_hook.py @@ -17,9 +17,10 @@ # specific language governing permissions and limitations # under the License. +import json + import MySQLdb import MySQLdb.cursors -import json from airflow.hooks.dbapi_hook import DbApiHook diff --git a/airflow/hooks/oracle_hook.py b/airflow/hooks/oracle_hook.py index b889958f1fb480..c3241258c6dad3 100644 --- a/airflow/hooks/oracle_hook.py +++ b/airflow/hooks/oracle_hook.py @@ -17,11 +17,12 @@ # specific language governing permissions and limitations # under the License. +from datetime import datetime + import cx_Oracle +import numpy from airflow.hooks.dbapi_hook import DbApiHook -from datetime import datetime -import numpy class OracleHook(DbApiHook): diff --git a/airflow/hooks/postgres_hook.py b/airflow/hooks/postgres_hook.py index 6a318326fb0b76..8a4db571375ae2 100644 --- a/airflow/hooks/postgres_hook.py +++ b/airflow/hooks/postgres_hook.py @@ -18,9 +18,10 @@ # under the License. import os +from contextlib import closing + import psycopg2 import psycopg2.extensions -from contextlib import closing from airflow.hooks.dbapi_hook import DbApiHook diff --git a/airflow/hooks/samba_hook.py b/airflow/hooks/samba_hook.py index 0820500bbcedaf..03659c61afa1cf 100644 --- a/airflow/hooks/samba_hook.py +++ b/airflow/hooks/samba_hook.py @@ -17,9 +17,10 @@ # specific language governing permissions and limitations # under the License. -from smbclient import SambaClient import os +from smbclient import SambaClient + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/hooks/slack_hook.py b/airflow/hooks/slack_hook.py index 0b5157e0a853a7..400a2cc8625254 100644 --- a/airflow/hooks/slack_hook.py +++ b/airflow/hooks/slack_hook.py @@ -20,8 +20,9 @@ from typing import Optional from slackclient import SlackClient -from airflow.hooks.base_hook import BaseHook + from airflow.exceptions import AirflowException +from airflow.hooks.base_hook import BaseHook # noinspection PyAbstractClass diff --git a/airflow/hooks/webhdfs_hook.py b/airflow/hooks/webhdfs_hook.py index 1998f56ed67bed..ec19132ed8b53e 100644 --- a/airflow/hooks/webhdfs_hook.py +++ b/airflow/hooks/webhdfs_hook.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. """Hook for Web HDFS""" -from hdfs import InsecureClient, HdfsError +from hdfs import HdfsError, InsecureClient from airflow.configuration import conf from airflow.exceptions import AirflowException diff --git a/airflow/hooks/zendesk_hook.py b/airflow/hooks/zendesk_hook.py index a6f758345018b2..e8df2ef9bf42d7 100644 --- a/airflow/hooks/zendesk_hook.py +++ b/airflow/hooks/zendesk_hook.py @@ -18,7 +18,9 @@ # under the License. import time -from zdesk import Zendesk, RateLimitError, ZendeskError + +from zdesk import RateLimitError, Zendesk, ZendeskError + from airflow.hooks.base_hook import BaseHook diff --git a/airflow/jobs/__init__.py b/airflow/jobs/__init__.py index 00a149e650dc62..0006226233bdfd 100644 --- a/airflow/jobs/__init__.py +++ b/airflow/jobs/__init__.py @@ -18,7 +18,7 @@ # under the License. # -from airflow.jobs.base_job import BaseJob # noqa: F401 from airflow.jobs.backfill_job import BackfillJob # noqa: F401 -from airflow.jobs.scheduler_job import DagFileProcessor, SchedulerJob # noqa: F401 +from airflow.jobs.base_job import BaseJob # noqa: F401 from airflow.jobs.local_task_job import LocalTaskJob # noqa: F401 +from airflow.jobs.scheduler_job import DagFileProcessor, SchedulerJob # noqa: F401 diff --git a/airflow/jobs/backfill_job.py b/airflow/jobs/backfill_job.py index 45d052bf34a013..f70d65da594047 100644 --- a/airflow/jobs/backfill_job.py +++ b/airflow/jobs/backfill_job.py @@ -18,26 +18,23 @@ # under the License. # -from datetime import datetime import time from collections import OrderedDict +from datetime import datetime -from sqlalchemy.orm.session import make_transient, Session +from sqlalchemy.orm.session import Session, make_transient from airflow import executors, models from airflow.exceptions import ( - AirflowException, - DagConcurrencyLimitReached, - NoAvailablePoolSlot, - PoolNotFound, + AirflowException, DagConcurrencyLimitReached, NoAvailablePoolSlot, PoolNotFound, TaskConcurrencyLimitReached, ) +from airflow.jobs.base_job import BaseJob from airflow.models import DAG, DagPickle, DagRun -from airflow.ti_deps.dep_context import DepContext, BACKFILL_QUEUED_DEPS +from airflow.ti_deps.dep_context import BACKFILL_QUEUED_DEPS, DepContext from airflow.utils import timezone from airflow.utils.configuration import tmp_configuration_copy from airflow.utils.db import provide_session -from airflow.jobs.base_job import BaseJob from airflow.utils.state import State diff --git a/airflow/jobs/base_job.py b/airflow/jobs/base_job.py index b9ef7da46d7ad8..2cb450c573b602 100644 --- a/airflow/jobs/base_job.py +++ b/airflow/jobs/base_job.py @@ -20,18 +20,16 @@ import getpass from time import sleep +from typing import Optional from sqlalchemy import Column, Index, Integer, String, and_, or_ from sqlalchemy.exc import OperationalError from sqlalchemy.orm.session import make_transient -from typing import Optional -from airflow.configuration import conf from airflow import executors, models -from airflow.exceptions import ( - AirflowException, -) -from airflow.models.base import Base, ID_LEN +from airflow.configuration import conf +from airflow.exceptions import AirflowException +from airflow.models.base import ID_LEN, Base from airflow.stats import Stats from airflow.utils import helpers, timezone from airflow.utils.db import create_session, provide_session diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index c34bc58d7f5730..f7c08695d24e98 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -24,11 +24,11 @@ from airflow.configuration import conf from airflow.exceptions import AirflowException +from airflow.jobs.base_job import BaseJob from airflow.stats import Stats from airflow.task.task_runner import get_task_runner from airflow.utils.db import provide_session from airflow.utils.net import get_hostname -from airflow.jobs.base_job import BaseJob from airflow.utils.state import State diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py index b2e912f372ae22..94065bf91e199c 100644 --- a/airflow/jobs/scheduler_job.py +++ b/airflow/jobs/scheduler_job.py @@ -34,21 +34,19 @@ from sqlalchemy import and_, func, not_, or_ from sqlalchemy.orm.session import make_transient -from airflow.configuration import conf from airflow import executors, models, settings +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.jobs.base_job import BaseJob from airflow.models import DAG, DagRun, SlaMiss, errors from airflow.stats import Stats -from airflow.ti_deps.dep_context import DepContext, SCHEDULEABLE_STATES, SCHEDULED_DEPS +from airflow.ti_deps.dep_context import SCHEDULEABLE_STATES, SCHEDULED_DEPS, DepContext from airflow.ti_deps.deps.pool_slots_available_dep import STATES_TO_COUNT_AS_RUNNING from airflow.utils import asciiart, helpers, timezone -from airflow.utils.dag_processing import (AbstractDagFileProcessor, - DagFileProcessorAgent, - SimpleDag, - SimpleDagBag, - SimpleTaskInstance, - list_py_file_paths) +from airflow.utils.dag_processing import ( + AbstractDagFileProcessor, DagFileProcessorAgent, SimpleDag, SimpleDagBag, SimpleTaskInstance, + list_py_file_paths, +) from airflow.utils.db import provide_session from airflow.utils.email import get_email_address_list, send_email from airflow.utils.log.logging_mixin import LoggingMixin, StreamLogWriter, set_context diff --git a/airflow/kubernetes/k8s_model.py b/airflow/kubernetes/k8s_model.py index bca1cc0b9e1250..3ba1410154e91c 100644 --- a/airflow/kubernetes/k8s_model.py +++ b/airflow/kubernetes/k8s_model.py @@ -19,8 +19,9 @@ """ from abc import ABC, abstractmethod -from typing import List, Optional from functools import reduce +from typing import List, Optional + import kubernetes.client.models as k8s diff --git a/airflow/kubernetes/pod.py b/airflow/kubernetes/pod.py index bdce1ce50a042e..2568e635218a92 100644 --- a/airflow/kubernetes/pod.py +++ b/airflow/kubernetes/pod.py @@ -19,7 +19,9 @@ """ import copy + import kubernetes.client.models as k8s + from airflow.kubernetes.k8s_model import K8SModel diff --git a/airflow/kubernetes/pod_generator.py b/airflow/kubernetes/pod_generator.py index d12b7525785df0..429e4351326055 100644 --- a/airflow/kubernetes/pod_generator.py +++ b/airflow/kubernetes/pod_generator.py @@ -25,6 +25,7 @@ import uuid import kubernetes.client.models as k8s + from airflow.executors import Executors diff --git a/airflow/kubernetes/pod_launcher.py b/airflow/kubernetes/pod_launcher.py index fda9290ae44870..2d0ea7d4930e5b 100644 --- a/airflow/kubernetes/pod_launcher.py +++ b/airflow/kubernetes/pod_launcher.py @@ -18,22 +18,20 @@ import json import time from datetime import datetime as dt -from typing import Tuple, Optional - -from requests.exceptions import BaseHTTPError +from typing import Optional, Tuple import tenacity - -from kubernetes import watch, client +from kubernetes import client, watch +from kubernetes.client.models.v1_pod import V1Pod from kubernetes.client.rest import ApiException from kubernetes.stream import stream as kubernetes_stream -from kubernetes.client.models.v1_pod import V1Pod +from requests.exceptions import BaseHTTPError +from airflow import AirflowException +from airflow.kubernetes.pod_generator import PodDefaults from airflow.settings import pod_mutation_hook from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State -from airflow import AirflowException -from airflow.kubernetes.pod_generator import PodDefaults from .kube_client import get_kube_client diff --git a/airflow/kubernetes/pod_runtime_info_env.py b/airflow/kubernetes/pod_runtime_info_env.py index bf1320dc431716..09e651f2bb68de 100644 --- a/airflow/kubernetes/pod_runtime_info_env.py +++ b/airflow/kubernetes/pod_runtime_info_env.py @@ -19,7 +19,9 @@ """ import copy + import kubernetes.client.models as k8s + from airflow.kubernetes.k8s_model import K8SModel diff --git a/airflow/kubernetes/secret.py b/airflow/kubernetes/secret.py index 3d33739cbd6729..1d160dd8ec8506 100644 --- a/airflow/kubernetes/secret.py +++ b/airflow/kubernetes/secret.py @@ -18,10 +18,12 @@ Classes for interacting with Kubernetes API """ -import uuid import copy +import uuid from typing import Tuple + import kubernetes.client.models as k8s + from airflow.exceptions import AirflowConfigException from airflow.kubernetes.k8s_model import K8SModel diff --git a/airflow/kubernetes/volume.py b/airflow/kubernetes/volume.py index 679671cc87b335..066d8db24922a0 100644 --- a/airflow/kubernetes/volume.py +++ b/airflow/kubernetes/volume.py @@ -20,7 +20,9 @@ import copy from typing import Dict + import kubernetes.client.models as k8s + from airflow.kubernetes.k8s_model import K8SModel diff --git a/airflow/kubernetes/volume_mount.py b/airflow/kubernetes/volume_mount.py index 3f7b2b5bef5ec9..8b6d7dd131a8ee 100644 --- a/airflow/kubernetes/volume_mount.py +++ b/airflow/kubernetes/volume_mount.py @@ -19,7 +19,9 @@ """ import copy + import kubernetes.client.models as k8s + from airflow.kubernetes.k8s_model import K8SModel diff --git a/airflow/kubernetes/worker_configuration.py b/airflow/kubernetes/worker_configuration.py index 90e36f206ac983..12c2dfaf1643de 100644 --- a/airflow/kubernetes/worker_configuration.py +++ b/airflow/kubernetes/worker_configuration.py @@ -16,14 +16,15 @@ # under the License. """Configuration of the worker""" import os -from typing import List, Dict +from typing import Dict, List + import kubernetes.client.models as k8s from airflow.configuration import conf +from airflow.kubernetes.k8s_model import append_to_pod from airflow.kubernetes.pod_generator import PodGenerator -from airflow.utils.log.logging_mixin import LoggingMixin from airflow.kubernetes.secret import Secret -from airflow.kubernetes.k8s_model import append_to_pod +from airflow.utils.log.logging_mixin import LoggingMixin class WorkerConfiguration(LoggingMixin): diff --git a/airflow/lineage/__init__.py b/airflow/lineage/__init__.py index f444139f009794..bb5024e4151df4 100644 --- a/airflow/lineage/__init__.py +++ b/airflow/lineage/__init__.py @@ -17,14 +17,13 @@ # specific language governing permissions and limitations # under the License. from functools import wraps +from itertools import chain from airflow.configuration import conf from airflow.lineage.datasets import DataSet from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string -from itertools import chain - PIPELINE_OUTLETS = "pipeline_outlets" PIPELINE_INLETS = "pipeline_inlets" diff --git a/airflow/lineage/backend/atlas/__init__.py b/airflow/lineage/backend/atlas/__init__.py index 8e0dfc5dc3dfc1..26b91e0b0e5e7e 100644 --- a/airflow/lineage/backend/atlas/__init__.py +++ b/airflow/lineage/backend/atlas/__init__.py @@ -17,15 +17,15 @@ # specific language governing permissions and limitations # under the License. # +from atlasclient.client import Atlas +from atlasclient.exceptions import HttpError + from airflow.configuration import conf from airflow.lineage import datasets from airflow.lineage.backend import LineageBackend from airflow.lineage.backend.atlas.typedefs import operator_typedef from airflow.utils.timezone import convert_to_utc -from atlasclient.client import Atlas -from atlasclient.exceptions import HttpError - SERIALIZED_DATE_FORMAT_STR = "%Y-%m-%dT%H:%M:%S.%fZ" _username = conf.get("atlas", "username") diff --git a/airflow/lineage/datasets.py b/airflow/lineage/datasets.py index 5778086ee7bc1d..e0b7842261de19 100644 --- a/airflow/lineage/datasets.py +++ b/airflow/lineage/datasets.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. import json - from typing import List + from jinja2 import Environment diff --git a/airflow/macros/__init__.py b/airflow/macros/__init__.py index 27205a22c8d2d2..25edc71bcf2de9 100644 --- a/airflow/macros/__init__.py +++ b/airflow/macros/__init__.py @@ -17,12 +17,14 @@ # specific language governing permissions and limitations # under the License. +import time # noqa +import uuid # noqa from datetime import datetime, timedelta -import dateutil # noqa -from random import random # noqa -import time # noqa -from . import hive # noqa -import uuid # noqa +from random import random # noqa + +import dateutil # noqa + +from . import hive # noqa def ds_add(ds, days): diff --git a/airflow/migrations/env.py b/airflow/migrations/env.py index 95ac21ffdb7b49..e4c307be2c93a7 100644 --- a/airflow/migrations/env.py +++ b/airflow/migrations/env.py @@ -21,8 +21,7 @@ from alembic import context -from airflow import settings -from airflow import models +from airflow import models, settings def include_object(_, name, type_, *args): diff --git a/airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py b/airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py index 4182a2d4aa969d..76245097b4946a 100644 --- a/airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py +++ b/airflow/migrations/versions/004c1210f153_increase_queue_name_size_limit.py @@ -24,8 +24,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '004c1210f153' diff --git a/airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py b/airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py index e7767cc8dc3efe..9e95568a132756 100644 --- a/airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py +++ b/airflow/migrations/versions/0a2a5b66e19d_add_task_reschedule_table.py @@ -22,8 +22,8 @@ Create Date: 2018-06-17 22:50:00.053620 """ -from alembic import op import sqlalchemy as sa +from alembic import op from sqlalchemy.dialects import mysql # revision identifiers, used by Alembic. diff --git a/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py b/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py index b52f3459453fab..16fabccb45d638 100644 --- a/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py +++ b/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py @@ -23,9 +23,9 @@ """ +import sqlalchemy as sa from alembic import op from sqlalchemy.dialects import mysql -import sqlalchemy as sa # revision identifiers, used by Alembic. revision = "0e2a74e0fc9f" diff --git a/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py b/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py index fe84254c38caa3..95f49a7d382f88 100644 --- a/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py +++ b/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py @@ -24,8 +24,8 @@ Create Date: 2015-08-18 18:57:51.927315 """ -from alembic import op import sqlalchemy as sa +from alembic import op from sqlalchemy.engine.reflection import Inspector # revision identifiers, used by Alembic. diff --git a/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py b/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py index 16ab3495634281..cd9f72ffd4e746 100644 --- a/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py +++ b/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py @@ -24,8 +24,8 @@ Create Date: 2016-02-02 17:20:55.692295 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '1968acfc09e3' diff --git a/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py b/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py index 50d53652c4734d..d5d3e0eac45f6b 100644 --- a/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py +++ b/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py @@ -25,8 +25,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '1b38cef5b76e' diff --git a/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py b/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py index 7d6dcb9e1cc24d..c02a1f5242eac0 100644 --- a/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py +++ b/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py @@ -24,9 +24,9 @@ """ -from alembic import op -import sqlalchemy as sa import dill +import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '27c6a30d7c24' diff --git a/airflow/migrations/versions/2e541a1dcfed_task_duration.py b/airflow/migrations/versions/2e541a1dcfed_task_duration.py index 595a5774a6b27e..6fc39f63188548 100644 --- a/airflow/migrations/versions/2e541a1dcfed_task_duration.py +++ b/airflow/migrations/versions/2e541a1dcfed_task_duration.py @@ -25,8 +25,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op from sqlalchemy.dialects import mysql # revision identifiers, used by Alembic. diff --git a/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py b/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py index 9fe515064c35b7..da59ea025006eb 100644 --- a/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py +++ b/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py @@ -24,8 +24,8 @@ Create Date: 2015-08-25 06:09:20.460147 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '338e90f54d61' diff --git a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py index c1e62a2cc2a5f8..dfaa4c51a73a7b 100644 --- a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py +++ b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py @@ -23,8 +23,8 @@ Create Date: 2017-09-11 15:26:47.598494 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '33ae817a1ff4' diff --git a/airflow/migrations/versions/40e67319e3a9_dagrun_config.py b/airflow/migrations/versions/40e67319e3a9_dagrun_config.py index 3da4d5f543038d..703a0f0416cbf2 100644 --- a/airflow/migrations/versions/40e67319e3a9_dagrun_config.py +++ b/airflow/migrations/versions/40e67319e3a9_dagrun_config.py @@ -24,8 +24,8 @@ Create Date: 2015-10-29 08:36:31.726728 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '40e67319e3a9' diff --git a/airflow/migrations/versions/41f5f12752f8_add_superuser_field.py b/airflow/migrations/versions/41f5f12752f8_add_superuser_field.py index 6e02582b7e8406..8b3e9b4a41de56 100644 --- a/airflow/migrations/versions/41f5f12752f8_add_superuser_field.py +++ b/airflow/migrations/versions/41f5f12752f8_add_superuser_field.py @@ -23,9 +23,8 @@ """ -from alembic import op import sqlalchemy as sa - +from alembic import op # revision identifiers, used by Alembic. revision = '41f5f12752f8' diff --git a/airflow/migrations/versions/4446e08588_dagrun_start_end.py b/airflow/migrations/versions/4446e08588_dagrun_start_end.py index 29932c92060e9b..ef7578e6cdca4f 100644 --- a/airflow/migrations/versions/4446e08588_dagrun_start_end.py +++ b/airflow/migrations/versions/4446e08588_dagrun_start_end.py @@ -25,8 +25,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '4446e08588' diff --git a/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py index e7e832da51637c..551c0beb40b6da 100644 --- a/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py +++ b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py @@ -24,9 +24,8 @@ """ -from alembic import op +from alembic import context, op from sqlalchemy.dialects import mysql -from alembic import context # revision identifiers, used by Alembic. revision = '4addfa1236f1' diff --git a/airflow/migrations/versions/502898887f84_adding_extra_to_log.py b/airflow/migrations/versions/502898887f84_adding_extra_to_log.py index 632720a4e27b0c..51f79f6dba5f4d 100644 --- a/airflow/migrations/versions/502898887f84_adding_extra_to_log.py +++ b/airflow/migrations/versions/502898887f84_adding_extra_to_log.py @@ -24,8 +24,8 @@ Create Date: 2015-11-03 22:50:49.794097 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '502898887f84' diff --git a/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py b/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py index a26a105ac82991..58d93df4a27890 100644 --- a/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py +++ b/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py @@ -24,8 +24,8 @@ Create Date: 2015-11-30 06:51:25.872557 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '561833c1c74b' diff --git a/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py b/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py index 77a35db48fad51..062ec93aa6f71b 100644 --- a/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py +++ b/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py @@ -24,8 +24,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '5e7d17757c7a' diff --git a/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py b/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py index 2def57e9048704..e23ff78856dd0a 100644 --- a/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py +++ b/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py @@ -23,8 +23,8 @@ Create Date: 2016-08-03 14:02:59.203021 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '64de9cddf6c9' diff --git a/airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py b/airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py index 0f6763ef4989a0..d4ad10b68ecb35 100644 --- a/airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py +++ b/airflow/migrations/versions/6e96a59344a4_make_taskinstance_pool_not_nullable.py @@ -24,9 +24,9 @@ """ -from alembic import op import dill import sqlalchemy as sa +from alembic import op from sqlalchemy import Column, Float, Integer, PickleType, String from sqlalchemy.ext.declarative import declarative_base diff --git a/airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py b/airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py index 30a4365e32262c..774e21ca53c00b 100644 --- a/airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py +++ b/airflow/migrations/versions/74effc47d867_change_datetime_to_datetime2_6_on_mssql_.py @@ -25,6 +25,7 @@ """ from collections import defaultdict + from alembic import op from sqlalchemy.dialects import mssql diff --git a/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py b/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py index 2abccdad5d5658..1cb84f7480678c 100644 --- a/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py +++ b/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py @@ -23,8 +23,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '856955da8476' diff --git a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py index 86ad47131e4a1f..520c505f55b2b2 100644 --- a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py +++ b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py @@ -23,8 +23,8 @@ Create Date: 2018-04-03 15:31:20.814328 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = '86770d1215c0' diff --git a/airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py b/airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py index 89dba33c3e3947..a8c95d7cf7dbaa 100644 --- a/airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py +++ b/airflow/migrations/versions/a56c9515abdc_remove_dag_stat_table.py @@ -24,8 +24,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'a56c9515abdc' diff --git a/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py b/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py index 503cd0b6f04068..e2e1a6298433d8 100644 --- a/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py +++ b/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py @@ -25,8 +25,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'bba5a7cfc896' diff --git a/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py b/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py index 9855a6d4daf3c4..6c720344380522 100644 --- a/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py +++ b/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py @@ -24,8 +24,8 @@ Create Date: 2016-01-14 18:05:54.871682 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'bbc73705a13e' diff --git a/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py b/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py index a1a5270c7bff40..d69d24aed6e916 100644 --- a/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py +++ b/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py @@ -23,9 +23,9 @@ Create Date: 2017-08-14 16:06:31.568971 """ -from alembic import op import dill import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'bdaa763e6c56' diff --git a/airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py b/airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py index 70282715edfa6f..ff02efbf9f1a01 100644 --- a/airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py +++ b/airflow/migrations/versions/c8ffec048a3b_add_fields_to_dag.py @@ -24,8 +24,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'c8ffec048a3b' diff --git a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py index 9967586fccba37..ae4192ce0a5d07 100644 --- a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py +++ b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py @@ -23,15 +23,15 @@ """ -from alembic import op import sqlalchemy as sa -from airflow import settings -from airflow.models import DagBag - +from alembic import op from sqlalchemy import Column, Integer, String from sqlalchemy.engine.reflection import Inspector from sqlalchemy.ext.declarative import declarative_base +from airflow import settings +from airflow.models import DagBag + # revision identifiers, used by Alembic. revision = 'cc1e65623dc7' down_revision = '127d2bf2dfa7' diff --git a/airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py b/airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py index 243431402618f4..3c40f77ea6eb90 100644 --- a/airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py +++ b/airflow/migrations/versions/cf5dc11e79ad_drop_user_and_chart.py @@ -22,11 +22,10 @@ Create Date: 2019-01-24 15:30:35.834740 """ +import sqlalchemy as sa from alembic import op from sqlalchemy.dialects import mysql from sqlalchemy.engine.reflection import Inspector -import sqlalchemy as sa - # revision identifiers, used by Alembic. revision = 'cf5dc11e79ad' diff --git a/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py b/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py index db5afaf023e546..fb198d0557faf7 100644 --- a/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py +++ b/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py @@ -23,9 +23,8 @@ Create Date: 2017-08-18 17:07:16.686130 """ -from alembic import op +from alembic import context, op from sqlalchemy.dialects import mysql -from alembic import context # revision identifiers, used by Alembic. revision = 'd2ae31099d61' diff --git a/airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py b/airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py index 7ae562756b63a5..776527e16633b1 100644 --- a/airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py +++ b/airflow/migrations/versions/dd4ecb8fbee3_add_schedule_interval_to_dag.py @@ -24,8 +24,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'dd4ecb8fbee3' diff --git a/airflow/migrations/versions/e3a246e0dc1_current_schema.py b/airflow/migrations/versions/e3a246e0dc1_current_schema.py index c915ed6f763681..747234911b3696 100644 --- a/airflow/migrations/versions/e3a246e0dc1_current_schema.py +++ b/airflow/migrations/versions/e3a246e0dc1_current_schema.py @@ -25,8 +25,8 @@ """ -from alembic import op import sqlalchemy as sa +from alembic import op from sqlalchemy import func from sqlalchemy.engine.reflection import Inspector diff --git a/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py b/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py index e14b4b80251091..e3679830bc8ad3 100644 --- a/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py +++ b/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py @@ -23,8 +23,8 @@ Create Date: 2016-07-20 15:08:28.247537 """ -from alembic import op import sqlalchemy as sa +from alembic import op # revision identifiers, used by Alembic. revision = 'f2ca10b85618' diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index 5ec40be69da09e..fc2a6eb8f67322 100644 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. """Airflow models""" -from airflow.models.base import Base, ID_LEN # noqa: F401 +from airflow.models.base import ID_LEN, Base # noqa: F401 from airflow.models.baseoperator import BaseOperator # noqa: F401 from airflow.models.connection import Connection # noqa: F401 from airflow.models.dag import DAG, DagModel # noqa: F401 @@ -25,13 +25,13 @@ from airflow.models.dagpickle import DagPickle # noqa: F401 from airflow.models.dagrun import DagRun # noqa: F401 from airflow.models.errors import ImportError # noqa: F401, pylint:disable=redefined-builtin -from airflow.models.kubernetes import KubeWorkerIdentifier, KubeResourceVersion # noqa: F401 +from airflow.models.kubernetes import KubeResourceVersion, KubeWorkerIdentifier # noqa: F401 from airflow.models.log import Log # noqa: F401 from airflow.models.pool import Pool # noqa: F401 -from airflow.models.taskfail import TaskFail # noqa: F401 from airflow.models.skipmixin import SkipMixin # noqa: F401 from airflow.models.slamiss import SlaMiss # noqa: F401 -from airflow.models.taskinstance import clear_task_instances, TaskInstance # noqa: F401 +from airflow.models.taskfail import TaskFail # noqa: F401 +from airflow.models.taskinstance import TaskInstance, clear_task_instances # noqa: F401 from airflow.models.taskreschedule import TaskReschedule # noqa: F401 from airflow.models.variable import Variable # noqa: F401 -from airflow.models.xcom import XCom, XCOM_RETURN_KEY # noqa: F401 +from airflow.models.xcom import XCOM_RETURN_KEY, XCom # noqa: F401 diff --git a/airflow/models/base.py b/airflow/models/base.py index 3dacb8adcedb1e..29fcc5cb76d9fd 100644 --- a/airflow/models/base.py +++ b/airflow/models/base.py @@ -18,6 +18,7 @@ # under the License. from typing import Any + from sqlalchemy import MetaData from sqlalchemy.ext.declarative import declarative_base diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 20303fac4d6757..ac094849b97198 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -19,25 +19,23 @@ """ Base operator for all operators. """ -from abc import ABCMeta, abstractmethod import copy import functools import logging import sys import warnings -from datetime import timedelta, datetime -from typing import Callable, Dict, Iterable, List, Optional, Set, Any, Union - -from dateutil.relativedelta import relativedelta - -from cached_property import cached_property +from abc import ABCMeta, abstractmethod +from datetime import datetime, timedelta +from typing import Any, Callable, Dict, Iterable, List, Optional, Set, Union import jinja2 +from cached_property import cached_property +from dateutil.relativedelta import relativedelta from airflow import settings from airflow.configuration import conf from airflow.exceptions import AirflowException -from airflow.lineage import prepare_lineage, apply_lineage, DataSet +from airflow.lineage import DataSet, apply_lineage, prepare_lineage from airflow.models.dag import DAG from airflow.models.pool import Pool from airflow.models.taskinstance import TaskInstance, clear_task_instances diff --git a/airflow/models/connection.py b/airflow/models/connection.py index 00f14875a8ed59..b334a0b91708a3 100644 --- a/airflow/models/connection.py +++ b/airflow/models/connection.py @@ -18,15 +18,15 @@ # under the License. import json -from urllib.parse import urlparse, unquote, parse_qsl +from urllib.parse import parse_qsl, unquote, urlparse -from sqlalchemy import Column, Integer, String, Boolean +from sqlalchemy import Boolean, Column, Integer, String from sqlalchemy.ext.declarative import declared_attr from sqlalchemy.orm import synonym from airflow import LoggingMixin from airflow.exceptions import AirflowException -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.models.crypto import get_fernet diff --git a/airflow/models/crypto.py b/airflow/models/crypto.py index 8ba656e52f68eb..a9e5b6b86e510f 100644 --- a/airflow/models/crypto.py +++ b/airflow/models/crypto.py @@ -17,10 +17,11 @@ # specific language governing permissions and limitations # under the License. -from airflow.typing import Protocol from typing import Optional + from airflow.configuration import conf from airflow.exceptions import AirflowException +from airflow.typing import Protocol from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/models/dag.py b/airflow/models/dag.py index bd47429bb6d3ad..a6939bfee402b9 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -26,21 +26,21 @@ import traceback import warnings from collections import OrderedDict, defaultdict -from datetime import timedelta, datetime -from typing import Union, Optional, Iterable, Dict, Type, Callable, List, TYPE_CHECKING +from datetime import datetime, timedelta +from typing import TYPE_CHECKING, Callable, Dict, Iterable, List, Optional, Type, Union import jinja2 import pendulum from croniter import croniter from dateutil.relativedelta import relativedelta -from sqlalchemy import Column, String, Boolean, Integer, Text, func, or_ +from sqlalchemy import Boolean, Column, Integer, String, Text, func, or_ from airflow import settings, utils from airflow.configuration import conf from airflow.dag.base_dag import BaseDag -from airflow.exceptions import AirflowException, AirflowDagCycleException +from airflow.exceptions import AirflowDagCycleException, AirflowException from airflow.executors import LocalExecutor, get_default_executor -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.models.dagbag import DagBag from airflow.models.dagpickle import DagPickle from airflow.models.dagrun import DagRun @@ -50,7 +50,7 @@ from airflow.utils.db import provide_session from airflow.utils.helpers import validate_key from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.utils.sqlalchemy import UtcDateTime, Interval +from airflow.utils.sqlalchemy import Interval, UtcDateTime from airflow.utils.state import State if TYPE_CHECKING: diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index b8328566b81221..b09f47f095ec28 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -27,7 +27,7 @@ from collections import namedtuple from datetime import datetime, timedelta -from croniter import croniter, CroniterBadCronError, CroniterBadDateError, CroniterNotAlphaError +from croniter import CroniterBadCronError, CroniterBadDateError, CroniterNotAlphaError, croniter from sqlalchemy import or_ from airflow import settings @@ -37,7 +37,7 @@ from airflow.executors import get_default_executor from airflow.stats import Stats from airflow.utils import timezone -from airflow.utils.dag_processing import list_py_file_paths, correct_maybe_zipped +from airflow.utils.dag_processing import correct_maybe_zipped, list_py_file_paths from airflow.utils.db import provide_session from airflow.utils.helpers import pprinttable from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index e92d1b92504bc5..3d3536cd18b2ba 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -19,14 +19,14 @@ from typing import Optional, cast from sqlalchemy import ( - Column, Integer, String, Boolean, PickleType, Index, UniqueConstraint, func, DateTime, or_, - and_ + Boolean, Column, DateTime, Index, Integer, PickleType, String, UniqueConstraint, and_, func, or_, ) from sqlalchemy.ext.declarative import declared_attr from sqlalchemy.orm import synonym from sqlalchemy.orm.session import Session + from airflow.exceptions import AirflowException -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext from airflow.utils import timezone diff --git a/airflow/models/errors.py b/airflow/models/errors.py index 6a3797ca3df9b4..f25a788c46ba75 100644 --- a/airflow/models/errors.py +++ b/airflow/models/errors.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. -from sqlalchemy import Integer, Column, String, Text +from sqlalchemy import Column, Integer, String, Text from airflow.models.base import Base from airflow.utils.sqlalchemy import UtcDateTime diff --git a/airflow/models/kubernetes.py b/airflow/models/kubernetes.py index a18689eefd3167..50c205f0e8358f 100644 --- a/airflow/models/kubernetes.py +++ b/airflow/models/kubernetes.py @@ -19,7 +19,7 @@ import uuid -from sqlalchemy import Column, Boolean, true as sqltrue, String +from sqlalchemy import Boolean, Column, String, true as sqltrue from airflow.models.base import Base from airflow.utils.db import provide_session diff --git a/airflow/models/log.py b/airflow/models/log.py index fa3fed60ee2401..f58eb5227dc023 100644 --- a/airflow/models/log.py +++ b/airflow/models/log.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. -from sqlalchemy import Column, Integer, String, Text, Index +from sqlalchemy import Column, Index, Integer, String, Text -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.utils import timezone from airflow.utils.sqlalchemy import UtcDateTime diff --git a/airflow/models/pool.py b/airflow/models/pool.py index 4e99b02bd38dff..1cb31182383156 100644 --- a/airflow/models/pool.py +++ b/airflow/models/pool.py @@ -20,9 +20,9 @@ from sqlalchemy import Column, Integer, String, Text from airflow.models.base import Base -from airflow.utils.state import State from airflow.ti_deps.deps.pool_slots_available_dep import STATES_TO_COUNT_AS_RUNNING from airflow.utils.db import provide_session +from airflow.utils.state import State class Pool(Base): diff --git a/airflow/models/skipmixin.py b/airflow/models/skipmixin.py index c92d8131e83513..ce90e170c83963 100644 --- a/airflow/models/skipmixin.py +++ b/airflow/models/skipmixin.py @@ -17,14 +17,14 @@ # specific language governing permissions and limitations # under the License. +from typing import Iterable, Set, Union + from airflow.models.taskinstance import TaskInstance from airflow.utils import timezone from airflow.utils.db import provide_session from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State -from typing import Union, Iterable, Set - class SkipMixin(LoggingMixin): @provide_session diff --git a/airflow/models/slamiss.py b/airflow/models/slamiss.py index 0981be886ea015..44cb0b6c6b725d 100644 --- a/airflow/models/slamiss.py +++ b/airflow/models/slamiss.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. -from sqlalchemy import Boolean, Column, String, Index, Text +from sqlalchemy import Boolean, Column, Index, String, Text -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.utils.sqlalchemy import UtcDateTime diff --git a/airflow/models/taskfail.py b/airflow/models/taskfail.py index 3fb08b3cee0009..36a979d823fed2 100644 --- a/airflow/models/taskfail.py +++ b/airflow/models/taskfail.py @@ -19,7 +19,7 @@ """Taskfail tracks the failed run durations of each task instance""" from sqlalchemy import Column, Index, Integer, String -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.utils.sqlalchemy import UtcDateTime diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 10a4dfa147c3d7..2717da276e45be 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -29,19 +29,20 @@ from datetime import timedelta from typing import Optional from urllib.parse import quote -import lazy_object_proxy -import pendulum import dill +import lazy_object_proxy +import pendulum from sqlalchemy import Column, Float, Index, Integer, PickleType, String, func from sqlalchemy.orm import reconstructor from sqlalchemy.orm.session import Session -from airflow.exceptions import (AirflowException, AirflowRescheduleException, - AirflowSkipException, AirflowTaskTimeout) from airflow import settings from airflow.configuration import conf -from airflow.models.base import Base, ID_LEN +from airflow.exceptions import ( + AirflowException, AirflowRescheduleException, AirflowSkipException, AirflowTaskTimeout, +) +from airflow.models.base import ID_LEN, Base from airflow.models.log import Log from airflow.models.pool import Pool from airflow.models.taskfail import TaskFail @@ -50,7 +51,7 @@ from airflow.models.xcom import XCOM_RETURN_KEY, XCom from airflow.sentry import Sentry from airflow.stats import Stats -from airflow.ti_deps.dep_context import DepContext, REQUEUEABLE_DEPS, RUNNING_DEPS +from airflow.ti_deps.dep_context import REQUEUEABLE_DEPS, RUNNING_DEPS, DepContext from airflow.utils import timezone from airflow.utils.db import provide_session from airflow.utils.email import send_email diff --git a/airflow/models/taskreschedule.py b/airflow/models/taskreschedule.py index ebca6ee4e4ee6f..b79294c6f95c35 100644 --- a/airflow/models/taskreschedule.py +++ b/airflow/models/taskreschedule.py @@ -19,7 +19,7 @@ """TaskReschedule tracks rescheduled task instances.""" from sqlalchemy import Column, ForeignKeyConstraint, Index, Integer, String, asc -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.utils.db import provide_session from airflow.utils.sqlalchemy import UtcDateTime diff --git a/airflow/models/variable.py b/airflow/models/variable.py index aa459d7fd9b615..3c8142ba34a0d6 100644 --- a/airflow/models/variable.py +++ b/airflow/models/variable.py @@ -20,12 +20,12 @@ import json from typing import Any -from sqlalchemy import Column, Integer, String, Text, Boolean +from sqlalchemy import Boolean, Column, Integer, String, Text from sqlalchemy.ext.declarative import declared_attr from sqlalchemy.orm import synonym -from airflow.models.base import Base, ID_LEN -from airflow.models.crypto import get_fernet, InvalidFernetToken +from airflow.models.base import ID_LEN, Base +from airflow.models.crypto import InvalidFernetToken, get_fernet from airflow.utils.db import provide_session from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/models/xcom.py b/airflow/models/xcom.py index eb7ced61376fb5..7a078e652dfbb1 100644 --- a/airflow/models/xcom.py +++ b/airflow/models/xcom.py @@ -20,18 +20,17 @@ import json import pickle -from sqlalchemy import Column, Integer, String, Index, LargeBinary, and_ +from sqlalchemy import Column, Index, Integer, LargeBinary, String, and_ from sqlalchemy.orm import reconstructor from airflow.configuration import conf -from airflow.models.base import Base, ID_LEN +from airflow.models.base import ID_LEN, Base from airflow.utils import timezone from airflow.utils.db import provide_session from airflow.utils.helpers import as_tuple from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.sqlalchemy import UtcDateTime - # MAX XCOM Size is 48KB # https://github.com/apache/airflow/pull/1618#discussion_r68249677 MAX_XCOM_SIZE = 49344 diff --git a/airflow/operators/bash_operator.py b/airflow/operators/bash_operator.py index f7b5cfa4aa18c8..ea609e6ac243be 100644 --- a/airflow/operators/bash_operator.py +++ b/airflow/operators/bash_operator.py @@ -20,8 +20,8 @@ import os import signal -from subprocess import Popen, STDOUT, PIPE -from tempfile import gettempdir, NamedTemporaryFile +from subprocess import PIPE, STDOUT, Popen +from tempfile import NamedTemporaryFile, gettempdir from typing import Dict, Optional from airflow.exceptions import AirflowException diff --git a/airflow/operators/bigquery_to_bigquery.py b/airflow/operators/bigquery_to_bigquery.py index 4927abf196e48f..3595926ada90d2 100644 --- a/airflow/operators/bigquery_to_bigquery.py +++ b/airflow/operators/bigquery_to_bigquery.py @@ -20,11 +20,11 @@ This module contains Google BigQuery to BigQuery operator. """ import warnings -from typing import List, Optional, Union, Dict +from typing import Dict, List, Optional, Union +from airflow.gcp.hooks.bigquery import BigQueryHook from airflow.models.baseoperator import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.gcp.hooks.bigquery import BigQueryHook class BigQueryToBigQueryOperator(BaseOperator): diff --git a/airflow/operators/bigquery_to_gcs.py b/airflow/operators/bigquery_to_gcs.py index 4764d4a1b44cd7..e4293d7f2625da 100644 --- a/airflow/operators/bigquery_to_gcs.py +++ b/airflow/operators/bigquery_to_gcs.py @@ -20,11 +20,11 @@ This module contains Google BigQuery to Google CLoud Storage operator. """ import warnings -from typing import List, Optional, Dict +from typing import Dict, List, Optional +from airflow.gcp.hooks.bigquery import BigQueryHook from airflow.models.baseoperator import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.gcp.hooks.bigquery import BigQueryHook class BigQueryToCloudStorageOperator(BaseOperator): diff --git a/airflow/operators/bigquery_to_mysql.py b/airflow/operators/bigquery_to_mysql.py index 0bc26ca705db25..967830b2a03311 100644 --- a/airflow/operators/bigquery_to_mysql.py +++ b/airflow/operators/bigquery_to_mysql.py @@ -21,10 +21,10 @@ """ from typing import Optional -from airflow.models.baseoperator import BaseOperator -from airflow.utils.decorators import apply_defaults from airflow.gcp.hooks.bigquery import BigQueryHook from airflow.hooks.mysql_hook import MySqlHook +from airflow.models.baseoperator import BaseOperator +from airflow.utils.decorators import apply_defaults class BigQueryToMySqlOperator(BaseOperator): diff --git a/airflow/operators/branch_operator.py b/airflow/operators/branch_operator.py index a1e2c72faee3ba..77c47ff475361c 100644 --- a/airflow/operators/branch_operator.py +++ b/airflow/operators/branch_operator.py @@ -18,7 +18,7 @@ # under the License. """Branching operators""" -from typing import Union, Iterable, Dict +from typing import Dict, Iterable, Union from airflow.models import BaseOperator, SkipMixin diff --git a/airflow/operators/cassandra_to_gcs.py b/airflow/operators/cassandra_to_gcs.py index e11daa7ebead26..2b0c3943652f00 100644 --- a/airflow/operators/cassandra_to_gcs.py +++ b/airflow/operators/cassandra_to_gcs.py @@ -29,11 +29,11 @@ from typing import Optional from uuid import UUID -from cassandra.util import Date, Time, SortedSet, OrderedMapSerializedKey +from cassandra.util import Date, OrderedMapSerializedKey, SortedSet, Time -from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.contrib.hooks.cassandra_hook import CassandraHook from airflow.exceptions import AirflowException +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/operators/check_operator.py b/airflow/operators/check_operator.py index 5def5d55db07e5..0da9480c07d33b 100644 --- a/airflow/operators/check_operator.py +++ b/airflow/operators/check_operator.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. -from typing import Optional, Any, Iterable, Dict, SupportsAbs +from typing import Any, Dict, Iterable, Optional, SupportsAbs from airflow.exceptions import AirflowException from airflow.hooks.base_hook import BaseHook diff --git a/airflow/operators/dagrun_operator.py b/airflow/operators/dagrun_operator.py index 4919aecae5cc24..7d082fab85f3df 100644 --- a/airflow/operators/dagrun_operator.py +++ b/airflow/operators/dagrun_operator.py @@ -18,14 +18,13 @@ # under the License. import datetime -from typing import Callable, Union, Optional, Dict +import json +from typing import Callable, Dict, Optional, Union +from airflow.api.common.experimental.trigger_dag import trigger_dag from airflow.models import BaseOperator from airflow.utils import timezone from airflow.utils.decorators import apply_defaults -from airflow.api.common.experimental.trigger_dag import trigger_dag - -import json class DagRunOrder: diff --git a/airflow/operators/docker_operator.py b/airflow/operators/docker_operator.py index 377dd48567bca2..1dcd8f79a208bd 100644 --- a/airflow/operators/docker_operator.py +++ b/airflow/operators/docker_operator.py @@ -19,14 +19,14 @@ """ Implements Docker operator """ +import ast import json -from typing import Union, List, Dict, Iterable, Optional +from typing import Dict, Iterable, List, Optional, Union -import ast from docker import APIClient, tls -from airflow.hooks.docker_hook import DockerHook from airflow.exceptions import AirflowException +from airflow.hooks.docker_hook import DockerHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults from airflow.utils.file import TemporaryDirectory diff --git a/airflow/operators/email_operator.py b/airflow/operators/email_operator.py index 81a89a3af6ec37..5731de5f7b3bef 100644 --- a/airflow/operators/email_operator.py +++ b/airflow/operators/email_operator.py @@ -16,11 +16,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, List, Optional +from typing import List, Optional, Union from airflow.models import BaseOperator -from airflow.utils.email import send_email from airflow.utils.decorators import apply_defaults +from airflow.utils.email import send_email class EmailOperator(BaseOperator): diff --git a/airflow/operators/gcs_to_bq.py b/airflow/operators/gcs_to_bq.py index 0890f4847a44a1..cc660efee4066e 100644 --- a/airflow/operators/gcs_to_bq.py +++ b/airflow/operators/gcs_to_bq.py @@ -23,8 +23,8 @@ import json from airflow import AirflowException -from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.gcp.hooks.bigquery import BigQueryHook +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/operators/gcs_to_gcs.py b/airflow/operators/gcs_to_gcs.py index a4d51e9a5157ac..944892a62616e4 100644 --- a/airflow/operators/gcs_to_gcs.py +++ b/airflow/operators/gcs_to_gcs.py @@ -22,10 +22,10 @@ import warnings from typing import Optional +from airflow.exceptions import AirflowException from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException WILDCARD = '*' diff --git a/airflow/operators/gcs_to_s3.py b/airflow/operators/gcs_to_s3.py index 51e4227c399e3e..3c1dcbfb6d2be3 100644 --- a/airflow/operators/gcs_to_s3.py +++ b/airflow/operators/gcs_to_s3.py @@ -21,10 +21,10 @@ """ import warnings -from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.contrib.operators.gcs_list_operator import GoogleCloudStorageListOperator -from airflow.utils.decorators import apply_defaults +from airflow.gcp.hooks.gcs import GoogleCloudStorageHook from airflow.hooks.S3_hook import S3Hook +from airflow.utils.decorators import apply_defaults class GoogleCloudStorageToS3Operator(GoogleCloudStorageListOperator): diff --git a/airflow/operators/generic_transfer.py b/airflow/operators/generic_transfer.py index dc9f809af14775..99955a0bd81f59 100644 --- a/airflow/operators/generic_transfer.py +++ b/airflow/operators/generic_transfer.py @@ -16,11 +16,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, List, Optional +from typing import List, Optional, Union +from airflow.hooks.base_hook import BaseHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.hooks.base_hook import BaseHook class GenericTransfer(BaseOperator): diff --git a/airflow/operators/google_api_to_s3_transfer.py b/airflow/operators/google_api_to_s3_transfer.py index f65c5da126347e..d05ff4a42b372a 100644 --- a/airflow/operators/google_api_to_s3_transfer.py +++ b/airflow/operators/google_api_to_s3_transfer.py @@ -23,13 +23,12 @@ import json import sys +from airflow.gcp.hooks.discovery_api import GoogleDiscoveryApiHook +from airflow.hooks.S3_hook import S3Hook from airflow.models import BaseOperator from airflow.models.xcom import MAX_XCOM_SIZE from airflow.utils.decorators import apply_defaults -from airflow.gcp.hooks.discovery_api import GoogleDiscoveryApiHook -from airflow.hooks.S3_hook import S3Hook - class GoogleApiToS3Transfer(BaseOperator): """ diff --git a/airflow/operators/hive_operator.py b/airflow/operators/hive_operator.py index 1d1a79e3fa35df..e842e6f2f89121 100644 --- a/airflow/operators/hive_operator.py +++ b/airflow/operators/hive_operator.py @@ -20,8 +20,8 @@ import re from typing import Dict, Optional -from airflow.hooks.hive_hooks import HiveCliHook from airflow.configuration import conf +from airflow.hooks.hive_hooks import HiveCliHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults from airflow.utils.operator_helpers import context_to_airflow_vars diff --git a/airflow/operators/hive_to_druid.py b/airflow/operators/hive_to_druid.py index f994e59869edb1..e6ff67284f2185 100644 --- a/airflow/operators/hive_to_druid.py +++ b/airflow/operators/hive_to_druid.py @@ -16,10 +16,10 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import List, Dict, Optional +from typing import Dict, List, Optional -from airflow.hooks.hive_hooks import HiveCliHook, HiveMetastoreHook from airflow.hooks.druid_hook import DruidHook +from airflow.hooks.hive_hooks import HiveCliHook, HiveMetastoreHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/operators/http_operator.py b/airflow/operators/http_operator.py index e7af7105da1ac2..150b3b7d9330c2 100644 --- a/airflow/operators/http_operator.py +++ b/airflow/operators/http_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Any, Dict, Callable, Optional +from typing import Any, Callable, Dict, Optional from airflow.exceptions import AirflowException from airflow.hooks.http_hook import HttpHook diff --git a/airflow/operators/jdbc_operator.py b/airflow/operators/jdbc_operator.py index 3a9b5f69236817..27f38b670757dd 100644 --- a/airflow/operators/jdbc_operator.py +++ b/airflow/operators/jdbc_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, Iterable, Mapping, Optional +from typing import Iterable, Mapping, Optional, Union from airflow.hooks.jdbc_hook import JdbcHook from airflow.models import BaseOperator diff --git a/airflow/operators/mssql_operator.py b/airflow/operators/mssql_operator.py index 5423beab586af2..e1029f340c9294 100644 --- a/airflow/operators/mssql_operator.py +++ b/airflow/operators/mssql_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, Mapping, Iterable, Optional +from typing import Iterable, Mapping, Optional, Union from airflow.hooks.mssql_hook import MsSqlHook from airflow.models import BaseOperator diff --git a/airflow/operators/mssql_to_gcs.py b/airflow/operators/mssql_to_gcs.py index d8573a9518d4b4..294a8902f1df6e 100644 --- a/airflow/operators/mssql_to_gcs.py +++ b/airflow/operators/mssql_to_gcs.py @@ -22,9 +22,9 @@ import decimal -from airflow.utils.decorators import apply_defaults from airflow.hooks.mssql_hook import MsSqlHook from airflow.operators.sql_to_gcs import BaseSQLToGoogleCloudStorageOperator +from airflow.utils.decorators import apply_defaults class MsSqlToGoogleCloudStorageOperator(BaseSQLToGoogleCloudStorageOperator): diff --git a/airflow/operators/mysql_operator.py b/airflow/operators/mysql_operator.py index 69ac03be991a65..91f3277ea0daa5 100644 --- a/airflow/operators/mysql_operator.py +++ b/airflow/operators/mysql_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, Mapping, Iterable, Optional +from typing import Iterable, Mapping, Optional, Union from airflow.hooks.mysql_hook import MySqlHook from airflow.models import BaseOperator diff --git a/airflow/operators/mysql_to_gcs.py b/airflow/operators/mysql_to_gcs.py index d7c07c81936b0a..9241994bd1fdf2 100644 --- a/airflow/operators/mysql_to_gcs.py +++ b/airflow/operators/mysql_to_gcs.py @@ -28,8 +28,8 @@ from MySQLdb.constants import FIELD_TYPE from airflow.hooks.mysql_hook import MySqlHook -from airflow.utils.decorators import apply_defaults from airflow.operators.sql_to_gcs import BaseSQLToGoogleCloudStorageOperator +from airflow.utils.decorators import apply_defaults class MySqlToGoogleCloudStorageOperator(BaseSQLToGoogleCloudStorageOperator): diff --git a/airflow/operators/mysql_to_hive.py b/airflow/operators/mysql_to_hive.py index 5d2ee0aed06cad..111f8829d03a2a 100644 --- a/airflow/operators/mysql_to_hive.py +++ b/airflow/operators/mysql_to_hive.py @@ -18,11 +18,11 @@ # under the License. from collections import OrderedDict +from tempfile import NamedTemporaryFile from typing import Dict, Optional -import unicodecsv as csv -from tempfile import NamedTemporaryFile import MySQLdb +import unicodecsv as csv from airflow.hooks.hive_hooks import HiveCliHook from airflow.hooks.mysql_hook import MySqlHook diff --git a/airflow/operators/oracle_operator.py b/airflow/operators/oracle_operator.py index 03d7116597f010..d43fef380f094c 100644 --- a/airflow/operators/oracle_operator.py +++ b/airflow/operators/oracle_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, Mapping, Iterable, Optional +from typing import Iterable, Mapping, Optional, Union from airflow.hooks.oracle_hook import OracleHook from airflow.models import BaseOperator diff --git a/airflow/operators/papermill_operator.py b/airflow/operators/papermill_operator.py index e0f02952602798..66a570bb4ae028 100644 --- a/airflow/operators/papermill_operator.py +++ b/airflow/operators/papermill_operator.py @@ -20,8 +20,8 @@ import papermill as pm -from airflow.models import BaseOperator from airflow.lineage.datasets import DataSet +from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/operators/postgres_operator.py b/airflow/operators/postgres_operator.py index e818b1cc491b8c..0be17312d0969e 100644 --- a/airflow/operators/postgres_operator.py +++ b/airflow/operators/postgres_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, Mapping, Iterable, Optional +from typing import Iterable, Mapping, Optional, Union from airflow.hooks.postgres_hook import PostgresHook from airflow.models import BaseOperator diff --git a/airflow/operators/postgres_to_gcs.py b/airflow/operators/postgres_to_gcs.py index a50a0a3d838b6b..442c48cf035ad3 100644 --- a/airflow/operators/postgres_to_gcs.py +++ b/airflow/operators/postgres_to_gcs.py @@ -20,13 +20,13 @@ PostgreSQL to GCS operator. """ -import time import datetime +import time from decimal import Decimal from airflow.hooks.postgres_hook import PostgresHook -from airflow.utils.decorators import apply_defaults from airflow.operators.sql_to_gcs import BaseSQLToGoogleCloudStorageOperator +from airflow.utils.decorators import apply_defaults class PostgresToGoogleCloudStorageOperator(BaseSQLToGoogleCloudStorageOperator): diff --git a/airflow/operators/presto_check_operator.py b/airflow/operators/presto_check_operator.py index b457ba90e1b5e4..c3226760a40b0f 100644 --- a/airflow/operators/presto_check_operator.py +++ b/airflow/operators/presto_check_operator.py @@ -19,8 +19,7 @@ from typing import Any, Dict from airflow.hooks.presto_hook import PrestoHook -from airflow.operators.check_operator import CheckOperator, \ - ValueCheckOperator, IntervalCheckOperator +from airflow.operators.check_operator import CheckOperator, IntervalCheckOperator, ValueCheckOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py index 84f64dcde9c71f..7d18576cb87486 100644 --- a/airflow/operators/python_operator.py +++ b/airflow/operators/python_operator.py @@ -26,7 +26,7 @@ from inspect import signature from itertools import islice from textwrap import dedent -from typing import Optional, Iterable, Dict, Callable, List +from typing import Callable, Dict, Iterable, List, Optional import dill diff --git a/airflow/operators/redshift_to_s3_operator.py b/airflow/operators/redshift_to_s3_operator.py index 107036d2eaef0b..11612cf04301ec 100644 --- a/airflow/operators/redshift_to_s3_operator.py +++ b/airflow/operators/redshift_to_s3_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, List, Optional +from typing import List, Optional, Union from airflow.hooks.postgres_hook import PostgresHook from airflow.hooks.S3_hook import S3Hook diff --git a/airflow/operators/s3_file_transform_operator.py b/airflow/operators/s3_file_transform_operator.py index 6e168fc264d222..bfcb733478f66a 100644 --- a/airflow/operators/s3_file_transform_operator.py +++ b/airflow/operators/s3_file_transform_operator.py @@ -17,10 +17,10 @@ # specific language governing permissions and limitations # under the License. -from tempfile import NamedTemporaryFile import subprocess import sys -from typing import Union, Optional +from tempfile import NamedTemporaryFile +from typing import Optional, Union from airflow.exceptions import AirflowException from airflow.hooks.S3_hook import S3Hook diff --git a/airflow/operators/s3_to_hive_operator.py b/airflow/operators/s3_to_hive_operator.py index 69e48e220413b3..9c174e25849ae7 100644 --- a/airflow/operators/s3_to_hive_operator.py +++ b/airflow/operators/s3_to_hive_operator.py @@ -17,21 +17,20 @@ # specific language governing permissions and limitations # under the License. -from tempfile import NamedTemporaryFile -from typing import Dict, Union, Optional - -from airflow.utils.file import TemporaryDirectory -import gzip import bz2 -import tempfile +import gzip import os +import tempfile +from tempfile import NamedTemporaryFile +from typing import Dict, Optional, Union from airflow.exceptions import AirflowException -from airflow.hooks.S3_hook import S3Hook from airflow.hooks.hive_hooks import HiveCliHook +from airflow.hooks.S3_hook import S3Hook from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults from airflow.utils.compression import uncompress_file +from airflow.utils.decorators import apply_defaults +from airflow.utils.file import TemporaryDirectory class S3ToHiveTransfer(BaseOperator): diff --git a/airflow/operators/s3_to_redshift_operator.py b/airflow/operators/s3_to_redshift_operator.py index f31d10e2d07039..0b393edfca1cb6 100644 --- a/airflow/operators/s3_to_redshift_operator.py +++ b/airflow/operators/s3_to_redshift_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, List, Optional +from typing import List, Optional, Union from airflow.hooks.postgres_hook import PostgresHook from airflow.hooks.S3_hook import S3Hook diff --git a/airflow/operators/slack_operator.py b/airflow/operators/slack_operator.py index 225573ab04d37b..c3998ea5675bb1 100644 --- a/airflow/operators/slack_operator.py +++ b/airflow/operators/slack_operator.py @@ -18,12 +18,12 @@ # under the License. import json -from typing import Dict, Optional, List +from typing import Dict, List, Optional +from airflow.exceptions import AirflowException +from airflow.hooks.slack_hook import SlackHook from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults -from airflow.hooks.slack_hook import SlackHook -from airflow.exceptions import AirflowException class SlackAPIOperator(BaseOperator): diff --git a/airflow/operators/sqlite_operator.py b/airflow/operators/sqlite_operator.py index 203972c4a98a2c..4879de12b965d8 100644 --- a/airflow/operators/sqlite_operator.py +++ b/airflow/operators/sqlite_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Union, Mapping, Iterable, Optional +from typing import Iterable, Mapping, Optional, Union from airflow.hooks.sqlite_hook import SqliteHook from airflow.models import BaseOperator diff --git a/airflow/operators/subdag_operator.py b/airflow/operators/subdag_operator.py index f0fbb1fdf2aef6..84ce1a4295b7ac 100644 --- a/airflow/operators/subdag_operator.py +++ b/airflow/operators/subdag_operator.py @@ -17,9 +17,10 @@ # specific language governing permissions and limitations # under the License. -from sqlalchemy.orm.session import Session from typing import Optional +from sqlalchemy.orm.session import Session + from airflow import settings from airflow.exceptions import AirflowException from airflow.models.dag import DAG @@ -27,9 +28,9 @@ from airflow.models.pool import Pool from airflow.models.taskinstance import TaskInstance from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.utils.state import State -from airflow.utils.decorators import apply_defaults from airflow.utils.db import create_session, provide_session +from airflow.utils.decorators import apply_defaults +from airflow.utils.state import State class SubDagOperator(BaseSensorOperator): diff --git a/airflow/plugins_manager.py b/airflow/plugins_manager.py index acf3a7494e5741..d2d9a8984dddf7 100644 --- a/airflow/plugins_manager.py +++ b/airflow/plugins_manager.py @@ -21,8 +21,9 @@ import inspect import os import re +from typing import Any, List + import pkg_resources -from typing import List, Any from airflow import settings from airflow.models.baseoperator import BaseOperatorLink diff --git a/airflow/security/kerberos.py b/airflow/security/kerberos.py index 18f52c3834c43e..a913e9efae24bd 100644 --- a/airflow/security/kerberos.py +++ b/airflow/security/kerberos.py @@ -33,11 +33,11 @@ # limitations under the License. """Kerberos security provider""" -from typing import Optional import socket import subprocess import sys import time +from typing import Optional from airflow import LoggingMixin from airflow.configuration import conf diff --git a/airflow/sensors/base_sensor_operator.py b/airflow/sensors/base_sensor_operator.py index 9d421932c8675e..b257e24d4e2dab 100644 --- a/airflow/sensors/base_sensor_operator.py +++ b/airflow/sensors/base_sensor_operator.py @@ -18,16 +18,17 @@ # under the License. -from time import sleep from datetime import timedelta +from time import sleep from typing import Dict, Iterable -from airflow.exceptions import AirflowException, AirflowSensorTimeout, \ - AirflowSkipException, AirflowRescheduleException +from airflow.exceptions import ( + AirflowException, AirflowRescheduleException, AirflowSensorTimeout, AirflowSkipException, +) from airflow.models import BaseOperator, SkipMixin, TaskReschedule +from airflow.ti_deps.deps.ready_to_reschedule import ReadyToRescheduleDep from airflow.utils import timezone from airflow.utils.decorators import apply_defaults -from airflow.ti_deps.deps.ready_to_reschedule import ReadyToRescheduleDep class BaseSensorOperator(BaseOperator, SkipMixin): diff --git a/airflow/sensors/external_task_sensor.py b/airflow/sensors/external_task_sensor.py index 9e181fa64b5742..71b1517caeed1a 100644 --- a/airflow/sensors/external_task_sensor.py +++ b/airflow/sensors/external_task_sensor.py @@ -20,7 +20,7 @@ import os from airflow.exceptions import AirflowException -from airflow.models import TaskInstance, DagBag, DagModel, DagRun +from airflow.models import DagBag, DagModel, DagRun, TaskInstance from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.db import provide_session from airflow.utils.decorators import apply_defaults diff --git a/airflow/sensors/http_sensor.py b/airflow/sensors/http_sensor.py index ab07e759733497..ee036095ca090c 100644 --- a/airflow/sensors/http_sensor.py +++ b/airflow/sensors/http_sensor.py @@ -16,12 +16,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Dict, Callable, Optional - -from airflow.operators.python_operator import PythonOperator +from typing import Callable, Dict, Optional from airflow.exceptions import AirflowException from airflow.hooks.http_hook import HttpHook +from airflow.operators.python_operator import PythonOperator from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.decorators import apply_defaults diff --git a/airflow/sensors/s3_prefix_sensor.py b/airflow/sensors/s3_prefix_sensor.py index 6035847719d16e..18db735d10cfab 100644 --- a/airflow/sensors/s3_prefix_sensor.py +++ b/airflow/sensors/s3_prefix_sensor.py @@ -18,7 +18,6 @@ # under the License. from airflow.sensors.base_sensor_operator import BaseSensorOperator - from airflow.utils.decorators import apply_defaults diff --git a/airflow/sentry.py b/airflow/sentry.py index efd91340b7c2b4..c101f4c71a5c38 100644 --- a/airflow/sentry.py +++ b/airflow/sentry.py @@ -27,7 +27,6 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State - log = LoggingMixin().log diff --git a/airflow/settings.py b/airflow/settings.py index ba6f4abd035d6b..252a70c4893a26 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -17,21 +17,21 @@ # specific language governing permissions and limitations # under the License. -from typing import Optional import atexit import logging import os -import pendulum import sys +from typing import Optional +import pendulum from sqlalchemy import create_engine, exc from sqlalchemy.engine import Engine from sqlalchemy.orm import scoped_session, sessionmaker -from sqlalchemy.pool import NullPool from sqlalchemy.orm.session import Session as SASession +from sqlalchemy.pool import NullPool import airflow -from airflow.configuration import conf, AIRFLOW_HOME, WEBSERVER_CONFIG # NOQA F401 +from airflow.configuration import AIRFLOW_HOME, WEBSERVER_CONFIG, conf # NOQA F401 from airflow.logging_config import configure_logging from airflow.utils.sqlalchemy import setup_event_handlers diff --git a/airflow/stats.py b/airflow/stats.py index f784212389cd85..64694ea0fa0afa 100644 --- a/airflow/stats.py +++ b/airflow/stats.py @@ -18,11 +18,11 @@ # under the License. -from functools import wraps import logging import socket import string import textwrap +from functools import wraps from typing import Any from airflow.configuration import conf diff --git a/airflow/task/task_runner/__init__.py b/airflow/task/task_runner/__init__.py index 0c422013dbe192..6d0d1d92e7f3c3 100644 --- a/airflow/task/task_runner/__init__.py +++ b/airflow/task/task_runner/__init__.py @@ -18,8 +18,8 @@ # under the License. from airflow.configuration import conf -from airflow.task.task_runner.standard_task_runner import StandardTaskRunner from airflow.exceptions import AirflowException +from airflow.task.task_runner.standard_task_runner import StandardTaskRunner _TASK_RUNNER = conf.get('core', 'TASK_RUNNER') diff --git a/airflow/task/task_runner/base_task_runner.py b/airflow/task/task_runner/base_task_runner.py index a46a3bfe19b353..f944a474635317 100644 --- a/airflow/task/task_runner/base_task_runner.py +++ b/airflow/task/task_runner/base_task_runner.py @@ -22,11 +22,9 @@ import subprocess import threading -from airflow.utils.log.logging_mixin import LoggingMixin - from airflow.configuration import conf from airflow.utils.configuration import tmp_configuration_copy - +from airflow.utils.log.logging_mixin import LoggingMixin PYTHONPATH_VAR = 'PYTHONPATH' diff --git a/airflow/utils/cli_action_loggers.py b/airflow/utils/cli_action_loggers.py index 72d1effb2f2556..bbb30bb8b6ca16 100644 --- a/airflow/utils/cli_action_loggers.py +++ b/airflow/utils/cli_action_loggers.py @@ -23,7 +23,7 @@ """ import logging -from typing import List, Callable +from typing import Callable, List from airflow.utils.db import create_session diff --git a/airflow/utils/compression.py b/airflow/utils/compression.py index 2565299175bb07..07a2790d9d6bf7 100644 --- a/airflow/utils/compression.py +++ b/airflow/utils/compression.py @@ -17,10 +17,10 @@ # specific language governing permissions and limitations # under the License. -from tempfile import NamedTemporaryFile -import shutil -import gzip import bz2 +import gzip +import shutil +from tempfile import NamedTemporaryFile def uncompress_file(input_file_name, file_extension, dest_dir): diff --git a/airflow/utils/configuration.py b/airflow/utils/configuration.py index 79e5f1bb1f3ab6..a0bc80412252a8 100644 --- a/airflow/utils/configuration.py +++ b/airflow/utils/configuration.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -import os import json +import os from tempfile import mkstemp from airflow.configuration import conf diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index bf425a8857a9f6..aadbac9d4d69e4 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -17,6 +17,7 @@ # specific language governing permissions and limitations # under the License. +import enum import importlib import logging import multiprocessing @@ -29,8 +30,7 @@ from abc import ABCMeta, abstractmethod from datetime import datetime from importlib import import_module -import enum -from typing import Optional, NamedTuple, Iterable +from typing import Iterable, NamedTuple, Optional import psutil from setproctitle import setproctitle @@ -44,8 +44,8 @@ from airflow.models import errors from airflow.stats import Stats from airflow.utils import timezone -from airflow.utils.helpers import reap_process_group from airflow.utils.db import provide_session +from airflow.utils.helpers import reap_process_group from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/utils/dates.py b/airflow/utils/dates.py index 5bbe8a18d91feb..89e67db73adfc2 100644 --- a/airflow/utils/dates.py +++ b/airflow/utils/dates.py @@ -17,11 +17,12 @@ # specific language governing permissions and limitations # under the License. -from airflow.utils import timezone from datetime import datetime, timedelta -from dateutil.relativedelta import relativedelta # noqa: F401 for doctest from croniter import croniter +from dateutil.relativedelta import relativedelta # noqa: F401 for doctest + +from airflow.utils import timezone cron_presets = { '@hourly': '0 * * * *', diff --git a/airflow/utils/db.py b/airflow/utils/db.py index 7b99a019dfbad9..32d7450f6bcb90 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -17,10 +17,9 @@ # specific language governing permissions and limitations # under the License. -from functools import wraps - -import os import contextlib +import os +from functools import wraps from airflow import settings from airflow.configuration import conf diff --git a/airflow/utils/decorators.py b/airflow/utils/decorators.py index 9776d16fe91163..2e77aaee8a783c 100644 --- a/airflow/utils/decorators.py +++ b/airflow/utils/decorators.py @@ -20,7 +20,6 @@ import inspect import os - from copy import copy from functools import wraps diff --git a/airflow/utils/email.py b/airflow/utils/email.py index 46153b01958e3a..12123030e897a3 100644 --- a/airflow/utils/email.py +++ b/airflow/utils/email.py @@ -20,10 +20,9 @@ import importlib import os import smtplib - -from email.mime.text import MIMEText -from email.mime.multipart import MIMEMultipart from email.mime.application import MIMEApplication +from email.mime.multipart import MIMEMultipart +from email.mime.text import MIMEText from email.utils import formatdate from airflow.configuration import conf diff --git a/airflow/utils/file.py b/airflow/utils/file.py index b0ece81223b35a..2d654df04e9977 100644 --- a/airflow/utils/file.py +++ b/airflow/utils/file.py @@ -20,9 +20,8 @@ import errno import os import shutil -from tempfile import mkdtemp - from contextlib import contextmanager +from tempfile import mkdtemp @contextmanager diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py index 31df6ac1beb2c5..5f034bcf168984 100644 --- a/airflow/utils/helpers.py +++ b/airflow/utils/helpers.py @@ -18,27 +18,26 @@ # under the License. import errno - -import psutil - -from datetime import datetime -from functools import reduce -try: - # Fix Python > 3.7 deprecation - from collections.abc import Iterable -except ImportError: - # Preserve Python < 3.3 compatibility - from collections import Iterable import os import re import signal import subprocess +from datetime import datetime +from functools import reduce +import psutil from jinja2 import Template from airflow.configuration import conf from airflow.exceptions import AirflowException +try: + # Fix Python > 3.7 deprecation + from collections.abc import Iterable +except ImportError: + # Preserve Python < 3.3 compatibility + from collections import Iterable + # When killing processes, time to wait after issuing a SIGTERM before issuing a # SIGKILL. DEFAULT_TIME_TO_WAIT_AFTER_SIGTERM = conf.getint( diff --git a/airflow/utils/json.py b/airflow/utils/json.py index 54795efff758ca..ebf9ea5b17cc17 100644 --- a/airflow/utils/json.py +++ b/airflow/utils/json.py @@ -17,13 +17,14 @@ # specific language governing permissions and limitations # under the License. -from datetime import datetime, date import json -import numpy as np +from datetime import date, datetime +import numpy as np # Dates and JSON encoding/decoding + class AirflowJsonEncoder(json.JSONEncoder): def default(self, obj): diff --git a/airflow/utils/log/colored_log.py b/airflow/utils/log/colored_log.py index 496885cfa58f24..7f364306447d3c 100644 --- a/airflow/utils/log/colored_log.py +++ b/airflow/utils/log/colored_log.py @@ -21,9 +21,9 @@ """ import re import sys +from logging import LogRecord from typing import Any, Union -from logging import LogRecord from colorlog import TTYColoredFormatter from termcolor import colored diff --git a/airflow/utils/log/es_task_handler.py b/airflow/utils/log/es_task_handler.py index 054fafce3f1239..984b88564fdc75 100644 --- a/airflow/utils/log/es_task_handler.py +++ b/airflow/utils/log/es_task_handler.py @@ -17,19 +17,20 @@ # specific language governing permissions and limitations # under the License. -# Using `from elasticsearch import *` would break elasticsearch mocking used in unit test. -import elasticsearch import logging import sys + +# Using `from elasticsearch import *` would break elasticsearch mocking used in unit test. +import elasticsearch import pendulum from elasticsearch_dsl import Search +from airflow.configuration import conf from airflow.utils import timezone from airflow.utils.helpers import parse_template_string from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.json_formatter import JSONFormatter from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.configuration import conf class ElasticsearchTaskHandler(FileTaskHandler, LoggingMixin): diff --git a/airflow/utils/log/file_processor_handler.py b/airflow/utils/log/file_processor_handler.py index 5c9f19a0d8e60e..c3b508548f6f2d 100644 --- a/airflow/utils/log/file_processor_handler.py +++ b/airflow/utils/log/file_processor_handler.py @@ -19,10 +19,10 @@ import logging import os +from datetime import datetime from airflow import settings from airflow.utils.helpers import parse_template_string -from datetime import datetime class FileProcessorHandler(logging.Handler): diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index b389dee4e18f2e..a53461299392ac 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -23,8 +23,7 @@ import requests -from airflow.configuration import conf -from airflow.configuration import AirflowConfigException +from airflow.configuration import AirflowConfigException, conf from airflow.models import TaskInstance from airflow.utils.file import mkdirs from airflow.utils.helpers import parse_template_string diff --git a/airflow/utils/log/gcs_task_handler.py b/airflow/utils/log/gcs_task_handler.py index b685d2d0fdbde5..2f25be0a636c29 100644 --- a/airflow/utils/log/gcs_task_handler.py +++ b/airflow/utils/log/gcs_task_handler.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. import os +from urllib.parse import urlparse from cached_property import cached_property -from urllib.parse import urlparse from airflow.configuration import conf from airflow.exceptions import AirflowException diff --git a/airflow/utils/log/json_formatter.py b/airflow/utils/log/json_formatter.py index 4e7e4a0fffa5f4..b7f9bc7eb6f87b 100644 --- a/airflow/utils/log/json_formatter.py +++ b/airflow/utils/log/json_formatter.py @@ -21,8 +21,8 @@ json_formatter module stores all related to ElasticSearch specific logger classes """ -import logging import json +import logging def merge_dicts(dict1, dict2): diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index b7ee7098018ad7..ac4a5e00af4144 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -16,10 +16,9 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import re import logging +import re import sys - from contextlib import contextmanager from logging import Handler, StreamHandler diff --git a/airflow/utils/log/s3_task_handler.py b/airflow/utils/log/s3_task_handler.py index 848a5bc859e2f3..95b69bf7e65b4a 100644 --- a/airflow/utils/log/s3_task_handler.py +++ b/airflow/utils/log/s3_task_handler.py @@ -21,8 +21,8 @@ from cached_property import cached_property from airflow.configuration import conf -from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.logging_mixin import LoggingMixin class S3TaskHandler(FileTaskHandler, LoggingMixin): diff --git a/airflow/utils/log/wasb_task_handler.py b/airflow/utils/log/wasb_task_handler.py index d480a86931ef5f..aa57bb64c39a3e 100644 --- a/airflow/utils/log/wasb_task_handler.py +++ b/airflow/utils/log/wasb_task_handler.py @@ -19,12 +19,12 @@ import os import shutil +from azure.common import AzureHttpError from cached_property import cached_property from airflow.configuration import conf -from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.log.file_task_handler import FileTaskHandler -from azure.common import AzureHttpError +from airflow.utils.log.logging_mixin import LoggingMixin class WasbTaskHandler(FileTaskHandler, LoggingMixin): diff --git a/airflow/utils/net.py b/airflow/utils/net.py index 206102591344ae..318cfa31b234cd 100644 --- a/airflow/utils/net.py +++ b/airflow/utils/net.py @@ -19,7 +19,8 @@ # import importlib import socket -from airflow.configuration import (conf, AirflowConfigException) + +from airflow.configuration import AirflowConfigException, conf def get_host_ip_address(): diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index 423cd283c39b27..bab7f7e0755672 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -18,13 +18,13 @@ # under the License. import datetime -import os import json -import pendulum +import os +import pendulum from dateutil import relativedelta from sqlalchemy import event, exc -from sqlalchemy.types import Text, DateTime, TypeDecorator +from sqlalchemy.types import DateTime, Text, TypeDecorator from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/utils/timeout.py b/airflow/utils/timeout.py index 896293ebe28556..6a50ebe67815f5 100644 --- a/airflow/utils/timeout.py +++ b/airflow/utils/timeout.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -import signal import os +import signal from airflow.exceptions import AirflowTaskTimeout from airflow.utils.log.logging_mixin import LoggingMixin diff --git a/airflow/utils/timezone.py b/airflow/utils/timezone.py index fd3dffa908b4d7..13b0a047d7e3f7 100644 --- a/airflow/utils/timezone.py +++ b/airflow/utils/timezone.py @@ -18,11 +18,11 @@ # under the License. # import datetime as dt + import pendulum from airflow.settings import TIMEZONE - # UTC time zone as a tzinfo instance. utc = pendulum.timezone('UTC') diff --git a/airflow/www/api/experimental/endpoints.py b/airflow/www/api/experimental/endpoints.py index 5a3b10eea94476..27ec69b39cba72 100644 --- a/airflow/www/api/experimental/endpoints.py +++ b/airflow/www/api/experimental/endpoints.py @@ -16,22 +16,20 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from flask import Blueprint, g, jsonify, request, url_for + import airflow.api -from airflow.api.common.experimental import delete_dag as delete -from airflow.api.common.experimental import pool as pool_api -from airflow.api.common.experimental import trigger_dag as trigger +from airflow import models +from airflow.api.common.experimental import delete_dag as delete, pool as pool_api, trigger_dag as trigger +from airflow.api.common.experimental.get_code import get_code +from airflow.api.common.experimental.get_dag_run_state import get_dag_run_state from airflow.api.common.experimental.get_dag_runs import get_dag_runs from airflow.api.common.experimental.get_task import get_task from airflow.api.common.experimental.get_task_instance import get_task_instance -from airflow.api.common.experimental.get_code import get_code -from airflow.api.common.experimental.get_dag_run_state import get_dag_run_state from airflow.exceptions import AirflowException -from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils import timezone +from airflow.utils.log.logging_mixin import LoggingMixin from airflow.www.app import csrf -from airflow import models - -from flask import g, Blueprint, jsonify, request, url_for _log = LoggingMixin().log diff --git a/airflow/www/app.py b/airflow/www/app.py index 0ce638e9015b5c..71cfc3fc89c52f 100644 --- a/airflow/www/app.py +++ b/airflow/www/app.py @@ -17,21 +17,20 @@ # specific language governing permissions and limitations # under the License. # -from typing import Optional import logging import socket +from typing import Any, Optional +from urllib.parse import urlparse from flask import Flask -from flask_appbuilder import AppBuilder, SQLA +from flask_appbuilder import SQLA, AppBuilder from flask_caching import Cache from flask_wtf.csrf import CSRFProtect -from typing import Any -from urllib.parse import urlparse -from werkzeug.middleware.proxy_fix import ProxyFix from werkzeug.middleware.dispatcher import DispatcherMiddleware +from werkzeug.middleware.proxy_fix import ProxyFix -from airflow.configuration import conf from airflow import settings +from airflow.configuration import conf from airflow.logging_config import configure_logging from airflow.utils.json import AirflowJsonEncoder from airflow.www.static_config import configure_manifest_files diff --git a/airflow/www/decorators.py b/airflow/www/decorators.py index 91ef51e4f0383a..c91e08c458d77b 100644 --- a/airflow/www/decorators.py +++ b/airflow/www/decorators.py @@ -17,11 +17,13 @@ # specific language governing permissions and limitations # under the License. -import gzip import functools -import pendulum +import gzip from io import BytesIO as IO -from flask import after_this_request, flash, redirect, request, url_for, g + +import pendulum +from flask import after_this_request, flash, g, redirect, request, url_for + from airflow.models import Log from airflow.utils.db import create_session diff --git a/airflow/www/forms.py b/airflow/www/forms.py index 89e8ffc3226bf8..af75cccad5444f 100644 --- a/airflow/www/forms.py +++ b/airflow/www/forms.py @@ -17,19 +17,19 @@ # specific language governing permissions and limitations # under the License. -from airflow.models import Connection -from airflow.utils import timezone - +from flask_appbuilder.fieldwidgets import ( + BS3PasswordFieldWidget, BS3TextAreaFieldWidget, BS3TextFieldWidget, DateTimePickerWidget, Select2Widget, +) from flask_appbuilder.forms import DynamicForm -from flask_appbuilder.fieldwidgets import (BS3TextFieldWidget, BS3TextAreaFieldWidget, - BS3PasswordFieldWidget, Select2Widget, - DateTimePickerWidget) from flask_babel import lazy_gettext from flask_wtf import FlaskForm - from wtforms import validators -from wtforms.fields import (IntegerField, SelectField, TextAreaField, PasswordField, - StringField, DateTimeField, BooleanField) +from wtforms.fields import ( + BooleanField, DateTimeField, IntegerField, PasswordField, SelectField, StringField, TextAreaField, +) + +from airflow.models import Connection +from airflow.utils import timezone class DateTimeForm(FlaskForm): diff --git a/airflow/www/gunicorn_config.py b/airflow/www/gunicorn_config.py index db76b6de0c5af3..4eb1d5fd31eba9 100644 --- a/airflow/www/gunicorn_config.py +++ b/airflow/www/gunicorn_config.py @@ -19,6 +19,7 @@ # under the License. import setproctitle + from airflow import settings diff --git a/airflow/www/security.py b/airflow/www/security.py index 36fe22fd415d07..48cd177fb52c44 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -25,10 +25,9 @@ from airflow import models from airflow.exceptions import AirflowException -from airflow.www.app import appbuilder from airflow.utils.db import provide_session from airflow.utils.log.logging_mixin import LoggingMixin - +from airflow.www.app import appbuilder EXISTING_ROLES = { 'Admin', diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 5a77822c23308b..4902a3c1daca9c 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -19,22 +19,22 @@ import functools import inspect +import io import json -import time -import markdown +import os import re -from typing import Any, Optional +import time import zipfile -import os -import io +from typing import Any, Optional +from urllib.parse import urlencode -from pygments import highlight, lexers -from pygments.formatters import HtmlFormatter -from flask import request, Response, Markup, url_for -from flask_appbuilder.models.sqla.interface import SQLAInterface import flask_appbuilder.models.sqla.filters as fab_sqlafilters +import markdown import sqlalchemy as sqla -from urllib.parse import urlencode +from flask import Markup, Response, request, url_for +from flask_appbuilder.models.sqla.interface import SQLAInterface +from pygments import highlight, lexers +from pygments.formatters import HtmlFormatter from airflow.configuration import conf from airflow.models import BaseOperator diff --git a/airflow/www/validators.py b/airflow/www/validators.py index 62b7db82523bc2..ae20e450218eee 100644 --- a/airflow/www/validators.py +++ b/airflow/www/validators.py @@ -17,8 +17,7 @@ # specific language governing permissions and limitations # under the License. -from wtforms.validators import EqualTo -from wtforms.validators import ValidationError +from wtforms.validators import EqualTo, ValidationError class GreaterEqualThan(EqualTo): diff --git a/airflow/www/views.py b/airflow/www/views.py index 1c12c5aa53cfe9..75f051052199ee 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -28,33 +28,31 @@ import traceback from collections import defaultdict from datetime import timedelta +from urllib.parse import quote +import lazy_object_proxy import markdown import pendulum import sqlalchemy as sqla -from flask import (Markup, Response, flash, jsonify, make_response, redirect, - render_template, request, url_for) +from flask import Markup, Response, flash, jsonify, make_response, redirect, render_template, request, url_for from flask_appbuilder import BaseView, ModelView, expose, has_access from flask_appbuilder.actions import action from flask_appbuilder.models.sqla.filters import BaseFilter from flask_babel import lazy_gettext -import lazy_object_proxy from pygments import highlight, lexers from pygments.formatters import HtmlFormatter -from sqlalchemy import or_, desc, and_, union_all -from urllib.parse import quote +from sqlalchemy import and_, desc, or_, union_all from wtforms import SelectField, validators import airflow -from airflow.configuration import conf, AIRFLOW_CONFIG -from airflow import jobs, models -from airflow import settings +from airflow import jobs, models, settings from airflow._vendor import nvd3 -from airflow.api.common.experimental.mark_tasks import (set_dag_run_state_to_failed, - set_dag_run_state_to_success) -from airflow.models import Connection, DagModel, DagRun, Log, SlaMiss, TaskFail, XCom, \ - errors -from airflow.ti_deps.dep_context import DepContext, SCHEDULER_QUEUED_DEPS +from airflow.api.common.experimental.mark_tasks import ( + set_dag_run_state_to_failed, set_dag_run_state_to_success, +) +from airflow.configuration import AIRFLOW_CONFIG, conf +from airflow.models import Connection, DagModel, DagRun, Log, SlaMiss, TaskFail, XCom, errors +from airflow.ti_deps.dep_context import SCHEDULER_QUEUED_DEPS, DepContext from airflow.utils import timezone from airflow.utils.dates import infer_time_unit, scale_time_units from airflow.utils.db import create_session, provide_session @@ -63,9 +61,9 @@ from airflow.www import utils as wwwutils from airflow.www.app import app, appbuilder from airflow.www.decorators import action_logging, gzipped, has_dag_access -from airflow.www.forms import (ConnectionForm, DagRunForm, DateTimeForm, - DateTimeWithNumRunsForm, - DateTimeWithNumRunsWithDagRunsForm) +from airflow.www.forms import ( + ConnectionForm, DagRunForm, DateTimeForm, DateTimeWithNumRunsForm, DateTimeWithNumRunsWithDagRunsForm, +) from airflow.www.widgets import AirflowModelListWidget PAGE_SIZE = conf.getint('webserver', 'page_size') diff --git a/dags/test_dag.py b/dags/test_dag.py index 245764ae1999ce..358a5de1a1ca5c 100644 --- a/dags/test_dag.py +++ b/dags/test_dag.py @@ -21,8 +21,7 @@ """ from datetime import datetime, timedelta -from airflow import DAG -from airflow import utils +from airflow import DAG, utils from airflow.operators.dummy_operator import DummyOperator now = datetime.now() diff --git a/dev/airflow-jira b/dev/airflow-jira index 36d09d3017394d..07ab1952c3666b 100755 --- a/dev/airflow-jira +++ b/dev/airflow-jira @@ -29,10 +29,11 @@ # This tool is based on the Spark merge_spark_pr script: # https://github.com/apache/spark/blob/master/dev/merge_spark_pr.py -import jira import re import sys +import jira + TMP_CREDENTIALS = {} PROJECT = "AIRFLOW" diff --git a/dev/airflow-license b/dev/airflow-license index 91a283c78f3eb5..367d8ffb58103f 100755 --- a/dev/airflow-license +++ b/dev/airflow-license @@ -19,9 +19,10 @@ import os import re -import slugify import string +import slugify + # order is important _licenses = {'MIT': ['Permission is hereby granted free of charge', 'The above copyright notice and this permission notice shall'], 'BSD-3': ['Redistributions of source code must retain the above copyright', 'Redistributions in binary form must reproduce the above copyright', 'specific prior written permission'], diff --git a/docs/conf.py b/docs/conf.py index ac943d9be8f3dd..2ed3b44b831cd6 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -281,7 +281,7 @@ # Add any paths that contain custom themes here, relative to this directory. # html_theme_path = [] -import sphinx_rtd_theme # pylint: disable=wrong-import-position,wrong-import-order +import sphinx_rtd_theme # isort:skip pylint: disable=wrong-import-position,wrong-import-order html_theme_path = [sphinx_rtd_theme.get_html_theme_path()] diff --git a/docs/exts/exampleinclude.py b/docs/exts/exampleinclude.py index 3f802411ac7374..f1f5e6a1da08c7 100644 --- a/docs/exts/exampleinclude.py +++ b/docs/exts/exampleinclude.py @@ -29,8 +29,7 @@ from sphinx.directives.code import LiteralIncludeReader from sphinx.locale import _ from sphinx.pycode import ModuleAnalyzer -from sphinx.util import logging -from sphinx.util import parselinenos +from sphinx.util import logging, parselinenos from sphinx.util.docutils import SphinxDirective from sphinx.util.nodes import set_source_info diff --git a/docs/exts/removemarktransform.py b/docs/exts/removemarktransform.py index 4b045af235c254..ed768ecc5c5a03 100644 --- a/docs/exts/removemarktransform.py +++ b/docs/exts/removemarktransform.py @@ -24,7 +24,7 @@ from docutils import nodes # noinspection PyUnresolvedReferences -from pygments.lexers import guess_lexer, Python3Lexer, PythonLexer # pylint: disable=no-name-in-module +from pygments.lexers import Python3Lexer, PythonLexer, guess_lexer # pylint: disable=no-name-in-module from sphinx.transforms import SphinxTransform from sphinx.transforms.post_transforms.code import TrimDoctestFlagsTransform diff --git a/pylintrc b/pylintrc index 2d4c5c67b73698..a18bf4d01030d8 100644 --- a/pylintrc +++ b/pylintrc @@ -168,7 +168,8 @@ disable=print-statement, arguments-differ, # Doesn't always raise valid messages import-error, # Requires installing Airflow environment in CI task which takes long, therefore ignored. Tests should fail anyways if deps are missing. Possibly un-ignore in the future if we ever use pre-built Docker images for CI. fixme, # There should be a good reason for adding a TODO - pointless-statement # Is raised on the bitshift operator. Could be disabled only on /example_dags after https://github.com/PyCQA/pylint/projects/1. + pointless-statement, # Is raised on the bitshift operator. Could be disabled only on /example_dags after https://github.com/PyCQA/pylint/projects/1. + ungrouped-imports # Disabled to avoid conflict with flake8-isort import order rules, which is enabled in the project. # Enable the message, report, category or checker with the given id(s). You can # either give multiple identifier separated by comma (,) or put this option diff --git a/scripts/ci/ci_isort.sh b/scripts/ci/ci_isort.sh new file mode 100755 index 00000000000000..a8ada59956858b --- /dev/null +++ b/scripts/ci/ci_isort.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash +# 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. + +set -euo pipefail + +MY_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" + +export AIRFLOW_CI_SILENT=${AIRFLOW_CI_SILENT:="true"} +export ASSUME_QUIT_TO_ALL_QUESTIONS=${ASSUME_QUIT_TO_ALL_QUESTIONS:="true"} + +# shellcheck source=scripts/ci/_utils.sh +. "${MY_DIR}/_utils.sh" + +basic_sanity_checks + +force_python_3_5 + +script_start + +rebuild_ci_slim_image_if_needed + +run_flake8 "$@" + +script_end diff --git a/scripts/perf/scheduler_ops_metrics.py b/scripts/perf/scheduler_ops_metrics.py index c44ade0e41667c..26b07e8481503d 100644 --- a/scripts/perf/scheduler_ops_metrics.py +++ b/scripts/perf/scheduler_ops_metrics.py @@ -18,9 +18,10 @@ # under the License. import logging -import pandas as pd import sys +import pandas as pd + from airflow import settings from airflow.configuration import conf from airflow.jobs import SchedulerJob diff --git a/setup.cfg b/setup.cfg index ab69124adc6bd8..98bf437382da29 100644 --- a/setup.cfg +++ b/setup.cfg @@ -47,3 +47,13 @@ ignore_errors = True [mypy-airflow.contrib.auth.*] ignore_errors = True + +[isort] +line_length=110 +combine_as_imports = true +default_section = THIRDPARTY +include_trailing_comma = true +known_first_party=airflow,tests +multi_line_output=5 +# Need to be consistent with the exclude config defined in pre-commit-config.yaml +skip=airflow/_vendor,build,.tox,venv diff --git a/setup.py b/setup.py index 2a06c664f716f1..fe7d8131372c8b 100644 --- a/setup.py +++ b/setup.py @@ -18,16 +18,16 @@ # under the License. """Setup.py for the Airflow project.""" -from importlib import util import io import logging import os import subprocess import sys import unittest +from importlib import util from typing import List -from setuptools import setup, find_packages, Command +from setuptools import Command, find_packages, setup logger = logging.getLogger(__name__) @@ -283,6 +283,7 @@ def write_version(filename: str = os.path.join(*["airflow", "git_version"])): 'dumb-init>=1.2.2', 'flake8>=3.6.0', 'flake8-colors', + 'flake8-isort', 'freezegun', 'ipdb', 'jira', diff --git a/tests/api/client/test_local_client.py b/tests/api/client/test_local_client.py index b8f1c314c5e9e1..6ac017b9806831 100644 --- a/tests/api/client/test_local_client.py +++ b/tests/api/client/test_local_client.py @@ -23,11 +23,10 @@ from freezegun import freeze_time -from airflow import AirflowException -from airflow import models +from airflow import AirflowException, models from airflow.api.client.local_client import Client from airflow.example_dags import example_bash_operator -from airflow.models import DagModel, DagBag +from airflow.models import DagBag, DagModel from airflow.utils import timezone from airflow.utils.db import create_session from airflow.utils.state import State diff --git a/tests/api/common/experimental/test_mark_tasks.py b/tests/api/common/experimental/test_mark_tasks.py index 2685cc66943a4d..e234d34245a28b 100644 --- a/tests/api/common/experimental/test_mark_tasks.py +++ b/tests/api/common/experimental/test_mark_tasks.py @@ -17,20 +17,21 @@ # specific language governing permissions and limitations # under the License. -import unittest import time +import unittest from datetime import datetime, timedelta from airflow import models -from airflow.configuration import conf from airflow.api.common.experimental.mark_tasks import ( - set_state, _create_dagruns, set_dag_run_state_to_success, set_dag_run_state_to_failed, - set_dag_run_state_to_running) + _create_dagruns, set_dag_run_state_to_failed, set_dag_run_state_to_running, set_dag_run_state_to_success, + set_state, +) +from airflow.configuration import conf +from airflow.models import DagRun from airflow.utils import timezone -from airflow.utils.db import create_session, provide_session from airflow.utils.dates import days_ago +from airflow.utils.db import create_session, provide_session from airflow.utils.state import State -from airflow.models import DagRun from tests.test_utils.db import clear_db_runs DEV_NULL = "/dev/null" diff --git a/tests/api/common/experimental/test_pool.py b/tests/api/common/experimental/test_pool.py index 1a639b84dabef2..25bcce6782f3ba 100644 --- a/tests/api/common/experimental/test_pool.py +++ b/tests/api/common/experimental/test_pool.py @@ -20,9 +20,9 @@ import unittest from airflow import models -from airflow.models.pool import Pool from airflow.api.common.experimental import pool as pool_api from airflow.exceptions import AirflowBadRequest, PoolNotFound +from airflow.models.pool import Pool from airflow.utils.db import create_session from tests.test_utils.db import clear_db_pools diff --git a/tests/api/common/experimental/test_trigger_dag.py b/tests/api/common/experimental/test_trigger_dag.py index 47afdfa58dc250..b787f840ffbc39 100644 --- a/tests/api/common/experimental/test_trigger_dag.py +++ b/tests/api/common/experimental/test_trigger_dag.py @@ -17,13 +17,13 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock -import unittest import json +import unittest +from unittest import mock +from airflow.api.common.experimental.trigger_dag import _trigger_dag from airflow.exceptions import AirflowException from airflow.models import DAG, DagRun -from airflow.api.common.experimental.trigger_dag import _trigger_dag class TestTriggerDag(unittest.TestCase): diff --git a/tests/cli/test_cli.py b/tests/cli/test_cli.py index 903ea53f60d2fa..7c33f6dab871d3 100644 --- a/tests/cli/test_cli.py +++ b/tests/cli/test_cli.py @@ -18,28 +18,27 @@ # under the License. import contextlib import io +import os +import subprocess import sys import unittest -from unittest.mock import patch, Mock, MagicMock - -from datetime import datetime, timedelta, time +from argparse import Namespace +from datetime import datetime, time, timedelta from time import sleep +from unittest.mock import MagicMock, Mock, patch + import psutil import pytz -import subprocess -from argparse import Namespace -from airflow import settings + import airflow.bin.cli as cli -from airflow.bin.cli import get_num_ready_workers_running, run, get_dag +from airflow import models, settings +from airflow.bin.cli import get_dag, get_num_ready_workers_running, run from airflow.models import TaskInstance +from airflow.settings import Session from airflow.utils import timezone from airflow.utils.state import State -from airflow.settings import Session -from airflow import models from tests.compat import mock -import os - dag_folder_path = '/'.join(os.path.realpath(__file__).split('/')[:-1]) DEFAULT_DATE = timezone.make_aware(datetime(2015, 1, 1)) diff --git a/tests/cli/test_worker_initialisation.py b/tests/cli/test_worker_initialisation.py index 5f0e24060336f4..0415081ac2af8c 100644 --- a/tests/cli/test_worker_initialisation.py +++ b/tests/cli/test_worker_initialisation.py @@ -18,14 +18,16 @@ # under the License. import unittest +from argparse import Namespace + import sqlalchemy + import airflow -from argparse import Namespace +from airflow.bin import cli # noqa from tests.compat import mock, patch from tests.test_utils.config import conf_vars patch('airflow.utils.cli.action_logging', lambda x: x).start() -from airflow.bin import cli # noqa mock_args = Namespace(queues=1, concurrency=1) diff --git a/tests/contrib/hooks/test_aws_glue_catalog_hook.py b/tests/contrib/hooks/test_aws_glue_catalog_hook.py index 85b2777e88c51f..e105559e681d48 100644 --- a/tests/contrib/hooks/test_aws_glue_catalog_hook.py +++ b/tests/contrib/hooks/test_aws_glue_catalog_hook.py @@ -19,6 +19,7 @@ import unittest import boto3 + from airflow.contrib.hooks.aws_glue_catalog_hook import AwsGlueCatalogHook from tests.compat import mock diff --git a/tests/contrib/hooks/test_aws_hook.py b/tests/contrib/hooks/test_aws_hook.py index 09f7b4928dc125..959704df0aa41a 100644 --- a/tests/contrib/hooks/test_aws_hook.py +++ b/tests/contrib/hooks/test_aws_hook.py @@ -22,8 +22,8 @@ import boto3 -from airflow.models import Connection from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.models import Connection from tests.compat import mock try: diff --git a/tests/contrib/hooks/test_aws_lambda_hook.py b/tests/contrib/hooks/test_aws_lambda_hook.py index ec97098e5119f3..4147fe44e03a28 100644 --- a/tests/contrib/hooks/test_aws_lambda_hook.py +++ b/tests/contrib/hooks/test_aws_lambda_hook.py @@ -18,10 +18,10 @@ # under the License. # -import unittest import io import json import textwrap +import unittest import zipfile from airflow.contrib.hooks.aws_lambda_hook import AwsLambdaHook diff --git a/tests/contrib/hooks/test_azure_container_instance_hook.py b/tests/contrib/hooks/test_azure_container_instance_hook.py index ba7d8e07422cac..d4b842795a45b6 100644 --- a/tests/contrib/hooks/test_azure_container_instance_hook.py +++ b/tests/contrib/hooks/test_azure_container_instance_hook.py @@ -21,16 +21,14 @@ import unittest from unittest.mock import patch -from airflow.models import Connection +from azure.mgmt.containerinstance.models import ( + Container, ContainerGroup, Logs, ResourceRequests, ResourceRequirements, +) + from airflow.contrib.hooks.azure_container_instance_hook import AzureContainerInstanceHook +from airflow.models import Connection from airflow.utils import db -from azure.mgmt.containerinstance.models import (Container, - ContainerGroup, - Logs, - ResourceRequests, - ResourceRequirements) - class TestAzureContainerInstanceHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_azure_container_registry_hook.py b/tests/contrib/hooks/test_azure_container_registry_hook.py index 9d0c2536409fb4..b521603c2f899f 100644 --- a/tests/contrib/hooks/test_azure_container_registry_hook.py +++ b/tests/contrib/hooks/test_azure_container_registry_hook.py @@ -19,8 +19,8 @@ import unittest -from airflow.models import Connection from airflow.contrib.hooks.azure_container_registry_hook import AzureContainerRegistryHook +from airflow.models import Connection from airflow.utils import db diff --git a/tests/contrib/hooks/test_azure_container_volume_hook.py b/tests/contrib/hooks/test_azure_container_volume_hook.py index 484f0b6def476f..a1c7c7ed40fead 100644 --- a/tests/contrib/hooks/test_azure_container_volume_hook.py +++ b/tests/contrib/hooks/test_azure_container_volume_hook.py @@ -19,8 +19,8 @@ import unittest -from airflow.models import Connection from airflow.contrib.hooks.azure_container_volume_hook import AzureContainerVolumeHook +from airflow.models import Connection from airflow.utils import db diff --git a/tests/contrib/hooks/test_azure_cosmos_hook.py b/tests/contrib/hooks/test_azure_cosmos_hook.py index 08afca3e850eb7..ee0c39a6274648 100644 --- a/tests/contrib/hooks/test_azure_cosmos_hook.py +++ b/tests/contrib/hooks/test_azure_cosmos_hook.py @@ -20,18 +20,16 @@ import json +import logging import unittest import uuid -from airflow.exceptions import AirflowException from airflow.contrib.hooks.azure_cosmos_hook import AzureCosmosDBHook - +from airflow.exceptions import AirflowException from airflow.models import Connection from airflow.utils import db from tests.compat import mock -import logging - class TestAzureCosmosDbHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_cassandra_hook.py b/tests/contrib/hooks/test_cassandra_hook.py index 6d9e823c4f9d45..e2a91c966227c9 100644 --- a/tests/contrib/hooks/test_cassandra_hook.py +++ b/tests/contrib/hooks/test_cassandra_hook.py @@ -19,11 +19,12 @@ import unittest -from airflow.contrib.hooks.cassandra_hook import CassandraHook from cassandra.cluster import Cluster from cassandra.policies import ( - TokenAwarePolicy, RoundRobinPolicy, DCAwareRoundRobinPolicy, WhiteListRoundRobinPolicy + DCAwareRoundRobinPolicy, RoundRobinPolicy, TokenAwarePolicy, WhiteListRoundRobinPolicy, ) + +from airflow.contrib.hooks.cassandra_hook import CassandraHook from airflow.models import Connection from airflow.utils import db from tests.compat import mock, patch diff --git a/tests/contrib/hooks/test_databricks_hook.py b/tests/contrib/hooks/test_databricks_hook.py index 27e42917578a38..8b96622ca21f12 100644 --- a/tests/contrib/hooks/test_databricks_hook.py +++ b/tests/contrib/hooks/test_databricks_hook.py @@ -25,11 +25,7 @@ from requests import exceptions as requests_exceptions from airflow import __version__ -from airflow.contrib.hooks.databricks_hook import ( - DatabricksHook, - RunState, - SUBMIT_RUN_ENDPOINT -) +from airflow.contrib.hooks.databricks_hook import SUBMIT_RUN_ENDPOINT, DatabricksHook, RunState from airflow.exceptions import AirflowException from airflow.models import Connection from airflow.utils import db diff --git a/tests/contrib/hooks/test_datadog_hook.py b/tests/contrib/hooks/test_datadog_hook.py index c035781e7690b0..df6d1bcdc59620 100644 --- a/tests/contrib/hooks/test_datadog_hook.py +++ b/tests/contrib/hooks/test_datadog_hook.py @@ -18,15 +18,13 @@ # under the License. # import json -from unittest import mock import unittest +from unittest import mock +from airflow.contrib.hooks.datadog_hook import DatadogHook from airflow.exceptions import AirflowException from airflow.models import Connection -from airflow.contrib.hooks.datadog_hook import DatadogHook - - APP_KEY = 'app_key' API_KEY = 'api_key' METRIC_NAME = 'metric' diff --git a/tests/contrib/hooks/test_dingding_hook.py b/tests/contrib/hooks/test_dingding_hook.py index a246aacf92a746..57e51f78680fe9 100644 --- a/tests/contrib/hooks/test_dingding_hook.py +++ b/tests/contrib/hooks/test_dingding_hook.py @@ -20,10 +20,9 @@ import json import unittest -from airflow.utils import db - from airflow.contrib.hooks.dingding_hook import DingdingHook from airflow.models import Connection +from airflow.utils import db class TestDingdingHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_discord_webhook_hook.py b/tests/contrib/hooks/test_discord_webhook_hook.py index 346cc8d3042e1e..c70e920c169189 100644 --- a/tests/contrib/hooks/test_discord_webhook_hook.py +++ b/tests/contrib/hooks/test_discord_webhook_hook.py @@ -21,11 +21,10 @@ import unittest from airflow import AirflowException +from airflow.contrib.hooks.discord_webhook_hook import DiscordWebhookHook from airflow.models import Connection from airflow.utils import db -from airflow.contrib.hooks.discord_webhook_hook import DiscordWebhookHook - class TestDiscordWebhookHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_emr_hook.py b/tests/contrib/hooks/test_emr_hook.py index b3c77535d93c82..f02fbc31d6cba6 100644 --- a/tests/contrib/hooks/test_emr_hook.py +++ b/tests/contrib/hooks/test_emr_hook.py @@ -19,6 +19,7 @@ # import unittest + import boto3 from airflow.contrib.hooks.emr_hook import EmrHook diff --git a/tests/contrib/hooks/test_ftp_hook.py b/tests/contrib/hooks/test_ftp_hook.py index df313a0e3bb717..625de52f7d70b9 100644 --- a/tests/contrib/hooks/test_ftp_hook.py +++ b/tests/contrib/hooks/test_ftp_hook.py @@ -18,8 +18,8 @@ # under the License. import io -from unittest import mock import unittest +from unittest import mock from airflow.contrib.hooks import ftp_hook as fh diff --git a/tests/contrib/hooks/test_grpc_hook.py b/tests/contrib/hooks/test_grpc_hook.py index 5da9b681c3ee4b..1c50f909309059 100644 --- a/tests/contrib/hooks/test_grpc_hook.py +++ b/tests/contrib/hooks/test_grpc_hook.py @@ -19,8 +19,8 @@ import unittest from io import StringIO -from airflow.exceptions import AirflowConfigException from airflow.contrib.hooks.grpc_hook import GrpcHook +from airflow.exceptions import AirflowConfigException from airflow.models import Connection from tests.compat import mock diff --git a/tests/contrib/hooks/test_imap_hook.py b/tests/contrib/hooks/test_imap_hook.py index 72fab45067a8b3..7f6496197d71d1 100644 --- a/tests/contrib/hooks/test_imap_hook.py +++ b/tests/contrib/hooks/test_imap_hook.py @@ -19,8 +19,7 @@ import imaplib import unittest - -from unittest.mock import Mock, patch, mock_open +from unittest.mock import Mock, mock_open, patch from airflow import AirflowException from airflow.contrib.hooks.imap_hook import ImapHook diff --git a/tests/contrib/hooks/test_jdbc_hook.py b/tests/contrib/hooks/test_jdbc_hook.py index 80d39ecd32ed92..bf7c0c971af10e 100644 --- a/tests/contrib/hooks/test_jdbc_hook.py +++ b/tests/contrib/hooks/test_jdbc_hook.py @@ -18,9 +18,9 @@ # under the License. # +import json import unittest from unittest.mock import Mock, patch -import json from airflow.hooks.jdbc_hook import JdbcHook from airflow.models import Connection diff --git a/tests/contrib/hooks/test_jira_hook.py b/tests/contrib/hooks/test_jira_hook.py index b4d73b68a89f9f..da5fe8c1ce1473 100644 --- a/tests/contrib/hooks/test_jira_hook.py +++ b/tests/contrib/hooks/test_jira_hook.py @@ -19,9 +19,7 @@ # import unittest - -from unittest.mock import Mock -from unittest.mock import patch +from unittest.mock import Mock, patch from airflow.contrib.hooks.jira_hook import JiraHook from airflow.models import Connection diff --git a/tests/contrib/hooks/test_mongo_hook.py b/tests/contrib/hooks/test_mongo_hook.py index c00ab50362859f..9223c47ddfed5c 100644 --- a/tests/contrib/hooks/test_mongo_hook.py +++ b/tests/contrib/hooks/test_mongo_hook.py @@ -17,16 +17,18 @@ # specific language governing permissions and limitations # under the License. import unittest + import pymongo -try: - import mongomock -except ImportError: - mongomock = None from airflow.contrib.hooks.mongo_hook import MongoHook from airflow.models import Connection from airflow.utils import db +try: + import mongomock +except ImportError: + mongomock = None + class MongoHookTest(MongoHook): """ diff --git a/tests/contrib/hooks/test_openfaas_hook.py b/tests/contrib/hooks/test_openfaas_hook.py index d39710ac41ab90..778d8612c42f14 100644 --- a/tests/contrib/hooks/test_openfaas_hook.py +++ b/tests/contrib/hooks/test_openfaas_hook.py @@ -19,11 +19,13 @@ # import unittest + import requests_mock -from airflow.models import Connection + +from airflow import AirflowException from airflow.contrib.hooks.openfaas_hook import OpenFaasHook from airflow.hooks.base_hook import BaseHook -from airflow import AirflowException +from airflow.models import Connection from tests.compat import mock FUNCTION_NAME = "function_name" diff --git a/tests/contrib/hooks/test_opsgenie_alert_hook.py b/tests/contrib/hooks/test_opsgenie_alert_hook.py index 611885f327b51b..6d0306e41fff15 100644 --- a/tests/contrib/hooks/test_opsgenie_alert_hook.py +++ b/tests/contrib/hooks/test_opsgenie_alert_hook.py @@ -17,14 +17,15 @@ # specific language governing permissions and limitations # under the License. # +import json import unittest + import requests_mock -import json +from airflow import AirflowException +from airflow.contrib.hooks.opsgenie_alert_hook import OpsgenieAlertHook from airflow.models import Connection from airflow.utils import db -from airflow.contrib.hooks.opsgenie_alert_hook import OpsgenieAlertHook -from airflow import AirflowException class TestOpsgenieAlertHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_pinot_hook.py b/tests/contrib/hooks/test_pinot_hook.py index 5e9a5e1d4113d8..72cee01eaddeab 100644 --- a/tests/contrib/hooks/test_pinot_hook.py +++ b/tests/contrib/hooks/test_pinot_hook.py @@ -18,8 +18,8 @@ # under the License. # -from unittest import mock import unittest +from unittest import mock from airflow.contrib.hooks.pinot_hook import PinotDbApiHook diff --git a/tests/contrib/hooks/test_qubole_check_hook.py b/tests/contrib/hooks/test_qubole_check_hook.py index e6cd56b3f05dc4..f9a6d83ffc0db0 100644 --- a/tests/contrib/hooks/test_qubole_check_hook.py +++ b/tests/contrib/hooks/test_qubole_check_hook.py @@ -18,6 +18,7 @@ # under the License. # import unittest + from airflow.contrib.hooks.qubole_check_hook import parse_first_row diff --git a/tests/contrib/hooks/test_redis_hook.py b/tests/contrib/hooks/test_redis_hook.py index e6696b6b47a7d6..954f2bbce3dfc0 100644 --- a/tests/contrib/hooks/test_redis_hook.py +++ b/tests/contrib/hooks/test_redis_hook.py @@ -19,6 +19,7 @@ import unittest + from airflow.contrib.hooks.redis_hook import RedisHook diff --git a/tests/contrib/hooks/test_redshift_hook.py b/tests/contrib/hooks/test_redshift_hook.py index 08c85727d59680..923c954e846355 100644 --- a/tests/contrib/hooks/test_redshift_hook.py +++ b/tests/contrib/hooks/test_redshift_hook.py @@ -19,10 +19,11 @@ # import unittest + import boto3 -from airflow.contrib.hooks.redshift_hook import RedshiftHook from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.hooks.redshift_hook import RedshiftHook try: from moto import mock_redshift diff --git a/tests/contrib/hooks/test_sagemaker_hook.py b/tests/contrib/hooks/test_sagemaker_hook.py index af550f8187f78e..14fb3f785c20cc 100644 --- a/tests/contrib/hooks/test_sagemaker_hook.py +++ b/tests/contrib/hooks/test_sagemaker_hook.py @@ -18,19 +18,20 @@ # under the License. # -import unittest import time +import unittest from datetime import datetime + from tzlocal import get_localzone -from airflow.contrib.hooks.sagemaker_hook import (SageMakerHook, secondary_training_status_changed, - secondary_training_status_message, LogState) from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook -from airflow.hooks.S3_hook import S3Hook +from airflow.contrib.hooks.sagemaker_hook import ( + LogState, SageMakerHook, secondary_training_status_changed, secondary_training_status_message, +) from airflow.exceptions import AirflowException +from airflow.hooks.S3_hook import S3Hook from tests.compat import mock - role = 'arn:aws:iam:role/test-role' path = 'local/data' diff --git a/tests/contrib/hooks/test_salesforce_hook.py b/tests/contrib/hooks/test_salesforce_hook.py index a0067e13904ee6..620385616b4ea1 100644 --- a/tests/contrib/hooks/test_salesforce_hook.py +++ b/tests/contrib/hooks/test_salesforce_hook.py @@ -19,8 +19,7 @@ # import unittest - -from unittest.mock import patch, Mock +from unittest.mock import Mock, patch import pandas as pd from simple_salesforce import Salesforce diff --git a/tests/contrib/hooks/test_segment_hook.py b/tests/contrib/hooks/test_segment_hook.py index 471847898d6998..ca100eaf72c078 100644 --- a/tests/contrib/hooks/test_segment_hook.py +++ b/tests/contrib/hooks/test_segment_hook.py @@ -17,11 +17,10 @@ # specific language governing permissions and limitations # under the License. # -from unittest import mock import unittest +from unittest import mock from airflow import AirflowException - from airflow.contrib.hooks.segment_hook import SegmentHook TEST_CONN_ID = 'test_segment' diff --git a/tests/contrib/hooks/test_sftp_hook.py b/tests/contrib/hooks/test_sftp_hook.py index 4810ffd705ac87..848c1493bb4440 100644 --- a/tests/contrib/hooks/test_sftp_hook.py +++ b/tests/contrib/hooks/test_sftp_hook.py @@ -17,10 +17,11 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock -import unittest -import shutil import os +import shutil +import unittest +from unittest import mock + import pysftp from airflow.contrib.hooks.sftp_hook import SFTPHook diff --git a/tests/contrib/hooks/test_slack_webhook_hook.py b/tests/contrib/hooks/test_slack_webhook_hook.py index 4a5951c9dc8802..132976565f556d 100644 --- a/tests/contrib/hooks/test_slack_webhook_hook.py +++ b/tests/contrib/hooks/test_slack_webhook_hook.py @@ -18,13 +18,14 @@ # under the License. # import json -from requests.exceptions import MissingSchema import unittest from unittest import mock +from requests.exceptions import MissingSchema + +from airflow.contrib.hooks.slack_webhook_hook import SlackWebhookHook from airflow.models import Connection from airflow.utils import db -from airflow.contrib.hooks.slack_webhook_hook import SlackWebhookHook class TestSlackWebhookHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_snowflake_hook.py b/tests/contrib/hooks/test_snowflake_hook.py index f985122dddc7d8..0c29870b21b9f1 100644 --- a/tests/contrib/hooks/test_snowflake_hook.py +++ b/tests/contrib/hooks/test_snowflake_hook.py @@ -18,12 +18,11 @@ # under the License. # import os - -from unittest import mock import unittest +from unittest import mock -from cryptography.hazmat.primitives import serialization from cryptography.hazmat.backends import default_backend +from cryptography.hazmat.primitives import serialization from cryptography.hazmat.primitives.asymmetric import rsa from airflow.contrib.hooks.snowflake_hook import SnowflakeHook diff --git a/tests/contrib/hooks/test_spark_jdbc_hook.py b/tests/contrib/hooks/test_spark_jdbc_hook.py index bb0a7a25ace893..3a10996983bf9d 100644 --- a/tests/contrib/hooks/test_spark_jdbc_hook.py +++ b/tests/contrib/hooks/test_spark_jdbc_hook.py @@ -19,11 +19,10 @@ # import unittest +from airflow.contrib.hooks.spark_jdbc_hook import SparkJDBCHook from airflow.models import Connection from airflow.utils import db -from airflow.contrib.hooks.spark_jdbc_hook import SparkJDBCHook - class TestSparkJDBCHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_spark_sql_hook.py b/tests/contrib/hooks/test_spark_sql_hook.py index e1e2acb39e55fc..0e565af5711d20 100644 --- a/tests/contrib/hooks/test_spark_sql_hook.py +++ b/tests/contrib/hooks/test_spark_sql_hook.py @@ -19,12 +19,12 @@ import io import unittest -from unittest.mock import patch, call from itertools import dropwhile +from unittest.mock import call, patch +from airflow.contrib.hooks.spark_sql_hook import SparkSqlHook from airflow.models import Connection from airflow.utils import db -from airflow.contrib.hooks.spark_sql_hook import SparkSqlHook def get_after(sentinel, iterable): diff --git a/tests/contrib/hooks/test_spark_submit_hook.py b/tests/contrib/hooks/test_spark_submit_hook.py index 136e6daf85b390..b73024cec48b27 100644 --- a/tests/contrib/hooks/test_spark_submit_hook.py +++ b/tests/contrib/hooks/test_spark_submit_hook.py @@ -19,13 +19,12 @@ import io import unittest +from unittest.mock import call, patch from airflow import AirflowException +from airflow.contrib.hooks.spark_submit_hook import SparkSubmitHook from airflow.models import Connection from airflow.utils import db -from unittest.mock import patch, call - -from airflow.contrib.hooks.spark_submit_hook import SparkSubmitHook class TestSparkSubmitHook(unittest.TestCase): diff --git a/tests/contrib/hooks/test_sqoop_hook.py b/tests/contrib/hooks/test_sqoop_hook.py index 66ea0dd8e8b357..05e8c47688a3ae 100644 --- a/tests/contrib/hooks/test_sqoop_hook.py +++ b/tests/contrib/hooks/test_sqoop_hook.py @@ -21,16 +21,14 @@ import collections import json import unittest +from io import StringIO +from unittest.mock import call, patch from airflow.contrib.hooks.sqoop_hook import SqoopHook from airflow.exceptions import AirflowException from airflow.models import Connection from airflow.utils import db -from unittest.mock import patch, call - -from io import StringIO - class TestSqoopHook(unittest.TestCase): _config = { diff --git a/tests/contrib/hooks/test_ssh_hook.py b/tests/contrib/hooks/test_ssh_hook.py index d804de7a666f48..88d58d5b46e20d 100644 --- a/tests/contrib/hooks/test_ssh_hook.py +++ b/tests/contrib/hooks/test_ssh_hook.py @@ -17,13 +17,15 @@ # specific language governing permissions and limitations # under the License. -import unittest import json +import unittest from io import StringIO + import paramiko + +from airflow.contrib.hooks.ssh_hook import SSHHook from airflow.models import Connection from airflow.utils import db -from airflow.contrib.hooks.ssh_hook import SSHHook from airflow.utils.db import create_session from tests.compat import mock diff --git a/tests/contrib/hooks/test_winrm_hook.py b/tests/contrib/hooks/test_winrm_hook.py index 00f82427de2e34..b7fe08461d19f8 100644 --- a/tests/contrib/hooks/test_winrm_hook.py +++ b/tests/contrib/hooks/test_winrm_hook.py @@ -19,7 +19,6 @@ # import unittest - from unittest.mock import patch from airflow import AirflowException diff --git a/tests/contrib/hooks/test_zendesk_hook.py b/tests/contrib/hooks/test_zendesk_hook.py index 81bdf2b60915c3..6bc1d1110b3424 100644 --- a/tests/contrib/hooks/test_zendesk_hook.py +++ b/tests/contrib/hooks/test_zendesk_hook.py @@ -19,12 +19,12 @@ # import unittest - from unittest import mock -from airflow.hooks.zendesk_hook import ZendeskHook from zdesk import RateLimitError +from airflow.hooks.zendesk_hook import ZendeskHook + class TestZendeskHook(unittest.TestCase): diff --git a/tests/contrib/operators/test_aws_athena_operator.py b/tests/contrib/operators/test_aws_athena_operator.py index 9a6124ae04f4a2..bd8b7a5c0dc213 100644 --- a/tests/contrib/operators/test_aws_athena_operator.py +++ b/tests/contrib/operators/test_aws_athena_operator.py @@ -20,8 +20,8 @@ import unittest -from airflow.contrib.operators.aws_athena_operator import AWSAthenaOperator from airflow.contrib.hooks.aws_athena_hook import AWSAthenaHook +from airflow.contrib.operators.aws_athena_operator import AWSAthenaOperator from airflow.models import DAG, TaskInstance from airflow.utils import timezone from airflow.utils.timezone import datetime diff --git a/tests/contrib/operators/test_aws_sqs_publish_operator.py b/tests/contrib/operators/test_aws_sqs_publish_operator.py index db48abbd3c7d10..8af85b593dbdd5 100644 --- a/tests/contrib/operators/test_aws_sqs_publish_operator.py +++ b/tests/contrib/operators/test_aws_sqs_publish_operator.py @@ -19,12 +19,14 @@ import unittest -from airflow import DAG -from airflow.contrib.operators.aws_sqs_publish_operator import SQSPublishOperator -from airflow.utils import timezone from unittest.mock import MagicMock + from moto import mock_sqs + +from airflow import DAG from airflow.contrib.hooks.aws_sqs_hook import SQSHook +from airflow.contrib.operators.aws_sqs_publish_operator import SQSPublishOperator +from airflow.utils import timezone DEFAULT_DATE = timezone.datetime(2019, 1, 1) diff --git a/tests/contrib/operators/test_awsbatch_operator.py b/tests/contrib/operators/test_awsbatch_operator.py index 3135a8eef0cff0..8814857c147369 100644 --- a/tests/contrib/operators/test_awsbatch_operator.py +++ b/tests/contrib/operators/test_awsbatch_operator.py @@ -21,8 +21,8 @@ import sys import unittest -from airflow.exceptions import AirflowException from airflow.contrib.operators.awsbatch_operator import AWSBatchOperator +from airflow.exceptions import AirflowException from tests.compat import mock RESPONSE_WITHOUT_FAILURES = { diff --git a/tests/contrib/operators/test_azure_container_instances_operator.py b/tests/contrib/operators/test_azure_container_instances_operator.py index dc18237192ec7a..56f4f6ff77764d 100644 --- a/tests/contrib/operators/test_azure_container_instances_operator.py +++ b/tests/contrib/operators/test_azure_container_instances_operator.py @@ -18,17 +18,15 @@ # under the License. # +import unittest from collections import namedtuple -from airflow.exceptions import AirflowException -from airflow.contrib.operators.azure_container_instances_operator import AzureContainerInstancesOperator -from azure.mgmt.containerinstance.models import (ContainerState, - Event) +from azure.mgmt.containerinstance.models import ContainerState, Event +from airflow.contrib.operators.azure_container_instances_operator import AzureContainerInstancesOperator +from airflow.exceptions import AirflowException from tests.compat import mock -import unittest - def make_mock_cg(container_state, events=[]): """ diff --git a/tests/contrib/operators/test_azure_cosmos_insertdocument_operator.py b/tests/contrib/operators/test_azure_cosmos_insertdocument_operator.py index f776eb56ad0449..7a4e73dc7c0907 100644 --- a/tests/contrib/operators/test_azure_cosmos_insertdocument_operator.py +++ b/tests/contrib/operators/test_azure_cosmos_insertdocument_operator.py @@ -24,7 +24,6 @@ import uuid from airflow.contrib.operators.azure_cosmos_operator import AzureCosmosInsertDocumentOperator - from airflow.models import Connection from airflow.utils import db from tests.compat import mock diff --git a/tests/contrib/operators/test_databricks_operator.py b/tests/contrib/operators/test_databricks_operator.py index 73fc002f783e6f..78d140684293af 100644 --- a/tests/contrib/operators/test_databricks_operator.py +++ b/tests/contrib/operators/test_databricks_operator.py @@ -21,10 +21,11 @@ import unittest from datetime import datetime -from airflow.contrib.hooks.databricks_hook import RunState import airflow.contrib.operators.databricks_operator as databricks_operator -from airflow.contrib.operators.databricks_operator import DatabricksSubmitRunOperator -from airflow.contrib.operators.databricks_operator import DatabricksRunNowOperator +from airflow.contrib.hooks.databricks_hook import RunState +from airflow.contrib.operators.databricks_operator import ( + DatabricksRunNowOperator, DatabricksSubmitRunOperator, +) from airflow.exceptions import AirflowException from airflow.models import DAG from tests.compat import mock diff --git a/tests/contrib/operators/test_dataproc_operator_system_helper.py b/tests/contrib/operators/test_dataproc_operator_system_helper.py index d67b113aec12ea..d74ce85a7d89a6 100644 --- a/tests/contrib/operators/test_dataproc_operator_system_helper.py +++ b/tests/contrib/operators/test_dataproc_operator_system_helper.py @@ -18,8 +18,9 @@ # specific language governing permissions and limitations # under the License. import os -from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor + from airflow.utils.file import TemporaryDirectory +from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor class DataprocTestHelper(LoggingCommandExecutor): diff --git a/tests/contrib/operators/test_dingding_operator.py b/tests/contrib/operators/test_dingding_operator.py index e1283e4bf5600d..2d35d42e1d738d 100644 --- a/tests/contrib/operators/test_dingding_operator.py +++ b/tests/contrib/operators/test_dingding_operator.py @@ -18,7 +18,6 @@ # under the License. import unittest - from unittest import mock from airflow import DAG diff --git a/tests/contrib/operators/test_discord_webhook_operator.py b/tests/contrib/operators/test_discord_webhook_operator.py index 70077fd5b4d097..2189ec485f36ee 100644 --- a/tests/contrib/operators/test_discord_webhook_operator.py +++ b/tests/contrib/operators/test_discord_webhook_operator.py @@ -20,7 +20,6 @@ import unittest from airflow import DAG - from airflow.contrib.operators.discord_webhook_operator import DiscordWebhookOperator from airflow.utils import timezone diff --git a/tests/contrib/operators/test_druid_operator.py b/tests/contrib/operators/test_druid_operator.py index b9b64f4222131d..9bedca38bbeb74 100644 --- a/tests/contrib/operators/test_druid_operator.py +++ b/tests/contrib/operators/test_druid_operator.py @@ -18,13 +18,13 @@ # under the License. # -from unittest import mock import unittest +from unittest import mock from airflow import DAG from airflow.contrib.operators.druid_operator import DruidOperator -from airflow.utils import timezone from airflow.models import TaskInstance +from airflow.utils import timezone DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/operators/test_dynamodb_to_s3.py b/tests/contrib/operators/test_dynamodb_to_s3.py index f0462a67991d0b..ea09ebaba1ed08 100644 --- a/tests/contrib/operators/test_dynamodb_to_s3.py +++ b/tests/contrib/operators/test_dynamodb_to_s3.py @@ -18,9 +18,9 @@ # under the License. # -from multiprocessing import SimpleQueue import unittest -from unittest.mock import patch, MagicMock +from multiprocessing import SimpleQueue +from unittest.mock import MagicMock, patch from boto.compat import json # type: ignore diff --git a/tests/contrib/operators/test_ecs_operator.py b/tests/contrib/operators/test_ecs_operator.py index 454e694fc3fae1..8fbe58912f41ae 100644 --- a/tests/contrib/operators/test_ecs_operator.py +++ b/tests/contrib/operators/test_ecs_operator.py @@ -24,8 +24,8 @@ from parameterized import parameterized -from airflow.exceptions import AirflowException from airflow.contrib.operators.ecs_operator import ECSOperator +from airflow.exceptions import AirflowException from tests.compat import mock RESPONSE_WITHOUT_FAILURES = { diff --git a/tests/contrib/operators/test_emr_add_steps_operator.py b/tests/contrib/operators/test_emr_add_steps_operator.py index 1395352be76233..f8127a0b7c16c3 100644 --- a/tests/contrib/operators/test_emr_add_steps_operator.py +++ b/tests/contrib/operators/test_emr_add_steps_operator.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest.mock import MagicMock, patch from datetime import timedelta +from unittest.mock import MagicMock, patch from airflow import DAG from airflow.contrib.operators.emr_add_steps_operator import EmrAddStepsOperator diff --git a/tests/contrib/operators/test_emr_create_job_flow_operator.py b/tests/contrib/operators/test_emr_create_job_flow_operator.py index 17911c0003774b..ed5f1a77bb34e1 100644 --- a/tests/contrib/operators/test_emr_create_job_flow_operator.py +++ b/tests/contrib/operators/test_emr_create_job_flow_operator.py @@ -19,8 +19,8 @@ # import unittest -from unittest.mock import MagicMock, patch from datetime import timedelta +from unittest.mock import MagicMock, patch from airflow import DAG from airflow.contrib.operators.emr_create_job_flow_operator import EmrCreateJobFlowOperator diff --git a/tests/contrib/operators/test_gcs_to_gdrive.py b/tests/contrib/operators/test_gcs_to_gdrive.py index 61027d87d691cf..0f4d4cb53cd833 100644 --- a/tests/contrib/operators/test_gcs_to_gdrive.py +++ b/tests/contrib/operators/test_gcs_to_gdrive.py @@ -22,7 +22,6 @@ from airflow import AirflowException from airflow.contrib.operators.gcs_to_gdrive_operator import GcsToGDriveOperator - MODULE = "airflow.contrib.operators.gcs_to_gdrive_operator" diff --git a/tests/contrib/operators/test_grpc_operator.py b/tests/contrib/operators/test_grpc_operator.py index 91648e2dbf521b..da39d6a88203fd 100644 --- a/tests/contrib/operators/test_grpc_operator.py +++ b/tests/contrib/operators/test_grpc_operator.py @@ -19,7 +19,6 @@ import unittest from airflow.contrib.operators.grpc_operator import GrpcOperator - from tests.compat import mock diff --git a/tests/contrib/operators/test_hive_to_dynamodb_operator.py b/tests/contrib/operators/test_hive_to_dynamodb_operator.py index 5b80d321101543..5948e83946cda4 100644 --- a/tests/contrib/operators/test_hive_to_dynamodb_operator.py +++ b/tests/contrib/operators/test_hive_to_dynamodb_operator.py @@ -18,18 +18,17 @@ # under the License. # +import datetime import json import unittest from unittest import mock -import datetime import pandas as pd +import airflow.contrib.operators.hive_to_dynamodb from airflow import DAG from airflow.contrib.hooks.aws_dynamodb_hook import AwsDynamoDBHook -import airflow.contrib.operators.hive_to_dynamodb - DEFAULT_DATE = datetime.datetime(2015, 1, 1) DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat() DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10] diff --git a/tests/contrib/operators/test_imap_attachment_to_s3_operator.py b/tests/contrib/operators/test_imap_attachment_to_s3_operator.py index 85743a39f70cf0..b563c974a74f64 100644 --- a/tests/contrib/operators/test_imap_attachment_to_s3_operator.py +++ b/tests/contrib/operators/test_imap_attachment_to_s3_operator.py @@ -18,7 +18,6 @@ # under the License. import unittest - from unittest.mock import patch from airflow.contrib.operators.imap_attachment_to_s3_operator import ImapAttachmentToS3Operator diff --git a/tests/contrib/operators/test_jenkins_operator.py b/tests/contrib/operators/test_jenkins_operator.py index 2cec0dd18e8a41..e637d649ffcf65 100644 --- a/tests/contrib/operators/test_jenkins_operator.py +++ b/tests/contrib/operators/test_jenkins_operator.py @@ -21,10 +21,8 @@ import jenkins -from airflow.contrib.operators.jenkins_job_trigger_operator \ - import JenkinsJobTriggerOperator from airflow.contrib.hooks.jenkins_hook import JenkinsHook - +from airflow.contrib.operators.jenkins_job_trigger_operator import JenkinsJobTriggerOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_jira_operator_test.py b/tests/contrib/operators/test_jira_operator_test.py index 9e6bd116182cfa..f5d39e31908e25 100644 --- a/tests/contrib/operators/test_jira_operator_test.py +++ b/tests/contrib/operators/test_jira_operator_test.py @@ -19,15 +19,12 @@ # import unittest - -from unittest.mock import Mock -from unittest.mock import patch +from unittest.mock import Mock, patch from airflow import DAG from airflow.contrib.operators.jira_operator import JiraOperator from airflow.models import Connection -from airflow.utils import db -from airflow.utils import timezone +from airflow.utils import db, timezone DEFAULT_DATE = timezone.datetime(2017, 1, 1) jira_client_mock = Mock( diff --git a/tests/contrib/operators/test_opsgenie_alert_operator.py b/tests/contrib/operators/test_opsgenie_alert_operator.py index ed1b6f60e09983..36d8e03beccbe4 100644 --- a/tests/contrib/operators/test_opsgenie_alert_operator.py +++ b/tests/contrib/operators/test_opsgenie_alert_operator.py @@ -21,7 +21,6 @@ import unittest from airflow import DAG - from airflow.contrib.operators.opsgenie_alert_operator import OpsgenieAlertOperator from airflow.utils import timezone diff --git a/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py b/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py index 2844175ab2b777..7702dad0b85137 100644 --- a/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py +++ b/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py @@ -17,14 +17,14 @@ # specific language governing permissions and limitations # under the License. +import os import unittest -from airflow.contrib.operators.oracle_to_azure_data_lake_transfer \ - import OracleToAzureDataLakeTransfer -from airflow.utils.file import TemporaryDirectory + import unicodecsv as csv -import os -from tests.compat import mock, MagicMock +from airflow.contrib.operators.oracle_to_azure_data_lake_transfer import OracleToAzureDataLakeTransfer +from airflow.utils.file import TemporaryDirectory +from tests.compat import MagicMock, mock class TestOracleToAzureDataLakeTransfer(unittest.TestCase): diff --git a/tests/contrib/operators/test_oracle_to_oracle_transfer.py b/tests/contrib/operators/test_oracle_to_oracle_transfer.py index f9d7fc83d86e76..c97af6e8749f3c 100644 --- a/tests/contrib/operators/test_oracle_to_oracle_transfer.py +++ b/tests/contrib/operators/test_oracle_to_oracle_transfer.py @@ -19,8 +19,8 @@ import unittest from unittest import mock -from airflow.contrib.operators.oracle_to_oracle_transfer \ - import OracleToOracleTransfer + +from airflow.contrib.operators.oracle_to_oracle_transfer import OracleToOracleTransfer from tests.compat import MagicMock diff --git a/tests/contrib/operators/test_qubole_check_operator.py b/tests/contrib/operators/test_qubole_check_operator.py index db06ced12f2432..d64879d822c948 100644 --- a/tests/contrib/operators/test_qubole_check_operator.py +++ b/tests/contrib/operators/test_qubole_check_operator.py @@ -19,12 +19,14 @@ # import unittest from datetime import datetime -from airflow.models import DAG -from airflow.exceptions import AirflowException -from airflow.contrib.operators.qubole_check_operator import QuboleValueCheckOperator + +from qds_sdk.commands import HiveCommand + from airflow.contrib.hooks.qubole_check_hook import QuboleCheckHook from airflow.contrib.hooks.qubole_hook import QuboleHook -from qds_sdk.commands import HiveCommand +from airflow.contrib.operators.qubole_check_operator import QuboleValueCheckOperator +from airflow.exceptions import AirflowException +from airflow.models import DAG from tests.compat import mock diff --git a/tests/contrib/operators/test_qubole_operator.py b/tests/contrib/operators/test_qubole_operator.py index a8ab1ee4fce551..0a8f37c32cf85c 100644 --- a/tests/contrib/operators/test_qubole_operator.py +++ b/tests/contrib/operators/test_qubole_operator.py @@ -19,15 +19,14 @@ # import unittest -from airflow.utils.timezone import datetime from airflow import settings -from airflow.models import Connection, DAG -from airflow.models.taskinstance import TaskInstance -from airflow.utils import db - from airflow.contrib.hooks.qubole_hook import QuboleHook from airflow.contrib.operators.qubole_operator import QuboleOperator +from airflow.models import DAG, Connection +from airflow.models.taskinstance import TaskInstance +from airflow.utils import db +from airflow.utils.timezone import datetime DAG_ID = "qubole_test_dag" TASK_ID = "test_task" diff --git a/tests/contrib/operators/test_redis_publish_operator.py b/tests/contrib/operators/test_redis_publish_operator.py index 04820d524e0541..792cd7779e761d 100644 --- a/tests/contrib/operators/test_redis_publish_operator.py +++ b/tests/contrib/operators/test_redis_publish_operator.py @@ -19,11 +19,13 @@ import unittest + +from mock import MagicMock + from airflow import DAG -from airflow.contrib.operators.redis_publish_operator import RedisPublishOperator from airflow.contrib.hooks.redis_hook import RedisHook +from airflow.contrib.operators.redis_publish_operator import RedisPublishOperator from airflow.utils import timezone -from mock import MagicMock DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/operators/test_s3_to_gcs_operator.py b/tests/contrib/operators/test_s3_to_gcs_operator.py index 57b64974815f0d..42821a755fe881 100644 --- a/tests/contrib/operators/test_s3_to_gcs_operator.py +++ b/tests/contrib/operators/test_s3_to_gcs_operator.py @@ -19,8 +19,7 @@ import unittest -from airflow.contrib.operators.s3_to_gcs_operator import \ - S3ToGoogleCloudStorageOperator +from airflow.contrib.operators.s3_to_gcs_operator import S3ToGoogleCloudStorageOperator from tests.compat import mock TASK_ID = 'test-s3-gcs-operator' diff --git a/tests/contrib/operators/test_s3_to_sftp_operator.py b/tests/contrib/operators/test_s3_to_sftp_operator.py index 4d1e2fd5134b98..829e57c4d33f66 100644 --- a/tests/contrib/operators/test_s3_to_sftp_operator.py +++ b/tests/contrib/operators/test_s3_to_sftp_operator.py @@ -19,17 +19,17 @@ import unittest -from airflow.configuration import conf +import boto3 +from moto import mock_s3 + from airflow import models +from airflow.configuration import conf from airflow.contrib.operators.s3_to_sftp_operator import S3ToSFTPOperator from airflow.contrib.operators.ssh_operator import SSHOperator from airflow.models import DAG, TaskInstance from airflow.settings import Session from airflow.utils import timezone from airflow.utils.timezone import datetime -import boto3 -from moto import mock_s3 - TASK_ID = 'test_s3_to_sftp' BUCKET = 'test-s3-bucket' diff --git a/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py b/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py index a294ae04dd6daf..cd1bcbf76fb99a 100644 --- a/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py +++ b/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py @@ -20,8 +20,7 @@ import unittest from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_endpoint_config_operator \ - import SageMakerEndpointConfigOperator +from airflow.contrib.operators.sagemaker_endpoint_config_operator import SageMakerEndpointConfigOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_sagemaker_endpoint_operator.py b/tests/contrib/operators/test_sagemaker_endpoint_operator.py index eb89f06132e25c..cc895941738a50 100644 --- a/tests/contrib/operators/test_sagemaker_endpoint_operator.py +++ b/tests/contrib/operators/test_sagemaker_endpoint_operator.py @@ -20,8 +20,7 @@ import unittest from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_endpoint_operator \ - import SageMakerEndpointOperator +from airflow.contrib.operators.sagemaker_endpoint_operator import SageMakerEndpointOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_sagemaker_model_operator.py b/tests/contrib/operators/test_sagemaker_model_operator.py index bc5fa6537be8c1..252120dda4b2ad 100644 --- a/tests/contrib/operators/test_sagemaker_model_operator.py +++ b/tests/contrib/operators/test_sagemaker_model_operator.py @@ -20,8 +20,7 @@ import unittest from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_model_operator \ - import SageMakerModelOperator +from airflow.contrib.operators.sagemaker_model_operator import SageMakerModelOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_sagemaker_training_operator.py b/tests/contrib/operators/test_sagemaker_training_operator.py index 919b3f1bcac0c2..b385af9c141e0e 100644 --- a/tests/contrib/operators/test_sagemaker_training_operator.py +++ b/tests/contrib/operators/test_sagemaker_training_operator.py @@ -20,8 +20,7 @@ import unittest from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_training_operator \ - import SageMakerTrainingOperator +from airflow.contrib.operators.sagemaker_training_operator import SageMakerTrainingOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_sagemaker_transform_operator.py b/tests/contrib/operators/test_sagemaker_transform_operator.py index 8340c62ac92962..01adcf12b4cc31 100644 --- a/tests/contrib/operators/test_sagemaker_transform_operator.py +++ b/tests/contrib/operators/test_sagemaker_transform_operator.py @@ -20,8 +20,7 @@ import unittest from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_transform_operator \ - import SageMakerTransformOperator +from airflow.contrib.operators.sagemaker_transform_operator import SageMakerTransformOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_sagemaker_tuning_operator.py b/tests/contrib/operators/test_sagemaker_tuning_operator.py index d388d67f5d025b..e5dfe4955fd99c 100644 --- a/tests/contrib/operators/test_sagemaker_tuning_operator.py +++ b/tests/contrib/operators/test_sagemaker_tuning_operator.py @@ -20,8 +20,7 @@ import unittest from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_tuning_operator \ - import SageMakerTuningOperator +from airflow.contrib.operators.sagemaker_tuning_operator import SageMakerTuningOperator from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/operators/test_segment_track_event_operator.py b/tests/contrib/operators/test_segment_track_event_operator.py index 552f79b04add4e..81cb4208116f62 100644 --- a/tests/contrib/operators/test_segment_track_event_operator.py +++ b/tests/contrib/operators/test_segment_track_event_operator.py @@ -17,14 +17,12 @@ # specific language governing permissions and limitations # under the License. # -from unittest import mock import unittest +from unittest import mock from airflow import AirflowException - from airflow.contrib.hooks.segment_hook import SegmentHook -from airflow.contrib.operators.segment_track_event_operator \ - import SegmentTrackEventOperator +from airflow.contrib.operators.segment_track_event_operator import SegmentTrackEventOperator TEST_CONN_ID = 'test_segment' WRITE_KEY = 'foo' diff --git a/tests/contrib/operators/test_sftp_operator.py b/tests/contrib/operators/test_sftp_operator.py index b5aa6526911589..662b2fad1a588e 100644 --- a/tests/contrib/operators/test_sftp_operator.py +++ b/tests/contrib/operators/test_sftp_operator.py @@ -22,7 +22,7 @@ from base64 import b64encode from airflow import models -from airflow.contrib.operators.sftp_operator import SFTPOperator, SFTPOperation +from airflow.contrib.operators.sftp_operator import SFTPOperation, SFTPOperator from airflow.contrib.operators.ssh_operator import SSHOperator from airflow.models import DAG, TaskInstance from airflow.settings import Session diff --git a/tests/contrib/operators/test_sftp_to_s3_operator.py b/tests/contrib/operators/test_sftp_to_s3_operator.py index 00214107f6f6d8..1e444ecb76b005 100644 --- a/tests/contrib/operators/test_sftp_to_s3_operator.py +++ b/tests/contrib/operators/test_sftp_to_s3_operator.py @@ -23,17 +23,16 @@ from moto import mock_s3 from airflow import models +from airflow.contrib.hooks.ssh_hook import SSHHook from airflow.contrib.operators.sftp_to_s3_operator import SFTPToS3Operator from airflow.contrib.operators.ssh_operator import SSHOperator +from airflow.hooks.S3_hook import S3Hook from airflow.models import DAG, TaskInstance from airflow.settings import Session from airflow.utils import timezone from airflow.utils.timezone import datetime -from airflow.contrib.hooks.ssh_hook import SSHHook -from airflow.hooks.S3_hook import S3Hook from tests.test_utils.config import conf_vars - BUCKET = 'test-bucket' S3_KEY = 'test/test_1_file.csv' SFTP_PATH = '/tmp/remote_path.txt' diff --git a/tests/contrib/operators/test_snowflake_operator.py b/tests/contrib/operators/test_snowflake_operator.py index c12e72bc05935c..fa34102198671c 100644 --- a/tests/contrib/operators/test_snowflake_operator.py +++ b/tests/contrib/operators/test_snowflake_operator.py @@ -20,12 +20,10 @@ import unittest from airflow import DAG -from airflow.utils import timezone - from airflow.contrib.operators.snowflake_operator import SnowflakeOperator +from airflow.utils import timezone from tests.compat import mock - DEFAULT_DATE = timezone.datetime(2015, 1, 1) DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat() DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10] diff --git a/tests/contrib/operators/test_sns_publish_operator.py b/tests/contrib/operators/test_sns_publish_operator.py index e204ecdc8ceca5..8920fddefc62a7 100644 --- a/tests/contrib/operators/test_sns_publish_operator.py +++ b/tests/contrib/operators/test_sns_publish_operator.py @@ -18,8 +18,8 @@ # under the License. # -from unittest import mock import unittest +from unittest import mock from airflow.contrib.operators.sns_publish_operator import SnsPublishOperator diff --git a/tests/contrib/operators/test_spark_jdbc_operator.py b/tests/contrib/operators/test_spark_jdbc_operator.py index 339ef368e4ffdc..42be02b3e2c356 100644 --- a/tests/contrib/operators/test_spark_jdbc_operator.py +++ b/tests/contrib/operators/test_spark_jdbc_operator.py @@ -21,7 +21,6 @@ import unittest from airflow import DAG - from airflow.contrib.operators.spark_jdbc_operator import SparkJDBCOperator from airflow.utils import timezone diff --git a/tests/contrib/operators/test_spark_submit_operator.py b/tests/contrib/operators/test_spark_submit_operator.py index 959c64d7df31cf..061681475d8e83 100644 --- a/tests/contrib/operators/test_spark_submit_operator.py +++ b/tests/contrib/operators/test_spark_submit_operator.py @@ -19,16 +19,13 @@ # import unittest +from datetime import timedelta from airflow import DAG -from airflow.models import TaskInstance - from airflow.contrib.operators.spark_submit_operator import SparkSubmitOperator +from airflow.models import TaskInstance from airflow.utils import timezone -from datetime import timedelta - - DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/operators/test_vertica_operator.py b/tests/contrib/operators/test_vertica_operator.py index 79d4127117ded4..be082999f1eb1c 100644 --- a/tests/contrib/operators/test_vertica_operator.py +++ b/tests/contrib/operators/test_vertica_operator.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock from airflow.contrib.operators.vertica_operator import VerticaOperator diff --git a/tests/contrib/operators/test_vertica_to_mysql.py b/tests/contrib/operators/test_vertica_to_mysql.py index 11ae8d79491c4f..5305d2eeb036bd 100644 --- a/tests/contrib/operators/test_vertica_to_mysql.py +++ b/tests/contrib/operators/test_vertica_to_mysql.py @@ -18,9 +18,8 @@ # under the License. import datetime - -from unittest import mock import unittest +from unittest import mock from airflow import DAG from airflow.contrib.operators.vertica_to_mysql import VerticaToMySqlTransfer diff --git a/tests/contrib/operators/test_winrm_operator.py b/tests/contrib/operators/test_winrm_operator.py index 25e2dc273373ea..f9d15fb590d82c 100644 --- a/tests/contrib/operators/test_winrm_operator.py +++ b/tests/contrib/operators/test_winrm_operator.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock from airflow.contrib.operators.winrm_operator import WinRMOperator from airflow.exceptions import AirflowException diff --git a/tests/contrib/sensors/test_athena_sensor.py b/tests/contrib/sensors/test_athena_sensor.py index aa08a839b2e0a7..af25de706841fa 100644 --- a/tests/contrib/sensors/test_athena_sensor.py +++ b/tests/contrib/sensors/test_athena_sensor.py @@ -18,9 +18,10 @@ # under the License. import unittest + from airflow import AirflowException -from airflow.contrib.sensors.aws_athena_sensor import AthenaSensor from airflow.contrib.hooks.aws_athena_hook import AWSAthenaHook +from airflow.contrib.sensors.aws_athena_sensor import AthenaSensor from tests.compat import mock diff --git a/tests/contrib/sensors/test_bash_sensor.py b/tests/contrib/sensors/test_bash_sensor.py index 72fa38469c8fe8..02ab3746d3133a 100644 --- a/tests/contrib/sensors/test_bash_sensor.py +++ b/tests/contrib/sensors/test_bash_sensor.py @@ -18,9 +18,8 @@ # under the License. # -import unittest - import datetime +import unittest from airflow import DAG from airflow.contrib.sensors.bash_sensor import BashSensor diff --git a/tests/contrib/sensors/test_cassandra_sensor.py b/tests/contrib/sensors/test_cassandra_sensor.py index 97132541a78ed5..fcbfb5fe899082 100644 --- a/tests/contrib/sensors/test_cassandra_sensor.py +++ b/tests/contrib/sensors/test_cassandra_sensor.py @@ -19,7 +19,6 @@ import unittest - from unittest.mock import patch from airflow import DAG @@ -27,7 +26,6 @@ from airflow.contrib.sensors.cassandra_table_sensor import CassandraTableSensor from airflow.utils import timezone - DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/sensors/test_emr_job_flow_sensor.py b/tests/contrib/sensors/test_emr_job_flow_sensor.py index 7440dbd00bcfab..b0937ed8477367 100644 --- a/tests/contrib/sensors/test_emr_job_flow_sensor.py +++ b/tests/contrib/sensors/test_emr_job_flow_sensor.py @@ -17,9 +17,10 @@ # specific language governing permissions and limitations # under the License. +import datetime import unittest from unittest.mock import MagicMock, patch -import datetime + from dateutil.tz import tzlocal from airflow import AirflowException diff --git a/tests/contrib/sensors/test_emr_step_sensor.py b/tests/contrib/sensors/test_emr_step_sensor.py index f2a0d63106502d..fa7756f53331e4 100644 --- a/tests/contrib/sensors/test_emr_step_sensor.py +++ b/tests/contrib/sensors/test_emr_step_sensor.py @@ -18,8 +18,9 @@ # under the License. import unittest -from unittest.mock import MagicMock, patch from datetime import datetime +from unittest.mock import MagicMock, patch + from dateutil.tz import tzlocal from airflow import AirflowException diff --git a/tests/contrib/sensors/test_file_sensor.py b/tests/contrib/sensors/test_file_sensor.py index 6d7b3f3d5e0e8f..6adf4920178797 100644 --- a/tests/contrib/sensors/test_file_sensor.py +++ b/tests/contrib/sensors/test_file_sensor.py @@ -18,14 +18,14 @@ # under the License. # -import unittest +import os.path import shutil import tempfile -import os.path +import unittest -from airflow import models, DAG -from airflow.exceptions import AirflowSensorTimeout +from airflow import DAG, models from airflow.contrib.sensors.file_sensor import FileSensor +from airflow.exceptions import AirflowSensorTimeout from airflow.settings import Session from airflow.utils.timezone import datetime diff --git a/tests/contrib/sensors/test_ftp_sensor.py b/tests/contrib/sensors/test_ftp_sensor.py index cbd7951beb7247..62fbbd15e68813 100644 --- a/tests/contrib/sensors/test_ftp_sensor.py +++ b/tests/contrib/sensors/test_ftp_sensor.py @@ -18,9 +18,8 @@ # under the License. import unittest -from unittest import mock - from ftplib import error_perm +from unittest import mock from airflow.contrib.hooks.ftp_hook import FTPHook from airflow.contrib.sensors.ftp_sensor import FTPSensor diff --git a/tests/contrib/sensors/test_hdfs_sensor.py b/tests/contrib/sensors/test_hdfs_sensor.py index f7f7b55d09b158..6b9d726248b713 100644 --- a/tests/contrib/sensors/test_hdfs_sensor.py +++ b/tests/contrib/sensors/test_hdfs_sensor.py @@ -17,9 +17,8 @@ # specific language governing permissions and limitations # under the License. import logging -import unittest - import re +import unittest from datetime import timedelta from airflow.contrib.sensors.hdfs_sensor import HdfsSensorFolder, HdfsSensorRegex diff --git a/tests/contrib/sensors/test_imap_attachment_sensor.py b/tests/contrib/sensors/test_imap_attachment_sensor.py index 3c5ab37df4bbf5..93401b27a70598 100644 --- a/tests/contrib/sensors/test_imap_attachment_sensor.py +++ b/tests/contrib/sensors/test_imap_attachment_sensor.py @@ -18,7 +18,7 @@ # under the License. import unittest -from unittest.mock import patch, Mock +from unittest.mock import Mock, patch from parameterized import parameterized diff --git a/tests/contrib/sensors/test_jira_sensor_test.py b/tests/contrib/sensors/test_jira_sensor_test.py index 04acc0ab510adc..e61ea63ea24e2f 100644 --- a/tests/contrib/sensors/test_jira_sensor_test.py +++ b/tests/contrib/sensors/test_jira_sensor_test.py @@ -19,9 +19,7 @@ # import unittest - -from unittest.mock import Mock -from unittest.mock import patch +from unittest.mock import Mock, patch from airflow import DAG from airflow.contrib.sensors.jira_sensor import JiraTicketSensor diff --git a/tests/contrib/sensors/test_mongo_sensor.py b/tests/contrib/sensors/test_mongo_sensor.py index b7cb85634c0166..3ee75cd88b2f2d 100644 --- a/tests/contrib/sensors/test_mongo_sensor.py +++ b/tests/contrib/sensors/test_mongo_sensor.py @@ -26,7 +26,6 @@ from airflow.models import Connection from airflow.utils import db, timezone - DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/sensors/test_python_sensor.py b/tests/contrib/sensors/test_python_sensor.py index 18b26a8461832a..21d068fac98043 100644 --- a/tests/contrib/sensors/test_python_sensor.py +++ b/tests/contrib/sensors/test_python_sensor.py @@ -25,7 +25,6 @@ from airflow.exceptions import AirflowSensorTimeout from airflow.utils.timezone import datetime - DEFAULT_DATE = datetime(2015, 1, 1) TEST_DAG_ID = 'python_sensor_dag' diff --git a/tests/contrib/sensors/test_qubole_sensor.py b/tests/contrib/sensors/test_qubole_sensor.py index 2b1c4c51e3b7db..17828d56422d90 100644 --- a/tests/contrib/sensors/test_qubole_sensor.py +++ b/tests/contrib/sensors/test_qubole_sensor.py @@ -19,13 +19,12 @@ # import unittest -from unittest.mock import patch - from datetime import datetime +from unittest.mock import patch from airflow.contrib.sensors.qubole_sensor import QuboleFileSensor, QubolePartitionSensor from airflow.exceptions import AirflowException -from airflow.models import Connection, DAG +from airflow.models import DAG, Connection from airflow.utils import db DAG_ID = "qubole_test_dag" diff --git a/tests/contrib/sensors/test_redis_pub_sub_sensor.py b/tests/contrib/sensors/test_redis_pub_sub_sensor.py index 9a065831528ca9..1792f61ed7aa6a 100644 --- a/tests/contrib/sensors/test_redis_pub_sub_sensor.py +++ b/tests/contrib/sensors/test_redis_pub_sub_sensor.py @@ -19,11 +19,12 @@ import unittest +from unittest.mock import MagicMock, call, patch + from airflow import DAG +from airflow.contrib.hooks.redis_hook import RedisHook from airflow.contrib.sensors.redis_pub_sub_sensor import RedisPubSubSensor from airflow.utils import timezone -from airflow.contrib.hooks.redis_hook import RedisHook -from unittest.mock import patch, call, MagicMock DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/sensors/test_redis_sensor.py b/tests/contrib/sensors/test_redis_sensor.py index 82ab2cbd58f1f0..cd16ceafa8867b 100644 --- a/tests/contrib/sensors/test_redis_sensor.py +++ b/tests/contrib/sensors/test_redis_sensor.py @@ -19,6 +19,7 @@ import unittest + from airflow import DAG from airflow.contrib.hooks.redis_hook import RedisHook from airflow.contrib.sensors.redis_key_sensor import RedisKeySensor diff --git a/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py b/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py index 0c1ec3ed937af8..65800da219da56 100644 --- a/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py @@ -19,9 +19,8 @@ import unittest -from airflow.contrib.sensors.sagemaker_endpoint_sensor \ - import SageMakerEndpointSensor from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_endpoint_sensor import SageMakerEndpointSensor from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/sensors/test_sagemaker_training_sensor.py b/tests/contrib/sensors/test_sagemaker_training_sensor.py index b6c5af5943a458..c380e42bc406b4 100644 --- a/tests/contrib/sensors/test_sagemaker_training_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_training_sensor.py @@ -20,10 +20,9 @@ import unittest from datetime import datetime -from airflow.contrib.sensors.sagemaker_training_sensor \ - import SageMakerTrainingSensor -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook, LogState from airflow.contrib.hooks.aws_logs_hook import AwsLogsHook +from airflow.contrib.hooks.sagemaker_hook import LogState, SageMakerHook +from airflow.contrib.sensors.sagemaker_training_sensor import SageMakerTrainingSensor from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/sensors/test_sagemaker_transform_sensor.py b/tests/contrib/sensors/test_sagemaker_transform_sensor.py index d238df5c6c0d06..e26bf708e31195 100644 --- a/tests/contrib/sensors/test_sagemaker_transform_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_transform_sensor.py @@ -19,9 +19,8 @@ import unittest -from airflow.contrib.sensors.sagemaker_transform_sensor \ - import SageMakerTransformSensor from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_transform_sensor import SageMakerTransformSensor from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py index 696654fa37173e..be6cbc55331670 100644 --- a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py @@ -19,9 +19,8 @@ import unittest -from airflow.contrib.sensors.sagemaker_tuning_sensor \ - import SageMakerTuningSensor from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_tuning_sensor import SageMakerTuningSensor from airflow.exceptions import AirflowException from tests.compat import mock diff --git a/tests/contrib/sensors/test_sftp_sensor.py b/tests/contrib/sensors/test_sftp_sensor.py index 9ed9dc658efbbe..eba8ef5e62cb20 100644 --- a/tests/contrib/sensors/test_sftp_sensor.py +++ b/tests/contrib/sensors/test_sftp_sensor.py @@ -19,7 +19,9 @@ import unittest from unittest.mock import patch -from paramiko import SFTP_NO_SUCH_FILE, SFTP_FAILURE + +from paramiko import SFTP_FAILURE, SFTP_NO_SUCH_FILE + from airflow.contrib.sensors.sftp_sensor import SFTPSensor diff --git a/tests/contrib/sensors/test_sqs_sensor.py b/tests/contrib/sensors/test_sqs_sensor.py index 04e50ae7b54591..ca391b1dc044ff 100644 --- a/tests/contrib/sensors/test_sqs_sensor.py +++ b/tests/contrib/sensors/test_sqs_sensor.py @@ -19,13 +19,15 @@ import unittest +from unittest.mock import MagicMock, patch + +from moto import mock_sqs + from airflow import DAG +from airflow.contrib.hooks.aws_sqs_hook import SQSHook from airflow.contrib.sensors.aws_sqs_sensor import SQSSensor -from airflow.utils import timezone -from unittest.mock import patch, MagicMock from airflow.exceptions import AirflowException -from moto import mock_sqs -from airflow.contrib.hooks.aws_sqs_hook import SQSHook +from airflow.utils import timezone DEFAULT_DATE = timezone.datetime(2017, 1, 1) diff --git a/tests/contrib/sensors/test_wasb_sensor.py b/tests/contrib/sensors/test_wasb_sensor.py index fb16ef044d5977..9fa35bd4e4d8aa 100644 --- a/tests/contrib/sensors/test_wasb_sensor.py +++ b/tests/contrib/sensors/test_wasb_sensor.py @@ -18,13 +18,11 @@ # under the License. # -import unittest - import datetime +import unittest from airflow import DAG -from airflow.contrib.sensors.wasb_sensor import WasbBlobSensor -from airflow.contrib.sensors.wasb_sensor import WasbPrefixSensor +from airflow.contrib.sensors.wasb_sensor import WasbBlobSensor, WasbPrefixSensor from tests.compat import mock diff --git a/tests/contrib/sensors/test_weekday_sensor.py b/tests/contrib/sensors/test_weekday_sensor.py index 536725008d7d80..1a8b5d2f7c1dff 100644 --- a/tests/contrib/sensors/test_weekday_sensor.py +++ b/tests/contrib/sensors/test_weekday_sensor.py @@ -19,6 +19,7 @@ # import unittest + from airflow import DAG, models from airflow.contrib.sensors.weekday_sensor import DayOfWeekSensor from airflow.contrib.utils.weekday import WeekDay diff --git a/tests/contrib/task_runner/test_cgroup_task_runner.py b/tests/contrib/task_runner/test_cgroup_task_runner.py index 7adbb2f4d9d289..957065848dcffe 100644 --- a/tests/contrib/task_runner/test_cgroup_task_runner.py +++ b/tests/contrib/task_runner/test_cgroup_task_runner.py @@ -16,8 +16,8 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock from airflow.contrib.task_runner.cgroup_task_runner import CgroupTaskRunner diff --git a/tests/contrib/utils/logging_command_executor.py b/tests/contrib/utils/logging_command_executor.py index 615b92dfc07040..f2d7f3f63325c3 100644 --- a/tests/contrib/utils/logging_command_executor.py +++ b/tests/contrib/utils/logging_command_executor.py @@ -19,7 +19,7 @@ import os import subprocess -from airflow import LoggingMixin, AirflowException +from airflow import AirflowException, LoggingMixin class LoggingCommandExecutor(LoggingMixin): diff --git a/tests/contrib/utils/test_sendgrid.py b/tests/contrib/utils/test_sendgrid.py index 3561548341e113..bc76b8048c91c5 100644 --- a/tests/contrib/utils/test_sendgrid.py +++ b/tests/contrib/utils/test_sendgrid.py @@ -19,9 +19,9 @@ # import copy -import unittest -import tempfile import os +import tempfile +import unittest from airflow.contrib.utils.sendgrid import send_email from tests.compat import mock diff --git a/tests/contrib/utils/test_task_handler_with_custom_formatter.py b/tests/contrib/utils/test_task_handler_with_custom_formatter.py index e69c434395462d..95d6c857ae387f 100644 --- a/tests/contrib/utils/test_task_handler_with_custom_formatter.py +++ b/tests/contrib/utils/test_task_handler_with_custom_formatter.py @@ -20,12 +20,12 @@ import logging import unittest -from airflow.models import TaskInstance, DAG from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG +from airflow.configuration import conf +from airflow.models import DAG, TaskInstance from airflow.operators.dummy_operator import DummyOperator -from airflow.utils.timezone import datetime from airflow.utils.log.logging_mixin import set_context -from airflow.configuration import conf +from airflow.utils.timezone import datetime DEFAULT_DATE = datetime(2019, 1, 1) TASK_LOGGER = 'airflow.task' diff --git a/tests/contrib/utils/test_weekday.py b/tests/contrib/utils/test_weekday.py index 906eec52005ed3..9b4007425f8621 100644 --- a/tests/contrib/utils/test_weekday.py +++ b/tests/contrib/utils/test_weekday.py @@ -17,7 +17,6 @@ # specific language governing permissions and limitations # under the License. import unittest - from enum import Enum from airflow.contrib.utils.weekday import WeekDay diff --git a/tests/core.py b/tests/core.py index 50de527745cdd4..58f036ecc9f9ce 100644 --- a/tests/core.py +++ b/tests/core.py @@ -17,7 +17,6 @@ # specific language governing permissions and limitations # under the License. -from typing import Optional import io import json import multiprocessing @@ -35,6 +34,7 @@ from email.mime.text import MIMEText from tempfile import NamedTemporaryFile from time import sleep +from typing import Optional from unittest import mock import sqlalchemy @@ -42,24 +42,16 @@ from numpy.testing import assert_array_almost_equal from pendulum import utcnow -from airflow import configuration, models -from airflow import jobs, DAG, utils, settings, exceptions +from airflow import DAG, configuration, exceptions, jobs, models, settings, utils from airflow.bin import cli -from airflow.configuration import AirflowConfigException, run_command, conf +from airflow.configuration import AirflowConfigException, conf, run_command from airflow.exceptions import AirflowException from airflow.executors import SequentialExecutor +from airflow.hooks import hdfs_hook from airflow.hooks.base_hook import BaseHook from airflow.hooks.sqlite_hook import SqliteHook from airflow.models import ( - BaseOperator, - Connection, - TaskFail, - DagBag, - DagRun, - Pool, - DagModel, - TaskInstance, - Variable, + BaseOperator, Connection, DagBag, DagModel, DagRun, Pool, TaskFail, TaskInstance, Variable, ) from airflow.operators.bash_operator import BashOperator from airflow.operators.check_operator import CheckOperator, ValueCheckOperator @@ -68,13 +60,9 @@ from airflow.operators.python_operator import PythonOperator from airflow.settings import Session from airflow.utils import timezone -from airflow.utils.dates import ( - days_ago, infer_time_unit, round_time, - scale_time_units -) +from airflow.utils.dates import days_ago, infer_time_unit, round_time, scale_time_units from airflow.utils.state import State from airflow.utils.timezone import datetime -from airflow.hooks import hdfs_hook from tests.test_utils.config import conf_vars DEV_NULL = '/dev/null' diff --git a/tests/dags/test_dag_serialization.py b/tests/dags/test_dag_serialization.py index 1934c642a4232b..c175ddf79c9a95 100644 --- a/tests/dags/test_dag_serialization.py +++ b/tests/dags/test_dag_serialization.py @@ -22,15 +22,15 @@ import json import multiprocessing import unittest -from unittest import mock from datetime import datetime +from unittest import mock from airflow import example_dags from airflow.contrib import example_dags as contrib_example_dags from airflow.dag.serialization import Serialization, SerializedBaseOperator, SerializedDAG from airflow.dag.serialization.enum import Encoding from airflow.hooks.base_hook import BaseHook -from airflow.models import BaseOperator, Connection, DAG, DagBag +from airflow.models import DAG, BaseOperator, Connection, DagBag from airflow.operators.bash_operator import BashOperator # airflow/example_dags/ diff --git a/tests/dags/test_impersonation_custom.py b/tests/dags/test_impersonation_custom.py index 35c104dca08c6c..2d73046f6f5dde 100644 --- a/tests/dags/test_impersonation_custom.py +++ b/tests/dags/test_impersonation_custom.py @@ -19,8 +19,6 @@ from datetime import datetime -from airflow.models import DAG -from airflow.operators.python_operator import PythonOperator # AIRFLOW-1893 - Originally, impersonation tests were incomplete missing the use case when # DAGs access custom packages usually made available through the PYTHONPATH environment # variable. This file includes a DAG that imports a custom package made available and if @@ -30,6 +28,9 @@ # variable correctly. from fake_datetime import FakeDatetime +from airflow.models import DAG +from airflow.operators.python_operator import PythonOperator + DEFAULT_DATE = datetime(2016, 1, 1) args = { diff --git a/tests/dags/test_impersonation_subdag.py b/tests/dags/test_impersonation_subdag.py index c3862a2469a47e..e7da2da01b3cce 100644 --- a/tests/dags/test_impersonation_subdag.py +++ b/tests/dags/test_impersonation_subdag.py @@ -20,10 +20,9 @@ from datetime import datetime from airflow.models import DAG +from airflow.operators.bash_operator import BashOperator from airflow.operators.python_operator import PythonOperator from airflow.operators.subdag_operator import SubDagOperator -from airflow.operators.bash_operator import BashOperator - DEFAULT_DATE = datetime(2016, 1, 1) diff --git a/tests/dags/test_scheduler_dags.py b/tests/dags/test_scheduler_dags.py index 94e6f8216d226d..ec42a335608b99 100644 --- a/tests/dags/test_scheduler_dags.py +++ b/tests/dags/test_scheduler_dags.py @@ -21,6 +21,7 @@ from airflow.models import DAG from airflow.operators.dummy_operator import DummyOperator + DEFAULT_DATE = datetime(2016, 1, 1) # DAG tests backfill with pooled tasks diff --git a/tests/dags/test_subdag.py b/tests/dags/test_subdag.py index dab224d0fc3ddc..52bc9ec81c2b4d 100644 --- a/tests/dags/test_subdag.py +++ b/tests/dags/test_subdag.py @@ -22,7 +22,7 @@ A DAG with subdag for testing purpose. """ -from datetime import timedelta, datetime +from datetime import datetime, timedelta from airflow.models import DAG from airflow.operators.dummy_operator import DummyOperator diff --git a/tests/dags/test_task_view_type_check.py b/tests/dags/test_task_view_type_check.py index 4080117476fb27..7bad77ea96d30e 100644 --- a/tests/dags/test_task_view_type_check.py +++ b/tests/dags/test_task_view_type_check.py @@ -23,7 +23,6 @@ """ import functools import logging - from datetime import datetime from airflow.models import DAG diff --git a/tests/dags_with_system_exit/a_system_exit.py b/tests/dags_with_system_exit/a_system_exit.py index 8d51ddcbb833e4..67aa5ec42170f2 100644 --- a/tests/dags_with_system_exit/a_system_exit.py +++ b/tests/dags_with_system_exit/a_system_exit.py @@ -21,8 +21,8 @@ # Starts with 'a' to get listed first. import sys - from datetime import datetime + from airflow.models import DAG DEFAULT_DATE = datetime(2100, 1, 1) diff --git a/tests/dags_with_system_exit/c_system_exit.py b/tests/dags_with_system_exit/c_system_exit.py index bebaa3e8f39456..39721c186c7879 100644 --- a/tests/dags_with_system_exit/c_system_exit.py +++ b/tests/dags_with_system_exit/c_system_exit.py @@ -21,8 +21,8 @@ # Start with 'z' to get listed last. import sys - from datetime import datetime + from airflow.models import DAG DEFAULT_DATE = datetime(2100, 1, 1) diff --git a/tests/executors/test_base_executor.py b/tests/executors/test_base_executor.py index dba3bafeaf4714..3509634edcbb3c 100644 --- a/tests/executors/test_base_executor.py +++ b/tests/executors/test_base_executor.py @@ -18,13 +18,12 @@ # under the License. import unittest +from datetime import datetime from unittest import mock from airflow.executors.base_executor import BaseExecutor from airflow.utils.state import State -from datetime import datetime - class TestBaseExecutor(unittest.TestCase): def test_get_event_buffer(self): diff --git a/tests/executors/test_celery_executor.py b/tests/executors/test_celery_executor.py index f86ff25e395381..bf5f6b4714e022 100644 --- a/tests/executors/test_celery_executor.py +++ b/tests/executors/test_celery_executor.py @@ -16,27 +16,23 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import contextlib import os import sys import unittest -from unittest import mock -import contextlib from multiprocessing import Pool +from unittest import mock - -from celery import Celery -from celery import states as celery_states +# leave this it is used by the test worker +import celery.contrib.testing.tasks # noqa: F401 pylint: disable=ungrouped-imports +from celery import Celery, states as celery_states from celery.contrib.testing.worker import start_worker from kombu.asynchronous import set_event_loop from parameterized import parameterized -from airflow.utils.state import State -from airflow.executors import celery_executor - from airflow.configuration import conf - -# leave this it is used by the test worker -import celery.contrib.testing.tasks # noqa: F401 pylint: disable=ungrouped-imports +from airflow.executors import celery_executor +from airflow.utils.state import State def _prepare_test_bodies(): diff --git a/tests/executors/test_dask_executor.py b/tests/executors/test_dask_executor.py index 08f1ce373b2110..b9d8c0105910fe 100644 --- a/tests/executors/test_dask_executor.py +++ b/tests/executors/test_dask_executor.py @@ -18,15 +18,14 @@ # under the License. import unittest +from datetime import timedelta from unittest import mock from airflow.configuration import conf -from airflow.models import DagBag from airflow.jobs import BackfillJob +from airflow.models import DagBag from airflow.utils import timezone -from datetime import timedelta - try: from airflow.executors.dask_executor import DaskExecutor from distributed import LocalCluster diff --git a/tests/executors/test_executor.py b/tests/executors/test_executor.py index aac3feea0412e3..f501e56d3f6bac 100644 --- a/tests/executors/test_executor.py +++ b/tests/executors/test_executor.py @@ -20,8 +20,8 @@ from collections import defaultdict from airflow.executors.base_executor import BaseExecutor -from airflow.utils.state import State from airflow.utils.db import create_session +from airflow.utils.state import State class TestExecutor(BaseExecutor): diff --git a/tests/executors/test_kubernetes_executor.py b/tests/executors/test_kubernetes_executor.py index 1d523a6611bcc6..b7db93a27506eb 100644 --- a/tests/executors/test_kubernetes_executor.py +++ b/tests/executors/test_kubernetes_executor.py @@ -16,15 +16,17 @@ # under the License. # -from datetime import datetime -import unittest +import random import re import string -import random +import unittest +from datetime import datetime + from urllib3 import HTTPResponse from tests.compat import mock from tests.test_utils.config import conf_vars + try: from kubernetes.client.rest import ApiException from airflow.executors.kubernetes_executor import AirflowKubernetesScheduler diff --git a/tests/gcp/hooks/test_automl.py b/tests/gcp/hooks/test_automl.py index e8c25746545b26..174ac8f2a9a1b0 100644 --- a/tests/gcp/hooks/test_automl.py +++ b/tests/gcp/hooks/test_automl.py @@ -22,9 +22,8 @@ from google.cloud.automl_v1beta1 import AutoMlClient, PredictionServiceClient from airflow.gcp.hooks.automl import CloudAutoMLHook -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id from tests.compat import mock - +from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id CREDENTIALS = "test-creds" CLIENT_INFO = "client-info" diff --git a/tests/gcp/hooks/test_base.py b/tests/gcp/hooks/test_base.py index 424dcdd7a13553..f0865dba9ed879 100644 --- a/tests/gcp/hooks/test_base.py +++ b/tests/gcp/hooks/test_base.py @@ -22,14 +22,13 @@ import unittest from io import StringIO -from parameterized import parameterized - import google.auth +from google.api_core.exceptions import AlreadyExists, RetryError from google.auth.environment_vars import CREDENTIALS from google.auth.exceptions import GoogleAuthError -from google.api_core.exceptions import RetryError, AlreadyExists from google.cloud.exceptions import MovedPermanently from googleapiclient.errors import HttpError +from parameterized import parameterized from airflow import AirflowException, LoggingMixin from airflow.gcp.hooks import base as hook diff --git a/tests/gcp/hooks/test_bigquery.py b/tests/gcp/hooks/test_bigquery.py index 3f19da1ed2a290..cbf84c14ec35f7 100644 --- a/tests/gcp/hooks/test_bigquery.py +++ b/tests/gcp/hooks/test_bigquery.py @@ -19,16 +19,17 @@ # import unittest -from unittest import mock from typing import List, Optional +from unittest import mock from google.auth.exceptions import GoogleAuthError from googleapiclient.errors import HttpError from airflow.gcp.hooks import bigquery as hook -from airflow.gcp.hooks.bigquery import _cleanse_time_partitioning, \ - _validate_value, _api_resource_configs_duplication_check, \ - _validate_src_fmt_configs +from airflow.gcp.hooks.bigquery import ( + _api_resource_configs_duplication_check, _cleanse_time_partitioning, _validate_src_fmt_configs, + _validate_value, +) bq_available = True diff --git a/tests/gcp/hooks/test_bigquery_dts.py b/tests/gcp/hooks/test_bigquery_dts.py index 4609538c426dff..ba828783a51646 100644 --- a/tests/gcp/hooks/test_bigquery_dts.py +++ b/tests/gcp/hooks/test_bigquery_dts.py @@ -20,17 +20,15 @@ import unittest from copy import deepcopy -from google.protobuf.json_format import ParseDict from google.cloud.bigquery_datatransfer_v1 import DataTransferServiceClient from google.cloud.bigquery_datatransfer_v1.types import TransferConfig +from google.protobuf.json_format import ParseDict +from airflow.gcp.hooks.bigquery_dts import BiqQueryDataTransferServiceHook +from airflow.version import version from tests.compat import mock from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id -from airflow.version import version -from airflow.gcp.hooks.bigquery_dts import BiqQueryDataTransferServiceHook - - CREDENTIALS = "test-creds" PROJECT_ID = "id" diff --git a/tests/gcp/hooks/test_bigtable.py b/tests/gcp/hooks/test_bigtable.py index c4b790cccb8fba..0edd8c44ddcc66 100644 --- a/tests/gcp/hooks/test_bigtable.py +++ b/tests/gcp/hooks/test_bigtable.py @@ -20,16 +20,16 @@ import unittest import google - from google.cloud.bigtable import Client from google.cloud.bigtable.instance import Instance -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id, \ - mock_base_gcp_hook_default_project_id, GCP_PROJECT_ID_HOOK_UNIT_TEST -from tests.compat import mock, PropertyMock - from airflow import AirflowException from airflow.gcp.hooks.bigtable import BigtableHook +from tests.compat import PropertyMock, mock +from tests.gcp.utils.base_gcp_mock import ( + GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id, + mock_base_gcp_hook_no_default_project_id, +) CBT_INSTANCE = 'instance' CBT_CLUSTER = 'cluster' diff --git a/tests/gcp/hooks/test_cloud_build.py b/tests/gcp/hooks/test_cloud_build.py index 95884f3ba43a41..d9e17199a028fa 100644 --- a/tests/gcp/hooks/test_cloud_build.py +++ b/tests/gcp/hooks/test_cloud_build.py @@ -19,18 +19,17 @@ """ Tests for Google Cloud Build Hook """ -from typing import Optional import unittest +from typing import Optional from unittest import mock from airflow import AirflowException from airflow.gcp.hooks.cloud_build import CloudBuildHook from tests.compat import PropertyMock from tests.gcp.utils.base_gcp_mock import ( - mock_base_gcp_hook_default_project_id, + GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id, mock_base_gcp_hook_no_default_project_id, - GCP_PROJECT_ID_HOOK_UNIT_TEST) - +) TEST_CREATE_BODY = { "source": {"storageSource": {"bucket": "cloud-build-examples", "object": "node-docker-example.tar.gz"}}, diff --git a/tests/gcp/hooks/test_cloud_memorystore.py b/tests/gcp/hooks/test_cloud_memorystore.py index 6d0fefb7da027c..d3017bfc68930a 100644 --- a/tests/gcp/hooks/test_cloud_memorystore.py +++ b/tests/gcp/hooks/test_cloud_memorystore.py @@ -23,16 +23,14 @@ from google.cloud.exceptions import NotFound from google.cloud.redis_v1.types import Instance +from airflow import AirflowException, version +from airflow.gcp.hooks.cloud_memorystore import CloudMemorystoreHook from tests.compat import PropertyMock from tests.gcp.utils.base_gcp_mock import ( - GCP_PROJECT_ID_HOOK_UNIT_TEST, - mock_base_gcp_hook_default_project_id, + GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id, mock_base_gcp_hook_no_default_project_id, ) -from airflow import AirflowException, version -from airflow.gcp.hooks.cloud_memorystore import CloudMemorystoreHook - TEST_GCP_CONN_ID = "test-gcp-conn-id" # type: str TEST_DELEGATE_TO = "test-delegate-to" # type: str TEST_LOCATION = "test-location" # type: str diff --git a/tests/gcp/hooks/test_cloud_sql.py b/tests/gcp/hooks/test_cloud_sql.py index 9a7d006540eb1a..b423ef7b4cc9f3 100644 --- a/tests/gcp/hooks/test_cloud_sql.py +++ b/tests/gcp/hooks/test_cloud_sql.py @@ -23,15 +23,15 @@ import unittest from googleapiclient.errors import HttpError - from parameterized import parameterized -from airflow.gcp.hooks.cloud_sql import CloudSqlHook, CloudSqlDatabaseHook from airflow.exceptions import AirflowException +from airflow.gcp.hooks.cloud_sql import CloudSqlDatabaseHook, CloudSqlHook from airflow.models import Connection -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id, \ - mock_base_gcp_hook_no_default_project_id -from tests.compat import mock, PropertyMock +from tests.compat import PropertyMock, mock +from tests.gcp.utils.base_gcp_mock import ( + mock_base_gcp_hook_default_project_id, mock_base_gcp_hook_no_default_project_id, +) class TestGcpSqlHookDefaultProjectId(unittest.TestCase): diff --git a/tests/gcp/hooks/test_cloud_storage_transfer_service.py b/tests/gcp/hooks/test_cloud_storage_transfer_service.py index b3b79934baa863..f707377adaa266 100644 --- a/tests/gcp/hooks/test_cloud_storage_transfer_service.py +++ b/tests/gcp/hooks/test_cloud_storage_transfer_service.py @@ -25,26 +25,15 @@ from airflow import AirflowException from airflow.gcp.hooks.cloud_storage_transfer_service import ( - GCPTransferServiceHook, - TIME_TO_SLEEP_IN_SECONDS, - GcpTransferOperationStatus, - GcpTransferJobsStatus, - PROJECT_ID, - STATUS, - DESCRIPTION, - FILTER_PROJECT_ID, - FILTER_JOB_NAMES, - TRANSFER_JOBS, - OPERATIONS, - METADATA, - TRANSFER_JOB, - TRANSFER_JOB_FIELD_MASK, + DESCRIPTION, FILTER_JOB_NAMES, FILTER_PROJECT_ID, METADATA, OPERATIONS, PROJECT_ID, STATUS, + TIME_TO_SLEEP_IN_SECONDS, TRANSFER_JOB, TRANSFER_JOB_FIELD_MASK, TRANSFER_JOBS, GcpTransferJobsStatus, + GcpTransferOperationStatus, GCPTransferServiceHook, ) +from tests.compat import PropertyMock, mock from tests.gcp.utils.base_gcp_mock import ( + GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id, mock_base_gcp_hook_no_default_project_id, - mock_base_gcp_hook_default_project_id, - GCP_PROJECT_ID_HOOK_UNIT_TEST) -from tests.compat import mock, PropertyMock +) NAME = "name" diff --git a/tests/gcp/hooks/test_dataflow.py b/tests/gcp/hooks/test_dataflow.py index e5df8e971642e3..b1a30f1efe6463 100644 --- a/tests/gcp/hooks/test_dataflow.py +++ b/tests/gcp/hooks/test_dataflow.py @@ -23,12 +23,7 @@ from parameterized import parameterized -from airflow.gcp.hooks.dataflow import ( - DataFlowHook, - _Dataflow, - _DataflowJob, - DataflowJobStatus -) +from airflow.gcp.hooks.dataflow import DataFlowHook, DataflowJobStatus, _Dataflow, _DataflowJob from tests.compat import MagicMock, mock TASK_ID = 'test-dataflow-operator' diff --git a/tests/gcp/hooks/test_dataproc.py b/tests/gcp/hooks/test_dataproc.py index ffcffe9d160750..ff0d12d0c6e3c0 100644 --- a/tests/gcp/hooks/test_dataproc.py +++ b/tests/gcp/hooks/test_dataproc.py @@ -19,12 +19,12 @@ # import unittest + from mock import MagicMock -from airflow.gcp.hooks.dataproc import _DataProcJob -from airflow.gcp.hooks.dataproc import DataProcHook -from tests.gcp.utils.base_gcp_mock import GCP_PROJECT_ID_HOOK_UNIT_TEST +from airflow.gcp.hooks.dataproc import DataProcHook, _DataProcJob from tests.compat import mock +from tests.gcp.utils.base_gcp_mock import GCP_PROJECT_ID_HOOK_UNIT_TEST JOB = 'test-job' GCP_REGION = 'global' diff --git a/tests/gcp/hooks/test_datastore.py b/tests/gcp/hooks/test_datastore.py index 45b77b2bdaee57..293e5b86f1ee3c 100644 --- a/tests/gcp/hooks/test_datastore.py +++ b/tests/gcp/hooks/test_datastore.py @@ -25,7 +25,6 @@ from airflow.gcp.hooks.datastore import DatastoreHook from tests.compat import mock - GCP_PROJECT_ID = "test" diff --git a/tests/gcp/hooks/test_dlp.py b/tests/gcp/hooks/test_dlp.py index 8cea4b1be691b2..7a709b2564d007 100644 --- a/tests/gcp/hooks/test_dlp.py +++ b/tests/gcp/hooks/test_dlp.py @@ -25,11 +25,12 @@ import unittest from typing import Any, Dict + from google.cloud.dlp_v2.types import DlpJob from airflow import AirflowException from airflow.gcp.hooks.dlp import CloudDLPHook -from tests.compat import mock, PropertyMock +from tests.compat import PropertyMock, mock from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id API_RESPONSE = {} # type: Dict[Any, Any] diff --git a/tests/gcp/hooks/test_functions.py b/tests/gcp/hooks/test_functions.py index 30f81902f837eb..e67322c38b61f4 100644 --- a/tests/gcp/hooks/test_functions.py +++ b/tests/gcp/hooks/test_functions.py @@ -21,10 +21,11 @@ from airflow import AirflowException from airflow.gcp.hooks.functions import GcfHook - -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id, \ - mock_base_gcp_hook_default_project_id, GCP_PROJECT_ID_HOOK_UNIT_TEST, get_open_mock -from tests.compat import mock, PropertyMock +from tests.compat import PropertyMock, mock +from tests.gcp.utils.base_gcp_mock import ( + GCP_PROJECT_ID_HOOK_UNIT_TEST, get_open_mock, mock_base_gcp_hook_default_project_id, + mock_base_gcp_hook_no_default_project_id, +) GCF_LOCATION = 'location' GCF_FUNCTION = 'function' diff --git a/tests/gcp/hooks/test_gcp_compute_hook.py b/tests/gcp/hooks/test_gcp_compute_hook.py index dfff2c6c093f30..01c2dc53904f50 100644 --- a/tests/gcp/hooks/test_gcp_compute_hook.py +++ b/tests/gcp/hooks/test_gcp_compute_hook.py @@ -21,12 +21,13 @@ import unittest -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id, \ - mock_base_gcp_hook_default_project_id, GCP_PROJECT_ID_HOOK_UNIT_TEST -from tests.compat import mock, PropertyMock - from airflow import AirflowException from airflow.gcp.hooks.compute import GceHook, GceOperationStatus +from tests.compat import PropertyMock, mock +from tests.gcp.utils.base_gcp_mock import ( + GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id, + mock_base_gcp_hook_no_default_project_id, +) GCE_ZONE = 'zone' GCE_INSTANCE = 'instance' diff --git a/tests/gcp/hooks/test_gcs.py b/tests/gcp/hooks/test_gcs.py index 797d8fbe11e29c..bdb7cacd18a343 100644 --- a/tests/gcp/hooks/test_gcs.py +++ b/tests/gcp/hooks/test_gcs.py @@ -17,20 +17,19 @@ # specific language governing permissions and limitations # under the License. # pylint: disable=too-many-lines +import copy +import gzip as gz import io import os import tempfile import unittest -import copy from datetime import datetime -import gzip as gz import dateutil -from google.cloud import storage -from google.cloud import exceptions +from google.cloud import exceptions, storage -from airflow.gcp.hooks import gcs from airflow.exceptions import AirflowException +from airflow.gcp.hooks import gcs from airflow.version import version from tests.compat import mock from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id diff --git a/tests/gcp/hooks/test_google_discovery_api.py b/tests/gcp/hooks/test_google_discovery_api.py index c15c7b49470126..6117656ca5162d 100644 --- a/tests/gcp/hooks/test_google_discovery_api.py +++ b/tests/gcp/hooks/test_google_discovery_api.py @@ -18,7 +18,7 @@ # under the License. # import unittest -from unittest.mock import patch, call +from unittest.mock import call, patch from airflow import configuration, models from airflow.gcp.hooks.discovery_api import GoogleDiscoveryApiHook diff --git a/tests/gcp/hooks/test_gsheets.py b/tests/gcp/hooks/test_gsheets.py index e26b9b42a11bb9..0069226c73ca85 100644 --- a/tests/gcp/hooks/test_gsheets.py +++ b/tests/gcp/hooks/test_gsheets.py @@ -23,12 +23,11 @@ import unittest +from airflow import AirflowException +from airflow.gcp.hooks.gsheets import GSheetsHook from tests.compat import mock from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id -from airflow.gcp.hooks.gsheets import GSheetsHook -from airflow import AirflowException - GCP_CONN_ID = 'test' SPREADHSEET_ID = '1234567890' RANGE_ = 'test!A:E' diff --git a/tests/gcp/hooks/test_kms.py b/tests/gcp/hooks/test_kms.py index b099bc8392175d..d9992bd7a070a1 100644 --- a/tests/gcp/hooks/test_kms.py +++ b/tests/gcp/hooks/test_kms.py @@ -18,13 +18,12 @@ # under the License. import unittest -from base64 import b64encode, b64decode +from base64 import b64decode, b64encode from collections import namedtuple from airflow.gcp.hooks.kms import GoogleCloudKMSHook from tests.compat import mock - Response = namedtuple("Response", ["plaintext", "ciphertext"]) PLAINTEXT = b"Test plaintext" diff --git a/tests/gcp/hooks/test_kubernetes_engine.py b/tests/gcp/hooks/test_kubernetes_engine.py index 4c7ccde3ff58e6..cb26b41f9b27b0 100644 --- a/tests/gcp/hooks/test_kubernetes_engine.py +++ b/tests/gcp/hooks/test_kubernetes_engine.py @@ -23,7 +23,7 @@ from airflow import AirflowException from airflow.gcp.hooks.kubernetes_engine import GKEClusterHook -from tests.compat import mock, PropertyMock +from tests.compat import PropertyMock, mock TASK_ID = 'test-gke-cluster-operator' CLUSTER_NAME = 'test-cluster' diff --git a/tests/gcp/hooks/test_mlengine.py b/tests/gcp/hooks/test_mlengine.py index c1f0bc239fa50f..aed895a9f50d3a 100644 --- a/tests/gcp/hooks/test_mlengine.py +++ b/tests/gcp/hooks/test_mlengine.py @@ -15,13 +15,12 @@ # specific language governing permissions and limitations # under the License. +import json import unittest from unittest import mock -from urllib.parse import urlparse, parse_qsl +from urllib.parse import parse_qsl, urlparse -import json import requests - from google.auth.exceptions import GoogleAuthError from googleapiclient.discovery import build_from_document from googleapiclient.errors import HttpError diff --git a/tests/gcp/hooks/test_natural_language.py b/tests/gcp/hooks/test_natural_language.py index b8b924536a5e0c..b1917dc0f5dde5 100644 --- a/tests/gcp/hooks/test_natural_language.py +++ b/tests/gcp/hooks/test_natural_language.py @@ -18,7 +18,7 @@ # under the License. # import unittest -from typing import Dict, Any +from typing import Any, Dict from google.cloud.language_v1.proto.language_service_pb2 import Document @@ -26,7 +26,6 @@ from tests.compat import mock from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id - API_RESPONSE = {} # type: Dict[Any, Any] DOCUMENT = Document( content="Airflow is a platform to programmatically author, schedule and monitor workflows." diff --git a/tests/gcp/hooks/test_pubsub.py b/tests/gcp/hooks/test_pubsub.py index e4b985423319ec..48da1e70be013a 100644 --- a/tests/gcp/hooks/test_pubsub.py +++ b/tests/gcp/hooks/test_pubsub.py @@ -24,8 +24,8 @@ from googleapiclient.errors import HttpError from parameterized import parameterized -from airflow.version import version from airflow.gcp.hooks.pubsub import PubSubException, PubSubHook +from airflow.version import version from tests.compat import mock BASE_STRING = 'airflow.gcp.hooks.base.{}' diff --git a/tests/gcp/hooks/test_spanner.py b/tests/gcp/hooks/test_spanner.py index 7746c5d1cde62d..25762078ad1e18 100644 --- a/tests/gcp/hooks/test_spanner.py +++ b/tests/gcp/hooks/test_spanner.py @@ -21,9 +21,11 @@ from airflow import AirflowException from airflow.gcp.hooks.spanner import CloudSpannerHook -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id, \ - GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id -from tests.compat import mock, PropertyMock +from tests.compat import PropertyMock, mock +from tests.gcp.utils.base_gcp_mock import ( + GCP_PROJECT_ID_HOOK_UNIT_TEST, mock_base_gcp_hook_default_project_id, + mock_base_gcp_hook_no_default_project_id, +) SPANNER_INSTANCE = 'instance' SPANNER_CONFIGURATION = 'configuration' diff --git a/tests/gcp/hooks/test_speech_to_text.py b/tests/gcp/hooks/test_speech_to_text.py index fbd833e58afc53..bc15f7d603b0fa 100644 --- a/tests/gcp/hooks/test_speech_to_text.py +++ b/tests/gcp/hooks/test_speech_to_text.py @@ -21,7 +21,7 @@ import unittest from airflow.gcp.hooks.speech_to_text import GCPSpeechToTextHook -from tests.compat import patch, PropertyMock +from tests.compat import PropertyMock, patch from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id PROJECT_ID = "project-id" diff --git a/tests/gcp/hooks/test_tasks.py b/tests/gcp/hooks/test_tasks.py index df332598c36f3d..b409921c228c00 100644 --- a/tests/gcp/hooks/test_tasks.py +++ b/tests/gcp/hooks/test_tasks.py @@ -18,7 +18,7 @@ # under the License. # import unittest -from typing import Dict, Any +from typing import Any, Dict from google.cloud.tasks_v2.types import Queue, Task @@ -26,7 +26,6 @@ from tests.compat import mock from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_no_default_project_id - API_RESPONSE = {} # type: Dict[Any, Any] PROJECT_ID = "test-project" LOCATION = "asia-east2" diff --git a/tests/gcp/hooks/test_text_to_speech.py b/tests/gcp/hooks/test_text_to_speech.py index 61350c3583406f..2fd2dc2d8a7caf 100644 --- a/tests/gcp/hooks/test_text_to_speech.py +++ b/tests/gcp/hooks/test_text_to_speech.py @@ -21,7 +21,7 @@ import unittest from airflow.gcp.hooks.text_to_speech import GCPTextToSpeechHook -from tests.compat import patch, PropertyMock +from tests.compat import PropertyMock, patch from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id INPUT = {"text": "test text"} diff --git a/tests/gcp/hooks/test_translate.py b/tests/gcp/hooks/test_translate.py index 32de80fb506486..3be5f709fa7326 100644 --- a/tests/gcp/hooks/test_translate.py +++ b/tests/gcp/hooks/test_translate.py @@ -20,8 +20,8 @@ import unittest from airflow.gcp.hooks.translate import CloudTranslateHook -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id from tests.compat import mock +from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id PROJECT_ID_TEST = 'project-id' diff --git a/tests/gcp/hooks/test_video_intelligence.py b/tests/gcp/hooks/test_video_intelligence.py index c730ba6fc0d381..d56bff8e9ac253 100644 --- a/tests/gcp/hooks/test_video_intelligence.py +++ b/tests/gcp/hooks/test_video_intelligence.py @@ -21,10 +21,9 @@ from google.cloud.videointelligence_v1 import enums -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id -from tests.compat import mock - from airflow.gcp.hooks.video_intelligence import CloudVideoIntelligenceHook +from tests.compat import mock +from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id INPUT_URI = "gs://bucket-name/input-file" OUTPUT_URI = "gs://bucket-name/output-file" diff --git a/tests/gcp/hooks/test_vision.py b/tests/gcp/hooks/test_vision.py index f2ecf58dddc068..b7b7b6de8a16d2 100644 --- a/tests/gcp/hooks/test_vision.py +++ b/tests/gcp/hooks/test_vision.py @@ -20,19 +20,17 @@ from google.cloud.vision import enums from google.cloud.vision_v1 import ProductSearchClient -from google.cloud.vision_v1.proto.product_search_service_pb2 import ProductSet, Product, ReferenceImage from google.cloud.vision_v1.proto.image_annotator_pb2 import ( - AnnotateImageResponse, - EntityAnnotation, - SafeSearchAnnotation, + AnnotateImageResponse, EntityAnnotation, SafeSearchAnnotation, ) +from google.cloud.vision_v1.proto.product_search_service_pb2 import Product, ProductSet, ReferenceImage from google.protobuf.json_format import MessageToDict from parameterized import parameterized from airflow import AirflowException -from airflow.gcp.hooks.vision import CloudVisionHook, ERR_DIFF_NAMES, ERR_UNABLE_TO_CREATE -from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id +from airflow.gcp.hooks.vision import ERR_DIFF_NAMES, ERR_UNABLE_TO_CREATE, CloudVisionHook from tests.compat import mock +from tests.gcp.utils.base_gcp_mock import mock_base_gcp_hook_default_project_id PROJECT_ID_TEST = 'project-id' PROJECT_ID_TEST_2 = 'project-id-2' diff --git a/tests/gcp/operators/test_automl.py b/tests/gcp/operators/test_automl.py index 002992d8bf5045..60c143f5fcd411 100644 --- a/tests/gcp/operators/test_automl.py +++ b/tests/gcp/operators/test_automl.py @@ -17,25 +17,16 @@ # specific language governing permissions and limitations # under the License. # -import unittest import copy +import unittest from google.cloud.automl_v1beta1 import AutoMlClient, PredictionServiceClient from airflow.gcp.operators.automl import ( - AutoMLTrainModelOperator, - AutoMLPredictOperator, - AutoMLBatchPredictOperator, - AutoMLCreateDatasetOperator, - AutoMLImportDataOperator, - AutoMLGetModelOperator, - AutoMLTablesListColumnSpecsOperator, - AutoMLDeployModelOperator, - AutoMLTablesUpdateDatasetOperator, - AutoMLTablesListTableSpecsOperator, - AutoMLListDatasetOperator, - AutoMLDeleteDatasetOperator, - AutoMLDeleteModelOperator, + AutoMLBatchPredictOperator, AutoMLCreateDatasetOperator, AutoMLDeleteDatasetOperator, + AutoMLDeleteModelOperator, AutoMLDeployModelOperator, AutoMLGetModelOperator, AutoMLImportDataOperator, + AutoMLListDatasetOperator, AutoMLPredictOperator, AutoMLTablesListColumnSpecsOperator, + AutoMLTablesListTableSpecsOperator, AutoMLTablesUpdateDatasetOperator, AutoMLTrainModelOperator, ) from tests.compat import mock diff --git a/tests/gcp/operators/test_automl_system.py b/tests/gcp/operators/test_automl_system.py index c338ca17733e91..a556df7a20d8da 100644 --- a/tests/gcp/operators/test_automl_system.py +++ b/tests/gcp/operators/test_automl_system.py @@ -19,9 +19,7 @@ import unittest from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - SKIP_LONG_TEST_WARNING, - TestDagGcpSystem, + SKIP_LONG_TEST_WARNING, SKIP_TEST_WARNING, TestDagGcpSystem, ) from tests.gcp.utils.gcp_authenticator import GCP_AUTOML_KEY diff --git a/tests/gcp/operators/test_bigquery.py b/tests/gcp/operators/test_bigquery.py index b14c380623b57f..0a7ebfccd21f5f 100644 --- a/tests/gcp/operators/test_bigquery.py +++ b/tests/gcp/operators/test_bigquery.py @@ -18,24 +18,17 @@ # under the License. import unittest -from unittest.mock import MagicMock from datetime import datetime +from unittest.mock import MagicMock from airflow import models -from airflow.gcp.operators.bigquery import ( - BigQueryCreateExternalTableOperator, - BigQueryCreateEmptyTableOperator, - BigQueryDeleteDatasetOperator, - BigQueryCreateEmptyDatasetOperator, - BigQueryOperator, - BigQueryConsoleLink, - BigQueryGetDatasetOperator, - BigQueryPatchDatasetOperator, - BigQueryUpdateDatasetOperator, - BigQueryTableDeleteOperator, - BigQueryGetDataOperator, - BigQueryGetDatasetTablesOperator) from airflow.exceptions import AirflowException +from airflow.gcp.operators.bigquery import ( + BigQueryConsoleLink, BigQueryCreateEmptyDatasetOperator, BigQueryCreateEmptyTableOperator, + BigQueryCreateExternalTableOperator, BigQueryDeleteDatasetOperator, BigQueryGetDataOperator, + BigQueryGetDatasetOperator, BigQueryGetDatasetTablesOperator, BigQueryOperator, + BigQueryPatchDatasetOperator, BigQueryTableDeleteOperator, BigQueryUpdateDatasetOperator, +) from airflow.models import DAG, TaskFail, TaskInstance, XCom from airflow.settings import Session from airflow.utils.db import provide_session diff --git a/tests/gcp/operators/test_bigquery_dts.py b/tests/gcp/operators/test_bigquery_dts.py index ed2858dc40c886..fbd392102199e1 100644 --- a/tests/gcp/operators/test_bigquery_dts.py +++ b/tests/gcp/operators/test_bigquery_dts.py @@ -18,14 +18,11 @@ # under the License. import unittest -from tests.compat import mock - from airflow.gcp.operators.bigquery_dts import ( - BigQueryCreateDataTransferOperator, + BigQueryCreateDataTransferOperator, BigQueryDataTransferServiceStartTransferRunsOperator, BigQueryDeleteDataTransferConfigOperator, - BigQueryDataTransferServiceStartTransferRunsOperator, ) - +from tests.compat import mock PROJECT_ID = "id" diff --git a/tests/gcp/operators/test_bigquery_dts_system.py b/tests/gcp/operators/test_bigquery_dts_system.py index 5478d75b05e70d..35ad6d921104b5 100644 --- a/tests/gcp/operators/test_bigquery_dts_system.py +++ b/tests/gcp/operators/test_bigquery_dts_system.py @@ -19,21 +19,12 @@ import unittest -from tests.gcp.operators.test_bigquery_dts_system_helper import ( - GcpBigqueryDtsTestHelper, -) -from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - TestDagGcpSystem, -) -from tests.gcp.utils.gcp_authenticator import GCP_BIGQUERY_KEY - from airflow.gcp.example_dags.example_bigquery_dts import ( - GCP_PROJECT_ID, - GCP_DTS_BQ_DATASET, - GCP_DTS_BQ_TABLE, - BUCKET_URI + BUCKET_URI, GCP_DTS_BQ_DATASET, GCP_DTS_BQ_TABLE, GCP_PROJECT_ID, ) +from tests.gcp.operators.test_bigquery_dts_system_helper import GcpBigqueryDtsTestHelper +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem +from tests.gcp.utils.gcp_authenticator import GCP_BIGQUERY_KEY @unittest.skipIf(TestDagGcpSystem.skip_check(GCP_BIGQUERY_KEY), SKIP_TEST_WARNING) diff --git a/tests/gcp/operators/test_bigquery_system_helper.py b/tests/gcp/operators/test_bigquery_system_helper.py index 2f6f3f8211bfbf..851c82cf8d6a12 100755 --- a/tests/gcp/operators/test_bigquery_system_helper.py +++ b/tests/gcp/operators/test_bigquery_system_helper.py @@ -23,8 +23,8 @@ import argparse from airflow.gcp.example_dags.example_bigquery import DATA_EXPORT_BUCKET_NAME -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_BIGQUERY_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_BIGQUERY_KEY, GcpAuthenticator class GCPBigQueryTestHelper(LoggingCommandExecutor): diff --git a/tests/gcp/operators/test_bigtable.py b/tests/gcp/operators/test_bigtable.py index f5bd3432aa4cb4..beba05245ca7be 100644 --- a/tests/gcp/operators/test_bigtable.py +++ b/tests/gcp/operators/test_bigtable.py @@ -18,19 +18,18 @@ # under the License. import unittest -from typing import List, Dict +from typing import Dict, List + import google.api_core.exceptions from google.cloud.bigtable.column_family import MaxVersionsGCRule from google.cloud.bigtable.instance import Instance from parameterized import parameterized from airflow import AirflowException -from airflow.gcp.operators.bigtable import \ - BigtableInstanceDeleteOperator, \ - BigtableTableDeleteOperator, \ - BigtableTableCreateOperator, \ - BigtableClusterUpdateOperator, \ - BigtableInstanceCreateOperator +from airflow.gcp.operators.bigtable import ( + BigtableClusterUpdateOperator, BigtableInstanceCreateOperator, BigtableInstanceDeleteOperator, + BigtableTableCreateOperator, BigtableTableDeleteOperator, +) from tests.compat import mock PROJECT_ID = 'test_project_id' diff --git a/tests/gcp/operators/test_bigtable_system.py b/tests/gcp/operators/test_bigtable_system.py index 45ea841833bb38..4a15071a7debf6 100644 --- a/tests/gcp/operators/test_bigtable_system.py +++ b/tests/gcp/operators/test_bigtable_system.py @@ -18,8 +18,7 @@ # under the License. import unittest -from tests.gcp.operators.test_bigtable_system_helper import \ - GCPBigtableTestHelper +from tests.gcp.operators.test_bigtable_system_helper import GCPBigtableTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_BIGTABLE_KEY diff --git a/tests/gcp/operators/test_bigtable_system_helper.py b/tests/gcp/operators/test_bigtable_system_helper.py index 52755f95b6f4b0..38a6f9769ca9f0 100755 --- a/tests/gcp/operators/test_bigtable_system_helper.py +++ b/tests/gcp/operators/test_bigtable_system_helper.py @@ -17,12 +17,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import os - import argparse +import os -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_SPANNER_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_SPANNER_KEY, GcpAuthenticator GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') CBT_INSTANCE = os.environ.get('CBT_INSTANCE_ID', 'testinstance') diff --git a/tests/gcp/operators/test_cloud_build.py b/tests/gcp/operators/test_cloud_build.py index 9067ee8682dca7..5ae3246984c4ef 100644 --- a/tests/gcp/operators/test_cloud_build.py +++ b/tests/gcp/operators/test_cloud_build.py @@ -22,11 +22,10 @@ from unittest import TestCase from parameterized import parameterized -from tests.compat import mock from airflow import AirflowException from airflow.gcp.operators.cloud_build import BuildProcessor, CloudBuildCreateBuildOperator - +from tests.compat import mock TEST_CREATE_BODY = { "source": {"storageSource": {"bucket": "cloud-build-examples", "object": "node-docker-example.tar.gz"}}, diff --git a/tests/gcp/operators/test_cloud_build_system_helper.py b/tests/gcp/operators/test_cloud_build_system_helper.py index 37b0fea23ba63a..e7a075fd856d52 100755 --- a/tests/gcp/operators/test_cloud_build_system_helper.py +++ b/tests/gcp/operators/test_cloud_build_system_helper.py @@ -25,8 +25,8 @@ from urllib.parse import urlparse from airflow.utils.file import TemporaryDirectory -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_CLOUD_BUILD_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_CLOUD_BUILD_KEY, GcpAuthenticator GCE_INSTANCE = os.environ.get("GCE_INSTANCE", "testinstance") GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") diff --git a/tests/gcp/operators/test_cloud_memorystore.py b/tests/gcp/operators/test_cloud_memorystore.py index 91930dc31b61d5..4f3eb7358b18b9 100644 --- a/tests/gcp/operators/test_cloud_memorystore.py +++ b/tests/gcp/operators/test_cloud_memorystore.py @@ -24,19 +24,13 @@ from google.cloud.redis_v1.types import Instance from airflow.gcp.operators.cloud_memorystore import ( - CloudMemorystoreCreateInstanceOperator, - CloudMemorystoreDeleteInstanceOperator, - CloudMemorystoreExportInstanceOperator, - CloudMemorystoreFailoverInstanceOperator, - CloudMemorystoreGetInstanceOperator, - CloudMemorystoreImportOperator, - CloudMemorystoreListInstancesOperator, - CloudMemorystoreUpdateInstanceOperator, - CloudMemorystoreScaleInstanceOperator, - CloudMemorystoreCreateInstanceAndImportOperator, + CloudMemorystoreCreateInstanceAndImportOperator, CloudMemorystoreCreateInstanceOperator, + CloudMemorystoreDeleteInstanceOperator, CloudMemorystoreExportInstanceOperator, + CloudMemorystoreFailoverInstanceOperator, CloudMemorystoreGetInstanceOperator, + CloudMemorystoreImportOperator, CloudMemorystoreListInstancesOperator, + CloudMemorystoreScaleInstanceOperator, CloudMemorystoreUpdateInstanceOperator, ) - TEST_GCP_CONN_ID = "test-gcp-conn-id" TEST_TASK_ID = "task-id" TEST_LOCATION = "test-location" diff --git a/tests/gcp/operators/test_cloud_memorystore_system_helper.py b/tests/gcp/operators/test_cloud_memorystore_system_helper.py index ea54b9370e967d..6739a7e4290637 100755 --- a/tests/gcp/operators/test_cloud_memorystore_system_helper.py +++ b/tests/gcp/operators/test_cloud_memorystore_system_helper.py @@ -24,8 +24,8 @@ import os from urllib.parse import urlparse -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_GCS_TRANSFER_KEY # TODO: Update it from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_GCS_TRANSFER_KEY, GcpAuthenticator # TODO: Update it GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") GCP_ARCHIVE_URL = os.environ.get("GCP_MEMORYSTORE_EXPORT_GCS_URL", "gs://test-memorystore/my-export.rdb") diff --git a/tests/gcp/operators/test_cloud_sql.py b/tests/gcp/operators/test_cloud_sql.py index 4f88f1e0f90709..5a2e3f239ed2fb 100644 --- a/tests/gcp/operators/test_cloud_sql.py +++ b/tests/gcp/operators/test_cloud_sql.py @@ -26,11 +26,12 @@ from parameterized import parameterized from airflow import AirflowException -from airflow.gcp.operators.cloud_sql import CloudSqlInstanceCreateOperator, \ - CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator, \ - CloudSqlInstanceDatabaseCreateOperator, CloudSqlInstanceDatabasePatchOperator, \ - CloudSqlInstanceExportOperator, CloudSqlInstanceImportOperator, \ - CloudSqlInstanceDatabaseDeleteOperator, CloudSqlQueryOperator +from airflow.gcp.operators.cloud_sql import ( + CloudSqlInstanceCreateOperator, CloudSqlInstanceDatabaseCreateOperator, + CloudSqlInstanceDatabaseDeleteOperator, CloudSqlInstanceDatabasePatchOperator, + CloudSqlInstanceDeleteOperator, CloudSqlInstanceExportOperator, CloudSqlInstanceImportOperator, + CloudSqlInstancePatchOperator, CloudSqlQueryOperator, +) from airflow.models import Connection from tests.compat import mock diff --git a/tests/gcp/operators/test_cloud_sql_operatorquery_system.py b/tests/gcp/operators/test_cloud_sql_operatorquery_system.py index 5f0cda0a33a81a..47e7c720f3c9ec 100644 --- a/tests/gcp/operators/test_cloud_sql_operatorquery_system.py +++ b/tests/gcp/operators/test_cloud_sql_operatorquery_system.py @@ -17,18 +17,16 @@ # specific language governing permissions and limitations # under the License. import os -from os.path import dirname import random import string -import unittest - import time +import unittest +from os.path import dirname from airflow import AirflowException from airflow.gcp.hooks.cloud_sql import CloudSqlProxyRunner +from tests.gcp.operators.test_cloud_sql_system_helper import CloudSqlQueryTestHelper from tests.gcp.utils.base_gcp_system_test_case import TestBaseGcpSystem, TestDagGcpSystem -from tests.gcp.operators.test_cloud_sql_system_helper import \ - CloudSqlQueryTestHelper from tests.gcp.utils.gcp_authenticator import GCP_CLOUDSQL_KEY GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'project-id') diff --git a/tests/gcp/operators/test_cloud_sql_operatorquery_system_helper.py b/tests/gcp/operators/test_cloud_sql_operatorquery_system_helper.py index 7007d2416c7f94..6498d3228f79ab 100755 --- a/tests/gcp/operators/test_cloud_sql_operatorquery_system_helper.py +++ b/tests/gcp/operators/test_cloud_sql_operatorquery_system_helper.py @@ -20,8 +20,8 @@ import argparse -from gcp.operators.test_cloud_sql_system_helper \ - import CloudSqlQueryTestHelper +from gcp.operators.test_cloud_sql_system_helper import CloudSqlQueryTestHelper + from tests.gcp.utils.gcp_authenticator import GCP_CLOUDSQL_KEY, GcpAuthenticator QUERY_SUFFIX = "_QUERY" diff --git a/tests/gcp/operators/test_cloud_sql_system.py b/tests/gcp/operators/test_cloud_sql_system.py index 988f1557158511..dd890f09a55d9a 100644 --- a/tests/gcp/operators/test_cloud_sql_system.py +++ b/tests/gcp/operators/test_cloud_sql_system.py @@ -20,9 +20,8 @@ import unittest from airflow import AirflowException +from tests.gcp.operators.test_cloud_sql_system_helper import CloudSqlQueryTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem -from tests.gcp.operators.test_cloud_sql_system_helper import \ - CloudSqlQueryTestHelper from tests.gcp.utils.gcp_authenticator import GCP_CLOUDSQL_KEY GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'project-id') diff --git a/tests/gcp/operators/test_cloud_sql_system_helper.py b/tests/gcp/operators/test_cloud_sql_system_helper.py index 0500062a3129bc..80adf6c6c8b8b9 100755 --- a/tests/gcp/operators/test_cloud_sql_system_helper.py +++ b/tests/gcp/operators/test_cloud_sql_system_helper.py @@ -17,21 +17,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import argparse import errno import json import os +import time from os.path import expanduser - -import argparse from threading import Thread - -import time - from urllib.parse import urlsplit -from tests.gcp.utils.base_gcp_system_test_case import RetrieveVariables -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_CLOUDSQL_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.base_gcp_system_test_case import RetrieveVariables +from tests.gcp.utils.gcp_authenticator import GCP_CLOUDSQL_KEY, GcpAuthenticator retrieve_variables = RetrieveVariables() diff --git a/tests/gcp/operators/test_cloud_storage_transfer_service.py b/tests/gcp/operators/test_cloud_storage_transfer_service.py index 65f850390d900e..2eb30114e08eaf 100644 --- a/tests/gcp/operators/test_cloud_storage_transfer_service.py +++ b/tests/gcp/operators/test_cloud_storage_transfer_service.py @@ -22,45 +22,25 @@ from datetime import date, time from typing import Dict +from botocore.credentials import Credentials from freezegun import freeze_time from parameterized import parameterized -from botocore.credentials import Credentials from airflow import AirflowException from airflow.gcp.hooks.cloud_storage_transfer_service import ( - FILTER_JOB_NAMES, - SCHEDULE_START_DATE, - SCHEDULE_END_DATE, - START_TIME_OF_DAY, - STATUS, - NAME, - AWS_S3_DATA_SOURCE, - GCS_DATA_SOURCE, - GCS_DATA_SINK, - AWS_ACCESS_KEY, - ACCESS_KEY_ID, - SECRET_ACCESS_KEY, - BUCKET_NAME, - SCHEDULE, - TRANSFER_SPEC, - HTTP_DATA_SOURCE, - LIST_URL, + ACCESS_KEY_ID, AWS_ACCESS_KEY, AWS_S3_DATA_SOURCE, BUCKET_NAME, FILTER_JOB_NAMES, GCS_DATA_SINK, + GCS_DATA_SOURCE, HTTP_DATA_SOURCE, LIST_URL, NAME, SCHEDULE, SCHEDULE_END_DATE, SCHEDULE_START_DATE, + SECRET_ACCESS_KEY, START_TIME_OF_DAY, STATUS, TRANSFER_SPEC, ) from airflow.gcp.operators.cloud_storage_transfer_service import ( - GcpTransferServiceOperationCancelOperator, - GcpTransferServiceOperationResumeOperator, - GcpTransferServiceOperationsListOperator, - TransferJobValidator, - TransferJobPreprocessor, - GcpTransferServiceJobCreateOperator, - GcpTransferServiceJobUpdateOperator, - GcpTransferServiceOperationGetOperator, - GcpTransferServiceOperationPauseOperator, - S3ToGoogleCloudStorageTransferOperator, - GoogleCloudStorageToGoogleCloudStorageTransferOperator, - GcpTransferServiceJobDeleteOperator, + GcpTransferServiceJobCreateOperator, GcpTransferServiceJobDeleteOperator, + GcpTransferServiceJobUpdateOperator, GcpTransferServiceOperationCancelOperator, + GcpTransferServiceOperationGetOperator, GcpTransferServiceOperationPauseOperator, + GcpTransferServiceOperationResumeOperator, GcpTransferServiceOperationsListOperator, + GoogleCloudStorageToGoogleCloudStorageTransferOperator, S3ToGoogleCloudStorageTransferOperator, + TransferJobPreprocessor, TransferJobValidator, ) -from airflow.models import TaskInstance, DAG +from airflow.models import DAG, TaskInstance from airflow.utils import timezone from tests.compat import mock diff --git a/tests/gcp/operators/test_cloud_storage_transfer_service_system.py b/tests/gcp/operators/test_cloud_storage_transfer_service_system.py index e7b0e536ee96be..1d5419ebe0c7f0 100644 --- a/tests/gcp/operators/test_cloud_storage_transfer_service_system.py +++ b/tests/gcp/operators/test_cloud_storage_transfer_service_system.py @@ -20,7 +20,6 @@ from tests.gcp.operators.test_cloud_storage_transfer_service_system_helper import GCPTransferTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem - from tests.gcp.utils.gcp_authenticator import GCP_GCS_TRANSFER_KEY diff --git a/tests/gcp/operators/test_cloud_storage_transfer_service_system_helper.py b/tests/gcp/operators/test_cloud_storage_transfer_service_system_helper.py index 81de5f7feb7a9b..a15f4ed4374587 100755 --- a/tests/gcp/operators/test_cloud_storage_transfer_service_system_helper.py +++ b/tests/gcp/operators/test_cloud_storage_transfer_service_system_helper.py @@ -21,13 +21,12 @@ import os import subprocess -from googleapiclient._auth import default_credentials, with_scopes from googleapiclient import discovery +from googleapiclient._auth import default_credentials, with_scopes -from tests.gcp.utils.base_gcp_system_test_case import RetrieveVariables -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_GCS_TRANSFER_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor - +from tests.gcp.utils.base_gcp_system_test_case import RetrieveVariables +from tests.gcp.utils.gcp_authenticator import GCP_GCS_TRANSFER_KEY, GcpAuthenticator retrieve_variables = RetrieveVariables() diff --git a/tests/gcp/operators/test_compute.py b/tests/gcp/operators/test_compute.py index 2c693044befbf0..0ab64e05a62655 100644 --- a/tests/gcp/operators/test_compute.py +++ b/tests/gcp/operators/test_compute.py @@ -27,10 +27,11 @@ from googleapiclient.errors import HttpError from airflow import AirflowException -from airflow.gcp.operators.compute import GceInstanceStartOperator, \ - GceInstanceStopOperator, GceSetMachineTypeOperator, GceInstanceTemplateCopyOperator, \ - GceInstanceGroupManagerUpdateTemplateOperator -from airflow.models import TaskInstance, DAG +from airflow.gcp.operators.compute import ( + GceInstanceGroupManagerUpdateTemplateOperator, GceInstanceStartOperator, GceInstanceStopOperator, + GceInstanceTemplateCopyOperator, GceSetMachineTypeOperator, +) +from airflow.models import DAG, TaskInstance from airflow.utils import timezone from tests.compat import mock diff --git a/tests/gcp/operators/test_compute_system.py b/tests/gcp/operators/test_compute_system.py index fed7b0bc7ba807..9f0d68f0646939 100644 --- a/tests/gcp/operators/test_compute_system.py +++ b/tests/gcp/operators/test_compute_system.py @@ -18,10 +18,8 @@ # under the License. import unittest +from tests.gcp.operators.test_compute_system_helper import GCPComputeTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem - -from tests.gcp.operators.test_compute_system_helper import \ - GCPComputeTestHelper from tests.gcp.utils.gcp_authenticator import GCP_COMPUTE_KEY diff --git a/tests/gcp/operators/test_compute_system_helper.py b/tests/gcp/operators/test_compute_system_helper.py index 2d35a573267c09..f7c8821b3af989 100755 --- a/tests/gcp/operators/test_compute_system_helper.py +++ b/tests/gcp/operators/test_compute_system_helper.py @@ -20,8 +20,8 @@ import argparse import os -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_COMPUTE_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_COMPUTE_KEY, GcpAuthenticator GCE_INSTANCE = os.environ.get('GCE_INSTANCE', 'testinstance') GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') diff --git a/tests/gcp/operators/test_dataflow.py b/tests/gcp/operators/test_dataflow.py index 7baed133a4b8f7..40b32ada2e7a48 100644 --- a/tests/gcp/operators/test_dataflow.py +++ b/tests/gcp/operators/test_dataflow.py @@ -20,14 +20,13 @@ import unittest -from airflow.gcp.operators.dataflow import \ - DataFlowPythonOperator, DataFlowJavaOperator, \ - DataflowTemplateOperator, GoogleCloudBucketHelper, CheckJobRunning - +from airflow.gcp.operators.dataflow import ( + CheckJobRunning, DataFlowJavaOperator, DataFlowPythonOperator, DataflowTemplateOperator, + GoogleCloudBucketHelper, +) from airflow.version import version from tests.compat import mock - TASK_ID = 'test-dataflow-operator' JOB_NAME = 'test-dataflow-pipeline' TEMPLATE = 'gs://dataflow-templates/wordcount/template_file' diff --git a/tests/gcp/operators/test_dataflow_system.py b/tests/gcp/operators/test_dataflow_system.py index ca6207f1121047..5c4142d930ace5 100644 --- a/tests/gcp/operators/test_dataflow_system.py +++ b/tests/gcp/operators/test_dataflow_system.py @@ -19,7 +19,7 @@ import unittest -from tests.gcp.utils.base_gcp_system_test_case import TestDagGcpSystem, SKIP_TEST_WARNING +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_DATAFLOW_KEY diff --git a/tests/gcp/operators/test_dataproc.py b/tests/gcp/operators/test_dataproc.py index cdddbc77ffd297..eb0b9f75fd8a24 100644 --- a/tests/gcp/operators/test_dataproc.py +++ b/tests/gcp/operators/test_dataproc.py @@ -21,27 +21,21 @@ import datetime import re -import unittest -from unittest.mock import MagicMock, Mock, patch, PropertyMock -from typing import Dict - import time +import unittest from copy import deepcopy +from typing import Dict +from unittest.mock import MagicMock, Mock, PropertyMock, patch from airflow import DAG, AirflowException -from airflow.gcp.hooks.dataproc import _DataProcJobBuilder -from airflow.gcp.operators.dataproc import \ - DataprocClusterCreateOperator, \ - DataprocClusterDeleteOperator, \ - DataProcHadoopOperator, \ - DataProcHiveOperator, \ - DataProcPigOperator, \ - DataProcPySparkOperator, \ - DataProcSparkOperator, \ - DataprocWorkflowTemplateInstantiateInlineOperator, \ - DataprocWorkflowTemplateInstantiateOperator, \ - DataprocClusterScaleOperator, DataProcJobBaseOperator from airflow.exceptions import AirflowTaskTimeout +from airflow.gcp.hooks.dataproc import _DataProcJobBuilder +from airflow.gcp.operators.dataproc import ( + DataprocClusterCreateOperator, DataprocClusterDeleteOperator, DataprocClusterScaleOperator, + DataProcHadoopOperator, DataProcHiveOperator, DataProcJobBaseOperator, DataProcPigOperator, + DataProcPySparkOperator, DataProcSparkOperator, DataprocWorkflowTemplateInstantiateInlineOperator, + DataprocWorkflowTemplateInstantiateOperator, +) from airflow.models.taskinstance import TaskInstance from airflow.utils.timezone import make_aware from airflow.version import version diff --git a/tests/gcp/operators/test_dataproc_system.py b/tests/gcp/operators/test_dataproc_system.py index 086a6ba22142b8..a9927165af03a8 100644 --- a/tests/gcp/operators/test_dataproc_system.py +++ b/tests/gcp/operators/test_dataproc_system.py @@ -19,14 +19,9 @@ import os import unittest -from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - TestDagGcpSystem, -) +from tests.contrib.operators.test_dataproc_operator_system_helper import DataprocTestHelper +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_DATAPROC_KEY -from tests.contrib.operators.test_dataproc_operator_system_helper import ( - DataprocTestHelper, -) BUCKET = os.environ.get("GCP_DATAPROC_BUCKET", "dataproc-system-tests") PYSPARK_MAIN = os.environ.get("PYSPARK_MAIN", "hello_world.py") diff --git a/tests/gcp/operators/test_datastore_system.py b/tests/gcp/operators/test_datastore_system.py index f3082e614a8ff8..98a030fbd14120 100644 --- a/tests/gcp/operators/test_datastore_system.py +++ b/tests/gcp/operators/test_datastore_system.py @@ -19,12 +19,9 @@ import unittest -from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - TestDagGcpSystem, -) -from tests.gcp.utils.gcp_authenticator import GCP_DATASTORE_KEY from tests.gcp.operators.test_datastore_system_helper import GcpDatastoreSystemTestHelper +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem +from tests.gcp.utils.gcp_authenticator import GCP_DATASTORE_KEY @unittest.skipIf(TestDagGcpSystem.skip_check(GCP_DATASTORE_KEY), SKIP_TEST_WARNING) diff --git a/tests/gcp/operators/test_dlp.py b/tests/gcp/operators/test_dlp.py index f2d73537af1310..44a1372f5a4faf 100644 --- a/tests/gcp/operators/test_dlp.py +++ b/tests/gcp/operators/test_dlp.py @@ -25,36 +25,18 @@ import unittest from airflow.gcp.operators.dlp import ( - CloudDLPCancelDLPJobOperator, - CloudDLPCreateDeidentifyTemplateOperator, - CloudDLPCreateDLPJobOperator, - CloudDLPCreateInspectTemplateOperator, - CloudDLPCreateJobTriggerOperator, - CloudDLPCreateStoredInfoTypeOperator, - CloudDLPDeidentifyContentOperator, - CloudDLPDeleteDeidentifyTemplateOperator, - CloudDLPDeleteInspectTemplateOperator, - CloudDLPDeleteDlpJobOperator, - CloudDLPDeleteJobTriggerOperator, - CloudDLPDeleteStoredInfoTypeOperator, - CloudDLPGetDeidentifyTemplateOperator, - CloudDLPGetDlpJobOperator, - CloudDLPGetInspectTemplateOperator, - CloudDLPGetJobTripperOperator, - CloudDLPGetStoredInfoTypeOperator, - CloudDLPInspectContentOperator, - CloudDLPListDeidentifyTemplatesOperator, - CloudDLPListDlpJobsOperator, - CloudDLPListInfoTypesOperator, - CloudDLPListInspectTemplatesOperator, - CloudDLPListJobTriggersOperator, - CloudDLPListStoredInfoTypesOperator, - CloudDLPRedactImageOperator, - CloudDLPReidentifyContentOperator, - CloudDLPUpdateDeidentifyTemplateOperator, - CloudDLPUpdateInspectTemplateOperator, - CloudDLPUpdateJobTriggerOperator, - CloudDLPUpdateStoredInfoTypeOperator, + CloudDLPCancelDLPJobOperator, CloudDLPCreateDeidentifyTemplateOperator, CloudDLPCreateDLPJobOperator, + CloudDLPCreateInspectTemplateOperator, CloudDLPCreateJobTriggerOperator, + CloudDLPCreateStoredInfoTypeOperator, CloudDLPDeidentifyContentOperator, + CloudDLPDeleteDeidentifyTemplateOperator, CloudDLPDeleteDlpJobOperator, + CloudDLPDeleteInspectTemplateOperator, CloudDLPDeleteJobTriggerOperator, + CloudDLPDeleteStoredInfoTypeOperator, CloudDLPGetDeidentifyTemplateOperator, CloudDLPGetDlpJobOperator, + CloudDLPGetInspectTemplateOperator, CloudDLPGetJobTripperOperator, CloudDLPGetStoredInfoTypeOperator, + CloudDLPInspectContentOperator, CloudDLPListDeidentifyTemplatesOperator, CloudDLPListDlpJobsOperator, + CloudDLPListInfoTypesOperator, CloudDLPListInspectTemplatesOperator, CloudDLPListJobTriggersOperator, + CloudDLPListStoredInfoTypesOperator, CloudDLPRedactImageOperator, CloudDLPReidentifyContentOperator, + CloudDLPUpdateDeidentifyTemplateOperator, CloudDLPUpdateInspectTemplateOperator, + CloudDLPUpdateJobTriggerOperator, CloudDLPUpdateStoredInfoTypeOperator, ) from tests.compat import mock diff --git a/tests/gcp/operators/test_functions.py b/tests/gcp/operators/test_functions.py index 1f0316eddb7e27..238af763529df6 100644 --- a/tests/gcp/operators/test_functions.py +++ b/tests/gcp/operators/test_functions.py @@ -18,20 +18,18 @@ # under the License. import unittest - from copy import deepcopy from googleapiclient.errors import HttpError from parameterized import parameterized -from airflow.gcp.operators.functions import \ - GcfFunctionDeployOperator, GcfFunctionDeleteOperator, \ - FUNCTION_NAME_PATTERN, GcfFunctionInvokeOperator from airflow import AirflowException +from airflow.gcp.operators.functions import ( + FUNCTION_NAME_PATTERN, GcfFunctionDeleteOperator, GcfFunctionDeployOperator, GcfFunctionInvokeOperator, +) from airflow.version import version from tests.compat import mock - EMPTY_CONTENT = b'' MOCK_RESP_404 = type('', (object,), {"status": 404})() diff --git a/tests/gcp/operators/test_gcs.py b/tests/gcp/operators/test_gcs.py index e55a5fa56d44f9..68b1cf93dbd8c2 100644 --- a/tests/gcp/operators/test_gcs.py +++ b/tests/gcp/operators/test_gcs.py @@ -20,13 +20,10 @@ import unittest from airflow.gcp.operators.gcs import ( - GoogleCloudStorageCreateBucketOperator, - GoogleCloudStorageBucketCreateAclEntryOperator, + GcsFileTransformOperator, GoogleCloudStorageBucketCreateAclEntryOperator, + GoogleCloudStorageCreateBucketOperator, GoogleCloudStorageDeleteOperator, + GoogleCloudStorageDownloadOperator, GoogleCloudStorageListOperator, GoogleCloudStorageObjectCreateAclEntryOperator, - GoogleCloudStorageDeleteOperator, - GoogleCloudStorageDownloadOperator, - GoogleCloudStorageListOperator, - GcsFileTransformOperator ) from tests.compat import mock diff --git a/tests/gcp/operators/test_gcs_system.py b/tests/gcp/operators/test_gcs_system.py index 97d77cd5008b7d..1f477326bba42a 100644 --- a/tests/gcp/operators/test_gcs_system.py +++ b/tests/gcp/operators/test_gcs_system.py @@ -18,12 +18,9 @@ # under the License. import unittest -from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - TestDagGcpSystem, -) -from tests.gcp.utils.gcp_authenticator import GCP_GCS_KEY from tests.gcp.operators.test_gcs_system_helper import GcsSystemTestHelper +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem +from tests.gcp.utils.gcp_authenticator import GCP_GCS_KEY @unittest.skipIf(TestDagGcpSystem.skip_check(GCP_GCS_KEY), SKIP_TEST_WARNING) diff --git a/tests/gcp/operators/test_kubernetes_engine.py b/tests/gcp/operators/test_kubernetes_engine.py index 2fc4c17f2f9f5e..bcd61694747a1c 100644 --- a/tests/gcp/operators/test_kubernetes_engine.py +++ b/tests/gcp/operators/test_kubernetes_engine.py @@ -20,15 +20,16 @@ import os import unittest -from parameterized import parameterized from google.auth.environment_vars import CREDENTIALS +from parameterized import parameterized from airflow import AirflowException -from airflow.gcp.operators.kubernetes_engine import GKEClusterCreateOperator, \ - GKEClusterDeleteOperator, GKEPodOperator from airflow.contrib.operators.kubernetes_pod_operator import KubernetesPodOperator +from airflow.gcp.operators.kubernetes_engine import ( + GKEClusterCreateOperator, GKEClusterDeleteOperator, GKEPodOperator, +) from airflow.models import Connection -from tests.compat import mock, PropertyMock +from tests.compat import PropertyMock, mock TEST_GCP_PROJECT_ID = 'test-id' PROJECT_LOCATION = 'test-location' diff --git a/tests/gcp/operators/test_kubernetes_engine_system.py b/tests/gcp/operators/test_kubernetes_engine_system.py index 7265f03076bc84..b06eba3142eafa 100644 --- a/tests/gcp/operators/test_kubernetes_engine_system.py +++ b/tests/gcp/operators/test_kubernetes_engine_system.py @@ -18,10 +18,7 @@ # under the License. import unittest -from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - TestDagGcpSystem, -) +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_GKE_KEY diff --git a/tests/gcp/operators/test_mlengine.py b/tests/gcp/operators/test_mlengine.py index b9190413d9f58b..8c839b61045aa0 100644 --- a/tests/gcp/operators/test_mlengine.py +++ b/tests/gcp/operators/test_mlengine.py @@ -18,18 +18,16 @@ import copy import datetime import unittest - from unittest.mock import ANY, patch import httplib2 - from googleapiclient.errors import HttpError from airflow import DAG -from airflow.gcp.operators.mlengine import (MLEngineBatchPredictionOperator, - MLEngineTrainingOperator, - MLEngineVersionOperator) from airflow.exceptions import AirflowException +from airflow.gcp.operators.mlengine import ( + MLEngineBatchPredictionOperator, MLEngineTrainingOperator, MLEngineVersionOperator, +) DEFAULT_DATE = datetime.datetime(2017, 6, 6) diff --git a/tests/gcp/operators/test_mlengine_utils.py b/tests/gcp/operators/test_mlengine_utils.py index 4022ca1c48b0af..6dc25afb0f1084 100644 --- a/tests/gcp/operators/test_mlengine_utils.py +++ b/tests/gcp/operators/test_mlengine_utils.py @@ -17,13 +17,11 @@ import datetime import unittest - -from unittest.mock import ANY -from unittest.mock import patch +from unittest.mock import ANY, patch from airflow import DAG -from airflow.gcp.utils import mlengine_operator_utils from airflow.exceptions import AirflowException +from airflow.gcp.utils import mlengine_operator_utils from airflow.version import version DEFAULT_DATE = datetime.datetime(2017, 6, 6) diff --git a/tests/gcp/operators/test_natural_language.py b/tests/gcp/operators/test_natural_language.py index 3d4dabddb1d0f8..38c7adfa87d419 100644 --- a/tests/gcp/operators/test_natural_language.py +++ b/tests/gcp/operators/test_natural_language.py @@ -20,22 +20,16 @@ import unittest from google.cloud.language_v1.proto.language_service_pb2 import ( - AnalyzeEntitiesResponse, - AnalyzeEntitySentimentResponse, - AnalyzeSentimentResponse, - ClassifyTextResponse, + AnalyzeEntitiesResponse, AnalyzeEntitySentimentResponse, AnalyzeSentimentResponse, ClassifyTextResponse, Document, ) from airflow.gcp.operators.natural_language import ( - CloudLanguageAnalyzeEntitiesOperator, - CloudLanguageAnalyzeEntitySentimentOperator, - CloudLanguageAnalyzeSentimentOperator, - CloudLanguageClassifyTextOperator, + CloudLanguageAnalyzeEntitiesOperator, CloudLanguageAnalyzeEntitySentimentOperator, + CloudLanguageAnalyzeSentimentOperator, CloudLanguageClassifyTextOperator, ) from tests.compat import patch - DOCUMENT = Document( content="Airflow is a platform to programmatically author, schedule and monitor workflows." ) diff --git a/tests/gcp/operators/test_pubsub.py b/tests/gcp/operators/test_pubsub.py index 4d0ad33950cb60..8ca437ae0f5337 100644 --- a/tests/gcp/operators/test_pubsub.py +++ b/tests/gcp/operators/test_pubsub.py @@ -20,9 +20,9 @@ import unittest from airflow.gcp.operators.pubsub import ( + PubSubPublishOperator, PubSubSubscriptionCreateOperator, PubSubSubscriptionDeleteOperator, PubSubTopicCreateOperator, PubSubTopicDeleteOperator, - PubSubSubscriptionCreateOperator, PubSubSubscriptionDeleteOperator, - PubSubPublishOperator) +) from tests.compat import mock TASK_ID = 'test-task-id' diff --git a/tests/gcp/operators/test_pubsub_system.py b/tests/gcp/operators/test_pubsub_system.py index 075dfbc2aae1d7..68f55bb4613200 100644 --- a/tests/gcp/operators/test_pubsub_system.py +++ b/tests/gcp/operators/test_pubsub_system.py @@ -18,10 +18,7 @@ # under the License. import unittest -from tests.gcp.utils.base_gcp_system_test_case import ( - SKIP_TEST_WARNING, - TestDagGcpSystem, -) +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_PUBSUB_KEY diff --git a/tests/gcp/operators/test_spanner.py b/tests/gcp/operators/test_spanner.py index 364c1f460f4be2..17085af14ef51c 100644 --- a/tests/gcp/operators/test_spanner.py +++ b/tests/gcp/operators/test_spanner.py @@ -21,16 +21,13 @@ from parameterized import parameterized from airflow import AirflowException -from airflow.gcp.operators.spanner import \ - CloudSpannerInstanceDeployOperator, \ - CloudSpannerInstanceDeleteOperator, \ - CloudSpannerInstanceDatabaseQueryOperator, \ - CloudSpannerInstanceDatabaseDeployOperator, \ - CloudSpannerInstanceDatabaseDeleteOperator, \ - CloudSpannerInstanceDatabaseUpdateOperator +from airflow.gcp.operators.spanner import ( + CloudSpannerInstanceDatabaseDeleteOperator, CloudSpannerInstanceDatabaseDeployOperator, + CloudSpannerInstanceDatabaseQueryOperator, CloudSpannerInstanceDatabaseUpdateOperator, + CloudSpannerInstanceDeleteOperator, CloudSpannerInstanceDeployOperator, +) from tests.compat import mock - PROJECT_ID = 'project-id' INSTANCE_ID = 'instance-id' DB_ID = 'db1' diff --git a/tests/gcp/operators/test_spanner_system.py b/tests/gcp/operators/test_spanner_system.py index 1a98a1a0aff76f..f77e800e063ad2 100644 --- a/tests/gcp/operators/test_spanner_system.py +++ b/tests/gcp/operators/test_spanner_system.py @@ -18,8 +18,7 @@ # under the License. import unittest -from tests.gcp.operators.test_spanner_system_helper import \ - GCPSpannerTestHelper +from tests.gcp.operators.test_spanner_system_helper import GCPSpannerTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_SPANNER_KEY diff --git a/tests/gcp/operators/test_spanner_system_helper.py b/tests/gcp/operators/test_spanner_system_helper.py index 71c5a668ea7b58..35e23a8fcbf1c4 100755 --- a/tests/gcp/operators/test_spanner_system_helper.py +++ b/tests/gcp/operators/test_spanner_system_helper.py @@ -17,12 +17,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import os - import argparse +import os -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_SPANNER_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_SPANNER_KEY, GcpAuthenticator GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') GCP_SPANNER_INSTANCE_ID = os.environ.get('GCP_SPANNER_INSTANCE_ID', 'testinstance') diff --git a/tests/gcp/operators/test_speech_system.py b/tests/gcp/operators/test_speech_system.py index 3a63c8be3a67bb..57d8b51fc74ec6 100644 --- a/tests/gcp/operators/test_speech_system.py +++ b/tests/gcp/operators/test_speech_system.py @@ -19,11 +19,10 @@ import unittest +from tests.gcp.operators.test_speech_system_helper import GCPTextToSpeechTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_GCS_KEY -from tests.gcp.operators.test_speech_system_helper import GCPTextToSpeechTestHelper - @unittest.skipIf(TestDagGcpSystem.skip_check(GCP_GCS_KEY), SKIP_TEST_WARNING) class GCPTextToSpeechExampleDagSystemTest(TestDagGcpSystem): diff --git a/tests/gcp/operators/test_speech_system_helper.py b/tests/gcp/operators/test_speech_system_helper.py index cac4bba333dfb6..e941b5da856fa8 100644 --- a/tests/gcp/operators/test_speech_system_helper.py +++ b/tests/gcp/operators/test_speech_system_helper.py @@ -21,10 +21,9 @@ import argparse import os -from tests.gcp.utils.base_gcp_system_test_case import RetrieveVariables -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_GCS_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor - +from tests.gcp.utils.base_gcp_system_test_case import RetrieveVariables +from tests.gcp.utils.gcp_authenticator import GCP_GCS_KEY, GcpAuthenticator retrieve_variables = RetrieveVariables() diff --git a/tests/gcp/operators/test_tasks.py b/tests/gcp/operators/test_tasks.py index b080ece9217406..77de3190daa86c 100644 --- a/tests/gcp/operators/test_tasks.py +++ b/tests/gcp/operators/test_tasks.py @@ -22,19 +22,11 @@ from google.cloud.tasks_v2.types import Queue, Task from airflow.gcp.operators.tasks import ( - CloudTasksQueueCreateOperator, - CloudTasksTaskCreateOperator, - CloudTasksQueueDeleteOperator, - CloudTasksTaskDeleteOperator, - CloudTasksQueueGetOperator, - CloudTasksTaskGetOperator, - CloudTasksQueuesListOperator, + CloudTasksQueueCreateOperator, CloudTasksQueueDeleteOperator, CloudTasksQueueGetOperator, + CloudTasksQueuePauseOperator, CloudTasksQueuePurgeOperator, CloudTasksQueueResumeOperator, + CloudTasksQueuesListOperator, CloudTasksQueueUpdateOperator, CloudTasksTaskCreateOperator, + CloudTasksTaskDeleteOperator, CloudTasksTaskGetOperator, CloudTasksTaskRunOperator, CloudTasksTasksListOperator, - CloudTasksQueuePauseOperator, - CloudTasksQueuePurgeOperator, - CloudTasksQueueResumeOperator, - CloudTasksTaskRunOperator, - CloudTasksQueueUpdateOperator, ) from tests.compat import mock diff --git a/tests/gcp/operators/test_text_to_speech.py b/tests/gcp/operators/test_text_to_speech.py index 070ad4816fa65d..db115fbfdcc18b 100644 --- a/tests/gcp/operators/test_text_to_speech.py +++ b/tests/gcp/operators/test_text_to_speech.py @@ -23,7 +23,7 @@ from airflow import AirflowException from airflow.gcp.operators.text_to_speech import GcpTextToSpeechSynthesizeOperator -from tests.compat import PropertyMock, Mock, patch, ANY +from tests.compat import ANY, Mock, PropertyMock, patch PROJECT_ID = "project-id" GCP_CONN_ID = "gcp-conn-id" diff --git a/tests/gcp/operators/test_translate_speech.py b/tests/gcp/operators/test_translate_speech.py index 3d08b0637adb42..68a22e89cbf840 100644 --- a/tests/gcp/operators/test_translate_speech.py +++ b/tests/gcp/operators/test_translate_speech.py @@ -18,8 +18,10 @@ # under the License. import unittest -from google.cloud.speech_v1.proto.cloud_speech_pb2 import RecognizeResponse, SpeechRecognitionResult, \ - SpeechRecognitionAlternative + +from google.cloud.speech_v1.proto.cloud_speech_pb2 import ( + RecognizeResponse, SpeechRecognitionAlternative, SpeechRecognitionResult, +) from airflow import AirflowException from airflow.gcp.operators.translate_speech import GcpTranslateSpeechOperator diff --git a/tests/gcp/operators/test_translate_system.py b/tests/gcp/operators/test_translate_system.py index 24edda39af89bd..3ed1efdb7ad965 100644 --- a/tests/gcp/operators/test_translate_system.py +++ b/tests/gcp/operators/test_translate_system.py @@ -19,7 +19,7 @@ import unittest -from tests.gcp.utils.base_gcp_system_test_case import TestDagGcpSystem, SKIP_TEST_WARNING +from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_AI_KEY diff --git a/tests/gcp/operators/test_video_intelligence.py b/tests/gcp/operators/test_video_intelligence.py index 6a9ddc0c1bccf9..b15ff5a5ad488e 100644 --- a/tests/gcp/operators/test_video_intelligence.py +++ b/tests/gcp/operators/test_video_intelligence.py @@ -21,10 +21,10 @@ from google.cloud.videointelligence_v1 import enums from google.cloud.videointelligence_v1.proto.video_intelligence_pb2 import AnnotateVideoResponse + from airflow.gcp.operators.video_intelligence import ( - CloudVideoIntelligenceDetectVideoLabelsOperator, + CloudVideoIntelligenceDetectVideoExplicitContentOperator, CloudVideoIntelligenceDetectVideoLabelsOperator, CloudVideoIntelligenceDetectVideoShotsOperator, - CloudVideoIntelligenceDetectVideoExplicitContentOperator, ) from tests.compat import mock diff --git a/tests/gcp/operators/test_video_intelligence_system.py b/tests/gcp/operators/test_video_intelligence_system.py index eae991ec232948..2d786704c9132e 100644 --- a/tests/gcp/operators/test_video_intelligence_system.py +++ b/tests/gcp/operators/test_video_intelligence_system.py @@ -18,9 +18,7 @@ # under the License. import unittest -from tests.gcp.operators.test_video_intelligence_system_helper import ( - GCPVideoIntelligenceHelper, -) +from tests.gcp.operators.test_video_intelligence_system_helper import GCPVideoIntelligenceHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem from tests.gcp.utils.gcp_authenticator import GCP_AI_KEY diff --git a/tests/gcp/operators/test_video_intelligence_system_helper.py b/tests/gcp/operators/test_video_intelligence_system_helper.py index 87b507d3dd7790..95808a4fca5489 100755 --- a/tests/gcp/operators/test_video_intelligence_system_helper.py +++ b/tests/gcp/operators/test_video_intelligence_system_helper.py @@ -17,12 +17,11 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import os - import argparse +import os -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_AI_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_AI_KEY, GcpAuthenticator GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") GCP_BUCKET_NAME = os.environ.get("GCP_VIDEO_INTELLIGENCE_BUCKET_NAME", "test-bucket-name") diff --git a/tests/gcp/operators/test_vision.py b/tests/gcp/operators/test_vision.py index 505351776258c7..5c6538a4a18277 100644 --- a/tests/gcp/operators/test_vision.py +++ b/tests/gcp/operators/test_vision.py @@ -20,27 +20,17 @@ import unittest from google.api_core.exceptions import AlreadyExists -from google.cloud.vision_v1.types import ProductSet, Product, ReferenceImage +from google.cloud.vision_v1.types import Product, ProductSet, ReferenceImage from airflow.gcp.operators.vision import ( - CloudVisionProductSetCreateOperator, - CloudVisionProductSetGetOperator, - CloudVisionProductSetUpdateOperator, - CloudVisionProductSetDeleteOperator, - CloudVisionProductCreateOperator, - CloudVisionProductGetOperator, - CloudVisionProductUpdateOperator, - CloudVisionProductDeleteOperator, - CloudVisionReferenceImageCreateOperator, - CloudVisionAddProductToProductSetOperator, - CloudVisionRemoveProductFromProductSetOperator, - CloudVisionAnnotateImageOperator, - CloudVisionDetectTextOperator, - CloudVisionDetectDocumentTextOperator, - CloudVisionDetectImageLabelsOperator, - CloudVisionDetectImageSafeSearchOperator, + CloudVisionAddProductToProductSetOperator, CloudVisionAnnotateImageOperator, + CloudVisionDetectDocumentTextOperator, CloudVisionDetectImageLabelsOperator, + CloudVisionDetectImageSafeSearchOperator, CloudVisionDetectTextOperator, CloudVisionProductCreateOperator, + CloudVisionProductDeleteOperator, CloudVisionProductGetOperator, CloudVisionProductSetCreateOperator, + CloudVisionProductSetDeleteOperator, CloudVisionProductSetGetOperator, + CloudVisionProductSetUpdateOperator, CloudVisionProductUpdateOperator, + CloudVisionReferenceImageCreateOperator, CloudVisionRemoveProductFromProductSetOperator, ) - from tests.compat import mock PRODUCTSET_TEST = ProductSet(display_name='Test Product Set') diff --git a/tests/gcp/operators/test_vision_system.py b/tests/gcp/operators/test_vision_system.py index 3624d723d8f24f..98fcc18d8f9a74 100644 --- a/tests/gcp/operators/test_vision_system.py +++ b/tests/gcp/operators/test_vision_system.py @@ -19,9 +19,9 @@ import unittest -from tests.gcp.utils.gcp_authenticator import GCP_AI_KEY from tests.gcp.operators.test_vision_system_helper import GCPVisionTestHelper from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem +from tests.gcp.utils.gcp_authenticator import GCP_AI_KEY VISION_HELPER = GCPVisionTestHelper() diff --git a/tests/gcp/operators/test_vision_system_helper.py b/tests/gcp/operators/test_vision_system_helper.py index d9e5e8509383c9..d0cf0e62c20449 100755 --- a/tests/gcp/operators/test_vision_system_helper.py +++ b/tests/gcp/operators/test_vision_system_helper.py @@ -17,13 +17,12 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import os - import argparse +import os from tempfile import NamedTemporaryFile -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_AI_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_AI_KEY, GcpAuthenticator GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') GCP_BUCKET_NAME = os.environ.get('GCP_VISION_BUCKET_NAME', 'vision-bucket-system-test') diff --git a/tests/gcp/sensors/test_bigquery_dts.py b/tests/gcp/sensors/test_bigquery_dts.py index 95fba396b90049..5908bdd9ae58f8 100644 --- a/tests/gcp/sensors/test_bigquery_dts.py +++ b/tests/gcp/sensors/test_bigquery_dts.py @@ -18,11 +18,10 @@ # under the License. import unittest + import mock -from airflow.gcp.sensors.bigquery_dts import ( - BigQueryDataTransferServiceTransferRunSensor, -) +from airflow.gcp.sensors.bigquery_dts import BigQueryDataTransferServiceTransferRunSensor TRANSFER_CONFIG_ID = "config_id" RUN_ID = "run_id" diff --git a/tests/gcp/sensors/test_bigtable.py b/tests/gcp/sensors/test_bigtable.py index 8b8f20b397ad28..977a0116637f2a 100644 --- a/tests/gcp/sensors/test_bigtable.py +++ b/tests/gcp/sensors/test_bigtable.py @@ -18,6 +18,7 @@ # under the License. import unittest + import google.api_core.exceptions from google.cloud.bigtable.instance import Instance from google.cloud.bigtable.table import ClusterState diff --git a/tests/gcp/sensors/test_gcs.py b/tests/gcp/sensors/test_gcs.py index b008d324b86edc..36b1469f1bad68 100644 --- a/tests/gcp/sensors/test_gcs.py +++ b/tests/gcp/sensors/test_gcs.py @@ -21,11 +21,13 @@ import pendulum +from airflow import DAG, models from airflow.exceptions import AirflowSensorTimeout -from airflow import models, DAG +from airflow.gcp.sensors.gcs import ( + GoogleCloudStorageObjectSensor, GoogleCloudStorageObjectUpdatedSensor, GoogleCloudStoragePrefixSensor, + ts_function, +) from airflow.settings import Session -from airflow.gcp.sensors.gcs import GoogleCloudStorageObjectSensor, \ - GoogleCloudStorageObjectUpdatedSensor, ts_function, GoogleCloudStoragePrefixSensor TEST_DAG_ID = 'test-dag_id' diff --git a/tests/gcp/sensors/test_pubsub.py b/tests/gcp/sensors/test_pubsub.py index 12261e380afe26..c90fec488c008b 100644 --- a/tests/gcp/sensors/test_pubsub.py +++ b/tests/gcp/sensors/test_pubsub.py @@ -20,10 +20,10 @@ import unittest from google.cloud.pubsub_v1.types import ReceivedMessage -from google.protobuf.json_format import ParseDict, MessageToDict +from google.protobuf.json_format import MessageToDict, ParseDict -from airflow.gcp.sensors.pubsub import PubSubPullSensor from airflow.exceptions import AirflowSensorTimeout +from airflow.gcp.sensors.pubsub import PubSubPullSensor from tests.compat import mock TASK_ID = 'test-task-id' diff --git a/tests/gcp/utils/base_gcp_system_test_case.py b/tests/gcp/utils/base_gcp_system_test_case.py index 39dbc59ccf663a..edcd32d6d98a81 100644 --- a/tests/gcp/utils/base_gcp_system_test_case.py +++ b/tests/gcp/utils/base_gcp_system_test_case.py @@ -23,11 +23,11 @@ from shutil import move from tempfile import mkdtemp +from airflow import AirflowException, LoggingMixin, models from airflow.utils import db as db_utils -from airflow import models, AirflowException, LoggingMixin from airflow.utils.timezone import datetime -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator from tests.contrib.utils.run_once_decorator import run_once +from tests.gcp.utils.gcp_authenticator import GcpAuthenticator AIRFLOW_MAIN_FOLDER = os.path.realpath(os.path.join( os.path.dirname(os.path.realpath(__file__)), diff --git a/tests/gcp/utils/gcp_authenticator.py b/tests/gcp/utils/gcp_authenticator.py index 01d051a175c4dc..15c96d8be59e39 100644 --- a/tests/gcp/utils/gcp_authenticator.py +++ b/tests/gcp/utils/gcp_authenticator.py @@ -16,14 +16,13 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from typing import Optional # noqa: W0611 import json import os import subprocess +from typing import Optional # noqa: W0611 -from airflow import settings, AirflowException +from airflow import AirflowException, settings from airflow.models import Connection - from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor # Please keep these variables in alphabetical order. diff --git a/tests/hooks/test_dbapi_hook.py b/tests/hooks/test_dbapi_hook.py index f845d0576f2e1d..911d4f86c7b32b 100644 --- a/tests/hooks/test_dbapi_hook.py +++ b/tests/hooks/test_dbapi_hook.py @@ -18,8 +18,8 @@ # under the License. # -from unittest import mock import unittest +from unittest import mock from airflow.hooks.dbapi_hook import DbApiHook diff --git a/tests/hooks/test_druid_hook.py b/tests/hooks/test_druid_hook.py index 4d5043016ba68b..867757a10988d0 100644 --- a/tests/hooks/test_druid_hook.py +++ b/tests/hooks/test_druid_hook.py @@ -18,10 +18,11 @@ # under the License. # +import unittest from unittest.mock import MagicMock, patch + import requests import requests_mock -import unittest from airflow.exceptions import AirflowException from airflow.hooks.druid_hook import DruidDbApiHook, DruidHook diff --git a/tests/hooks/test_hive_hook.py b/tests/hooks/test_hive_hook.py index 2f6eb0118e8f5f..52af425917e02b 100644 --- a/tests/hooks/test_hive_hook.py +++ b/tests/hooks/test_hive_hook.py @@ -23,8 +23,8 @@ import os import random import unittest -from unittest import mock from collections import OrderedDict +from unittest import mock import pandas as pd from hmsclient import HMSClient diff --git a/tests/hooks/test_mysql_hook.py b/tests/hooks/test_mysql_hook.py index be3afd71ae4138..0af32910f08272 100644 --- a/tests/hooks/test_mysql_hook.py +++ b/tests/hooks/test_mysql_hook.py @@ -19,8 +19,8 @@ # import json -from unittest import mock import unittest +from unittest import mock import MySQLdb.cursors diff --git a/tests/hooks/test_oracle_hook.py b/tests/hooks/test_oracle_hook.py index d2217477ed766d..2b2f870d035ca6 100644 --- a/tests/hooks/test_oracle_hook.py +++ b/tests/hooks/test_oracle_hook.py @@ -21,16 +21,17 @@ import unittest from datetime import datetime -try: - import cx_Oracle -except ImportError: - cx_Oracle = None import numpy from airflow.hooks.oracle_hook import OracleHook from airflow.models import Connection from tests.compat import mock +try: + import cx_Oracle +except ImportError: + cx_Oracle = None + @unittest.skipIf(cx_Oracle is None, 'cx_Oracle package not present') class TestOracleHookConn(unittest.TestCase): diff --git a/tests/hooks/test_pig_hook.py b/tests/hooks/test_pig_hook.py index dc9c4bbdab6572..529da68e6f93db 100644 --- a/tests/hooks/test_pig_hook.py +++ b/tests/hooks/test_pig_hook.py @@ -18,6 +18,7 @@ # under the License. import unittest + from airflow.hooks.pig_hook import PigCliHook from tests.compat import mock diff --git a/tests/hooks/test_postgres_hook.py b/tests/hooks/test_postgres_hook.py index d76ac916b6388e..903d45d7829e99 100644 --- a/tests/hooks/test_postgres_hook.py +++ b/tests/hooks/test_postgres_hook.py @@ -18,10 +18,9 @@ # under the License. # -from unittest import mock import unittest - from tempfile import NamedTemporaryFile +from unittest import mock from airflow.hooks.postgres_hook import PostgresHook from airflow.models import Connection diff --git a/tests/hooks/test_presto_hook.py b/tests/hooks/test_presto_hook.py index e12d9fd993dd8f..767e368552d36f 100644 --- a/tests/hooks/test_presto_hook.py +++ b/tests/hooks/test_presto_hook.py @@ -18,9 +18,8 @@ # under the License. # -from unittest import mock import unittest - +from unittest import mock from unittest.mock import patch from airflow.hooks.presto_hook import PrestoHook diff --git a/tests/hooks/test_slack_hook.py b/tests/hooks/test_slack_hook.py index cf8625a40e3973..7f45f996a2d9aa 100644 --- a/tests/hooks/test_slack_hook.py +++ b/tests/hooks/test_slack_hook.py @@ -18,6 +18,7 @@ # under the License. import unittest + from airflow.exceptions import AirflowException from airflow.hooks.slack_hook import SlackHook from tests.compat import mock diff --git a/tests/hooks/test_webhdfs_hook.py b/tests/hooks/test_webhdfs_hook.py index 3a5cbeb80ef801..bf2da6ed2c4d5c 100644 --- a/tests/hooks/test_webhdfs_hook.py +++ b/tests/hooks/test_webhdfs_hook.py @@ -18,11 +18,11 @@ # under the License. import unittest +from unittest.mock import call, patch from hdfs import HdfsError -from unittest.mock import patch, call -from airflow.hooks.webhdfs_hook import WebHDFSHook, AirflowWebHDFSHookException +from airflow.hooks.webhdfs_hook import AirflowWebHDFSHookException, WebHDFSHook from airflow.models.connection import Connection diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index abb35bec5f6caf..671473532237cc 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -28,10 +28,11 @@ from parameterized import parameterized from airflow import AirflowException, settings -from airflow.configuration import conf from airflow.bin import cli -from airflow.exceptions import AirflowTaskTimeout -from airflow.exceptions import DagConcurrencyLimitReached, NoAvailablePoolSlot, TaskConcurrencyLimitReached +from airflow.configuration import conf +from airflow.exceptions import ( + AirflowTaskTimeout, DagConcurrencyLimitReached, NoAvailablePoolSlot, TaskConcurrencyLimitReached, +) from airflow.jobs import BackfillJob, SchedulerJob from airflow.models import DAG, DagBag, DagRun, Pool, TaskInstance as TI from airflow.operators.dummy_operator import DummyOperator @@ -41,8 +42,7 @@ from airflow.utils.timeout import timeout from tests.compat import Mock, patch from tests.executors.test_executor import TestExecutor -from tests.test_utils.db import clear_db_pools, \ - clear_db_runs, set_default_pool_slots +from tests.test_utils.db import clear_db_pools, clear_db_runs, set_default_pool_slots logger = logging.getLogger(__name__) diff --git a/tests/jobs/test_base_job.py b/tests/jobs/test_base_job.py index be2cc63b5761c4..2cd4a1229e79d7 100644 --- a/tests/jobs/test_base_job.py +++ b/tests/jobs/test_base_job.py @@ -23,8 +23,8 @@ from airflow.jobs import BaseJob from airflow.utils import timezone -from airflow.utils.state import State from airflow.utils.db import create_session +from airflow.utils.state import State class TestBaseJob(unittest.TestCase): diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index 68fcdd6c132786..eb46f42166655a 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -33,8 +33,7 @@ from airflow.configuration import conf from airflow.executors import BaseExecutor from airflow.jobs import BackfillJob, SchedulerJob -from airflow.models import DAG, DagBag, DagModel, DagRun, Pool, SlaMiss, \ - TaskInstance as TI, errors +from airflow.models import DAG, DagBag, DagModel, DagRun, Pool, SlaMiss, TaskInstance as TI, errors from airflow.operators.bash_operator import BashOperator from airflow.operators.dummy_operator import DummyOperator from airflow.utils import timezone @@ -42,13 +41,12 @@ from airflow.utils.dates import days_ago from airflow.utils.db import create_session, provide_session from airflow.utils.state import State -from tests.compat import MagicMock, Mock, PropertyMock, patch -from tests.compat import mock +from tests.compat import MagicMock, Mock, PropertyMock, mock, patch from tests.core import TEST_DAG_FOLDER from tests.executors.test_executor import TestExecutor -from tests.test_utils.db import clear_db_dags, clear_db_errors, clear_db_pools, \ - clear_db_runs, clear_db_sla_miss, set_default_pool_slots - +from tests.test_utils.db import ( + clear_db_dags, clear_db_errors, clear_db_pools, clear_db_runs, clear_db_sla_miss, set_default_pool_slots, +) DEFAULT_DATE = timezone.datetime(2016, 1, 1) TRY_NUMBER = 1 diff --git a/tests/kubernetes/models/test_pod.py b/tests/kubernetes/models/test_pod.py index 24314ebefcfa54..727af29834396b 100644 --- a/tests/kubernetes/models/test_pod.py +++ b/tests/kubernetes/models/test_pod.py @@ -16,11 +16,13 @@ # under the License. import unittest import unittest.mock as mock -from kubernetes.client import ApiClient + import kubernetes.client.models as k8s +from kubernetes.client import ApiClient + +from airflow.kubernetes.k8s_model import append_to_pod from airflow.kubernetes.pod import Port from airflow.kubernetes.pod_generator import PodGenerator -from airflow.kubernetes.k8s_model import append_to_pod class TestPod(unittest.TestCase): diff --git a/tests/kubernetes/models/test_secret.py b/tests/kubernetes/models/test_secret.py index 1cdc87adcf055f..217bbe38dc9dd7 100644 --- a/tests/kubernetes/models/test_secret.py +++ b/tests/kubernetes/models/test_secret.py @@ -16,11 +16,13 @@ # under the License. import unittest import unittest.mock as mock -from kubernetes.client import ApiClient + import kubernetes.client.models as k8s -from airflow.kubernetes.secret import Secret -from airflow.kubernetes.pod_generator import PodGenerator +from kubernetes.client import ApiClient + from airflow.kubernetes.k8s_model import append_to_pod +from airflow.kubernetes.pod_generator import PodGenerator +from airflow.kubernetes.secret import Secret class TestSecret(unittest.TestCase): diff --git a/tests/kubernetes/test_pod_generator.py b/tests/kubernetes/test_pod_generator.py index 63e4535c290e1f..21a1b730325cde 100644 --- a/tests/kubernetes/test_pod_generator.py +++ b/tests/kubernetes/test_pod_generator.py @@ -17,12 +17,14 @@ import unittest import unittest.mock as mock + import kubernetes.client.models as k8s from kubernetes.client import ApiClient -from airflow.kubernetes.secret import Secret -from airflow.kubernetes.pod_generator import PodGenerator, PodDefaults -from airflow.kubernetes.pod import Resources + from airflow.kubernetes.k8s_model import append_to_pod +from airflow.kubernetes.pod import Resources +from airflow.kubernetes.pod_generator import PodDefaults, PodGenerator +from airflow.kubernetes.secret import Secret class TestPodGenerator(unittest.TestCase): diff --git a/tests/kubernetes/test_pod_launcher.py b/tests/kubernetes/test_pod_launcher.py index 9e0c288dd1d4fc..a7eca549c41a57 100644 --- a/tests/kubernetes/test_pod_launcher.py +++ b/tests/kubernetes/test_pod_launcher.py @@ -15,8 +15,8 @@ # specific language governing permissions and limitations # under the License. import unittest -import mock +import mock from requests.exceptions import BaseHTTPError from airflow import AirflowException diff --git a/tests/kubernetes/test_worker_configuration.py b/tests/kubernetes/test_worker_configuration.py index 8645c4bcbabf9d..cd6b0f8955cb4a 100644 --- a/tests/kubernetes/test_worker_configuration.py +++ b/tests/kubernetes/test_worker_configuration.py @@ -19,8 +19,10 @@ import unittest import uuid from datetime import datetime + from tests.compat import mock from tests.test_utils.config import conf_vars + try: from airflow.executors.kubernetes_executor import AirflowKubernetesScheduler from airflow.executors.kubernetes_executor import KubeConfig diff --git a/tests/lineage/backend/test_atlas.py b/tests/lineage/backend/test_atlas.py index ea9378f77af3ab..a174c6586db998 100644 --- a/tests/lineage/backend/test_atlas.py +++ b/tests/lineage/backend/test_atlas.py @@ -20,7 +20,7 @@ import unittest from configparser import DuplicateSectionError -from airflow.configuration import conf, AirflowConfigException +from airflow.configuration import AirflowConfigException, conf from airflow.lineage.backend.atlas import AtlasBackend from airflow.lineage.datasets import File from airflow.models import DAG, TaskInstance as TI diff --git a/tests/macros/test_hive.py b/tests/macros/test_hive.py index eb8dc1fc320734..d1c2e6a5b86e10 100644 --- a/tests/macros/test_hive.py +++ b/tests/macros/test_hive.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from datetime import datetime import unittest +from datetime import datetime from airflow.macros import hive diff --git a/tests/minikube/test_kubernetes_executor.py b/tests/minikube/test_kubernetes_executor.py index b595243a28696f..a085db536dcbdb 100644 --- a/tests/minikube/test_kubernetes_executor.py +++ b/tests/minikube/test_kubernetes_executor.py @@ -20,10 +20,10 @@ import re import time import unittest - from subprocess import check_call, check_output -import requests.exceptions + import requests +import requests.exceptions from requests.adapters import HTTPAdapter from urllib3.util.retry import Retry diff --git a/tests/minikube/test_kubernetes_pod_operator.py b/tests/minikube/test_kubernetes_pod_operator.py index d88779ab0adfba..69687e978a451e 100644 --- a/tests/minikube/test_kubernetes_pod_operator.py +++ b/tests/minikube/test_kubernetes_pod_operator.py @@ -15,27 +15,26 @@ # specific language governing permissions and limitations # under the License. -import unittest -from unittest import mock -from unittest.mock import ANY - +import json import os import shutil -import json +import unittest from subprocess import check_call +from unittest import mock +from unittest.mock import ANY -from kubernetes.client.rest import ApiException import kubernetes.client.models as k8s from kubernetes.client.api_client import ApiClient +from kubernetes.client.rest import ApiException -from airflow.contrib.operators.kubernetes_pod_operator import KubernetesPodOperator -from airflow.kubernetes.secret import Secret from airflow import AirflowException -from airflow.kubernetes.pod_launcher import PodLauncher +from airflow.contrib.operators.kubernetes_pod_operator import KubernetesPodOperator from airflow.kubernetes.pod import Port from airflow.kubernetes.pod_generator import PodDefaults -from airflow.kubernetes.volume_mount import VolumeMount +from airflow.kubernetes.pod_launcher import PodLauncher +from airflow.kubernetes.secret import Secret from airflow.kubernetes.volume import Volume +from airflow.kubernetes.volume_mount import VolumeMount try: check_call(["/usr/local/bin/kubectl", "get", "pods"]) diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index c3139bbe50e1a9..2716422edc4225 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -19,9 +19,9 @@ import datetime import unittest -from unittest import mock import uuid from collections import namedtuple +from unittest import mock import jinja2 from parameterized import parameterized diff --git a/tests/models/test_cleartasks.py b/tests/models/test_cleartasks.py index dd69084f2f292b..3950b77901ef58 100644 --- a/tests/models/test_cleartasks.py +++ b/tests/models/test_cleartasks.py @@ -23,7 +23,7 @@ from airflow import settings from airflow.configuration import conf -from airflow.models import DAG, TaskInstance as TI, clear_task_instances, XCom +from airflow.models import DAG, TaskInstance as TI, XCom, clear_task_instances from airflow.operators.dummy_operator import DummyOperator from airflow.utils import timezone from airflow.utils.state import State diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 1416d35349e380..74f98128cb9a5d 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -31,7 +31,7 @@ from airflow import models, settings from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowDagCycleException +from airflow.exceptions import AirflowDagCycleException, AirflowException from airflow.models import DAG, DagModel, TaskInstance as TI from airflow.operators.dummy_operator import DummyOperator from airflow.operators.subdag_operator import SubDagOperator diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index 84f0eb99507a3c..e3ab7874495a4f 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -17,25 +17,25 @@ # specific language governing permissions and limitations # under the License. -from datetime import datetime, timezone, timedelta import inspect import os import shutil import textwrap import unittest -from unittest.mock import patch, ANY -from tempfile import mkdtemp, NamedTemporaryFile +from datetime import datetime, timedelta, timezone +from tempfile import NamedTemporaryFile, mkdtemp +from unittest.mock import ANY, patch +import airflow.example_dags from airflow import models from airflow.configuration import conf from airflow.jobs import LocalTaskJob as LJ -from airflow.models import DagModel, DagBag, TaskInstance as TI +from airflow.models import DagBag, DagModel, TaskInstance as TI from airflow.utils.db import create_session from airflow.utils.state import State from airflow.utils.timezone import utcnow -from tests.models import TEST_DAGS_FOLDER, DEFAULT_DATE +from tests.models import DEFAULT_DATE, TEST_DAGS_FOLDER from tests.test_utils.config import conf_vars -import airflow.example_dags class TestDagBag(unittest.TestCase): diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 8fd35ead053f68..0d9e920d791fd3 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -20,10 +20,9 @@ import datetime import unittest -from airflow import settings, models +from airflow import models, settings from airflow.jobs import BackfillJob -from airflow.models import DAG, DagRun, clear_task_instances -from airflow.models import TaskInstance as TI +from airflow.models import DAG, DagRun, TaskInstance as TI, clear_task_instances from airflow.operators.dummy_operator import DummyOperator from airflow.operators.python_operator import ShortCircuitOperator from airflow.utils import timezone diff --git a/tests/models/test_kubernetes.py b/tests/models/test_kubernetes.py index 7b0f64079f5867..405d31d5ddddb0 100644 --- a/tests/models/test_kubernetes.py +++ b/tests/models/test_kubernetes.py @@ -18,7 +18,6 @@ # under the License. import unittest - from unittest.mock import patch from airflow import settings diff --git a/tests/models/test_skipmixin.py b/tests/models/test_skipmixin.py index 0d8b2bcf2712df..148476112a63d7 100644 --- a/tests/models/test_skipmixin.py +++ b/tests/models/test_skipmixin.py @@ -19,13 +19,12 @@ import datetime import unittest +from unittest.mock import Mock, patch import pendulum -from unittest.mock import patch, Mock from airflow import settings -from airflow.models import DAG, TaskInstance as TI -from airflow.models import SkipMixin +from airflow.models import DAG, SkipMixin, TaskInstance as TI from airflow.operators.dummy_operator import DummyOperator from airflow.utils import timezone from airflow.utils.state import State diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index ebcdedb6bee29f..1b1f52c60de55e 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -20,13 +20,15 @@ import datetime import time import unittest -from unittest.mock import patch, mock_open import urllib -from typing import Union, List +from typing import List, Union +from unittest.mock import mock_open, patch + import pendulum from freezegun import freeze_time -from parameterized import parameterized, param +from parameterized import param, parameterized from sqlalchemy.orm.session import Session + from airflow import models, settings from airflow.configuration import conf from airflow.contrib.sensors.python_sensor import PythonSensor @@ -40,11 +42,10 @@ from airflow.ti_deps.deps.base_ti_dep import TIDepStatus from airflow.ti_deps.deps.trigger_rule_dep import TriggerRuleDep from airflow.utils import timezone -from airflow.utils.db import create_session +from airflow.utils.db import create_session, provide_session from airflow.utils.state import State from tests.models import DEFAULT_DATE from tests.test_utils import db -from airflow.utils.db import provide_session class TestTaskInstance(unittest.TestCase): diff --git a/tests/models/test_variable.py b/tests/models/test_variable.py index 919428d2c92ed5..c7f249091fa9d8 100644 --- a/tests/models/test_variable.py +++ b/tests/models/test_variable.py @@ -22,7 +22,7 @@ from cryptography.fernet import Fernet from airflow import settings -from airflow.models import crypto, Variable +from airflow.models import Variable, crypto from tests.test_utils.config import conf_vars diff --git a/tests/operators/test_adls_to_gcs_operator.py b/tests/operators/test_adls_to_gcs_operator.py index ccdbce6e2e81e9..df0ac2c667060d 100644 --- a/tests/operators/test_adls_to_gcs_operator.py +++ b/tests/operators/test_adls_to_gcs_operator.py @@ -19,8 +19,7 @@ import unittest -from airflow.operators.adls_to_gcs import \ - AdlsToGoogleCloudStorageOperator +from airflow.operators.adls_to_gcs import AdlsToGoogleCloudStorageOperator from tests.compat import mock TASK_ID = 'test-adls-gcs-operator' diff --git a/tests/operators/test_bash_operator.py b/tests/operators/test_bash_operator.py index de79e2f3452b3d..6a413df8f8c988 100644 --- a/tests/operators/test_bash_operator.py +++ b/tests/operators/test_bash_operator.py @@ -21,12 +21,12 @@ import unittest from datetime import datetime, timedelta from tempfile import NamedTemporaryFile -from tests.compat import mock from airflow import DAG, configuration from airflow.operators.bash_operator import BashOperator from airflow.utils import timezone from airflow.utils.state import State +from tests.compat import mock DEFAULT_DATE = datetime(2016, 1, 1, tzinfo=timezone.utc) END_DATE = datetime(2016, 1, 2, tzinfo=timezone.utc) diff --git a/tests/operators/test_branch_operator.py b/tests/operators/test_branch_operator.py index 864b0a6abe5073..bde83d24cbcab9 100644 --- a/tests/operators/test_branch_operator.py +++ b/tests/operators/test_branch_operator.py @@ -17,12 +17,12 @@ # specific language governing permissions and limitations # under the License. -import unittest import datetime +import unittest -from airflow.models import TaskInstance as TI, DAG, DagRun -from airflow.operators.dummy_operator import DummyOperator +from airflow.models import DAG, DagRun, TaskInstance as TI from airflow.operators.branch_operator import BaseBranchOperator +from airflow.operators.dummy_operator import DummyOperator from airflow.utils import timezone from airflow.utils.db import create_session from airflow.utils.state import State diff --git a/tests/operators/test_cassandra_to_gcs.py b/tests/operators/test_cassandra_to_gcs.py index a950f20dcb9a2e..1a10228f1035fa 100644 --- a/tests/operators/test_cassandra_to_gcs.py +++ b/tests/operators/test_cassandra_to_gcs.py @@ -19,11 +19,10 @@ import unittest from unittest import mock + from mock import call -from airflow.operators.cassandra_to_gcs import ( - CassandraToGoogleCloudStorageOperator, -) +from airflow.operators.cassandra_to_gcs import CassandraToGoogleCloudStorageOperator TMP_FILE_NAME = "temp-file" diff --git a/tests/operators/test_check_operator.py b/tests/operators/test_check_operator.py index d95cb3ab54c440..5ff5af6fce297d 100644 --- a/tests/operators/test_check_operator.py +++ b/tests/operators/test_check_operator.py @@ -22,7 +22,7 @@ from airflow.exceptions import AirflowException from airflow.models import DAG -from airflow.operators.check_operator import ValueCheckOperator, CheckOperator, IntervalCheckOperator +from airflow.operators.check_operator import CheckOperator, IntervalCheckOperator, ValueCheckOperator from tests.compat import mock diff --git a/tests/operators/test_docker_operator.py b/tests/operators/test_docker_operator.py index 68b8450d9dad57..4ae40fff1c06ca 100644 --- a/tests/operators/test_docker_operator.py +++ b/tests/operators/test_docker_operator.py @@ -17,10 +17,12 @@ # specific language governing permissions and limitations # under the License. -import unittest import logging +import unittest from airflow.exceptions import AirflowException +from tests.compat import mock + try: from airflow.operators.docker_operator import DockerOperator from airflow.hooks.docker_hook import DockerHook @@ -28,8 +30,6 @@ except ImportError: pass -from tests.compat import mock - class TestDockerOperator(unittest.TestCase): @mock.patch('airflow.utils.file.mkdtemp') diff --git a/tests/operators/test_docker_swarm_operator.py b/tests/operators/test_docker_swarm_operator.py index 74c5dfe8a37301..1e3046409464d6 100644 --- a/tests/operators/test_docker_swarm_operator.py +++ b/tests/operators/test_docker_swarm_operator.py @@ -20,10 +20,10 @@ import unittest from docker import APIClient -from tests.compat import mock from airflow.contrib.operators.docker_swarm_operator import DockerSwarmOperator from airflow.exceptions import AirflowException +from tests.compat import mock class TestDockerSwarmOperator(unittest.TestCase): diff --git a/tests/operators/test_druid_check_operator.py b/tests/operators/test_druid_check_operator.py index 6e7979f15d612f..43d13cc74889e8 100644 --- a/tests/operators/test_druid_check_operator.py +++ b/tests/operators/test_druid_check_operator.py @@ -18,11 +18,11 @@ # under the License. -from datetime import datetime import unittest +from datetime import datetime -from airflow.models import DAG from airflow.exceptions import AirflowException +from airflow.models import DAG from airflow.operators.druid_check_operator import DruidCheckOperator from tests.compat import mock diff --git a/tests/operators/test_email_operator.py b/tests/operators/test_email_operator.py index c6961b29f47c76..521bb6234965a9 100644 --- a/tests/operators/test_email_operator.py +++ b/tests/operators/test_email_operator.py @@ -18,8 +18,8 @@ # under the License. import datetime -from unittest import mock import unittest +from unittest import mock from airflow import DAG from airflow.operators.email_operator import EmailOperator diff --git a/tests/operators/test_gcs_to_gcs.py b/tests/operators/test_gcs_to_gcs.py index de05189f14daff..ee7d1c07f3488b 100644 --- a/tests/operators/test_gcs_to_gcs.py +++ b/tests/operators/test_gcs_to_gcs.py @@ -20,9 +20,10 @@ import unittest from datetime import datetime -from airflow.operators.gcs_to_gcs import \ - GoogleCloudStorageToGoogleCloudStorageOperator, WILDCARD, GoogleCloudStorageSynchronizeBuckets from airflow.exceptions import AirflowException +from airflow.operators.gcs_to_gcs import ( + WILDCARD, GoogleCloudStorageSynchronizeBuckets, GoogleCloudStorageToGoogleCloudStorageOperator, +) from tests.compat import mock, patch TASK_ID = 'test-gcs-to-gcs-operator' diff --git a/tests/operators/test_gcs_to_gcs_operator_system.py b/tests/operators/test_gcs_to_gcs_operator_system.py index 78fbd0f3a795b0..82cdc6b36e0e26 100644 --- a/tests/operators/test_gcs_to_gcs_operator_system.py +++ b/tests/operators/test_gcs_to_gcs_operator_system.py @@ -19,10 +19,11 @@ """System tests for Google Cloud Build operators""" import unittest -from tests.operators.test_gcs_to_gcs_system_helper import GcsToGcsTestHelper -from tests.gcp.utils.base_gcp_system_test_case import SKIP_TEST_WARNING, TestDagGcpSystem, \ - OPERATORS_EXAMPLES_DAG_FOLDER +from tests.gcp.utils.base_gcp_system_test_case import ( + OPERATORS_EXAMPLES_DAG_FOLDER, SKIP_TEST_WARNING, TestDagGcpSystem, +) from tests.gcp.utils.gcp_authenticator import GCP_GCS_KEY +from tests.operators.test_gcs_to_gcs_system_helper import GcsToGcsTestHelper @unittest.skipIf(TestDagGcpSystem.skip_check(GCP_GCS_KEY), SKIP_TEST_WARNING) diff --git a/tests/operators/test_gcs_to_gcs_system_helper.py b/tests/operators/test_gcs_to_gcs_system_helper.py index 084e2acf7ab2c8..b95646f795a612 100755 --- a/tests/operators/test_gcs_to_gcs_system_helper.py +++ b/tests/operators/test_gcs_to_gcs_system_helper.py @@ -24,15 +24,10 @@ import os from airflow.example_dags.example_gcs_to_gcs import ( - BUCKET_1_SRC, - BUCKET_1_DST, - BUCKET_2_SRC, - BUCKET_2_DST, - BUCKET_3_SRC, - BUCKET_3_DST, + BUCKET_1_DST, BUCKET_1_SRC, BUCKET_2_DST, BUCKET_2_SRC, BUCKET_3_DST, BUCKET_3_SRC, ) -from tests.gcp.utils.gcp_authenticator import GcpAuthenticator, GCP_GCS_KEY from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor +from tests.gcp.utils.gcp_authenticator import GCP_GCS_KEY, GcpAuthenticator GCP_PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project") diff --git a/tests/operators/test_gcs_to_s3.py b/tests/operators/test_gcs_to_s3.py index 95f8c1499ee6df..4e9d2d2a8df5a0 100644 --- a/tests/operators/test_gcs_to_s3.py +++ b/tests/operators/test_gcs_to_s3.py @@ -19,8 +19,8 @@ import unittest -from airflow.operators.gcs_to_s3 import GoogleCloudStorageToS3Operator from airflow.hooks.S3_hook import S3Hook +from airflow.operators.gcs_to_s3 import GoogleCloudStorageToS3Operator from tests.compat import mock try: diff --git a/tests/operators/test_google_api_to_s3_transfer.py b/tests/operators/test_google_api_to_s3_transfer.py index 7034008cc7e7d6..14cea11c017351 100644 --- a/tests/operators/test_google_api_to_s3_transfer.py +++ b/tests/operators/test_google_api_to_s3_transfer.py @@ -18,11 +18,11 @@ # under the License. # import unittest -from unittest.mock import patch, Mock +from unittest.mock import Mock, patch from airflow import configuration, models -from airflow.operators.google_api_to_s3_transfer import GoogleApiToS3Transfer from airflow.models.xcom import MAX_XCOM_SIZE +from airflow.operators.google_api_to_s3_transfer import GoogleApiToS3Transfer from airflow.utils import db diff --git a/tests/operators/test_hive_operator.py b/tests/operators/test_hive_operator.py index 833e5207b2d00c..c7e943600ec058 100644 --- a/tests/operators/test_hive_operator.py +++ b/tests/operators/test_hive_operator.py @@ -20,8 +20,8 @@ import datetime import os import unittest - from unittest import mock + import nose from airflow import DAG, operators @@ -30,7 +30,6 @@ from airflow.operators.hive_operator import HiveOperator from airflow.utils import timezone - DEFAULT_DATE = datetime.datetime(2015, 1, 1) DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat() DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10] diff --git a/tests/operators/test_hive_stats_operator.py b/tests/operators/test_hive_stats_operator.py index 2a0643c13e34f0..a8841ca715699c 100644 --- a/tests/operators/test_hive_stats_operator.py +++ b/tests/operators/test_hive_stats_operator.py @@ -18,9 +18,8 @@ # under the License. import unittest -from unittest.mock import patch from collections import OrderedDict - +from unittest.mock import patch from airflow import AirflowException from airflow.operators.hive_stats_operator import HiveStatsCollectionOperator diff --git a/tests/operators/test_hive_to_druid.py b/tests/operators/test_hive_to_druid.py index e0c2b5bd71675b..50dce3eb5e52b4 100644 --- a/tests/operators/test_hive_to_druid.py +++ b/tests/operators/test_hive_to_druid.py @@ -18,9 +18,10 @@ # under the License. # +import unittest + import requests import requests_mock -import unittest from airflow import DAG from airflow.operators.hive_to_druid import HiveToDruidTransfer diff --git a/tests/operators/test_hive_to_mysql.py b/tests/operators/test_hive_to_mysql.py index a3b4e74672a8f7..85f26c46a35820 100644 --- a/tests/operators/test_hive_to_mysql.py +++ b/tests/operators/test_hive_to_mysql.py @@ -18,8 +18,7 @@ # under the License. import unittest - -from unittest.mock import patch, PropertyMock +from unittest.mock import PropertyMock, patch from airflow.operators.hive_to_mysql import HiveToMySqlTransfer from airflow.utils.operator_helpers import context_to_airflow_vars diff --git a/tests/operators/test_hive_to_samba_operator.py b/tests/operators/test_hive_to_samba_operator.py index 916e05ee1bab53..c41cbe60488133 100644 --- a/tests/operators/test_hive_to_samba_operator.py +++ b/tests/operators/test_hive_to_samba_operator.py @@ -18,8 +18,7 @@ # under the License. import unittest - -from unittest.mock import patch, PropertyMock, Mock +from unittest.mock import Mock, PropertyMock, patch from airflow.operators.hive_to_samba_operator import Hive2SambaOperator from airflow.utils.operator_helpers import context_to_airflow_vars diff --git a/tests/operators/test_http_operator.py b/tests/operators/test_http_operator.py index 3aeedff2ae2a40..1555d44f3d667e 100644 --- a/tests/operators/test_http_operator.py +++ b/tests/operators/test_http_operator.py @@ -21,8 +21,9 @@ import unittest import requests_mock -from airflow.operators.http_operator import SimpleHttpOperator + from airflow.exceptions import AirflowException +from airflow.operators.http_operator import SimpleHttpOperator from tests.compat import mock diff --git a/tests/operators/test_jdbc_operator.py b/tests/operators/test_jdbc_operator.py index 51cdc3736bad2e..a90b045f71f3e5 100644 --- a/tests/operators/test_jdbc_operator.py +++ b/tests/operators/test_jdbc_operator.py @@ -18,7 +18,6 @@ # under the License. import unittest - from unittest.mock import patch from airflow.operators.jdbc_operator import JdbcOperator diff --git a/tests/operators/test_latest_only_operator.py b/tests/operators/test_latest_only_operator.py index 94d36a29db8663..bbd7a7a582241c 100644 --- a/tests/operators/test_latest_only_operator.py +++ b/tests/operators/test_latest_only_operator.py @@ -24,8 +24,8 @@ from airflow import DAG, settings from airflow.models import TaskInstance -from airflow.operators.latest_only_operator import LatestOnlyOperator from airflow.operators.dummy_operator import DummyOperator +from airflow.operators.latest_only_operator import LatestOnlyOperator from airflow.utils import timezone from airflow.utils.state import State diff --git a/tests/operators/test_mssql_to_gcs.py b/tests/operators/test_mssql_to_gcs.py index fc729eec62d400..bf94fc707d50be 100644 --- a/tests/operators/test_mssql_to_gcs.py +++ b/tests/operators/test_mssql_to_gcs.py @@ -19,8 +19,7 @@ import unittest -from airflow.operators.mssql_to_gcs import \ - MsSqlToGoogleCloudStorageOperator +from airflow.operators.mssql_to_gcs import MsSqlToGoogleCloudStorageOperator from tests.compat import mock TASK_ID = 'test-mssql-to-gcs' diff --git a/tests/operators/test_mssql_to_hive.py b/tests/operators/test_mssql_to_hive.py index 2dc37ebeccadf3..40e06802306ba0 100644 --- a/tests/operators/test_mssql_to_hive.py +++ b/tests/operators/test_mssql_to_hive.py @@ -18,16 +18,16 @@ # under the License. import unittest -from unittest.mock import patch, PropertyMock, Mock from collections import OrderedDict +from unittest.mock import Mock, PropertyMock, patch + +from airflow.operators.mssql_to_hive import MsSqlToHiveTransfer try: import pymssql except ImportError: pymssql = None -from airflow.operators.mssql_to_hive import MsSqlToHiveTransfer - @unittest.skipIf(pymssql is None, 'pymssql package not present') class TestMsSqlToHiveTransfer(unittest.TestCase): diff --git a/tests/operators/test_mysql_to_gcs.py b/tests/operators/test_mysql_to_gcs.py index 077134b249a0fd..dd4d394d4e40c5 100644 --- a/tests/operators/test_mysql_to_gcs.py +++ b/tests/operators/test_mysql_to_gcs.py @@ -21,12 +21,10 @@ import decimal import unittest -from parameterized import parameterized - from _mysql_exceptions import ProgrammingError +from parameterized import parameterized -from airflow.operators.mysql_to_gcs import \ - MySqlToGoogleCloudStorageOperator +from airflow.operators.mysql_to_gcs import MySqlToGoogleCloudStorageOperator from tests.compat import mock TASK_ID = 'test-mysql-to-gcs' diff --git a/tests/operators/test_operators.py b/tests/operators/test_operators.py index 7e54bb30fb3ab9..e2612ccd1101cf 100644 --- a/tests/operators/test_operators.py +++ b/tests/operators/test_operators.py @@ -17,14 +17,13 @@ # specific language governing permissions and limitations # under the License. -from airflow import DAG, configuration, operators -from airflow.utils import timezone - -from collections import OrderedDict - import os -from unittest import mock import unittest +from collections import OrderedDict +from unittest import mock + +from airflow import DAG, configuration, operators +from airflow.utils import timezone DEFAULT_DATE = timezone.datetime(2015, 1, 1) DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat() diff --git a/tests/operators/test_papermill_operator.py b/tests/operators/test_papermill_operator.py index f9123b08ffa3f4..9236ab12a92886 100644 --- a/tests/operators/test_papermill_operator.py +++ b/tests/operators/test_papermill_operator.py @@ -17,7 +17,6 @@ # specific language governing permissions and limitations # under the License. import unittest - from unittest.mock import patch from airflow.operators.papermill_operator import PapermillOperator diff --git a/tests/operators/test_postgres_to_gcs.py b/tests/operators/test_postgres_to_gcs.py index 8ba160f3c3d0e1..56276442e1838f 100644 --- a/tests/operators/test_postgres_to_gcs.py +++ b/tests/operators/test_postgres_to_gcs.py @@ -20,8 +20,7 @@ import unittest from airflow.hooks.postgres_hook import PostgresHook -from airflow.operators.postgres_to_gcs import \ - PostgresToGoogleCloudStorageOperator +from airflow.operators.postgres_to_gcs import PostgresToGoogleCloudStorageOperator from tests.compat import patch TABLES = {'postgres_to_gcs_operator', 'postgres_to_gcs_operator_empty'} diff --git a/tests/operators/test_presto_to_mysql.py b/tests/operators/test_presto_to_mysql.py index b650d4de4e080c..67ed4a8d2a8dcd 100644 --- a/tests/operators/test_presto_to_mysql.py +++ b/tests/operators/test_presto_to_mysql.py @@ -18,7 +18,6 @@ # under the License. import unittest - from unittest.mock import patch from airflow.operators.presto_to_mysql import PrestoToMySqlTransfer diff --git a/tests/operators/test_python_operator.py b/tests/operators/test_python_operator.py index 497a0019392627..7cbe080b85162c 100644 --- a/tests/operators/test_python_operator.py +++ b/tests/operators/test_python_operator.py @@ -22,13 +22,12 @@ import os import unittest from collections import namedtuple -from datetime import timedelta, date +from datetime import date, timedelta from airflow.exceptions import AirflowException -from airflow.models import TaskInstance as TI, DAG, DagRun +from airflow.models import DAG, DagRun, TaskInstance as TI from airflow.operators.dummy_operator import DummyOperator -from airflow.operators.python_operator import PythonOperator, BranchPythonOperator -from airflow.operators.python_operator import ShortCircuitOperator +from airflow.operators.python_operator import BranchPythonOperator, PythonOperator, ShortCircuitOperator from airflow.utils import timezone from airflow.utils.db import create_session from airflow.utils.state import State diff --git a/tests/operators/test_redshift_to_s3_operator.py b/tests/operators/test_redshift_to_s3_operator.py index baf2afb7e98435..8b3903271bb45e 100644 --- a/tests/operators/test_redshift_to_s3_operator.py +++ b/tests/operators/test_redshift_to_s3_operator.py @@ -18,10 +18,11 @@ # under the License. # -from unittest import mock import unittest +from unittest import mock from boto3.session import Session + from airflow.operators.redshift_to_s3_operator import RedshiftToS3Transfer from airflow.utils.tests import assertEqualIgnoreMultipleSpaces diff --git a/tests/operators/test_s3_file_transform_operator.py b/tests/operators/test_s3_file_transform_operator.py index 95bec2e630d5b4..f7dbed2f6a2bc2 100644 --- a/tests/operators/test_s3_file_transform_operator.py +++ b/tests/operators/test_s3_file_transform_operator.py @@ -24,11 +24,10 @@ import shutil import sys import unittest -from unittest import mock from tempfile import mkdtemp +from unittest import mock import boto3 - from moto import mock_s3 from airflow.exceptions import AirflowException diff --git a/tests/operators/test_s3_to_hive_operator.py b/tests/operators/test_s3_to_hive_operator.py index ac36ea2a546c8f..d1241981bc5600 100644 --- a/tests/operators/test_s3_to_hive_operator.py +++ b/tests/operators/test_s3_to_hive_operator.py @@ -17,20 +17,20 @@ # specific language governing permissions and limitations # under the License. -import unittest - -from tests.compat import mock +import bz2 +import errno +import filecmp import logging -from itertools import product -from airflow.operators.s3_to_hive_operator import S3ToHiveTransfer +import shutil +import unittest from collections import OrderedDict -from airflow.exceptions import AirflowException -from tempfile import NamedTemporaryFile, mkdtemp from gzip import GzipFile -import bz2 -import shutil -import filecmp -import errno +from itertools import product +from tempfile import NamedTemporaryFile, mkdtemp + +from airflow.exceptions import AirflowException +from airflow.operators.s3_to_hive_operator import S3ToHiveTransfer +from tests.compat import mock try: import boto3 diff --git a/tests/operators/test_s3_to_redshift_operator.py b/tests/operators/test_s3_to_redshift_operator.py index 5f3bd444db5410..28c8cc137fa9ab 100644 --- a/tests/operators/test_s3_to_redshift_operator.py +++ b/tests/operators/test_s3_to_redshift_operator.py @@ -18,10 +18,11 @@ # under the License. # -from unittest import mock import unittest +from unittest import mock from boto3.session import Session + from airflow.operators.s3_to_redshift_operator import S3ToRedshiftTransfer from airflow.utils.tests import assertEqualIgnoreMultipleSpaces diff --git a/tests/operators/test_slack_operator.py b/tests/operators/test_slack_operator.py index 61836fdf692878..617bb9531ace3f 100644 --- a/tests/operators/test_slack_operator.py +++ b/tests/operators/test_slack_operator.py @@ -17,8 +17,9 @@ # specific language governing permissions and limitations # under the License. -import unittest import json +import unittest + from airflow.exceptions import AirflowException from airflow.operators.slack_operator import SlackAPIPostOperator from tests.compat import mock diff --git a/tests/operators/test_subdag_operator.py b/tests/operators/test_subdag_operator.py index af50190b2b80e7..044bdee5bf977d 100644 --- a/tests/operators/test_subdag_operator.py +++ b/tests/operators/test_subdag_operator.py @@ -25,12 +25,11 @@ from airflow.models import DAG, DagRun, TaskInstance from airflow.operators.dummy_operator import DummyOperator from airflow.operators.subdag_operator import SubDagOperator +from airflow.utils.db import create_session from airflow.utils.state import State from airflow.utils.timezone import datetime -from airflow.utils.db import create_session from tests.test_utils.db import clear_db_runs - DEFAULT_DATE = datetime(2016, 1, 1) default_args = dict( diff --git a/tests/operators/test_virtualenv_operator.py b/tests/operators/test_virtualenv_operator.py index 97c3dcc4eb473f..d59804dcd91e6d 100644 --- a/tests/operators/test_virtualenv_operator.py +++ b/tests/operators/test_virtualenv_operator.py @@ -18,19 +18,17 @@ # under the License. import datetime - -import funcsigs import sys import unittest - from subprocess import CalledProcessError +import funcsigs + from airflow import DAG +from airflow.exceptions import AirflowException from airflow.operators.python_operator import PythonVirtualenvOperator from airflow.utils import timezone -from airflow.exceptions import AirflowException - DEFAULT_DATE = timezone.datetime(2016, 1, 1) END_DATE = timezone.datetime(2016, 1, 2) INTERVAL = datetime.timedelta(hours=12) diff --git a/tests/plugins/test_plugin.py b/tests/plugins/test_plugin.py index 4e4ae620fd27b5..f7999beeb6bc4a 100644 --- a/tests/plugins/test_plugin.py +++ b/tests/plugins/test_plugin.py @@ -17,17 +17,16 @@ # specific language governing permissions and limitations # under the License. -# This is the class you derive to create a plugin -from airflow.plugins_manager import AirflowPlugin - from flask import Blueprint -from flask_appbuilder import expose, BaseView as AppBuilderBaseView +from flask_appbuilder import BaseView as AppBuilderBaseView, expose +from airflow.executors.base_executor import BaseExecutor # Importing base classes that we need to derive from airflow.hooks.base_hook import BaseHook -from airflow.models.baseoperator import BaseOperatorLink, BaseOperator +from airflow.models.baseoperator import BaseOperator, BaseOperatorLink +# This is the class you derive to create a plugin +from airflow.plugins_manager import AirflowPlugin from airflow.sensors.base_sensor_operator import BaseSensorOperator -from airflow.executors.base_executor import BaseExecutor # Will show up under airflow.hooks.test_plugin.PluginHook diff --git a/tests/plugins/test_plugins_manager.py b/tests/plugins/test_plugins_manager.py index b6830842c91b25..975977566dd19a 100644 --- a/tests/plugins/test_plugins_manager.py +++ b/tests/plugins/test_plugins_manager.py @@ -19,7 +19,6 @@ import unittest - from airflow.www import app as application diff --git a/tests/security/test_kerberos.py b/tests/security/test_kerberos.py index c5aeef20aabe4c..87fc61224c9644 100644 --- a/tests/security/test_kerberos.py +++ b/tests/security/test_kerberos.py @@ -21,9 +21,9 @@ import unittest from argparse import Namespace +from airflow import LoggingMixin from airflow.configuration import conf from airflow.security.kerberos import renew_from_kt -from airflow import LoggingMixin from tests.test_utils.config import conf_vars diff --git a/tests/sensors/test_base_sensor.py b/tests/sensors/test_base_sensor.py index 7be6e786e3c2eb..d5b92ee8b09904 100644 --- a/tests/sensors/test_base_sensor.py +++ b/tests/sensors/test_base_sensor.py @@ -18,11 +18,14 @@ # under the License. import unittest +from datetime import timedelta +from time import sleep from unittest.mock import Mock +from freezegun import freeze_time + from airflow import DAG, settings -from airflow.exceptions import (AirflowSensorTimeout, AirflowException, - AirflowRescheduleException) +from airflow.exceptions import AirflowException, AirflowRescheduleException, AirflowSensorTimeout from airflow.models import DagRun, TaskInstance, TaskReschedule from airflow.operators.dummy_operator import DummyOperator from airflow.sensors.base_sensor_operator import BaseSensorOperator @@ -30,9 +33,6 @@ from airflow.utils import timezone from airflow.utils.state import State from airflow.utils.timezone import datetime -from datetime import timedelta -from time import sleep -from freezegun import freeze_time DEFAULT_DATE = datetime(2015, 1, 1) TEST_DAG_ID = 'unit_test_dag' diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index f479627b5aadfa..4a84b97fc5e2fa 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -17,12 +17,11 @@ # specific language governing permissions and limitations # under the License. import unittest -from datetime import timedelta, time +from datetime import time, timedelta -from airflow import DAG, settings -from airflow import exceptions +from airflow import DAG, exceptions, settings from airflow.exceptions import AirflowException, AirflowSensorTimeout -from airflow.models import TaskInstance, DagBag +from airflow.models import DagBag, TaskInstance from airflow.operators.bash_operator import BashOperator from airflow.operators.dummy_operator import DummyOperator from airflow.sensors.external_task_sensor import ExternalTaskSensor diff --git a/tests/sensors/test_hdfs_sensor.py b/tests/sensors/test_hdfs_sensor.py index 3ad1ed1f31564e..b4f9c2676bde57 100644 --- a/tests/sensors/test_hdfs_sensor.py +++ b/tests/sensors/test_hdfs_sensor.py @@ -17,7 +17,6 @@ # specific language governing permissions and limitations # under the License. import unittest - from datetime import timedelta from airflow.exceptions import AirflowSensorTimeout diff --git a/tests/sensors/test_http_sensor.py b/tests/sensors/test_http_sensor.py index 1b6c3667ba6531..f741d96a7be5ca 100644 --- a/tests/sensors/test_http_sensor.py +++ b/tests/sensors/test_http_sensor.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. import unittest +from unittest.mock import patch import requests -from unittest.mock import patch from airflow import DAG from airflow.exceptions import AirflowException, AirflowSensorTimeout diff --git a/tests/sensors/test_named_hive_partition_sensor.py b/tests/sensors/test_named_hive_partition_sensor.py index 045fa95622414e..2e5a1a0537362a 100644 --- a/tests/sensors/test_named_hive_partition_sensor.py +++ b/tests/sensors/test_named_hive_partition_sensor.py @@ -21,9 +21,9 @@ from datetime import timedelta from airflow import DAG, operators +from airflow.hooks.hive_hooks import HiveMetastoreHook from airflow.sensors.named_hive_partition_sensor import NamedHivePartitionSensor from airflow.utils.timezone import datetime -from airflow.hooks.hive_hooks import HiveMetastoreHook DEFAULT_DATE = datetime(2015, 1, 1) DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat() diff --git a/tests/sensors/test_s3_key_sensor.py b/tests/sensors/test_s3_key_sensor.py index 053a83f008a7f1..70ebfebf389a8d 100644 --- a/tests/sensors/test_s3_key_sensor.py +++ b/tests/sensors/test_s3_key_sensor.py @@ -17,8 +17,9 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock + from parameterized import parameterized from airflow.exceptions import AirflowException diff --git a/tests/sensors/test_s3_prefix_sensor.py b/tests/sensors/test_s3_prefix_sensor.py index 21ff5ee5272a97..f7a5bfa87d6172 100644 --- a/tests/sensors/test_s3_prefix_sensor.py +++ b/tests/sensors/test_s3_prefix_sensor.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock from airflow.sensors.s3_prefix_sensor import S3PrefixSensor diff --git a/tests/sensors/test_sql_sensor.py b/tests/sensors/test_sql_sensor.py index c7987970d7101e..e11bf56030e7e5 100644 --- a/tests/sensors/test_sql_sensor.py +++ b/tests/sensors/test_sql_sensor.py @@ -16,8 +16,8 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock from airflow import DAG from airflow.configuration import conf diff --git a/tests/sensors/test_timedelta_sensor.py b/tests/sensors/test_timedelta_sensor.py index 1ce1aed720ba4d..3e84f0a16d9876 100644 --- a/tests/sensors/test_timedelta_sensor.py +++ b/tests/sensors/test_timedelta_sensor.py @@ -17,10 +17,9 @@ # specific language governing permissions and limitations # under the License. import unittest - from datetime import timedelta -from airflow import models, DAG +from airflow import DAG, models from airflow.sensors.time_delta_sensor import TimeDeltaSensor from airflow.utils.timezone import datetime diff --git a/tests/sensors/test_timeout_sensor.py b/tests/sensors/test_timeout_sensor.py index 438b9329a5f278..340c16d952eb02 100644 --- a/tests/sensors/test_timeout_sensor.py +++ b/tests/sensors/test_timeout_sensor.py @@ -16,9 +16,8 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import unittest - import time +import unittest from datetime import timedelta from airflow import DAG diff --git a/tests/task/task_runner/test_standard_task_runner.py b/tests/task/task_runner/test_standard_task_runner.py index 41fed206df024c..0c5e79bfd54791 100644 --- a/tests/task/task_runner/test_standard_task_runner.py +++ b/tests/task/task_runner/test_standard_task_runner.py @@ -17,20 +17,19 @@ # specific language governing permissions and limitations # under the License. import os -import psutil import time import unittest +from logging.config import dictConfig from unittest import mock +import psutil + from airflow import models, settings from airflow.jobs import LocalTaskJob from airflow.models import TaskInstance as TI from airflow.task.task_runner import StandardTaskRunner from airflow.utils import timezone from airflow.utils.state import State - -from logging.config import dictConfig - from tests.core import TEST_DAG_FOLDER DEFAULT_DATE = timezone.datetime(2016, 1, 1) diff --git a/tests/test_configuration.py b/tests/test_configuration.py index 2f1066ef29a719..182158a0f9c55a 100644 --- a/tests/test_configuration.py +++ b/tests/test_configuration.py @@ -19,14 +19,13 @@ import contextlib import os +import unittest import warnings from collections import OrderedDict +from unittest import mock from airflow import configuration -from airflow.configuration import conf, AirflowConfigParser, parameterized_config - -import unittest -from unittest import mock +from airflow.configuration import AirflowConfigParser, conf, parameterized_config @contextlib.contextmanager diff --git a/tests/test_core_to_contrib.py b/tests/test_core_to_contrib.py index 42fd340ec50ec2..afd3a8f7d1dfdc 100644 --- a/tests/test_core_to_contrib.py +++ b/tests/test_core_to_contrib.py @@ -16,9 +16,8 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from unittest import TestCase import importlib - +from unittest import TestCase from parameterized import parameterized diff --git a/tests/test_impersonation.py b/tests/test_impersonation.py index 31c84096f45674..659efbd4f38fd5 100644 --- a/tests/test_impersonation.py +++ b/tests/test_impersonation.py @@ -18,10 +18,10 @@ # under the License. import errno +import logging import os import subprocess import unittest -import logging from airflow import jobs, models from airflow.utils.db import add_default_pool_if_not_exists diff --git a/tests/test_local_settings.py b/tests/test_local_settings.py index 6a5962f3ab03f2..ea774c20edc44f 100644 --- a/tests/test_local_settings.py +++ b/tests/test_local_settings.py @@ -23,7 +23,6 @@ import unittest from unittest.mock import MagicMock, call - SETTINGS_FILE_POLICY = """ def policy(task_instance): task_instance.run_as_user = "myself" diff --git a/tests/test_logging_config.py b/tests/test_logging_config.py index 2b31e4e40ab156..8c46532018ae21 100644 --- a/tests/test_logging_config.py +++ b/tests/test_logging_config.py @@ -22,13 +22,12 @@ import pathlib import sys import tempfile +import unittest from airflow.configuration import conf from tests.compat import patch from tests.test_utils.config import conf_vars -import unittest - SETTINGS_FILE_VALID = """ LOGGING_CONFIG = { 'version': 1, diff --git a/tests/test_sentry.py b/tests/test_sentry.py index f4d39056340274..6c6d11ec85e3f3 100644 --- a/tests/test_sentry.py +++ b/tests/test_sentry.py @@ -19,18 +19,17 @@ import datetime import unittest -from unittest.mock import Mock, MagicMock -from freezegun import freeze_time +from unittest.mock import MagicMock, Mock +from freezegun import freeze_time from sentry_sdk import configure_scope from airflow.models import TaskInstance -from airflow.settings import Session from airflow.sentry import ConfiguredSentry +from airflow.settings import Session from airflow.utils import timezone from airflow.utils.state import State - EXECUTION_DATE = timezone.utcnow() DAG_ID = "test_dag" TASK_ID = "test_task" diff --git a/tests/test_stats.py b/tests/test_stats.py index 3f44c24e35c4e4..297bf1cc6d61a2 100644 --- a/tests/test_stats.py +++ b/tests/test_stats.py @@ -18,10 +18,10 @@ # under the License. import unittest - -from airflow.stats import SafeStatsdLogger, AllowListValidator from unittest.mock import Mock +from airflow.stats import AllowListValidator, SafeStatsdLogger + class TestStats(unittest.TestCase): diff --git a/tests/test_utils/config.py b/tests/test_utils/config.py index 0b8687da7c9ae1..06a0c8658ae30b 100644 --- a/tests/test_utils/config.py +++ b/tests/test_utils/config.py @@ -20,8 +20,8 @@ import contextlib import os -from airflow.configuration import conf from airflow import settings +from airflow.configuration import conf @contextlib.contextmanager diff --git a/tests/test_utils/db.py b/tests/test_utils/db.py index c0f4b927c968d5..d967a23eeb9c6b 100644 --- a/tests/test_utils/db.py +++ b/tests/test_utils/db.py @@ -16,9 +16,8 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from airflow.models import DagModel, DagRun, errors, Pool, SlaMiss, TaskInstance -from airflow.utils.db import add_default_pool_if_not_exists -from airflow.utils.db import create_session +from airflow.models import DagModel, DagRun, Pool, SlaMiss, TaskInstance, errors +from airflow.utils.db import add_default_pool_if_not_exists, create_session def clear_db_runs(): diff --git a/tests/test_utils/reset_warning_registry.py b/tests/test_utils/reset_warning_registry.py index 46fdc3df37978d..7f71e0873471d3 100644 --- a/tests/test_utils/reset_warning_registry.py +++ b/tests/test_utils/reset_warning_registry.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. -from typing import Optional, Dict, Match import re import sys +from typing import Dict, Match, Optional # We need to explicitly clear the warning registry context diff --git a/tests/ti_deps/deps/test_dagrun_exists_dep.py b/tests/ti_deps/deps/test_dagrun_exists_dep.py index 42e5cde6b58f00..23dfae1868bfcc 100644 --- a/tests/ti_deps/deps/test_dagrun_exists_dep.py +++ b/tests/ti_deps/deps/test_dagrun_exists_dep.py @@ -18,11 +18,11 @@ # under the License. import unittest -from airflow.utils.state import State from unittest.mock import Mock, patch from airflow.models import DAG, DagRun from airflow.ti_deps.deps.dagrun_exists_dep import DagrunRunningDep +from airflow.utils.state import State class TestDagrunRunningDep(unittest.TestCase): diff --git a/tests/ti_deps/deps/test_not_in_retry_period_dep.py b/tests/ti_deps/deps/test_not_in_retry_period_dep.py index 9d587e9aecef5a..de1f574327e7c1 100644 --- a/tests/ti_deps/deps/test_not_in_retry_period_dep.py +++ b/tests/ti_deps/deps/test_not_in_retry_period_dep.py @@ -18,8 +18,9 @@ # under the License. import unittest -from unittest.mock import Mock from datetime import timedelta +from unittest.mock import Mock + from freezegun import freeze_time from airflow.models import TaskInstance diff --git a/tests/ti_deps/deps/test_pool_slots_available_dep.py b/tests/ti_deps/deps/test_pool_slots_available_dep.py index b63538e1ee0f68..ff8d1502d90ded 100644 --- a/tests/ti_deps/deps/test_pool_slots_available_dep.py +++ b/tests/ti_deps/deps/test_pool_slots_available_dep.py @@ -21,8 +21,7 @@ from mock import Mock, patch from airflow.models import Pool -from airflow.ti_deps.deps.pool_slots_available_dep import PoolSlotsAvailableDep, \ - STATES_TO_COUNT_AS_RUNNING +from airflow.ti_deps.deps.pool_slots_available_dep import STATES_TO_COUNT_AS_RUNNING, PoolSlotsAvailableDep from airflow.utils.db import create_session from tests.test_utils import db diff --git a/tests/ti_deps/deps/test_prev_dagrun_dep.py b/tests/ti_deps/deps/test_prev_dagrun_dep.py index 3ecb9c33d3533a..53cc822a58e633 100644 --- a/tests/ti_deps/deps/test_prev_dagrun_dep.py +++ b/tests/ti_deps/deps/test_prev_dagrun_dep.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest.mock import Mock from datetime import datetime +from unittest.mock import Mock from airflow.models import DAG, BaseOperator from airflow.ti_deps.dep_context import DepContext diff --git a/tests/ti_deps/deps/test_ready_to_reschedule_dep.py b/tests/ti_deps/deps/test_ready_to_reschedule_dep.py index 73b1228d411712..0408a05ed265f4 100644 --- a/tests/ti_deps/deps/test_ready_to_reschedule_dep.py +++ b/tests/ti_deps/deps/test_ready_to_reschedule_dep.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest.mock import Mock, patch from datetime import timedelta +from unittest.mock import Mock, patch from airflow.models import DAG, TaskInstance, TaskReschedule from airflow.ti_deps.dep_context import DepContext diff --git a/tests/ti_deps/deps/test_runnable_exec_date_dep.py b/tests/ti_deps/deps/test_runnable_exec_date_dep.py index 9925f2e40ad74f..796161b2c1e027 100644 --- a/tests/ti_deps/deps/test_runnable_exec_date_dep.py +++ b/tests/ti_deps/deps/test_runnable_exec_date_dep.py @@ -18,9 +18,10 @@ # under the License. import unittest -from freezegun import freeze_time from unittest.mock import Mock +from freezegun import freeze_time + from airflow.models import TaskInstance from airflow.ti_deps.deps.runnable_exec_date_dep import RunnableExecDateDep from airflow.utils.timezone import datetime diff --git a/tests/ti_deps/deps/test_task_concurrency.py b/tests/ti_deps/deps/test_task_concurrency.py index 53793ec359b442..76987ca2f72320 100644 --- a/tests/ti_deps/deps/test_task_concurrency.py +++ b/tests/ti_deps/deps/test_task_concurrency.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest.mock import Mock from datetime import datetime +from unittest.mock import Mock from airflow.models import DAG, BaseOperator from airflow.ti_deps.dep_context import DepContext diff --git a/tests/ti_deps/deps/test_trigger_rule_dep.py b/tests/ti_deps/deps/test_trigger_rule_dep.py index 7fec61ecc80e23..e86fb3e8cbea4b 100644 --- a/tests/ti_deps/deps/test_trigger_rule_dep.py +++ b/tests/ti_deps/deps/test_trigger_rule_dep.py @@ -21,10 +21,10 @@ from datetime import datetime from airflow.models import BaseOperator, TaskInstance -from airflow.utils.trigger_rule import TriggerRule from airflow.ti_deps.deps.trigger_rule_dep import TriggerRuleDep from airflow.utils.db import create_session from airflow.utils.state import State +from airflow.utils.trigger_rule import TriggerRule class TestTriggerRuleDep(unittest.TestCase): diff --git a/tests/ti_deps/deps/test_valid_state_dep.py b/tests/ti_deps/deps/test_valid_state_dep.py index 3b4499d881b65f..198cb82ca016a3 100644 --- a/tests/ti_deps/deps/test_valid_state_dep.py +++ b/tests/ti_deps/deps/test_valid_state_dep.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest.mock import Mock from datetime import datetime +from unittest.mock import Mock from airflow import AirflowException from airflow.ti_deps.deps.valid_state_dep import ValidStateDep diff --git a/tests/utils/log/elasticmock/__init__.py b/tests/utils/log/elasticmock/__init__.py index 287963368c19c8..3e5aad9bbfe667 100644 --- a/tests/utils/log/elasticmock/__init__.py +++ b/tests/utils/log/elasticmock/__init__.py @@ -43,6 +43,7 @@ from typing import Dict from elasticsearch.client import _normalize_hosts + from tests.compat import patch from .fake_elasticsearch import FakeElasticsearch diff --git a/tests/utils/log/test_es_task_handler.py b/tests/utils/log/test_es_task_handler.py index 11b58c51dfa083..64126b2096b644 100644 --- a/tests/utils/log/test_es_task_handler.py +++ b/tests/utils/log/test_es_task_handler.py @@ -17,22 +17,22 @@ # specific language governing permissions and limitations # under the License. +import logging import os import shutil import unittest -import logging +from unittest import mock import elasticsearch -from unittest import mock import pendulum -from airflow.models import TaskInstance, DAG +from airflow.configuration import conf +from airflow.models import DAG, TaskInstance from airflow.operators.dummy_operator import DummyOperator from airflow.utils import timezone from airflow.utils.log.es_task_handler import ElasticsearchTaskHandler from airflow.utils.state import State from airflow.utils.timezone import datetime -from airflow.configuration import conf from .elasticmock import elasticmock diff --git a/tests/utils/log/test_file_processor_handler.py b/tests/utils/log/test_file_processor_handler.py index 1a0949c7f48b92..07c4792e357809 100644 --- a/tests/utils/log/test_file_processor_handler.py +++ b/tests/utils/log/test_file_processor_handler.py @@ -17,14 +17,15 @@ # specific language governing permissions and limitations # under the License. -import shutil import os +import shutil import unittest +from datetime import timedelta + from freezegun import freeze_time -from airflow.utils.log.file_processor_handler import FileProcessorHandler from airflow.utils import timezone -from datetime import timedelta +from airflow.utils.log.file_processor_handler import FileProcessorHandler class TestFileProcessorHandler(unittest.TestCase): diff --git a/tests/utils/log/test_s3_task_handler.py b/tests/utils/log/test_s3_task_handler.py index 4ce9eb15ce2df6..2557d796bdca42 100644 --- a/tests/utils/log/test_s3_task_handler.py +++ b/tests/utils/log/test_s3_task_handler.py @@ -17,16 +17,16 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock -import unittest import os +import unittest +from unittest import mock +from airflow.hooks.S3_hook import S3Hook +from airflow.models import DAG, TaskInstance +from airflow.operators.dummy_operator import DummyOperator from airflow.utils.log.s3_task_handler import S3TaskHandler from airflow.utils.state import State from airflow.utils.timezone import datetime -from airflow.hooks.S3_hook import S3Hook -from airflow.models import TaskInstance, DAG -from airflow.operators.dummy_operator import DummyOperator try: import boto3 diff --git a/tests/utils/test_dag_processing.py b/tests/utils/test_dag_processing.py index 26ec144e6fd280..b1e20ff52f4794 100644 --- a/tests/utils/test_dag_processing.py +++ b/tests/utils/test_dag_processing.py @@ -17,20 +17,21 @@ # specific language governing permissions and limitations # under the License. -from datetime import (datetime, timedelta) import os import pathlib import sys import tempfile import unittest +from datetime import datetime, timedelta from unittest import mock -from unittest.mock import (MagicMock, PropertyMock) +from unittest.mock import MagicMock, PropertyMock from airflow.configuration import conf from airflow.jobs import DagFileProcessor from airflow.utils import timezone -from airflow.utils.dag_processing import (DagFileProcessorAgent, DagFileProcessorManager, - DagFileStat, correct_maybe_zipped) +from airflow.utils.dag_processing import ( + DagFileProcessorAgent, DagFileProcessorManager, DagFileStat, correct_maybe_zipped, +) TEST_DAG_FOLDER = os.path.join( os.path.dirname(os.path.realpath(__file__)), os.pardir, 'dags') diff --git a/tests/utils/test_dates.py b/tests/utils/test_dates.py index fdd73fd4a6faeb..a4e0af917501d8 100644 --- a/tests/utils/test_dates.py +++ b/tests/utils/test_dates.py @@ -17,12 +17,12 @@ # specific language governing permissions and limitations # under the License. +import unittest from datetime import datetime, timedelta + import pendulum -import unittest -from airflow.utils import dates -from airflow.utils import timezone +from airflow.utils import dates, timezone class TestDates(unittest.TestCase): diff --git a/tests/utils/test_db.py b/tests/utils/test_db.py index b762247f94c4f7..f68ed1e4d21819 100644 --- a/tests/utils/test_db.py +++ b/tests/utils/test_db.py @@ -19,13 +19,13 @@ import unittest -from airflow.models import Base as airflow_base - -from airflow.settings import engine from alembic.autogenerate import compare_metadata from alembic.migration import MigrationContext from sqlalchemy import MetaData +from airflow.models import Base as airflow_base +from airflow.settings import engine + class TestDb(unittest.TestCase): diff --git a/tests/utils/test_decorators.py b/tests/utils/test_decorators.py index 00745b6cad3482..6623ac3cf5c176 100644 --- a/tests/utils/test_decorators.py +++ b/tests/utils/test_decorators.py @@ -19,8 +19,8 @@ import unittest -from airflow.utils.decorators import apply_defaults from airflow.exceptions import AirflowException +from airflow.utils.decorators import apply_defaults # Essentially similar to airflow.models.BaseOperator diff --git a/tests/utils/test_dot_renderer.py b/tests/utils/test_dot_renderer.py index 87c930087de1ea..4ce13704d31cae 100644 --- a/tests/utils/test_dot_renderer.py +++ b/tests/utils/test_dot_renderer.py @@ -27,7 +27,6 @@ from airflow.operators.python_operator import PythonOperator from airflow.utils import dot_renderer - START_DATE = datetime.datetime.now() diff --git a/tests/utils/test_email.py b/tests/utils/test_email.py index 024bddd55c1f8c..fb91ed24b12b4e 100644 --- a/tests/utils/test_email.py +++ b/tests/utils/test_email.py @@ -18,6 +18,7 @@ # under the License. import unittest + from airflow.utils.email import get_email_address_list EMAILS = ['test1@example.com', 'test2@example.com'] diff --git a/tests/utils/test_helpers.py b/tests/utils/test_helpers.py index d5bcb715eea7fe..33850833026b3f 100644 --- a/tests/utils/test_helpers.py +++ b/tests/utils/test_helpers.py @@ -28,10 +28,10 @@ import psutil from airflow import DAG -from airflow.utils import helpers +from airflow.exceptions import AirflowException from airflow.models import TaskInstance from airflow.operators.dummy_operator import DummyOperator -from airflow.exceptions import AirflowException +from airflow.utils import helpers class TestHelpers(unittest.TestCase): diff --git a/tests/utils/test_json.py b/tests/utils/test_json.py index b314b51149ff9e..a266f8c06608af 100644 --- a/tests/utils/test_json.py +++ b/tests/utils/test_json.py @@ -17,9 +17,9 @@ # specific language governing permissions and limitations # under the License. -from datetime import datetime, date import json import unittest +from datetime import date, datetime import numpy as np diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 7126871cb5807c..cb4ca31f38dcf1 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -22,15 +22,15 @@ import os import unittest -from airflow.models import TaskInstance, DAG, DagRun from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG +from airflow.models import DAG, DagRun, TaskInstance from airflow.operators.dummy_operator import DummyOperator from airflow.operators.python_operator import PythonOperator -from airflow.utils.timezone import datetime -from airflow.utils.log.logging_mixin import set_context -from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.db import create_session +from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.logging_mixin import set_context from airflow.utils.state import State +from airflow.utils.timezone import datetime DEFAULT_DATE = datetime(2016, 1, 1) TASK_LOGGER = 'airflow.task' diff --git a/tests/utils/test_logging_mixin.py b/tests/utils/test_logging_mixin.py index 449b6b9de8b3a6..3aecc191897a36 100644 --- a/tests/utils/test_logging_mixin.py +++ b/tests/utils/test_logging_mixin.py @@ -17,11 +17,11 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest import warnings +from unittest import mock -from airflow.utils.log.logging_mixin import set_context, StreamLogWriter +from airflow.utils.log.logging_mixin import StreamLogWriter, set_context class TestLoggingMixin(unittest.TestCase): diff --git a/tests/utils/test_operator_helpers.py b/tests/utils/test_operator_helpers.py index b707713c28d3c9..d988d926099a88 100644 --- a/tests/utils/test_operator_helpers.py +++ b/tests/utils/test_operator_helpers.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest import mock from datetime import datetime +from unittest import mock from airflow.utils import operator_helpers diff --git a/tests/utils/test_tests.py b/tests/utils/test_tests.py index 793d0f77060ad9..1b95165838a9ca 100644 --- a/tests/utils/test_tests.py +++ b/tests/utils/test_tests.py @@ -18,6 +18,7 @@ # under the License. import unittest + from airflow.utils.tests import assertEqualIgnoreMultipleSpaces diff --git a/tests/utils/test_timezone.py b/tests/utils/test_timezone.py index 10446304add40d..95f766653b5083 100644 --- a/tests/utils/test_timezone.py +++ b/tests/utils/test_timezone.py @@ -18,9 +18,10 @@ # under the License. import datetime -import pendulum import unittest +import pendulum + from airflow.utils import timezone CET = pendulum.timezone("Europe/Paris") diff --git a/tests/utils/test_trigger_rule.py b/tests/utils/test_trigger_rule.py index 1167ac8c36edb9..8934cc500289cf 100644 --- a/tests/utils/test_trigger_rule.py +++ b/tests/utils/test_trigger_rule.py @@ -18,6 +18,7 @@ # under the License. import unittest + from airflow.utils.trigger_rule import TriggerRule diff --git a/tests/utils/test_weight_rule.py b/tests/utils/test_weight_rule.py index c862263387cc31..947281359aa4c0 100644 --- a/tests/utils/test_weight_rule.py +++ b/tests/utils/test_weight_rule.py @@ -18,6 +18,7 @@ # under the License. import unittest + from airflow.utils.weight_rule import WeightRule diff --git a/tests/www/api/experimental/test_endpoints.py b/tests/www/api/experimental/test_endpoints.py index e28d58db7c5de5..99fa80241bd46a 100644 --- a/tests/www/api/experimental/test_endpoints.py +++ b/tests/www/api/experimental/test_endpoints.py @@ -17,17 +17,16 @@ # specific language governing permissions and limitations # under the License. -from datetime import timedelta import json import unittest +from datetime import timedelta from urllib.parse import quote_plus - from airflow import settings from airflow.api.common.experimental.trigger_dag import trigger_dag from airflow.models import DagBag, DagRun, Pool, TaskInstance from airflow.settings import Session -from airflow.utils.timezone import datetime, utcnow, parse as parse_datetime +from airflow.utils.timezone import datetime, parse as parse_datetime, utcnow from airflow.www import app as application from tests.test_utils.db import clear_db_pools diff --git a/tests/www/api/experimental/test_kerberos_endpoints.py b/tests/www/api/experimental/test_kerberos_endpoints.py index ecd8fedf13f083..2a98b4ff334757 100644 --- a/tests/www/api/experimental/test_kerberos_endpoints.py +++ b/tests/www/api/experimental/test_kerberos_endpoints.py @@ -18,15 +18,14 @@ # under the License. import json -import unittest -from unittest import mock import os import socket - +import unittest from datetime import datetime +from unittest import mock -from airflow.configuration import conf from airflow.api.auth.backend.kerberos_auth import CLIENT_AUTH +from airflow.configuration import conf from airflow.www import app as application diff --git a/tests/www/test_security.py b/tests/www/test_security.py index a4515038db5544..285c2c1a693f4f 100644 --- a/tests/www/test_security.py +++ b/tests/www/test_security.py @@ -17,22 +17,20 @@ # specific language governing permissions and limitations # under the License. +import logging import unittest from unittest import mock -import logging from flask import Flask -from flask_appbuilder import AppBuilder, SQLA, Model, has_access, expose +from flask_appbuilder import SQLA, AppBuilder, Model, expose, has_access from flask_appbuilder.models.sqla.interface import SQLAInterface from flask_appbuilder.security.sqla import models as sqla_models -from flask_appbuilder.views import ModelView, BaseView - -from sqlalchemy import Column, Integer, String, Date, Float +from flask_appbuilder.views import BaseView, ModelView +from sqlalchemy import Column, Date, Float, Integer, String from airflow.exceptions import AirflowException from airflow.www.security import AirflowSecurityManager - logging.basicConfig(format='%(asctime)s:%(levelname)s:%(name)s:%(message)s') logging.getLogger().setLevel(logging.DEBUG) log = logging.getLogger(__name__) diff --git a/tests/www/test_utils.py b/tests/www/test_utils.py index cdd40bc7f8acc2..002d19eda03b6f 100644 --- a/tests/www/test_utils.py +++ b/tests/www/test_utils.py @@ -18,8 +18,8 @@ # under the License. import unittest -from unittest import mock from datetime import datetime +from unittest import mock from urllib.parse import parse_qs from bs4 import BeautifulSoup diff --git a/tests/www/test_validators.py b/tests/www/test_validators.py index c4e50041c5ed7d..e60a30c180f9da 100644 --- a/tests/www/test_validators.py +++ b/tests/www/test_validators.py @@ -17,8 +17,8 @@ # specific language governing permissions and limitations # under the License. -from unittest import mock import unittest +from unittest import mock from airflow.www import validators diff --git a/tests/www/test_views.py b/tests/www/test_views.py index 021d9fe8ac0f84..28767e5c96d926 100644 --- a/tests/www/test_views.py +++ b/tests/www/test_views.py @@ -26,10 +26,10 @@ import sys import tempfile import unittest -from unittest import mock import urllib -from urllib.parse import quote_plus from datetime import timedelta +from unittest import mock +from urllib.parse import quote_plus import jinja2 from flask import Markup, url_for @@ -37,11 +37,11 @@ from werkzeug.test import Client from werkzeug.wrappers import BaseResponse -from airflow.configuration import conf from airflow import models, settings from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG +from airflow.configuration import conf from airflow.jobs import BaseJob -from airflow.models import BaseOperator, Connection, DAG, DagRun, TaskInstance +from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance from airflow.models.baseoperator import BaseOperatorLink from airflow.operators.dummy_operator import DummyOperator from airflow.settings import Session