Skip to content

Commit

Permalink
Remove prefect-agent as a possible work pool type (#13444)
Browse files Browse the repository at this point in the history
  • Loading branch information
desertaxle committed May 18, 2024
1 parent 295184f commit 73b8e73
Show file tree
Hide file tree
Showing 6 changed files with 36 additions and 34 deletions.
8 changes: 6 additions & 2 deletions src/prefect/server/api/collections.py
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,12 @@ async def get_collection_view(view: str):
resp = await client.get(KNOWN_VIEWS[view])
resp.raise_for_status()

GLOBAL_COLLECTIONS_VIEW_CACHE[view] = resp.json()
return resp.json()
data = resp.json()
if view == "aggregate-worker-metadata":
data.get("prefect", {}).pop("prefect-agent", None)

GLOBAL_COLLECTIONS_VIEW_CACHE[view] = data
return data
except Exception:
local_file = Path(__file__).parent / Path(f"collections_data/views/{view}.json")
if await local_file.exists():
Expand Down
Original file line number Diff line number Diff line change
@@ -1,14 +1,5 @@
{
"prefect": {
"prefect-agent": {
"type": "prefect-agent",
"documentation_url": "https://docs.prefect.io/latest/concepts/work-pools/#agent-overview",
"display_name": "Prefect Agent",
"logo_url": "https://cdn.sanity.io/images/3ugk85nk/production/c771bb53894c877e169c8db158c5598558b8f175-24x24.svg",
"install_command": "pip install prefect",
"default_base_job_configuration": {},
"description": "Execute flow runs on heterogeneous infrastructure using infrastructure blocks."
},
"process": {
"default_base_job_configuration": {
"job_configuration": {
Expand Down
22 changes: 11 additions & 11 deletions tests/cli/test_work_pool.py
Original file line number Diff line number Diff line change
Expand Up @@ -57,13 +57,13 @@ async def test_create_work_pool(self, prefect_client, mock_collection_registry):
pool_name = "my-pool"
res = await run_sync_in_worker_thread(
invoke_and_assert,
f"work-pool create {pool_name} -t prefect-agent",
f"work-pool create {pool_name} -t fake",
)
assert res.exit_code == 0
assert f"Created work pool {pool_name!r}" in res.output
client_res = await prefect_client.read_work_pool(pool_name)
assert client_res.name == pool_name
assert client_res.base_job_template == {}
assert client_res.base_job_template == FAKE_DEFAULT_BASE_JOB_TEMPLATE
assert isinstance(client_res, WorkPool)

@pytest.mark.usefixtures("mock_collection_registry")
Expand Down Expand Up @@ -114,7 +114,7 @@ async def test_create_work_pool_with_empty_name(
):
await run_sync_in_worker_thread(
invoke_and_assert,
"work-pool create '' -t prefect-agent",
"work-pool create '' -t process",
expected_code=1,
expected_output_contains=["name cannot be empty"],
)
Expand All @@ -126,13 +126,13 @@ async def test_create_work_pool_name_conflict(
pool_name = "my-pool"
await run_sync_in_worker_thread(
invoke_and_assert,
f"work-pool create {pool_name} -t prefect-agent",
f"work-pool create {pool_name} -t process",
expected_code=0,
expected_output_contains=[f"Created work pool {pool_name!r}"],
)
await run_sync_in_worker_thread(
invoke_and_assert,
f"work-pool create {pool_name} -t prefect-agent",
f"work-pool create {pool_name} -t process",
expected_code=1,
expected_output_contains=[
f"Work pool named {pool_name!r} already exists. Please try creating"
Expand All @@ -145,18 +145,18 @@ async def test_default_template(self, prefect_client):
pool_name = "my-pool"
res = await run_sync_in_worker_thread(
invoke_and_assert,
f"work-pool create {pool_name} -t prefect-agent",
f"work-pool create {pool_name} -t fake",
)
assert res.exit_code == 0
client_res = await prefect_client.read_work_pool(pool_name)
assert client_res.base_job_template == dict()
assert client_res.base_job_template == FAKE_DEFAULT_BASE_JOB_TEMPLATE

@pytest.mark.usefixtures("mock_collection_registry")
async def test_default_paused(self, prefect_client):
pool_name = "my-pool"
res = await run_sync_in_worker_thread(
invoke_and_assert,
f"work-pool create {pool_name} -t prefect-agent",
f"work-pool create {pool_name} -t process",
)
assert res.exit_code == 0
client_res = await prefect_client.read_work_pool(pool_name)
Expand All @@ -167,7 +167,7 @@ async def test_paused_true(self, prefect_client):
pool_name = "my-pool"
res = await run_sync_in_worker_thread(
invoke_and_assert,
f"work-pool create {pool_name} --paused -t prefect-agent",
f"work-pool create {pool_name} --paused -t process",
)
assert res.exit_code == 0
client_res = await prefect_client.read_work_pool(pool_name)
Expand Down Expand Up @@ -239,7 +239,7 @@ async def test_create_interactive_first_type(self, prefect_client):
)
client_res = await prefect_client.read_work_pool(work_pool_name)
assert client_res.name == work_pool_name
assert client_res.type == "prefect-agent"
assert client_res.type == "fake"
assert isinstance(client_res, WorkPool)

@pytest.mark.usefixtures("interactive_console", "mock_collection_registry")
Expand All @@ -254,7 +254,7 @@ async def test_create_interactive_second_type(self, prefect_client):
)
client_res = await prefect_client.read_work_pool(work_pool_name)
assert client_res.name == work_pool_name
assert client_res.type == "fake"
assert client_res.type == "cloud-run:push"
assert isinstance(client_res, WorkPool)

@pytest.mark.usefixtures("mock_collection_registry")
Expand Down
8 changes: 0 additions & 8 deletions tests/fixtures/collections_registry.py
Original file line number Diff line number Diff line change
Expand Up @@ -393,14 +393,6 @@ def mock_collection_registry(
k8s_default_base_job_template,
):
mock_body = {
"prefect": {
"prefect-agent": {
"type": "prefect-agent",
"default_base_job_configuration": {},
"display_name": "Prefect Agent",
"description": "A Prefect Agent pool.",
}
},
"prefect-fake": {
"fake": {
"type": "fake",
Expand Down
21 changes: 19 additions & 2 deletions tests/server/api/test_collections.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,14 +41,23 @@ def mock_collection_response(self):
},
}

@pytest.fixture
def mock_worker_response(self):
return {
"prefect": {
"prefect-agent": {},
"process": {},
}
}

@respx.mock
@pytest.fixture
def mock_get_view(
self,
respx_mock,
mock_flow_response,
mock_block_response,
mock_collection_response,
mock_worker_response,
):
respx_mock.get(self.collection_view_url("flow")).mock(
return_value=Response(200, json=mock_flow_response)
Expand All @@ -57,7 +66,7 @@ def mock_get_view(
return_value=Response(200, json=mock_block_response)
)
respx_mock.get(self.collection_view_url("worker")).mock(
return_value=Response(404, json=mock_collection_response)
return_value=Response(200, json=mock_worker_response)
)

return respx_mock
Expand Down Expand Up @@ -131,3 +140,11 @@ async def test_read_worker_view_failed_fetch(self, client, mock_get_missing_view
assert res.status_code == 200
# check for expected key to ensure it isn't an error
assert isinstance(res.json()["prefect"], dict)

async def test_prefect_agent_excluded_from_worker_metadata(
self, client, mock_get_view
):
res = await client.get("/collections/views/aggregate-worker-metadata")

assert res.status_code == 200
assert "prefect-agent" not in res.json()["prefect"]
2 changes: 0 additions & 2 deletions tests/workers/test_utilities.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@ def available():
"fake",
"faker",
"kubernetes",
"prefect-agent",
"process",
]

Expand All @@ -74,7 +73,6 @@ def available():
"docker",
"ecs",
"kubernetes",
"prefect-agent",
"process",
"vertex-ai",
}
Expand Down

0 comments on commit 73b8e73

Please sign in to comment.