From 40bf8f28f97f17f40d993d207ea740eba54593ee Mon Sep 17 00:00:00 2001 From: Ivan Date: Thu, 18 Jun 2020 10:33:12 +0100 Subject: [PATCH] Detect automatically the lack of reference to the guide in the operator descriptions (#9290) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: ivan.afonichkin Co-authored-by: Kamil BreguĊ‚a --- airflow/providers/amazon/aws/operators/ecs.py | 4 + .../aws/transfers/imap_attachment_to_s3.py | 4 + .../amazon/aws/transfers/s3_to_redshift.py | 4 + .../apache/spark/operators/spark_jdbc.py | 4 + .../apache/spark/operators/spark_sql.py | 4 + .../apache/spark/operators/spark_submit.py | 4 + .../kubernetes/operators/kubernetes_pod.py | 4 + .../google/cloud/operators/bigquery.py | 61 +++++++++++++ .../google/cloud/operators/dataproc.py | 4 + .../google/cloud/operators/life_sciences.py | 4 + .../google/cloud/transfers/gcs_to_local.py | 4 + .../google/cloud/transfers/local_to_gcs.py | 4 + .../marketing_platform/operators/analytics.py | 2 +- .../google/suite/transfers/gcs_to_gdrive.py | 4 + airflow/providers/http/operators/http.py | 4 + airflow/providers/http/sensors/http.py | 3 + .../yandex/operators/yandexcloud_dataproc.py | 4 + docs/build | 86 +++++++++++++++++++ docs/howto/operator/http/http.rst | 6 ++ 19 files changed, 213 insertions(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/ecs.py b/airflow/providers/amazon/aws/operators/ecs.py index ab729c52e8b6..07dd0b28d354 100644 --- a/airflow/providers/amazon/aws/operators/ecs.py +++ b/airflow/providers/amazon/aws/operators/ecs.py @@ -61,6 +61,10 @@ class ECSOperator(BaseOperator): # pylint: disable=too-many-instance-attributes """ Execute a task on AWS ECS (Elastic Container Service) + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ECSOperator` + :param task_definition: the task definition name on Elastic Container Service :type task_definition: str :param cluster: the cluster name on Elastic Container Service diff --git a/airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py b/airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py index 2a1d7f07d96f..a894ae4d2cfa 100644 --- a/airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/imap_attachment_to_s3.py @@ -28,6 +28,10 @@ class ImapAttachmentToS3Operator(BaseOperator): """ Transfers a mail attachment from a mail server into s3 bucket. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:ImapAttachmentToS3Operator` + :param imap_attachment_name: The file name of the mail attachment that you want to transfer. :type imap_attachment_name: str :param s3_key: The destination file name in the s3 bucket for the attachment. diff --git a/airflow/providers/amazon/aws/transfers/s3_to_redshift.py b/airflow/providers/amazon/aws/transfers/s3_to_redshift.py index f8d28a15a093..f2228e90a0f4 100644 --- a/airflow/providers/amazon/aws/transfers/s3_to_redshift.py +++ b/airflow/providers/amazon/aws/transfers/s3_to_redshift.py @@ -27,6 +27,10 @@ class S3ToRedshiftOperator(BaseOperator): """ Executes an COPY command to load files from s3 to Redshift + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:S3ToRedshiftOperator` + :param schema: reference to a specific schema in redshift database :type schema: str :param table: reference to a specific table in redshift database diff --git a/airflow/providers/apache/spark/operators/spark_jdbc.py b/airflow/providers/apache/spark/operators/spark_jdbc.py index 6a45b8cf4334..807f26749221 100644 --- a/airflow/providers/apache/spark/operators/spark_jdbc.py +++ b/airflow/providers/apache/spark/operators/spark_jdbc.py @@ -29,6 +29,10 @@ class SparkJDBCOperator(SparkSubmitOperator): SparkSubmitOperator, it assumes that the "spark-submit" binary is available on the PATH. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:SparkJDBCOperator` + :param spark_app_name: Name of the job (default airflow-spark-jdbc) :type spark_app_name: str :param spark_conn_id: Connection id as configured in Airflow administration diff --git a/airflow/providers/apache/spark/operators/spark_sql.py b/airflow/providers/apache/spark/operators/spark_sql.py index ac887af4d5b9..7230919790f4 100644 --- a/airflow/providers/apache/spark/operators/spark_sql.py +++ b/airflow/providers/apache/spark/operators/spark_sql.py @@ -25,6 +25,10 @@ class SparkSqlOperator(BaseOperator): """ Execute Spark SQL query + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:SparkSqlOperator` + :param sql: The SQL query to execute. (templated) :type sql: str :param conf: arbitrary Spark configuration property diff --git a/airflow/providers/apache/spark/operators/spark_submit.py b/airflow/providers/apache/spark/operators/spark_submit.py index 31553684e98f..393362463c52 100644 --- a/airflow/providers/apache/spark/operators/spark_submit.py +++ b/airflow/providers/apache/spark/operators/spark_submit.py @@ -29,6 +29,10 @@ class SparkSubmitOperator(BaseOperator): It requires that the "spark-submit" binary is in the PATH or the spark-home is set in the extra on the connection. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:SparkSubmitOperator` + :param application: The application that submitted as a job, either jar or py file. (templated) :type application: str :param conf: Arbitrary Spark configuration properties (templated) diff --git a/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py b/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py index 3c4bf6fbd0e9..1541a07e2f7a 100644 --- a/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py +++ b/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py @@ -39,6 +39,10 @@ class KubernetesPodOperator(BaseOperator): # pylint: disable=too-many-instance- """ Execute a task in a Kubernetes Pod + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:KubernetesPodOperator` + .. note:: If you use `Google Kubernetes Engine `__, use :class:`~airflow.providers.google.cloud.operators.kubernetes_engine.GKEStartPodOperator`, which diff --git a/airflow/providers/google/cloud/operators/bigquery.py b/airflow/providers/google/cloud/operators/bigquery.py index 3571a31b19c7..3eb15ce15ee8 100644 --- a/airflow/providers/google/cloud/operators/bigquery.py +++ b/airflow/providers/google/cloud/operators/bigquery.py @@ -95,6 +95,10 @@ class BigQueryCheckOperator(CheckOperator): first row is evaluated using python ``bool`` casting. If any of the values return ``False`` the check is failed and errors out. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryCheckOperator` + Note that Python bool casting evals the following as ``False``: * ``False`` @@ -168,6 +172,10 @@ class BigQueryValueCheckOperator(ValueCheckOperator): """ Performs a simple value check using sql code. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryValueCheckOperator` + :param sql: the sql to be executed :type sql: str :param use_legacy_sql: Whether to use legacy SQL (true) @@ -233,6 +241,10 @@ class BigQueryIntervalCheckOperator(IntervalCheckOperator): SELECT {metrics_threshold_dict_key} FROM {table} WHERE {date_filter_column}= + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryIntervalCheckOperator` + :param table: the table name :type table: str :param days_back: number of days between ds and the ds we want to check @@ -305,6 +317,10 @@ class BigQueryGetDataOperator(BaseOperator): **Example Result**: ``[['Tony', '10'], ['Mike', '20'], ['Steve', '15']]`` + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryGetDataOperator` + .. note:: If you pass fields to ``selected_fields`` which are in different order than the order of columns already in @@ -650,6 +666,10 @@ class BigQueryCreateEmptyTableOperator(BaseOperator): Google Cloud Storage must be a JSON file with the schema fields in it. You can also create a table without schema. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryCreateEmptyTableOperator` + :param project_id: The project to create the table into. (templated) :type project_id: str :param dataset_id: The dataset to create the table into. (templated) @@ -854,6 +874,10 @@ class BigQueryCreateExternalTableOperator(BaseOperator): point the operator to a Google Cloud Storage object name. The object in Google Cloud Storage must be a JSON file with the schema fields in it. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryCreateExternalTableOperator` + :param bucket: The bucket to point the external table to. (templated) :type bucket: str :param source_objects: List of Google Cloud Storage URIs to point @@ -1076,6 +1100,10 @@ class BigQueryDeleteDatasetOperator(BaseOperator): This operator deletes an existing dataset from your Project in Big query. https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets/delete + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryDeleteDatasetOperator` + :param project_id: The project id of the dataset. :type project_id: str :param dataset_id: The dataset to be deleted. @@ -1150,6 +1178,10 @@ class BigQueryCreateEmptyDatasetOperator(BaseOperator): This operator is used to create new dataset for your Project in BigQuery. https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryCreateEmptyDatasetOperator` + :param project_id: The name of the project where we want to create the dataset. :type project_id: str :param dataset_id: The id of dataset. Don't need to provide, if datasetId in dataset_reference. @@ -1231,6 +1263,10 @@ class BigQueryGetDatasetOperator(BaseOperator): """ This operator is used to return the dataset specified by dataset_id. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryGetDatasetOperator` + :param dataset_id: The id of dataset. Don't need to provide, if datasetId in dataset_reference. :type dataset_id: str @@ -1275,6 +1311,10 @@ class BigQueryGetDatasetTablesOperator(BaseOperator): """ This operator retrieves the list of tables in the specified dataset. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryGetDatasetTablesOperator` + :param dataset_id: the dataset ID of the requested dataset. :type dataset_id: str :param project_id: (Optional) the project of the requested dataset. If None, @@ -1327,6 +1367,10 @@ class BigQueryPatchDatasetOperator(BaseOperator): This operator is used to patch dataset for your Project in BigQuery. It only replaces fields that are provided in the submitted dataset resource. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryPatchDatasetOperator` + :param dataset_id: The id of dataset. Don't need to provide, if datasetId in dataset_reference. :type dataset_id: str @@ -1388,6 +1432,10 @@ class BigQueryUpdateDatasetOperator(BaseOperator): If no ``fields`` are provided then all fields of provided ``dataset_reources`` will be used. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryUpdateDatasetOperator` + :param dataset_id: The id of dataset. Don't need to provide, if datasetId in dataset_reference. :type dataset_id: str @@ -1447,6 +1495,10 @@ class BigQueryDeleteTableOperator(BaseOperator): """ Deletes BigQuery tables + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryDeleteTableOperator` + :param deletion_dataset_table: A dotted ``(.|:).`` that indicates which table will be deleted. (templated) @@ -1512,6 +1564,10 @@ class BigQueryUpsertTableOperator(BaseOperator): """ Upsert BigQuery table + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryUpsertTableOperator` + :param dataset_id: A dotted ``(.|:)`` that indicates which dataset will be updated. (templated) @@ -1586,6 +1642,11 @@ class BigQueryInsertJobOperator(BaseOperator): https://cloud.google.com/bigquery/docs/reference/v2/jobs + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:BigQueryInsertJobOperator` + + :param configuration: The configuration parameter maps directly to BigQuery's configuration field in the job object. For more details see https://cloud.google.com/bigquery/docs/reference/v2/jobs diff --git a/airflow/providers/google/cloud/operators/dataproc.py b/airflow/providers/google/cloud/operators/dataproc.py index 9d537bf02dc0..b9b3da179106 100644 --- a/airflow/providers/google/cloud/operators/dataproc.py +++ b/airflow/providers/google/cloud/operators/dataproc.py @@ -428,6 +428,10 @@ class DataprocCreateClusterOperator(BaseOperator): for a detailed explanation on the different parameters. Most of the configuration parameters detailed in the link are available as a parameter to this operator. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:DataprocCreateClusterOperator` + :param project_id: The ID of the google cloud project in which to create the cluster. (templated) :type project_id: str diff --git a/airflow/providers/google/cloud/operators/life_sciences.py b/airflow/providers/google/cloud/operators/life_sciences.py index 45c3bb3ef101..98ee1ead09bf 100644 --- a/airflow/providers/google/cloud/operators/life_sciences.py +++ b/airflow/providers/google/cloud/operators/life_sciences.py @@ -29,6 +29,10 @@ class LifeSciencesRunPipelineOperator(BaseOperator): """ Runs a Life Sciences Pipeline + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:LifeSciencesRunPipelineOperator` + :param body: The request body :type body: dict :param location: The location of the project diff --git a/airflow/providers/google/cloud/transfers/gcs_to_local.py b/airflow/providers/google/cloud/transfers/gcs_to_local.py index ffd33157dde9..9c564bda4f0f 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_local.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_local.py @@ -34,6 +34,10 @@ class GCSToLocalFilesystemOperator(BaseOperator): set the ``store_to_xcom_key`` parameter to True push the file content into xcom. When the file size exceeds the maximum size for xcom it is recommended to write to a file. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:GCSToLocalFilesystemOperator` + :param bucket: The Google Cloud Storage bucket where the object is. Must not contain 'gs://' prefix. (templated) :type bucket: str diff --git a/airflow/providers/google/cloud/transfers/local_to_gcs.py b/airflow/providers/google/cloud/transfers/local_to_gcs.py index 1a5cea9eb291..51a4661f6776 100644 --- a/airflow/providers/google/cloud/transfers/local_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/local_to_gcs.py @@ -30,6 +30,10 @@ class LocalFilesystemToGCSOperator(BaseOperator): Uploads a file to Google Cloud Storage. Optionally can compress the file for upload. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:LocalFilesystemToGCSOperator` + :param src: Path to the local file. (templated) :type src: str :param dst: The object name to set when uploading the file. (templated) diff --git a/airflow/providers/google/marketing_platform/operators/analytics.py b/airflow/providers/google/marketing_platform/operators/analytics.py index 621ac0205b6e..9564596b7485 100644 --- a/airflow/providers/google/marketing_platform/operators/analytics.py +++ b/airflow/providers/google/marketing_platform/operators/analytics.py @@ -143,7 +143,7 @@ class GoogleAnalyticsRetrieveAdsLinksListOperator(BaseOperator): .. seealso:: For more information on how to use this operator, take a look at the guide: - :ref:`howto/operator:GoogleAnalyticsListAccountsOperator` + :ref:`howto/operator:GoogleAnalyticsRetrieveAdsLinksListOperator` :param account_id: ID of the account which the given web property belongs to. :type account_id: str diff --git a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py index ec2882aa4769..b3fa6e455296 100644 --- a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py +++ b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py @@ -41,6 +41,10 @@ class GCSToGoogleDriveOperator(BaseOperator): https://www.googleapis.com/auth/drive + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:GCSToGoogleDriveOperator` + :param source_bucket: The source Google Cloud Storage bucket where the object is. (templated) :type source_bucket: str :param source_object: The source name of the object to copy in the Google cloud diff --git a/airflow/providers/http/operators/http.py b/airflow/providers/http/operators/http.py index b18ffc05e6e6..87df78df6998 100644 --- a/airflow/providers/http/operators/http.py +++ b/airflow/providers/http/operators/http.py @@ -27,6 +27,10 @@ class SimpleHttpOperator(BaseOperator): """ Calls an endpoint on an HTTP system to execute an action + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:SimpleHttpOperator` + :param http_conn_id: The connection to run the operator against :type http_conn_id: str :param endpoint: The relative part of the full url. (templated) diff --git a/airflow/providers/http/sensors/http.py b/airflow/providers/http/sensors/http.py index 89eef4723971..f4eb00a53791 100644 --- a/airflow/providers/http/sensors/http.py +++ b/airflow/providers/http/sensors/http.py @@ -44,6 +44,9 @@ def response_check(response, task_instance): HttpSensor(task_id='my_http_sensor', ..., response_check=response_check) + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:HttpSensor` :param http_conn_id: The connection to run the sensor against :type http_conn_id: str diff --git a/airflow/providers/yandex/operators/yandexcloud_dataproc.py b/airflow/providers/yandex/operators/yandexcloud_dataproc.py index 8ba4899a7d4a..e72a516334ed 100644 --- a/airflow/providers/yandex/operators/yandexcloud_dataproc.py +++ b/airflow/providers/yandex/operators/yandexcloud_dataproc.py @@ -25,6 +25,10 @@ class DataprocCreateClusterOperator(BaseOperator): """Creates Yandex.Cloud Data Proc cluster. + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:DataprocCreateClusterOperator` + :param folder_id: ID of the folder in which cluster should be created. :type folder_id: Optional[str] :param cluster_name: Cluster name. Must be unique inside the folder. diff --git a/docs/build b/docs/build index 8266243c0034..e9ddcc020de3 100755 --- a/docs/build +++ b/docs/build @@ -15,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import ast import atexit import os import re @@ -24,6 +25,7 @@ import sys from contextlib import suppress from functools import total_ordering from glob import glob +from itertools import chain from subprocess import run from tempfile import NamedTemporaryFile from typing import Iterable, List, NamedTuple, Optional, Set @@ -127,6 +129,89 @@ def display_errors_summary() -> None: print("=" * 50) +def find_existing_guide_operator_names(): + operator_names = set() + + paths = glob("howto/operator/**/*.rst", recursive=True) + for path in paths: + with open(path) as f: + operator_names |= set(re.findall(".. _howto/operator:(.+?):", f.read())) + + return operator_names + + +def extract_ast_class_def_by_name(ast_tree, class_name): + class ClassVisitor(ast.NodeVisitor): + def __init__(self): + self.found_class_node = None + + def visit_ClassDef(self, node): + if node.name == class_name: + self.found_class_node = node + + visitor = ClassVisitor() + visitor.visit(ast_tree) + + return visitor.found_class_node + + +def check_guide_links_in_operator_descriptions(): + def generate_build_error(path, line_no, operator_name): + return DocBuildError( + file_path=path, + line_no=line_no, + message=( + f"Link to the guide is missing in operator's description: {operator_name}.\n" + f"Please add link to the guide to the description in the following form:\n" + f"\n" + f".. seealso::\n" + f" For more information on how to use this operator, take a look at the guide:\n" + f" :ref:`howto/operator:{operator_name}`\n" + ) + ) + + # Extract operators for which there are existing .rst guides + operator_names = find_existing_guide_operator_names() + + # Extract all potential python modules that can contain operators + python_module_paths = chain( + glob(f"{ROOT_PACKAGE_DIR}/operators/*.py"), + glob(f"{ROOT_PACKAGE_DIR}/sensors/*.py"), + glob(f"{ROOT_PACKAGE_DIR}/providers/**/operators/*.py", recursive=True), + glob(f"{ROOT_PACKAGE_DIR}/providers/**/sensors/*.py", recursive=True), + glob(f"{ROOT_PACKAGE_DIR}/providers/**/transfers/*.py", recursive=True), + ) + + for py_module_path in python_module_paths: + with open(py_module_path) as f: + py_content = f.read() + + if "This module is deprecated" in py_content: + continue + + for existing_operator in operator_names: + if f"class {existing_operator}" not in py_content: + continue + # This is a potential file with necessary class definition. + # To make sure it's a real Python class definition, we build AST tree + ast_tree = ast.parse(py_content) + class_def = extract_ast_class_def_by_name(ast_tree, existing_operator) + + if class_def is None: + continue + + docstring = ast.get_docstring(class_def) + if "This class is deprecated." in docstring: + continue + + if f":ref:`howto/operator:{existing_operator}`" in ast.get_docstring(class_def): + continue + + build_errors.append( + generate_build_error(py_module_path, class_def.lineno, existing_operator) + ) + + def assert_file_not_contains(file_path: str, pattern: str, message: str) -> None: with open(file_path, "rb", 0) as doc_file: pattern_compiled = re.compile(pattern) @@ -401,6 +486,7 @@ print("Current working directory: ", os.getcwd()) prepare_directories() clean_files() +check_guide_links_in_operator_descriptions() check_class_links_in_operators_and_hooks_ref() check_guide_links_in_operators_and_hooks_ref() check_exampleinclude_for_example_dags() diff --git a/docs/howto/operator/http/http.rst b/docs/howto/operator/http/http.rst index b00a4f395017..edda7b686b9d 100644 --- a/docs/howto/operator/http/http.rst +++ b/docs/howto/operator/http/http.rst @@ -23,6 +23,9 @@ The following code examples use the ``http_default`` connection which means the .. _howto/operator:HttpSensor: +HttpSensor +========== + Use the :class:`~airflow.providers.http.sensors.http.HttpSensor` to poke until the ``response_check`` callable evaluates to ``true``. @@ -35,6 +38,9 @@ Here we are poking until httpbin gives us a response text containing ``httpbin`` .. _howto/operator:SimpleHttpOperator: +SimpleHttpOperator +================== + Use the :class:`~airflow.providers.http.operators.http.SimpleHttpOperator` to call HTTP requests and get the response text back.