Skip to content

Docker Image Build Issue: Outdated Docker SDK Causes ECS Fargate Pull Failures #19337

@johan-octonomy

Description

@johan-octonomy

Summary

Prefect's DockerImage class uses the legacy Docker Python SDK client.api.build() method which cannot disable Docker buildx attestations (--provenance and --sbom). This causes ECS Fargate to fail pulling images built on Apple Silicon machines with the error: CannotPullContainerError: wrong diff id calculated on extraction

This blocks the primary deployment workflow for Apple Silicon developers targeting serverless platforms.


Environment

  • Prefect Version: 3.5.0
  • Docker Python SDK: 7.1.0
  • Docker Desktop: 26.0+ (buildx enabled by default since v23.0)
  • Build Machine: Apple Silicon (ARM64) macOS
  • Target Platform: AWS ECS Fargate (AMD64)

Root Cause

Location: src/prefect/utilities/dockerutils.py:174

events = client.api.build(
    path=str(context),
    tag=tag,
    dockerfile=dockerfile,
    pull=pull,
    decode=True,
    labels=IMAGE_LABELS,
    platform=platform,
    **kwargs,
)

Problem:

  1. Since Docker v23.0, buildx is the default builder with attestations (--provenance and --sbom) enabled
  2. These attestation layers corrupt image manifests during cross-platform builds (ARM → AMD64)
  3. The Docker Python SDK's client.api.build() only supports legacy builder parameters
  4. There is no way to pass --provenance=false or --sbom=false through the Python SDK

Docker CLI equivalent (what works):

docker buildx build --provenance=false --sbom=false --platform linux/amd64 -t image:tag .

Prefect's current approach (what fails):

DockerImage(
    name="image",
    tag="tag",
    platform="linux/amd64",
    # No way to disable attestations ❌
)

Impact

Severity: 🔴 CRITICAL - Blocks primary deployment workflow

Affected Users:

  • All Apple Silicon developers deploying to:
    • AWS ECS Fargate
    • Google Cloud Run
    • Azure Container Instances
    • Any AMD64 serverless platform

Error Manifestation:

  1. Image builds successfully on Apple Silicon
  2. Image pushes successfully to ECR/GCR/ACR
  3. ECS/Cloud Run/ACI fails to start container with:
    CannotPullContainerError: Error response from daemon: 
    failed to register layer: wrong diff id calculated on extraction
    

Workaround Required: Users must use shell scripts instead of Python:

#!/bin/bash
# Manual workaround - defeats purpose of Python-based deployments
docker buildx build \
  --provenance=false \
  --sbom=false \
  --platform linux/amd64 \
  -t "$IMAGE" .
docker push "$IMAGE"

Then reference the pre-built image:

DockerImage(
    name="pre-built-image",
    tag="manual-tag",
    build=False,  # Can't use Prefect's build
    push=False
)

Reproduction Steps

  1. On Apple Silicon Mac:

    from prefect import flow
    from prefect.docker import DockerImage
    
    @flow
    def hello():
        print("Hello from ECS")
    
    if __name__ == "__main__":
        hello.deploy(
            name="ecs-deploy",
            work_pool_name="my-ecs-pool",
            image=DockerImage(
                name="123456789.dkr.ecr.us-east-1.amazonaws.com/test",
                tag="v1",
                platform="linux/amd64"
            )
        )
  2. Deploy flow → Image builds and pushes successfully

  3. Trigger flow on ECSTask fails with:

    CannotPullContainerError: failed to register layer: 
    wrong diff id calculated on extraction
    
  4. Check stopped ECS tasks → All show TaskFailedToStart status code


Proposed Solutions

Option 1: Upgrade to Docker BuildKit APIRECOMMENDED

Change: Replace client.api.build() with Docker BuildKit API

Benefits:

  • Native buildx support
  • Access to modern build features
  • Future-proof implementation

Implementation:

# In dockerutils.py
from docker.types import BuildArgs

def build_image(
    context: Path,
    dockerfile: str = "Dockerfile",
    platform: Optional[str] = None,
    provenance: bool = True,  # NEW PARAM
    sbom: bool = True,        # NEW PARAM
    **kwargs: Any,
) -> str:
    with docker_client() as client:
        # Use BuildKit API instead of legacy client.api.build()
        build_args = BuildArgs(
            path=str(context),
            dockerfile=dockerfile,
            platform=platform,
            provenance=provenance,
            sbom=sbom,
            **kwargs
        )
        
        image_id = None
        for event in client.images.build(**build_args):
            # Process build events...
            pass
        
        return image_id

Signature Change:

# DockerImage class
def __init__(
    self,
    name: str,
    tag: Optional[str] = None,
    dockerfile: str = "auto",
    provenance: bool = True,  # NEW - default True for backward compat
    sbom: bool = True,        # NEW - default True for backward compat
    **build_kwargs: Any,
):

Option 2: Shell Out to docker buildx CLI

Change: Execute docker buildx as subprocess when cross-platform build detected

Benefits:

  • Immediate fix
  • Minimal code changes
  • Works with existing Docker installations

Implementation:

def build_image(
    context: Path,
    dockerfile: str = "Dockerfile",
    platform: Optional[str] = None,
    provenance: bool = True,
    sbom: bool = True,
    **kwargs: Any,
) -> str:
    import subprocess
    import platform as sys_platform
    
    # Detect cross-platform build
    host_arch = sys_platform.machine()
    is_cross_platform = (
        platform and 
        ("arm" in host_arch.lower() and "amd64" in platform.lower())
    )
    
    # Use docker buildx for cross-platform builds
    if is_cross_platform and not provenance and not sbom:
        cmd = [
            "docker", "buildx", "build",
            "--provenance=false",
            "--sbom=false",
            f"--platform={platform}",
            f"--tag={tag}",
            str(context)
        ]
        
        result = subprocess.run(cmd, capture_output=True, text=True)
        if result.returncode != 0:
            raise BuildError(result.stderr)
        
        # Extract image ID from output
        return parse_image_id(result.stdout)
    
    # Fall back to legacy API for non-cross-platform builds
    return legacy_build_image(context, **kwargs)

Option 3: Add Configuration Override

Change: Provide environment variable or config option to disable attestations globally

Benefits:

  • Quick fix for affected users
  • No API changes required
  • Can be documented immediately

Implementation:

# In dockerutils.py
import os

def build_image(context: Path, **kwargs) -> str:
    # Check environment variable
    disable_attestations = os.getenv("PREFECT_DOCKER_DISABLE_ATTESTATIONS", "false").lower() == "true"
    
    if disable_attestations:
        # Build with docker buildx CLI to disable attestations
        return build_with_buildx_cli(context, provenance=False, sbom=False, **kwargs)
    else:
        # Use legacy client.api.build()
        return legacy_build_image(context, **kwargs)

Usage:

export PREFECT_DOCKER_DISABLE_ATTESTATIONS=true
python deploy.py

Testing Checklist

When implementing the fix, please test:

  • Apple Silicon → ECS Fargate: ARM64 Mac → AMD64 ECS (primary use case)
  • Apple Silicon → Cloud Run: ARM64 Mac → AMD64 GCR
  • Apple Silicon → ACI: ARM64 Mac → AMD64 Azure Container Instances
  • AMD64 → ECS: x86_64 Linux → AMD64 ECS (ensure no regression)
  • Native builds: ARM64 → ARM64 (ensure attestations still work when desired)
  • Backward compatibility: Existing deployments without provenance/sbom params
  • prefect.yaml builds: Ensure prefect_docker.deployments.steps.build_docker_image works
  • Python-based builds: Ensure DockerImage.build() works with new params

References

  1. Docker Buildx Attestations Documentation:

  2. AWS ECS Issue with Attestations:

  3. Docker Python SDK Limitations:

  4. Prefect Source Files:

    • src/prefect/utilities/dockerutils.py:174 (build_image function)
    • src/prefect/docker/docker_image.py:64 (DockerImage.build method)
    • src/integrations/prefect-docker/prefect_docker/deployments/steps.py:261 (build_docker_image step)

Priority Justification

Why CRITICAL:

  1. Blocks primary deployment path for fastest-growing developer segment (Apple Silicon)
  2. No Python-based workaround - requires shell scripts, defeating Prefect's Python-first philosophy
  3. Silent failure mode - builds succeed, deployments fail at runtime
  4. Affects multiple cloud providers - ECS, Cloud Run, ACI all impacted
  5. Modern Docker versions - All Docker Desktop 23.0+ users affected (released 2023)

Current Workaround (For Affected Users)

Until this is fixed, Apple Silicon users must:

  1. Create manual build script (build_and_push.sh):

    #!/bin/bash
    IMAGE="${1:-my-registry/my-image:latest}"
    
    docker buildx build \
      --provenance=false \
      --sbom=false \
      --platform linux/amd64 \
      -t "$IMAGE" \
      --push \
      .
  2. Run build manually before deployment:

    ./build_and_push.sh my-registry/my-image:v1.0.0
  3. Reference pre-built image in Python:

    from prefect.docker import DockerImage
    
    image = DockerImage(
        name="my-registry/my-image",
        tag="v1.0.0",
        build=False,  # Don't rebuild
        push=False    # Already pushed
    )
    
    flow.deploy(
        name="my-deployment",
        work_pool_name="my-ecs-pool",
        image=image
    )

Related Issues

  • Please link this to any existing issues about:
    • Apple Silicon deployment failures
    • ECS container pull errors
    • Docker buildx compatibility
    • Cross-platform build problems

Additional Context

Discovered during: Full ECS Fargate deployment setup on Apple Silicon Mac
Verified on: Prefect 3.5.0, Docker Desktop 26.0.0, macOS 14.x Sonoma
Confirmed working workaround: Shell script with docker buildx --provenance=false --sbom=false
Ideal user experience: DockerImage(name, tag, platform="linux/amd64") should "just work" from any host architecture

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething isn't working

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions