Skip to content

EcsRunTaskOperator leaks ECS task on failure with partial IAM permissions #61050

@SameerMesiah97

Description

@SameerMesiah97

Apache Airflow Provider(s)

amazon

Versions of Apache Airflow Providers

apache-airflow-providers-amazon==9.20.0

Apache Airflow version

main

Operating System

Debian GNU/Linux 12 (bookworm)

Deployment

Other

Deployment details

No response

What happened

When using EcsRunTaskOperator, an ECS task may be successfully started even when the task execution role has partial ECS permissions, for example lacking ecs:DescribeTasks.

In this scenario, the operator successfully calls RunTask and starts an ECS task. However, subsequent calls (such as waiting for task completion when wait_for_completion=True) fail due to insufficient permissions. The task then fails, but the ECS task continues running in AWS, resulting in leaked infrastructure.

This can occur, for example, when the execution role allows ecs:RunTask but denies ecs:DescribeTasks, which is required by the waiter used to monitor task completion.

What you think should happen instead

If the operator fails after successfully starting an ECS task (for example due to missing DescribeTasks or other follow-up permissions), it should make a best-effort attempt to clean up the partially created resource by stopping the ECS task.

Cleanup should be attempted opportunistically (i.e. only if the task ARN is known and the necessary permissions are available), and failure to clean up should not mask or replace the original exception.

How to reproduce

  1. Create an IAM role that allows ecs:RunTask but denies ecs:DescribeTasks.

  2. Configure an AWS connection in Airflow using this role.
    (The connection ID aws_test_conn is used for this reproduction.)

  3. Create an ECS cluster.
    (The cluster name airflow-partial-auth-cluster is used for this reproduction.)

  4. Create an ECS task definition with at least one container.
    (The task definition my-task-def with container my-task-def-container is used for this reproduction.)

  5. Use the following DAG:

from datetime import datetime

from airflow import DAG
from airflow.providers.amazon.aws.operators.ecs import EcsRunTaskOperator


with DAG(
    dag_id="ecs_partial_auth_leak_repro",
    start_date=datetime(2025, 1, 1),
    schedule=None,
    catchup=False,
) as dag:
    run_task = EcsRunTaskOperator(
        task_id="run_task",
        aws_conn_id="aws_test_conn",
        cluster="airflow-partial-auth-cluster",
        task_definition="my-task-def:1",
        launch_type="FARGATE",
        overrides={
            "containerOverrides": [
                {
                    "name": "my-task-def-container",
                }
            ]
        },
        network_configuration={
            "awsvpcConfiguration": {
                "subnets": ["subnet-xxxxxxxxxxxxxxxxx"],
                "assignPublicIp": "ENABLED",
            }
        },
        wait_for_completion=True,  # triggers DescribeTasks via waiter
    )
  1. Trigger the DAG.

Expected Result

The task fails due to missing ecs:DescribeTasks permissions, but the ECS task continues running in AWS and is not stopped automatically.

Anything else

This behavior can be surprising and potentially costly, as infrastructure is created even though the Airflow task fails. Other Airflow operators that manage external resources already attempt best-effort cleanup on failure to avoid leaking infrastructure.

In particular, this pattern has recently been introduced for both EC2 and EMR operators. The EC2 change has already been merged (PR #60904), and a corresponding EMR change has been proposed (PR #61010). Given this precedent, similar best-effort cleanup behavior would be expected for ECS operators as well.

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions