diff --git a/airflow/providers/amazon/aws/example_dags/example_athena.py b/airflow/providers/amazon/aws/example_dags/example_athena.py
index fe248b2fb34e6..26b47bdfa19b5 100644
--- a/airflow/providers/amazon/aws/example_dags/example_athena.py
+++ b/airflow/providers/amazon/aws/example_dags/example_athena.py
@@ -84,21 +84,21 @@ def read_results_from_s3(query_execution_id):
output_location=f's3://{S3_BUCKET}/{S3_KEY}',
)
- # [START howto_athena_operator]
+ # [START howto_operator_athena]
read_table = AthenaOperator(
task_id='read_table',
query=QUERY_READ_TABLE,
database=ATHENA_DATABASE,
output_location=f's3://{S3_BUCKET}/{S3_KEY}',
)
- # [END howto_athena_operator]
+ # [END howto_operator_athena]
- # [START howto_athena_sensor]
+ # [START howto_sensor_athena]
await_query = AthenaSensor(
task_id='await_query',
query_execution_id=read_table.output,
)
- # [END howto_athena_sensor]
+ # [END howto_sensor_athena]
drop_table = AthenaOperator(
task_id='drop_table',
diff --git a/airflow/providers/amazon/aws/example_dags/example_dms.py b/airflow/providers/amazon/aws/example_dags/example_dms.py
index e2ce8b852b255..caffe44353fda 100644
--- a/airflow/providers/amazon/aws/example_dags/example_dms.py
+++ b/airflow/providers/amazon/aws/example_dags/example_dms.py
@@ -319,12 +319,12 @@ def clean_up():
# [END howto_operator_dms_stop_task]
# TaskCompletedSensor actually waits until task reaches the "Stopped" state, so it will work here.
- # [START howto_operator_dms_task_completed_sensor]
+ # [START howto_sensor_dms_task_completed]
await_task_stop = DmsTaskCompletedSensor(
task_id='await_task_stop',
replication_task_arn=create_task.output,
)
- # [END howto_operator_dms_task_completed_sensor]
+ # [END howto_sensor_dms_task_completed]
# [START howto_operator_dms_delete_task]
delete_task = DmsDeleteTaskOperator(
diff --git a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py
index d8cc76450e718..66334fc996522 100644
--- a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py
+++ b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py
@@ -18,6 +18,7 @@
from os import environ
from airflow import DAG
+from airflow.models.baseoperator import chain
from airflow.providers.amazon.aws.transfers.dynamodb_to_s3 import DynamoDBToS3Operator
TABLE_NAME = environ.get('DYNAMO_TABLE_NAME', 'ExistingDynamoDbTableName')
@@ -31,7 +32,6 @@
tags=['example'],
catchup=False,
) as dag:
-
# [START howto_transfer_dynamodb_to_s3]
backup_db = DynamoDBToS3Operator(
task_id='backup_db',
@@ -41,3 +41,32 @@
file_size=1000,
)
# [END howto_transfer_dynamodb_to_s3]
+
+ # [START howto_transfer_dynamodb_to_s3_segmented]
+ # Segmenting allows the transfer to be parallelized into {segment} number of parallel tasks.
+ backup_db_segment_1 = DynamoDBToS3Operator(
+ task_id='backup-1',
+ dynamodb_table_name=TABLE_NAME,
+ s3_bucket_name=BUCKET_NAME,
+ # Max output file size in bytes. If the Table is too large, multiple files will be created.
+ file_size=1000,
+ dynamodb_scan_kwargs={
+ "TotalSegments": 2,
+ "Segment": 0,
+ },
+ )
+
+ backup_db_segment_2 = DynamoDBToS3Operator(
+ task_id="backup-2",
+ dynamodb_table_name=TABLE_NAME,
+ s3_bucket_name=BUCKET_NAME,
+ # Max output file size in bytes. If the Table is too large, multiple files will be created.
+ file_size=1000,
+ dynamodb_scan_kwargs={
+ "TotalSegments": 2,
+ "Segment": 1,
+ },
+ )
+ # [END howto_transfer_dynamodb_to_s3_segmented]
+
+ chain(backup_db, [backup_db_segment_1, backup_db_segment_2])
diff --git a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3_segmented.py b/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3_segmented.py
deleted file mode 100644
index dc1bc13ae0274..0000000000000
--- a/airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3_segmented.py
+++ /dev/null
@@ -1,60 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied. See the License for the
-# specific language governing permissions and limitations
-# under the License.
-from datetime import datetime
-from os import environ
-
-from airflow import DAG
-from airflow.providers.amazon.aws.transfers.dynamodb_to_s3 import DynamoDBToS3Operator
-
-TABLE_NAME = environ.get('DYNAMO_TABLE_NAME', 'ExistingDynamoDbTableName')
-BUCKET_NAME = environ.get('S3_BUCKET_NAME', 'ExistingS3BucketName')
-
-
-with DAG(
- dag_id='example_dynamodb_to_s3_segmented',
- schedule_interval=None,
- start_date=datetime(2021, 1, 1),
- tags=['example'],
- catchup=False,
-) as dag:
-
- # [START howto_transfer_dynamodb_to_s3_segmented]
- # Segmenting allows the transfer to be parallelized into {segment} number of parallel tasks.
- backup_db_segment_1 = DynamoDBToS3Operator(
- task_id='backup-1',
- dynamodb_table_name=TABLE_NAME,
- s3_bucket_name=BUCKET_NAME,
- # Max output file size in bytes. If the Table is too large, multiple files will be created.
- file_size=1000,
- dynamodb_scan_kwargs={
- "TotalSegments": 2,
- "Segment": 0,
- },
- )
-
- backup_db_segment_2 = DynamoDBToS3Operator(
- task_id="backup-2",
- dynamodb_table_name=TABLE_NAME,
- s3_bucket_name=BUCKET_NAME,
- # Max output file size in bytes. If the Table is too large, multiple files will be created.
- file_size=1000,
- dynamodb_scan_kwargs={
- "TotalSegments": 2,
- "Segment": 1,
- },
- )
- # [END howto_transfer_dynamodb_to_s3_segmented]
diff --git a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py
index 2c9f732caf444..2df40a7d0c37b 100644
--- a/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py
+++ b/airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py
@@ -32,20 +32,20 @@
start_date=datetime(2021, 1, 1), # Override to match your needs
catchup=False,
) as dag:
- # [START howto_glacier_create_job_operator]
+ # [START howto_operator_glacier_create_job]
create_glacier_job = GlacierCreateJobOperator(task_id="create_glacier_job", vault_name=VAULT_NAME)
JOB_ID = '{{ task_instance.xcom_pull("create_glacier_job")["jobId"] }}'
- # [END howto_glacier_create_job_operator]
+ # [END howto_operator_glacier_create_job]
- # [START howto_glacier_job_operation_sensor]
+ # [START howto_sensor_glacier_job_operation]
wait_for_operation_complete = GlacierJobOperationSensor(
vault_name=VAULT_NAME,
job_id=JOB_ID,
task_id="wait_for_operation_complete",
)
- # [END howto_glacier_job_operation_sensor]
+ # [END howto_sensor_glacier_job_operation]
- # [START howto_glacier_transfer_data_to_gcs]
+ # [START howto_transfer_glacier_to_gcs]
transfer_archive_to_gcs = GlacierToGCSOperator(
task_id="transfer_archive_to_gcs",
vault_name=VAULT_NAME,
@@ -57,6 +57,6 @@
# then whole file will be downloaded
chunk_size=1024,
)
- # [END howto_glacier_transfer_data_to_gcs]
+ # [END howto_transfer_glacier_to_gcs]
create_glacier_job >> wait_for_operation_complete >> transfer_archive_to_gcs
diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py
index fb9c06d85c558..7b6e4b291a66c 100644
--- a/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py
+++ b/airflow/providers/amazon/aws/example_dags/example_google_api_sheets_to_s3.py
@@ -15,7 +15,7 @@
# specific language governing permissions and limitations
# under the License.
"""
-This is a basic example dag for using `GoogleApiToS3Transfer` to retrieve Google Sheets data
+This is a basic example dag for using `GoogleApiToS3Operator` to retrieve Google Sheets data
You need to set all env variables to request the data.
"""
diff --git a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py
index 241a551320efd..682cc83919129 100644
--- a/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py
+++ b/airflow/providers/amazon/aws/example_dags/example_google_api_youtube_to_s3.py
@@ -15,7 +15,7 @@
# specific language governing permissions and limitations
# under the License.
"""
-This is a more advanced example dag for using `GoogleApiToS3Transfer` which uses xcom to pass data between
+This is a more advanced example dag for using `GoogleApiToS3Operator` which uses xcom to pass data between
tasks to retrieve specific information about YouTube videos:
First it searches for up to 50 videos (due to pagination) in a given time range
diff --git a/airflow/providers/amazon/aws/example_dags/example_lambda.py b/airflow/providers/amazon/aws/example_dags/example_lambda.py
index 17ba0dab48eb9..3b87c3aa3c3fb 100644
--- a/airflow/providers/amazon/aws/example_dags/example_lambda.py
+++ b/airflow/providers/amazon/aws/example_dags/example_lambda.py
@@ -36,10 +36,10 @@
tags=['example'],
catchup=False,
) as dag:
- # [START howto_lambda_operator]
+ # [START howto_operator_lambda]
invoke_lambda_function = AwsLambdaInvokeFunctionOperator(
task_id='setup__invoke_lambda_function',
function_name=LAMBDA_FUNCTION_NAME,
payload=SAMPLE_EVENT,
)
- # [END howto_lambda_operator]
+ # [END howto_operator_lambda]
diff --git a/airflow/providers/amazon/aws/example_dags/example_redshift_data_execute_sql.py b/airflow/providers/amazon/aws/example_dags/example_redshift_data_execute_sql.py
index 38ac3ef1424c7..cfa3e4cefcb2d 100644
--- a/airflow/providers/amazon/aws/example_dags/example_redshift_data_execute_sql.py
+++ b/airflow/providers/amazon/aws/example_dags/example_redshift_data_execute_sql.py
@@ -57,7 +57,7 @@ def output_query_results(statement_id):
catchup=False,
tags=['example'],
) as dag:
- # [START howto_redshift_data]
+ # [START howto_operator_redshift_data]
task_query = RedshiftDataOperator(
task_id='redshift_query',
cluster_identifier=REDSHIFT_CLUSTER_IDENTIFIER,
@@ -67,6 +67,6 @@ def output_query_results(statement_id):
poll_interval=POLL_INTERVAL,
await_result=True,
)
- # [END howto_redshift_data]
+ # [END howto_operator_redshift_data]
task_output = output_query_results(task_query.output)
diff --git a/airflow/providers/amazon/aws/example_dags/example_sagemaker.py b/airflow/providers/amazon/aws/example_dags/example_sagemaker.py
index df69013e1c473..41e7666eabb8a 100644
--- a/airflow/providers/amazon/aws/example_dags/example_sagemaker.py
+++ b/airflow/providers/amazon/aws/example_dags/example_sagemaker.py
@@ -393,12 +393,12 @@ def cleanup():
)
# [END howto_operator_sagemaker_training]
- # [START howto_operator_sagemaker_training_sensor]
+ # [START howto_sensor_sagemaker_training]
await_training = SageMakerTrainingSensor(
task_id='await_training',
job_name=TRAINING_JOB_NAME,
)
- # [END howto_operator_sagemaker_training_sensor]
+ # [END howto_sensor_sagemaker_training]
# [START howto_operator_sagemaker_model]
create_model = SageMakerModelOperator(
@@ -418,12 +418,12 @@ def cleanup():
)
# [END howto_operator_sagemaker_tuning]
- # [START howto_operator_sagemaker_tuning_sensor]
+ # [START howto_sensor_sagemaker_tuning]
await_tune = SageMakerTuningSensor(
task_id='await_tuning',
job_name=TUNING_JOB_NAME,
)
- # [END howto_operator_sagemaker_tuning_sensor]
+ # [END howto_sensor_sagemaker_tuning]
# [START howto_operator_sagemaker_transform]
test_model = SageMakerTransformOperator(
@@ -435,12 +435,12 @@ def cleanup():
)
# [END howto_operator_sagemaker_transform]
- # [START howto_operator_sagemaker_transform_sensor]
+ # [START howto_sensor_sagemaker_transform]
await_transform = SageMakerTransformSensor(
task_id='await_transform',
job_name=TRANSFORM_JOB_NAME,
)
- # [END howto_operator_sagemaker_transform_sensor]
+ # [END howto_sensor_sagemaker_transform]
# Trigger rule set to "all_done" so clean up will run regardless of success on other tasks.
# [START howto_operator_sagemaker_delete_model]
diff --git a/airflow/providers/amazon/aws/example_dags/example_sagemaker_endpoint.py b/airflow/providers/amazon/aws/example_dags/example_sagemaker_endpoint.py
index b4207a9b163fc..52ee5303f2553 100644
--- a/airflow/providers/amazon/aws/example_dags/example_sagemaker_endpoint.py
+++ b/airflow/providers/amazon/aws/example_dags/example_sagemaker_endpoint.py
@@ -202,13 +202,13 @@ def cleanup():
)
# [END howto_operator_sagemaker_endpoint]
- # [START howto_operator_sagemaker_endpoint_sensor]
+ # [START howto_sensor_sagemaker_endpoint]
await_endpoint = SageMakerEndpointSensor(
task_id='await_endpoint',
endpoint_name=ENDPOINT_NAME,
do_xcom_push=False,
)
- # [END howto_operator_sagemaker_endpoint_sensor]
+ # [END howto_sensor_sagemaker_endpoint]
# Trigger rule set to "all_done" so clean up will run regardless of success on other tasks.
delete_model = SageMakerDeleteModelOperator(
diff --git a/airflow/providers/amazon/aws/example_dags/example_sqs.py b/airflow/providers/amazon/aws/example_dags/example_sqs.py
index 996d098521374..69ff1e5b75831 100644
--- a/airflow/providers/amazon/aws/example_dags/example_sqs.py
+++ b/airflow/providers/amazon/aws/example_dags/example_sqs.py
@@ -52,20 +52,20 @@ def delete_queue_fn(queue_url):
create_queue = create_queue_fn()
- # [START howto_sqs_operator]
+ # [START howto_operator_sqs]
publish_to_queue = SqsPublishOperator(
task_id='publish_to_queue',
sqs_queue=create_queue,
message_content="{{ task_instance }}-{{ execution_date }}",
)
- # [END howto_sqs_operator]
+ # [END howto_operator_sqs]
- # [START howto_sqs_sensor]
+ # [START howto_sensor_sqs]
read_from_queue = SqsSensor(
task_id='read_from_queue',
sqs_queue=create_queue,
)
- # [END howto_sqs_sensor]
+ # [END howto_sensor_sqs]
delete_queue = delete_queue_fn(create_queue)
diff --git a/airflow/providers/amazon/aws/example_dags/example_step_functions.py b/airflow/providers/amazon/aws/example_dags/example_step_functions.py
index 9a0ac2474cfc0..02763e3ea13f1 100644
--- a/airflow/providers/amazon/aws/example_dags/example_step_functions.py
+++ b/airflow/providers/amazon/aws/example_dags/example_step_functions.py
@@ -41,11 +41,11 @@
)
# [END howto_operator_step_function_start_execution]
- # [START howto_operator_step_function_execution_sensor]
+ # [START howto_sensor_step_function_execution]
wait_for_execution = StepFunctionExecutionSensor(
task_id='wait_for_execution', execution_arn=start_execution.output
)
- # [END howto_operator_step_function_execution_sensor]
+ # [END howto_sensor_step_function_execution]
# [START howto_operator_step_function_get_execution_output]
get_execution_output = StepFunctionGetExecutionOutputOperator(
diff --git a/airflow/providers/amazon/aws/sensors/athena.py b/airflow/providers/amazon/aws/sensors/athena.py
index c94053ead6efa..927f512143362 100644
--- a/airflow/providers/amazon/aws/sensors/athena.py
+++ b/airflow/providers/amazon/aws/sensors/athena.py
@@ -38,7 +38,7 @@ class AthenaSensor(BaseSensorOperator):
.. seealso::
For more information on how to use this operator, take a look at the guide:
- :ref:`howto/operator:AthenaSensor`
+ :ref:`howto/sensor:AthenaSensor`
:param query_execution_id: query_execution_id to check the state of
diff --git a/airflow/providers/amazon/aws/sensors/cloud_formation.py b/airflow/providers/amazon/aws/sensors/cloud_formation.py
index 2add211bdf0e4..fb01bdc7f6827 100644
--- a/airflow/providers/amazon/aws/sensors/cloud_formation.py
+++ b/airflow/providers/amazon/aws/sensors/cloud_formation.py
@@ -37,7 +37,7 @@ class CloudFormationCreateStackSensor(BaseSensorOperator):
.. seealso::
For more information on how to use this operator, take a look at the guide:
- :ref:`howto/operator:CloudFormationCreateStackSensor`
+ :ref:`howto/sensor:CloudFormationCreateStackSensor`
:param stack_name: The name of the stack to wait for (templated)
@@ -75,8 +75,7 @@ class CloudFormationDeleteStackSensor(BaseSensorOperator):
.. seealso::
For more information on how to use this operator, take a look at the guide:
- :ref:`howto/operator:CloudFormationDeleteStackSensor`
-
+ :ref:`howto/sensor:CloudFormationDeleteStackSensor`
:param stack_name: The name of the stack to wait for (templated)
:param aws_conn_id: ID of the Airflow connection where credentials and extra configuration are
diff --git a/airflow/providers/amazon/aws/sensors/glacier.py b/airflow/providers/amazon/aws/sensors/glacier.py
index dec97f7293a79..e92f5a4326b3a 100644
--- a/airflow/providers/amazon/aws/sensors/glacier.py
+++ b/airflow/providers/amazon/aws/sensors/glacier.py
@@ -39,7 +39,7 @@ class GlacierJobOperationSensor(BaseSensorOperator):
.. seealso::
For more information on how to use this operator, take a look at the guide:
- :ref:`howto/operator:GlacierJobOperationSensor`
+ :ref:`howto/sensor:GlacierJobOperationSensor`
:param aws_conn_id: The reference to the AWS connection details
:param vault_name: name of Glacier vault on which job is executed
diff --git a/airflow/providers/amazon/aws/sensors/sqs.py b/airflow/providers/amazon/aws/sensors/sqs.py
index 2d1b800c4f789..cc026ec5e8306 100644
--- a/airflow/providers/amazon/aws/sensors/sqs.py
+++ b/airflow/providers/amazon/aws/sensors/sqs.py
@@ -39,7 +39,7 @@ class SqsSensor(BaseSensorOperator):
.. seealso::
For more information on how to use this sensor, take a look at the guide:
- :ref:`howto/operator:SqsSensor`
+ :ref:`howto/sensor:SqsSensor`
:param aws_conn_id: AWS connection id
:param sqs_queue: The SQS queue url (templated)
diff --git a/airflow/providers/amazon/aws/sensors/step_function.py b/airflow/providers/amazon/aws/sensors/step_function.py
index 085ca21fc4901..6f82c0bc99d8c 100644
--- a/airflow/providers/amazon/aws/sensors/step_function.py
+++ b/airflow/providers/amazon/aws/sensors/step_function.py
@@ -37,7 +37,7 @@ class StepFunctionExecutionSensor(BaseSensorOperator):
.. seealso::
For more information on how to use this sensor, take a look at the guide:
- :ref:`howto/operator:StepFunctionExecutionSensor`
+ :ref:`howto/sensor:StepFunctionExecutionSensor`
:param execution_arn: execution_arn to check the state of
:param aws_conn_id: aws connection to use, defaults to 'aws_default'
diff --git a/airflow/providers/amazon/aws/transfers/google_api_to_s3.py b/airflow/providers/amazon/aws/transfers/google_api_to_s3.py
index dd794c348b5bf..34b590cb2d209 100644
--- a/airflow/providers/amazon/aws/transfers/google_api_to_s3.py
+++ b/airflow/providers/amazon/aws/transfers/google_api_to_s3.py
@@ -43,6 +43,10 @@ class GoogleApiToS3Operator(BaseOperator):
Therefore it is recommended that you use the custom Google Cloud Service Operators for working
with the Google Cloud Platform.
+ .. seealso::
+ For more information on how to use this operator, take a look at the guide:
+ :ref:`howto/operator:GoogleApiToS3Operator`
+
:param google_api_service_name: The specific API service that is being requested.
:param google_api_service_version: The version of the API that is being requested.
:param google_api_endpoint_path: The client libraries path to the api call's executing method.
diff --git a/docs/apache-airflow-providers-amazon/operators/athena.rst b/docs/apache-airflow-providers-amazon/operators/athena.rst
index ea96d8127df24..1e132dc192003 100644
--- a/docs/apache-airflow-providers-amazon/operators/athena.rst
+++ b/docs/apache-airflow-providers-amazon/operators/athena.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-Amazon Athena Operators
-=======================
+=============
+Amazon Athena
+=============
`Amazon Athena `__ is an interactive query service
that makes it easy to analyze data in Amazon Simple Storage Service (S3) using
@@ -26,48 +26,50 @@ manage, and you pay only for the queries you run. To get started, simply point
to your data in S3, define the schema, and start querying using standard SQL.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
.. _howto/operator:AthenaOperator:
-Athena Operator
-^^^^^^^^^^^^^^^
+Run a query in Amazon Athena
+============================
Use the :class:`~airflow.providers.amazon.aws.operators.athena.AthenaOperator`
to run a query in Amazon Athena.
-
In the following example, we query an existing Athena table and send the results to
an existing Amazon S3 bucket. For more examples of how to use this operator, please
see the `Sample DAG `__.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_athena.py
:language: python
- :start-after: [START howto_athena_operator]
+ :start-after: [START howto_operator_athena]
:dedent: 4
- :end-before: [END howto_athena_operator]
+ :end-before: [END howto_operator_athena]
-.. _howto/operator:AthenaSensor:
+Sensors
+-------
-Athena Sensor
-^^^^^^^^^^^^^
+.. _howto/sensor:AthenaSensor:
+
+Wait on Amazon Athena query results
+===================================
Use the :class:`~airflow.providers.amazon.aws.sensors.athena.AthenaSensor`
to wait for the results of a query in Amazon Athena.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_athena.py
:language: python
- :start-after: [START howto_athena_sensor]
+ :start-after: [START howto_sensor_athena]
:dedent: 4
- :end-before: [END howto_athena_sensor]
+ :end-before: [END howto_sensor_athena]
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
-* `Boto3 Library Documentation for Athena `__
+* `AWS boto3 library documentation for Athena `__
diff --git a/docs/apache-airflow-providers-amazon/operators/batch.rst b/docs/apache-airflow-providers-amazon/operators/batch.rst
index 87d371b8c8a49..663cf24cece2a 100644
--- a/docs/apache-airflow-providers-amazon/operators/batch.rst
+++ b/docs/apache-airflow-providers-amazon/operators/batch.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-AWS Batch Operators
-===================
+=========
+AWS Batch
+=========
`AWS Batch `__ enables you to run batch computing workloads on the AWS Cloud.
Batch computing is a common way for developers, scientists, and engineers to access large amounts of compute
@@ -29,37 +29,41 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
-.. _howto/sensor:BatchSensor:
+Operators
+---------
-AWS Batch Sensor
-""""""""""""""""
+.. _howto/operator:BatchOperator:
-To wait on the state of an AWS Batch Job until it reaches a terminal state you can
-use :class:`~airflow.providers.amazon.aws.sensors.batch.BatchSensor`.
+Submit a new AWS Batch job
+==========================
+
+To submit a new AWS Batch job and monitor it until it reaches a terminal state you can
+use :class:`~airflow.providers.amazon.aws.operators.batch.BatchOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_batch.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_batch]
- :end-before: [END howto_sensor_batch]
+ :start-after: [START howto_operator_batch]
+ :end-before: [END howto_operator_batch]
-.. _howto/operator:BatchOperator:
+Sensors
+-------
-AWS Batch Operator
-""""""""""""""""""
+.. _howto/sensor:BatchSensor:
-To submit a new AWS Batch Job and monitor it until it reaches a terminal state you can
-use :class:`~airflow.providers.amazon.aws.operators.batch.BatchOperator`.
+Wait on an AWS Batch job state
+==============================
+
+To wait on the state of an AWS Batch Job until it reaches a terminal state you can
+use :class:`~airflow.providers.amazon.aws.sensors.batch.BatchSensor`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_batch.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_batch]
- :end-before: [END howto_operator_batch]
+ :start-after: [START howto_sensor_batch]
+ :end-before: [END howto_sensor_batch]
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for Batch `__
+* `AWS boto3 library documentation for Batch `__
diff --git a/docs/apache-airflow-providers-amazon/operators/cloudformation.rst b/docs/apache-airflow-providers-amazon/operators/cloudformation.rst
index 9ddb153bb9b06..f83a2861abead 100644
--- a/docs/apache-airflow-providers-amazon/operators/cloudformation.rst
+++ b/docs/apache-airflow-providers-amazon/operators/cloudformation.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-AWS CloudFormation Operators
-============================
+==================
+AWS CloudFormation
+==================
`AWS CloudFormation `__ enables you to create and provision AWS
infrastructure deployments predictably and repeatedly. It helps you leverage AWS products such as Amazon
@@ -31,10 +31,13 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:CloudFormationCreateStackOperator:
-AWS CloudFormation Create Stack Operator
-""""""""""""""""""""""""""""""""""""""""
+Create an AWS CloudFormation stack
+==================================
To create a new AWS CloudFormation stack use
:class:`~airflow.providers.amazon.aws.operators.cloud_formation.CloudFormationCreateStackOperator`.
@@ -45,40 +48,41 @@ To create a new AWS CloudFormation stack use
:start-after: [START howto_operator_cloudformation_create_stack]
:end-before: [END howto_operator_cloudformation_create_stack]
+.. _howto/operator:CloudFormationDeleteStackOperator:
-.. _howto/operator:CloudFormationCreateStackSensor:
-
-AWS CloudFormation Create Stack Sensor
-""""""""""""""""""""""""""""""""""""""
+Delete an AWS CloudFormation stack
+==================================
-To wait on the state of an AWS CloudFormation stack creation until it reaches a terminal state you can use
-:class:`~airflow.providers.amazon.aws.sensors.cloud_formation.CloudFormationCreateStackSensor`
+To delete an AWS CloudFormation stack you can use
+:class:`~airflow.providers.amazon.aws.operators.cloud_formation.CloudFormationDeleteStackOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_cloudformation.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_cloudformation_create_stack]
- :end-before: [END howto_sensor_cloudformation_create_stack]
+ :start-after: [START howto_operator_cloudformation_delete_stack]
+ :end-before: [END howto_operator_cloudformation_delete_stack]
-.. _howto/operator:CloudFormationDeleteStackOperator:
+Sensors
+-------
-AWS CloudFormation Delete Stack Operator
-""""""""""""""""""""""""""""""""""""""""
+.. _howto/sensor:CloudFormationCreateStackSensor:
-To delete an AWS CloudFormation stack you can use
-:class:`~airflow.providers.amazon.aws.operators.cloud_formation.CloudFormationDeleteStackOperator`.
+Wait on an AWS CloudFormation stack creation state
+==================================================
+
+To wait on the state of an AWS CloudFormation stack creation until it reaches a terminal state you can use
+:class:`~airflow.providers.amazon.aws.sensors.cloud_formation.CloudFormationCreateStackSensor`
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_cloudformation.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_cloudformation_delete_stack]
- :end-before: [END howto_operator_cloudformation_delete_stack]
-
+ :start-after: [START howto_sensor_cloudformation_create_stack]
+ :end-before: [END howto_sensor_cloudformation_create_stack]
-.. _howto/operator:CloudFormationDeleteStackSensor:
+.. _howto/sensor:CloudFormationDeleteStackSensor:
-AWS CloudFormation Delete Stack Sensor
-""""""""""""""""""""""""""""""""""""""
+Wait on an AWS CloudFormation stack deletion state
+==================================================
To wait on the state of an AWS CloudFormation stack deletion until it reaches a terminal state you can use
use :class:`~airflow.providers.amazon.aws.sensors.cloud_formation.CloudFormationDeleteStackSensor`
@@ -89,10 +93,7 @@ use :class:`~airflow.providers.amazon.aws.sensors.cloud_formation.CloudFormation
:start-after: [START howto_sensor_cloudformation_delete_stack]
:end-before: [END howto_sensor_cloudformation_delete_stack]
-
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for CloudFormation `__
+* `AWS boto3 library documentation for CloudFormation `__
diff --git a/docs/apache-airflow-providers-amazon/operators/datasync.rst b/docs/apache-airflow-providers-amazon/operators/datasync.rst
index 20f18fa0556ca..5380aac5fa594 100644
--- a/docs/apache-airflow-providers-amazon/operators/datasync.rst
+++ b/docs/apache-airflow-providers-amazon/operators/datasync.rst
@@ -15,8 +15,9 @@
specific language governing permissions and limitations
under the License.
-AWS DataSync Operator
-=====================
+============
+AWS DataSync
+============
`AWS DataSync `__ is a data-transfer service that simplifies, automates,
and accelerates moving and replicating data between on-premises storage systems and AWS storage services over
@@ -27,10 +28,13 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:DataSyncOperator:
Interact with AWS DataSync Tasks
---------------------------------
+================================
You can use :class:`~airflow.providers.amazon.aws.operators.datasync.DataSyncOperator` to
find, create, update, execute and delete AWS DataSync tasks.
@@ -107,6 +111,4 @@ to randomly choose from candidate Locations.
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for DataSync `__
+* `AWS boto3 library documentation for DataSync `__
diff --git a/docs/apache-airflow-providers-amazon/operators/dms.rst b/docs/apache-airflow-providers-amazon/operators/dms.rst
index 8ca092b6fc573..a293f016beaf0 100644
--- a/docs/apache-airflow-providers-amazon/operators/dms.rst
+++ b/docs/apache-airflow-providers-amazon/operators/dms.rst
@@ -16,9 +16,9 @@
under the License.
-==============================================
-AWS Database Migration Service (DMS) Operators
-==============================================
+====================================
+AWS Database Migration Service (DMS)
+====================================
`AWS Database Migration Service (AWS DMS) `__
is a web service you can use to migrate data from your database that is
@@ -36,11 +36,11 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
-.. _howto/operator:DmsCreateTaskOperator:
-
Operators
---------
+.. _howto/operator:DmsCreateTaskOperator:
+
Create a replication task
=========================
@@ -123,13 +123,11 @@ To check the state of a replication task until it is completed, you can use
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_dms.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_dms_task_completed_sensor]
- :end-before: [END howto_operator_dms_task_completed_sensor]
+ :start-after: [START howto_sensor_dms_task_completed]
+ :end-before: [END howto_sensor_dms_task_completed]
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for DMS `__
+* `AWS boto3 library documentation for DMS `__
diff --git a/docs/apache-airflow-providers-amazon/operators/ec2.rst b/docs/apache-airflow-providers-amazon/operators/ec2.rst
index 5b25fdf866cd5..86807341a0e4d 100644
--- a/docs/apache-airflow-providers-amazon/operators/ec2.rst
+++ b/docs/apache-airflow-providers-amazon/operators/ec2.rst
@@ -27,11 +27,11 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
-.. _howto/operator:EC2StartInstanceOperator:
-
Operators
---------
+.. _howto/operator:EC2StartInstanceOperator:
+
Start an Amazon EC2 instance
============================
@@ -63,8 +63,8 @@ Sensors
.. _howto/sensor:EC2InstanceStateSensor:
-Amazon EC2 instance state sensor
-================================
+Wait on an Amazon EC2 instance state
+====================================
To check the state of an Amazon EC2 instance and wait until it reaches the target state you can use
:class:`~airflow.providers.amazon.aws.sensors.ec2.EC2InstanceStateSensor`.
@@ -78,6 +78,4 @@ To check the state of an Amazon EC2 instance and wait until it reaches the targe
Reference
---------
-For further information, look at:
-
* `Boto3 Library Documentation for EC2 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/ecs.rst b/docs/apache-airflow-providers-amazon/operators/ecs.rst
index e03f25954d584..6a1bd0fce55d9 100644
--- a/docs/apache-airflow-providers-amazon/operators/ecs.rst
+++ b/docs/apache-airflow-providers-amazon/operators/ecs.rst
@@ -15,37 +15,39 @@
specific language governing permissions and limitations
under the License.
+======================================
+Amazon Elastic Container Service (ECS)
+======================================
-Amazon Elastic Container Service (ECS) Operators
-================================================
-
-`Amazon Elastic Container Service (Amazon ECS) `__ is a fully
+`Amazon Elastic Container Service (Amazon ECS) `__ is a fully
managed container orchestration service that makes it easy for you to deploy, manage, and
scale containerized applications.
Airflow provides operators to run Task Definitions on an ECS cluster.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: _partials/prerequisite_tasks.rst
-* You will need to have created your ECS Cluster, and have created a Task Definition before you can use this Operator. The Task Definition contains details of the containerized application you want to run.
-
-.. include::/operators/_partials/prerequisite_tasks.rst
+Operators
+---------
.. _howto/operator:EcsOperator:
-Overview
-^^^^^^^^
+Run a task definition
+=====================
To run a Task Definition defined in an Amazon ECS cluster you can use
:class:`~airflow.providers.amazon.aws.operators.ecs.EcsOperator`.
-This Operator support running your containers in ECS Clusters that are either Serverless (FARGATE), via EC2, or via external resources (EXTERNAL). The parameters you need to configure for this Operator will depend upon which ``launch_type`` you want to use.
+You need to have created your ECS Cluster, and have created a Task Definition before you can use this Operator.
+The Task Definition contains details of the containerized application you want to run.
+
+This Operator support running your containers in ECS Clusters that are either Serverless (FARGATE), via EC2,
+or via external resources (EXTERNAL).
+The parameters you need to configure for this Operator will depend upon which ``launch_type`` you want to use.
-Launch Types
-------------
.. code-block::
launch_type="EC2|FARGATE|EXTERNAL"
@@ -67,11 +69,12 @@ Launch Types
:start-after: [START howto_operator_ecs]
:end-before: [END howto_operator_ecs]
+Stream logs to AWS CloudWatch
+"""""""""""""""""""""""""""""
-CloudWatch Logging
-------------------
-
-To stream logs to AWS CloudWatch, you need to define these parameters. Using the example Operators above, we would add these additional parameters to enable logging to CloudWatch. You will need to ensure that you have the appropriate level of permissions (see next section)
+To stream logs to AWS CloudWatch, you need to define the parameters below.
+Using the example above, we would add these additional parameters to enable logging to CloudWatch.
+You need to ensure that you have the appropriate level of permissions (see next section).
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_ecs.py
:language: python
@@ -80,9 +83,10 @@ To stream logs to AWS CloudWatch, you need to define these parameters. Using the
:end-before: [END howto_awslogs_ecs]
IAM Permissions
----------------
+"""""""""""""""
-You will need to ensure you have the following IAM permissions to run Tasks via this Operator. In this example, the Operator will have permissions to run Tasks on an ECS Cluster called "cluster a" in a specific AWS region and account.
+You need to ensure you have the following IAM permissions to run tasks via this operator.
+In this example, the operator will have permissions to run tasks on an ECS Cluster called "cluster a" in a specific AWS region and account.
.. code-block::
@@ -95,7 +99,8 @@ You will need to ensure you have the following IAM permissions to run Tasks via
"Resource": : [ "arn:aws:ecs:{aws region}:{aws account number}:cluster/{custer a}"
}
-If you use the "reattach=True" (the default is False), you will need to add further permissions. You will need to add the following additional Actions to the IAM policy.
+If you use the "reattach=True" (the default is False), you need to add further permissions.
+You need to add the following additional Actions to the IAM policy.
.. code-block::
@@ -104,7 +109,7 @@ If you use the "reattach=True" (the default is False), you will need to add furt
**CloudWatch Permissions**
-If you plan on streaming Apache Airflow logs into AWS CloudWatch, you will need to ensure that you have configured the appropriate permissions set.
+If you plan on streaming Apache Airflow logs into AWS CloudWatch, you need to ensure that you have configured the appropriate permissions set.
.. code-block::
@@ -125,9 +130,7 @@ If you plan on streaming Apache Airflow logs into AWS CloudWatch, you will need
)
-More information
-----------------
-
-For further information, look at:
+Reference
+---------
-* `Boto3 Library Documentation for ECS `__
+* `AWS boto3 library documentation for ECS `__
diff --git a/docs/apache-airflow-providers-amazon/operators/eks.rst b/docs/apache-airflow-providers-amazon/operators/eks.rst
index d6cd5ad791973..ee38f3d31f050 100644
--- a/docs/apache-airflow-providers-amazon/operators/eks.rst
+++ b/docs/apache-airflow-providers-amazon/operators/eks.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-Amazon Elastic Kubernetes Service (EKS) Operators
-=================================================
+=======================================
+Amazon Elastic Kubernetes Service (EKS)
+=======================================
`Amazon Elastic Kubernetes Service (Amazon EKS) `__ is a managed service
that makes it easy for you to run Kubernetes on AWS without needing to stand up or maintain your own
@@ -27,50 +27,73 @@ and management of containerized applications.
Airflow provides operators to create and interact with the EKS clusters and compute infrastructure.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: _partials/prerequisite_tasks.rst
-Manage Amazon EKS Clusters
-^^^^^^^^^^^^^^^^^^^^^^^^^^
+Operators
+---------
-.. _howto/sensor:EksClusterStateSensor:
+.. _howto/operator:EksCreateClusterOperator:
-Amazon EKS Cluster State Sensor
-"""""""""""""""""""""""""""""""
+Create an Amazon EKS cluster
+============================
-To check the state of an Amazon EKS Cluster until it reaches the target state or another terminal
-state you can use :class:`~airflow.providers.amazon.aws.sensors.eks.EksClusterStateSensor`.
+To create an Amazon EKS Cluster you can use
+:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateClusterOperator`.
+
+Note: An AWS IAM role with the following permissions is required:
+ ``eks.amazonaws.com`` must be added to the Trusted Relationships
+ ``AmazonEKSClusterPolicy`` IAM Policy must be attached
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_eks_cluster]
- :end-before: [END howto_sensor_eks_cluster]
+ :start-after: [START howto_operator_eks_create_cluster]
+ :end-before: [END howto_operator_eks_create_cluster]
+Create an Amazon EKS cluster and node group in one step
+=======================================================
-.. _howto/operator:EksCreateClusterOperator:
+To create an Amazon EKS cluster and an EKS managed node group in one command, you can use
+:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateClusterOperator`.
-Create an Amazon EKS Cluster
-""""""""""""""""""""""""""""
+Note: An AWS IAM role with the following permissions is required:
+ ``ec2.amazon.aws.com`` must be in the Trusted Relationships
+ ``eks.amazonaws.com`` must be added to the Trusted Relationships
+ ``AmazonEC2ContainerRegistryReadOnly`` IAM Policy must be attached
+ ``AmazonEKSClusterPolicy`` IAM Policy must be attached
+ ``AmazonEKSWorkerNodePolicy`` IAM Policy must be attached
-To create an Amazon EKS Cluster you can use
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_operator_eks_create_cluster_with_nodegroup]
+ :end-before: [END howto_operator_eks_create_cluster_with_nodegroup]
+
+Create an Amazon EKS cluster and AWS Fargate profile in one step
+================================================================
+
+To create an Amazon EKS cluster and an AWS Fargate profile in one command, you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateClusterOperator`.
Note: An AWS IAM role with the following permissions is required:
+ ``ec2.amazon.aws.com`` must be in the Trusted Relationships
``eks.amazonaws.com`` must be added to the Trusted Relationships
+ ``AmazonEC2ContainerRegistryReadOnly`` IAM Policy must be attached
``AmazonEKSClusterPolicy`` IAM Policy must be attached
+ ``AmazonEKSWorkerNodePolicy`` IAM Policy must be attached
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_eks_create_cluster]
- :end-before: [END howto_operator_eks_create_cluster]
+ :start-after: [START howto_operator_eks_create_cluster_with_fargate_profile]
+ :end-before: [END howto_operator_eks_create_cluster_with_fargate_profile]
.. _howto/operator:EksDeleteClusterOperator:
Delete an Amazon EKS Cluster
-""""""""""""""""""""""""""""
+============================
To delete an existing Amazon EKS Cluster you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksDeleteClusterOperator`.
@@ -91,30 +114,12 @@ Note: If the cluster has any attached resources, such as an Amazon EKS Nodegroup
:start-after: [START howto_operator_eks_force_delete_cluster]
:end-before: [END howto_operator_eks_force_delete_cluster]
-
-Manage Amazon EKS Managed Nodegroups
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-.. _howto/sensor:EksNodegroupStateSensor:
-
-Amazon EKS Managed Nodegroup State Sensor
-"""""""""""""""""""""""""""""""""""""""""
-
-To check the state of an Amazon EKS managed node group until it reaches the target state or another terminal
-state you can use :class:`~airflow.providers.amazon.aws.sensors.eks.EksNodegroupStateSensor`.
-
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py
- :language: python
- :dedent: 4
- :start-after: [START howto_sensor_eks_nodegroup]
- :end-before: [END howto_sensor_eks_nodegroup]
-
.. _howto/operator:EksCreateNodegroupOperator:
-Create an Amazon EKS Managed NodeGroup
-""""""""""""""""""""""""""""""""""""""
+Create an Amazon EKS managed node group
+=======================================
-To create an Amazon EKS Managed Nodegroup you can use
+To create an Amazon EKS managed node group you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateNodegroupOperator`.
Note: An AWS IAM role with the following permissions is required:
@@ -130,10 +135,10 @@ Note: An AWS IAM role with the following permissions is required:
.. _howto/operator:EksDeleteNodegroupOperator:
-Delete an Amazon EKS Managed Nodegroup
-""""""""""""""""""""""""""""""""""""""
+Delete an Amazon EKS managed node group
+=======================================
-To delete an existing Amazon EKS Managed Nodegroup you can use
+To delete an existing Amazon EKS managed node group you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksDeleteNodegroupOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py
@@ -142,67 +147,10 @@ To delete an existing Amazon EKS Managed Nodegroup you can use
:start-after: [START howto_operator_eks_delete_nodegroup]
:end-before: [END howto_operator_eks_delete_nodegroup]
-
-Create an Amazon EKS Cluster and Nodegroup in one step
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-To create an Amazon EKS Cluster and an EKS Managed Nodegroup in one command, you can use
-:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateClusterOperator`.
-
-Note: An AWS IAM role with the following permissions is required:
- ``ec2.amazon.aws.com`` must be in the Trusted Relationships
- ``eks.amazonaws.com`` must be added to the Trusted Relationships
- ``AmazonEC2ContainerRegistryReadOnly`` IAM Policy must be attached
- ``AmazonEKSClusterPolicy`` IAM Policy must be attached
- ``AmazonEKSWorkerNodePolicy`` IAM Policy must be attached
-
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroup_in_one_step.py
- :language: python
- :dedent: 4
- :start-after: [START howto_operator_eks_create_cluster_with_nodegroup]
- :end-before: [END howto_operator_eks_create_cluster_with_nodegroup]
-
-Create an Amazon EKS Cluster and AWS Fargate profile in one step
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-To create an Amazon EKS Cluster and an AWS Fargate profile in one command, you can use
-:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateClusterOperator`.
-
-Note: An AWS IAM role with the following permissions is required:
- ``ec2.amazon.aws.com`` must be in the Trusted Relationships
- ``eks.amazonaws.com`` must be added to the Trusted Relationships
- ``AmazonEC2ContainerRegistryReadOnly`` IAM Policy must be attached
- ``AmazonEKSClusterPolicy`` IAM Policy must be attached
- ``AmazonEKSWorkerNodePolicy`` IAM Policy must be attached
-
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_in_one_step.py
- :language: python
- :dedent: 4
- :start-after: [START howto_operator_eks_create_cluster_with_fargate_profile]
- :end-before: [END howto_operator_eks_create_cluster_with_fargate_profile]
-
-Manage AWS Fargate Profiles
-^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-.. _howto/sensor:EksFargateProfileStateSensor:
-
-AWS Fargate Profile State Sensor
-""""""""""""""""""""""""""""""""
-
-To check the state of an AWS Fargate profile until it reaches the target state or another terminal
-state you can use :class:`~airflow.providers.amazon.aws.sensors.eks.EksFargateProfileSensor`.
-
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py
- :language: python
- :dedent: 4
- :start-after: [START howto_sensor_eks_fargate]
- :end-before: [END howto_sensor_eks_fargate]
-
-
.. _howto/operator:EksCreateFargateProfileOperator:
Create an AWS Fargate Profile
-"""""""""""""""""""""""""""""
+=============================
To create an AWS Fargate Profile you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksCreateFargateProfileOperator`.
@@ -221,7 +169,7 @@ Note: An AWS IAM role with the following permissions is required:
.. _howto/operator:EksDeleteFargateProfileOperator:
Delete an AWS Fargate Profile
-"""""""""""""""""""""""""""""
+=============================
To delete an existing AWS Fargate Profile you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksDeleteFargateProfileOperator`.
@@ -235,7 +183,7 @@ To delete an existing AWS Fargate Profile you can use
.. _howto/operator:EksPodOperator:
Perform a Task on an Amazon EKS Cluster
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+=======================================
To run a pod on an existing Amazon EKS Cluster, you can use
:class:`~airflow.providers.amazon.aws.operators.eks.EksPodOperator`.
@@ -248,9 +196,52 @@ Note: An Amazon EKS Cluster with underlying compute infrastructure is required.
:start-after: [START howto_operator_eks_pod_operator]
:end-before: [END howto_operator_eks_pod_operator]
-Reference
-^^^^^^^^^
+Sensors
+-------
+
+.. _howto/sensor:EksClusterStateSensor:
+
+Wait on an Amazon EKS cluster state
+===================================
-For further information, look at:
+To check the state of an Amazon EKS Cluster until it reaches the target state or another terminal
+state you can use :class:`~airflow.providers.amazon.aws.sensors.eks.EksClusterStateSensor`.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_eks_cluster]
+ :end-before: [END howto_sensor_eks_cluster]
+
+.. _howto/sensor:EksNodegroupStateSensor:
+
+Wait on an Amazon EKS managed node group state
+==============================================
+
+To check the state of an Amazon EKS managed node group until it reaches the target state or another terminal
+state you can use :class:`~airflow.providers.amazon.aws.sensors.eks.EksNodegroupStateSensor`.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_nodegroups.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_eks_nodegroup]
+ :end-before: [END howto_sensor_eks_nodegroup]
+
+.. _howto/sensor:EksFargateProfileStateSensor:
+
+Wait on an AWS Fargate profile state
+====================================
+
+To check the state of an AWS Fargate profile until it reaches the target state or another terminal
+state you can use :class:`~airflow.providers.amazon.aws.sensors.eks.EksFargateProfileSensor`.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_eks_with_fargate_profile.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_eks_fargate]
+ :end-before: [END howto_sensor_eks_fargate]
+
+Reference
+---------
-* `Boto3 Library Documentation for EKS `__
+* `AWS boto3 library documentation for EKS `__
diff --git a/docs/apache-airflow-providers-amazon/operators/emr.rst b/docs/apache-airflow-providers-amazon/operators/emr.rst
index a61f4e16ae087..a62b503740dcf 100644
--- a/docs/apache-airflow-providers-amazon/operators/emr.rst
+++ b/docs/apache-airflow-providers-amazon/operators/emr.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-Amazon EMR Operators
-====================
+==========
+Amazon EMR
+==========
`Amazon EMR `__ (previously called Amazon Elastic MapReduce)
is a managed cluster platform that simplifies running big data frameworks, such as Apache
@@ -32,15 +32,18 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. note::
- In order to run the two examples successfully, you need to create the IAM Service
+ In order to run the examples successfully, you need to create the IAM Service
Roles(``EMR_EC2_DefaultRole`` and ``EMR_DefaultRole``) for Amazon EMR. You can
create these roles using the AWS CLI: ``aws emr create-default-roles``.
.. _howto/operator:EmrCreateJobFlowOperator:
-Create EMR Job Flow
--------------------
+Create an EMR job flow
+======================
You can use :class:`~airflow.providers.amazon.aws.operators.emr.EmrCreateJobFlowOperator` to
create a new EMR job flow. The cluster will be terminated automatically after finishing the steps.
@@ -81,10 +84,10 @@ In the following code we are creating a new job flow using the configuration as
.. _howto/operator:EmrAddStepsOperator:
-Add Steps to an EMR Job Flow
-----------------------------
+Add Steps to an EMR job flow
+============================
-To add Steps to an existing EMR Job Flow you can use
+To add steps to an existing EMR Job flow you can use
:class:`~airflow.providers.amazon.aws.operators.emr.EmrAddStepsOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_job_flow_manual_steps.py
@@ -95,8 +98,8 @@ To add Steps to an existing EMR Job Flow you can use
.. _howto/operator:EmrTerminateJobFlowOperator:
-Terminate an EMR Job Flow
--------------------------
+Terminate an EMR job flow
+=========================
To terminate an EMR Job Flow you can use
:class:`~airflow.providers.amazon.aws.operators.emr.EmrTerminateJobFlowOperator`.
@@ -109,27 +112,29 @@ To terminate an EMR Job Flow you can use
.. _howto/operator:EmrModifyClusterOperator:
-Modify Amazon EMR Container
----------------------------
+Modify Amazon EMR container
+===========================
-To modify an existing EMR Container you can use
+To modify an existing EMR container you can use
:class:`~airflow.providers.amazon.aws.sensors.emr.EmrContainerSensor`.
+Sensors
+-------
+
.. _howto/sensor:EmrContainerSensor:
-Amazon EMR Container Sensor
----------------------------
+Wait on an Amazon EMR container state
+=====================================
-To monitor the state of an EMR Container you can use
+To monitor the state of an EMR container you can use
:class:`~airflow.providers.amazon.aws.sensors.emr.EmrContainerSensor`.
-
.. _howto/sensor:EmrJobFlowSensor:
-Amazon EMR Job Flow Sensor
----------------------------
+Wait on an Amazon EMR job flow state
+====================================
-To monitor the state of an EMR Job Flow you can use
+To monitor the state of an EMR job flow you can use
:class:`~airflow.providers.amazon.aws.sensors.emr.EmrJobFlowSensor`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_job_flow_automatic_steps.py
@@ -140,10 +145,10 @@ To monitor the state of an EMR Job Flow you can use
.. _howto/sensor:EmrStepSensor:
-Amazon EMR Step Sensor
-----------------------
+Wait on an Amazon EMR step state
+================================
-To monitor the state of a Step running an existing EMR Job Flow you can use
+To monitor the state of a step running an existing EMR Job flow you can use
:class:`~airflow.providers.amazon.aws.sensors.emr.EmrStepSensor`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_job_flow_manual_steps.py
@@ -155,8 +160,6 @@ To monitor the state of a Step running an existing EMR Job Flow you can use
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for EMR `__
+* `AWS boto3 library documentation for EMR `__
* `AWS CLI - create-default-roles `__
* `Configure IAM Service Roles for Amazon EMR Permissions `__
diff --git a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst
index 397bc1cb4d98c..b84fbf5ee5dd5 100644
--- a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst
+++ b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst
@@ -15,11 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-.. _howto/operator:EMRContainersOperators:
-
-Amazon EMR on EKS Operators
-===========================
+========================
+Amazon EMR on Amazon EKS
+========================
`Amazon EMR on EKS `__
provides a deployment option for Amazon EMR that allows you to run open-source big data frameworks on
@@ -33,32 +31,31 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
-This example assumes that you already have an EMR on EKS virtual cluster configured. See the
-`EMR on EKS Getting Started guide `__
-for more information.
+Operators
+---------
+.. _howto/operator:EMRContainersOperators:
Run a Spark job on EMR on EKS
------------------------------
+=============================
-Purpose
-"""""""
+.. note::
+ This example assumes that you already have an EMR on EKS virtual cluster configured. See the
+ `EMR on EKS Getting Started guide `__
+ for more information.
-The ``EMRContainerOperator`` will submit a new job to an EMR on EKS virtual cluster and wait for
+The ``EMRContainerOperator`` will submit a new job to an Amazon EMR on Amazon EKS virtual cluster and wait for
the job to complete. The example job below calculates the mathematical constant ``Pi``. In a
production job, you would usually refer to a Spark script on Amazon Simple Storage Service (S3).
-Job configuration
-"""""""""""""""""
-
-To create a job for EMR on EKS, you need to specify your virtual cluster ID, the release of EMR you
+To create a job for Amazon EMR on Amazon EKS, you need to specify your virtual cluster ID, the release of Amazon EMR you
want to use, your IAM execution role, and Spark submit parameters.
You can also optionally provide configuration overrides such as Spark, Hive, or Log4j properties as
-well as monitoring configuration that sends Spark logs to S3 or Amazon Cloudwatch.
+well as monitoring configuration that sends Spark logs to Amazon S3 or Amazon Cloudwatch.
In the example, we show how to add an ``applicationConfiguration`` to use the AWS Glue data catalog
-and ``monitoringConfiguration`` to send logs to the ``/aws/emr-eks-spark`` log group in CloudWatch.
+and ``monitoringConfiguration`` to send logs to the ``/aws/emr-eks-spark`` log group in Amazon CloudWatch.
Refer to the `EMR on EKS guide `__
for more details on job configuration.
@@ -79,13 +76,11 @@ that gets passed to the operator with the ``aws_conn_id`` parameter.
:start-after: [START howto_operator_emr_eks_job]
:end-before: [END howto_operator_emr_eks_job]
-With the EmrContainerOperator, it will wait until the successful completion of the job or raise
+With the ``EmrContainerOperator``, it will wait until the successful completion of the job or raise
an ``AirflowException`` if there is an error. The operator returns the Job ID of the job run.
Reference
---------
-For further information, look at:
-
* `Amazon EMR on EKS Job runs `__
* `EMR on EKS Best Practices `__
diff --git a/docs/apache-airflow-providers-amazon/operators/glacier.rst b/docs/apache-airflow-providers-amazon/operators/glacier.rst
index 747b46d46a024..df9b0fcf8b77c 100644
--- a/docs/apache-airflow-providers-amazon/operators/glacier.rst
+++ b/docs/apache-airflow-providers-amazon/operators/glacier.rst
@@ -15,21 +15,25 @@
specific language governing permissions and limitations
under the License.
+=================
+Amazon S3 Glacier
+=================
-Amazon S3 Glacier Operators
-===========================
-
-`Amazon Glacier `_ is a secure, durable, and extremely low-cost Amazon S3 cloud storage class for data archiving and long-term backup.
+`Amazon Glacier `_ is a secure, durable,
+and extremely low-cost Amazon S3 cloud storage class for data archiving and long-term backup.
Prerequisite Tasks
------------------
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:GlacierCreateJobOperator:
-Amazon Glacier Create Job Operator
-""""""""""""""""""""""""""""""""""
+Create an Amazon Glacier job
+============================
To initiate an Amazon Glacier inventory retrieval job
use :class:`~airflow.providers.amazon.aws.transfers.glacier_to_gcs.GlacierCreateJobOperator`
@@ -39,13 +43,16 @@ This Operator returns a dictionary of information related to the initiated job s
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py
:language: python
:dedent: 4
- :start-after: [START howto_glacier_create_job_operator]
- :end-before: [END howto_glacier_create_job_operator]
+ :start-after: [START howto_operator_glacier_create_job]
+ :end-before: [END howto_operator_glacier_create_job]
+
+Sensors
+-------
-.. _howto/operator:GlacierJobOperationSensor:
+.. _howto/sensor:GlacierJobOperationSensor:
-Amazon Glacier Job Sensor
-"""""""""""""""""""""""""
+Wait on an Amazon Glacier job state
+===================================
To wait on the status of an Amazon Glacier Job to reach a terminal state
use :class:`~airflow.providers.amazon.aws.sensors.glacier.GlacierJobOperationSensor`
@@ -53,13 +60,10 @@ use :class:`~airflow.providers.amazon.aws.sensors.glacier.GlacierJobOperationSen
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py
:language: python
:dedent: 4
- :start-after: [START howto_glacier_job_operation_sensor]
- :end-before: [END howto_glacier_job_operation_sensor]
-
+ :start-after: [START howto_sensor_glacier_job_operation]
+ :end-before: [END howto_sensor_glacier_job_operation]
References
----------
-For further information, look at:
-
-* `Boto3 Library Documentation for Amazon Glacier `__
+* `AWS boto3 library documentation for Amazon Glacier `__
diff --git a/docs/apache-airflow-providers-amazon/operators/glue.rst b/docs/apache-airflow-providers-amazon/operators/glue.rst
index 306354c500a1c..64726ef47393b 100644
--- a/docs/apache-airflow-providers-amazon/operators/glue.rst
+++ b/docs/apache-airflow-providers-amazon/operators/glue.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-AWS Glue Operators
-===================
+========
+AWS Glue
+========
`AWS Glue `__ is a serverless data integration service that makes it
easy to discover, prepare, and combine data for analytics, machine learning, and application development.
@@ -29,10 +29,13 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:GlueCrawlerOperator:
-AWS Glue Crawler Operator
-"""""""""""""""""""""""""
+Create an AWS Glue crawler
+==========================
AWS Glue Crawlers allow you to easily extract data from various data sources.
To create a new AWS Glue Crawler or run an existing one you can
@@ -44,44 +47,49 @@ use :class:`~airflow.providers.amazon.aws.operators.glue_crawler.GlueCrawlerOper
:start-after: [START howto_operator_glue_crawler]
:end-before: [END howto_operator_glue_crawler]
-Note that the AWS IAM role included in the ``config`` needs access to the source data location
-(e.g. s3:PutObject access if data is stored in Amazon S3) as well as the ``AWSGlueServiceRole``
-policy. See the References section below for a link to more details.
+.. note::
+ The AWS IAM role included in the ``config`` needs access to the source data location
+ (e.g. s3:PutObject access if data is stored in Amazon S3) as well as the ``AWSGlueServiceRole``
+ policy. See the References section below for a link to more details.
-.. _howto/sensor:GlueCrawlerSensor:
+.. _howto/operator:GlueJobOperator:
-AWS Glue Crawler Sensor
-"""""""""""""""""""""""
+Submit an AWS Glue job
+======================
-To wait on the state of an AWS Glue Crawler execution until it reaches a terminal state you can
-use :class:`~airflow.providers.amazon.aws.sensors.glue_crawler.GlueCrawlerSensor`.
+To submit a new AWS Glue job you can use :class:`~airflow.providers.amazon.aws.operators.glue.GlueJobOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_glue.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_glue_crawler]
- :end-before: [END howto_sensor_glue_crawler]
+ :start-after: [START howto_operator_glue]
+ :end-before: [END howto_operator_glue]
-.. _howto/operator:GlueJobOperator:
+.. note::
+ The same AWS IAM role used for the crawler can be used here as well, but it will need
+ policies to provide access to the output location for result data.
-AWS Glue Job Operator
-"""""""""""""""""""""
+Sensors
+-------
-To submit a new AWS Glue Job you can use :class:`~airflow.providers.amazon.aws.operators.glue.GlueJobOperator`.
+.. _howto/sensor:GlueCrawlerSensor:
+
+Wait on an AWS Glue crawler state
+=================================
+
+To wait on the state of an AWS Glue crawler execution until it reaches a terminal state you can
+use :class:`~airflow.providers.amazon.aws.sensors.glue_crawler.GlueCrawlerSensor`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_glue.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_glue]
- :end-before: [END howto_operator_glue]
-
-Note that the same AWS IAM role used for the Crawler can be used here as well, but it will need
-policies to provide access to the output location for result data.
+ :start-after: [START howto_sensor_glue_crawler]
+ :end-before: [END howto_sensor_glue_crawler]
.. _howto/sensor:GlueJobSensor:
-AWS Glue Job Sensor
-"""""""""""""""""""
+Wait on an AWS Glue job state
+=============================
To wait on the state of an AWS Glue Job until it reaches a terminal state you can
use :class:`~airflow.providers.amazon.aws.sensors.glue.GlueJobSensor`
@@ -95,7 +103,5 @@ use :class:`~airflow.providers.amazon.aws.sensors.glue.GlueJobSensor`
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for Glue `__
+* `AWS boto3 library documentation for Glue `__
* `Glue IAM Role creation `__
diff --git a/docs/apache-airflow-providers-amazon/operators/lambda.rst b/docs/apache-airflow-providers-amazon/operators/lambda.rst
index 79374eb506c34..e4516a170bad8 100644
--- a/docs/apache-airflow-providers-amazon/operators/lambda.rst
+++ b/docs/apache-airflow-providers-amazon/operators/lambda.rst
@@ -15,44 +15,40 @@
specific language governing permissions and limitations
under the License.
+==========
+AWS Lambda
+==========
-AWS Lambda Operators
-==================================================
-
-`AWS Lambda `__ is a
-serverless, event-driven compute service that lets you
-run code for virtually any type of application
-or backend service without provisioning or managing servers.
-You can trigger Lambda from over 200 AWS services and software as a service (SaaS) applications,
-and only pay for what you use.
-
-Airflow provides an operator to invoke an AWS Lambda function.
+With `AWS Lambda `__, you can run code without provisioning or managing servers.
+You pay only for the compute time that you consume—there's no charge when your code isn't running.
+You can run code for virtually any type of application or backend service—all with zero administration.
+Just upload your code and Lambda takes care of everything required to run and scale your code with high availability.
+You can set up your code to automatically trigger from other AWS services or call it directly from any web or mobile app.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
.. _howto/operator:AwsLambdaInvokeFunctionOperator:
-Invoke an existing AWS Lambda function with a payload
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Invoke an AWS Lambda function
+=============================
-To publish a message to an Amazon SNS Topic you can use
+To invoke an AWS lambda function you can use
:class:`~airflow.providers.amazon.aws.operators.aws_lambda.AwsLambdaInvokeFunctionOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_lambda.py
:language: python
:dedent: 4
- :start-after: [START howto_lambda_operator]
- :end-before: [END howto_lambda_operator]
-
+ :start-after: [START howto_operator_lambda]
+ :end-before: [END howto_operator_lambda]
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
-* `Boto3 Library Documentation for Lambda `__
+* `AWS boto3 library documentation for Lambda `__
diff --git a/docs/apache-airflow-providers-amazon/operators/quicksight.rst b/docs/apache-airflow-providers-amazon/operators/quicksight.rst
index 21948fe8486e6..c214a282705ff 100644
--- a/docs/apache-airflow-providers-amazon/operators/quicksight.rst
+++ b/docs/apache-airflow-providers-amazon/operators/quicksight.rst
@@ -66,6 +66,4 @@ The ``QuickSightSensor`` waits for an Amazon QuickSight create ingestion until i
Reference
---------
-For further information, look at:
-
* `Boto3 Library Documentation for QuickSight `__
diff --git a/docs/apache-airflow-providers-amazon/operators/redshift_cluster.rst b/docs/apache-airflow-providers-amazon/operators/redshift_cluster.rst
index d999052e5f713..a0875c3d771a3 100644
--- a/docs/apache-airflow-providers-amazon/operators/redshift_cluster.rst
+++ b/docs/apache-airflow-providers-amazon/operators/redshift_cluster.rst
@@ -15,30 +15,29 @@
specific language governing permissions and limitations
under the License.
-Amazon Redshift Operators
-=========================
+===============
+Amazon Redshift
+===============
`Amazon Redshift `__ manages all the work of setting up, operating, and scaling a data warehouse:
provisioning capacity, monitoring and backing up the cluster, and applying patches and upgrades to
the Amazon Redshift engine. You can focus on using your data to acquire new insights for your
business and customers.
-Airflow provides operators to manage your Redshift clusters.
-
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: _partials/prerequisite_tasks.rst
-Manage Amazon Redshift Clusters
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Operators
+---------
.. _howto/operator:RedshiftCreateClusterOperator:
-Amazon Redshift Cluster Operator
-""""""""""""""""""""""""""""""""
+Create an Amazon Redshift cluster
+=================================
-To create an Amazon Redshift Cluster with the specified parameters
+To create an Amazon Redshift Cluster with the specified parameters you can use
:class:`~airflow.providers.amazon.aws.operators.redshift_cluster.RedshiftCreateClusterOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_cluster.py
@@ -47,26 +46,12 @@ To create an Amazon Redshift Cluster with the specified parameters
:start-after: [START howto_operator_redshift_cluster]
:end-before: [END howto_operator_redshift_cluster]
-.. _howto/sensor:RedshiftClusterSensor:
-
-Amazon Redshift Cluster Sensor
-""""""""""""""""""""""""""""""
-
-To check the state of an Amazon Redshift Cluster until it reaches the target state or another terminal
-state you can use :class:`~airflow.providers.amazon.aws.sensors.redshift_cluster.RedshiftClusterSensor`.
-
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_cluster.py
- :language: python
- :dedent: 4
- :start-after: [START howto_sensor_redshift_cluster]
- :end-before: [END howto_sensor_redshift_cluster]
-
.. _howto/operator:RedshiftResumeClusterOperator:
-Resume an Amazon Redshift Cluster
-"""""""""""""""""""""""""""""""""
+Resume an Amazon Redshift cluster
+=================================
-To resume a 'paused' Amazon Redshift Cluster you can use
+To resume a 'paused' Amazon Redshift cluster you can use
:class:`RedshiftResumeClusterOperator `
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_cluster.py
@@ -77,10 +62,10 @@ To resume a 'paused' Amazon Redshift Cluster you can use
.. _howto/operator:RedshiftPauseClusterOperator:
-Pause an Amazon Redshift Cluster
-""""""""""""""""""""""""""""""""
+Pause an Amazon Redshift cluster
+================================
-To pause an 'available' Amazon Redshift Cluster you can use
+To pause an 'available' Amazon Redshift cluster you can use
:class:`RedshiftPauseClusterOperator `
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_cluster.py
@@ -91,10 +76,10 @@ To pause an 'available' Amazon Redshift Cluster you can use
.. _howto/operator:RedshiftDeleteClusterOperator:
-Delete an Amazon Redshift Cluster
-"""""""""""""""""""""""""""""""""
+Delete an Amazon Redshift cluster
+=================================
-To delete an Amazon Redshift Cluster you can use
+To delete an Amazon Redshift cluster you can use
:class:`RedshiftDeleteClusterOperator `
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_cluster.py
@@ -103,7 +88,24 @@ To delete an Amazon Redshift Cluster you can use
:start-after: [START howto_operator_redshift_delete_cluster]
:end-before: [END howto_operator_redshift_delete_cluster]
+Sensors
+-------
+
+.. _howto/sensor:RedshiftClusterSensor:
+
+Wait on an Amazon Redshift cluster state
+========================================
+
+To check the state of an Amazon Redshift Cluster until it reaches the target state or another terminal
+state you can use :class:`~airflow.providers.amazon.aws.sensors.redshift_cluster.RedshiftClusterSensor`.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_cluster.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_redshift_cluster]
+ :end-before: [END howto_sensor_redshift_cluster]
+
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon Redshift `__
+* `AWS boto3 library documentation for Amazon Redshift `__
diff --git a/docs/apache-airflow-providers-amazon/operators/redshift_data.rst b/docs/apache-airflow-providers-amazon/operators/redshift_data.rst
index 4080378e1a6bb..eb9dc5112952c 100644
--- a/docs/apache-airflow-providers-amazon/operators/redshift_data.rst
+++ b/docs/apache-airflow-providers-amazon/operators/redshift_data.rst
@@ -15,37 +15,41 @@
specific language governing permissions and limitations
under the License.
-Amazon Redshift Data Operators
-==============================
-
-Use the :class:`RedshiftDataOperator ` to execute
-statements against an Amazon Redshift cluster.
+====================
+Amazon Redshift Data
+====================
-This differs from ``RedshiftSQLOperator`` in that it allows users to query and retrieve data via the AWS API and avoid the necessity of a Postgres connection.
+`Amazon Redshift `__ manages all the work of setting up, operating, and scaling a data warehouse:
+provisioning capacity, monitoring and backing up the cluster, and applying patches and upgrades to
+the Amazon Redshift engine. You can focus on using your data to acquire new insights for your
+business and customers.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: _partials/prerequisite_tasks.rst
-Amazon Redshift Data
-^^^^^^^^^^^^^^^^^^^^
+Operators
+---------
.. _howto/operator:RedshiftDataOperator:
-Execute a statement on an Amazon Redshift Cluster
-"""""""""""""""""""""""""""""""""""""""""""""""""
+Execute a statement on an Amazon Redshift cluster
+=================================================
+
+Use the :class:`RedshiftDataOperator ` to execute
+statements against an Amazon Redshift cluster.
-This is a basic example DAG for using :class:`RedshiftDataOperator `
-to execute statements against an Amazon Redshift cluster.
+This differs from ``RedshiftSQLOperator`` in that it allows users to query and retrieve data via the AWS API and avoid
+the necessity of a Postgres connection.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_data_execute_sql.py
:language: python
:dedent: 4
- :start-after: [START howto_redshift_data]
- :end-before: [END howto_redshift_data]
+ :start-after: [START howto_operator_redshift_data]
+ :end-before: [END howto_operator_redshift_data]
Reference
-^^^^^^^^^
+---------
- * `AWS boto3 Library Documentation for Amazon Redshift Data `__
+ * `AWS boto3 library documentation for Amazon Redshift Data `__
diff --git a/docs/apache-airflow-providers-amazon/operators/redshift_sql.rst b/docs/apache-airflow-providers-amazon/operators/redshift_sql.rst
index 2363ec4b4a911..92ff0e1c4b4c5 100644
--- a/docs/apache-airflow-providers-amazon/operators/redshift_sql.rst
+++ b/docs/apache-airflow-providers-amazon/operators/redshift_sql.rst
@@ -17,28 +17,30 @@
.. _howto/operator:RedshiftSQLOperator:
-Amazon Redshift Operators
-=========================
+===============
+Amazon Redshift
+===============
`Amazon Redshift `__ manages all the work of setting up, operating, and scaling a data warehouse:
provisioning capacity, monitoring and backing up the cluster, and applying patches and upgrades to
the Amazon Redshift engine. You can focus on using your data to acquire new insights for your
business and customers.
-Airflow provides an operator to execute queries against an Amazon Redshift cluster.
-
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: _partials/prerequisite_tasks.rst
-Redshift SQL
-^^^^^^^^^^^^
-
-This operator executes a SQL query against an Amazon Redshift cluster.
+Operators
+---------
Execute a SQL query
-"""""""""""""""""""
+===================
+
+``RedshiftSQLOperator`` executes a SQL query against an Amazon Redshift cluster using a Postgres connection.
+
+To execute a SQL query against an Amazon Redshift cluster without using a Postgres connection,
+please check ``RedshiftDataOperator``.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_sql.py
:language: python
@@ -46,10 +48,7 @@ Execute a SQL query
:start-after: [START howto_operator_redshift_sql]
:end-before: [END howto_operator_redshift_sql]
-Execute a SQL query with parameters
-"""""""""""""""""""""""""""""""""""
-
-RedshiftSQLOperator supports the ``parameters`` attribute which allows us to dynamically pass
+``RedshiftSQLOperator`` supports the ``parameters`` attribute which allows us to dynamically pass
parameters into SQL statements.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift_sql.py
@@ -59,8 +58,7 @@ parameters into SQL statements.
:end-before: [END howto_operator_redshift_sql_with_params]
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
+* `AWS boto3 library documentation for Amazon Redshift `__
* `Amazon Redshift Python connector `__
diff --git a/docs/apache-airflow-providers-amazon/operators/s3.rst b/docs/apache-airflow-providers-amazon/operators/s3.rst
index 977be924520bb..bbb13d976366c 100644
--- a/docs/apache-airflow-providers-amazon/operators/s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/s3.rst
@@ -15,34 +15,25 @@
specific language governing permissions and limitations
under the License.
+=========
+Amazon S3
+=========
-Amazon S3 Operators
-====================
-
-Airflow to Amazon Simple Storage Service (S3) integration provides several operators to create and interact with S3 buckets.
-
- - :class:`~airflow.providers.amazon.aws.sensors.s3.S3KeySensor`
- - :class:`~airflow.providers.amazon.aws.sensors.s3.S3KeysUnchangedSensor`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3CreateBucketOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteBucketOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteBucketTaggingOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3GetBucketTaggingOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3PutBucketTaggingOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3CreateObjectOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3CopyObjectOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteObjectsOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3FileTransformOperator`
- - :class:`~airflow.providers.amazon.aws.operators.s3.S3ListOperator`
+`Amazon Simple Storage Service (Amazon S3) `__ is storage for the internet.
+You can use Amazon S3 to store and retrieve any amount of data at any time, from anywhere on the web.
Prerequisite Tasks
------------------
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:S3CreateBucketOperator:
-Create an Amazon S3 Bucket
---------------------------
+Create an Amazon S3 bucket
+==========================
To create an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3CreateBucketOperator`.
@@ -55,8 +46,8 @@ To create an Amazon S3 bucket you can use
.. _howto/operator:S3DeleteBucketOperator:
-Delete an Amazon S3 Bucket
---------------------------
+Delete an Amazon S3 bucket
+==========================
To delete an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteBucketOperator`.
@@ -69,8 +60,8 @@ To delete an Amazon S3 bucket you can use
.. _howto/operator:S3PutBucketTaggingOperator:
-Set the tags for an Amazon S3 Bucket
-------------------------------------
+Set the tags for an Amazon S3 bucket
+====================================
To set the tags for an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3PutBucketTaggingOperator`.
@@ -83,8 +74,8 @@ To set the tags for an Amazon S3 bucket you can use
.. _howto/operator:S3GetBucketTaggingOperator:
-Get the tag of an Amazon S3 Bucket
-----------------------------------
+Get the tag of an Amazon S3 bucket
+==================================
To get the tag set associated with an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3GetBucketTaggingOperator`.
@@ -95,11 +86,10 @@ To get the tag set associated with an Amazon S3 bucket you can use
:start-after: [START howto_operator_s3_get_bucket_tagging]
:end-before: [END howto_operator_s3_get_bucket_tagging]
-
.. _howto/operator:S3DeleteBucketTaggingOperator:
-Delete the tags of an Amazon S3 Bucket
---------------------------------------
+Delete the tags of an Amazon S3 bucket
+======================================
To delete the tags of an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteBucketTaggingOperator`.
@@ -110,96 +100,69 @@ To delete the tags of an Amazon S3 bucket you can use
:start-after: [START howto_operator_s3_delete_bucket_tagging]
:end-before: [END howto_operator_s3_delete_bucket_tagging]
-.. _howto/sensor:S3KeySensor:
-
-Amazon S3 Key Sensor
---------------------
-
-To wait for one or multiple keys to be present in an Amazon S3 bucket you can use
-:class:`~airflow.providers.amazon.aws.sensors.s3.S3KeySensor`.
-For each key, it calls
-`head_object `__
-API (or `list_objects_v2 `__
-API if ``wildcard_match`` is ``True``) to check whether it is present or not.
-Please keep in mind, especially when used to check a large volume of keys, that it makes one API call per key.
-
-To check one file:
+.. _howto/operator:S3CreateObjectOperator:
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
- :language: python
- :dedent: 4
- :start-after: [START howto_sensor_s3_key_single_key]
- :end-before: [END howto_sensor_s3_key_single_key]
+Create an Amazon S3 object
+==========================
-To check multiple files:
+To create a new (or replace) Amazon S3 object you can use
+:class:`~airflow.providers.amazon.aws.operators.s3.S3CreateObjectOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_s3_key_multiple_keys]
- :end-before: [END howto_sensor_s3_key_multiple_keys]
-
-To check with an additional custom check you can define a function which receives a list of matched S3 object
-attributes and returns a boolean:
-
-- ``True``: a certain criteria is met
-- ``False``: the criteria isn't met
-
-This function is called for each key passed as parameter in ``bucket_key``.
-The reason why the parameter of this function is a list of objects is when ``wildcard_match`` is ``True``,
-multiple files can match one key. The list of matched S3 object attributes contain only the size and is this format:
+ :start-after: [START howto_operator_s3_create_object]
+ :end-before: [END howto_operator_s3_create_object]
-.. code-block:: python
+.. _howto/operator:S3CopyObjectOperator:
- [{"Size": int}]
+Copy an Amazon S3 object
+========================
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
- :language: python
- :dedent: 4
- :start-after: [START howto_sensor_s3_key_function_definition]
- :end-before: [END howto_sensor_s3_key_function_definition]
+To copy an Amazon S3 object from one bucket to another you can use
+:class:`~airflow.providers.amazon.aws.operators.s3.S3CopyObjectOperator`.
+The Amazon S3 connection used here needs to have access to both source and destination bucket/key.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_s3_key_function]
- :end-before: [END howto_sensor_s3_key_function]
+ :start-after: [START howto_operator_s3_copy_object]
+ :end-before: [END howto_operator_s3_copy_object]
-.. _howto/sensor:S3KeysUnchangedSensor:
+.. _howto/operator:S3DeleteObjectsOperator:
-Amazon S3 Key Unchanged Sensor
-------------------------------
+Delete Amazon S3 objects
+========================
-To check for changes in the number of objects at a specific prefix in an Amazon S3 bucket and waits until
-the inactivity period has passed with no increase in the number of objects you can use
-:class:`~airflow.providers.amazon.aws.sensors.s3.S3KeysUnchangedSensor`.
-Note, this sensor will not behave correctly in reschedule mode,
-as the state of the listed objects in the Amazon S3 bucket will be lost between rescheduled invocations.
+To delete one or multiple Amazon S3 objects you can use
+:class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteObjectsOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_sensor_s3_keys_unchanged]
- :end-before: [END howto_sensor_s3_keys_unchanged]
+ :start-after: [START howto_operator_s3_delete_objects]
+ :end-before: [END howto_operator_s3_delete_objects]
-.. _howto/operator:S3CreateObjectOperator:
+.. _howto/operator:S3FileTransformOperator:
-Create an Amazon S3 object
---------------------------
+Transform an Amazon S3 object
+=============================
-To create a new (or replace) Amazon S3 object you can use
-:class:`~airflow.providers.amazon.aws.operators.s3.S3CreateObjectOperator`.
+To transform the data from one Amazon S3 object and save it to another object you can use
+:class:`~airflow.providers.amazon.aws.operators.s3.S3FileTransformOperator`.
+You can also apply an optional [Amazon S3 Select expression](https://docs.aws.amazon.com/AmazonS3/latest/userguide/s3-glacier-select-sql-reference-select.html)
+to select the data you want to retrieve from ``source_s3_key`` using ``select_expression``.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_s3_create_object]
- :end-before: [END howto_operator_s3_create_object]
+ :start-after: [START howto_operator_s3_file_transform]
+ :end-before: [END howto_operator_s3_file_transform]
.. _howto/operator:S3ListPrefixesOperator:
List Amazon S3 prefixes
------------------------
+=======================
To list all Amazon S3 prefixes within an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3ListPrefixesOperator`.
@@ -215,7 +178,7 @@ for more information about Amazon S3 prefixes.
.. _howto/operator:S3ListOperator:
List Amazon S3 objects
-----------------------
+======================
To list all Amazon S3 objects within an Amazon S3 bucket you can use
:class:`~airflow.providers.amazon.aws.operators.s3.S3ListOperator`.
@@ -227,54 +190,82 @@ You can specify a ``prefix`` to filter the objects whose name begins with such p
:start-after: [START howto_operator_s3_list]
:end-before: [END howto_operator_s3_list]
-.. _howto/operator:S3CopyObjectOperator:
+Sensors
+-------
-Copy an Amazon S3 object
-------------------------
+.. _howto/sensor:S3KeySensor:
-To copy an Amazon S3 object from one bucket to another you can use
-:class:`~airflow.providers.amazon.aws.operators.s3.S3CopyObjectOperator`.
-The Amazon S3 connection used here needs to have access to both source and destination bucket/key.
+Wait on an Amazon S3 key
+========================
+
+To wait for one or multiple keys to be present in an Amazon S3 bucket you can use
+:class:`~airflow.providers.amazon.aws.sensors.s3.S3KeySensor`.
+For each key, it calls
+`head_object `__
+API (or `list_objects_v2 `__
+API if ``wildcard_match`` is ``True``) to check whether it is present or not.
+Please keep in mind, especially when used to check a large volume of keys, that it makes one API call per key.
+
+To check one file:
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_s3_copy_object]
- :end-before: [END howto_operator_s3_copy_object]
+ :start-after: [START howto_sensor_s3_key_single_key]
+ :end-before: [END howto_sensor_s3_key_single_key]
-.. _howto/operator:S3DeleteObjectsOperator:
+To check multiple files:
-Delete Amazon S3 objects
-------------------------
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_s3_key_multiple_keys]
+ :end-before: [END howto_sensor_s3_key_multiple_keys]
-To delete one or multiple Amazon S3 objects you can use
-:class:`~airflow.providers.amazon.aws.operators.s3.S3DeleteObjectsOperator`.
+To check with an additional custom check you can define a function which receives a list of matched S3 object
+attributes and returns a boolean:
+
+- ``True``: a certain criteria is met
+- ``False``: the criteria isn't met
+
+This function is called for each key passed as parameter in ``bucket_key``.
+The reason why the parameter of this function is a list of objects is when ``wildcard_match`` is ``True``,
+multiple files can match one key. The list of matched S3 object attributes contain only the size and is this format:
+
+.. code-block:: python
+
+ [{"Size": int}]
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_s3_delete_objects]
- :end-before: [END howto_operator_s3_delete_objects]
+ :start-after: [START howto_sensor_s3_key_function_definition]
+ :end-before: [END howto_sensor_s3_key_function_definition]
-.. _howto/operator:S3FileTransformOperator:
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
+ :language: python
+ :dedent: 4
+ :start-after: [START howto_sensor_s3_key_function]
+ :end-before: [END howto_sensor_s3_key_function]
-Transform an Amazon S3 object
------------------------------
+.. _howto/sensor:S3KeysUnchangedSensor:
-To transform the data from one Amazon S3 object and save it to another object you can use
-:class:`~airflow.providers.amazon.aws.operators.s3.S3FileTransformOperator`.
-You can also apply an optional [Amazon S3 Select expression](https://docs.aws.amazon.com/AmazonS3/latest/userguide/s3-glacier-select-sql-reference-select.html)
-to select the data you want to retrieve from ``source_s3_key`` using ``select_expression``.
+Wait on Amazon S3 prefix changes
+================================
+
+To check for changes in the number of objects at a specific prefix in an Amazon S3 bucket and waits until
+the inactivity period has passed with no increase in the number of objects you can use
+:class:`~airflow.providers.amazon.aws.sensors.s3.S3KeysUnchangedSensor`.
+Note, this sensor will not behave correctly in reschedule mode,
+as the state of the listed objects in the Amazon S3 bucket will be lost between rescheduled invocations.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_s3.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_s3_file_transform]
- :end-before: [END howto_operator_s3_file_transform]
+ :start-after: [START howto_sensor_s3_keys_unchanged]
+ :end-before: [END howto_sensor_s3_keys_unchanged]
Reference
---------
-For further information, look at:
-
-* `Boto3 Library Documentation for S3 `__
+* `AWS boto3 library documentation for S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/sagemaker.rst b/docs/apache-airflow-providers-amazon/operators/sagemaker.rst
index a31527d4ca305..262d7d84f2767 100644
--- a/docs/apache-airflow-providers-amazon/operators/sagemaker.rst
+++ b/docs/apache-airflow-providers-amazon/operators/sagemaker.rst
@@ -15,8 +15,9 @@
specific language governing permissions and limitations
under the License.
-Amazon SageMaker Operators
-==========================
+================
+Amazon SageMaker
+================
`Amazon SageMaker `__ is a fully managed
machine learning service. With Amazon SageMaker, data scientists and developers
@@ -30,13 +31,13 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
-Manage Amazon SageMaker Jobs
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Operators
+---------
.. _howto/operator:SageMakerProcessingOperator:
-Create an Amazon SageMaker Processing Job
-"""""""""""""""""""""""""""""""""""""""""
+Create an Amazon SageMaker processing job
+=========================================
To create an Amazon Sagemaker processing job to sanitize your dataset you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerProcessingOperator`.
@@ -47,11 +48,10 @@ To create an Amazon Sagemaker processing job to sanitize your dataset you can us
:start-after: [START howto_operator_sagemaker_processing]
:end-before: [END howto_operator_sagemaker_processing]
-
.. _howto/operator:SageMakerTrainingOperator:
-Create an Amazon SageMaker Training Job
-"""""""""""""""""""""""""""""""""""""""
+Create an Amazon SageMaker training job
+=======================================
To create an Amazon Sagemaker training job you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerTrainingOperator`.
@@ -64,8 +64,8 @@ To create an Amazon Sagemaker training job you can use
.. _howto/operator:SageMakerModelOperator:
-Create an Amazon SageMaker Model
-""""""""""""""""""""""""""""""""
+Create an Amazon SageMaker model
+================================
To create an Amazon Sagemaker model you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerModelOperator`.
@@ -78,8 +78,8 @@ To create an Amazon Sagemaker model you can use
.. _howto/operator:SageMakerTuningOperator:
-Start a Hyperparameter Tuning Job
-"""""""""""""""""""""""""""""""""
+Start a hyperparameter tuning job
+=================================
To start a hyperparameter tuning job for an Amazon Sagemaker model you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerTuningOperator`.
@@ -92,8 +92,8 @@ To start a hyperparameter tuning job for an Amazon Sagemaker model you can use
.. _howto/operator:SageMakerDeleteModelOperator:
-Delete an Amazon SageMaker Model
-""""""""""""""""""""""""""""""""
+Delete an Amazon SageMaker model
+================================
To delete an Amazon Sagemaker model you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerDeleteModelOperator`.
@@ -106,8 +106,8 @@ To delete an Amazon Sagemaker model you can use
.. _howto/operator:SageMakerTransformOperator:
-Create an Amazon SageMaker Transform Job
-""""""""""""""""""""""""""""""""""""""""
+Create an Amazon SageMaker transform job
+========================================
To create an Amazon Sagemaker transform job you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerTransformOperator`.
@@ -120,8 +120,8 @@ To create an Amazon Sagemaker transform job you can use
.. _howto/operator:SageMakerEndpointConfigOperator:
-Create an Amazon SageMaker Endpoint Config Job
-""""""""""""""""""""""""""""""""""""""""""""""
+Create an Amazon SageMaker endpoint config job
+==============================================
To create an Amazon Sagemaker endpoint config job you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerEndpointConfigOperator`.
@@ -134,8 +134,8 @@ To create an Amazon Sagemaker endpoint config job you can use
.. _howto/operator:SageMakerEndpointOperator:
-Create an Amazon SageMaker Endpoint Job
-"""""""""""""""""""""""""""""""""""""""
+Create an Amazon SageMaker endpoint job
+=======================================
To create an Amazon Sagemaker endpoint you can use
:class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerEndpointOperator`.
@@ -146,14 +146,13 @@ To create an Amazon Sagemaker endpoint you can use
:start-after: [START howto_operator_sagemaker_endpoint]
:end-before: [END howto_operator_sagemaker_endpoint]
-
-Amazon SageMaker Sensors
-^^^^^^^^^^^^^^^^^^^^^^^^
+Sensors
+-------
.. _howto/sensor:SageMakerTrainingSensor:
-Amazon SageMaker Training Sensor
-""""""""""""""""""""""""""""""""
+Wait on an Amazon SageMaker training job state
+==============================================
To check the state of an Amazon Sagemaker training job until it reaches a terminal state
you can use :class:`~airflow.providers.amazon.aws.sensors.sagemaker.SageMakerTrainingSensor`.
@@ -161,13 +160,13 @@ you can use :class:`~airflow.providers.amazon.aws.sensors.sagemaker.SageMakerTra
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sagemaker.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_sagemaker_training_sensor]
- :end-before: [END howto_operator_sagemaker_training_sensor]
+ :start-after: [START howto_sensor_sagemaker_training]
+ :end-before: [END howto_sensor_sagemaker_training]
.. _howto/sensor:SageMakerTransformSensor:
-Amazon SageMaker Transform Sensor
-"""""""""""""""""""""""""""""""""""
+Wait on an Amazon SageMaker transform job state
+===============================================
To check the state of an Amazon Sagemaker transform job until it reaches a terminal state
you can use :class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerTransformOperator`.
@@ -175,13 +174,13 @@ you can use :class:`~airflow.providers.amazon.aws.operators.sagemaker.SageMakerT
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sagemaker.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_sagemaker_transform_sensor]
- :end-before: [END howto_operator_sagemaker_transform_sensor]
+ :start-after: [START howto_sensor_sagemaker_transform]
+ :end-before: [END howto_sensor_sagemaker_transform]
.. _howto/sensor:SageMakerTuningSensor:
-Amazon SageMaker Tuning Sensor
-""""""""""""""""""""""""""""""
+Wait on an Amazon SageMaker tuning job state
+============================================
To check the state of an Amazon Sagemaker hyperparameter tuning job until it reaches a terminal state
you can use :class:`~airflow.providers.amazon.aws.sensors.sagemaker.SageMakerTuningSensor`.
@@ -189,27 +188,25 @@ you can use :class:`~airflow.providers.amazon.aws.sensors.sagemaker.SageMakerTun
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sagemaker.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_sagemaker_tuning_sensor]
- :end-before: [END howto_operator_sagemaker_tuning_sensor]
+ :start-after: [START howto_sensor_sagemaker_tuning]
+ :end-before: [END howto_sensor_sagemaker_tuning]
.. _howto/sensor:SageMakerEndpointSensor:
-Amazon SageMaker Endpoint Sensor
-""""""""""""""""""""""""""""""""
+Wait on an Amazon SageMaker endpoint state
+==========================================
-To check the state of an Amazon Sagemaker hyperparameter tuning job until it reaches a terminal state
+To check the state of an Amazon Sagemaker endpoint until it reaches a terminal state
you can use :class:`~airflow.providers.amazon.aws.sensors.sagemaker.SageMakerEndpointSensor`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sagemaker_endpoint.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_sagemaker_endpoint_sensor]
- :end-before: [END howto_operator_sagemaker_endpoint_sensor]
+ :start-after: [START howto_sensor_sagemaker_endpoint]
+ :end-before: [END howto_sensor_sagemaker_endpoint]
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
-* `Boto3 Library Documentation for Sagemaker `__
+* `AWS boto3 library documentation for Sagemaker `__
* `Amazon SageMaker Developer Guide `__
diff --git a/docs/apache-airflow-providers-amazon/operators/sns.rst b/docs/apache-airflow-providers-amazon/operators/sns.rst
index 1853bf27ae9dc..a6001cdc39e09 100644
--- a/docs/apache-airflow-providers-amazon/operators/sns.rst
+++ b/docs/apache-airflow-providers-amazon/operators/sns.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-Amazon Simple Notification Service (SNS) Operators
-==================================================
+========================================
+Amazon Simple Notification Service (SNS)
+========================================
`Amazon Simple Notification Service (Amazon SNS) `__ is a managed
service that provides message delivery from publishers to subscribers (also known as producers
@@ -27,33 +27,29 @@ SNS topic and receive published messages using a supported endpoint type, such a
Data Firehose, Amazon SQS, AWS Lambda, HTTP, email, mobile push notifications, and mobile text
messages (SMS).
-Airflow provides an operator to publish messages to an SNS Topic.
-
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
-.. include::/operators/_partials/prerequisite_tasks.rst
+.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
.. _howto/operator:SnsPublishOperator:
-Publish A Message To An Existing SNS Topic
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Publish a message to an existing SNS topic
+==========================================
To publish a message to an Amazon SNS Topic you can use
:class:`~airflow.providers.amazon.aws.operators.sns.SnsPublishOperator`.
-
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sns.py
:language: python
:dedent: 4
:start-after: [START howto_operator_sns_publish_operator]
:end-before: [END howto_operator_sns_publish_operator]
-
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
-* `Boto3 Library Documentation for SNS `__
+* `AWS boto3 library documentation for SNS `__
diff --git a/docs/apache-airflow-providers-amazon/operators/sqs.rst b/docs/apache-airflow-providers-amazon/operators/sqs.rst
index aedc037515d3e..dcaa6a13f9ceb 100644
--- a/docs/apache-airflow-providers-amazon/operators/sqs.rst
+++ b/docs/apache-airflow-providers-amazon/operators/sqs.rst
@@ -15,8 +15,9 @@
specific language governing permissions and limitations
under the License.
-Amazon SQS Operators
-====================
+==========
+Amazon SQS
+==========
`Amazon Simple Queue Service (SQS) `__ is a fully managed message queuing
service that enables you to decouple and scale microservices, distributed systems, and serverless
@@ -30,38 +31,44 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:SqsPublishOperator:
-Amazon SQS Publish Operator
-"""""""""""""""""""""""""""
+Publish a message to an Amazon SQS queue
+========================================
-To publish a message to an Amazon SQS queue use the
+To publish a message to an Amazon SQS queue you can use the
:class:`~airflow.providers.amazon.aws.operators.sqs.SqsPublishOperator`
-In the following example, the task "publish_to_queue" publishes a message containing
+In the following example, the task ``publish_to_queue`` publishes a message containing
the task instance and the execution date to a queue with a default name of ``Airflow-Example-Queue``.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sqs.py
:language: python
- :start-after: [START howto_sqs_operator]
- :end-before: [END howto_sqs_operator]
+ :dedent: 4
+ :start-after: [START howto_operator_sqs]
+ :end-before: [END howto_operator_sqs]
+
+Sensors
+-------
-.. _howto/operator:SqsSensor:
+.. _howto/sensor:SqsSensor:
-Amazon SQS Sensor
-"""""""""""""""""
+Read messages from an Amazon SQS queue
+======================================
To read messages from an Amazon SQS queue until exhausted use the
-:class:`~airflow.providers.amazon.aws.operators.sqs.SqsPublishOperator`
+:class:`~airflow.providers.amazon.aws.sensors.sqs.SqsSensor`
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_sqs.py
:language: python
- :start-after: [START howto_sqs_sensor]
- :end-before: [END howto_sqs_sensor]
+ :dedent: 4
+ :start-after: [START howto_sensor_sqs]
+ :end-before: [END howto_sensor_sqs]
References
----------
-For further information, look at:
-
-* `Boto3 Library Documentation for SQS `__
+* `AWS boto3 library documentation for SQS `__
diff --git a/docs/apache-airflow-providers-amazon/operators/step_functions.rst b/docs/apache-airflow-providers-amazon/operators/step_functions.rst
index 8d14af78fcbe7..e29d3194f6908 100644
--- a/docs/apache-airflow-providers-amazon/operators/step_functions.rst
+++ b/docs/apache-airflow-providers-amazon/operators/step_functions.rst
@@ -15,9 +15,9 @@
specific language governing permissions and limitations
under the License.
-
-AWS Step Functions Operators
-============================
+==================
+AWS Step Functions
+==================
`AWS Step Functions `__ makes it easy to coordinate the components
of distributed applications as a series of steps in a visual workflow. You can quickly build and run state
@@ -28,13 +28,16 @@ Prerequisite Tasks
.. include:: _partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:StepFunctionStartExecutionOperator:
-AWS Step Functions Start Execution Operator
-"""""""""""""""""""""""""""""""""""""""""""
+Start an AWS Step Functions state machine execution
+===================================================
-To start a new AWS Step Functions State Machine execution
-use :class:`~airflow.providers.amazon.aws.operators.step_function.StepFunctionStartExecutionOperator`.
+To start a new AWS Step Functions state machine execution you can use
+:class:`~airflow.providers.amazon.aws.operators.step_function.StepFunctionStartExecutionOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_step_functions.py
:language: python
@@ -42,37 +45,38 @@ use :class:`~airflow.providers.amazon.aws.operators.step_function.StepFunctionSt
:start-after: [START howto_operator_step_function_start_execution]
:end-before: [END howto_operator_step_function_start_execution]
-.. _howto/operator:StepFunctionExecutionSensor:
+.. _howto/operator:StepFunctionGetExecutionOutputOperator:
-AWS Step Functions Execution Sensor
-"""""""""""""""""""""""""""""""""""
+Get an AWS Step Functions execution output
+==========================================
-To wait on the state of an AWS Step Function State Machine execution until it reaches a terminal state you can
-use :class:`~airflow.providers.amazon.aws.sensors.step_function.StepFunctionExecutionSensor`.
+To fetch the output from an AWS Step Function state machine execution you can
+use :class:`~airflow.providers.amazon.aws.operators.step_function.StepFunctionGetExecutionOutputOperator`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_step_functions.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_step_function_execution_sensor]
- :end-before: [END howto_operator_step_function_execution_sensor]
+ :start-after: [START howto_operator_step_function_get_execution_output]
+ :end-before: [END howto_operator_step_function_get_execution_output]
-.. _howto/operator:StepFunctionGetExecutionOutputOperator:
+Sensors
+-------
-AWS Step Functions Get Execution Output Operator
-""""""""""""""""""""""""""""""""""""""""""""""""
+.. _howto/sensor:StepFunctionExecutionSensor:
-To fetch the output from an AWS Step Function State Machine execution you can
-use :class:`~airflow.providers.amazon.aws.operators.step_function.StepFunctionGetExecutionOutputOperator`.
+Wait on an AWS Step Functions state machine execution state
+===========================================================
+
+To wait on the state of an AWS Step Function state machine execution until it reaches a terminal state you can
+use :class:`~airflow.providers.amazon.aws.sensors.step_function.StepFunctionExecutionSensor`.
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_step_functions.py
:language: python
:dedent: 4
- :start-after: [START howto_operator_step_function_get_execution_output]
- :end-before: [END howto_operator_step_function_get_execution_output]
+ :start-after: [START howto_sensor_step_function_execution]
+ :end-before: [END howto_sensor_step_function_execution]
References
----------
-For further information, look at:
-
-* `Boto3 Library Documentation for Step Functions `__
+* `AWS boto3 library documentation for Step Functions `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/dynamodb_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/dynamodb_to_s3.rst
index 2fad595e81c5b..61727140a048c 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/dynamodb_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/dynamodb_to_s3.rst
@@ -15,29 +15,32 @@
specific language governing permissions and limitations
under the License.
+============================
+Amazon DynamoDB to Amazon S3
+============================
-Amazon DynamoDB to Amazon S3 Transfer Operator
-==============================================
-
-Use the DynamoDBToS3Operator transfer to copy the contents of an existing Amazon DynamoDB table
+Use the ``DynamoDBToS3Operator`` transfer to copy the content of an existing Amazon DynamoDB table
to an existing Amazon Simple Storage Service (S3) bucket.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/transfer:DynamoDBToS3Operator:
-DynamoDB To S3 Operator
-^^^^^^^^^^^^^^^^^^^^^^^
+Amazon DynamoDB To Amazon S3 transfer operator
+==============================================
-This operator replicates records from a DynamoDB table to a file in an S3 bucket.
-It scans a DynamoDB table and writes the received records to a file on the local
-filesystem. It flushes the file to S3 once the file size exceeds the file size limit
+This operator replicates records from an Amazon DynamoDB table to a file in an Amazon S3 bucket.
+It scans an Amazon DynamoDB table and writes the received records to a file on the local
+filesystem. It flushes the file to Amazon S3 once the file size exceeds the file size limit
specified by the user.
-Users can also specify a filtering criteria using dynamodb_scan_kwargs to only replicate
+Users can also specify a filtering criteria using ``dynamodb_scan_kwargs`` to only replicate
records that satisfy the criteria.
To get more information visit:
@@ -51,11 +54,17 @@ Example usage:
:start-after: [START howto_transfer_dynamodb_to_s3]
:end-before: [END howto_transfer_dynamodb_to_s3]
-To parallelize the replication, users can create multiple DynamoDBToS3Operator tasks using the
+To parallelize the replication, users can create multiple ``DynamoDBToS3Operator`` tasks using the
``TotalSegments`` parameter. For instance to replicate with parallelism of 2, create two tasks:
-.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3_segmented.py
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_dynamodb_to_s3.py
:language: python
:dedent: 4
:start-after: [START howto_transfer_dynamodb_to_s3_segmented]
:end-before: [END howto_transfer_dynamodb_to_s3_segmented]
+
+Reference
+---------
+
+* `AWS boto3 library documentation for Amazon DynamoDB `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/ftp_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/ftp_to_s3.rst
index eb1fe8ff1e140..eb4b63c6a5aba 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/ftp_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/ftp_to_s3.rst
@@ -15,20 +15,24 @@
specific language governing permissions and limitations
under the License.
-FTP to Amazon S3 Transfer Operator
-==================================
+================
+FTP to Amazon S3
+================
-Use the FTPToS3Operator transfer to copy data from a FTP server to an Amazon Simple Storage Service (S3) file.
+Use the ``FTPToS3Operator`` transfer to copy data from a FTP server to an Amazon Simple Storage Service (S3) file.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:FTPToS3Operator:
-FTP to Amazon S3
-^^^^^^^^^^^^^^^^
+FTP to Amazon S3 transfer operator
+==================================
This operator copies data from a FTP server to an Amazon S3 file.
@@ -44,6 +48,6 @@ Example usage:
:end-before: [END howto_transfer_ftp_to_s3]
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/gcs_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/gcs_to_s3.rst
index f19d005e943cf..6ad2efe169046 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/gcs_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/gcs_to_s3.rst
@@ -48,5 +48,5 @@ Example usage:
Reference
---------
-* `Google Cloud Storage Client library `__
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `Google Cloud Storage client library `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/glacier_to_gcs.rst b/docs/apache-airflow-providers-amazon/operators/transfer/glacier_to_gcs.rst
index b23e0cef92e6c..e51354f1e7790 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/glacier_to_gcs.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/glacier_to_gcs.rst
@@ -15,31 +15,41 @@
specific language governing permissions and limitations
under the License.
+========================
+Amazon S3 Glacier to GCS
+========================
-Amazon S3 Glacier to GCS Transfer Operator
-==========================================
-
-`Amazon Glacier `_ is a secure, durable, and extremely low-cost Amazon S3 cloud storage class for data archiving and long-term backup.
+`Amazon Glacier `_ is a secure, durable,
+and extremely low-cost Amazon S3 cloud storage class for data archiving and long-term backup.
Prerequisite Tasks
------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:GlacierToGCSOperator:
-Glacier To GCS
-""""""""""""""
+Amazon S3 Glacier To GCS transfer operator
+==========================================
-To transfer data from an Amazon Glacier vault to Google Cloud Storage.
-use :class:`~airflow.providers.amazon.aws.transfers.glacier_to_gcs.GlacierToGCSOperator`
+To transfer data from an Amazon Glacier vault to Google Cloud Storage you can use
+:class:`~airflow.providers.amazon.aws.transfers.glacier_to_gcs.GlacierToGCSOperator`
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_glacier_to_gcs.py
:language: python
:dedent: 4
- :start-after: [START howto_glacier_transfer_data_to_gcs]
- :end-before: [END howto_glacier_transfer_data_to_gcs]
+ :start-after: [START howto_transfer_glacier_to_gcs]
+ :end-before: [END howto_transfer_glacier_to_gcs]
.. note::
Please be aware that GlacierToGCSOperator depends on available memory.
Transferring large files may exhaust memory on the worker host.
+
+Reference
+---------
+
+* `AWS boto3 library documentation for Amazon S3 Glacier `__
+* `Google Cloud Storage client library `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/google_api_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/google_api_to_s3.rst
index 7e5ddaae9868e..50259b5e19c0b 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/google_api_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/google_api_to_s3.rst
@@ -16,21 +16,25 @@
specific language governing permissions and limitations
under the License.
-Google API to Amazon S3 Transfer Operator
-=========================================
+=======================
+Google API to Amazon S3
+=======================
-Use the GoogleApiToS3Transfer transfer to call requests to any Google API which supports discovery and save its response
-on Amazon S3.
+Use the ``GoogleApiToS3Operator`` transfer to make requests to any Google API which supports discovery and save
+its response in an Amazon S3 file.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
-.. _howto/operator:GoogleApiToS3Transfer:
+Operators
+---------
-Google Sheets to Amazon S3
-^^^^^^^^^^^^^^^^^^^^^^^^^^
+.. _howto/operator:GoogleApiToS3Operator:
+
+Google Sheets to Amazon S3 transfer operator
+============================================
This example loads data from Google Sheets and save it to an Amazon S3 file.
@@ -44,14 +48,11 @@ You can find more information about the Google API endpoint used
`here `__.
Google Youtube to Amazon S3
-^^^^^^^^^^^^^^^^^^^^^^^^^^^
+===========================
-This is a more advanced example dag for using ``GoogleApiToS3Transfer`` which uses xcom to pass data between
+This is a more advanced example dag for using ``GoogleApiToS3Operator`` which uses xcom to pass data between
tasks to retrieve specific information about YouTube videos.
-Get YouTube Videos
-""""""""""""""""""
-
It searches for up to 50 videos (due to pagination) in a given time range
(``YOUTUBE_VIDEO_PUBLISHED_AFTER``, ``YOUTUBE_VIDEO_PUBLISHED_BEFORE``) on a YouTube channel (``YOUTUBE_CHANNEL_ID``)
saves the response in Amazon S3 and also pushes the data to xcom.
@@ -72,11 +73,9 @@ information (``YOUTUBE_VIDEO_FIELDS``) for the requested videos and saves them i
:end-before: [END howto_transfer_google_api_youtube_list_to_s3]
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
-* `Google API Client library `__
-* `Google Sheets API v4 Documentation `__
-* `YouTube Data API v3 Documentation `__
-* `AWS boto3 Library Documentation for S3 `__
+* `Google API client library `__
+* `Google Sheets API v4 documentation `__
+* `YouTube Data API v3 documentation `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/hive_to_dynamodb.rst b/docs/apache-airflow-providers-amazon/operators/transfer/hive_to_dynamodb.rst
index cf40526d95f1e..3ef934761d974 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/hive_to_dynamodb.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/hive_to_dynamodb.rst
@@ -15,24 +15,27 @@
specific language governing permissions and limitations
under the License.
+==============================
+Apache Hive to Amazon DynamoDB
+==============================
-Apache Hive to Amazon DynamoDB Transfer Operator
-================================================
-
-Use the HiveToDynamoDBOperator transfer to copy the contents of an
+Use the ``HiveToDynamoDBOperator`` transfer to copy the contents of an
existing Apache Hive table to an existing Amazon DynamoDB table.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/transfer:HiveToDynamoDBOperator:
-Hive to DynamoDB Operator
-^^^^^^^^^^^^^^^^^^^^^^^^^
+Apache Hive to Amazon DynamoDB transfer operator
+================================================
-This operator replicates records from a Hive table to a DynamoDB table. The user must
+This operator replicates records from an Apache Hive table to an Amazon DynamoDB table. The user must
specify an `HQL query `__
to use as filtering criteria.
@@ -48,9 +51,7 @@ Example usage:
:end-before: [END howto_transfer_hive_to_dynamodb]
Reference
-^^^^^^^^^
-
-For further information, look at:
+---------
-* `Boto3 Library Documentation for DynamoDB `__
-* `Hive Language Manual `__
+* `Apache Hive language manual `__
+* `AWS boto3 library documentation for Amazon DynamoDB `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/imap_attachment_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/imap_attachment_to_s3.rst
index 2e9b6a758f10f..a104889595ce6 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/imap_attachment_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/imap_attachment_to_s3.rst
@@ -15,21 +15,28 @@
specific language governing permissions and limitations
under the License.
-Imap Attachment to Amazon S3 Transfer Operator
-==============================================
+============================
+Imap Attachment to Amazon S3
+============================
The ``ImapAttachmentToS3Operator`` transfers an email attachment via IMAP
-protocol from a mail server to an Amazon S3 Bucket.
+protocol from an email server to an Amazon S3 Bucket.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:ImapAttachmentToS3Operator:
-Imap Attachment To Amazon S3
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Imap Attachment To Amazon S3 transfer operator
+==============================================
+
+To save an email attachment via IMAP protocol from an email server to an Amazon S3 Bucket you can use
+:class:`~airflow.providers.amazon.aws.transfers.imap_attachment_to_s3.ImapAttachmentToS3Operator`
.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_imap_attachment_to_s3.py
:language: python
@@ -38,7 +45,7 @@ Imap Attachment To Amazon S3
:end-before: [END howto_transfer_imap_attachment_to_s3]
Reference
-^^^^^^^^^
+---------
-* `IMAP Library Documentation `__
-* `AWS boto3 Library Documentation for S3 `__
+* `IMAP Library documentation `__
+* `AWS boto3 library documentation for S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/local_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/local_to_s3.rst
index 53df0c629e043..549ad8e2b1c93 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/local_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/local_to_s3.rst
@@ -15,21 +15,25 @@
specific language governing permissions and limitations
under the License.
-Local Filesystem to Amazon S3 Transfer Operator
-===============================================
+=============================
+Local Filesystem to Amazon S3
+=============================
-Use the LocalFilesystemToS3Operator transfer to copy data from the Airflow local filesystem
+Use the ``LocalFilesystemToS3Operator`` transfer to copy data from the Airflow local filesystem
to an Amazon Simple Storage Service (S3) file.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:LocalFilesystemToS3Operator:
-Local to Amazon S3
-^^^^^^^^^^^^^^^^^^
+Local to Amazon S3 transfer operator
+====================================
This operator copies data from the local filesystem to an Amazon S3 file.
@@ -45,6 +49,6 @@ Example usage:
:end-before: [END howto_transfer_local_to_s3]
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/mongo_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/mongo_to_s3.rst
index 209461f5be57c..aeab7083cbd32 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/mongo_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/mongo_to_s3.rst
@@ -15,20 +15,25 @@
specific language governing permissions and limitations
under the License.
-MongoDB to Amazon S3 Transfer Operator
-======================================
+====================
+MongoDB to Amazon S3
+====================
-Use the MongoToS3Operator transfer to copy data from a MongoDB collection into an Amazon Simple Storage Service (S3) file.
+Use the ``MongoToS3Operator`` transfer to copy data from a MongoDB collection into an Amazon Simple Storage Service
+(S3) file.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:MongoToS3Operator:
-MongoDB To Amazon S3
-^^^^^^^^^^^^^^^^^^^^
+MongoDB To Amazon S3 transfer operator
+======================================
This operator copies a set of data from a MongoDB collection to an Amazon S3 files.
In order to select the data you want to copy, you need to use the ``mongo_query`` parameter.
@@ -48,7 +53,7 @@ You can find more information about ``PyMongo`` used by Airflow to communicate w
`here `__.
Reference
-^^^^^^^^^
+---------
* `PyMongo `__
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/redshift_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/redshift_to_s3.rst
index 72306a89a0995..2235bf8bb3bb5 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/redshift_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/redshift_to_s3.rst
@@ -15,21 +15,25 @@
specific language governing permissions and limitations
under the License.
-Amazon Redshift to Amazon S3 Transfer Operator
-==============================================
+============================
+Amazon Redshift to Amazon S3
+============================
-Use the RedshiftToS3Operator transfer to copy the data from an Amazon Redshift table into an Amazon Simple Storage
+Use the ``RedshiftToS3Operator`` transfer to copy the data from an Amazon Redshift table into an Amazon Simple Storage
Service (S3) file.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:RedshiftToS3Operator:
-Amazon Redshift To Amazon S3
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Amazon Redshift To Amazon S3 transfer operator
+==============================================
This operator loads data from an Amazon Redshift table to an existing Amazon S3 bucket.
@@ -48,8 +52,8 @@ You can find more information to the ``UNLOAD`` command used
`here `__.
Reference
-^^^^^^^^^
+---------
-* `AWS UNLOAD from Amazon Redshift Documentation `__
-* `AWS boto3 Library Documentation for Amazon Redshift `__
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS UNLOAD from Amazon Redshift documentation `__
+* `AWS boto3 library documentation for Amazon Redshift `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_ftp.rst b/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_ftp.rst
index 4757705c6ce80..953ded33ba067 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_ftp.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_ftp.rst
@@ -15,21 +15,25 @@
specific language governing permissions and limitations
under the License.
-Amazon S3 to FTP Transfer Operator
-==================================
+================
+Amazon S3 to FTP
+================
-Use the S3ToFTPOperator transfer to copy data from an Amazon Simple Storage Service (S3) file into a remote file
+Use the ``S3ToFTPOperator`` transfer to copy data from an Amazon Simple Storage Service (S3) file into a remote file
using FTP protocol.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:S3ToFTPOperator:
-Amazon S3 To FTP
-^^^^^^^^^^^^^^^^
+Amazon S3 To FTP transfer operator
+==================================
This operator copies data from Amazon S3 to a FTP server.
@@ -45,6 +49,6 @@ Example usage:
:end-before: [END howto_transfer_s3_to_ftp]
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_redshift.rst b/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_redshift.rst
index 877e7dde594eb..cfc9d404f5949 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_redshift.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_redshift.rst
@@ -15,21 +15,25 @@
specific language governing permissions and limitations
under the License.
-Amazon S3 to Amazon Redshift Transfer Operator
-==============================================
+============================
+Amazon S3 to Amazon Redshift
+============================
-Use the S3ToRedshiftOperator transfer to copy the data from an Amazon Simple Storage Service (S3) file into an
+Use the ``S3ToRedshiftOperator`` transfer to copy the data from an Amazon Simple Storage Service (S3) file into an
Amazon Redshift table.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:S3ToRedshiftOperator:
-Amazon S3 To Amazon Redshift
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Amazon S3 To Amazon Redshift transfer operator
+==============================================
This operator loads data from Amazon S3 to an existing Amazon Redshift table.
@@ -48,8 +52,8 @@ You can find more information to the ``COPY`` command used
`here `__.
Reference
-^^^^^^^^^
+---------
-* `AWS COPY from Amazon S3 Documentation `__
-* `AWS boto3 Library Documentation for Amazon S3 `__
-* `AWS boto3 Library Documentation for Amazon Redshift `__
+* `AWS COPY from Amazon S3 documentation `__
+* `AWS boto3 library documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon Redshift `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_sftp.rst b/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_sftp.rst
index 2b3b8fcfc9837..a1c9d4c1efe1f 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_sftp.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/s3_to_sftp.rst
@@ -15,22 +15,26 @@
specific language governing permissions and limitations
under the License.
-Amazon S3 to SFTP Transfer Operator
-===================================
+=================
+Amazon S3 to SFTP
+=================
-Use the S3ToSFTPOperator transfer to copy the data from an Amazon Simple Storage Service (S3) file into a remote file
+Use the ``S3ToSFTPOperator`` transfer to copy the data from an Amazon Simple Storage Service (S3) file into a remote file
using SFTP protocol.
For more information about the service visits `Amazon Transfer for SFTP API documentation `_.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:S3ToSFTPOperator:
-Amazon S3 To SFTP
-^^^^^^^^^^^^^^^^^
+Amazon S3 To SFTP transfer operator
+===================================
This operator loads data from Amazon S3 to a SFTP server.
@@ -46,6 +50,6 @@ Example usage:
:end-before: [END howto_transfer_s3_to_sftp]
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/salesforce_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/salesforce_to_s3.rst
index 06bd1bb4e7de3..1f9725eb2f5c5 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/salesforce_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/salesforce_to_s3.rst
@@ -15,22 +15,25 @@
specific language governing permissions and limitations
under the License.
-Salesforce to Amazon S3 Transfer Operator
-=========================================
+=======================
+Salesforce to Amazon S3
+=======================
-Use the
-:class:`~airflow.providers.amazon.aws.transfers.salesforce_to_s3.SalesforceToS3Operator`
-to execute a Salesforce query to fetch data and upload to an Amazon S3 bucket.
+Use the ``SalesforceToS3Operator`` transfer to execute a Salesforce query to fetch data and upload to an Amazon Simple
+Storage Service (S3) file.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:SalesforceToS3Operator:
-Extract Account data from Salesforce
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Extract data from Salesforce to Amazon S3 transfer operator
+===========================================================
The following example demonstrates a use case of extracting account data from a Salesforce
instance and upload to an Amazon S3 bucket.
@@ -45,3 +48,4 @@ Reference
---------
* `Simple Salesforce Documentation `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/sftp_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/sftp_to_s3.rst
index 00556e7abe0c0..67f91c9edb408 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/sftp_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/sftp_to_s3.rst
@@ -15,21 +15,25 @@
specific language governing permissions and limitations
under the License.
-SFTP to Amazon S3 Transfer Operator
-===================================
+=================
+SFTP to Amazon S3
+=================
-Use the SFTPToS3Operator transfer to copy the data from a SFTP server to an Amazon Simple Storage Service (S3) file.
+Use the ``SFTPToS3Operator`` transfer to copy the data from a SFTP server to an Amazon Simple Storage Service (S3) file.
For more information about the service visits `Amazon Transfer for SFTP API documentation `_.
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:SFTPToS3Operator:
-SFTP to Amazon S3
-^^^^^^^^^^^^^^^^^
+SFTP to Amazon S3 transfer operator
+===================================
This operator loads data from a SFTP server to an Amazon S3 file.
@@ -45,6 +49,6 @@ Example usage:
:end-before: [END howto_transfer_sftp_to_s3]
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/docs/apache-airflow-providers-amazon/operators/transfer/sql_to_s3.rst b/docs/apache-airflow-providers-amazon/operators/transfer/sql_to_s3.rst
index 20c8a1fe232c8..0a75b7cd9c7ef 100644
--- a/docs/apache-airflow-providers-amazon/operators/transfer/sql_to_s3.rst
+++ b/docs/apache-airflow-providers-amazon/operators/transfer/sql_to_s3.rst
@@ -15,23 +15,27 @@
specific language governing permissions and limitations
under the License.
-SQL to Amazon S3 Transfer Operator
-==================================
+================
+SQL to Amazon S3
+================
-Use SqlToS3Operator to copy data from a SQL server to an Amazon Simple Storage Service (S3) file.
-SqlToS3Operator is compatible with any SQL connection as long as the SQL hook has function that
+Use ``SqlToS3Operator`` to copy data from a SQL server to an Amazon Simple Storage Service (S3) file.
+``SqlToS3Operator`` is compatible with any SQL connection as long as the SQL hook has function that
converts the SQL result to `pandas dataframe `__
(e.g. MySQL, Hive, ...).
Prerequisite Tasks
-^^^^^^^^^^^^^^^^^^
+------------------
.. include:: ../_partials/prerequisite_tasks.rst
+Operators
+---------
+
.. _howto/operator:SqlToS3Operator:
-MySQL to Amazon S3
-^^^^^^^^^^^^^^^^^^
+MySQL to Amazon S3 transfer operator
+====================================
This example sends the response of a MySQL query to an Amazon S3 file.
@@ -47,6 +51,6 @@ Example usage:
:end-before: [END howto_transfer_sql_to_s3]
Reference
-^^^^^^^^^
+---------
-* `AWS boto3 Library Documentation for Amazon S3 `__
+* `AWS boto3 library documentation for Amazon S3 `__
diff --git a/tests/providers/amazon/aws/transfers/test_google_api_to_s3_system.py b/tests/providers/amazon/aws/transfers/test_google_api_to_s3_system.py
index 8f55a73d04339..9f06d72275759 100644
--- a/tests/providers/amazon/aws/transfers/test_google_api_to_s3_system.py
+++ b/tests/providers/amazon/aws/transfers/test_google_api_to_s3_system.py
@@ -52,7 +52,7 @@ def provide_s3_bucket_youtube():
@pytest.mark.backend("mysql", "postgres")
@pytest.mark.credential_file(GMP_KEY)
-class GoogleApiToS3TransferExampleDagsSystemTest(GoogleSystemTest, AmazonSystemTest):
+class GoogleApiToS3OperatorExampleDagsSystemTest(GoogleSystemTest, AmazonSystemTest):
@pytest.mark.usefixtures("provide_s3_bucket_sheets")
@provide_aws_context()
@provide_gcp_context(GMP_KEY, scopes=['https://www.googleapis.com/auth/spreadsheets.readonly'])