From 08b5bfa081d4abbf22a6e1c109d59c73ce28f152 Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Wed, 30 Jun 2021 10:01:17 -0700 Subject: [PATCH 1/8] Add an Amazon EMR on EKS provider package - Adds an operator, sensor, and hook for running Spark jobs on EMR on EKS as well as docs and an example DAG --- .../aws/example_dags/example_emr_eks_job.py | 71 ++++++ .../amazon/aws/hooks/emr_containers.py | 209 ++++++++++++++++++ .../amazon/aws/operators/emr_containers.py | 146 ++++++++++++ .../amazon/aws/sensors/emr_containers.py | 93 ++++++++ airflow/providers/amazon/provider.yaml | 15 ++ .../operators/emr_eks.rst | 84 +++++++ .../amazon/aws/hooks/test_emr_containers.py | 57 +++++ .../aws/operators/test_emr_containers.py | 102 +++++++++ .../amazon/aws/sensors/test_emr_containers.py | 72 ++++++ 9 files changed, 849 insertions(+) create mode 100644 airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py create mode 100644 airflow/providers/amazon/aws/hooks/emr_containers.py create mode 100644 airflow/providers/amazon/aws/operators/emr_containers.py create mode 100644 airflow/providers/amazon/aws/sensors/emr_containers.py create mode 100644 docs/apache-airflow-providers-amazon/operators/emr_eks.rst create mode 100644 tests/providers/amazon/aws/hooks/test_emr_containers.py create mode 100644 tests/providers/amazon/aws/operators/test_emr_containers.py create mode 100644 tests/providers/amazon/aws/sensors/test_emr_containers.py diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py b/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py new file mode 100644 index 0000000000000..aa935a5a14694 --- /dev/null +++ b/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py @@ -0,0 +1,71 @@ +# 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. +""" +This is an example dag for an Amazon EMR on EKS Spark job. +""" +import os +from datetime import timedelta + +from airflow import DAG +from airflow.providers.amazon.aws.operators.emr_containers import EMRContainerOperator +from airflow.utils.dates import days_ago + +# [START howto_operator_emr_eks_env_variables] +VIRTUAL_CLUSTER_ID = os.getenv("VIRTUAL_CLUSTER_ID", "test-cluster") +JOB_ROLE_ARN = os.getenv("JOB_ROLE_ARN", "arn:aws:iam::012345678912:role/emr_eks_default_role") +# [END howto_operator_emr_eks_env_variables] + + +# [START howto_operator_emr_eks_config] +JOB_DRIVER_ARG = { + "sparkSubmitJobDriver": { + "entryPoint": "local:///usr/lib/spark/examples/src/main/python/pi.py", + "sparkSubmitParameters": "--conf spark.executors.instances=2 --conf spark.executors.memory=2G --conf spark.executor.cores=2 --conf spark.driver.cores=1", # noqa: E501 + } +} + +CONFIGURATION_OVERRIDES_ARG = { + "monitoringConfiguration": { + "cloudWatchMonitoringConfiguration": { + "logGroupName": "/aws/emr-eks-spark", + "logStreamNamePrefix": "airflow", + } + } +} +# [END howto_operator_emr_eks_config] + +with DAG( + dag_id='emr_eks_pi_job', + dagrun_timeout=timedelta(hours=2), + start_date=days_ago(1), + schedule_interval="@once", + tags=["emr_containers", "example"], +) as dag: + + # An example of how to get the cluster id and arn from an Airflow connection + # VIRTUAL_CLUSTER_ID = '{{ conn.emr_eks.extra_dejson["virtual_cluster_id"] }}' + # JOB_ROLE_ARN = '{{ conn.emr_eks.extra_dejson["job_role_arn"] }}' + + job_starter = EMRContainerOperator( + task_id="start_job", + virtual_cluster_id=VIRTUAL_CLUSTER_ID, + execution_role_arn=JOB_ROLE_ARN, + release_label="emr-6.3.0-latest", + job_driver=JOB_DRIVER_ARG, + configuration_overrides=CONFIGURATION_OVERRIDES_ARG, + name="pi.py", + ) diff --git a/airflow/providers/amazon/aws/hooks/emr_containers.py b/airflow/providers/amazon/aws/hooks/emr_containers.py new file mode 100644 index 0000000000000..7d03d6f2a0744 --- /dev/null +++ b/airflow/providers/amazon/aws/hooks/emr_containers.py @@ -0,0 +1,209 @@ +# 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 time import sleep +from typing import Any, Dict, Optional + +from botocore.exceptions import ClientError + +from airflow.exceptions import AirflowException +from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook + + +class EMRContainerHook(AwsBaseHook): + """ + Interact with AWS EMR Virtual Cluster to run, poll jobs and return job status + Additional arguments (such as ``aws_conn_id``) may be specified and + are passed down to the underlying AwsBaseHook. + + .. seealso:: + :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook` + + :param virtual_cluster_id: Cluster ID of the EMR on EKS virtual cluster + :type virtual_cluster_id: str + """ + + INTERMEDIATE_STATES = ( + "PENDING", + "SUBMITTED", + "RUNNING", + ) + FAILURE_STATES = ( + "FAILED", + "CANCELLED", + "CANCEL_PENDING", + ) + SUCCESS_STATES = ("COMPLETED",) + + def __init__(self, *args: Any, virtual_cluster_id: str = None, **kwargs: Any) -> None: + super().__init__(client_type="emr-containers", *args, **kwargs) # type: ignore + self.virtual_cluster_id = virtual_cluster_id + + def submit_job( + self, + name: str, + execution_role_arn: str, + release_label: str, + job_driver: dict, + configuration_overrides: Optional[dict] = None, + client_request_token: Optional[str] = None, + ) -> str: + """ + Submit a job to the EMR Containers API and and return the job ID. + A job run is a unit of work, such as a Spark jar, PySpark script, + or SparkSQL query, that you submit to Amazon EMR on EKS. + See: https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/emr-containers.html#EMRContainers.Client.start_job_run # noqa: E501 + + :param name: The name of the job run. + :type name: str + :param execution_role_arn: The IAM role ARN associated with the job run. + :type execution_role_arn: str + :param release_label: The Amazon EMR release version to use for the job run. + :type release_label: str + :param job_driver: Job configuration details, e.g. the Spark job parameters. + :type job_driver: dict + :param configuration_overrides: The configuration overrides for the job run, + specifically either application configuration or monitoring configuration. + :type configuration_overrides: dict + :param client_request_token: The client idempotency token of the job run request. + Use this if you want to specify a unique ID to prevent two jobs from getting started. + :type client_request_token: str + :return: Job ID + """ + params = { + "name": name, + "virtualClusterId": self.virtual_cluster_id, + "executionRoleArn": execution_role_arn, + "releaseLabel": release_label, + "jobDriver": job_driver, + "configurationOverrides": configuration_overrides or {}, + } + if client_request_token: + params["clientToken"] = client_request_token + + response = self.conn.start_job_run(**params) + + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException(f'Start Job Run failed: {response}') + else: + self.log.info( + f"Start Job Run success - Job Id %s and virtual cluster id %s", + response['id'], + response['virtualClusterId'], + ) + return response['id'] + + def get_job_failure_reason(self, job_id: str) -> Optional[str]: + """ + Fetch the reason for a job failure (e.g. error message). Returns None or reason string. + + :param job_id: Id of submitted job run + :type job_id: str + :return: str + """ + # We absorb any errors if we can't retrieve the job status + reason = None + + try: + response = self.conn.describe_job_run( + virtualClusterId=self.virtual_cluster_id, + id=job_id, + ) + reason = response['jobRun']['failureReason'] + except KeyError: + self.log.error('Could not get status of the EMR on EKS job') + except ClientError as ex: + self.log.error('AWS request failed, check logs for more info: %s', ex) + + return reason + + def check_query_status(self, job_id: str) -> Optional[str]: + """ + Fetch the status of submitted job run. Returns None or one of valid query states. + See: https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/emr-containers.html#EMRContainers.Client.describe_job_run # noqa: E501 + :param job_id: Id of submitted job run + :type job_id: str + :return: str + """ + try: + response = self.conn.describe_job_run( + virtualClusterId=self.virtual_cluster_id, + id=job_id, + ) + return response["jobRun"]["state"] + except self.conn.exceptions.ResourceNotFoundException: + # If the job is not found, we raise an exception as something fatal has happened. + raise AirflowException(f'Job ID {job_id} not found on Virtual Cluster {self.virtual_cluster_id}') + except ClientError as ex: + # If we receive a generic ClientError, we swallow the exception so that the + self.log.error('AWS request failed, check logs for more info: %s', ex) + return None + + def poll_query_status( + self, job_id: str, max_tries: Optional[int] = None, poll_interval: int = 30 + ) -> Optional[str]: + """ + Poll the status of submitted job run until query state reaches final state. + Returns one of the final states. + + :param job_id: Id of submitted job run + :type job_id: str + :param max_tries: Number of times to poll for query state before function exits + :type max_tries: int + :param poll_interval: Time (in seconds) to wait between calls to check query status on EMR + :type poll_interval: int + :return: str + """ + try_number = 1 + final_query_state = None # Query state when query reaches final state or max_tries reached + + # TODO: Make this logic a little bit more robust. + # Currently this polls until the state is *not* one of the INTERMEDIATE_STATES + # While that should work in most cases...it might not. :) + while True: + query_state = self.check_query_status(job_id) + if query_state is None: + self.log.info(f"Try %s: Invalid query state. Retrying again", try_number) + elif query_state in self.INTERMEDIATE_STATES: + self.log.info( + f"Try %s: Query is still in an intermediate state - %s", try_number, query_state + ) + else: + self.log.info( + f"Try %s: Query execution completed. Final state is %s", try_number, query_state + ) + final_query_state = query_state + break + if max_tries and try_number >= max_tries: # Break loop if max_tries reached + final_query_state = query_state + break + try_number += 1 + sleep(poll_interval) + return final_query_state + + def stop_query(self, job_id: str) -> Dict: + """ + Cancel the submitted job_run + + :param job_id: Id of submitted job_run + :type job_id: str + :return: dict + """ + return self.conn.cancel_job_run( + virtualClusterId=self.virtual_cluster_id, + id=job_id, + ) diff --git a/airflow/providers/amazon/aws/operators/emr_containers.py b/airflow/providers/amazon/aws/operators/emr_containers.py new file mode 100644 index 0000000000000..b8a5edb394cf9 --- /dev/null +++ b/airflow/providers/amazon/aws/operators/emr_containers.py @@ -0,0 +1,146 @@ +# 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 typing import Any, Optional +from uuid import uuid4 + +from airflow.exceptions import AirflowException + +try: + from functools import cached_property +except ImportError: + from cached_property import cached_property + +from airflow.models import BaseOperator +from airflow.providers.amazon.aws.hooks.emr_containers import EMRContainerHook +from airflow.utils.decorators import apply_defaults + + +class EMRContainerOperator(BaseOperator): + """ + An operator that submits jobs to EMR on EKS virtual clusters. + + :param name: The name of the job run. + :type name: str + :param virtual_cluster_id: The EMR on EKS virtual cluster ID + :type virtual_cluster_id: str + :param execution_role_arn: The IAM role ARN associated with the job run. + :type execution_role_arn: str + :param release_label: The Amazon EMR release version to use for the job run. + :type release_label: str + :param job_driver: Job configuration details, e.g. the Spark job parameters. + :type job_driver: dict + :param configuration_overrides: The configuration overrides for the job run, + specifically either application configuration or monitoring configuration. + :type configuration_overrides: dict + :param client_request_token: The client idempotency token of the job run request. + Use this if you want to specify a unique ID to prevent two jobs from getting started. + If no token is provided, a UUIDv4 token will be generated for you. + :type client_request_token: str + :param aws_conn_id: The Airflow connection used for AWS credentials. + :type aws_conn_id: str + :param poll_interval: Time (in seconds) to wait between two consecutive calls to check query status on EMR + :type poll_interval: int + :param max_tries: Maximum number of times to wait for the job run to finish. + Defaults to None, which will poll until the job is *not* in a pending, submitted, or running state. + :type max_tries: int + """ + + template_fields = ["name", "virtual_cluster_id", "execution_role_arn", "release_label", "job_driver"] + ui_color = "#f9c915" + + @apply_defaults + def __init__( # pylint: disable=too-many-arguments + self, + *, + name: str, + virtual_cluster_id: str, + execution_role_arn: str, + release_label: str, + job_driver: dict, + configuration_overrides: Optional[dict] = None, + client_request_token: Optional[str] = None, + aws_conn_id: str = "aws_default", + poll_interval: int = 30, + max_tries: Optional[int] = None, + **kwargs: Any, + ) -> None: + super().__init__(**kwargs) + self.name = name + self.virtual_cluster_id = virtual_cluster_id + self.execution_role_arn = execution_role_arn + self.release_label = release_label + self.job_driver = job_driver + self.configuration_overrides = configuration_overrides or {} + self.aws_conn_id = aws_conn_id + self.client_request_token = client_request_token or str(uuid4()) + self.poll_interval = poll_interval + self.max_tries = max_tries + self.job_id = None + + @cached_property + def hook(self) -> EMRContainerHook: + """Create and return an EMRContainerHook.""" + return EMRContainerHook( + self.aws_conn_id, + virtual_cluster_id=self.virtual_cluster_id, + ) + + def execute(self, context: dict) -> Optional[str]: + """Run job on EMR Containers""" + self.job_id = self.hook.submit_job( + self.name, + self.execution_role_arn, + self.release_label, + self.job_driver, + self.configuration_overrides, + self.client_request_token, + ) + query_status = self.hook.poll_query_status(self.job_id, self.max_tries, self.poll_interval) + + if query_status in EMRContainerHook.FAILURE_STATES: + error_message = self.hook.get_job_failure_reason(self.job_id) + raise AirflowException( + f"EMR Containers job failed. Final state is {query_status}, query_execution_id is {self.job_id}. Error: {error_message}" + ) + elif not query_status or query_status in EMRContainerHook.INTERMEDIATE_STATES: + raise AirflowException( + "Final state of EMR Containers job is {query_status}. " + "Max tries of poll status exceeded, query_execution_id is {self.job_id}." + ) + + return self.job_id + + def on_kill(self) -> None: + """Cancel the submitted job run""" + if self.job_id: + self.log.info("Stopping job run with jobId - %s", self.job_id) + response = self.hook.stop_query(self.job_id) + http_status_code = None + try: + http_status_code = response["ResponseMetadata"]["HTTPStatusCode"] + except Exception as ex: + self.log.error("Exception while cancelling query: %s", ex) + finally: + if http_status_code is None or http_status_code != 200: + self.log.error("Unable to request query cancel on EMR. Exiting") + else: + self.log.info( + "Polling EMR for query with id %s to reach final state", + self.job_id, + ) + self.hook.poll_query_status(self.job_id) diff --git a/airflow/providers/amazon/aws/sensors/emr_containers.py b/airflow/providers/amazon/aws/sensors/emr_containers.py new file mode 100644 index 0000000000000..692911375a62e --- /dev/null +++ b/airflow/providers/amazon/aws/sensors/emr_containers.py @@ -0,0 +1,93 @@ +# 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 typing import Any, Optional + +try: + from functools import cached_property +except ImportError: + from cached_property import cached_property + +from airflow.exceptions import AirflowException +from airflow.providers.amazon.aws.hooks.emr_containers import EMRContainerHook +from airflow.sensors.base import BaseSensorOperator + + +class EMRContainerSensor(BaseSensorOperator): + """ + Asks for the state of the job run until it reaches a failure state or success state. + If the job run fails, the task will fail. + + :param job_id: job_id to check the state of + :type job_id: str + :param max_retries: Number of times to poll for query state before + returning the current state, defaults to None + :type max_retries: int + :param aws_conn_id: aws connection to use, defaults to 'aws_default' + :type aws_conn_id: str + :param poll_interval: Time in seconds to wait between two consecutive call to + check query status on athena, defaults to 10 + :type poll_interval: int + """ + + INTERMEDIATE_STATES = ( + "PENDING", + "SUBMITTED", + "RUNNING", + ) + FAILURE_STATES = ( + "FAILED", + "CANCELLED", + "CANCEL_PENDING", + ) + SUCCESS_STATES = ("COMPLETED",) + + template_fields = ['virtual_cluster_id', 'job_id'] + template_ext = () + ui_color = '#66c3ff' + + def __init__( + self, + *, + virtual_cluster_id: str, + job_id: str, + max_retries: Optional[int] = None, + aws_conn_id: str = 'aws_default', + poll_interval: int = 10, + **kwargs: Any, + ) -> None: + super().__init__(**kwargs) + self.aws_conn_id = aws_conn_id + self.virtual_cluster_id = virtual_cluster_id + self.job_id = job_id + self.poll_interval = poll_interval + self.max_retries = max_retries + + def poke(self, context: dict) -> bool: + state = self.hook.poll_query_status(self.job_id, self.max_retries, self.poll_interval) + + if state in self.FAILURE_STATES: + raise AirflowException('EMR Containers sensor failed') + + if state in self.INTERMEDIATE_STATES: + return False + return True + + @cached_property + def hook(self) -> EMRContainerHook: + """Create and return an EMRContainerHook""" + return EMRContainerHook(self.aws_conn_id, virtual_cluster_id=self.virtual_cluster_id) diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index 56b6621a9ad15..ce1255a1b5e24 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -79,6 +79,12 @@ integrations: - /docs/apache-airflow-providers-amazon/operators/emr.rst logo: /integration-logos/aws/Amazon-EMR_light-bg@4x.png tags: [aws] + - integration-name: Amazon EMR on EKS + external-doc-url: https://docs.aws.amazon.com/emr/latest/EMR-on-EKS-DevelopmentGuide/emr-eks.html + how-to-guide: + - /docs/apache-airflow-providers-amazon/operators/emr_eks.rst + logo: /integration-logos/aws/Amazon-EMR_light-bg@4x.png + tags: [aws] - integration-name: Amazon Glacier external-doc-url: https://aws.amazon.com/glacier/ logo: /integration-logos/aws/Amazon-S3-Glacier_light-bg@4x.png @@ -188,6 +194,9 @@ operators: - airflow.providers.amazon.aws.operators.emr_create_job_flow - airflow.providers.amazon.aws.operators.emr_modify_cluster - airflow.providers.amazon.aws.operators.emr_terminate_job_flow + - integration-name: Amazon EMR on EKS + python-modules: + - airflow.providers.amazon.aws.operators.emr_containers - integration-name: Amazon Glacier python-modules: - airflow.providers.amazon.aws.operators.glacier @@ -245,6 +254,9 @@ sensors: - airflow.providers.amazon.aws.sensors.emr_base - airflow.providers.amazon.aws.sensors.emr_job_flow - airflow.providers.amazon.aws.sensors.emr_step + - integration-name: Amazon EMR on EKS + python-modules: + - airflow.providers.amazon.aws.sensors.emr_containers - integration-name: Amazon Glacier python-modules: - airflow.providers.amazon.aws.sensors.glacier @@ -311,6 +323,9 @@ hooks: - integration-name: Amazon EMR python-modules: - airflow.providers.amazon.aws.hooks.emr + - integration-name: Amazon EMR on EKS + python-modules: + - airflow.providers.amazon.aws.hooks.emr_containers - integration-name: Amazon Glacier python-modules: - airflow.providers.amazon.aws.hooks.glacier diff --git a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst new file mode 100644 index 0000000000000..c6eb05d5c0fc7 --- /dev/null +++ b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst @@ -0,0 +1,84 @@ + .. 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. + + +.. _howto/operator:EMRContainersOperators: + +Amazon EMR on EKS Operators +=========================== + +.. contents:: + :depth: 1 + :local: + +Prerequisite Tasks +------------------ + +.. include:: _partials/prerequisite_tasks.rst + +Overview +-------- + +Airflow to Amazon EMR on Amazon EKS integration provides a way to run Apache Spark jobs on Kubernetes. + +- :class:`~airflow.providers.amazon.aws.operators.emr_containers.EMRContainerOperator` + + +Create EMR on EKS job with sample script +---------------------------------------- + +Purpose +""""""" + +This example dag ``example_emr_eks_job.py`` uses ``EMRContainerOperator`` to create a new EMR on EKS job calculating the mathematical constant ``Pi``, and monitors the progress +with ``EMRContainerSensor``. + +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. + +Environment variables +""""""""""""""""""""" + +This example relies on the following variables, which can be passed via OS environment variables. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py + :language: python + :start-after: [START howto_operator_emr_eks_env_variables] + :end-before: [END howto_operator_emr_eks_env_variables] + +Job configuration +""""""""""""""""" + +To create a job for EMR on EKS, you need to specify your job configuration, any monitoring configuration, and a few other details for the job. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py + :language: python + :start-after: [START howto_operator_emr_eks_config] + :end-before: [END howto_operator_emr_eks_config] + +With EMR on EKS, you specify your Spark configuration, the EMR release you want to use, the IAM role to use for the job, and the EMR virtual cluster ID. + +We pass the ``virtual_cluster_id`` and ``execution_role_arn`` values as operator parameters, but you can store them in a Connection or provide them in the DAG. + +With the EMRContainerOperator, it will wait until the successful completion of the job or raise an ``AirflowException`` if there is an error. + +Reference +--------- + +For further information, look at: + +* `Amazon EMR on EKS Job runs `__ +* `EMR on EKS Best Practices `__ diff --git a/tests/providers/amazon/aws/hooks/test_emr_containers.py b/tests/providers/amazon/aws/hooks/test_emr_containers.py new file mode 100644 index 0000000000000..8b8db5d330d53 --- /dev/null +++ b/tests/providers/amazon/aws/hooks/test_emr_containers.py @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +import unittest +from unittest import mock + +from airflow.providers.amazon.aws.hooks.emr_containers import EMRContainerHook + +SUBMIT_JOB_SUCCESS_RETURN = { + 'ResponseMetadata': {'HTTPStatusCode': 200}, + 'id': 'job123456', + 'virtualClusterId': 'vc1234', +} + + +class TestEMRContainerHook(unittest.TestCase): + def setUp(self): + self.emr_containers = EMRContainerHook(virtual_cluster_id='vc1234') + + def test_init(self): + assert self.emr_containers.aws_conn_id == 'aws_default' + assert self.emr_containers.virtual_cluster_id == 'vc1234' + + @mock.patch("boto3.session.Session") + def test_submit_job(self, mock_session): + # Mock out the emr_client creator + emr_client_mock = mock.MagicMock() + emr_client_mock.start_job_run.return_value = SUBMIT_JOB_SUCCESS_RETURN + emr_session_mock = mock.MagicMock() + emr_session_mock.client.return_value = emr_client_mock + mock_session.return_value = emr_session_mock + + emr_containers_job = self.emr_containers.submit_job( + name="test-job-run", + execution_role_arn="arn:aws:somerole", + release_label="emr-6.3.0-latest", + job_driver={}, + configuration_overrides={}, + client_request_token="uuidtoken", + ) + assert emr_containers_job == 'job123456' diff --git a/tests/providers/amazon/aws/operators/test_emr_containers.py b/tests/providers/amazon/aws/operators/test_emr_containers.py new file mode 100644 index 0000000000000..7e17778a33f35 --- /dev/null +++ b/tests/providers/amazon/aws/operators/test_emr_containers.py @@ -0,0 +1,102 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +from unittest import mock +from unittest.mock import MagicMock, patch + +import pytest + +from airflow import configuration +from airflow.exceptions import AirflowException +from airflow.providers.amazon.aws.hooks.emr_containers import EMRContainerHook +from airflow.providers.amazon.aws.operators.emr_containers import EMRContainerOperator + +SUBMIT_JOB_SUCCESS_RETURN = { + 'ResponseMetadata': {'HTTPStatusCode': 200}, + 'id': 'job123456', + 'virtualClusterId': 'vc1234', +} + +GENERATED_UUID = '800647a9-adda-4237-94e6-f542c85fa55b' + + +class TestEMRContainerOperator(unittest.TestCase): + @mock.patch('airflow.providers.amazon.aws.hooks.emr_containers.EMRContainerHook') + def setUp(self, emr_hook_mock): + configuration.load_test_config() + + self.emr_hook_mock = emr_hook_mock + self.emr_container = EMRContainerOperator( + task_id='start_job', + name='test_emr_job', + virtual_cluster_id='vzw123456', + execution_role_arn='arn:aws:somerole', + release_label='6.3.0-latest', + job_driver={}, + configuration_overrides={}, + poll_interval=0, + client_request_token=GENERATED_UUID, + ) + + @mock.patch.object(EMRContainerHook, 'submit_job') + @mock.patch.object(EMRContainerHook, 'check_query_status') + def test_execute_without_failure( + self, + mock_check_query_status, + mock_submit_job, + ): + mock_submit_job.return_value = "jobid_123456" + mock_check_query_status.return_value = 'COMPLETED' + + self.emr_container.execute(None) + + mock_submit_job.assert_called_once_with( + 'test_emr_job', 'arn:aws:somerole', '6.3.0-latest', {}, {}, GENERATED_UUID + ) + mock_check_query_status.assert_called_once_with('jobid_123456') + assert self.emr_container.release_label == '6.3.0-latest' + + @mock.patch.object( + EMRContainerHook, + 'check_query_status', + side_effect=['PENDING', 'PENDING', 'SUBMITTED', 'RUNNING', 'COMPLETED'], + ) + def test_execute_with_polling(self, mock_check_query_status): + # Mock out the emr_client creator + emr_client_mock = MagicMock() + emr_client_mock.start_job_run.return_value = SUBMIT_JOB_SUCCESS_RETURN + emr_session_mock = MagicMock() + emr_session_mock.client.return_value = emr_client_mock + boto3_session_mock = MagicMock(return_value=emr_session_mock) + + with patch('boto3.session.Session', boto3_session_mock): + assert self.emr_container.execute(None) == 'job123456' + assert mock_check_query_status.call_count == 5 + + @mock.patch.object(EMRContainerHook, 'submit_job') + @mock.patch.object(EMRContainerHook, 'check_query_status') + @mock.patch.object(EMRContainerHook, 'get_job_failure_reason') + def test_execute_with_failure( + self, mock_get_job_failure_reason, mock_check_query_status, mock_submit_job + ): + mock_submit_job.return_value = "jobid_123456" + mock_check_query_status.return_value = 'FAILED' + mock_get_job_failure_reason.return_value = "CLUSTER_UNAVAILABLE" + with pytest.raises(AirflowException) as ctx: + self.emr_container.execute(None) + assert 'EMR Containers job failed' in str(ctx.value) diff --git a/tests/providers/amazon/aws/sensors/test_emr_containers.py b/tests/providers/amazon/aws/sensors/test_emr_containers.py new file mode 100644 index 0000000000000..8b49854201462 --- /dev/null +++ b/tests/providers/amazon/aws/sensors/test_emr_containers.py @@ -0,0 +1,72 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +from unittest import mock + +import pytest + +from airflow.exceptions import AirflowException +from airflow.providers.amazon.aws.hooks.emr_containers import EMRContainerHook +from airflow.providers.amazon.aws.sensors.emr_containers import EMRContainerSensor + + +class TestEMRContainerSensor(unittest.TestCase): + def setUp(self): + self.sensor = EMRContainerSensor( + task_id='test_emrcontainer_sensor', + virtual_cluster_id='vzwemreks', + job_id='job1234', + poll_interval=5, + max_retries=1, + aws_conn_id='aws_default', + ) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("PENDING",)) + def test_poke_pending(self, mock_check_query_status): + assert not self.sensor.poke(None) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("SUBMITTED",)) + def test_poke_submitted(self, mock_check_query_status): + assert not self.sensor.poke(None) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("RUNNING",)) + def test_poke_running(self, mock_check_query_status): + assert not self.sensor.poke(None) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("COMPLETED",)) + def test_poke_completed(self, mock_check_query_status): + assert self.sensor.poke(None) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("FAILED",)) + def test_poke_failed(self, mock_check_query_status): + with pytest.raises(AirflowException) as ctx: + self.sensor.poke(None) + assert 'EMR Containers sensor failed' in str(ctx.value) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("CANCELLED",)) + def test_poke_cancelled(self, mock_check_query_status): + with pytest.raises(AirflowException) as ctx: + self.sensor.poke(None) + assert 'EMR Containers sensor failed' in str(ctx.value) + + @mock.patch.object(EMRContainerHook, 'check_query_status', side_effect=("CANCEL_PENDING",)) + def test_poke_cancel_pending(self, mock_check_query_status): + with pytest.raises(AirflowException) as ctx: + self.sensor.poke(None) + assert 'EMR Containers sensor failed' in str(ctx.value) From 534b6c7656bc07be6787b6881f11353cb0770899 Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Wed, 14 Jul 2021 10:58:41 -0700 Subject: [PATCH 2/8] Fixup exception messages and add an additional test for polling timeout. --- .../amazon/aws/operators/emr_containers.py | 7 ++-- .../aws/operators/test_emr_containers.py | 34 +++++++++++++++++++ 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/airflow/providers/amazon/aws/operators/emr_containers.py b/airflow/providers/amazon/aws/operators/emr_containers.py index b8a5edb394cf9..ca3c9363f3666 100644 --- a/airflow/providers/amazon/aws/operators/emr_containers.py +++ b/airflow/providers/amazon/aws/operators/emr_containers.py @@ -115,12 +115,13 @@ def execute(self, context: dict) -> Optional[str]: if query_status in EMRContainerHook.FAILURE_STATES: error_message = self.hook.get_job_failure_reason(self.job_id) raise AirflowException( - f"EMR Containers job failed. Final state is {query_status}, query_execution_id is {self.job_id}. Error: {error_message}" + f"EMR Containers job failed. Final state is {query_status}. " + f"query_execution_id is {self.job_id}. Error: {error_message}" ) elif not query_status or query_status in EMRContainerHook.INTERMEDIATE_STATES: raise AirflowException( - "Final state of EMR Containers job is {query_status}. " - "Max tries of poll status exceeded, query_execution_id is {self.job_id}." + f"Final state of EMR Containers job is {query_status}. " + f"Max tries of poll status exceeded, query_execution_id is {self.job_id}." ) return self.job_id diff --git a/tests/providers/amazon/aws/operators/test_emr_containers.py b/tests/providers/amazon/aws/operators/test_emr_containers.py index 7e17778a33f35..f7684dca83854 100644 --- a/tests/providers/amazon/aws/operators/test_emr_containers.py +++ b/tests/providers/amazon/aws/operators/test_emr_containers.py @@ -100,3 +100,37 @@ def test_execute_with_failure( with pytest.raises(AirflowException) as ctx: self.emr_container.execute(None) assert 'EMR Containers job failed' in str(ctx.value) + assert 'Error: CLUSTER_UNAVAILABLE' in str(ctx.value) + + @mock.patch.object( + EMRContainerHook, + 'check_query_status', + side_effect=['PENDING', 'PENDING', 'SUBMITTED', 'RUNNING', 'COMPLETED'], + ) + def test_execute_with_polling_timeout(self, mock_check_query_status): + # Mock out the emr_client creator + emr_client_mock = MagicMock() + emr_client_mock.start_job_run.return_value = SUBMIT_JOB_SUCCESS_RETURN + emr_session_mock = MagicMock() + emr_session_mock.client.return_value = emr_client_mock + boto3_session_mock = MagicMock(return_value=emr_session_mock) + + timeout_container = EMRContainerOperator( + task_id='start_job', + name='test_emr_job', + virtual_cluster_id='vzw123456', + execution_role_arn='arn:aws:somerole', + release_label='6.3.0-latest', + job_driver={}, + configuration_overrides={}, + poll_interval=0, + max_tries=3, + ) + + with patch('boto3.session.Session', boto3_session_mock): + with pytest.raises(AirflowException) as ctx: + timeout_container.execute(None) + + assert mock_check_query_status.call_count == 3 + assert 'Final state of EMR Containers job is SUBMITTED' in str(ctx.value) + assert 'Max tries of poll status exceeded' in str(ctx.value) From d5bacb379ef00c8d71735e41870fe407c1f44499 Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Wed, 14 Jul 2021 11:33:41 -0700 Subject: [PATCH 3/8] Remove f-string from log statements --- airflow/providers/amazon/aws/hooks/emr_containers.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/airflow/providers/amazon/aws/hooks/emr_containers.py b/airflow/providers/amazon/aws/hooks/emr_containers.py index 7d03d6f2a0744..dd65940805898 100644 --- a/airflow/providers/amazon/aws/hooks/emr_containers.py +++ b/airflow/providers/amazon/aws/hooks/emr_containers.py @@ -101,7 +101,7 @@ def submit_job( raise AirflowException(f'Start Job Run failed: {response}') else: self.log.info( - f"Start Job Run success - Job Id %s and virtual cluster id %s", + "Start Job Run success - Job Id %s and virtual cluster id %s", response['id'], response['virtualClusterId'], ) @@ -177,15 +177,11 @@ def poll_query_status( while True: query_state = self.check_query_status(job_id) if query_state is None: - self.log.info(f"Try %s: Invalid query state. Retrying again", try_number) + self.log.info("Try %s: Invalid query state. Retrying again", try_number) elif query_state in self.INTERMEDIATE_STATES: - self.log.info( - f"Try %s: Query is still in an intermediate state - %s", try_number, query_state - ) + self.log.info("Try %s: Query is still in an intermediate state - %s", try_number, query_state) else: - self.log.info( - f"Try %s: Query execution completed. Final state is %s", try_number, query_state - ) + self.log.info("Try %s: Query execution completed. Final state is %s", try_number, query_state) final_query_state = query_state break if max_tries and try_number >= max_tries: # Break loop if max_tries reached From 1c4091d303d595eb24bbe92a7201ba4d3e6d299e Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Thu, 22 Jul 2021 16:23:27 -0700 Subject: [PATCH 4/8] Update documentation and example --- .../aws/example_dags/example_emr_eks_job.py | 12 +++++- .../operators/emr_eks.rst | 38 +++++++++---------- 2 files changed, 28 insertions(+), 22 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py b/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py index aa935a5a14694..99e33e083f33c 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py @@ -39,12 +39,20 @@ } CONFIGURATION_OVERRIDES_ARG = { + "applicationConfiguration": [ + { + "classification": "spark-defaults", + "properties": { + "spark.hadoop.hive.metastore.client.factory.class": "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory", + }, + } + ], "monitoringConfiguration": { "cloudWatchMonitoringConfiguration": { "logGroupName": "/aws/emr-eks-spark", "logStreamNamePrefix": "airflow", } - } + }, } # [END howto_operator_emr_eks_config] @@ -60,6 +68,7 @@ # VIRTUAL_CLUSTER_ID = '{{ conn.emr_eks.extra_dejson["virtual_cluster_id"] }}' # JOB_ROLE_ARN = '{{ conn.emr_eks.extra_dejson["job_role_arn"] }}' + # [START howto_operator_emr_eks_jobrun] job_starter = EMRContainerOperator( task_id="start_job", virtual_cluster_id=VIRTUAL_CLUSTER_ID, @@ -69,3 +78,4 @@ configuration_overrides=CONFIGURATION_OVERRIDES_ARG, name="pi.py", ) + # [END howto_operator_emr_eks_jobrun] diff --git a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst index c6eb05d5c0fc7..09e98abef48bb 100644 --- a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst +++ b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst @@ -21,6 +21,10 @@ Amazon EMR on EKS Operators =========================== +`Amazon EMR on EKS `__ provides a deployment option for Amazon EMR that allows you to run open-source big data frameworks on Amazon Elastic Kubernetes Service (Amazon EKS). + +Airflow provides the :class:`~airflow.providers.amazon.aws.operators.emr_containers.EMRContainerOperator` to submit Spark jobs to your EMR on EKS virtual cluster. + .. contents:: :depth: 1 :local: @@ -30,12 +34,7 @@ Prerequisite Tasks .. include:: _partials/prerequisite_tasks.rst -Overview --------- - -Airflow to Amazon EMR on Amazon EKS integration provides a way to run Apache Spark jobs on Kubernetes. - -- :class:`~airflow.providers.amazon.aws.operators.emr_containers.EMRContainerOperator` +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. Create EMR on EKS job with sample script @@ -47,33 +46,30 @@ Purpose This example dag ``example_emr_eks_job.py`` uses ``EMRContainerOperator`` to create a new EMR on EKS job calculating the mathematical constant ``Pi``, and monitors the progress with ``EMRContainerSensor``. -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. - -Environment variables -""""""""""""""""""""" - -This example relies on the following variables, which can be passed via OS environment variables. - -.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py - :language: python - :start-after: [START howto_operator_emr_eks_env_variables] - :end-before: [END howto_operator_emr_eks_env_variables] - Job configuration """"""""""""""""" -To create a job for EMR on EKS, you need to specify your job configuration, any monitoring configuration, and a few other details for the job. +To create a job for EMR on EKS, you need to specify your virtual cluster ID, the release of 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 Cloudwatch. + +In the example, we show how to use use the AWS Glue data catalog and send logs to the ``/aws/emr-eks-spark`` log group in CloudWatch. .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py :language: python :start-after: [START howto_operator_emr_eks_config] :end-before: [END howto_operator_emr_eks_config] -With EMR on EKS, you specify your Spark configuration, the EMR release you want to use, the IAM role to use for the job, and the EMR virtual cluster ID. We pass the ``virtual_cluster_id`` and ``execution_role_arn`` values as operator parameters, but you can store them in a Connection or provide them in the DAG. -With the EMRContainerOperator, it will wait until the successful completion of the job or raise an ``AirflowException`` if there is an error. +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_emr_eks_jobrun] + :end-before: [END howto_operator_emr_eks_jobrun] + +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 --------- From 5eb6cd453de54c040e555857a968e2e65ca7afbb Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Thu, 22 Jul 2021 22:33:18 -0700 Subject: [PATCH 5/8] Address flake8 error --- .../providers/amazon/aws/example_dags/example_emr_eks_job.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py b/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py index 99e33e083f33c..76e848558ff54 100644 --- a/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py +++ b/airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py @@ -43,7 +43,7 @@ { "classification": "spark-defaults", "properties": { - "spark.hadoop.hive.metastore.client.factory.class": "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory", + "spark.hadoop.hive.metastore.client.factory.class": "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory", # noqa: E501 }, } ], From 5ff432631615cb8182f8484877695e60244fe752 Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Fri, 23 Jul 2021 14:46:55 -0700 Subject: [PATCH 6/8] Trailing space :) --- docs/apache-airflow-providers-amazon/operators/emr_eks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst index 09e98abef48bb..b9ad24bdf06f5 100644 --- a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst +++ b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst @@ -21,7 +21,7 @@ Amazon EMR on EKS Operators =========================== -`Amazon EMR on EKS `__ provides a deployment option for Amazon EMR that allows you to run open-source big data frameworks on Amazon Elastic Kubernetes Service (Amazon EKS). +`Amazon EMR on EKS `__ provides a deployment option for Amazon EMR that allows you to run open-source big data frameworks on Amazon Elastic Kubernetes Service (Amazon EKS). Airflow provides the :class:`~airflow.providers.amazon.aws.operators.emr_containers.EMRContainerOperator` to submit Spark jobs to your EMR on EKS virtual cluster. From ff678417edabeb23fd4d53c635a148fd69b0321c Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Fri, 20 Aug 2021 10:07:12 -0700 Subject: [PATCH 7/8] Documentation update --- .../operators/emr_eks.rst | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst index b9ad24bdf06f5..4c8ebc20a0dff 100644 --- a/docs/apache-airflow-providers-amazon/operators/emr_eks.rst +++ b/docs/apache-airflow-providers-amazon/operators/emr_eks.rst @@ -37,14 +37,13 @@ Prerequisite Tasks 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. -Create EMR on EKS job with sample script ----------------------------------------- +Run a Spark job on EMR on EKS +----------------------------- Purpose """"""" -This example dag ``example_emr_eks_job.py`` uses ``EMRContainerOperator`` to create a new EMR on EKS job calculating the mathematical constant ``Pi``, and monitors the progress -with ``EMRContainerSensor``. +The ``EMRContainerOperator`` will submit a new job to an EMR on EKS virtual cluster and wait for the job to complete. The example job below calculates the mathematical constant ``Pi``, and monitors the progress with ``EMRContainerSensor``. In a production job, you would usually refer to a Spark script on Amazon S3. Job configuration """"""""""""""""" @@ -53,7 +52,7 @@ To create a job for EMR on EKS, you need to specify your virtual cluster ID, the 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 Cloudwatch. -In the example, we show how to use use the AWS Glue data catalog and send logs to the ``/aws/emr-eks-spark`` log group in 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. Refer to the `EMR on EKS guide `__ for more details on job configuration. .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py :language: python @@ -61,7 +60,7 @@ In the example, we show how to use use the AWS Glue data catalog and send logs t :end-before: [END howto_operator_emr_eks_config] -We pass the ``virtual_cluster_id`` and ``execution_role_arn`` values as operator parameters, but you can store them in a Connection or provide them in the DAG. +We pass the ``virtual_cluster_id`` and ``execution_role_arn`` values as operator parameters, but you can store them in a connection or provide them in the DAG. Your AWS region should be defined either in the ``aws_default`` connection as ``{"region_name": "us-east-1"}`` or a custom connection name that gets passed to the operator with the ``aws_conn_id`` parameter. .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_emr_eks_job.py :language: python From 3e31d6fb7c3dd86798ca6d969d4c878fddf2055a Mon Sep 17 00:00:00 2001 From: "Damon P. Cortesi" Date: Fri, 27 Aug 2021 08:53:35 -0700 Subject: [PATCH 8/8] Trigger Build