|
| 1 | +--- |
| 2 | +title: "Integrating AWS EMR Serverless with Dagster Pipes | Dagster Docs" |
| 3 | +description: "Learn to integrate Dagster Pipes with AWS EMR Serverless to launch external code from Dagster assets." |
| 4 | +--- |
| 5 | + |
| 6 | +# AWS EMR Serverless & Dagster Pipes |
| 7 | + |
| 8 | +This tutorial gives a short overview on how to use [Dagster Pipes](/concepts/dagster-pipes) with [AWS EMR Serverless](https://aws.amazon.com/emr-serverless/). |
| 9 | + |
| 10 | +The [dagster-aws](/\_apidocs/libraries/dagster-aws) integration library provides the <PyObject object="PipesEMRServerlessClient" module="dagster_aws.pipes" /> resource, which can be used to launch AWS EMR Serverless jobs from Dagster assets and ops. Dagster can receive regular events such as logs, asset checks, or asset materializations from jobs launched with this client. Using it requires minimal code changes to your EMR jobs. |
| 11 | + |
| 12 | +--- |
| 13 | + |
| 14 | +## Prerequisites |
| 15 | + |
| 16 | +- **In the orchestration environment**, you'll need to: |
| 17 | + |
| 18 | + - Install the following packages: |
| 19 | + |
| 20 | + ```shell |
| 21 | + pip install dagster dagster-webserver dagster-aws |
| 22 | + ``` |
| 23 | + |
| 24 | + Refer to the [Dagster installation guide](/getting-started/install) for more info. |
| 25 | + |
| 26 | + - **AWS authentication credentials configured.** If you don't have this set up already, refer to the [boto3 quickstart](https://boto3.amazonaws.com/v1/documentation/api/latest/guide/quickstart.html). |
| 27 | + |
| 28 | +- **In AWS**: |
| 29 | + |
| 30 | + - An existing AWS account |
| 31 | + - An AWS EMR Serverless job. AWS CloudWatch logging has to be enabled in order to receive logs from the job: |
| 32 | + |
| 33 | + ```json |
| 34 | + { |
| 35 | + "monitoringConfiguration": { |
| 36 | + "cloudWatchLoggingConfiguration": { "enabled": true } |
| 37 | + } |
| 38 | + } |
| 39 | + ``` |
| 40 | + |
| 41 | +--- |
| 42 | + |
| 43 | +## Step 1: Install the dagster-pipes module |
| 44 | + |
| 45 | +There are a [few options](https://docs.aws.amazon.com/emr/latest/EMR-Serverless-UserGuide/using-python-libraries.html) available for shipping Python packages to a PySpark job. For example, [install it in your Docker image](https://docs.aws.amazon.com/emr/latest/EMR-Serverless-UserGuide/application-custom-image.html): |
| 46 | + |
| 47 | +Install the `dagster-pipes` module in the image used for your EMR job. For example, you can install the dependency with `pip` in your image Dockerfile: |
| 48 | + |
| 49 | +```Dockerfile |
| 50 | +# start from EMR image |
| 51 | +FROM public.ecr.aws/emr-serverless/spark/emr-7.2.0:latest |
| 52 | + |
| 53 | +USER root |
| 54 | + |
| 55 | +RUN python -m pip install dagster-pipes |
| 56 | + |
| 57 | +# copy the job script |
| 58 | +COPY . . |
| 59 | + |
| 60 | +USER hadoop |
| 61 | +``` |
| 62 | + |
| 63 | +--- |
| 64 | + |
| 65 | +## Step 2: Add dagster-pipes to the EMR Serverless job script |
| 66 | + |
| 67 | +Call `open_dagster_pipes` in the EMR Serverless script to create a context that can be used to send messages to Dagster: |
| 68 | + |
| 69 | +```python file=/guides/dagster/dagster_pipes/emr-serverless/script.py |
| 70 | +from dagster_pipes import open_dagster_pipes |
| 71 | +from pyspark.sql import SparkSession |
| 72 | + |
| 73 | + |
| 74 | +def main(): |
| 75 | + with open_dagster_pipes() as pipes: |
| 76 | + pipes.log.info("Hello from AWS EMR Serverless!") |
| 77 | + |
| 78 | + spark = SparkSession.builder.appName("HelloWorld").getOrCreate() |
| 79 | + |
| 80 | + df = spark.createDataFrame( |
| 81 | + [(1, "Alice", 34), (2, "Bob", 45), (3, "Charlie", 56)], |
| 82 | + ["id", "name", "age"], |
| 83 | + ) |
| 84 | + |
| 85 | + # calculate a really important statistic |
| 86 | + avg_age = float(df.agg({"age": "avg"}).collect()[0][0]) |
| 87 | + |
| 88 | + # attach it to the asset materialization in Dagster |
| 89 | + pipes.report_asset_materialization( |
| 90 | + metadata={"average_age": {"raw_value": avg_age, "type": "float"}}, |
| 91 | + data_version="alpha", |
| 92 | + ) |
| 93 | + |
| 94 | + spark.stop() |
| 95 | + |
| 96 | + |
| 97 | +if __name__ == "__main__": |
| 98 | + main() |
| 99 | +``` |
| 100 | + |
| 101 | +--- |
| 102 | + |
| 103 | +## Step 3: Create an asset using the PipesEMRServerlessClient to launch the job |
| 104 | + |
| 105 | +In the Dagster asset/op code, use the `PipesEMRServerlessClient` resource to launch the job: |
| 106 | + |
| 107 | +```python file=/guides/dagster/dagster_pipes/emr-serverless/dagster_code.py startafter=start_asset_marker endbefore=end_asset_marker |
| 108 | +import os |
| 109 | + |
| 110 | +import boto3 |
| 111 | +from dagster_aws.pipes import PipesEMRServerlessClient |
| 112 | + |
| 113 | +from dagster import AssetExecutionContext, asset |
| 114 | + |
| 115 | + |
| 116 | +@asset |
| 117 | +def emr_serverless_asset( |
| 118 | + context: AssetExecutionContext, |
| 119 | + pipes_emr_serverless_client: PipesEMRServerlessClient, |
| 120 | +): |
| 121 | + return pipes_emr_serverless_client.run( |
| 122 | + context=context, |
| 123 | + start_job_run_params={ |
| 124 | + "applicationId": "<app-id>", |
| 125 | + "executionRoleArn": "<emr-role>", |
| 126 | + "clientToken": context.run_id, # idempotency identifier for the job run |
| 127 | + "configurationOverrides": { |
| 128 | + "monitoringConfiguration": { |
| 129 | + "cloudWatchLoggingConfiguration": {"enabled": True} |
| 130 | + } |
| 131 | + }, |
| 132 | + }, |
| 133 | + ).get_results() |
| 134 | +``` |
| 135 | + |
| 136 | +This will launch the AWS EMR Serverless job and wait for it completion. If the job fails, the Dagster process will raise an exception. If the Dagster process is interrupted while the job is still running, the job will be terminated. |
| 137 | + |
| 138 | +--- |
| 139 | + |
| 140 | +## Step 4: Create Dagster definitions |
| 141 | + |
| 142 | +Next, add the `PipesEMRServerlessClient` resource to your project's <PyObject object="Definitions" /> object: |
| 143 | + |
| 144 | +```python file=/guides/dagster/dagster_pipes/emr-serverless/dagster_code.py startafter=start_definitions_marker endbefore=end_definitions_marker |
| 145 | +from dagster import Definitions # noqa |
| 146 | + |
| 147 | + |
| 148 | +defs = Definitions( |
| 149 | + assets=[emr_serverless_asset], |
| 150 | + resources={"pipes_emr_serverless_client": PipesEMRServerlessClient()}, |
| 151 | +) |
| 152 | +``` |
| 153 | + |
| 154 | +Dagster will now be able to launch the AWS EMR Serverless task from the `emr_serverless_asset` asset, and receive logs and events from the job. If using the default `message_reader` `PipesCloudwatchLogReader`, driver logs will be forwarded to the Dagster process. |
| 155 | + |
| 156 | +--- |
| 157 | + |
| 158 | +## Related |
| 159 | + |
| 160 | +<ArticleList> |
| 161 | + <ArticleListItem |
| 162 | + title="Dagster Pipes" |
| 163 | + href="/concepts/dagster-pipes" |
| 164 | + ></ArticleListItem> |
| 165 | + <ArticleListItem |
| 166 | + title="AWS EMR Serverless Pipes API reference" |
| 167 | + href="/_apidocs/libraries/dagster-aws#dagster_aws.pipes.PipesEMRServerlessClient" |
| 168 | + ></ArticleListItem> |
| 169 | +</ArticleList> |
0 commit comments