Skip to content

Commit

Permalink
Avoid launching ECS runs with large overrides (#8152)
Browse files Browse the repository at this point in the history
Per the AWS docs:

> A total of 8192 characters are allowed for overrides. This limit includes
> the JSON formatting characters of the override structure.

https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_RunTask.html

This is a server-side restriction and consequently isn't enforced via
botocore's stubber. This commit extends our existing ECS stub so we can
unit test against this restriction.

* Avoid launching ECS runs with large overrides

When users have very large container contexts (for example, if they set
many secrets), then we can run up against ECS's run task limits on
request size.

We don't actually need the container context to launch the run; only to
create its task definition. So this strips the container context out of
the command that gets passed to `dagster api execute_run` to keep our
override commands smaller.
  • Loading branch information
jmsanders committed Jun 1, 2022
1 parent 992e271 commit f2beff3
Show file tree
Hide file tree
Showing 4 changed files with 62 additions and 1 deletion.
16 changes: 15 additions & 1 deletion python_modules/libraries/dagster-aws/dagster_aws/ecs/launcher.py
Original file line number Diff line number Diff line change
Expand Up @@ -197,8 +197,22 @@ def launch_run(self, context: LaunchRunContext) -> None:
"family"
]

# ECS limits overrides to 8192 characters including json formatting
# https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_RunTask.html
# When container_context is serialized as part of the ExecuteRunArgs, we risk
# going over this limit (for example, if many secrets have been set). This strips
# the container context off of our pipeline origin because we don't actually need
# it to launch the run; we only needed it to create the task definition.
repository_origin = pipeline_origin.repository_origin
# pylint: disable=protected-access
stripped_repository_origin = repository_origin._replace(container_context={})
stripped_pipeline_origin = pipeline_origin._replace(
repository_origin=stripped_repository_origin
)
# pylint: enable=protected-access

args = ExecuteRunArgs(
pipeline_origin=pipeline_origin,
pipeline_origin=stripped_pipeline_origin,
pipeline_run_id=run.run_id,
instance_ref=self._instance.get_ref(),
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,8 +10,10 @@
from dagster_aws.ecs.tasks import TaskMetadata

from dagster._check import CheckError
from dagster.core.code_pointer import FileCodePointer
from dagster.core.events import MetadataEntry
from dagster.core.launcher.base import WorkerStatus
from dagster.core.origin import PipelinePythonOrigin, RepositoryPythonOrigin


@pytest.mark.parametrize("task_long_arn_format", ["enabled", "disabled"])
Expand Down Expand Up @@ -415,3 +417,34 @@ def test_status(ecs, instance, workspace, run):

task["lastStatus"] = "foo"
assert instance.run_launcher.check_run_worker_health(run).status == WorkerStatus.UNKNOWN


def test_overrides_too_long(
instance,
workspace,
pipeline,
external_pipeline,
):

large_container_context = {i: "boom" for i in range(10000)}

mock_pipeline_code_origin = PipelinePythonOrigin(
pipeline_name="test",
repository_origin=RepositoryPythonOrigin(
executable_path="/",
code_pointer=FileCodePointer(
python_file="foo.py",
fn_name="foo",
),
container_image="test:latest",
container_context=large_container_context,
),
)

run = instance.create_run_for_pipeline(
pipeline,
external_pipeline_origin=external_pipeline.get_external_origin(),
pipeline_code_origin=mock_pipeline_code_origin,
)

instance.launch_run(run.run_id, workspace)
Original file line number Diff line number Diff line change
Expand Up @@ -342,6 +342,11 @@ def run_task(self, **kwargs):
raise StubbedEcsError

overrides = kwargs.get("overrides", {})
# overrides is limited to 8192 characters including json formatting
# https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_RunTask.html
if len(str(overrides)) > 8192:
self.stubber.add_client_error(method="run_task", expected_params={**kwargs})

cpu = overrides.get("cpu") or task_definition.get("cpu")
memory = overrides.get("memory") or task_definition.get("memory")
if not self._valid_cpu_and_memory(cpu=cpu, memory=memory):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -371,6 +371,15 @@ def test_run_task(ecs, ec2, subnet):
assert response["tasks"][0]["overrides"]["cpu"] == "512"
assert response["tasks"][0]["overrides"]["memory"] == "1024"

# With very long overrides
with pytest.raises(Exception):
ecs.run_task(
taskDefinition="container",
# overrides is limited to 8192 characters including json formatting
# https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_RunTask.html
overrides={"containerOverrides": ["boom" for i in range(10000)]},
)


def test_stop_task(ecs):
with pytest.raises(ClientError):
Expand Down

0 comments on commit f2beff3

Please sign in to comment.