-
Notifications
You must be signed in to change notification settings - Fork 2k
Open
Labels
bugSomething isn't workingSomething isn't working
Description
Bug summary
When running a flow with a deployment over a process pool, if the run is deleted through the Prefect UI, after a couple of seconds the worker dies. When running Prefect through Docker Compose, this cause either the death of the worker or a restart.
I'm running a minimal example with the following docker-compose.yml
services:
postgres:
image: postgres:17.5
environment:
POSTGRES_USER: prefect
POSTGRES_PASSWORD: prefect
POSTGRES_DB: prefect
volumes:
- postgres_data:/var/lib/postgresql/data
healthcheck:
test: ["CMD-SHELL", "pg_isready -U prefect"]
interval: 5s
timeout: 5s
retries: 5
prefect-server:
image: prefecthq/prefect:3-latest
depends_on:
postgres:
condition: service_healthy
environment:
PREFECT_API_DATABASE_CONNECTION_URL: postgresql+asyncpg://prefect:prefect@postgres:5432/prefect
PREFECT_SERVER_API_HOST: 0.0.0.0
command: prefect server start --no-services
ports:
- "4200:4200"
prefect-services:
image: prefecthq/prefect:3-latest
depends_on:
postgres:
condition: service_healthy
environment:
PREFECT_API_DATABASE_CONNECTION_URL: postgresql+asyncpg://prefect:prefect@postgres:5432/prefect
command: prefect server services start
prefect-worker:
image: prefecthq/prefect:3-latest
depends_on:
prefect-server:
condition: service_started
environment:
PREFECT_API_URL: http://prefect-server:4200/api
command: prefect worker start --pool process-pool --type process
volumes:
- ./main.py:/opt/prefect/main.py
volumes:
postgres_data:
The main.py
file it's just a toy flow used to create a deployment. I then run this deployment through the UI and stop it when it's running.
import time
from prefect import flow
from prefect.deployments import run_deployment
from prefect.deployments.runner import deploy
from prefect.runner.storage import LocalStorage
@flow
def main():
time.sleep(30)
print("Hello from prefect-experiment!")
return None
if __name__ == "__main__":
deployment = main.to_deployment(
name="test",
description="test",
work_pool_name="process-pool",
)
entrypoint = "/opt/prefect"
storage = LocalStorage(
path=entrypoint,
)
deployment.storage = storage # type: ignore
deploy(deployment, work_pool_name="process-pool", build=False, push=False) # type: ignore
I remember not having this kind of behaviour in older Prefect versions, as I deleted runs multiple times without problems.
Version info
I'm running the latest prefect version, even tho I've tried with multiple versions like 3.4.15 and 3.4.1.
Additional context
Here's the error logs I see on the process-worker container
15:03:46.758 | ERROR | Flow run 'inquisitive-mastiff' - Encountered exception during execution: ObjectNotFound(None)
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 397, in set_flow_run_state
response = self.request(
^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/base.py", line 33, in request
return self._client.send(request)
^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 622, in send
response.raise_for_status()
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 163, in raise_for_status
raise PrefectHTTPStatusError.from_httpx_error(exc) from exc.__cause__
prefect.exceptions.PrefectHTTPStatusError: Client error '404 Not Found' for url 'http://prefect-server:4200/api/flow_runs/9576e0fd-b8a3-4850-a801-4ea9f553c560/set_state'
Response: {'exception_message': 'Flow run with id 9576e0fd-b8a3-4850-a801-4ea9f553c560 not found'}
For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/404
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 781, in run_context
yield self
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 1395, in run_flow_sync
engine.call_flow_fn()
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 802, in call_flow_fn
self.handle_success(result)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 384, in handle_success
self.set_state(terminal_state)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 335, in set_state
state = propose_state_sync(
^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 475, in propose_state_sync
response = set_state_and_handle_waits(set_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 461, in set_state_and_handle_waits
response = set_state_func()
^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 408, in set_flow_run_state
raise ObjectNotFound(http_exc=e) from e
prefect.exceptions.ObjectNotFound: None
15:03:46.778 | INFO | Flow run 'inquisitive-mastiff' - Finished in state Running()
15:03:46.780 | ERROR | Flow run 'inquisitive-mastiff' - Engine execution exited with unexpected exception
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 397, in set_flow_run_state
response = self.request(
^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/base.py", line 33, in request
return self._client.send(request)
^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 622, in send
response.raise_for_status()
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 163, in raise_for_status
raise PrefectHTTPStatusError.from_httpx_error(exc) from exc.__cause__
prefect.exceptions.PrefectHTTPStatusError: Client error '404 Not Found' for url 'http://prefect-server:4200/api/flow_runs/9576e0fd-b8a3-4850-a801-4ea9f553c560/set_state'
Response: {'exception_message': 'Flow run with id 9576e0fd-b8a3-4850-a801-4ea9f553c560 not found'}
For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/404
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 1552, in run_flow
ret_val = run_flow_sync(**kwargs)
^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 1394, in run_flow_sync
with engine.run_context():
^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/contextlib.py", line 158, in __exit__
self.gen.throw(value)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 786, in run_context
self.handle_exception(exc)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 410, in handle_exception
state = self.set_state(terminal_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 335, in set_state
state = propose_state_sync(
^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 475, in propose_state_sync
response = set_state_and_handle_waits(set_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 461, in set_state_and_handle_waits
response = set_state_func()
^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 408, in set_flow_run_state
raise ObjectNotFound(http_exc=e) from e
prefect.exceptions.ObjectNotFound: None
15:03:46.782 | ERROR | prefect.engine - Execution of flow run '9576e0fd-b8a3-4850-a801-4ea9f553c560' exited with unexpected exception
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 397, in set_flow_run_state
response = self.request(
^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/base.py", line 33, in request
return self._client.send(request)
^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 622, in send
response.raise_for_status()
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 163, in raise_for_status
raise PrefectHTTPStatusError.from_httpx_error(exc) from exc.__cause__
prefect.exceptions.PrefectHTTPStatusError: Client error '404 Not Found' for url 'http://prefect-server:4200/api/flow_runs/9576e0fd-b8a3-4850-a801-4ea9f553c560/set_state'
Response: {'exception_message': 'Flow run with id 9576e0fd-b8a3-4850-a801-4ea9f553c560 not found'}
For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/404
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/engine.py", line 57, in handle_engine_signals
yield
File "/usr/local/lib/python3.12/site-packages/prefect/engine.py", line 124, in <module>
run_flow(flow, flow_run=flow_run, error_logger=run_logger)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 1552, in run_flow
ret_val = run_flow_sync(**kwargs)
^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 1394, in run_flow_sync
with engine.run_context():
^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/contextlib.py", line 158, in __exit__
self.gen.throw(value)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 786, in run_context
self.handle_exception(exc)
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 410, in handle_exception
state = self.set_state(terminal_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/flow_engine.py", line 335, in set_state
state = propose_state_sync(
^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 475, in propose_state_sync
response = set_state_and_handle_waits(set_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 461, in set_state_and_handle_waits
response = set_state_func()
^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 408, in set_flow_run_state
raise ObjectNotFound(http_exc=e) from e
prefect.exceptions.ObjectNotFound: None
Hello from prefect-experiment!
15:03:46.958 | ERROR | prefect.flow_runs.runner - Process for flow run 'inquisitive-mastiff' exited with status code: 1
15:03:46.977 | ERROR | prefect.flow_runs.worker - Failed to update state of flow run '9576e0fd-b8a3-4850-a801-4ea9f553c560'
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 895, in set_flow_run_state
response = await self.request(
^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/base.py", line 53, in request
return await self._client.send(request)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 379, in send
response.raise_for_status()
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 163, in raise_for_status
raise PrefectHTTPStatusError.from_httpx_error(exc) from exc.__cause__
prefect.exceptions.PrefectHTTPStatusError: Client error '404 Not Found' for url 'http://prefect-server:4200/api/flow_runs/9576e0fd-b8a3-4850-a801-4ea9f553c560/set_state'
Response: {'exception_message': 'Flow run with id 9576e0fd-b8a3-4850-a801-4ea9f553c560 not found'}
For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/404
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/workers/base.py", line 1459, in _propose_crashed_state
state = await propose_state(
^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 379, in propose_state
response = await set_state_and_handle_waits(set_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 366, in set_state_and_handle_waits
response = await set_state_func()
^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 906, in set_flow_run_state
raise ObjectNotFound(http_exc=e) from e
prefect.exceptions.ObjectNotFound: None
15:03:46.983 | ERROR | prefect.flow_runs.runner - Failed to update state of flow run '9576e0fd-b8a3-4850-a801-4ea9f553c560'
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 895, in set_flow_run_state
response = await self.request(
^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/base.py", line 53, in request
return await self._client.send(request)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 379, in send
response.raise_for_status()
File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 163, in raise_for_status
raise PrefectHTTPStatusError.from_httpx_error(exc) from exc.__cause__
prefect.exceptions.PrefectHTTPStatusError: Client error '404 Not Found' for url 'http://prefect-server:4200/api/flow_runs/9576e0fd-b8a3-4850-a801-4ea9f553c560/set_state'
Response: {'exception_message': 'Flow run with id 9576e0fd-b8a3-4850-a801-4ea9f553c560 not found'}
For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/404
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/usr/local/lib/python3.12/site-packages/prefect/runner/runner.py", line 1396, in _propose_crashed_state
state = await propose_state(
^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 379, in propose_state
response = await set_state_and_handle_waits(set_state)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/utilities/engine.py", line 366, in set_state_and_handle_waits
response = await set_state_func()
^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 906, in set_flow_run_state
raise ObjectNotFound(http_exc=e) from e
prefect.exceptions.ObjectNotFound: None
+ Exception Group Traceback (most recent call last):
| File "/usr/local/lib/python3.12/site-packages/prefect/cli/_utilities.py", line 44, in wrapper
| return fn(*args, **kwargs)
| ^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/cli/_types.py", line 156, in sync_fn
| return asyncio.run(async_fn(*args, **kwargs))
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/asyncio/runners.py", line 195, in run
| return runner.run(main)
| ^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/asyncio/runners.py", line 118, in run
| return self._loop.run_until_complete(task)
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/asyncio/base_events.py", line 691, in run_until_complete
| return future.result()
| ^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/cli/worker.py", line 168, in start
| await worker.start(
| File "/usr/local/lib/python3.12/site-packages/prefect/workers/process.py", line 158, in start
| async with self as worker:
| ^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/workers/process.py", line 316, in __aexit__
| await super().__aexit__(*exc_info)
| File "/usr/local/lib/python3.12/site-packages/prefect/workers/base.py", line 1578, in __aexit__
| raise exceptions[0] from None
| File "/usr/local/lib/python3.12/contextlib.py", line 737, in __aexit__
| cb_suppress = await cb(*exc_details)
| ^^^^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/runner/runner.py", line 1537, in __aexit__
| await self._exit_stack.__aexit__(*exc_info)
| File "/usr/local/lib/python3.12/contextlib.py", line 754, in __aexit__
| raise exc_details[1]
| File "/usr/local/lib/python3.12/contextlib.py", line 737, in __aexit__
| cb_suppress = await cb(*exc_details)
| ^^^^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/anyio/_backends/_asyncio.py", line 772, in __aexit__
| raise BaseExceptionGroup(
| ExceptionGroup: unhandled errors in a TaskGroup (1 sub-exception)
+-+---------------- 1 ----------------
| Traceback (most recent call last):
| File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 716, in read_flow_run
| response = await self.request(
| ^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/base.py", line 53, in request
| return await self._client.send(request)
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 379, in send
| response.raise_for_status()
| File "/usr/local/lib/python3.12/site-packages/prefect/client/base.py", line 163, in raise_for_status
| raise PrefectHTTPStatusError.from_httpx_error(exc) from exc.__cause__
| prefect.exceptions.PrefectHTTPStatusError: Client error '404 Not Found' for url 'http://prefect-server:4200/api/flow_runs/9576e0fd-b8a3-4850-a801-4ea9f553c560'
| Response: {'detail': 'Flow run not found'}
| For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/404
|
| The above exception was the direct cause of the following exception:
|
| Traceback (most recent call last):
| File "/usr/local/lib/python3.12/site-packages/prefect/runner/runner.py", line 1333, in _submit_run_and_capture_errors
| api_flow_run = await self._client.read_flow_run(flow_run_id=flow_run.id)
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
| File "/usr/local/lib/python3.12/site-packages/prefect/client/orchestration/_flow_runs/client.py", line 721, in read_flow_run
| raise ObjectNotFound(http_exc=e) from e
| prefect.exceptions.ObjectNotFound: None
Metadata
Metadata
Assignees
Labels
bugSomething isn't workingSomething isn't working