Skip to content

Commit

Permalink
[dagster-aws] [docs] add documentation for ECS Pipes
Browse files Browse the repository at this point in the history
  • Loading branch information
danielgafni committed Aug 16, 2024
1 parent 76314fe commit a812f06
Show file tree
Hide file tree
Showing 4 changed files with 93 additions and 0 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
FROM python:3.11-slim

RUN python -m pip install --no-cache dagster-pipes boto3

# TODO: replace this with `pip install dagster-pipes` once the docs are finished
RUN pip install --no-cache-dir dagster-pipes boto3

WORKDIR /app

COPY task.py .
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
# start_asset_marker
import os

# dagster_glue_pipes.py
import boto3
from dagster_aws.pipes import PipesECSClient
from docutils.nodes import entry

from dagster import AssetExecutionContext, asset



@asset
def ecs_pipes_asset(context: AssetExecutionContext, pipes_ecs_client: PipesECSClient):
return pipes_ecs_client.run(
context=context,
task_definition="dagster-pipes",
launch_type="FARGATE",
network_configuration={
"awsvpcConfiguration": {
"subnets": [
"subnet-a96acdc0"
],
"securityGroups": ["sg-028d32553728591b2"],
"assignPublicIp": "ENABLED",
}
},
).get_materialize_result()


# end_asset_marker

# start_definitions_marker

from dagster import Definitions # noqa
from dagster_aws.pipes import PipesS3MessageReader


defs = Definitions(
assets=[ecs_pipes_asset],
resources={
"pipes_ecs_client": PipesECSClient(
client=boto3.client("ecs"),
)
},
)

# end_definitions_marker
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
# this Dockerfile can be used for ECS Pipes development

FROM python:3.11-slim

RUN python -m pip install --no-cache dagster-pipes boto3

RUN --mount=type=cache,target=/root/.cache/pip pip install boto3

COPY python_modules/dagster-pipes /src/dagster-pipes

RUN pip install -e /src/dagster-pipes

WORKDIR /app
COPY examples/docs_snippets/docs_snippets/guides/dagster/dagster_pipes/ecs/task.py .
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
import boto3
from dagster_pipes import (
PipesEnvVarParamsLoader,
PipesS3ContextLoader,
open_dagster_pipes,
)

client = boto3.client("s3")


def main():
with open_dagster_pipes() as pipes:
pipes.log.info("Hello from AWS ECS task!")
pipes.report_asset_materialization(
metadata={"some_metric": {"raw_value": 0, "type": "int"}},
data_version="alpha",
)


if __name__ == "__main__":
main()

0 comments on commit a812f06

Please sign in to comment.