From dfad3a55d51c059c0995263ee81ee06bca99383a Mon Sep 17 00:00:00 2001 From: Josh Fell Date: Mon, 9 May 2022 15:15:58 -0400 Subject: [PATCH 1/3] Clean up f-strings in logging calls --- airflow/dag_processing/manager.py | 2 +- airflow/kubernetes/pod_launcher_deprecated.py | 5 +- airflow/models/connection.py | 4 +- airflow/models/dagrun.py | 2 +- airflow/providers/amazon/aws/operators/ecs.py | 2 +- .../amazon/aws/operators/redshift_data.py | 4 +- .../amazon/aws/operators/redshift_sql.py | 2 +- .../amazon/aws/operators/sagemaker.py | 2 +- .../amazon/aws/transfers/ftp_to_s3.py | 6 +- .../amazon/aws/transfers/salesforce_to_s3.py | 2 +- .../providers/arangodb/sensors/arangodb.py | 4 +- .../cncf/kubernetes/hooks/kubernetes.py | 2 +- .../cncf/kubernetes/utils/pod_manager.py | 5 +- .../operators/vertex_ai/endpoint_service.py | 2 +- .../google/cloud/sensors/dataproc.py | 5 +- .../google/suite/transfers/sql_to_sheets.py | 2 +- .../microsoft/azure/hooks/data_factory.py | 4 +- .../microsoft/azure/operators/data_factory.py | 10 +- airflow/providers_manager.py | 9 +- airflow/settings.py | 2 +- airflow/www/fab_security/manager.py | 55 ++-- airflow/www/fab_security/sqla/manager.py | 4 +- .../src/airflow_breeze/pre_commit_ids.py | 1 + images/breeze/output-static-checks.svg | 271 +++++++++++------- .../pre_commit_check_logging_for_fstrings.py | 102 +++++++ 25 files changed, 345 insertions(+), 164 deletions(-) create mode 100755 scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py index dcfb6bedafb90..3dc72caa99866 100644 --- a/airflow/dag_processing/manager.py +++ b/airflow/dag_processing/manager.py @@ -509,7 +509,7 @@ def _deactivate_stale_dags(self, session=None): dag.fileloc in last_parsed and (dag.last_parsed_time + self._processor_timeout) < last_parsed[dag.fileloc] ): - self.log.info(f"DAG {dag.dag_id} is missing and will be deactivated.") + self.log.info("DAG %s is missing and will be deactivated.", dag.dag_id) to_deactivate.add(dag.dag_id) if to_deactivate: diff --git a/airflow/kubernetes/pod_launcher_deprecated.py b/airflow/kubernetes/pod_launcher_deprecated.py index ccf03b1e493c2..97845dad51d5a 100644 --- a/airflow/kubernetes/pod_launcher_deprecated.py +++ b/airflow/kubernetes/pod_launcher_deprecated.py @@ -186,8 +186,9 @@ def parse_log_line(self, line: str) -> Tuple[Optional[str], str]: split_at = line.find(' ') if split_at == -1: self.log.error( - f"Error parsing timestamp (no timestamp in message: '{line}'). " - "Will continue execution but won't update timestamp" + "Error parsing timestamp (no timestamp in message: %r). " + "Will continue execution but won't update timestamp", + line, ) return None, line timestamp = line[:split_at] diff --git a/airflow/models/connection.py b/airflow/models/connection.py index 8134f372ca6ab..d5e6c78c2fd7a 100644 --- a/airflow/models/connection.py +++ b/airflow/models/connection.py @@ -208,8 +208,8 @@ def get_uri(self) -> str: """Return connection in URI format""" if '_' in self.conn_type: self.log.warning( - f"Connection schemes (type: {str(self.conn_type)}) " - f"shall not contain '_' according to RFC3986." + "Connection schemes (type: %s) shall not contain '_' according to RFC3986.", + str(self.conn_type), ) uri = f"{str(self.conn_type).lower().replace('_', '-')}://" diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index fdb566e467150..da0b1ea058ce7 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -790,7 +790,7 @@ def _emit_true_scheduling_delay_stats_for_finished_state(self, finished_tis: Lis if true_delay.total_seconds() > 0: Stats.timing(f'dagrun.{dag.dag_id}.first_task_scheduling_delay', true_delay) except Exception as e: - self.log.warning(f'Failed to record first_task_scheduling_delay metric:\n{e}') + self.log.warning('Failed to record first_task_scheduling_delay metric:\n%s', e) def _emit_duration_stats_for_finished_state(self): if self.state == State.RUNNING: diff --git a/airflow/providers/amazon/aws/operators/ecs.py b/airflow/providers/amazon/aws/operators/ecs.py index 66aaa19389055..d1112edf445ee 100644 --- a/airflow/providers/amazon/aws/operators/ecs.py +++ b/airflow/providers/amazon/aws/operators/ecs.py @@ -375,7 +375,7 @@ def _start_task(self, context): self.arn = response['tasks'][0]['taskArn'] self.ecs_task_id = self.arn.split("/")[-1] - self.log.info(f"ECS task ID is: {self.ecs_task_id}") + self.log.info("ECS task ID is: %s", self.ecs_task_id) if self.reattach: # Save the task ARN in XCom to be able to reattach it if needed diff --git a/airflow/providers/amazon/aws/operators/redshift_data.py b/airflow/providers/amazon/aws/operators/redshift_data.py index 3961833475ae3..f2d47da655835 100644 --- a/airflow/providers/amazon/aws/operators/redshift_data.py +++ b/airflow/providers/amazon/aws/operators/redshift_data.py @@ -136,12 +136,12 @@ def wait_for_results(self, statement_id): elif status == 'FAILED' or status == 'ABORTED': raise ValueError(f"Statement {statement_id!r} terminated with status {status}.") else: - self.log.info(f"Query {status}") + self.log.info("Query %s", status) sleep(self.poll_interval) def execute(self, context: 'Context') -> None: """Execute a statement against Amazon Redshift""" - self.log.info(f"Executing statement: {self.sql}") + self.log.info("Executing statement: %s", self.sql) self.statement_id = self.execute_query() diff --git a/airflow/providers/amazon/aws/operators/redshift_sql.py b/airflow/providers/amazon/aws/operators/redshift_sql.py index e525e7963aa90..c7ad77acb5341 100644 --- a/airflow/providers/amazon/aws/operators/redshift_sql.py +++ b/airflow/providers/amazon/aws/operators/redshift_sql.py @@ -73,6 +73,6 @@ def get_hook(self) -> RedshiftSQLHook: def execute(self, context: 'Context') -> None: """Execute a statement against Amazon Redshift""" - self.log.info(f"Executing statement: {self.sql}") + self.log.info("Executing statement: %s", self.sql) hook = self.get_hook() hook.run(self.sql, autocommit=self.autocommit, parameters=self.parameters) diff --git a/airflow/providers/amazon/aws/operators/sagemaker.py b/airflow/providers/amazon/aws/operators/sagemaker.py index 590342344b74a..084e3e857e9be 100644 --- a/airflow/providers/amazon/aws/operators/sagemaker.py +++ b/airflow/providers/amazon/aws/operators/sagemaker.py @@ -693,4 +693,4 @@ def __init__(self, *, config, aws_conn_id: str = DEFAULT_CONN_ID, **kwargs): def execute(self, context: 'Context') -> Any: sagemaker_hook = SageMakerHook(aws_conn_id=self.aws_conn_id) sagemaker_hook.delete_model(model_name=self.config['ModelName']) - self.log.info(f"Model {self.config['ModelName']} deleted Successfully.") + self.log.info("Model %s deleted successfully.", self.config['ModelName']) diff --git a/airflow/providers/amazon/aws/transfers/ftp_to_s3.py b/airflow/providers/amazon/aws/transfers/ftp_to_s3.py index ac882acb1a7a8..1426599bc4763 100644 --- a/airflow/providers/amazon/aws/transfers/ftp_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/ftp_to_s3.py @@ -108,7 +108,7 @@ def __upload_to_s3_from_ftp(self, remote_filename, s3_file_key): gzip=self.gzip, acl_policy=self.acl_policy, ) - self.log.info(f'File upload to {s3_file_key}') + self.log.info('File upload to %s', s3_file_key) def execute(self, context: 'Context'): self.ftp_hook = FTPHook(ftp_conn_id=self.ftp_conn_id) @@ -116,7 +116,7 @@ def execute(self, context: 'Context'): if self.ftp_filenames: if isinstance(self.ftp_filenames, str): - self.log.info(f'Getting files in {self.ftp_path}') + self.log.info('Getting files in %s', self.ftp_path) list_dir = self.ftp_hook.list_directory( path=self.ftp_path, @@ -129,7 +129,7 @@ def execute(self, context: 'Context'): files = list(filter(lambda f: ftp_filename in f, list_dir)) for file in files: - self.log.info(f'Moving file {file}') + self.log.info('Moving file %s', file) if self.s3_filenames and isinstance(self.s3_filenames, str): filename = file.replace(self.ftp_filenames, self.s3_filenames) diff --git a/airflow/providers/amazon/aws/transfers/salesforce_to_s3.py b/airflow/providers/amazon/aws/transfers/salesforce_to_s3.py index b754bf9564287..a953693f1f193 100644 --- a/airflow/providers/amazon/aws/transfers/salesforce_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/salesforce_to_s3.py @@ -127,6 +127,6 @@ def execute(self, context: 'Context') -> str: ) s3_uri = f"s3://{self.s3_bucket_name}/{self.s3_key}" - self.log.info(f"Salesforce data uploaded to S3 at {s3_uri}.") + self.log.info("Salesforce data uploaded to S3 at %s.", s3_uri) return s3_uri diff --git a/airflow/providers/arangodb/sensors/arangodb.py b/airflow/providers/arangodb/sensors/arangodb.py index e27bd78a61acf..ee9d0d2a9004d 100644 --- a/airflow/providers/arangodb/sensors/arangodb.py +++ b/airflow/providers/arangodb/sensors/arangodb.py @@ -47,8 +47,8 @@ def __init__(self, *, query: str, arangodb_conn_id: str = "arangodb_default", ** self.query = query def poke(self, context: 'Context') -> bool: - self.log.info(f"Sensor running following query: {self.query}") + self.log.info("Sensor running the following query: %s", self.query) hook = ArangoDBHook(self.arangodb_conn_id) records = hook.query(self.query, count=True).count() - self.log.info(f"Total Records found: {records}") + self.log.info("Total records found: %d", records) return 0 != records diff --git a/airflow/providers/cncf/kubernetes/hooks/kubernetes.py b/airflow/providers/cncf/kubernetes/hooks/kubernetes.py index b6115b87f8560..5719918ce7b91 100644 --- a/airflow/providers/cncf/kubernetes/hooks/kubernetes.py +++ b/airflow/providers/cncf/kubernetes/hooks/kubernetes.py @@ -228,7 +228,7 @@ def create_custom_object( ) self.log.warning("Deleted SparkApplication with the same name.") except client.rest.ApiException: - self.log.info(f"SparkApp {body_dict['metadata']['name']} not found.") + self.log.info("SparkApp %s not found.", body_dict['metadata']['name']) try: response = api.create_namespaced_custom_object( diff --git a/airflow/providers/cncf/kubernetes/utils/pod_manager.py b/airflow/providers/cncf/kubernetes/utils/pod_manager.py index 153abee1dbf95..27c9439dbde1d 100644 --- a/airflow/providers/cncf/kubernetes/utils/pod_manager.py +++ b/airflow/providers/cncf/kubernetes/utils/pod_manager.py @@ -285,8 +285,9 @@ def parse_log_line(self, line: str) -> Tuple[Optional[DateTime], str]: split_at = line.find(' ') if split_at == -1: self.log.error( - f"Error parsing timestamp (no timestamp in message '${line}'). " - "Will continue execution but won't update timestamp" + "Error parsing timestamp (no timestamp in message %r). " + "Will continue execution but won't update timestamp", + line, ) return None, line timestamp = line[:split_at] diff --git a/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py b/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py index 3706836e58d12..64e2b4816af07 100644 --- a/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py +++ b/airflow/providers/google/cloud/operators/vertex_ai/endpoint_service.py @@ -557,7 +557,7 @@ def execute(self, context: 'Context'): impersonation_chain=self.impersonation_chain, ) - self.log.info(f"Removing a DeployedModel {self.deployed_model_id}") + self.log.info("Removing a DeployedModel %s", self.deployed_model_id) operation = hook.undeploy_model( project_id=self.project_id, region=self.region, diff --git a/airflow/providers/google/cloud/sensors/dataproc.py b/airflow/providers/google/cloud/sensors/dataproc.py index 929a5a8fe177d..d9cacef097eec 100644 --- a/airflow/providers/google/cloud/sensors/dataproc.py +++ b/airflow/providers/google/cloud/sensors/dataproc.py @@ -79,8 +79,9 @@ def poke(self, context: "Context") -> bool: job_id=self.dataproc_job_id, region=self.region, project_id=self.project_id ) except ServerError as err: - self.log.info(f"DURATION RUN: {self._duration()}") - if self._duration() > self.wait_timeout: + duration = self._duration() + self.log.info("DURATION RUN: %f") + if duration > self.wait_timeout: raise AirflowException( f"Timeout: dataproc job {self.dataproc_job_id} " f"is not ready after {self.wait_timeout}s" diff --git a/airflow/providers/google/suite/transfers/sql_to_sheets.py b/airflow/providers/google/suite/transfers/sql_to_sheets.py index 6f972d076debe..b5460f16138bb 100644 --- a/airflow/providers/google/suite/transfers/sql_to_sheets.py +++ b/airflow/providers/google/suite/transfers/sql_to_sheets.py @@ -120,7 +120,7 @@ def execute(self, context: Any) -> None: impersonation_chain=self.impersonation_chain, ) - self.log.info(f"Uploading data to https://docs.google.com/spreadsheets/d/{self.spreadsheet_id}") + self.log.info("Uploading data to https://docs.google.com/spreadsheets/d/%s", self.spreadsheet_id) sheet_hook.update_values( spreadsheet_id=self.spreadsheet_id, diff --git a/airflow/providers/microsoft/azure/hooks/data_factory.py b/airflow/providers/microsoft/azure/hooks/data_factory.py index 3b1a79675f9dc..03fb99272d911 100644 --- a/airflow/providers/microsoft/azure/hooks/data_factory.py +++ b/airflow/providers/microsoft/azure/hooks/data_factory.py @@ -637,13 +637,13 @@ def get_pipeline_run_status( :param factory_name: The factory name. :return: The status of the pipeline run. """ - self.log.info(f"Getting the status of run ID {run_id}.") + self.log.info("Getting the status of run ID %s.", run_id) pipeline_run_status = self.get_pipeline_run( run_id=run_id, factory_name=factory_name, resource_group_name=resource_group_name, ).status - self.log.info(f"Current status of pipeline run {run_id}: {pipeline_run_status}") + self.log.info("Current status of pipeline run %s: %s", run_id, pipeline_run_status) return pipeline_run_status diff --git a/airflow/providers/microsoft/azure/operators/data_factory.py b/airflow/providers/microsoft/azure/operators/data_factory.py index ef848b3512e0b..b354ea86ea9ca 100644 --- a/airflow/providers/microsoft/azure/operators/data_factory.py +++ b/airflow/providers/microsoft/azure/operators/data_factory.py @@ -156,7 +156,7 @@ def __init__( def execute(self, context: "Context") -> None: self.hook = AzureDataFactoryHook(azure_data_factory_conn_id=self.azure_data_factory_conn_id) - self.log.info(f"Executing the {self.pipeline_name} pipeline.") + self.log.info("Executing the %s pipeline.", self.pipeline_name) response = self.hook.run_pipeline( pipeline_name=self.pipeline_name, resource_group_name=self.resource_group_name, @@ -174,7 +174,7 @@ def execute(self, context: "Context") -> None: context["ti"].xcom_push(key="run_id", value=self.run_id) if self.wait_for_termination: - self.log.info(f"Waiting for pipeline run {self.run_id} to terminate.") + self.log.info("Waiting for pipeline run %s to terminate.", self.run_id) if self.hook.wait_for_pipeline_run_status( run_id=self.run_id, @@ -184,10 +184,10 @@ def execute(self, context: "Context") -> None: resource_group_name=self.resource_group_name, factory_name=self.factory_name, ): - self.log.info(f"Pipeline run {self.run_id} has completed successfully.") + self.log.info("Pipeline run %s has completed successfully.", self.run_id) else: raise AzureDataFactoryPipelineRunException( - f"Pipeline run {self.run_id} has failed or has been cancelled." + "Pipeline run %s has failed or has been cancelled.", self.run_id ) def on_kill(self) -> None: @@ -207,6 +207,6 @@ def on_kill(self) -> None: resource_group_name=self.resource_group_name, factory_name=self.factory_name, ): - self.log.info(f"Pipeline run {self.run_id} has been cancelled successfully.") + self.log.info("Pipeline run %s has been cancelled successfully.", self.run_id) else: raise AzureDataFactoryPipelineRunException(f"Pipeline run {self.run_id} was not cancelled.") diff --git a/airflow/providers_manager.py b/airflow/providers_manager.py index b96299753308d..b5d0297e90abf 100644 --- a/airflow/providers_manager.py +++ b/airflow/providers_manager.py @@ -377,9 +377,12 @@ def _verify_all_providers_all_compatible(self): if min_version: if packaging_version.parse(min_version) > packaging_version.parse(info.version): log.warning( - f"The package {provider_id} is not compatible with this version of Airflow. " - f"The package has version {info.version} but the minimum supported version " - f"of the package is {min_version}" + "The package %s is not compatible with this version of Airflow. " + "The package has version %s but the minimum supported version " + "of the package is %s", + provider_id, + info.version, + min_version, ) @provider_info_cache("hooks") diff --git a/airflow/settings.py b/airflow/settings.py index 8b50bde0a179e..260d306193ad7 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -293,7 +293,7 @@ def configure_orm(disable_connection_pool=False): data = result.fetchone()[0] if data != 1: log.critical("MSSQL database MUST have READ_COMMITTED_SNAPSHOT enabled.") - log.critical(f"The database {engine.url.database} has it disabled.") + log.critical("The database %s has it disabled.", engine.url.database) log.critical("This will cause random deadlocks, Refusing to start.") log.critical( "See https://airflow.apache.org/docs/apache-airflow/stable/howto/" diff --git a/airflow/www/fab_security/manager.py b/airflow/www/fab_security/manager.py index e34a3f736a8ad..f56e9fec6e074 100644 --- a/airflow/www/fab_security/manager.py +++ b/airflow/www/fab_security/manager.py @@ -82,7 +82,7 @@ def _oauth_tokengetter(token=None): from session cookie. """ token = session.get("oauth") - log.debug(f"Token Get: {token}") + log.debug("Token Get: %s", token) return token @@ -256,7 +256,7 @@ def __init__(self, appbuilder): self.oauth_remotes = {} for _provider in self.oauth_providers: provider_name = _provider["name"] - log.debug(f"OAuth providers init {provider_name}") + log.debug("OAuth providers init %s", provider_name) obj_provider = self.oauth.register(provider_name, **_provider["remote_app"]) obj_provider._tokengetter = self.oauth_tokengetter if not self.oauth_user_info: @@ -320,7 +320,7 @@ def get_roles_from_keys(self, role_keys: List[str]) -> Set[RoleModelView]: if fab_role: _roles.add(fab_role) else: - log.warning(f"Can't find role specified in AUTH_ROLES_MAPPING: {fab_role_name}") + log.warning("Can't find role specified in AUTH_ROLES_MAPPING: %s", fab_role_name) return _roles @property @@ -538,7 +538,7 @@ def wraps(provider, response=None): ret = f(self, provider, response=response) # Checks if decorator is well behaved and returns a dict as supposed. if not type(ret) == dict: - log.error(f"OAuth user info decorated function did not returned a dict, but: {type(ret)}") + log.error("OAuth user info decorated function did not returned a dict, but: %s", type(ret)) return {} return ret @@ -586,13 +586,13 @@ def get_oauth_user_info(self, provider, resp): if provider == "github" or provider == "githublocal": me = self.appbuilder.sm.oauth_remotes[provider].get("user") data = me.json() - log.debug(f"User info from Github: {data}") + log.debug("User info from GitHub: %s", data) return {"username": "github_" + data.get("login")} # for twitter if provider == "twitter": me = self.appbuilder.sm.oauth_remotes[provider].get("account/settings.json") data = me.json() - log.debug(f"User info from Twitter: {data}") + log.debug("User info from Twitter: %s", data) return {"username": "twitter_" + data.get("screen_name", "")} # for linkedin if provider == "linkedin": @@ -600,7 +600,7 @@ def get_oauth_user_info(self, provider, resp): "people/~:(id,email-address,first-name,last-name)?format=json" ) data = me.json() - log.debug(f"User info from Linkedin: {data}") + log.debug("User info from LinkedIn: %s", data) return { "username": "linkedin_" + data.get("id", ""), "email": data.get("email-address", ""), @@ -611,7 +611,7 @@ def get_oauth_user_info(self, provider, resp): if provider == "google": me = self.appbuilder.sm.oauth_remotes[provider].get("userinfo") data = me.json() - log.debug(f"User info from Google: {data}") + log.debug("User info from Google: %s", data) return { "username": "google_" + data.get("id", ""), "first_name": data.get("given_name", ""), @@ -624,11 +624,11 @@ def get_oauth_user_info(self, provider, resp): # https://docs.microsoft.com/en-us/azure/active-directory/develop/ # active-directory-protocols-oauth-code if provider == "azure": - log.debug(f"Azure response received : {resp}") + log.debug("Azure response received : %s", resp) id_token = resp["id_token"] log.debug(str(id_token)) me = self._azure_jwt_token_parse(id_token) - log.debug(f"Parse JWT token : {me}") + log.debug("Parse JWT token : %s", me) return { "name": me.get("name", ""), "email": me["upn"], @@ -642,7 +642,7 @@ def get_oauth_user_info(self, provider, resp): if provider == "openshift": me = self.appbuilder.sm.oauth_remotes[provider].get("apis/user.openshift.io/v1/users/~") data = me.json() - log.debug(f"User info from OpenShift: {data}") + log.debug("User info from OpenShift: %s", data) return {"username": "openshift_" + data.get("metadata").get("name")} # for Okta if provider == "okta": @@ -904,12 +904,12 @@ def _search_ldap(self, ldap, con, username): # perform the LDAP search log.debug( - f"LDAP search for '{filter_str}' with fields {request_fields} in scope '{self.auth_ldap_search}'" + "LDAP search for %r with fields %s in scope %r", filter_str, request_fields, self.auth_ldap_search ) raw_search_result = con.search_s( self.auth_ldap_search, ldap.SCOPE_SUBTREE, filter_str, request_fields ) - log.debug(f"LDAP search returned: {raw_search_result}") + log.debug("LDAP search returned: %s", raw_search_result) # Remove any search referrals from results search_result = [ @@ -919,8 +919,9 @@ def _search_ldap(self, ldap, con, username): # only continue if 0 or 1 results were returned if len(search_result) > 1: log.error( - f"LDAP search for '{filter_str}' in scope " - f"'{self.auth_ldap_search!a}' returned multiple results" + "LDAP search for %r in scope '%a' returned multiple results", + self.auth_ldap_search, + filter_str, ) return None, None @@ -951,7 +952,7 @@ def _ldap_calculate_user_roles(self, user_attributes: Dict[str, List[bytes]]) -> if fab_role: user_role_objects.add(fab_role) else: - log.warning(f"Can't find AUTH_USER_REGISTRATION role: {registration_role_name}") + log.warning("Can't find AUTH_USER_REGISTRATION role: %s", registration_role_name) return list(user_role_objects) @@ -966,9 +967,9 @@ def _ldap_bind_indirect(self, ldap, con) -> None: assert self.auth_ldap_bind_user, "AUTH_LDAP_BIND_USER must be set" try: - log.debug(f"LDAP bind indirect TRY with username: '{self.auth_ldap_bind_user}'") + log.debug("LDAP bind indirect TRY with username: %r", self.auth_ldap_bind_user) con.simple_bind_s(self.auth_ldap_bind_user, self.auth_ldap_bind_password) - log.debug(f"LDAP bind indirect SUCCESS with username: '{self.auth_ldap_bind_user}'") + log.debug("LDAP bind indirect SUCCESS with username: %r", self.auth_ldap_bind_user) except ldap.INVALID_CREDENTIALS as ex: log.error( "AUTH_LDAP_BIND_USER and AUTH_LDAP_BIND_PASSWORD are" " not valid LDAP bind credentials" @@ -979,9 +980,9 @@ def _ldap_bind_indirect(self, ldap, con) -> None: def _ldap_bind(ldap, con, dn: str, password: str) -> bool: """Validates/binds the provided dn/password with the LDAP sever.""" try: - log.debug(f"LDAP bind TRY with username: '{dn}'") + log.debug("LDAP bind TRY with username: %r", dn) con.simple_bind_s(dn, password) - log.debug(f"LDAP bind SUCCESS with username: '{dn}'") + log.debug("LDAP bind SUCCESS with username: %r", dn) return True except ldap.INVALID_CREDENTIALS: return False @@ -1141,7 +1142,7 @@ def auth_user_ldap(self, username, password): # Sync the user's roles if user and user_attributes and self.auth_roles_sync_at_login: user.roles = self._ldap_calculate_user_roles(user_attributes) - log.debug(f"Calculated new roles for user='{user_dn}' as: {user.roles}") + log.debug("Calculated new roles for user=%r as: %s", user_dn, user.roles) # If the user is new, register them if (not user) and user_attributes and self.auth_user_registration: @@ -1156,7 +1157,7 @@ def auth_user_ldap(self, username, password): ), role=self._ldap_calculate_user_roles(user_attributes), ) - log.debug(f"New user registered: {user}") + log.debug("New user registered: %s", user) # If user registration failed, go away if not user: @@ -1248,7 +1249,7 @@ def _oauth_calculate_user_roles(self, userinfo) -> List[str]: if fab_role: user_role_objects.add(fab_role) else: - log.warning(f"Can't find AUTH_USER_REGISTRATION role: {registration_role_name}") + log.warning("Can't find AUTH_USER_REGISTRATION role: %s", registration_role_name) return list(user_role_objects) @@ -1265,7 +1266,7 @@ def auth_user_oauth(self, userinfo): elif "email" in userinfo: username = userinfo["email"] else: - log.error(f"OAUTH userinfo does not have username or email {userinfo}") + log.error("OAUTH userinfo does not have username or email %s", userinfo) return None # If username is empty, go away @@ -1286,7 +1287,7 @@ def auth_user_oauth(self, userinfo): # Sync the user's roles if user and self.auth_roles_sync_at_login: user.roles = self._oauth_calculate_user_roles(userinfo) - log.debug(f"Calculated new roles for user='{username}' as: {user.roles}") + log.debug("Calculated new roles for user=%r as: %s", username, user.roles) # If the user is new, register them if (not user) and self.auth_user_registration: @@ -1297,11 +1298,11 @@ def auth_user_oauth(self, userinfo): email=userinfo.get("email", "") or f"{username}@email.notfound", role=self._oauth_calculate_user_roles(userinfo), ) - log.debug(f"New user registered: {user}") + log.debug("New user registered: %s", user) # If user registration failed, go away if not user: - log.error(f"Error creating a new OAuth user {username}") + log.error("Error creating a new OAuth user %s", username) return None # LOGIN SUCCESS (only if user is now registered) diff --git a/airflow/www/fab_security/sqla/manager.py b/airflow/www/fab_security/sqla/manager.py index 9042a22f6a0e4..8ee1d900f6ce0 100644 --- a/airflow/www/fab_security/sqla/manager.py +++ b/airflow/www/fab_security/sqla/manager.py @@ -173,13 +173,13 @@ def find_user(self, username=None, email=None): .one_or_none() ) except MultipleResultsFound: - log.error(f"Multiple results found for user {username}") + log.error("Multiple results found for user %s", username) return None elif email: try: return self.get_session.query(self.user_model).filter_by(email=email).one_or_none() except MultipleResultsFound: - log.error(f"Multiple results found for user with email {email}") + log.error("Multiple results found for user with email %s", email) return None def get_all_users(self): diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py index 40ff16a7a03bd..6f10c1b7e9401 100644 --- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py +++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py @@ -40,6 +40,7 @@ 'check-extra-packages-references', 'check-extras-order', 'check-for-inclusive-language', + 'check-fstrings-logging', 'check-hooks-apply', 'check-incorrect-use-of-LoggingMixin', 'check-integrations-are-consistent', diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg index a5bdbd201b478..c364c338fd0f5 100644 --- a/images/breeze/output-static-checks.svg +++ b/images/breeze/output-static-checks.svg @@ -1,103 +1,174 @@ - - + - Command: static-checks - - - - - - -                                                                                                                          - Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...                                                               -                                                                                                                         - Run static checks.                                                                                                      -                                                                                                                         -╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --type                  -t  Type(s) of the static checks to run (multiple can be added).                             -                              (all | black | blacken-docs | check-airflow-2-1-compatibility |                          -                              check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |             -                              check-apache-license-rat | check-base-operator-usage |                                   -                              check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |              -                              check-builtin-literals | check-changelog-has-no-duplicates |                             -                              check-daysago-import-from-utils | check-docstring-param-types |                          -                              check-executables-have-shebangs | check-extra-packages-references | check-extras-order   -                              | check-for-inclusive-language | check-hooks-apply |                                     -                              check-incorrect-use-of-LoggingMixin | check-integrations-are-consistent |                -                              check-merge-conflict | check-newsfragments-are-valid |                                   -                              check-no-providers-in-core-examples | check-no-relative-imports |                        -                              check-persist-credentials-disabled-in-github-workflows |                                 -                              check-pre-commit-information-consistent | check-provide-create-sessions-imports |        -                              check-provider-yaml-valid | check-providers-init-file-missing |                          -                              check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                -                              check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         -                              check-start-date-not-used-in-defaults | check-system-tests-present | check-xml |         -                              codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer |         -                              fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort |          -                              lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript |     -                              lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |                    -                              pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade | rst-backticks |       -                              run-flake8 | run-mypy | run-shellcheck | static-check-autoflake | trailing-whitespace    -                              | update-breeze-file | update-breeze-readme-config-hash | update-extras |                -                              update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |                    -                              update-local-yml-file | update-migration-references | update-providers-dependencies |    -                              update-setup-cfg-file | update-spelling-wordlist-to-be-sorted |                          -                              update-supported-versions | update-vendored-in-k8s-json-schema | update-version |        -                              yamllint | yesqa)                                                                        -  --files                 -f  List of files to run the checks on. (TEXT)                                               -  --all-files             -a  Run checks on all files.                                                                 -  --show-diff-on-failure  -s  Show diff for files modified by the checks.                                              -  --last-commit           -c  Run checks for all files in last commit. Mutually exclusive with --commit-ref.           -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ -  --commit-ref         -r  Run checks for this commit reference only (can be any git commit-ish reference). Mutually   -                           exclusive with --last-commit.                                                               -                           (TEXT)                                                                                      -  --verbose            -v  Print verbose information about performed steps.                                            -  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 -  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           -  --help               -h  Show this message and exit.                                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ - - + + +
+
+
+ + + + + +
Command: static-checks
+
+
+
+
Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...
+
+
Run static checks.
+
+
╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
+
--type -t Type(s) of the static checks to run (multiple can be added).
+
(all | black | blacken-docs | check-airflow-2-1-compatibility |
+
check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |
+
check-apache-license-rat | check-base-operator-usage |
+
check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |
+
check-builtin-literals | check-changelog-has-no-duplicates |
+
check-daysago-import-from-utils | check-docstring-param-types |
+
check-executables-have-shebangs | check-extra-packages-references | check-extras-order
+
| check-for-inclusive-language | check-fstrings-logging | check-hooks-apply |
+
check-incorrect-use-of-LoggingMixin | check-integrations-are-consistent |
+
check-merge-conflict | check-newsfragments-are-valid |
+
check-no-providers-in-core-examples | check-no-relative-imports |
+
check-persist-credentials-disabled-in-github-workflows |
+
check-pre-commit-information-consistent | check-provide-create-sessions-imports |
+
check-provider-yaml-valid | check-providers-init-file-missing |
+
check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |
+
check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |
+
check-start-date-not-used-in-defaults | check-system-tests-present | check-xml |
+
codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer |
+
fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort |
+
lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript |
+
lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |
+
pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade | rst-backticks |
+
run-flake8 | run-mypy | run-shellcheck | static-check-autoflake | trailing-whitespace
+
| update-breeze-file | update-breeze-readme-config-hash | update-extras |
+
update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |
+
update-local-yml-file | update-migration-references | update-providers-dependencies |
+
update-setup-cfg-file | update-spelling-wordlist-to-be-sorted |
+
update-supported-versions | update-vendored-in-k8s-json-schema | update-version |
+
yamllint | yesqa)
+
--files -f List of files to run the checks on. (TEXT)
+
--all-files -a Run checks on all files.
+
--show-diff-on-failure -s Show diff for files modified by the checks.
+
--last-commit -c Run checks for all files in last commit. Mutually exclusive with --commit-ref.
+
╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
+
--commit-ref -r Run checks for this commit reference only (can be any git commit-ish reference). Mutually
+
exclusive with --last-commit.
+
(TEXT)
+
--verbose -v Print verbose information about performed steps.
+
--dry-run -D If dry-run is set, commands are only printed, not executed.
+
--github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]
+
--help -h Show this message and exit.
+
╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
+
+
+
+
+ +
diff --git a/scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py b/scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py new file mode 100755 index 0000000000000..684b3129a8442 --- /dev/null +++ b/scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py @@ -0,0 +1,102 @@ +#!/usr/bin/env python3 +# 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. + +import ast +import os +import sys +from glob import glob +from typing import Dict, List, Sequence, Union + +from rich.console import Console + +if __name__ not in ("__main__", "__mp_main__"): + raise SystemExit( + "This file is intended to be executed as an executable program. You cannot use it as a module. " + f"To run this script, run the ./{__file__} command [FILE] ..." + ) + +console = Console(color_system="standard", width=180) + +FstringInfo = Dict[str, Union[str, int]] + + +class LogFstringFinder(ast.NodeVisitor): + logging_levels: Sequence[str] = ("critical", "error", "warning", "info", "debug") + has_fstrings_in_logging: bool = False + fstrings_logging_info: List[FstringInfo] = [] + + def __init__(self, file_name: str) -> None: + self.file_name = file_name + + def _add_fstring_info(self, node): + fstring_info: FstringInfo = {"file_name": self.file_name, "line_number": node.lineno} + LogFstringFinder.fstrings_logging_info.append(fstring_info) + + def visit_Call(self, node: ast.Call) -> None: + # Validate callable name is a predefined logging level other than `notset` and argument to the + # logging call is an f-string. + if ( + isinstance(node.func, ast.Attribute) + and node.func.attr in self.logging_levels + and isinstance(node.args[0], ast.JoinedStr) + ): + # Check for logging calls of `self.log.info()`, `self.logger.warning()`, etc. form or similar. + if ( + isinstance(node.func.value, ast.Attribute) + and getattr(node.func.value.value, "id") == "self" + and node.func.value.attr.startswith("log") + ): + LogFstringFinder.has_fstrings_in_logging = True + self._add_fstring_info(node) + # Check if logging call does not utilize a class method such as ``logging.info()``. + elif isinstance(node.func.value, ast.Name) and node.func.value.id.startswith("log"): + LogFstringFinder.has_fstrings_in_logging = True + self._add_fstring_info(node) + + +def check_logging_for_fstrings(provider_files: List[str]): + for file_name in provider_files: + if file_name.endswith("/__pycache__"): + continue + + if file_name.endswith(".py"): + try: + with open(file_name, encoding="utf-8") as f: + tree = ast.parse(f.read(), file_name) + except Exception: + print(f"Error when opening file: {file_name}", file=sys.stderr) + raise + + visitor = LogFstringFinder(file_name) + visitor.visit(tree) + + if visitor.has_fstrings_in_logging: + console.print( + "[bold red]\nF-strings found in logging calls within the following file(s). " + "Please convert these calls to use %-formatting instead.[/]\n" + ) + for fstring_log in visitor.fstrings_logging_info: + console.print(f"{fstring_log['file_name']}:{fstring_log['line_number']}") + + sys.exit(1) + + +if __name__ == "__main__": + root_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir, os.pardir, os.pardir)) + all_airflow_python_files = sorted(glob(f"{root_dir}/airflow/**/*", recursive=True)) + check_logging_for_fstrings(all_airflow_python_files) From dfa297a4fd0d9d68665d1c03c02473d9934b9b36 Mon Sep 17 00:00:00 2001 From: Josh Fell Date: Fri, 20 May 2022 23:07:25 -0400 Subject: [PATCH 2/3] Implement feedback from initial code reviews --- airflow/models/connection.py | 2 +- airflow/models/dagrun.py | 4 +- .../google/cloud/sensors/dataproc.py | 2 +- .../google/suite/transfers/sql_to_sheets.py | 5 +- .../src/airflow_breeze/pre_commit_ids.py | 1 - images/breeze/output-static-checks.svg | 271 +++++++----------- .../pre_commit_check_logging_for_fstrings.py | 102 ------- 7 files changed, 108 insertions(+), 279 deletions(-) delete mode 100755 scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py diff --git a/airflow/models/connection.py b/airflow/models/connection.py index d5e6c78c2fd7a..b21e68b73b356 100644 --- a/airflow/models/connection.py +++ b/airflow/models/connection.py @@ -209,7 +209,7 @@ def get_uri(self) -> str: if '_' in self.conn_type: self.log.warning( "Connection schemes (type: %s) shall not contain '_' according to RFC3986.", - str(self.conn_type), + self.conn_type, ) uri = f"{str(self.conn_type).lower().replace('_', '-')}://" diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index da0b1ea058ce7..58974c39be239 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -789,8 +789,8 @@ def _emit_true_scheduling_delay_stats_for_finished_state(self, finished_tis: Lis true_delay = first_start_date - data_interval_end if true_delay.total_seconds() > 0: Stats.timing(f'dagrun.{dag.dag_id}.first_task_scheduling_delay', true_delay) - except Exception as e: - self.log.warning('Failed to record first_task_scheduling_delay metric:\n%s', e) + except Exception: + self.log.warning('Failed to record first_task_scheduling_delay metric:', exc_info=True) def _emit_duration_stats_for_finished_state(self): if self.state == State.RUNNING: diff --git a/airflow/providers/google/cloud/sensors/dataproc.py b/airflow/providers/google/cloud/sensors/dataproc.py index d9cacef097eec..02b2d5e14d7ab 100644 --- a/airflow/providers/google/cloud/sensors/dataproc.py +++ b/airflow/providers/google/cloud/sensors/dataproc.py @@ -80,7 +80,7 @@ def poke(self, context: "Context") -> bool: ) except ServerError as err: duration = self._duration() - self.log.info("DURATION RUN: %f") + self.log.info("DURATION RUN: %f", duration) if duration > self.wait_timeout: raise AirflowException( f"Timeout: dataproc job {self.dataproc_job_id} " diff --git a/airflow/providers/google/suite/transfers/sql_to_sheets.py b/airflow/providers/google/suite/transfers/sql_to_sheets.py index b5460f16138bb..8384868199b6f 100644 --- a/airflow/providers/google/suite/transfers/sql_to_sheets.py +++ b/airflow/providers/google/suite/transfers/sql_to_sheets.py @@ -17,6 +17,7 @@ import datetime +import logging import numbers from contextlib import closing from typing import Any, Iterable, Mapping, Optional, Sequence, Union @@ -120,7 +121,9 @@ def execute(self, context: Any) -> None: impersonation_chain=self.impersonation_chain, ) - self.log.info("Uploading data to https://docs.google.com/spreadsheets/d/%s", self.spreadsheet_id) + if self.log.isEnabledFor(logging.INFO): + url = f"https://docs.google.com/spreadsheets/d/{self.spreadsheet_id}" + self.log.info("Uploading data to %s", url) sheet_hook.update_values( spreadsheet_id=self.spreadsheet_id, diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py index 6f10c1b7e9401..40ff16a7a03bd 100644 --- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py +++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py @@ -40,7 +40,6 @@ 'check-extra-packages-references', 'check-extras-order', 'check-for-inclusive-language', - 'check-fstrings-logging', 'check-hooks-apply', 'check-incorrect-use-of-LoggingMixin', 'check-integrations-are-consistent', diff --git a/images/breeze/output-static-checks.svg b/images/breeze/output-static-checks.svg index c364c338fd0f5..a5bdbd201b478 100644 --- a/images/breeze/output-static-checks.svg +++ b/images/breeze/output-static-checks.svg @@ -1,174 +1,103 @@ - + + - - -
-
-
- - - - - -
Command: static-checks
-
-
-
-
Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...
-
-
Run static checks.
-
-
╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮
-
--type -t Type(s) of the static checks to run (multiple can be added).
-
(all | black | blacken-docs | check-airflow-2-1-compatibility |
-
check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |
-
check-apache-license-rat | check-base-operator-usage |
-
check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |
-
check-builtin-literals | check-changelog-has-no-duplicates |
-
check-daysago-import-from-utils | check-docstring-param-types |
-
check-executables-have-shebangs | check-extra-packages-references | check-extras-order
-
| check-for-inclusive-language | check-fstrings-logging | check-hooks-apply |
-
check-incorrect-use-of-LoggingMixin | check-integrations-are-consistent |
-
check-merge-conflict | check-newsfragments-are-valid |
-
check-no-providers-in-core-examples | check-no-relative-imports |
-
check-persist-credentials-disabled-in-github-workflows |
-
check-pre-commit-information-consistent | check-provide-create-sessions-imports |
-
check-provider-yaml-valid | check-providers-init-file-missing |
-
check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |
-
check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |
-
check-start-date-not-used-in-defaults | check-system-tests-present | check-xml |
-
codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer |
-
fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort |
-
lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript |
-
lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |
-
pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade | rst-backticks |
-
run-flake8 | run-mypy | run-shellcheck | static-check-autoflake | trailing-whitespace
-
| update-breeze-file | update-breeze-readme-config-hash | update-extras |
-
update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |
-
update-local-yml-file | update-migration-references | update-providers-dependencies |
-
update-setup-cfg-file | update-spelling-wordlist-to-be-sorted |
-
update-supported-versions | update-vendored-in-k8s-json-schema | update-version |
-
yamllint | yesqa)
-
--files -f List of files to run the checks on. (TEXT)
-
--all-files -a Run checks on all files.
-
--show-diff-on-failure -s Show diff for files modified by the checks.
-
--last-commit -c Run checks for all files in last commit. Mutually exclusive with --commit-ref.
-
╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-
╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮
-
--commit-ref -r Run checks for this commit reference only (can be any git commit-ish reference). Mutually
-
exclusive with --last-commit.
-
(TEXT)
-
--verbose -v Print verbose information about performed steps.
-
--dry-run -D If dry-run is set, commands are only printed, not executed.
-
--github-repository -g GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]
-
--help -h Show this message and exit.
-
╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯
-
-
-
-
- -
+ Command: static-checks + + + + + + +                                                                                                                          + Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]...                                                               +                                                                                                                         + Run static checks.                                                                                                      +                                                                                                                         +╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --type                  -t  Type(s) of the static checks to run (multiple can be added).                             +                              (all | black | blacken-docs | check-airflow-2-1-compatibility |                          +                              check-airflow-config-yaml-consistent | check-airflow-providers-have-extras |             +                              check-apache-license-rat | check-base-operator-usage |                                   +                              check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |              +                              check-builtin-literals | check-changelog-has-no-duplicates |                             +                              check-daysago-import-from-utils | check-docstring-param-types |                          +                              check-executables-have-shebangs | check-extra-packages-references | check-extras-order   +                              | check-for-inclusive-language | check-hooks-apply |                                     +                              check-incorrect-use-of-LoggingMixin | check-integrations-are-consistent |                +                              check-merge-conflict | check-newsfragments-are-valid |                                   +                              check-no-providers-in-core-examples | check-no-relative-imports |                        +                              check-persist-credentials-disabled-in-github-workflows |                                 +                              check-pre-commit-information-consistent | check-provide-create-sessions-imports |        +                              check-provider-yaml-valid | check-providers-init-file-missing |                          +                              check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |                +                              check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |         +                              check-start-date-not-used-in-defaults | check-system-tests-present | check-xml |         +                              codespell | debug-statements | detect-private-key | doctoc | end-of-file-fixer |         +                              fix-encoding-pragma | flynt | forbid-tabs | identity | insert-license | isort |          +                              lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart | lint-javascript |     +                              lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending |                    +                              pretty-format-json | pydocstyle | python-no-log-warn | pyupgrade | rst-backticks |       +                              run-flake8 | run-mypy | run-shellcheck | static-check-autoflake | trailing-whitespace    +                              | update-breeze-file | update-breeze-readme-config-hash | update-extras |                +                              update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |                    +                              update-local-yml-file | update-migration-references | update-providers-dependencies |    +                              update-setup-cfg-file | update-spelling-wordlist-to-be-sorted |                          +                              update-supported-versions | update-vendored-in-k8s-json-schema | update-version |        +                              yamllint | yesqa)                                                                        +  --files                 -f  List of files to run the checks on. (TEXT)                                               +  --all-files             -a  Run checks on all files.                                                                 +  --show-diff-on-failure  -s  Show diff for files modified by the checks.                                              +  --last-commit           -c  Run checks for all files in last commit. Mutually exclusive with --commit-ref.           +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Options ────────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +  --commit-ref         -r  Run checks for this commit reference only (can be any git commit-ish reference). Mutually   +                           exclusive with --last-commit.                                                               +                           (TEXT)                                                                                      +  --verbose            -v  Print verbose information about performed steps.                                            +  --dry-run            -D  If dry-run is set, commands are only printed, not executed.                                 +  --github-repository  -g  GitHub repository used to pull, push run images. (TEXT) [default: apache/airflow]           +  --help               -h  Show this message and exit.                                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ + +
diff --git a/scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py b/scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py deleted file mode 100755 index 684b3129a8442..0000000000000 --- a/scripts/ci/pre_commit/pre_commit_check_logging_for_fstrings.py +++ /dev/null @@ -1,102 +0,0 @@ -#!/usr/bin/env python3 -# 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. - -import ast -import os -import sys -from glob import glob -from typing import Dict, List, Sequence, Union - -from rich.console import Console - -if __name__ not in ("__main__", "__mp_main__"): - raise SystemExit( - "This file is intended to be executed as an executable program. You cannot use it as a module. " - f"To run this script, run the ./{__file__} command [FILE] ..." - ) - -console = Console(color_system="standard", width=180) - -FstringInfo = Dict[str, Union[str, int]] - - -class LogFstringFinder(ast.NodeVisitor): - logging_levels: Sequence[str] = ("critical", "error", "warning", "info", "debug") - has_fstrings_in_logging: bool = False - fstrings_logging_info: List[FstringInfo] = [] - - def __init__(self, file_name: str) -> None: - self.file_name = file_name - - def _add_fstring_info(self, node): - fstring_info: FstringInfo = {"file_name": self.file_name, "line_number": node.lineno} - LogFstringFinder.fstrings_logging_info.append(fstring_info) - - def visit_Call(self, node: ast.Call) -> None: - # Validate callable name is a predefined logging level other than `notset` and argument to the - # logging call is an f-string. - if ( - isinstance(node.func, ast.Attribute) - and node.func.attr in self.logging_levels - and isinstance(node.args[0], ast.JoinedStr) - ): - # Check for logging calls of `self.log.info()`, `self.logger.warning()`, etc. form or similar. - if ( - isinstance(node.func.value, ast.Attribute) - and getattr(node.func.value.value, "id") == "self" - and node.func.value.attr.startswith("log") - ): - LogFstringFinder.has_fstrings_in_logging = True - self._add_fstring_info(node) - # Check if logging call does not utilize a class method such as ``logging.info()``. - elif isinstance(node.func.value, ast.Name) and node.func.value.id.startswith("log"): - LogFstringFinder.has_fstrings_in_logging = True - self._add_fstring_info(node) - - -def check_logging_for_fstrings(provider_files: List[str]): - for file_name in provider_files: - if file_name.endswith("/__pycache__"): - continue - - if file_name.endswith(".py"): - try: - with open(file_name, encoding="utf-8") as f: - tree = ast.parse(f.read(), file_name) - except Exception: - print(f"Error when opening file: {file_name}", file=sys.stderr) - raise - - visitor = LogFstringFinder(file_name) - visitor.visit(tree) - - if visitor.has_fstrings_in_logging: - console.print( - "[bold red]\nF-strings found in logging calls within the following file(s). " - "Please convert these calls to use %-formatting instead.[/]\n" - ) - for fstring_log in visitor.fstrings_logging_info: - console.print(f"{fstring_log['file_name']}:{fstring_log['line_number']}") - - sys.exit(1) - - -if __name__ == "__main__": - root_dir = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir, os.pardir, os.pardir)) - all_airflow_python_files = sorted(glob(f"{root_dir}/airflow/**/*", recursive=True)) - check_logging_for_fstrings(all_airflow_python_files) From 2812f1457519fc59f8b3a61b4f99c813aee7250d Mon Sep 17 00:00:00 2001 From: Josh Fell Date: Fri, 20 May 2022 23:31:41 -0400 Subject: [PATCH 3/3] Fix TestAzureDataFactoryRunPipelineOperator --- airflow/providers/microsoft/azure/operators/data_factory.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/microsoft/azure/operators/data_factory.py b/airflow/providers/microsoft/azure/operators/data_factory.py index b354ea86ea9ca..488ccbced0702 100644 --- a/airflow/providers/microsoft/azure/operators/data_factory.py +++ b/airflow/providers/microsoft/azure/operators/data_factory.py @@ -187,7 +187,7 @@ def execute(self, context: "Context") -> None: self.log.info("Pipeline run %s has completed successfully.", self.run_id) else: raise AzureDataFactoryPipelineRunException( - "Pipeline run %s has failed or has been cancelled.", self.run_id + f"Pipeline run {self.run_id} has failed or has been cancelled." ) def on_kill(self) -> None: