-
Notifications
You must be signed in to change notification settings - Fork 16.4k
Add an Amazon EMR on EKS provider package #16766
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
08b5bfa
534b6c7
d5bacb3
1c4091d
5eb6cd4
5ff4326
ff67841
cef0f25
3e31d6f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,81 @@ | ||
| # 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 = { | ||
| "applicationConfiguration": [ | ||
| { | ||
| "classification": "spark-defaults", | ||
| "properties": { | ||
| "spark.hadoop.hive.metastore.client.factory.class": "com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory", # noqa: E501 | ||
| }, | ||
| } | ||
| ], | ||
| "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"] }}' | ||
|
|
||
| # [START howto_operator_emr_eks_jobrun] | ||
| 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", | ||
| ) | ||
| # [END howto_operator_emr_eks_jobrun] |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,205 @@ | ||
| # 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( | ||
| "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("Try %s: Invalid query state. Retrying again", try_number) | ||
| elif query_state in self.INTERMEDIATE_STATES: | ||
| self.log.info("Try %s: Query is still in an intermediate state - %s", try_number, query_state) | ||
| else: | ||
| 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 | ||
| 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 | ||
dacort marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| :type job_id: str | ||
| :return: dict | ||
| """ | ||
| return self.conn.cancel_job_run( | ||
| virtualClusterId=self.virtual_cluster_id, | ||
| id=job_id, | ||
| ) | ||
Uh oh!
There was an error while loading. Please reload this page.