diff --git a/python_modules/dagster/dagster/_daemon/monitoring/run_monitoring.py b/python_modules/dagster/dagster/_daemon/monitoring/run_monitoring.py index 88302f5fd5ba4..7715420f8ead3 100644 --- a/python_modules/dagster/dagster/_daemon/monitoring/run_monitoring.py +++ b/python_modules/dagster/dagster/_daemon/monitoring/run_monitoring.py @@ -137,7 +137,10 @@ def monitor_started_run( # Return rather than immediately checking for a timeout, since we only just resumed return else: - if instance.run_launcher.supports_resume_run: + if ( + instance.run_launcher.supports_resume_run + and instance.run_monitoring_max_resume_run_attempts > 0 + ): msg = ( f"Detected run worker status {check_health_result}. Marking run" f" {run.run_id} as failed, because it has surpassed the configured maximum"