From 9c85c4f0bd4adaa5ae5dfbd0afa8b44de5e08b53 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 14:54:41 -0600 Subject: [PATCH 01/19] init --- src/prefect/client/orchestration/__init__.py | 579 +-------- .../orchestration/_deployments/__init__.py | 0 .../orchestration/_deployments/client.py | 1117 +++++++++++++++++ 3 files changed, 1124 insertions(+), 572 deletions(-) create mode 100644 src/prefect/client/orchestration/_deployments/__init__.py create mode 100644 src/prefect/client/orchestration/_deployments/client.py diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index 9103d12d5b5d..cf0673d43620 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -40,6 +40,11 @@ VariableAsyncClient, ) +from prefect.client.orchestration._deployments.client import ( + DeploymentClient, + DeploymentAsyncClient, +) + import prefect import prefect.exceptions import prefect.settings @@ -52,19 +57,11 @@ BlockSchemaCreate, BlockTypeCreate, BlockTypeUpdate, - DeploymentCreate, - DeploymentFlowRunCreate, - DeploymentScheduleCreate, - DeploymentScheduleUpdate, - DeploymentUpdate, FlowCreate, FlowRunCreate, FlowRunNotificationPolicyCreate, FlowRunNotificationPolicyUpdate, FlowRunUpdate, - LogCreate, - GlobalConcurrencyLimitCreate, - GlobalConcurrencyLimitUpdate, TaskRunCreate, TaskRunUpdate, WorkPoolCreate, @@ -87,9 +84,7 @@ BlockDocument, BlockSchema, BlockType, - ConcurrencyOptions, Constant, - DeploymentSchedule, Flow, FlowRunInput, FlowRunNotificationPolicy, @@ -104,13 +99,9 @@ WorkQueueStatusDetail, ) from prefect.client.schemas.responses import ( - DeploymentResponse, - FlowRunResponse, WorkerFlowRunResponse, ) -from prefect.client.schemas.schedules import SCHEDULE_TYPES from prefect.client.schemas.sorting import ( - DeploymentSort, FlowRunSort, FlowSort, TaskRunSort, @@ -255,6 +246,7 @@ class PrefectClient( LogAsyncClient, VariableAsyncClient, ConcurrencyLimitAsyncClient, + DeploymentAsyncClient, ): """ An asynchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -612,78 +604,6 @@ async def read_flow_by_name( response = await self._client.get(f"/flows/name/{flow_name}") return Flow.model_validate(response.json()) - async def create_flow_run_from_deployment( - self, - deployment_id: UUID, - *, - parameters: Optional[dict[str, Any]] = None, - context: Optional[dict[str, Any]] = None, - state: Optional[prefect.states.State[Any]] = None, - name: Optional[str] = None, - tags: Optional[Iterable[str]] = None, - idempotency_key: Optional[str] = None, - parent_task_run_id: Optional[UUID] = None, - work_queue_name: Optional[str] = None, - job_variables: Optional[dict[str, Any]] = None, - labels: Optional[KeyValueLabelsField] = None, - ) -> FlowRun: - """ - Create a flow run for a deployment. - - Args: - deployment_id: The deployment ID to create the flow run from - parameters: Parameter overrides for this flow run. Merged with the - deployment defaults - context: Optional run context data - state: The initial state for the run. If not provided, defaults to - `Scheduled` for now. Should always be a `Scheduled` type. - name: An optional name for the flow run. If not provided, the server will - generate a name. - tags: An optional iterable of tags to apply to the flow run; these tags - are merged with the deployment's tags. - idempotency_key: Optional idempotency key for creation of the flow run. - If the key matches the key of an existing flow run, the existing run will - be returned instead of creating a new one. - parent_task_run_id: if a subflow run is being created, the placeholder task - run identifier in the parent flow - work_queue_name: An optional work queue name to add this run to. If not provided, - will default to the deployment's set work queue. If one is provided that does not - exist, a new work queue will be created within the deployment's work pool. - job_variables: Optional variables that will be supplied to the flow run job. - - Raises: - httpx.RequestError: if the Prefect API does not successfully create a run for any reason - - Returns: - The flow run model - """ - parameters = parameters or {} - context = context or {} - state = state or prefect.states.Scheduled() - tags = tags or [] - - flow_run_create = DeploymentFlowRunCreate( - parameters=parameters, - context=context, - state=state.to_state_create(), - tags=list(tags), - name=name, - idempotency_key=idempotency_key, - parent_task_run_id=parent_task_run_id, - job_variables=job_variables, - labels=labels, - ) - - # done separately to avoid including this field in payloads sent to older API versions - if work_queue_name: - flow_run_create.work_queue_name = work_queue_name - - response = await self._client.post( - f"/deployments/{deployment_id}/create_flow_run", - json=flow_run_create.model_dump(mode="json", exclude_unset=True), - ) - return FlowRun.model_validate(response.json()) - async def create_flow_run( self, flow: "FlowObject[Any, R]", @@ -1466,422 +1386,6 @@ async def read_block_documents_by_type( response.json() ) - async def create_deployment( - self, - flow_id: UUID, - name: str, - version: Optional[str] = None, - schedules: Optional[list[DeploymentScheduleCreate]] = None, - concurrency_limit: Optional[int] = None, - concurrency_options: Optional[ConcurrencyOptions] = None, - parameters: Optional[dict[str, Any]] = None, - description: Optional[str] = None, - work_queue_name: Optional[str] = None, - work_pool_name: Optional[str] = None, - tags: Optional[list[str]] = None, - storage_document_id: Optional[UUID] = None, - path: Optional[str] = None, - entrypoint: Optional[str] = None, - infrastructure_document_id: Optional[UUID] = None, - parameter_openapi_schema: Optional[dict[str, Any]] = None, - paused: Optional[bool] = None, - pull_steps: Optional[list[dict[str, Any]]] = None, - enforce_parameter_schema: Optional[bool] = None, - job_variables: Optional[dict[str, Any]] = None, - ) -> UUID: - """ - Create a deployment. - - Args: - flow_id: the flow ID to create a deployment for - name: the name of the deployment - version: an optional version string for the deployment - tags: an optional list of tags to apply to the deployment - storage_document_id: an reference to the storage block document - used for the deployed flow - infrastructure_document_id: an reference to the infrastructure block document - to use for this deployment - job_variables: A dictionary of dot delimited infrastructure overrides that - will be applied at runtime; for example `env.CONFIG_KEY=config_value` or - `namespace='prefect'`. This argument was previously named `infra_overrides`. - Both arguments are supported for backwards compatibility. - - Raises: - httpx.RequestError: if the deployment was not created for any reason - - Returns: - the ID of the deployment in the backend - """ - - if parameter_openapi_schema is None: - parameter_openapi_schema = {} - deployment_create = DeploymentCreate( - flow_id=flow_id, - name=name, - version=version, - parameters=dict(parameters or {}), - tags=list(tags or []), - work_queue_name=work_queue_name, - description=description, - storage_document_id=storage_document_id, - path=path, - entrypoint=entrypoint, - infrastructure_document_id=infrastructure_document_id, - job_variables=dict(job_variables or {}), - parameter_openapi_schema=parameter_openapi_schema, - paused=paused, - schedules=schedules or [], - concurrency_limit=concurrency_limit, - concurrency_options=concurrency_options, - pull_steps=pull_steps, - enforce_parameter_schema=enforce_parameter_schema, - ) - - if work_pool_name is not None: - deployment_create.work_pool_name = work_pool_name - - # Exclude newer fields that are not set to avoid compatibility issues - exclude = { - field - for field in ["work_pool_name", "work_queue_name"] - if field not in deployment_create.model_fields_set - } - - if deployment_create.paused is None: - exclude.add("paused") - - if deployment_create.pull_steps is None: - exclude.add("pull_steps") - - if deployment_create.enforce_parameter_schema is None: - exclude.add("enforce_parameter_schema") - - json = deployment_create.model_dump(mode="json", exclude=exclude) - response = await self._client.post( - "/deployments/", - json=json, - ) - deployment_id = response.json().get("id") - if not deployment_id: - raise httpx.RequestError(f"Malformed response: {response}") - - return UUID(deployment_id) - - async def set_deployment_paused_state( - self, deployment_id: UUID, paused: bool - ) -> None: - await self._client.patch( - f"/deployments/{deployment_id}", json={"paused": paused} - ) - - async def update_deployment( - self, - deployment_id: UUID, - deployment: DeploymentUpdate, - ) -> None: - await self._client.patch( - f"/deployments/{deployment_id}", - json=deployment.model_dump(mode="json", exclude_unset=True), - ) - - async def _create_deployment_from_schema(self, schema: DeploymentCreate) -> UUID: - """ - Create a deployment from a prepared `DeploymentCreate` schema. - """ - # TODO: We are likely to remove this method once we have considered the - # packaging interface for deployments further. - response = await self._client.post( - "/deployments/", json=schema.model_dump(mode="json") - ) - deployment_id = response.json().get("id") - if not deployment_id: - raise httpx.RequestError(f"Malformed response: {response}") - - return UUID(deployment_id) - - async def read_deployment( - self, - deployment_id: Union[UUID, str], - ) -> DeploymentResponse: - """ - Query the Prefect API for a deployment by id. - - Args: - deployment_id: the deployment ID of interest - - Returns: - a [Deployment model][prefect.client.schemas.objects.Deployment] representation of the deployment - """ - if not isinstance(deployment_id, UUID): - try: - deployment_id = UUID(deployment_id) - except ValueError: - raise ValueError(f"Invalid deployment ID: {deployment_id}") - - try: - response = await self._client.get(f"/deployments/{deployment_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - return DeploymentResponse.model_validate(response.json()) - - async def read_deployment_by_name( - self, - name: str, - ) -> DeploymentResponse: - """ - Query the Prefect API for a deployment by name. - - Args: - name: A deployed flow's name: / - - Raises: - prefect.exceptions.ObjectNotFound: If request returns 404 - httpx.RequestError: If request fails - - Returns: - a Deployment model representation of the deployment - """ - try: - response = await self._client.get(f"/deployments/name/{name}") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - from prefect.utilities.text import fuzzy_match_string - - deployments = await self.read_deployments() - flow_name_map = { - flow.id: flow.name - for flow in await asyncio.gather( - *[ - self.read_flow(flow_id) - for flow_id in {d.flow_id for d in deployments} - ] - ) - } - - raise prefect.exceptions.ObjectNotFound( - http_exc=e, - help_message=( - f"Deployment {name!r} not found; did you mean {fuzzy_match!r}?" - if ( - fuzzy_match := fuzzy_match_string( - name, - [ - f"{flow_name_map[d.flow_id]}/{d.name}" - for d in deployments - ], - ) - ) - else f"Deployment {name!r} not found. Try `prefect deployment ls` to find available deployments." - ), - ) from e - else: - raise - - return DeploymentResponse.model_validate(response.json()) - - async def read_deployments( - self, - *, - flow_filter: Optional[FlowFilter] = None, - flow_run_filter: Optional[FlowRunFilter] = None, - task_run_filter: Optional[TaskRunFilter] = None, - deployment_filter: Optional[DeploymentFilter] = None, - work_pool_filter: Optional[WorkPoolFilter] = None, - work_queue_filter: Optional[WorkQueueFilter] = None, - limit: Optional[int] = None, - sort: Optional[DeploymentSort] = None, - offset: int = 0, - ) -> list[DeploymentResponse]: - """ - Query the Prefect API for deployments. Only deployments matching all - the provided criteria will be returned. - - Args: - flow_filter: filter criteria for flows - flow_run_filter: filter criteria for flow runs - task_run_filter: filter criteria for task runs - deployment_filter: filter criteria for deployments - work_pool_filter: filter criteria for work pools - work_queue_filter: filter criteria for work pool queues - limit: a limit for the deployment query - offset: an offset for the deployment query - - Returns: - a list of Deployment model representations - of the deployments - """ - body: dict[str, Any] = { - "flows": flow_filter.model_dump(mode="json") if flow_filter else None, - "flow_runs": ( - flow_run_filter.model_dump(mode="json", exclude_unset=True) - if flow_run_filter - else None - ), - "task_runs": ( - task_run_filter.model_dump(mode="json") if task_run_filter else None - ), - "deployments": ( - deployment_filter.model_dump(mode="json") if deployment_filter else None - ), - "work_pools": ( - work_pool_filter.model_dump(mode="json") if work_pool_filter else None - ), - "work_pool_queues": ( - work_queue_filter.model_dump(mode="json") if work_queue_filter else None - ), - "limit": limit, - "offset": offset, - "sort": sort, - } - - response = await self._client.post("/deployments/filter", json=body) - return pydantic.TypeAdapter(list[DeploymentResponse]).validate_python( - response.json() - ) - - async def delete_deployment( - self, - deployment_id: UUID, - ) -> None: - """ - Delete deployment by id. - - Args: - deployment_id: The deployment id of interest. - Raises: - prefect.exceptions.ObjectNotFound: If request returns 404 - httpx.RequestError: If requests fails - """ - try: - await self._client.delete(f"/deployments/{deployment_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == 404: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - - async def create_deployment_schedules( - self, - deployment_id: UUID, - schedules: list[tuple[SCHEDULE_TYPES, bool]], - ) -> list[DeploymentSchedule]: - """ - Create deployment schedules. - - Args: - deployment_id: the deployment ID - schedules: a list of tuples containing the schedule to create - and whether or not it should be active. - - Raises: - httpx.RequestError: if the schedules were not created for any reason - - Returns: - the list of schedules created in the backend - """ - deployment_schedule_create = [ - DeploymentScheduleCreate(schedule=schedule[0], active=schedule[1]) - for schedule in schedules - ] - - json = [ - deployment_schedule_create.model_dump(mode="json") - for deployment_schedule_create in deployment_schedule_create - ] - response = await self._client.post( - f"/deployments/{deployment_id}/schedules", json=json - ) - return pydantic.TypeAdapter(list[DeploymentSchedule]).validate_python( - response.json() - ) - - async def read_deployment_schedules( - self, - deployment_id: UUID, - ) -> list[DeploymentSchedule]: - """ - Query the Prefect API for a deployment's schedules. - - Args: - deployment_id: the deployment ID - - Returns: - a list of DeploymentSchedule model representations of the deployment schedules - """ - try: - response = await self._client.get(f"/deployments/{deployment_id}/schedules") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - return pydantic.TypeAdapter(list[DeploymentSchedule]).validate_python( - response.json() - ) - - async def update_deployment_schedule( - self, - deployment_id: UUID, - schedule_id: UUID, - active: Optional[bool] = None, - schedule: Optional[SCHEDULE_TYPES] = None, - ) -> None: - """ - Update a deployment schedule by ID. - - Args: - deployment_id: the deployment ID - schedule_id: the deployment schedule ID of interest - active: whether or not the schedule should be active - schedule: the cron, rrule, or interval schedule this deployment schedule should use - """ - kwargs: dict[str, Any] = {} - if active is not None: - kwargs["active"] = active - if schedule is not None: - kwargs["schedule"] = schedule - - deployment_schedule_update = DeploymentScheduleUpdate(**kwargs) - json = deployment_schedule_update.model_dump(mode="json", exclude_unset=True) - - try: - await self._client.patch( - f"/deployments/{deployment_id}/schedules/{schedule_id}", json=json - ) - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - - async def delete_deployment_schedule( - self, - deployment_id: UUID, - schedule_id: UUID, - ) -> None: - """ - Delete a deployment schedule. - - Args: - deployment_id: the deployment ID - schedule_id: the ID of the deployment schedule to delete. - - Raises: - httpx.RequestError: if the schedules were not deleted for any reason - """ - try: - await self._client.delete( - f"/deployments/{deployment_id}/schedules/{schedule_id}" - ) - except httpx.HTTPStatusError as e: - if e.response.status_code == 404: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - async def read_flow_run(self, flow_run_id: UUID) -> FlowRun: """ Query the Prefect API for a flow run by id. @@ -2666,27 +2170,6 @@ async def read_work_queues( return pydantic.TypeAdapter(list[WorkQueue]).validate_python(response.json()) - async def get_scheduled_flow_runs_for_deployments( - self, - deployment_ids: list[UUID], - scheduled_before: Optional[datetime.datetime] = None, - limit: Optional[int] = None, - ) -> list[FlowRunResponse]: - body: dict[str, Any] = dict(deployment_ids=[str(id) for id in deployment_ids]) - if scheduled_before: - body["scheduled_before"] = str(scheduled_before) - if limit: - body["limit"] = limit - - response = await self._client.post( - "/deployments/get_scheduled_flow_runs", - json=body, - ) - - return pydantic.TypeAdapter(list[FlowRunResponse]).validate_python( - response.json() - ) - async def get_scheduled_flow_runs_for_work_pool( self, work_pool_name: str, @@ -3026,6 +2509,7 @@ class SyncPrefectClient( LogClient, VariableClient, ConcurrencyLimitClient, + DeploymentClient, ): """ A synchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -3760,55 +3244,6 @@ def read_task_run_states(self, task_run_id: UUID) -> list[prefect.states.State]: response.json() ) - def read_deployment( - self, - deployment_id: UUID, - ) -> DeploymentResponse: - """ - Query the Prefect API for a deployment by id. - - Args: - deployment_id: the deployment ID of interest - - Returns: - a [Deployment model][prefect.client.schemas.objects.Deployment] representation of the deployment - """ - try: - response = self._client.get(f"/deployments/{deployment_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - return DeploymentResponse.model_validate(response.json()) - - def read_deployment_by_name( - self, - name: str, - ) -> DeploymentResponse: - """ - Query the Prefect API for a deployment by name. - - Args: - name: A deployed flow's name: / - - Raises: - prefect.exceptions.ObjectNotFound: If request returns 404 - httpx.RequestError: If request fails - - Returns: - a Deployment model representation of the deployment - """ - try: - response = self._client.get(f"/deployments/name/{name}") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - - return DeploymentResponse.model_validate(response.json()) - def update_flow_run_labels( self, flow_run_id: UUID, labels: KeyValueLabelsField ) -> None: diff --git a/src/prefect/client/orchestration/_deployments/__init__.py b/src/prefect/client/orchestration/_deployments/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_deployments/client.py b/src/prefect/client/orchestration/_deployments/client.py new file mode 100644 index 000000000000..0cc6b58dc220 --- /dev/null +++ b/src/prefect/client/orchestration/_deployments/client.py @@ -0,0 +1,1117 @@ +from __future__ import annotations + +from collections.abc import Iterable +from typing import TYPE_CHECKING, Any, Union + +from httpx import HTTPStatusError, RequestError + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.exceptions import ObjectNotFound +from prefect.states import Scheduled + +if TYPE_CHECKING: + import datetime + from uuid import UUID + + from prefect.client.schemas import FlowRun + from prefect.client.schemas.actions import ( + DeploymentCreate, + DeploymentScheduleCreate, + DeploymentUpdate, + ) + from prefect.client.schemas.filters import ( + DeploymentFilter, + FlowFilter, + FlowRunFilter, + TaskRunFilter, + WorkPoolFilter, + WorkQueueFilter, + ) + from prefect.client.schemas.objects import ( + ConcurrencyOptions, + DeploymentSchedule, + ) + from prefect.client.schemas.responses import ( + DeploymentResponse, + FlowRunResponse, + ) + from prefect.client.schemas.schedules import SCHEDULE_TYPES + from prefect.client.schemas.sorting import ( + DeploymentSort, + ) + from prefect.states import State + from prefect.types import KeyValueLabelsField + + +class DeploymentClient(BaseClient): + def create_deployment( + self, + flow_id: "UUID", + name: str, + version: str | None = None, + schedules: list["DeploymentScheduleCreate"] | None = None, + concurrency_limit: int | None = None, + concurrency_options: "ConcurrencyOptions | None" = None, + parameters: dict[str, Any] | None = None, + description: str | None = None, + work_queue_name: str | None = None, + work_pool_name: str | None = None, + tags: list[str] | None = None, + storage_document_id: "UUID | None" = None, + path: str | None = None, + entrypoint: str | None = None, + infrastructure_document_id: "UUID | None" = None, + parameter_openapi_schema: dict[str, Any] | None = None, + paused: bool | None = None, + pull_steps: list[dict[str, Any]] | None = None, + enforce_parameter_schema: bool | None = None, + job_variables: dict[str, Any] | None = None, + ) -> "UUID": + """ + Create a deployment. + + Args: + flow_id: the flow ID to create a deployment for + name: the name of the deployment + version: an optional version string for the deployment + tags: an optional list of tags to apply to the deployment + storage_document_id: an reference to the storage block document + used for the deployed flow + infrastructure_document_id: an reference to the infrastructure block document + to use for this deployment + job_variables: A dictionary of dot delimited infrastructure overrides that + will be applied at runtime; for example `env.CONFIG_KEY=config_value` or + `namespace='prefect'`. This argument was previously named `infra_overrides`. + Both arguments are supported for backwards compatibility. + + Raises: + RequestError: if the deployment was not created for any reason + + Returns: + the ID of the deployment in the backend + """ + from uuid import UUID + + from prefect.client.schemas.actions import DeploymentCreate + + if parameter_openapi_schema is None: + parameter_openapi_schema = {} + deployment_create = DeploymentCreate( + flow_id=flow_id, + name=name, + version=version, + parameters=dict(parameters or {}), + tags=list(tags or []), + work_queue_name=work_queue_name, + description=description, + storage_document_id=storage_document_id, + path=path, + entrypoint=entrypoint, + infrastructure_document_id=infrastructure_document_id, + job_variables=dict(job_variables or {}), + parameter_openapi_schema=parameter_openapi_schema, + paused=paused, + schedules=schedules or [], + concurrency_limit=concurrency_limit, + concurrency_options=concurrency_options, + pull_steps=pull_steps, + enforce_parameter_schema=enforce_parameter_schema, + ) + + if work_pool_name is not None: + deployment_create.work_pool_name = work_pool_name + + # Exclude newer fields that are not set to avoid compatibility issues + exclude = { + field + for field in ["work_pool_name", "work_queue_name"] + if field not in deployment_create.model_fields_set + } + + if deployment_create.paused is None: + exclude.add("paused") + + if deployment_create.pull_steps is None: + exclude.add("pull_steps") + + if deployment_create.enforce_parameter_schema is None: + exclude.add("enforce_parameter_schema") + + json = deployment_create.model_dump(mode="json", exclude=exclude) + response = self.request( + "POST", + "/deployments/", + json=json, + ) + deployment_id = response.json().get("id") + if not deployment_id: + raise RequestError(f"Malformed response: {response}") + + return UUID(deployment_id) + + def set_deployment_paused_state(self, deployment_id: "UUID", paused: bool) -> None: + self.request( + "PATCH", + "/deployments/{id}", + path_params={"id": deployment_id}, + json={"paused": paused}, + ) + + def update_deployment( + self, + deployment_id: "UUID", + deployment: "DeploymentUpdate", + ) -> None: + self.request( + "PATCH", + "/deployments/{id}", + path_params={"id": deployment_id}, + json=deployment.model_dump(mode="json", exclude_unset=True), + ) + + def _create_deployment_from_schema(self, schema: "DeploymentCreate") -> "UUID": + """ + Create a deployment from a prepared `DeploymentCreate` schema. + """ + from uuid import UUID + + # TODO: We are likely to remove this method once we have considered the + # packaging interface for deployments further. + response = self.request( + "POST", "/deployments/", json=schema.model_dump(mode="json") + ) + deployment_id = response.json().get("id") + if not deployment_id: + raise RequestError(f"Malformed response: {response}") + + return UUID(deployment_id) + + def read_deployment( + self, + deployment_id: Union["UUID", str], + ) -> "DeploymentResponse": + """ + Query the Prefect API for a deployment by id. + + Args: + deployment_id: the deployment ID of interest + + Returns: + a [Deployment model][prefect.client.schemas.objects.Deployment] representation of the deployment + """ + from uuid import UUID + + from prefect.client.schemas.responses import DeploymentResponse + + if not isinstance(deployment_id, UUID): + try: + deployment_id = UUID(deployment_id) + except ValueError: + raise ValueError(f"Invalid deployment ID: {deployment_id}") + + try: + response = self.request( + "GET", + "/deployments/{id}", + path_params={"id": deployment_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + return DeploymentResponse.model_validate(response.json()) + + def read_deployment_by_name( + self, + name: str, + ) -> "DeploymentResponse": + """ + Query the Prefect API for a deployment by name. + + Args: + name: A deployed flow's name: / + + Raises: + ObjectNotFound: If request returns 404 + RequestError: If request fails + + Returns: + a Deployment model representation of the deployment + """ + from prefect.client.schemas.responses import DeploymentResponse + + flow_name, deployment_name = name.split("/") + try: + response = self.request( + "GET", + "/deployments/name/{flow_name}/{deployment_name}", + path_params={ + "flow_name": flow_name, + "deployment_name": deployment_name, + }, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + return DeploymentResponse.model_validate(response.json()) + + def read_deployments( + self, + *, + flow_filter: "FlowFilter | None" = None, + flow_run_filter: "FlowRunFilter | None" = None, + task_run_filter: "TaskRunFilter | None" = None, + deployment_filter: "DeploymentFilter | None" = None, + work_pool_filter: "WorkPoolFilter | None" = None, + work_queue_filter: "WorkQueueFilter | None" = None, + limit: int | None = None, + sort: "DeploymentSort | None" = None, + offset: int = 0, + ) -> list["DeploymentResponse"]: + """ + Query the Prefect API for deployments. Only deployments matching all + the provided criteria will be returned. + + Args: + flow_filter: filter criteria for flows + flow_run_filter: filter criteria for flow runs + task_run_filter: filter criteria for task runs + deployment_filter: filter criteria for deployments + work_pool_filter: filter criteria for work pools + work_queue_filter: filter criteria for work pool queues + limit: a limit for the deployment query + offset: an offset for the deployment query + + Returns: + a list of Deployment model representations + of the deployments + """ + from prefect.client.schemas.responses import DeploymentResponse + + body: dict[str, Any] = { + "flows": flow_filter.model_dump(mode="json") if flow_filter else None, + "flow_runs": ( + flow_run_filter.model_dump(mode="json", exclude_unset=True) + if flow_run_filter + else None + ), + "task_runs": ( + task_run_filter.model_dump(mode="json") if task_run_filter else None + ), + "deployments": ( + deployment_filter.model_dump(mode="json") if deployment_filter else None + ), + "work_pools": ( + work_pool_filter.model_dump(mode="json") if work_pool_filter else None + ), + "work_pool_queues": ( + work_queue_filter.model_dump(mode="json") if work_queue_filter else None + ), + "limit": limit, + "offset": offset, + "sort": sort, + } + + response = self.request("POST", "/deployments/filter", json=body) + return DeploymentResponse.model_validate_list(response.json()) + + def delete_deployment( + self, + deployment_id: "UUID", + ) -> None: + """ + Delete deployment by id. + + Args: + deployment_id: The deployment id of interest. + Raises: + ObjectNotFound: If request returns 404 + RequestError: If requests fails + """ + try: + self.request( + "DELETE", + "/deployments/{id}", + path_params={"id": deployment_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def create_deployment_schedules( + self, + deployment_id: "UUID", + schedules: list[tuple["SCHEDULE_TYPES", bool]], + ) -> list["DeploymentSchedule"]: + """ + Create deployment schedules. + + Args: + deployment_id: the deployment ID + schedules: a list of tuples containing the schedule to create + and whether or not it should be active. + + Raises: + RequestError: if the schedules were not created for any reason + + Returns: + the list of schedules created in the backend + """ + from prefect.client.schemas.actions import DeploymentScheduleCreate + from prefect.client.schemas.objects import DeploymentSchedule + + deployment_schedule_create = [ + DeploymentScheduleCreate(schedule=schedule[0], active=schedule[1]) + for schedule in schedules + ] + + json = [ + deployment_schedule_create.model_dump(mode="json") + for deployment_schedule_create in deployment_schedule_create + ] + response = self.request( + "POST", + "/deployments/{id}/schedules", + path_params={"id": deployment_id}, + json=json, + ) + return DeploymentSchedule.model_validate_list(response.json()) + + def read_deployment_schedules( + self, + deployment_id: "UUID", + ) -> list["DeploymentSchedule"]: + """ + Query the Prefect API for a deployment's schedules. + + Args: + deployment_id: the deployment ID + + Returns: + a list of DeploymentSchedule model representations of the deployment schedules + """ + from prefect.client.schemas.objects import DeploymentSchedule + + try: + response = self.request( + "GET", + "/deployments/{id}/schedules", + path_params={"id": deployment_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + return DeploymentSchedule.model_validate_list(response.json()) + + def update_deployment_schedule( + self, + deployment_id: "UUID", + schedule_id: "UUID", + active: bool | None = None, + schedule: "SCHEDULE_TYPES | None" = None, + ) -> None: + """ + Update a deployment schedule by ID. + + Args: + deployment_id: the deployment ID + schedule_id: the deployment schedule ID of interest + active: whether or not the schedule should be active + schedule: the cron, rrule, or interval schedule this deployment schedule should use + """ + from prefect.client.schemas.actions import DeploymentScheduleUpdate + + kwargs: dict[str, Any] = {} + if active is not None: + kwargs["active"] = active + if schedule is not None: + kwargs["schedule"] = schedule + + deployment_schedule_update = DeploymentScheduleUpdate(**kwargs) + json = deployment_schedule_update.model_dump(mode="json", exclude_unset=True) + + try: + self.request( + "PATCH", + "/deployments/{id}/schedules/{schedule_id}", + path_params={"id": deployment_id, "schedule_id": schedule_id}, + json=json, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def delete_deployment_schedule( + self, + deployment_id: "UUID", + schedule_id: "UUID", + ) -> None: + """ + Delete a deployment schedule. + + Args: + deployment_id: the deployment ID + schedule_id: the ID of the deployment schedule to delete. + + Raises: + RequestError: if the schedules were not deleted for any reason + """ + try: + self.request( + "DELETE", + "/deployments/{id}/schedules/{schedule_id}", + path_params={"id": deployment_id, "schedule_id": schedule_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def get_scheduled_flow_runs_for_deployments( + self, + deployment_ids: list["UUID"], + scheduled_before: "datetime.datetime | None" = None, + limit: int | None = None, + ) -> list["FlowRunResponse"]: + from prefect.client.schemas.responses import FlowRunResponse + + body: dict[str, Any] = dict(deployment_ids=[str(id) for id in deployment_ids]) + if scheduled_before: + body["scheduled_before"] = str(scheduled_before) + if limit: + body["limit"] = limit + + response = self.request( + "POST", + "/deployments/get_scheduled_flow_runs", + json=body, + ) + + return FlowRunResponse.model_validate_list(response.json()) + + def create_flow_run_from_deployment( + self, + deployment_id: "UUID", + *, + parameters: dict[str, Any] | None = None, + context: dict[str, Any] | None = None, + state: State[Any] | None = None, + name: str | None = None, + tags: Iterable[str] | None = None, + idempotency_key: str | None = None, + parent_task_run_id: "UUID | None" = None, + work_queue_name: str | None = None, + job_variables: dict[str, Any] | None = None, + labels: "KeyValueLabelsField | None" = None, + ) -> "FlowRun": + """ + Create a flow run for a deployment. + + Args: + deployment_id: The deployment ID to create the flow run from + parameters: Parameter overrides for this flow run. Merged with the + deployment defaults + context: Optional run context data + state: The initial state for the run. If not provided, defaults to + `Scheduled` for now. Should always be a `Scheduled` type. + name: An optional name for the flow run. If not provided, the server will + generate a name. + tags: An optional iterable of tags to apply to the flow run; these tags + are merged with the deployment's tags. + idempotency_key: Optional idempotency key for creation of the flow run. + If the key matches the key of an existing flow run, the existing run will + be returned instead of creating a new one. + parent_task_run_id: if a subflow run is being created, the placeholder task + run identifier in the parent flow + work_queue_name: An optional work queue name to add this run to. If not provided, + will default to the deployment's set work queue. If one is provided that does not + exist, a new work queue will be created within the deployment's work pool. + job_variables: Optional variables that will be supplied to the flow run job. + + Raises: + RequestError: if the Prefect API does not successfully create a run for any reason + + Returns: + The flow run model + """ + from prefect.client.schemas.actions import DeploymentFlowRunCreate + from prefect.client.schemas.objects import FlowRun + + parameters = parameters or {} + context = context or {} + state = state or Scheduled() + tags = tags or [] + + flow_run_create = DeploymentFlowRunCreate( + parameters=parameters, + context=context, + state=state.to_state_create(), + tags=list(tags), + name=name, + idempotency_key=idempotency_key, + parent_task_run_id=parent_task_run_id, + job_variables=job_variables, + labels=labels, + ) + + # done separately to avoid including this field in payloads sent to older API versions + if work_queue_name: + flow_run_create.work_queue_name = work_queue_name + + response = self.request( + "POST", + "/deployments/{id}/create_flow_run", + path_params={"id": deployment_id}, + json=flow_run_create.model_dump(mode="json", exclude_unset=True), + ) + return FlowRun.model_validate(response.json()) + + +class DeploymentAsyncClient(BaseAsyncClient): + async def create_deployment( + self, + flow_id: "UUID", + name: str, + version: str | None = None, + schedules: list["DeploymentScheduleCreate"] | None = None, + concurrency_limit: int | None = None, + concurrency_options: "ConcurrencyOptions | None" = None, + parameters: dict[str, Any] | None = None, + description: str | None = None, + work_queue_name: str | None = None, + work_pool_name: str | None = None, + tags: list[str] | None = None, + storage_document_id: "UUID | None" = None, + path: str | None = None, + entrypoint: str | None = None, + infrastructure_document_id: "UUID | None" = None, + parameter_openapi_schema: dict[str, Any] | None = None, + paused: bool | None = None, + pull_steps: list[dict[str, Any]] | None = None, + enforce_parameter_schema: bool | None = None, + job_variables: dict[str, Any] | None = None, + ) -> "UUID": + """ + Create a deployment. + + Args: + flow_id: the flow ID to create a deployment for + name: the name of the deployment + version: an optional version string for the deployment + tags: an optional list of tags to apply to the deployment + storage_document_id: an reference to the storage block document + used for the deployed flow + infrastructure_document_id: an reference to the infrastructure block document + to use for this deployment + job_variables: A dictionary of dot delimited infrastructure overrides that + will be applied at runtime; for example `env.CONFIG_KEY=config_value` or + `namespace='prefect'`. This argument was previously named `infra_overrides`. + Both arguments are supported for backwards compatibility. + + Raises: + RequestError: if the deployment was not created for any reason + + Returns: + the ID of the deployment in the backend + """ + from uuid import UUID + + from prefect.client.schemas.actions import DeploymentCreate + + if parameter_openapi_schema is None: + parameter_openapi_schema = {} + deployment_create = DeploymentCreate( + flow_id=flow_id, + name=name, + version=version, + parameters=dict(parameters or {}), + tags=list(tags or []), + work_queue_name=work_queue_name, + description=description, + storage_document_id=storage_document_id, + path=path, + entrypoint=entrypoint, + infrastructure_document_id=infrastructure_document_id, + job_variables=dict(job_variables or {}), + parameter_openapi_schema=parameter_openapi_schema, + paused=paused, + schedules=schedules or [], + concurrency_limit=concurrency_limit, + concurrency_options=concurrency_options, + pull_steps=pull_steps, + enforce_parameter_schema=enforce_parameter_schema, + ) + + if work_pool_name is not None: + deployment_create.work_pool_name = work_pool_name + + # Exclude newer fields that are not set to avoid compatibility issues + exclude = { + field + for field in ["work_pool_name", "work_queue_name"] + if field not in deployment_create.model_fields_set + } + + if deployment_create.paused is None: + exclude.add("paused") + + if deployment_create.pull_steps is None: + exclude.add("pull_steps") + + if deployment_create.enforce_parameter_schema is None: + exclude.add("enforce_parameter_schema") + + json = deployment_create.model_dump(mode="json", exclude=exclude) + response = await self.request( + "POST", + "/deployments/", + json=json, + ) + deployment_id = response.json().get("id") + if not deployment_id: + raise RequestError(f"Malformed response: {response}") + + return UUID(deployment_id) + + async def set_deployment_paused_state( + self, deployment_id: "UUID", paused: bool + ) -> None: + await self.request( + "PATCH", + "/deployments/{id}", + path_params={"id": deployment_id}, + json={"paused": paused}, + ) + + async def update_deployment( + self, + deployment_id: "UUID", + deployment: "DeploymentUpdate", + ) -> None: + await self.request( + "PATCH", + "/deployments/{id}", + path_params={"id": deployment_id}, + json=deployment.model_dump(mode="json", exclude_unset=True), + ) + + async def _create_deployment_from_schema( + self, schema: "DeploymentCreate" + ) -> "UUID": + """ + Create a deployment from a prepared `DeploymentCreate` schema. + """ + from uuid import UUID + + # TODO: We are likely to remove this method once we have considered the + # packaging interface for deployments further. + response = await self.request( + "POST", "/deployments/", json=schema.model_dump(mode="json") + ) + deployment_id = response.json().get("id") + if not deployment_id: + raise RequestError(f"Malformed response: {response}") + + return UUID(deployment_id) + + async def read_deployment( + self, + deployment_id: Union["UUID", str], + ) -> "DeploymentResponse": + """ + Query the Prefect API for a deployment by id. + + Args: + deployment_id: the deployment ID of interest + + Returns: + a [Deployment model][prefect.client.schemas.objects.Deployment] representation of the deployment + """ + from uuid import UUID + + from prefect.client.schemas.responses import DeploymentResponse + + if not isinstance(deployment_id, UUID): + try: + deployment_id = UUID(deployment_id) + except ValueError: + raise ValueError(f"Invalid deployment ID: {deployment_id}") + + try: + response = await self.request( + "GET", + "/deployments/{id}", + path_params={"id": deployment_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + return DeploymentResponse.model_validate(response.json()) + + async def read_deployment_by_name( + self, + name: str, + ) -> "DeploymentResponse": + """ + Query the Prefect API for a deployment by name. + + Args: + name: A deployed flow's name: / + + Raises: + ObjectNotFound: If request returns 404 + RequestError: If request fails + + Returns: + a Deployment model representation of the deployment + """ + from prefect.client.schemas.responses import DeploymentResponse + + flow_name, deployment_name = name.split("/") + try: + response = await self.request( + "GET", + "/deployments/name/{flow_name}/{deployment_name}", + path_params={ + "flow_name": flow_name, + "deployment_name": deployment_name, + }, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + return DeploymentResponse.model_validate(response.json()) + + async def read_deployments( + self, + *, + flow_filter: "FlowFilter | None" = None, + flow_run_filter: "FlowRunFilter | None" = None, + task_run_filter: "TaskRunFilter | None" = None, + deployment_filter: "DeploymentFilter | None" = None, + work_pool_filter: "WorkPoolFilter | None" = None, + work_queue_filter: "WorkQueueFilter | None" = None, + limit: int | None = None, + sort: "DeploymentSort | None" = None, + offset: int = 0, + ) -> list["DeploymentResponse"]: + """ + Query the Prefect API for deployments. Only deployments matching all + the provided criteria will be returned. + + Args: + flow_filter: filter criteria for flows + flow_run_filter: filter criteria for flow runs + task_run_filter: filter criteria for task runs + deployment_filter: filter criteria for deployments + work_pool_filter: filter criteria for work pools + work_queue_filter: filter criteria for work pool queues + limit: a limit for the deployment query + offset: an offset for the deployment query + + Returns: + a list of Deployment model representations + of the deployments + """ + from prefect.client.schemas.responses import DeploymentResponse + + body: dict[str, Any] = { + "flows": flow_filter.model_dump(mode="json") if flow_filter else None, + "flow_runs": ( + flow_run_filter.model_dump(mode="json", exclude_unset=True) + if flow_run_filter + else None + ), + "task_runs": ( + task_run_filter.model_dump(mode="json") if task_run_filter else None + ), + "deployments": ( + deployment_filter.model_dump(mode="json") if deployment_filter else None + ), + "work_pools": ( + work_pool_filter.model_dump(mode="json") if work_pool_filter else None + ), + "work_pool_queues": ( + work_queue_filter.model_dump(mode="json") if work_queue_filter else None + ), + "limit": limit, + "offset": offset, + "sort": sort, + } + + response = await self.request("POST", "/deployments/filter", json=body) + return DeploymentResponse.model_validate_list(response.json()) + + async def delete_deployment( + self, + deployment_id: "UUID", + ) -> None: + """ + Delete deployment by id. + + Args: + deployment_id: The deployment id of interest. + Raises: + ObjectNotFound: If request returns 404 + RequestError: If requests fails + """ + try: + await self.request( + "DELETE", + "/deployments/{id}", + path_params={"id": deployment_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def create_deployment_schedules( + self, + deployment_id: "UUID", + schedules: list[tuple["SCHEDULE_TYPES", bool]], + ) -> list["DeploymentSchedule"]: + """ + Create deployment schedules. + + Args: + deployment_id: the deployment ID + schedules: a list of tuples containing the schedule to create + and whether or not it should be active. + + Raises: + RequestError: if the schedules were not created for any reason + + Returns: + the list of schedules created in the backend + """ + from prefect.client.schemas.actions import DeploymentScheduleCreate + from prefect.client.schemas.objects import DeploymentSchedule + + deployment_schedule_create = [ + DeploymentScheduleCreate(schedule=schedule[0], active=schedule[1]) + for schedule in schedules + ] + + json = [ + deployment_schedule_create.model_dump(mode="json") + for deployment_schedule_create in deployment_schedule_create + ] + response = await self.request( + "POST", + "/deployments/{id}/schedules", + path_params={"id": deployment_id}, + json=json, + ) + return DeploymentSchedule.model_validate_list(response.json()) + + async def read_deployment_schedules( + self, + deployment_id: "UUID", + ) -> list["DeploymentSchedule"]: + """ + Query the Prefect API for a deployment's schedules. + + Args: + deployment_id: the deployment ID + + Returns: + a list of DeploymentSchedule model representations of the deployment schedules + """ + from prefect.client.schemas.objects import DeploymentSchedule + + try: + response = await self.request( + "GET", + "/deployments/{id}/schedules", + path_params={"id": deployment_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + return DeploymentSchedule.model_validate_list(response.json()) + + async def update_deployment_schedule( + self, + deployment_id: "UUID", + schedule_id: "UUID", + active: bool | None = None, + schedule: "SCHEDULE_TYPES | None" = None, + ) -> None: + """ + Update a deployment schedule by ID. + + Args: + deployment_id: the deployment ID + schedule_id: the deployment schedule ID of interest + active: whether or not the schedule should be active + schedule: the cron, rrule, or interval schedule this deployment schedule should use + """ + from prefect.client.schemas.actions import DeploymentScheduleUpdate + + kwargs: dict[str, Any] = {} + if active is not None: + kwargs["active"] = active + if schedule is not None: + kwargs["schedule"] = schedule + + deployment_schedule_update = DeploymentScheduleUpdate(**kwargs) + json = deployment_schedule_update.model_dump(mode="json", exclude_unset=True) + + try: + await self.request( + "PATCH", + "/deployments/{id}/schedules/{schedule_id}", + path_params={"id": deployment_id, "schedule_id": schedule_id}, + json=json, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def delete_deployment_schedule( + self, + deployment_id: "UUID", + schedule_id: "UUID", + ) -> None: + """ + Delete a deployment schedule. + + Args: + deployment_id: the deployment ID + schedule_id: the ID of the deployment schedule to delete. + + Raises: + RequestError: if the schedules were not deleted for any reason + """ + try: + await self.request( + "DELETE", + "/deployments/{id}/schedules/{schedule_id}", + path_params={"id": deployment_id, "schedule_id": schedule_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def get_scheduled_flow_runs_for_deployments( + self, + deployment_ids: list["UUID"], + scheduled_before: "datetime.datetime | None" = None, + limit: int | None = None, + ) -> list["FlowRunResponse"]: + from prefect.client.schemas.responses import FlowRunResponse + + body: dict[str, Any] = dict(deployment_ids=[str(id) for id in deployment_ids]) + if scheduled_before: + body["scheduled_before"] = str(scheduled_before) + if limit: + body["limit"] = limit + + response = await self.request( + "POST", + "/deployments/get_scheduled_flow_runs", + json=body, + ) + + return FlowRunResponse.model_validate_list(response.json()) + + async def create_flow_run_from_deployment( + self, + deployment_id: "UUID", + *, + parameters: dict[str, Any] | None = None, + context: dict[str, Any] | None = None, + state: State[Any] | None = None, + name: str | None = None, + tags: Iterable[str] | None = None, + idempotency_key: str | None = None, + parent_task_run_id: "UUID | None" = None, + work_queue_name: str | None = None, + job_variables: dict[str, Any] | None = None, + labels: "KeyValueLabelsField | None" = None, + ) -> "FlowRun": + """ + Create a flow run for a deployment. + + Args: + deployment_id: The deployment ID to create the flow run from + parameters: Parameter overrides for this flow run. Merged with the + deployment defaults + context: Optional run context data + state: The initial state for the run. If not provided, defaults to + `Scheduled` for now. Should always be a `Scheduled` type. + name: An optional name for the flow run. If not provided, the server will + generate a name. + tags: An optional iterable of tags to apply to the flow run; these tags + are merged with the deployment's tags. + idempotency_key: Optional idempotency key for creation of the flow run. + If the key matches the key of an existing flow run, the existing run will + be returned instead of creating a new one. + parent_task_run_id: if a subflow run is being created, the placeholder task + run identifier in the parent flow + work_queue_name: An optional work queue name to add this run to. If not provided, + will default to the deployment's set work queue. If one is provided that does not + exist, a new work queue will be created within the deployment's work pool. + job_variables: Optional variables that will be supplied to the flow run job. + + Raises: + RequestError: if the Prefect API does not successfully create a run for any reason + + Returns: + The flow run model + """ + from prefect.client.schemas.actions import DeploymentFlowRunCreate + from prefect.client.schemas.objects import FlowRun + + parameters = parameters or {} + context = context or {} + state = state or Scheduled() + tags = tags or [] + + flow_run_create = DeploymentFlowRunCreate( + parameters=parameters, + context=context, + state=state.to_state_create(), + tags=list(tags), + name=name, + idempotency_key=idempotency_key, + parent_task_run_id=parent_task_run_id, + job_variables=job_variables, + labels=labels, + ) + + # done separately to avoid including this field in payloads sent to older API versions + if work_queue_name: + flow_run_create.work_queue_name = work_queue_name + + response = await self.request( + "POST", + "/deployments/{id}/create_flow_run", + path_params={"id": deployment_id}, + json=flow_run_create.model_dump(mode="json", exclude_unset=True), + ) + return FlowRun.model_validate(response.json()) From d540c1114e878137108fed3057f8006b285c7771 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 15:34:29 -0600 Subject: [PATCH 02/19] rm unnecessary test, add split under try block --- .../orchestration/_deployments/client.py | 4 ++-- tests/client/test_prefect_client.py | 21 ------------------- 2 files changed, 2 insertions(+), 23 deletions(-) diff --git a/src/prefect/client/orchestration/_deployments/client.py b/src/prefect/client/orchestration/_deployments/client.py index 0cc6b58dc220..6cab0ae9dc6c 100644 --- a/src/prefect/client/orchestration/_deployments/client.py +++ b/src/prefect/client/orchestration/_deployments/client.py @@ -241,8 +241,8 @@ def read_deployment_by_name( """ from prefect.client.schemas.responses import DeploymentResponse - flow_name, deployment_name = name.split("/") try: + flow_name, deployment_name = name.split("/") response = self.request( "GET", "/deployments/name/{flow_name}/{deployment_name}", @@ -780,8 +780,8 @@ async def read_deployment_by_name( """ from prefect.client.schemas.responses import DeploymentResponse - flow_name, deployment_name = name.split("/") try: + flow_name, deployment_name = name.split("/") response = await self.request( "GET", "/deployments/name/{flow_name}/{deployment_name}", diff --git a/tests/client/test_prefect_client.py b/tests/client/test_prefect_client.py index fb3a0dd056b5..2b73e82f9521 100644 --- a/tests/client/test_prefect_client.py +++ b/tests/client/test_prefect_client.py @@ -841,27 +841,6 @@ def moo_deng(): assert len(deployment_responses) == 0 -async def test_read_deployment_by_name_fails_with_helpful_suggestion(prefect_client): - """this is a regression test for https://github.com/PrefectHQ/prefect/issues/15571""" - - @flow - def moo_deng(): - pass - - flow_id = await prefect_client.create_flow(moo_deng) - - await prefect_client.create_deployment( - flow_id=flow_id, - name="moisturized-deployment", - ) - - with pytest.raises( - prefect.exceptions.ObjectNotFound, - match="Deployment 'moo_deng/moisturized-deployment' not found; did you mean 'moo-deng/moisturized-deployment'?", - ): - await prefect_client.read_deployment_by_name("moo_deng/moisturized-deployment") - - async def test_create_then_delete_deployment(prefect_client): @flow def foo(): From 777c4aee5a48bdfa919aa79d477f5c6fd19c1b1f Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 16:12:52 -0600 Subject: [PATCH 03/19] account for ValueError --- .../client/orchestration/_deployments/client.py | 16 ++++++++++++---- tests/client/test_prefect_client.py | 2 +- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/prefect/client/orchestration/_deployments/client.py b/src/prefect/client/orchestration/_deployments/client.py index 6cab0ae9dc6c..dbf5b5d31536 100644 --- a/src/prefect/client/orchestration/_deployments/client.py +++ b/src/prefect/client/orchestration/_deployments/client.py @@ -251,9 +251,13 @@ def read_deployment_by_name( "deployment_name": deployment_name, }, ) - except HTTPStatusError as e: - if e.response.status_code == 404: + except (HTTPStatusError, ValueError) as e: + if isinstance(e, HTTPStatusError) and e.response.status_code == 404: raise ObjectNotFound(http_exc=e) from e + elif isinstance(e, ValueError): + raise ValueError( + f"Invalid deployment name format: {name}. Expected format: /" + ) from e else: raise @@ -790,9 +794,13 @@ async def read_deployment_by_name( "deployment_name": deployment_name, }, ) - except HTTPStatusError as e: - if e.response.status_code == 404: + except (HTTPStatusError, ValueError) as e: + if isinstance(e, HTTPStatusError) and e.response.status_code == 404: raise ObjectNotFound(http_exc=e) from e + elif isinstance(e, ValueError): + raise ValueError( + f"Invalid deployment name format: {name}. Expected format: /" + ) from e else: raise diff --git a/tests/client/test_prefect_client.py b/tests/client/test_prefect_client.py index 2b73e82f9521..438371ea2f3d 100644 --- a/tests/client/test_prefect_client.py +++ b/tests/client/test_prefect_client.py @@ -859,7 +859,7 @@ def foo(): async def test_read_nonexistent_deployment_by_name(prefect_client): - with pytest.raises(prefect.exceptions.ObjectNotFound): + with pytest.raises((prefect.exceptions.ObjectNotFound, ValueError)): await prefect_client.read_deployment_by_name("not-a-real-deployment") From f1b7bb5531c719dafca636d8331df432bb1061bf Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 16:46:25 -0600 Subject: [PATCH 04/19] init --- src/prefect/client/orchestration/__init__.py | 131 +------ .../orchestration/_automations/__init__.py | 0 .../orchestration/_automations/client.py | 328 ++++++++++++++++++ 3 files changed, 334 insertions(+), 125 deletions(-) create mode 100644 src/prefect/client/orchestration/_automations/__init__.py create mode 100644 src/prefect/client/orchestration/_automations/client.py diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index cf0673d43620..f19cf770dbf3 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -44,6 +44,10 @@ DeploymentClient, DeploymentAsyncClient, ) +from prefect.client.orchestration._automations.client import ( + AutomationClient, + AutomationAsyncClient, +) import prefect import prefect.exceptions @@ -247,6 +251,7 @@ class PrefectClient( VariableAsyncClient, ConcurrencyLimitAsyncClient, DeploymentAsyncClient, + AutomationAsyncClient, ): """ An asynchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -2270,131 +2275,6 @@ async def delete_flow_run_input(self, flow_run_id: UUID, key: str) -> None: response = await self._client.delete(f"/flow_runs/{flow_run_id}/input/{key}") response.raise_for_status() - async def create_automation(self, automation: AutomationCore) -> UUID: - """Creates an automation in Prefect Cloud.""" - response = await self._client.post( - "/automations/", - json=automation.model_dump(mode="json"), - ) - - return UUID(response.json()["id"]) - - async def update_automation( - self, automation_id: UUID, automation: AutomationCore - ) -> None: - """Updates an automation in Prefect Cloud.""" - response = await self._client.put( - f"/automations/{automation_id}", - json=automation.model_dump(mode="json", exclude_unset=True), - ) - response.raise_for_status - - async def read_automations(self) -> list[Automation]: - response = await self._client.post("/automations/filter") - response.raise_for_status() - return pydantic.TypeAdapter(list[Automation]).validate_python(response.json()) - - async def find_automation( - self, id_or_name: Union[str, UUID] - ) -> Optional[Automation]: - if isinstance(id_or_name, str): - name = id_or_name - try: - id = UUID(id_or_name) - except ValueError: - id = None - else: - id = id_or_name - name = str(id) - - if id: - try: - automation = await self.read_automation(id) - return automation - except prefect.exceptions.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - - automations = await self.read_automations() - - # Look for it by an exact name - for automation in automations: - if automation.name == name: - return automation - - # Look for it by a case-insensitive name - for automation in automations: - if automation.name.lower() == name.lower(): - return automation - - return None - - async def read_automation( - self, automation_id: Union[UUID, str] - ) -> Optional[Automation]: - response = await self._client.get(f"/automations/{automation_id}") - if response.status_code == 404: - return None - response.raise_for_status() - return Automation.model_validate(response.json()) - - async def read_automations_by_name(self, name: str) -> list[Automation]: - """ - Query the Prefect API for an automation by name. Only automations matching the provided name will be returned. - - Args: - name: the name of the automation to query - - Returns: - a list of Automation model representations of the automations - """ - automation_filter = filters.AutomationFilter( - name=filters.AutomationFilterName(any_=[name]) - ) - - response = await self._client.post( - "/automations/filter", - json={ - "sort": sorting.AutomationSort.UPDATED_DESC, - "automations": automation_filter.model_dump(mode="json") - if automation_filter - else None, - }, - ) - - response.raise_for_status() - - return pydantic.TypeAdapter(list[Automation]).validate_python(response.json()) - - async def pause_automation(self, automation_id: UUID) -> None: - response = await self._client.patch( - f"/automations/{automation_id}", json={"enabled": False} - ) - response.raise_for_status() - - async def resume_automation(self, automation_id: UUID) -> None: - response = await self._client.patch( - f"/automations/{automation_id}", json={"enabled": True} - ) - response.raise_for_status() - - async def delete_automation(self, automation_id: UUID) -> None: - response = await self._client.delete(f"/automations/{automation_id}") - if response.status_code == 404: - return - - response.raise_for_status() - - async def read_resource_related_automations( - self, resource_id: str - ) -> list[Automation]: - response = await self._client.get(f"/automations/related-to/{resource_id}") - response.raise_for_status() - return pydantic.TypeAdapter(list[Automation]).validate_python(response.json()) - - async def delete_resource_owned_automations(self, resource_id: str) -> None: - await self._client.delete(f"/automations/owned-by/{resource_id}") - async def api_version(self) -> str: res = await self._client.get("/admin/version") return res.json() @@ -2510,6 +2390,7 @@ class SyncPrefectClient( VariableClient, ConcurrencyLimitClient, DeploymentClient, + AutomationClient, ): """ A synchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). diff --git a/src/prefect/client/orchestration/_automations/__init__.py b/src/prefect/client/orchestration/_automations/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_automations/client.py b/src/prefect/client/orchestration/_automations/client.py new file mode 100644 index 000000000000..cf973f067230 --- /dev/null +++ b/src/prefect/client/orchestration/_automations/client.py @@ -0,0 +1,328 @@ +from __future__ import annotations + +from typing import TYPE_CHECKING + +from httpx import HTTPStatusError + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.exceptions import ObjectNotFound + +if TYPE_CHECKING: + from uuid import UUID + + from prefect.client.schemas.sorting import AutomationSort + from prefect.events.schemas.automations import Automation, AutomationCore + + +class AutomationClient(BaseClient): + def create_automation(self, automation: "AutomationCore") -> "UUID": + """Creates an automation in Prefect Cloud.""" + response = self.request( + "POST", + "/automations/", + json=automation.model_dump(mode="json"), + ) + from uuid import UUID + + return UUID(response.json()["id"]) + + def update_automation( + self, automation_id: "UUID", automation: "AutomationCore" + ) -> None: + """Updates an automation in Prefect Cloud.""" + response = self.request( + "PUT", + "/automations/{id}", + path_params={"id": automation_id}, + json=automation.model_dump(mode="json", exclude_unset=True), + ) + response.raise_for_status() + + def read_automations(self) -> list["Automation"]: + response = self.request("POST", "/automations/filter") + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate_list(response.json()) + + def find_automation(self, id_or_name: "str | UUID") -> "Automation | None": + if isinstance(id_or_name, str): + name = id_or_name + try: + id = UUID(id_or_name) + except ValueError: + id = None + else: + id = id_or_name + name = str(id) + + if id: + try: + automation = self.read_automation(id) + return automation + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + + automations = self.read_automations() + + # Look for it by an exact name + for automation in automations: + if automation.name == name: + return automation + + # Look for it by a case-insensitive name + for automation in automations: + if automation.name.lower() == name.lower(): + return automation + + return None + + def read_automation(self, automation_id: "UUID | str") -> "Automation | None": + response = self.request( + "GET", "/automations/{id}", path_params={"id": automation_id} + ) + if response.status_code == 404: + return None + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate(response.json()) + + def read_automations_by_name(self, name: str) -> list["Automation"]: + """ + Query the Prefect API for an automation by name. Only automations matching the provided name will be returned. + + Args: + name: the name of the automation to query + + Returns: + a list of Automation model representations of the automations + """ + from prefect.events.filters import ( + AutomationFilter, + AutomationFilterName, + ) + + automation_filter = AutomationFilter(name=AutomationFilterName(any_=[name])) + + response = self.request( + "POST", + "/automations/filter", + json={ + "sort": AutomationSort.UPDATED_DESC, + "automations": automation_filter.model_dump(mode="json") + if automation_filter + else None, + }, + ) + + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate_list(response.json()) + + def pause_automation(self, automation_id: "UUID") -> None: + response = self.request( + "PATCH", + "/automations/{id}", + path_params={"id": automation_id}, + json={"enabled": False}, + ) + response.raise_for_status() + + def resume_automation(self, automation_id: "UUID") -> None: + response = self.request( + "PATCH", + "/automations/{id}", + path_params={"id": automation_id}, + json={"enabled": True}, + ) + response.raise_for_status() + + def delete_automation(self, automation_id: "UUID") -> None: + response = self.request( + "DELETE", + "/automations/{id}", + path_params={"id": automation_id}, + ) + if response.status_code == 404: + return + + response.raise_for_status() + + def read_resource_related_automations(self, resource_id: str) -> list["Automation"]: + response = self.request( + "GET", + "/automations/related-to/{resource_id}", + path_params={"resource_id": resource_id}, + ) + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate_list(response.json()) + + def delete_resource_owned_automations(self, resource_id: str) -> None: + self.request( + "DELETE", + "/automations/owned-by/{resource_id}", + path_params={"resource_id": resource_id}, + ) + + +class AutomationAsyncClient(BaseAsyncClient): + async def create_automation(self, automation: "AutomationCore") -> "UUID": + """Creates an automation in Prefect Cloud.""" + response = await self.request( + "POST", + "/automations/", + json=automation.model_dump(mode="json"), + ) + from uuid import UUID + + return UUID(response.json()["id"]) + + async def update_automation( + self, automation_id: "UUID", automation: "AutomationCore" + ) -> None: + """Updates an automation in Prefect Cloud.""" + response = await self.request( + "PUT", + "/automations/{id}", + path_params={"id": automation_id}, + json=automation.model_dump(mode="json", exclude_unset=True), + ) + response.raise_for_status() + + async def read_automations(self) -> list["Automation"]: + response = await self.request("POST", "/automations/filter") + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate_list(response.json()) + + async def find_automation(self, id_or_name: "str | UUID") -> "Automation | None": + if isinstance(id_or_name, str): + name = id_or_name + try: + id = UUID(id_or_name) + except ValueError: + id = None + else: + id = id_or_name + name = str(id) + + if id: + try: + automation = await self.read_automation(id) + return automation + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + + automations = await self.read_automations() + + # Look for it by an exact name + for automation in automations: + if automation.name == name: + return automation + + # Look for it by a case-insensitive name + for automation in automations: + if automation.name.lower() == name.lower(): + return automation + + return None + + async def read_automation(self, automation_id: "UUID | str") -> "Automation | None": + response = await self.request( + "GET", "/automations/{id}", path_params={"id": automation_id} + ) + if response.status_code == 404: + return None + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate(response.json()) + + async def read_automations_by_name(self, name: str) -> list["Automation"]: + """ + Query the Prefect API for an automation by name. Only automations matching the provided name will be returned. + + Args: + name: the name of the automation to query + + Returns: + a list of Automation model representations of the automations + """ + from prefect.events.filters import ( + AutomationFilter, + AutomationFilterName, + ) + + automation_filter = AutomationFilter(name=AutomationFilterName(any_=[name])) + + response = await self.request( + "POST", + "/automations/filter", + json={ + "sort": AutomationSort.UPDATED_DESC, + "automations": automation_filter.model_dump(mode="json") + if automation_filter + else None, + }, + ) + + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate_list(response.json()) + + async def pause_automation(self, automation_id: "UUID") -> None: + response = await self.request( + "PATCH", + "/automations/{id}", + path_params={"id": automation_id}, + json={"enabled": False}, + ) + response.raise_for_status() + + async def resume_automation(self, automation_id: "UUID") -> None: + response = await self.request( + "PATCH", + "/automations/{id}", + path_params={"id": automation_id}, + json={"enabled": True}, + ) + response.raise_for_status() + + async def delete_automation(self, automation_id: "UUID") -> None: + response = await self.request( + "DELETE", + "/automations/{id}", + path_params={"id": automation_id}, + ) + if response.status_code == 404: + return + + response.raise_for_status() + + async def read_resource_related_automations( + self, resource_id: str + ) -> list["Automation"]: + response = await self.request( + "GET", + "/automations/related-to/{resource_id}", + path_params={"resource_id": resource_id}, + ) + response.raise_for_status() + from prefect.events.schemas.automations import Automation + + return Automation.model_validate_list(response.json()) + + async def delete_resource_owned_automations(self, resource_id: str) -> None: + await self.request( + "DELETE", + "/automations/owned-by/{resource_id}", + path_params={"resource_id": resource_id}, + ) From 3ac509e4a1222d111f255ded38ca2e5a8a7e2306 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 17:01:31 -0600 Subject: [PATCH 05/19] Update client.py --- src/prefect/client/orchestration/_automations/client.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/prefect/client/orchestration/_automations/client.py b/src/prefect/client/orchestration/_automations/client.py index cf973f067230..ddd61d422586 100644 --- a/src/prefect/client/orchestration/_automations/client.py +++ b/src/prefect/client/orchestration/_automations/client.py @@ -10,7 +10,6 @@ if TYPE_CHECKING: from uuid import UUID - from prefect.client.schemas.sorting import AutomationSort from prefect.events.schemas.automations import Automation, AutomationCore @@ -99,6 +98,7 @@ def read_automations_by_name(self, name: str) -> list["Automation"]: Returns: a list of Automation model representations of the automations """ + from prefect.client.schemas.sorting import AutomationSort from prefect.events.filters import ( AutomationFilter, AutomationFilterName, @@ -255,6 +255,7 @@ async def read_automations_by_name(self, name: str) -> list["Automation"]: Returns: a list of Automation model representations of the automations """ + from prefect.client.schemas.sorting import AutomationSort from prefect.events.filters import ( AutomationFilter, AutomationFilterName, From 600d3b75db2e413e9e825fef5a44fb4cf6f3fef3 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 17:01:49 -0600 Subject: [PATCH 06/19] init --- src/prefect/client/orchestration/__init__.py | 200 ++------------- .../orchestration/_flow_runs/__init__.py | 0 .../client/orchestration/_flow_runs/client.py | 11 + .../client/orchestration/_flows/__init__.py | 0 .../client/orchestration/_flows/client.py | 228 ++++++++++++++++++ 5 files changed, 253 insertions(+), 186 deletions(-) create mode 100644 src/prefect/client/orchestration/_flow_runs/__init__.py create mode 100644 src/prefect/client/orchestration/_flow_runs/client.py create mode 100644 src/prefect/client/orchestration/_flows/__init__.py create mode 100644 src/prefect/client/orchestration/_flows/client.py diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index f19cf770dbf3..67abacf51a95 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -49,12 +49,21 @@ AutomationAsyncClient, ) +from prefect.client.orchestration._flows.client import ( + FlowClient, + FlowAsyncClient, +) +from prefect.client.orchestration._flow_runs.client import ( + FlowRunClient, + FlowRunAsyncClient, +) + import prefect import prefect.exceptions import prefect.settings import prefect.states from prefect.client.constants import SERVER_API_VERSION -from prefect.client.schemas import FlowRun, OrchestrationResult, TaskRun, sorting +from prefect.client.schemas import FlowRun, OrchestrationResult, TaskRun from prefect.client.schemas.actions import ( BlockDocumentCreate, BlockDocumentUpdate, @@ -110,8 +119,6 @@ FlowSort, TaskRunSort, ) -from prefect.events import filters -from prefect.events.schemas.automations import Automation, AutomationCore from prefect.logging import get_logger from prefect.settings import ( PREFECT_API_AUTH_STRING, @@ -252,6 +259,8 @@ class PrefectClient( ConcurrencyLimitAsyncClient, DeploymentAsyncClient, AutomationAsyncClient, + FlowAsyncClient, + FlowRunAsyncClient, ): """ An asynchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -464,151 +473,6 @@ async def hello(self) -> httpx.Response: """ return await self._client.get("/hello") - async def create_flow(self, flow: "FlowObject[Any, Any]") -> UUID: - """ - Create a flow in the Prefect API. - - Args: - flow: a [Flow][prefect.flows.Flow] object - - Raises: - httpx.RequestError: if a flow was not created for any reason - - Returns: - the ID of the flow in the backend - """ - return await self.create_flow_from_name(flow.name) - - async def create_flow_from_name(self, flow_name: str) -> UUID: - """ - Create a flow in the Prefect API. - - Args: - flow_name: the name of the new flow - - Raises: - httpx.RequestError: if a flow was not created for any reason - - Returns: - the ID of the flow in the backend - """ - flow_data = FlowCreate(name=flow_name) - response = await self._client.post( - "/flows/", json=flow_data.model_dump(mode="json") - ) - - flow_id = response.json().get("id") - if not flow_id: - raise httpx.RequestError(f"Malformed response: {response}") - - # Return the id of the created flow - return UUID(flow_id) - - async def read_flow(self, flow_id: UUID) -> Flow: - """ - Query the Prefect API for a flow by id. - - Args: - flow_id: the flow ID of interest - - Returns: - a [Flow model][prefect.client.schemas.objects.Flow] representation of the flow - """ - response = await self._client.get(f"/flows/{flow_id}") - return Flow.model_validate(response.json()) - - async def delete_flow(self, flow_id: UUID) -> None: - """ - Delete a flow by UUID. - - Args: - flow_id: ID of the flow to be deleted - Raises: - prefect.exceptions.ObjectNotFound: If request returns 404 - httpx.RequestError: If requests fail - """ - try: - await self._client.delete(f"/flows/{flow_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - - async def read_flows( - self, - *, - flow_filter: Optional[FlowFilter] = None, - flow_run_filter: Optional[FlowRunFilter] = None, - task_run_filter: Optional[TaskRunFilter] = None, - deployment_filter: Optional[DeploymentFilter] = None, - work_pool_filter: Optional[WorkPoolFilter] = None, - work_queue_filter: Optional[WorkQueueFilter] = None, - sort: Optional[FlowSort] = None, - limit: Optional[int] = None, - offset: int = 0, - ) -> list[Flow]: - """ - Query the Prefect API for flows. Only flows matching all criteria will - be returned. - - Args: - flow_filter: filter criteria for flows - flow_run_filter: filter criteria for flow runs - task_run_filter: filter criteria for task runs - deployment_filter: filter criteria for deployments - work_pool_filter: filter criteria for work pools - work_queue_filter: filter criteria for work pool queues - sort: sort criteria for the flows - limit: limit for the flow query - offset: offset for the flow query - - Returns: - a list of Flow model representations of the flows - """ - body: dict[str, Any] = { - "flows": flow_filter.model_dump(mode="json") if flow_filter else None, - "flow_runs": ( - flow_run_filter.model_dump(mode="json", exclude_unset=True) - if flow_run_filter - else None - ), - "task_runs": ( - task_run_filter.model_dump(mode="json") if task_run_filter else None - ), - "deployments": ( - deployment_filter.model_dump(mode="json") if deployment_filter else None - ), - "work_pools": ( - work_pool_filter.model_dump(mode="json") if work_pool_filter else None - ), - "work_queues": ( - work_queue_filter.model_dump(mode="json") if work_queue_filter else None - ), - "sort": sort, - "limit": limit, - "offset": offset, - } - - response = await self._client.post("/flows/filter", json=body) - return pydantic.TypeAdapter(list[Flow]).validate_python(response.json()) - - async def read_flow_by_name( - self, - flow_name: str, - ) -> Flow: - """ - Query the Prefect API for a flow by name. - - Args: - flow_name: the name of a flow - - Returns: - a fully hydrated Flow model - """ - response = await self._client.get(f"/flows/name/{flow_name}") - return Flow.model_validate(response.json()) - async def create_flow_run( self, flow: "FlowObject[Any, R]", @@ -2391,6 +2255,8 @@ class SyncPrefectClient( ConcurrencyLimitClient, DeploymentClient, AutomationClient, + FlowClient, + FlowRunClient, ): """ A synchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -2649,44 +2515,6 @@ def raise_for_api_version_mismatch(self) -> None: f"Major versions must match." ) - def create_flow(self, flow: "FlowObject[Any, Any]") -> UUID: - """ - Create a flow in the Prefect API. - - Args: - flow: a [Flow][prefect.flows.Flow] object - - Raises: - httpx.RequestError: if a flow was not created for any reason - - Returns: - the ID of the flow in the backend - """ - return self.create_flow_from_name(flow.name) - - def create_flow_from_name(self, flow_name: str) -> UUID: - """ - Create a flow in the Prefect API. - - Args: - flow_name: the name of the new flow - - Raises: - httpx.RequestError: if a flow was not created for any reason - - Returns: - the ID of the flow in the backend - """ - flow_data = FlowCreate(name=flow_name) - response = self._client.post("/flows/", json=flow_data.model_dump(mode="json")) - - flow_id = response.json().get("id") - if not flow_id: - raise httpx.RequestError(f"Malformed response: {response}") - - # Return the id of the created flow - return UUID(flow_id) - def create_flow_run( self, flow: "FlowObject[Any, R]", diff --git a/src/prefect/client/orchestration/_flow_runs/__init__.py b/src/prefect/client/orchestration/_flow_runs/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_flow_runs/client.py b/src/prefect/client/orchestration/_flow_runs/client.py new file mode 100644 index 000000000000..73de2774e618 --- /dev/null +++ b/src/prefect/client/orchestration/_flow_runs/client.py @@ -0,0 +1,11 @@ +from __future__ import annotations + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient + + +class FlowRunClient(BaseClient): + pass + + +class FlowRunAsyncClient(BaseAsyncClient): + pass diff --git a/src/prefect/client/orchestration/_flows/__init__.py b/src/prefect/client/orchestration/_flows/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_flows/client.py b/src/prefect/client/orchestration/_flows/client.py new file mode 100644 index 000000000000..88aad924a385 --- /dev/null +++ b/src/prefect/client/orchestration/_flows/client.py @@ -0,0 +1,228 @@ +from __future__ import annotations + +from typing import TYPE_CHECKING, Any + +from httpx import HTTPStatusError, RequestError + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.exceptions import ObjectNotFound + +if TYPE_CHECKING: + from uuid import UUID + + from prefect.client.schemas.filters import ( + DeploymentFilter, + FlowFilter, + FlowRunFilter, + TaskRunFilter, + WorkPoolFilter, + WorkQueueFilter, + ) + from prefect.client.schemas.objects import ( + Flow, + ) + from prefect.client.schemas.sorting import FlowSort + from prefect.flows import Flow as FlowObject + + +class FlowClient(BaseClient): + def create_flow(self, flow: "FlowObject[Any, Any]") -> "UUID": + """ + Create a flow in the Prefect API. + + Args: + flow: a [Flow][prefect.flows.Flow] object + + Raises: + httpx.RequestError: if a flow was not created for any reason + + Returns: + the ID of the flow in the backend + """ + return self.create_flow_from_name(flow.name) + + def create_flow_from_name(self, flow_name: str) -> "UUID": + """ + Create a flow in the Prefect API. + + Args: + flow_name: the name of the new flow + + Raises: + httpx.RequestError: if a flow was not created for any reason + + Returns: + the ID of the flow in the backend + """ + from prefect.client.schemas.actions import FlowCreate + + flow_data = FlowCreate(name=flow_name) + response = self.request( + "POST", "/flows/", json=flow_data.model_dump(mode="json") + ) + + flow_id = response.json().get("id") + if not flow_id: + raise RequestError(f"Malformed response: {response}") + + # Return the id of the created flow + return UUID(flow_id) + + +class FlowAsyncClient(BaseAsyncClient): + async def create_flow(self, flow: "FlowObject[Any, Any]") -> "UUID": + """ + Create a flow in the Prefect API. + + Args: + flow: a [Flow][prefect.flows.Flow] object + + Raises: + httpx.RequestError: if a flow was not created for any reason + + Returns: + the ID of the flow in the backend + """ + return await self.create_flow_from_name(flow.name) + + async def create_flow_from_name(self, flow_name: str) -> "UUID": + """ + Create a flow in the Prefect API. + + Args: + flow_name: the name of the new flow + + Raises: + httpx.RequestError: if a flow was not created for any reason + + Returns: + the ID of the flow in the backend + """ + from prefect.client.schemas.actions import FlowCreate + + flow_data = FlowCreate(name=flow_name) + response = await self.request( + "POST", "/flows/", json=flow_data.model_dump(mode="json") + ) + + flow_id = response.json().get("id") + if not flow_id: + raise RequestError(f"Malformed response: {response}") + + # Return the id of the created flow + from uuid import UUID + + return UUID(flow_id) + + async def read_flow(self, flow_id: "UUID") -> "Flow": + """ + Query the Prefect API for a flow by id. + + Args: + flow_id: the flow ID of interest + + Returns: + a [Flow model][prefect.client.schemas.objects.Flow] representation of the flow + """ + response = await self.request("GET", "/flows/{id}", path_params={"id": flow_id}) + from prefect.client.schemas.objects import Flow + + return Flow.model_validate(response.json()) + + async def delete_flow(self, flow_id: "UUID") -> None: + """ + Delete a flow by UUID. + + Args: + flow_id: ID of the flow to be deleted + Raises: + prefect.exceptions.ObjectNotFound: If request returns 404 + httpx.RequestError: If requests fail + """ + try: + await self.request("DELETE", "/flows/{id}", path_params={"id": flow_id}) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def read_flows( + self, + *, + flow_filter: "FlowFilter | None" = None, + flow_run_filter: "FlowRunFilter | None" = None, + task_run_filter: "TaskRunFilter | None" = None, + deployment_filter: "DeploymentFilter | None" = None, + work_pool_filter: "WorkPoolFilter | None" = None, + work_queue_filter: "WorkQueueFilter | None" = None, + sort: "FlowSort | None" = None, + limit: int | None = None, + offset: int = 0, + ) -> list["Flow"]: + """ + Query the Prefect API for flows. Only flows matching all criteria will + be returned. + + Args: + flow_filter: filter criteria for flows + flow_run_filter: filter criteria for flow runs + task_run_filter: filter criteria for task runs + deployment_filter: filter criteria for deployments + work_pool_filter: filter criteria for work pools + work_queue_filter: filter criteria for work pool queues + sort: sort criteria for the flows + limit: limit for the flow query + offset: offset for the flow query + + Returns: + a list of Flow model representations of the flows + """ + body: dict[str, Any] = { + "flows": flow_filter.model_dump(mode="json") if flow_filter else None, + "flow_runs": ( + flow_run_filter.model_dump(mode="json", exclude_unset=True) + if flow_run_filter + else None + ), + "task_runs": ( + task_run_filter.model_dump(mode="json") if task_run_filter else None + ), + "deployments": ( + deployment_filter.model_dump(mode="json") if deployment_filter else None + ), + "work_pools": ( + work_pool_filter.model_dump(mode="json") if work_pool_filter else None + ), + "work_queues": ( + work_queue_filter.model_dump(mode="json") if work_queue_filter else None + ), + "sort": sort, + "limit": limit, + "offset": offset, + } + + response = await self.request("POST", "/flows/filter", json=body) + from prefect.client.schemas.objects import Flow + + return Flow.model_validate_list(response.json()) + + async def read_flow_by_name( + self, + flow_name: str, + ) -> "Flow": + """ + Query the Prefect API for a flow by name. + + Args: + flow_name: the name of a flow + + Returns: + a fully hydrated Flow model + """ + response = await self.request( + "GET", "/flows/name/{name}", path_params={"name": flow_name} + ) + from prefect.client.schemas.objects import Flow + + return Flow.model_validate(response.json()) From 5d415dff52fb1685d2a800b84035cb801230d819 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 17:16:20 -0600 Subject: [PATCH 07/19] Update client.py --- .../client/orchestration/_flow_runs/client.py | 678 +++++++++++++++++- 1 file changed, 676 insertions(+), 2 deletions(-) diff --git a/src/prefect/client/orchestration/_flow_runs/client.py b/src/prefect/client/orchestration/_flow_runs/client.py index 73de2774e618..b5319903650c 100644 --- a/src/prefect/client/orchestration/_flow_runs/client.py +++ b/src/prefect/client/orchestration/_flow_runs/client.py @@ -1,11 +1,685 @@ from __future__ import annotations +from collections.abc import Iterable +from typing import TYPE_CHECKING, Any, Optional, TypeVar, Union +from uuid import UUID, uuid4 + +import httpx +import pydantic +from starlette import status + +import prefect +import prefect.exceptions +import prefect.settings +import prefect.states from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +T = TypeVar("T") +R = TypeVar("R", infer_variance=True) + +if TYPE_CHECKING: + from prefect.client.schemas import FlowRun, OrchestrationResult + from prefect.client.schemas.actions import ( + FlowRunCreate, + FlowRunUpdate, + ) + from prefect.client.schemas.filters import ( + DeploymentFilter, + FlowFilter, + FlowRunFilter, + TaskRunFilter, + WorkPoolFilter, + WorkQueueFilter, + ) + from prefect.client.schemas.objects import ( + FlowRunInput, + FlowRunPolicy, + ) + from prefect.client.schemas.sorting import ( + FlowRunSort, + ) + from prefect.flows import Flow as FlowObject + from prefect.states import State + from prefect.types import KeyValueLabelsField + class FlowRunClient(BaseClient): - pass + def create_flow_run( + self, + flow: "FlowObject[Any, R]", + name: Optional[str] = None, + parameters: Optional[dict[str, Any]] = None, + context: Optional[dict[str, Any]] = None, + tags: Optional[Iterable[str]] = None, + parent_task_run_id: Optional[UUID] = None, + state: Optional["State[R]"] = None, + ) -> FlowRun: + """ + Create a flow run for a flow. + + Args: + flow: The flow model to create the flow run for + name: An optional name for the flow run + parameters: Parameter overrides for this flow run. + context: Optional run context data + tags: a list of tags to apply to this flow run + parent_task_run_id: if a subflow run is being created, the placeholder task + run identifier in the parent flow + state: The initial state for the run. If not provided, defaults to + `Scheduled` for now. Should always be a `Scheduled` type. + + Raises: + httpx.RequestError: if the Prefect API does not successfully create a run for any reason + + Returns: + The flow run model + """ + parameters = parameters or {} + context = context or {} + + if state is None: + state = prefect.states.Pending() + + # Retrieve the flow id + flow_id = self.create_flow(flow) + + flow_run_create = FlowRunCreate( + flow_id=flow_id, + flow_version=flow.version, + name=name, + parameters=parameters, + context=context, + tags=list(tags or []), + parent_task_run_id=parent_task_run_id, + state=state.to_state_create(), + empirical_policy=FlowRunPolicy( + retries=flow.retries, + retry_delay=int(flow.retry_delay_seconds or 0), + ), + ) + + flow_run_create_json = flow_run_create.model_dump(mode="json") + response = self._client.post("/flow_runs/", json=flow_run_create_json) + flow_run = FlowRun.model_validate(response.json()) + + # Restore the parameters to the local objects to retain expectations about + # Python objects + flow_run.parameters = parameters + + return flow_run + + def update_flow_run( + self, + flow_run_id: UUID, + flow_version: Optional[str] = None, + parameters: Optional[dict[str, Any]] = None, + name: Optional[str] = None, + tags: Optional[Iterable[str]] = None, + empirical_policy: Optional[FlowRunPolicy] = None, + infrastructure_pid: Optional[str] = None, + job_variables: Optional[dict[str, Any]] = None, + ) -> httpx.Response: + """ + Update a flow run's details. + + Args: + flow_run_id: The identifier for the flow run to update. + flow_version: A new version string for the flow run. + parameters: A dictionary of parameter values for the flow run. This will not + be merged with any existing parameters. + name: A new name for the flow run. + empirical_policy: A new flow run orchestration policy. This will not be + merged with any existing policy. + tags: An iterable of new tags for the flow run. These will not be merged with + any existing tags. + infrastructure_pid: The id of flow run as returned by an + infrastructure block. + + Returns: + an `httpx.Response` object from the PATCH request + """ + params: dict[str, Any] = {} + if flow_version is not None: + params["flow_version"] = flow_version + if parameters is not None: + params["parameters"] = parameters + if name is not None: + params["name"] = name + if tags is not None: + params["tags"] = tags + if empirical_policy is not None: + params["empirical_policy"] = empirical_policy.model_dump( + mode="json", exclude_unset=True + ) + if infrastructure_pid: + params["infrastructure_pid"] = infrastructure_pid + if job_variables is not None: + params["job_variables"] = job_variables + + flow_run_data = FlowRunUpdate(**params) + + return self._client.patch( + f"/flow_runs/{flow_run_id}", + json=flow_run_data.model_dump(mode="json", exclude_unset=True), + ) + + def read_flow_run(self, flow_run_id: UUID) -> FlowRun: + """ + Query the Prefect API for a flow run by id. + + Args: + flow_run_id: the flow run ID of interest + + Returns: + a Flow Run model representation of the flow run + """ + try: + response = self._client.get(f"/flow_runs/{flow_run_id}") + except httpx.HTTPStatusError as e: + if e.response.status_code == 404: + raise prefect.exceptions.ObjectNotFound(http_exc=e) from e + else: + raise + return FlowRun.model_validate(response.json()) + + def read_flow_runs( + self, + *, + flow_filter: Optional[FlowFilter] = None, + flow_run_filter: Optional[FlowRunFilter] = None, + task_run_filter: Optional[TaskRunFilter] = None, + deployment_filter: Optional[DeploymentFilter] = None, + work_pool_filter: Optional[WorkPoolFilter] = None, + work_queue_filter: Optional[WorkQueueFilter] = None, + sort: Optional[FlowRunSort] = None, + limit: Optional[int] = None, + offset: int = 0, + ) -> list[FlowRun]: + """ + Query the Prefect API for flow runs. Only flow runs matching all criteria will + be returned. + + Args: + flow_filter: filter criteria for flows + flow_run_filter: filter criteria for flow runs + task_run_filter: filter criteria for task runs + deployment_filter: filter criteria for deployments + work_pool_filter: filter criteria for work pools + work_queue_filter: filter criteria for work pool queues + sort: sort criteria for the flow runs + limit: limit for the flow run query + offset: offset for the flow run query + + Returns: + a list of Flow Run model representations + of the flow runs + """ + body: dict[str, Any] = { + "flows": flow_filter.model_dump(mode="json") if flow_filter else None, + "flow_runs": ( + flow_run_filter.model_dump(mode="json", exclude_unset=True) + if flow_run_filter + else None + ), + "task_runs": ( + task_run_filter.model_dump(mode="json") if task_run_filter else None + ), + "deployments": ( + deployment_filter.model_dump(mode="json") if deployment_filter else None + ), + "work_pools": ( + work_pool_filter.model_dump(mode="json") if work_pool_filter else None + ), + "work_pool_queues": ( + work_queue_filter.model_dump(mode="json") if work_queue_filter else None + ), + "sort": sort, + "limit": limit, + "offset": offset, + } + + response = self._client.post("/flow_runs/filter", json=body) + return pydantic.TypeAdapter(list[FlowRun]).validate_python(response.json()) + + def set_flow_run_state( + self, + flow_run_id: UUID, + state: "prefect.states.State[T]", + force: bool = False, + ) -> OrchestrationResult[T]: + """ + Set the state of a flow run. + + Args: + flow_run_id: the id of the flow run + state: the state to set + force: if True, disregard orchestration logic when setting the state, + forcing the Prefect API to accept the state + + Returns: + an OrchestrationResult model representation of state orchestration output + """ + state_create = state.to_state_create() + state_create.state_details.flow_run_id = flow_run_id + state_create.state_details.transition_id = uuid4() + try: + response = self._client.post( + f"/flow_runs/{flow_run_id}/set_state", + json=dict( + state=state_create.model_dump(mode="json", serialize_as_any=True), + force=force, + ), + ) + except httpx.HTTPStatusError as e: + if e.response.status_code == status.HTTP_404_NOT_FOUND: + raise prefect.exceptions.ObjectNotFound(http_exc=e) from e + else: + raise + + result: OrchestrationResult[T] = OrchestrationResult.model_validate( + response.json() + ) + return result + + def set_flow_run_name(self, flow_run_id: UUID, name: str) -> httpx.Response: + flow_run_data = FlowRunUpdate(name=name) + return self._client.patch( + f"/flow_runs/{flow_run_id}", + json=flow_run_data.model_dump(mode="json", exclude_unset=True), + ) + + def update_flow_run_labels( + self, flow_run_id: UUID, labels: "KeyValueLabelsField" + ) -> None: + """ + Updates the labels of a flow run. + """ + response = self._client.patch( + f"/flow_runs/{flow_run_id}/labels", + json=labels, + ) + response.raise_for_status() class FlowRunAsyncClient(BaseAsyncClient): - pass + async def create_flow_run( + self, + flow: "FlowObject[Any, R]", + name: Optional[str] = None, + parameters: Optional[dict[str, Any]] = None, + context: Optional[dict[str, Any]] = None, + tags: Optional[Iterable[str]] = None, + parent_task_run_id: Optional[UUID] = None, + state: Optional["State[R]"] = None, + ) -> FlowRun: + """ + Create a flow run for a flow. + + Args: + flow: The flow model to create the flow run for + name: An optional name for the flow run + parameters: Parameter overrides for this flow run. + context: Optional run context data + tags: a list of tags to apply to this flow run + parent_task_run_id: if a subflow run is being created, the placeholder task + run identifier in the parent flow + state: The initial state for the run. If not provided, defaults to + `Scheduled` for now. Should always be a `Scheduled` type. + + Raises: + httpx.RequestError: if the Prefect API does not successfully create a run for any reason + + Returns: + The flow run model + """ + parameters = parameters or {} + context = context or {} + + if state is None: + state = prefect.states.Pending() + + # Retrieve the flow id + flow_id = await self.create_flow(flow) + + flow_run_create = FlowRunCreate( + flow_id=flow_id, + flow_version=flow.version, + name=name, + parameters=parameters, + context=context, + tags=list(tags or []), + parent_task_run_id=parent_task_run_id, + state=state.to_state_create(), + empirical_policy=FlowRunPolicy( + retries=flow.retries, + retry_delay=int(flow.retry_delay_seconds or 0), + ), + ) + + flow_run_create_json = flow_run_create.model_dump(mode="json") + response = await self._client.post("/flow_runs/", json=flow_run_create_json) + flow_run = FlowRun.model_validate(response.json()) + + # Restore the parameters to the local objects to retain expectations about + # Python objects + flow_run.parameters = parameters + + return flow_run + + async def update_flow_run( + self, + flow_run_id: UUID, + flow_version: Optional[str] = None, + parameters: Optional[dict[str, Any]] = None, + name: Optional[str] = None, + tags: Optional[Iterable[str]] = None, + empirical_policy: Optional[FlowRunPolicy] = None, + infrastructure_pid: Optional[str] = None, + job_variables: Optional[dict[str, Any]] = None, + ) -> httpx.Response: + """ + Update a flow run's details. + + Args: + flow_run_id: The identifier for the flow run to update. + flow_version: A new version string for the flow run. + parameters: A dictionary of parameter values for the flow run. This will not + be merged with any existing parameters. + name: A new name for the flow run. + empirical_policy: A new flow run orchestration policy. This will not be + merged with any existing policy. + tags: An iterable of new tags for the flow run. These will not be merged with + any existing tags. + infrastructure_pid: The id of flow run as returned by an + infrastructure block. + + Returns: + an `httpx.Response` object from the PATCH request + """ + params: dict[str, Any] = {} + if flow_version is not None: + params["flow_version"] = flow_version + if parameters is not None: + params["parameters"] = parameters + if name is not None: + params["name"] = name + if tags is not None: + params["tags"] = tags + if empirical_policy is not None: + params["empirical_policy"] = empirical_policy + if infrastructure_pid: + params["infrastructure_pid"] = infrastructure_pid + if job_variables is not None: + params["job_variables"] = job_variables + + flow_run_data = FlowRunUpdate(**params) + + return await self._client.patch( + f"/flow_runs/{flow_run_id}", + json=flow_run_data.model_dump(mode="json", exclude_unset=True), + ) + + async def delete_flow_run( + self, + flow_run_id: UUID, + ) -> None: + """ + Delete a flow run by UUID. + + Args: + flow_run_id: The flow run UUID of interest. + Raises: + prefect.exceptions.ObjectNotFound: If request returns 404 + httpx.RequestError: If requests fails + """ + try: + await self._client.delete(f"/flow_runs/{flow_run_id}") + except httpx.HTTPStatusError as e: + if e.response.status_code == status.HTTP_404_NOT_FOUND: + raise prefect.exceptions.ObjectNotFound(http_exc=e) from e + else: + raise + + async def read_flow_run(self, flow_run_id: UUID) -> FlowRun: + """ + Query the Prefect API for a flow run by id. + + Args: + flow_run_id: the flow run ID of interest + + Returns: + a Flow Run model representation of the flow run + """ + try: + response = await self._client.get(f"/flow_runs/{flow_run_id}") + except httpx.HTTPStatusError as e: + if e.response.status_code == 404: + raise prefect.exceptions.ObjectNotFound(http_exc=e) from e + else: + raise + return FlowRun.model_validate(response.json()) + + async def resume_flow_run( + self, flow_run_id: UUID, run_input: Optional[dict[str, Any]] = None + ) -> OrchestrationResult[Any]: + """ + Resumes a paused flow run. + + Args: + flow_run_id: the flow run ID of interest + run_input: the input to resume the flow run with + + Returns: + an OrchestrationResult model representation of state orchestration output + """ + try: + response = await self._client.post( + f"/flow_runs/{flow_run_id}/resume", json={"run_input": run_input} + ) + except httpx.HTTPStatusError: + raise + + result: OrchestrationResult[Any] = OrchestrationResult.model_validate( + response.json() + ) + return result + + async def read_flow_runs( + self, + *, + flow_filter: Optional[FlowFilter] = None, + flow_run_filter: Optional[FlowRunFilter] = None, + task_run_filter: Optional[TaskRunFilter] = None, + deployment_filter: Optional[DeploymentFilter] = None, + work_pool_filter: Optional[WorkPoolFilter] = None, + work_queue_filter: Optional[WorkQueueFilter] = None, + sort: Optional[FlowRunSort] = None, + limit: Optional[int] = None, + offset: int = 0, + ) -> list[FlowRun]: + """ + Query the Prefect API for flow runs. Only flow runs matching all criteria will + be returned. + + Args: + flow_filter: filter criteria for flows + flow_run_filter: filter criteria for flow runs + task_run_filter: filter criteria for task runs + deployment_filter: filter criteria for deployments + work_pool_filter: filter criteria for work pools + work_queue_filter: filter criteria for work pool queues + sort: sort criteria for the flow runs + limit: limit for the flow run query + offset: offset for the flow run query + + Returns: + a list of Flow Run model representations + of the flow runs + """ + body: dict[str, Any] = { + "flows": flow_filter.model_dump(mode="json") if flow_filter else None, + "flow_runs": ( + flow_run_filter.model_dump(mode="json", exclude_unset=True) + if flow_run_filter + else None + ), + "task_runs": ( + task_run_filter.model_dump(mode="json") if task_run_filter else None + ), + "deployments": ( + deployment_filter.model_dump(mode="json") if deployment_filter else None + ), + "work_pools": ( + work_pool_filter.model_dump(mode="json") if work_pool_filter else None + ), + "work_pool_queues": ( + work_queue_filter.model_dump(mode="json") if work_queue_filter else None + ), + "sort": sort, + "limit": limit, + "offset": offset, + } + + response = await self._client.post("/flow_runs/filter", json=body) + return pydantic.TypeAdapter(list[FlowRun]).validate_python(response.json()) + + async def set_flow_run_state( + self, + flow_run_id: Union[UUID, str], + state: "prefect.states.State[T]", + force: bool = False, + ) -> OrchestrationResult[T]: + """ + Set the state of a flow run. + + Args: + flow_run_id: the id of the flow run + state: the state to set + force: if True, disregard orchestration logic when setting the state, + forcing the Prefect API to accept the state + + Returns: + an OrchestrationResult model representation of state orchestration output + """ + flow_run_id = ( + flow_run_id if isinstance(flow_run_id, UUID) else UUID(flow_run_id) + ) + state_create = state.to_state_create() + state_create.state_details.flow_run_id = flow_run_id + state_create.state_details.transition_id = uuid4() + try: + response = await self._client.post( + f"/flow_runs/{flow_run_id}/set_state", + json=dict( + state=state_create.model_dump(mode="json", serialize_as_any=True), + force=force, + ), + ) + except httpx.HTTPStatusError as e: + if e.response.status_code == status.HTTP_404_NOT_FOUND: + raise prefect.exceptions.ObjectNotFound(http_exc=e) from e + else: + raise + + result: OrchestrationResult[T] = OrchestrationResult.model_validate( + response.json() + ) + return result + + async def read_flow_run_states( + self, flow_run_id: UUID + ) -> list[prefect.states.State]: + """ + Query for the states of a flow run + + Args: + flow_run_id: the id of the flow run + + Returns: + a list of State model representations + of the flow run states + """ + response = await self._client.get( + "/flow_run_states/", params=dict(flow_run_id=str(flow_run_id)) + ) + return pydantic.TypeAdapter(list[prefect.states.State]).validate_python( + response.json() + ) + + async def set_flow_run_name(self, flow_run_id: UUID, name: str) -> httpx.Response: + flow_run_data = FlowRunUpdate(name=name) + return await self._client.patch( + f"/flow_runs/{flow_run_id}", + json=flow_run_data.model_dump(mode="json", exclude_unset=True), + ) + + async def create_flow_run_input( + self, flow_run_id: UUID, key: str, value: str, sender: Optional[str] = None + ) -> None: + """ + Creates a flow run input. + + Args: + flow_run_id: The flow run id. + key: The input key. + value: The input value. + sender: The sender of the input. + """ + + # Initialize the input to ensure that the key is valid. + FlowRunInput(flow_run_id=flow_run_id, key=key, value=value) + + response = await self._client.post( + f"/flow_runs/{flow_run_id}/input", + json={"key": key, "value": value, "sender": sender}, + ) + response.raise_for_status() + + async def filter_flow_run_input( + self, flow_run_id: UUID, key_prefix: str, limit: int, exclude_keys: set[str] + ) -> list[FlowRunInput]: + response = await self._client.post( + f"/flow_runs/{flow_run_id}/input/filter", + json={ + "prefix": key_prefix, + "limit": limit, + "exclude_keys": list(exclude_keys), + }, + ) + response.raise_for_status() + return pydantic.TypeAdapter(list[FlowRunInput]).validate_python(response.json()) + + async def read_flow_run_input(self, flow_run_id: UUID, key: str) -> str: + """ + Reads a flow run input. + + Args: + flow_run_id: The flow run id. + key: The input key. + """ + response = await self._client.get(f"/flow_runs/{flow_run_id}/input/{key}") + response.raise_for_status() + return response.content.decode() + + async def delete_flow_run_input(self, flow_run_id: UUID, key: str) -> None: + """ + Deletes a flow run input. + + Args: + flow_run_id: The flow run id. + key: The input key. + """ + response = await self._client.delete(f"/flow_runs/{flow_run_id}/input/{key}") + response.raise_for_status() + + async def update_flow_run_labels( + self, flow_run_id: UUID, labels: "KeyValueLabelsField" + ) -> None: + """ + Updates the labels of a flow run. + """ + + response = await self._client.patch( + f"/flow_runs/{flow_run_id}/labels", json=labels + ) + response.raise_for_status() From b25f4b507ab4d9363d1f507a31e7aa0c95353001 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 17:56:59 -0600 Subject: [PATCH 08/19] Update client.py --- .../client/orchestration/_flow_runs/client.py | 429 +++++++++++++----- 1 file changed, 307 insertions(+), 122 deletions(-) diff --git a/src/prefect/client/orchestration/_flow_runs/client.py b/src/prefect/client/orchestration/_flow_runs/client.py index b5319903650c..bb766724787a 100644 --- a/src/prefect/client/orchestration/_flow_runs/client.py +++ b/src/prefect/client/orchestration/_flow_runs/client.py @@ -1,23 +1,22 @@ from __future__ import annotations from collections.abc import Iterable -from typing import TYPE_CHECKING, Any, Optional, TypeVar, Union -from uuid import UUID, uuid4 +from typing import TYPE_CHECKING, Any, TypeVar import httpx -import pydantic -from starlette import status import prefect import prefect.exceptions import prefect.settings import prefect.states -from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.client.orchestration._flows.client import FlowAsyncClient, FlowClient T = TypeVar("T") R = TypeVar("R", infer_variance=True) if TYPE_CHECKING: + from uuid import UUID, uuid4 + from prefect.client.schemas import FlowRun, OrchestrationResult from prefect.client.schemas.actions import ( FlowRunCreate, @@ -43,17 +42,17 @@ from prefect.types import KeyValueLabelsField -class FlowRunClient(BaseClient): +class FlowRunClient(FlowClient): def create_flow_run( self, flow: "FlowObject[Any, R]", - name: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - context: Optional[dict[str, Any]] = None, - tags: Optional[Iterable[str]] = None, - parent_task_run_id: Optional[UUID] = None, - state: Optional["State[R]"] = None, - ) -> FlowRun: + name: "str | None" = None, + parameters: "dict[str, Any] | None" = None, + context: "dict[str, Any] | None" = None, + tags: "Iterable[str] | None" = None, + parent_task_run_id: "UUID | None" = None, + state: "State[R] | None" = None, + ) -> "FlowRun": """ Create a flow run for a flow. @@ -99,7 +98,7 @@ def create_flow_run( ) flow_run_create_json = flow_run_create.model_dump(mode="json") - response = self._client.post("/flow_runs/", json=flow_run_create_json) + response = self.request("POST", "/flow_runs/", json=flow_run_create_json) flow_run = FlowRun.model_validate(response.json()) # Restore the parameters to the local objects to retain expectations about @@ -110,14 +109,14 @@ def create_flow_run( def update_flow_run( self, - flow_run_id: UUID, - flow_version: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - name: Optional[str] = None, - tags: Optional[Iterable[str]] = None, - empirical_policy: Optional[FlowRunPolicy] = None, - infrastructure_pid: Optional[str] = None, - job_variables: Optional[dict[str, Any]] = None, + flow_run_id: "UUID", + flow_version: "str | None" = None, + parameters: "dict[str, Any] | None" = None, + name: "str | None" = None, + tags: "Iterable[str] | None" = None, + empirical_policy: "FlowRunPolicy | None" = None, + infrastructure_pid: "str | None" = None, + job_variables: "dict[str, Any] | None" = None, ) -> httpx.Response: """ Update a flow run's details. @@ -148,9 +147,7 @@ def update_flow_run( if tags is not None: params["tags"] = tags if empirical_policy is not None: - params["empirical_policy"] = empirical_policy.model_dump( - mode="json", exclude_unset=True - ) + params["empirical_policy"] = empirical_policy if infrastructure_pid: params["infrastructure_pid"] = infrastructure_pid if job_variables is not None: @@ -158,12 +155,35 @@ def update_flow_run( flow_run_data = FlowRunUpdate(**params) - return self._client.patch( - f"/flow_runs/{flow_run_id}", + return self.request( + "PATCH", + "/flow_runs/{id}", + path_params={"id": flow_run_id}, json=flow_run_data.model_dump(mode="json", exclude_unset=True), ) - def read_flow_run(self, flow_run_id: UUID) -> FlowRun: + def delete_flow_run( + self, + flow_run_id: "UUID", + ) -> None: + """ + Delete a flow run by UUID. + + Args: + flow_run_id: The flow run UUID of interest. + Raises: + prefect.exceptions.ObjectNotFound: If request returns 404 + httpx.RequestError: If requests fails + """ + try: + self.request("DELETE", "/flow_runs/{id}", path_params={"id": flow_run_id}) + except httpx.HTTPStatusError as e: + if e.response.status_code == 404: + raise prefect.exceptions.ObjectNotFound(http_exc=e) from e + else: + raise + + def read_flow_run(self, flow_run_id: "UUID") -> "FlowRun": """ Query the Prefect API for a flow run by id. @@ -174,7 +194,9 @@ def read_flow_run(self, flow_run_id: UUID) -> FlowRun: a Flow Run model representation of the flow run """ try: - response = self._client.get(f"/flow_runs/{flow_run_id}") + response = self.request( + "GET", "/flow_runs/{id}", path_params={"id": flow_run_id} + ) except httpx.HTTPStatusError as e: if e.response.status_code == 404: raise prefect.exceptions.ObjectNotFound(http_exc=e) from e @@ -182,19 +204,47 @@ def read_flow_run(self, flow_run_id: UUID) -> FlowRun: raise return FlowRun.model_validate(response.json()) + def resume_flow_run( + self, flow_run_id: "UUID", run_input: "dict[str, Any] | None" = None + ) -> "OrchestrationResult[Any]": + """ + Resumes a paused flow run. + + Args: + flow_run_id: the flow run ID of interest + run_input: the input to resume the flow run with + + Returns: + an OrchestrationResult model representation of state orchestration output + """ + try: + response = self.request( + "POST", + "/flow_runs/{id}/resume", + path_params={"id": flow_run_id}, + json={"run_input": run_input}, + ) + except httpx.HTTPStatusError: + raise + + result: OrchestrationResult[Any] = OrchestrationResult.model_validate( + response.json() + ) + return result + def read_flow_runs( self, *, - flow_filter: Optional[FlowFilter] = None, - flow_run_filter: Optional[FlowRunFilter] = None, - task_run_filter: Optional[TaskRunFilter] = None, - deployment_filter: Optional[DeploymentFilter] = None, - work_pool_filter: Optional[WorkPoolFilter] = None, - work_queue_filter: Optional[WorkQueueFilter] = None, - sort: Optional[FlowRunSort] = None, - limit: Optional[int] = None, + flow_filter: "FlowFilter | None" = None, + flow_run_filter: "FlowRunFilter | None" = None, + task_run_filter: "TaskRunFilter | None" = None, + deployment_filter: "DeploymentFilter | None" = None, + work_pool_filter: "WorkPoolFilter | None" = None, + work_queue_filter: "WorkQueueFilter | None" = None, + sort: "FlowRunSort | None" = None, + limit: "int | None" = None, offset: int = 0, - ) -> list[FlowRun]: + ) -> "list[FlowRun]": """ Query the Prefect API for flow runs. Only flow runs matching all criteria will be returned. @@ -238,15 +288,17 @@ def read_flow_runs( "offset": offset, } - response = self._client.post("/flow_runs/filter", json=body) - return pydantic.TypeAdapter(list[FlowRun]).validate_python(response.json()) + response = self.request("POST", "/flow_runs/filter", json=body) + from prefect.client.schemas.objects import FlowRun + + return FlowRun.model_validate_list(response.json()) def set_flow_run_state( self, - flow_run_id: UUID, - state: "prefect.states.State[T]", + flow_run_id: "UUID | str", + state: "State[T]", force: bool = False, - ) -> OrchestrationResult[T]: + ) -> "OrchestrationResult[T]": """ Set the state of a flow run. @@ -259,19 +311,24 @@ def set_flow_run_state( Returns: an OrchestrationResult model representation of state orchestration output """ + flow_run_id = ( + flow_run_id if isinstance(flow_run_id, UUID) else UUID(flow_run_id) + ) state_create = state.to_state_create() state_create.state_details.flow_run_id = flow_run_id state_create.state_details.transition_id = uuid4() try: - response = self._client.post( - f"/flow_runs/{flow_run_id}/set_state", + response = self.request( + "POST", + "/flow_runs/{id}/set_state", + path_params={"id": flow_run_id}, json=dict( state=state_create.model_dump(mode="json", serialize_as_any=True), force=force, ), ) except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: + if e.response.status_code == 404: raise prefect.exceptions.ObjectNotFound(http_exc=e) from e else: raise @@ -281,37 +338,133 @@ def set_flow_run_state( ) return result - def set_flow_run_name(self, flow_run_id: UUID, name: str) -> httpx.Response: + def read_flow_run_states(self, flow_run_id: "UUID") -> "list[prefect.states.State]": + """ + Query for the states of a flow run + + Args: + flow_run_id: the id of the flow run + + Returns: + a list of State model representations + of the flow run states + """ + response = self.request( + "GET", "/flow_run_states/", params=dict(flow_run_id=str(flow_run_id)) + ) + from prefect.states import State + + return State.model_validate_list(response.json()) + + def set_flow_run_name(self, flow_run_id: "UUID", name: str) -> httpx.Response: flow_run_data = FlowRunUpdate(name=name) - return self._client.patch( - f"/flow_runs/{flow_run_id}", + return self.request( + "PATCH", + "/flow_runs/{id}", + path_params={"id": flow_run_id}, json=flow_run_data.model_dump(mode="json", exclude_unset=True), ) + def create_flow_run_input( + self, flow_run_id: "UUID", key: str, value: str, sender: "str | None" = None + ) -> None: + """ + Creates a flow run input. + + Args: + flow_run_id: The flow run id. + key: The input key. + value: The input value. + sender: The sender of the input. + """ + + # Initialize the input to ensure that the key is valid. + FlowRunInput(flow_run_id=flow_run_id, key=key, value=value) + + response = self.request( + "POST", + "/flow_runs/{id}/input", + path_params={"id": flow_run_id}, + json={"key": key, "value": value, "sender": sender}, + ) + response.raise_for_status() + + def filter_flow_run_input( + self, flow_run_id: "UUID", key_prefix: str, limit: int, exclude_keys: "set[str]" + ) -> "list[FlowRunInput]": + response = self.request( + "POST", + "/flow_runs/{id}/input/filter", + path_params={"id": flow_run_id}, + json={ + "prefix": key_prefix, + "limit": limit, + "exclude_keys": list(exclude_keys), + }, + ) + response.raise_for_status() + from prefect.client.schemas.objects import FlowRunInput + + return FlowRunInput.model_validate_list(response.json()) + + def read_flow_run_input(self, flow_run_id: "UUID", key: str) -> str: + """ + Reads a flow run input. + + Args: + flow_run_id: The flow run id. + key: The input key. + """ + response = self.request( + "GET", + "/flow_runs/{id}/input/{key}", + path_params={"id": flow_run_id, "key": key}, + ) + response.raise_for_status() + return response.content.decode() + + def delete_flow_run_input(self, flow_run_id: "UUID", key: str) -> None: + """ + Deletes a flow run input. + + Args: + flow_run_id: The flow run id. + key: The input key. + """ + response = self.request( + "DELETE", + "/flow_runs/{id}/input/{key}", + path_params={"id": flow_run_id, "key": key}, + ) + response.raise_for_status() + def update_flow_run_labels( - self, flow_run_id: UUID, labels: "KeyValueLabelsField" + self, flow_run_id: "UUID", labels: "KeyValueLabelsField" ) -> None: """ Updates the labels of a flow run. """ - response = self._client.patch( - f"/flow_runs/{flow_run_id}/labels", + + response = self.request( + "PATCH", + "/flow_runs/{id}/labels", + path_params={"id": flow_run_id}, json=labels, ) response.raise_for_status() -class FlowRunAsyncClient(BaseAsyncClient): +class FlowRunAsyncClient(FlowAsyncClient): async def create_flow_run( self, flow: "FlowObject[Any, R]", - name: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - context: Optional[dict[str, Any]] = None, - tags: Optional[Iterable[str]] = None, - parent_task_run_id: Optional[UUID] = None, - state: Optional["State[R]"] = None, - ) -> FlowRun: + name: "str | None" = None, + parameters: "dict[str, Any] | None" = None, + context: "dict[str, Any] | None" = None, + tags: "Iterable[str] | None" = None, + parent_task_run_id: "UUID | None" = None, + state: "State[R] | None" = None, + ) -> "FlowRun": """ Create a flow run for a flow. @@ -357,7 +510,7 @@ async def create_flow_run( ) flow_run_create_json = flow_run_create.model_dump(mode="json") - response = await self._client.post("/flow_runs/", json=flow_run_create_json) + response = await self.request("POST", "/flow_runs/", json=flow_run_create_json) flow_run = FlowRun.model_validate(response.json()) # Restore the parameters to the local objects to retain expectations about @@ -368,14 +521,14 @@ async def create_flow_run( async def update_flow_run( self, - flow_run_id: UUID, - flow_version: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - name: Optional[str] = None, - tags: Optional[Iterable[str]] = None, - empirical_policy: Optional[FlowRunPolicy] = None, - infrastructure_pid: Optional[str] = None, - job_variables: Optional[dict[str, Any]] = None, + flow_run_id: "UUID", + flow_version: "str | None" = None, + parameters: "dict[str, Any] | None" = None, + name: "str | None" = None, + tags: "Iterable[str] | None" = None, + empirical_policy: "FlowRunPolicy | None" = None, + infrastructure_pid: "str | None" = None, + job_variables: "dict[str, Any] | None" = None, ) -> httpx.Response: """ Update a flow run's details. @@ -414,14 +567,16 @@ async def update_flow_run( flow_run_data = FlowRunUpdate(**params) - return await self._client.patch( - f"/flow_runs/{flow_run_id}", + return await self.request( + "PATCH", + "/flow_runs/{id}", + path_params={"id": flow_run_id}, json=flow_run_data.model_dump(mode="json", exclude_unset=True), ) async def delete_flow_run( self, - flow_run_id: UUID, + flow_run_id: "UUID", ) -> None: """ Delete a flow run by UUID. @@ -433,14 +588,16 @@ async def delete_flow_run( httpx.RequestError: If requests fails """ try: - await self._client.delete(f"/flow_runs/{flow_run_id}") + await self.request( + "DELETE", "/flow_runs/{id}", path_params={"id": flow_run_id} + ) except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: + if e.response.status_code == 404: raise prefect.exceptions.ObjectNotFound(http_exc=e) from e else: raise - async def read_flow_run(self, flow_run_id: UUID) -> FlowRun: + async def read_flow_run(self, flow_run_id: "UUID") -> "FlowRun": """ Query the Prefect API for a flow run by id. @@ -451,7 +608,9 @@ async def read_flow_run(self, flow_run_id: UUID) -> FlowRun: a Flow Run model representation of the flow run """ try: - response = await self._client.get(f"/flow_runs/{flow_run_id}") + response = await self.request( + "GET", "/flow_runs/{id}", path_params={"id": flow_run_id} + ) except httpx.HTTPStatusError as e: if e.response.status_code == 404: raise prefect.exceptions.ObjectNotFound(http_exc=e) from e @@ -460,8 +619,8 @@ async def read_flow_run(self, flow_run_id: UUID) -> FlowRun: return FlowRun.model_validate(response.json()) async def resume_flow_run( - self, flow_run_id: UUID, run_input: Optional[dict[str, Any]] = None - ) -> OrchestrationResult[Any]: + self, flow_run_id: "UUID", run_input: "dict[str, Any] | None" = None + ) -> "OrchestrationResult[Any]": """ Resumes a paused flow run. @@ -473,8 +632,11 @@ async def resume_flow_run( an OrchestrationResult model representation of state orchestration output """ try: - response = await self._client.post( - f"/flow_runs/{flow_run_id}/resume", json={"run_input": run_input} + response = await self.request( + "POST", + "/flow_runs/{id}/resume", + path_params={"id": flow_run_id}, + json={"run_input": run_input}, ) except httpx.HTTPStatusError: raise @@ -487,16 +649,16 @@ async def resume_flow_run( async def read_flow_runs( self, *, - flow_filter: Optional[FlowFilter] = None, - flow_run_filter: Optional[FlowRunFilter] = None, - task_run_filter: Optional[TaskRunFilter] = None, - deployment_filter: Optional[DeploymentFilter] = None, - work_pool_filter: Optional[WorkPoolFilter] = None, - work_queue_filter: Optional[WorkQueueFilter] = None, - sort: Optional[FlowRunSort] = None, - limit: Optional[int] = None, + flow_filter: "FlowFilter | None" = None, + flow_run_filter: "FlowRunFilter | None" = None, + task_run_filter: "TaskRunFilter | None" = None, + deployment_filter: "DeploymentFilter | None" = None, + work_pool_filter: "WorkPoolFilter | None" = None, + work_queue_filter: "WorkQueueFilter | None" = None, + sort: "FlowRunSort | None" = None, + limit: "int | None" = None, offset: int = 0, - ) -> list[FlowRun]: + ) -> "list[FlowRun]": """ Query the Prefect API for flow runs. Only flow runs matching all criteria will be returned. @@ -540,15 +702,17 @@ async def read_flow_runs( "offset": offset, } - response = await self._client.post("/flow_runs/filter", json=body) - return pydantic.TypeAdapter(list[FlowRun]).validate_python(response.json()) + response = await self.request("POST", "/flow_runs/filter", json=body) + from prefect.client.schemas.objects import FlowRun + + return FlowRun.model_validate_list(response.json()) async def set_flow_run_state( self, - flow_run_id: Union[UUID, str], - state: "prefect.states.State[T]", + flow_run_id: "UUID | str", + state: "State[T]", force: bool = False, - ) -> OrchestrationResult[T]: + ) -> "OrchestrationResult[T]": """ Set the state of a flow run. @@ -568,15 +732,17 @@ async def set_flow_run_state( state_create.state_details.flow_run_id = flow_run_id state_create.state_details.transition_id = uuid4() try: - response = await self._client.post( - f"/flow_runs/{flow_run_id}/set_state", + response = await self.request( + "POST", + "/flow_runs/{id}/set_state", + path_params={"id": flow_run_id}, json=dict( state=state_create.model_dump(mode="json", serialize_as_any=True), force=force, ), ) except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: + if e.response.status_code == 404: raise prefect.exceptions.ObjectNotFound(http_exc=e) from e else: raise @@ -587,8 +753,8 @@ async def set_flow_run_state( return result async def read_flow_run_states( - self, flow_run_id: UUID - ) -> list[prefect.states.State]: + self, flow_run_id: "UUID" + ) -> "list[prefect.states.State]": """ Query for the states of a flow run @@ -599,22 +765,24 @@ async def read_flow_run_states( a list of State model representations of the flow run states """ - response = await self._client.get( - "/flow_run_states/", params=dict(flow_run_id=str(flow_run_id)) - ) - return pydantic.TypeAdapter(list[prefect.states.State]).validate_python( - response.json() + response = await self.request( + "GET", "/flow_run_states/", params=dict(flow_run_id=str(flow_run_id)) ) + from prefect.states import State - async def set_flow_run_name(self, flow_run_id: UUID, name: str) -> httpx.Response: + return State.model_validate_list(response.json()) + + async def set_flow_run_name(self, flow_run_id: "UUID", name: str) -> httpx.Response: flow_run_data = FlowRunUpdate(name=name) - return await self._client.patch( - f"/flow_runs/{flow_run_id}", + return await self.request( + "PATCH", + "/flow_runs/{id}", + path_params={"id": flow_run_id}, json=flow_run_data.model_dump(mode="json", exclude_unset=True), ) async def create_flow_run_input( - self, flow_run_id: UUID, key: str, value: str, sender: Optional[str] = None + self, flow_run_id: "UUID", key: str, value: str, sender: "str | None" = None ) -> None: """ Creates a flow run input. @@ -629,17 +797,21 @@ async def create_flow_run_input( # Initialize the input to ensure that the key is valid. FlowRunInput(flow_run_id=flow_run_id, key=key, value=value) - response = await self._client.post( - f"/flow_runs/{flow_run_id}/input", + response = await self.request( + "POST", + "/flow_runs/{id}/input", + path_params={"id": flow_run_id}, json={"key": key, "value": value, "sender": sender}, ) response.raise_for_status() async def filter_flow_run_input( - self, flow_run_id: UUID, key_prefix: str, limit: int, exclude_keys: set[str] - ) -> list[FlowRunInput]: - response = await self._client.post( - f"/flow_runs/{flow_run_id}/input/filter", + self, flow_run_id: "UUID", key_prefix: str, limit: int, exclude_keys: "set[str]" + ) -> "list[FlowRunInput]": + response = await self.request( + "POST", + "/flow_runs/{id}/input/filter", + path_params={"id": flow_run_id}, json={ "prefix": key_prefix, "limit": limit, @@ -647,9 +819,11 @@ async def filter_flow_run_input( }, ) response.raise_for_status() - return pydantic.TypeAdapter(list[FlowRunInput]).validate_python(response.json()) + from prefect.client.schemas.objects import FlowRunInput + + return FlowRunInput.model_validate_list(response.json()) - async def read_flow_run_input(self, flow_run_id: UUID, key: str) -> str: + async def read_flow_run_input(self, flow_run_id: "UUID", key: str) -> str: """ Reads a flow run input. @@ -657,11 +831,15 @@ async def read_flow_run_input(self, flow_run_id: UUID, key: str) -> str: flow_run_id: The flow run id. key: The input key. """ - response = await self._client.get(f"/flow_runs/{flow_run_id}/input/{key}") + response = await self.request( + "GET", + "/flow_runs/{id}/input/{key}", + path_params={"id": flow_run_id, "key": key}, + ) response.raise_for_status() return response.content.decode() - async def delete_flow_run_input(self, flow_run_id: UUID, key: str) -> None: + async def delete_flow_run_input(self, flow_run_id: "UUID", key: str) -> None: """ Deletes a flow run input. @@ -669,17 +847,24 @@ async def delete_flow_run_input(self, flow_run_id: UUID, key: str) -> None: flow_run_id: The flow run id. key: The input key. """ - response = await self._client.delete(f"/flow_runs/{flow_run_id}/input/{key}") + response = await self.request( + "DELETE", + "/flow_runs/{id}/input/{key}", + path_params={"id": flow_run_id, "key": key}, + ) response.raise_for_status() async def update_flow_run_labels( - self, flow_run_id: UUID, labels: "KeyValueLabelsField" + self, flow_run_id: "UUID", labels: "KeyValueLabelsField" ) -> None: """ Updates the labels of a flow run. """ - response = await self._client.patch( - f"/flow_runs/{flow_run_id}/labels", json=labels + response = await self.request( + "PATCH", + "/flow_runs/{id}/labels", + path_params={"id": flow_run_id}, + json=labels, ) response.raise_for_status() From 27d14b858c7abbe279f05d5468f5810488a02f1c Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 18:22:51 -0600 Subject: [PATCH 09/19] add imports --- src/prefect/client/orchestration/__init__.py | 654 +----------------- .../client/orchestration/_flow_runs/client.py | 41 +- .../client/orchestration/_flows/client.py | 115 +++ 3 files changed, 151 insertions(+), 659 deletions(-) diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index 67abacf51a95..8ccd288ec738 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -7,7 +7,7 @@ from contextlib import AsyncExitStack from logging import Logger from typing import TYPE_CHECKING, Any, Literal, NoReturn, Optional, Union, overload -from uuid import UUID, uuid4 +from uuid import UUID import certifi import httpcore @@ -70,11 +70,8 @@ BlockSchemaCreate, BlockTypeCreate, BlockTypeUpdate, - FlowCreate, - FlowRunCreate, FlowRunNotificationPolicyCreate, FlowRunNotificationPolicyUpdate, - FlowRunUpdate, TaskRunCreate, TaskRunUpdate, WorkPoolCreate, @@ -98,10 +95,7 @@ BlockSchema, BlockType, Constant, - Flow, - FlowRunInput, FlowRunNotificationPolicy, - FlowRunPolicy, Parameter, TaskRunPolicy, TaskRunResult, @@ -115,8 +109,6 @@ WorkerFlowRunResponse, ) from prefect.client.schemas.sorting import ( - FlowRunSort, - FlowSort, TaskRunSort, ) from prefect.logging import get_logger @@ -135,10 +127,8 @@ PREFECT_TESTING_UNIT_TEST_MODE, get_current_settings, ) -from prefect.types import KeyValueLabelsField if TYPE_CHECKING: - from prefect.flows import Flow as FlowObject from prefect.tasks import Task as TaskObject from prefect.client.base import ( @@ -259,8 +249,8 @@ class PrefectClient( ConcurrencyLimitAsyncClient, DeploymentAsyncClient, AutomationAsyncClient, - FlowAsyncClient, FlowRunAsyncClient, + FlowAsyncClient, ): """ An asynchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -473,144 +463,6 @@ async def hello(self) -> httpx.Response: """ return await self._client.get("/hello") - async def create_flow_run( - self, - flow: "FlowObject[Any, R]", - name: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - context: Optional[dict[str, Any]] = None, - tags: Optional[Iterable[str]] = None, - parent_task_run_id: Optional[UUID] = None, - state: Optional["prefect.states.State[R]"] = None, - ) -> FlowRun: - """ - Create a flow run for a flow. - - Args: - flow: The flow model to create the flow run for - name: An optional name for the flow run - parameters: Parameter overrides for this flow run. - context: Optional run context data - tags: a list of tags to apply to this flow run - parent_task_run_id: if a subflow run is being created, the placeholder task - run identifier in the parent flow - state: The initial state for the run. If not provided, defaults to - `Scheduled` for now. Should always be a `Scheduled` type. - - Raises: - httpx.RequestError: if the Prefect API does not successfully create a run for any reason - - Returns: - The flow run model - """ - parameters = parameters or {} - context = context or {} - - if state is None: - state = prefect.states.Pending() - - # Retrieve the flow id - flow_id = await self.create_flow(flow) - - flow_run_create = FlowRunCreate( - flow_id=flow_id, - flow_version=flow.version, - name=name, - parameters=parameters, - context=context, - tags=list(tags or []), - parent_task_run_id=parent_task_run_id, - state=state.to_state_create(), - empirical_policy=FlowRunPolicy( - retries=flow.retries, - retry_delay=int(flow.retry_delay_seconds or 0), - ), - ) - - flow_run_create_json = flow_run_create.model_dump(mode="json") - response = await self._client.post("/flow_runs/", json=flow_run_create_json) - flow_run = FlowRun.model_validate(response.json()) - - # Restore the parameters to the local objects to retain expectations about - # Python objects - flow_run.parameters = parameters - - return flow_run - - async def update_flow_run( - self, - flow_run_id: UUID, - flow_version: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - name: Optional[str] = None, - tags: Optional[Iterable[str]] = None, - empirical_policy: Optional[FlowRunPolicy] = None, - infrastructure_pid: Optional[str] = None, - job_variables: Optional[dict[str, Any]] = None, - ) -> httpx.Response: - """ - Update a flow run's details. - - Args: - flow_run_id: The identifier for the flow run to update. - flow_version: A new version string for the flow run. - parameters: A dictionary of parameter values for the flow run. This will not - be merged with any existing parameters. - name: A new name for the flow run. - empirical_policy: A new flow run orchestration policy. This will not be - merged with any existing policy. - tags: An iterable of new tags for the flow run. These will not be merged with - any existing tags. - infrastructure_pid: The id of flow run as returned by an - infrastructure block. - - Returns: - an `httpx.Response` object from the PATCH request - """ - params: dict[str, Any] = {} - if flow_version is not None: - params["flow_version"] = flow_version - if parameters is not None: - params["parameters"] = parameters - if name is not None: - params["name"] = name - if tags is not None: - params["tags"] = tags - if empirical_policy is not None: - params["empirical_policy"] = empirical_policy - if infrastructure_pid: - params["infrastructure_pid"] = infrastructure_pid - if job_variables is not None: - params["job_variables"] = job_variables - - flow_run_data = FlowRunUpdate(**params) - - return await self._client.patch( - f"/flow_runs/{flow_run_id}", - json=flow_run_data.model_dump(mode="json", exclude_unset=True), - ) - - async def delete_flow_run( - self, - flow_run_id: UUID, - ) -> None: - """ - Delete a flow run by UUID. - - Args: - flow_run_id: The flow run UUID of interest. - Raises: - prefect.exceptions.ObjectNotFound: If request returns 404 - httpx.RequestError: If requests fails - """ - try: - await self._client.delete(f"/flow_runs/{flow_run_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - async def create_work_queue( self, name: str, @@ -1255,179 +1107,6 @@ async def read_block_documents_by_type( response.json() ) - async def read_flow_run(self, flow_run_id: UUID) -> FlowRun: - """ - Query the Prefect API for a flow run by id. - - Args: - flow_run_id: the flow run ID of interest - - Returns: - a Flow Run model representation of the flow run - """ - try: - response = await self._client.get(f"/flow_runs/{flow_run_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == 404: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - return FlowRun.model_validate(response.json()) - - async def resume_flow_run( - self, flow_run_id: UUID, run_input: Optional[dict[str, Any]] = None - ) -> OrchestrationResult[Any]: - """ - Resumes a paused flow run. - - Args: - flow_run_id: the flow run ID of interest - run_input: the input to resume the flow run with - - Returns: - an OrchestrationResult model representation of state orchestration output - """ - try: - response = await self._client.post( - f"/flow_runs/{flow_run_id}/resume", json={"run_input": run_input} - ) - except httpx.HTTPStatusError: - raise - - result: OrchestrationResult[Any] = OrchestrationResult.model_validate( - response.json() - ) - return result - - async def read_flow_runs( - self, - *, - flow_filter: Optional[FlowFilter] = None, - flow_run_filter: Optional[FlowRunFilter] = None, - task_run_filter: Optional[TaskRunFilter] = None, - deployment_filter: Optional[DeploymentFilter] = None, - work_pool_filter: Optional[WorkPoolFilter] = None, - work_queue_filter: Optional[WorkQueueFilter] = None, - sort: Optional[FlowRunSort] = None, - limit: Optional[int] = None, - offset: int = 0, - ) -> list[FlowRun]: - """ - Query the Prefect API for flow runs. Only flow runs matching all criteria will - be returned. - - Args: - flow_filter: filter criteria for flows - flow_run_filter: filter criteria for flow runs - task_run_filter: filter criteria for task runs - deployment_filter: filter criteria for deployments - work_pool_filter: filter criteria for work pools - work_queue_filter: filter criteria for work pool queues - sort: sort criteria for the flow runs - limit: limit for the flow run query - offset: offset for the flow run query - - Returns: - a list of Flow Run model representations - of the flow runs - """ - body: dict[str, Any] = { - "flows": flow_filter.model_dump(mode="json") if flow_filter else None, - "flow_runs": ( - flow_run_filter.model_dump(mode="json", exclude_unset=True) - if flow_run_filter - else None - ), - "task_runs": ( - task_run_filter.model_dump(mode="json") if task_run_filter else None - ), - "deployments": ( - deployment_filter.model_dump(mode="json") if deployment_filter else None - ), - "work_pools": ( - work_pool_filter.model_dump(mode="json") if work_pool_filter else None - ), - "work_pool_queues": ( - work_queue_filter.model_dump(mode="json") if work_queue_filter else None - ), - "sort": sort, - "limit": limit, - "offset": offset, - } - - response = await self._client.post("/flow_runs/filter", json=body) - return pydantic.TypeAdapter(list[FlowRun]).validate_python(response.json()) - - async def set_flow_run_state( - self, - flow_run_id: Union[UUID, str], - state: "prefect.states.State[T]", - force: bool = False, - ) -> OrchestrationResult[T]: - """ - Set the state of a flow run. - - Args: - flow_run_id: the id of the flow run - state: the state to set - force: if True, disregard orchestration logic when setting the state, - forcing the Prefect API to accept the state - - Returns: - an OrchestrationResult model representation of state orchestration output - """ - flow_run_id = ( - flow_run_id if isinstance(flow_run_id, UUID) else UUID(flow_run_id) - ) - state_create = state.to_state_create() - state_create.state_details.flow_run_id = flow_run_id - state_create.state_details.transition_id = uuid4() - try: - response = await self._client.post( - f"/flow_runs/{flow_run_id}/set_state", - json=dict( - state=state_create.model_dump(mode="json", serialize_as_any=True), - force=force, - ), - ) - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - - result: OrchestrationResult[T] = OrchestrationResult.model_validate( - response.json() - ) - return result - - async def read_flow_run_states( - self, flow_run_id: UUID - ) -> list[prefect.states.State]: - """ - Query for the states of a flow run - - Args: - flow_run_id: the id of the flow run - - Returns: - a list of State model representations - of the flow run states - """ - response = await self._client.get( - "/flow_run_states/", params=dict(flow_run_id=str(flow_run_id)) - ) - return pydantic.TypeAdapter(list[prefect.states.State]).validate_python( - response.json() - ) - - async def set_flow_run_name(self, flow_run_id: UUID, name: str) -> httpx.Response: - flow_run_data = FlowRunUpdate(name=name) - return await self._client.patch( - f"/flow_runs/{flow_run_id}", - json=flow_run_data.model_dump(mode="json", exclude_unset=True), - ) - async def set_task_run_name(self, task_run_id: UUID, name: str) -> httpx.Response: task_run_data = TaskRunUpdate(name=name) return await self._client.patch( @@ -2080,65 +1759,6 @@ async def read_worker_metadata(self) -> dict[str, Any]: response.raise_for_status() return response.json() - async def create_flow_run_input( - self, flow_run_id: UUID, key: str, value: str, sender: Optional[str] = None - ) -> None: - """ - Creates a flow run input. - - Args: - flow_run_id: The flow run id. - key: The input key. - value: The input value. - sender: The sender of the input. - """ - - # Initialize the input to ensure that the key is valid. - FlowRunInput(flow_run_id=flow_run_id, key=key, value=value) - - response = await self._client.post( - f"/flow_runs/{flow_run_id}/input", - json={"key": key, "value": value, "sender": sender}, - ) - response.raise_for_status() - - async def filter_flow_run_input( - self, flow_run_id: UUID, key_prefix: str, limit: int, exclude_keys: set[str] - ) -> list[FlowRunInput]: - response = await self._client.post( - f"/flow_runs/{flow_run_id}/input/filter", - json={ - "prefix": key_prefix, - "limit": limit, - "exclude_keys": list(exclude_keys), - }, - ) - response.raise_for_status() - return pydantic.TypeAdapter(list[FlowRunInput]).validate_python(response.json()) - - async def read_flow_run_input(self, flow_run_id: UUID, key: str) -> str: - """ - Reads a flow run input. - - Args: - flow_run_id: The flow run id. - key: The input key. - """ - response = await self._client.get(f"/flow_runs/{flow_run_id}/input/{key}") - response.raise_for_status() - return response.content.decode() - - async def delete_flow_run_input(self, flow_run_id: UUID, key: str) -> None: - """ - Deletes a flow run input. - - Args: - flow_run_id: The flow run id. - key: The input key. - """ - response = await self._client.delete(f"/flow_runs/{flow_run_id}/input/{key}") - response.raise_for_status() - async def api_version(self) -> str: res = await self._client.get("/admin/version") return res.json() @@ -2167,18 +1787,6 @@ async def raise_for_api_version_mismatch(self) -> None: f"Major versions must match." ) - async def update_flow_run_labels( - self, flow_run_id: UUID, labels: KeyValueLabelsField - ) -> None: - """ - Updates the labels of a flow run. - """ - - response = await self._client.patch( - f"/flow_runs/{flow_run_id}/labels", json=labels - ) - response.raise_for_status() - async def __aenter__(self) -> Self: """ Start the client. @@ -2255,8 +1863,8 @@ class SyncPrefectClient( ConcurrencyLimitClient, DeploymentClient, AutomationClient, - FlowClient, FlowRunClient, + FlowClient, ): """ A synchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -2515,250 +2123,6 @@ def raise_for_api_version_mismatch(self) -> None: f"Major versions must match." ) - def create_flow_run( - self, - flow: "FlowObject[Any, R]", - name: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - context: Optional[dict[str, Any]] = None, - tags: Optional[Iterable[str]] = None, - parent_task_run_id: Optional[UUID] = None, - state: Optional["prefect.states.State[R]"] = None, - ) -> FlowRun: - """ - Create a flow run for a flow. - - Args: - flow: The flow model to create the flow run for - name: An optional name for the flow run - parameters: Parameter overrides for this flow run. - context: Optional run context data - tags: a list of tags to apply to this flow run - parent_task_run_id: if a subflow run is being created, the placeholder task - run identifier in the parent flow - state: The initial state for the run. If not provided, defaults to - `Scheduled` for now. Should always be a `Scheduled` type. - - Raises: - httpx.RequestError: if the Prefect API does not successfully create a run for any reason - - Returns: - The flow run model - """ - parameters = parameters or {} - context = context or {} - - if state is None: - state = prefect.states.Pending() - - # Retrieve the flow id - flow_id = self.create_flow(flow) - - flow_run_create = FlowRunCreate( - flow_id=flow_id, - flow_version=flow.version, - name=name, - parameters=parameters, - context=context, - tags=list(tags or []), - parent_task_run_id=parent_task_run_id, - state=state.to_state_create(), - empirical_policy=FlowRunPolicy( - retries=flow.retries, - retry_delay=int(flow.retry_delay_seconds or 0), - ), - ) - - flow_run_create_json = flow_run_create.model_dump(mode="json") - response = self._client.post("/flow_runs/", json=flow_run_create_json) - flow_run = FlowRun.model_validate(response.json()) - - # Restore the parameters to the local objects to retain expectations about - # Python objects - flow_run.parameters = parameters - - return flow_run - - def update_flow_run( - self, - flow_run_id: UUID, - flow_version: Optional[str] = None, - parameters: Optional[dict[str, Any]] = None, - name: Optional[str] = None, - tags: Optional[Iterable[str]] = None, - empirical_policy: Optional[FlowRunPolicy] = None, - infrastructure_pid: Optional[str] = None, - job_variables: Optional[dict[str, Any]] = None, - ) -> httpx.Response: - """ - Update a flow run's details. - - Args: - flow_run_id: The identifier for the flow run to update. - flow_version: A new version string for the flow run. - parameters: A dictionary of parameter values for the flow run. This will not - be merged with any existing parameters. - name: A new name for the flow run. - empirical_policy: A new flow run orchestration policy. This will not be - merged with any existing policy. - tags: An iterable of new tags for the flow run. These will not be merged with - any existing tags. - infrastructure_pid: The id of flow run as returned by an - infrastructure block. - - Returns: - an `httpx.Response` object from the PATCH request - """ - params: dict[str, Any] = {} - if flow_version is not None: - params["flow_version"] = flow_version - if parameters is not None: - params["parameters"] = parameters - if name is not None: - params["name"] = name - if tags is not None: - params["tags"] = tags - if empirical_policy is not None: - params["empirical_policy"] = empirical_policy.model_dump( - mode="json", exclude_unset=True - ) - if infrastructure_pid: - params["infrastructure_pid"] = infrastructure_pid - if job_variables is not None: - params["job_variables"] = job_variables - - flow_run_data = FlowRunUpdate(**params) - - return self._client.patch( - f"/flow_runs/{flow_run_id}", - json=flow_run_data.model_dump(mode="json", exclude_unset=True), - ) - - def read_flow_run(self, flow_run_id: UUID) -> FlowRun: - """ - Query the Prefect API for a flow run by id. - - Args: - flow_run_id: the flow run ID of interest - - Returns: - a Flow Run model representation of the flow run - """ - try: - response = self._client.get(f"/flow_runs/{flow_run_id}") - except httpx.HTTPStatusError as e: - if e.response.status_code == 404: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - return FlowRun.model_validate(response.json()) - - def read_flow_runs( - self, - *, - flow_filter: Optional[FlowFilter] = None, - flow_run_filter: Optional[FlowRunFilter] = None, - task_run_filter: Optional[TaskRunFilter] = None, - deployment_filter: Optional[DeploymentFilter] = None, - work_pool_filter: Optional[WorkPoolFilter] = None, - work_queue_filter: Optional[WorkQueueFilter] = None, - sort: Optional[FlowRunSort] = None, - limit: Optional[int] = None, - offset: int = 0, - ) -> list[FlowRun]: - """ - Query the Prefect API for flow runs. Only flow runs matching all criteria will - be returned. - - Args: - flow_filter: filter criteria for flows - flow_run_filter: filter criteria for flow runs - task_run_filter: filter criteria for task runs - deployment_filter: filter criteria for deployments - work_pool_filter: filter criteria for work pools - work_queue_filter: filter criteria for work pool queues - sort: sort criteria for the flow runs - limit: limit for the flow run query - offset: offset for the flow run query - - Returns: - a list of Flow Run model representations - of the flow runs - """ - body: dict[str, Any] = { - "flows": flow_filter.model_dump(mode="json") if flow_filter else None, - "flow_runs": ( - flow_run_filter.model_dump(mode="json", exclude_unset=True) - if flow_run_filter - else None - ), - "task_runs": ( - task_run_filter.model_dump(mode="json") if task_run_filter else None - ), - "deployments": ( - deployment_filter.model_dump(mode="json") if deployment_filter else None - ), - "work_pools": ( - work_pool_filter.model_dump(mode="json") if work_pool_filter else None - ), - "work_pool_queues": ( - work_queue_filter.model_dump(mode="json") if work_queue_filter else None - ), - "sort": sort, - "limit": limit, - "offset": offset, - } - - response = self._client.post("/flow_runs/filter", json=body) - return pydantic.TypeAdapter(list[FlowRun]).validate_python(response.json()) - - def set_flow_run_state( - self, - flow_run_id: UUID, - state: "prefect.states.State[T]", - force: bool = False, - ) -> OrchestrationResult[T]: - """ - Set the state of a flow run. - - Args: - flow_run_id: the id of the flow run - state: the state to set - force: if True, disregard orchestration logic when setting the state, - forcing the Prefect API to accept the state - - Returns: - an OrchestrationResult model representation of state orchestration output - """ - state_create = state.to_state_create() - state_create.state_details.flow_run_id = flow_run_id - state_create.state_details.transition_id = uuid4() - try: - response = self._client.post( - f"/flow_runs/{flow_run_id}/set_state", - json=dict( - state=state_create.model_dump(mode="json", serialize_as_any=True), - force=force, - ), - ) - except httpx.HTTPStatusError as e: - if e.response.status_code == status.HTTP_404_NOT_FOUND: - raise prefect.exceptions.ObjectNotFound(http_exc=e) from e - else: - raise - - result: OrchestrationResult[T] = OrchestrationResult.model_validate( - response.json() - ) - return result - - def set_flow_run_name(self, flow_run_id: UUID, name: str) -> httpx.Response: - flow_run_data = FlowRunUpdate(name=name) - return self._client.patch( - f"/flow_runs/{flow_run_id}", - json=flow_run_data.model_dump(mode="json", exclude_unset=True), - ) - def set_task_run_name(self, task_run_id: UUID, name: str) -> httpx.Response: task_run_data = TaskRunUpdate(name=name) return self._client.patch( @@ -2953,18 +2317,6 @@ def read_task_run_states(self, task_run_id: UUID) -> list[prefect.states.State]: response.json() ) - def update_flow_run_labels( - self, flow_run_id: UUID, labels: KeyValueLabelsField - ) -> None: - """ - Updates the labels of a flow run. - """ - response = self._client.patch( - f"/flow_runs/{flow_run_id}/labels", - json=labels, - ) - response.raise_for_status() - def read_block_document_by_name( self, name: str, diff --git a/src/prefect/client/orchestration/_flow_runs/client.py b/src/prefect/client/orchestration/_flow_runs/client.py index bb766724787a..31f0ccda8ff7 100644 --- a/src/prefect/client/orchestration/_flow_runs/client.py +++ b/src/prefect/client/orchestration/_flow_runs/client.py @@ -1,9 +1,8 @@ -from __future__ import annotations - from collections.abc import Iterable -from typing import TYPE_CHECKING, Any, TypeVar +from typing import TYPE_CHECKING, Any import httpx +from typing_extensions import TypeVar import prefect import prefect.exceptions @@ -15,13 +14,9 @@ R = TypeVar("R", infer_variance=True) if TYPE_CHECKING: - from uuid import UUID, uuid4 + from uuid import UUID from prefect.client.schemas import FlowRun, OrchestrationResult - from prefect.client.schemas.actions import ( - FlowRunCreate, - FlowRunUpdate, - ) from prefect.client.schemas.filters import ( DeploymentFilter, FlowFilter, @@ -81,6 +76,8 @@ def create_flow_run( # Retrieve the flow id flow_id = self.create_flow(flow) + from prefect.client.schemas.actions import FlowRunCreate + from prefect.client.schemas.objects import FlowRunPolicy flow_run_create = FlowRunCreate( flow_id=flow_id, @@ -99,6 +96,8 @@ def create_flow_run( flow_run_create_json = flow_run_create.model_dump(mode="json") response = self.request("POST", "/flow_runs/", json=flow_run_create_json) + from prefect.client.schemas.objects import FlowRun + flow_run = FlowRun.model_validate(response.json()) # Restore the parameters to the local objects to retain expectations about @@ -153,6 +152,8 @@ def update_flow_run( if job_variables is not None: params["job_variables"] = job_variables + from prefect.client.schemas.actions import FlowRunUpdate + flow_run_data = FlowRunUpdate(**params) return self.request( @@ -202,6 +203,8 @@ def read_flow_run(self, flow_run_id: "UUID") -> "FlowRun": raise prefect.exceptions.ObjectNotFound(http_exc=e) from e else: raise + from prefect.client.schemas.objects import FlowRun + return FlowRun.model_validate(response.json()) def resume_flow_run( @@ -226,6 +229,7 @@ def resume_flow_run( ) except httpx.HTTPStatusError: raise + from prefect.client.schemas import OrchestrationResult result: OrchestrationResult[Any] = OrchestrationResult.model_validate( response.json() @@ -311,6 +315,8 @@ def set_flow_run_state( Returns: an OrchestrationResult model representation of state orchestration output """ + from uuid import UUID, uuid4 + flow_run_id = ( flow_run_id if isinstance(flow_run_id, UUID) else UUID(flow_run_id) ) @@ -333,6 +339,8 @@ def set_flow_run_state( else: raise + from prefect.client.schemas import OrchestrationResult + result: OrchestrationResult[T] = OrchestrationResult.model_validate( response.json() ) @@ -357,6 +365,8 @@ def read_flow_run_states(self, flow_run_id: "UUID") -> "list[prefect.states.Stat return State.model_validate_list(response.json()) def set_flow_run_name(self, flow_run_id: "UUID", name: str) -> httpx.Response: + from prefect.client.schemas.actions import FlowRunUpdate + flow_run_data = FlowRunUpdate(name=name) return self.request( "PATCH", @@ -493,6 +503,8 @@ async def create_flow_run( # Retrieve the flow id flow_id = await self.create_flow(flow) + from prefect.client.schemas.actions import FlowRunCreate + from prefect.client.schemas.objects import FlowRunPolicy flow_run_create = FlowRunCreate( flow_id=flow_id, @@ -511,6 +523,8 @@ async def create_flow_run( flow_run_create_json = flow_run_create.model_dump(mode="json") response = await self.request("POST", "/flow_runs/", json=flow_run_create_json) + from prefect.client.schemas.objects import FlowRun + flow_run = FlowRun.model_validate(response.json()) # Restore the parameters to the local objects to retain expectations about @@ -564,6 +578,7 @@ async def update_flow_run( params["infrastructure_pid"] = infrastructure_pid if job_variables is not None: params["job_variables"] = job_variables + from prefect.client.schemas.actions import FlowRunUpdate flow_run_data = FlowRunUpdate(**params) @@ -616,6 +631,8 @@ async def read_flow_run(self, flow_run_id: "UUID") -> "FlowRun": raise prefect.exceptions.ObjectNotFound(http_exc=e) from e else: raise + from prefect.client.schemas.objects import FlowRun + return FlowRun.model_validate(response.json()) async def resume_flow_run( @@ -640,6 +657,7 @@ async def resume_flow_run( ) except httpx.HTTPStatusError: raise + from prefect.client.schemas import OrchestrationResult result: OrchestrationResult[Any] = OrchestrationResult.model_validate( response.json() @@ -725,6 +743,8 @@ async def set_flow_run_state( Returns: an OrchestrationResult model representation of state orchestration output """ + from uuid import UUID, uuid4 + flow_run_id = ( flow_run_id if isinstance(flow_run_id, UUID) else UUID(flow_run_id) ) @@ -746,6 +766,7 @@ async def set_flow_run_state( raise prefect.exceptions.ObjectNotFound(http_exc=e) from e else: raise + from prefect.client.schemas import OrchestrationResult result: OrchestrationResult[T] = OrchestrationResult.model_validate( response.json() @@ -773,6 +794,8 @@ async def read_flow_run_states( return State.model_validate_list(response.json()) async def set_flow_run_name(self, flow_run_id: "UUID", name: str) -> httpx.Response: + from prefect.client.schemas.actions import FlowRunUpdate + flow_run_data = FlowRunUpdate(name=name) return await self.request( "PATCH", @@ -795,6 +818,8 @@ async def create_flow_run_input( """ # Initialize the input to ensure that the key is valid. + from prefect.client.schemas.objects import FlowRunInput + FlowRunInput(flow_run_id=flow_run_id, key=key, value=value) response = await self.request( diff --git a/src/prefect/client/orchestration/_flows/client.py b/src/prefect/client/orchestration/_flows/client.py index 88aad924a385..b3bde13d349e 100644 --- a/src/prefect/client/orchestration/_flows/client.py +++ b/src/prefect/client/orchestration/_flows/client.py @@ -66,8 +66,123 @@ def create_flow_from_name(self, flow_name: str) -> "UUID": raise RequestError(f"Malformed response: {response}") # Return the id of the created flow + from uuid import UUID + return UUID(flow_id) + def read_flow(self, flow_id: "UUID") -> "Flow": + """ + Query the Prefect API for a flow by id. + + Args: + flow_id: the flow ID of interest + + Returns: + a [Flow model][prefect.client.schemas.objects.Flow] representation of the flow + """ + response = self.request("GET", "/flows/{id}", path_params={"id": flow_id}) + from prefect.client.schemas.objects import Flow + + return Flow.model_validate(response.json()) + + def delete_flow(self, flow_id: "UUID") -> None: + """ + Delete a flow by UUID. + + Args: + flow_id: ID of the flow to be deleted + Raises: + prefect.exceptions.ObjectNotFound: If request returns 404 + httpx.RequestError: If requests fail + """ + try: + self.request("DELETE", "/flows/{id}", path_params={"id": flow_id}) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def read_flows( + self, + *, + flow_filter: "FlowFilter | None" = None, + flow_run_filter: "FlowRunFilter | None" = None, + task_run_filter: "TaskRunFilter | None" = None, + deployment_filter: "DeploymentFilter | None" = None, + work_pool_filter: "WorkPoolFilter | None" = None, + work_queue_filter: "WorkQueueFilter | None" = None, + sort: "FlowSort | None" = None, + limit: int | None = None, + offset: int = 0, + ) -> list["Flow"]: + """ + Query the Prefect API for flows. Only flows matching all criteria will + be returned. + + Args: + flow_filter: filter criteria for flows + flow_run_filter: filter criteria for flow runs + task_run_filter: filter criteria for task runs + deployment_filter: filter criteria for deployments + work_pool_filter: filter criteria for work pools + work_queue_filter: filter criteria for work pool queues + sort: sort criteria for the flows + limit: limit for the flow query + offset: offset for the flow query + + Returns: + a list of Flow model representations of the flows + """ + body: dict[str, Any] = { + "flows": flow_filter.model_dump(mode="json") if flow_filter else None, + "flow_runs": ( + flow_run_filter.model_dump(mode="json", exclude_unset=True) + if flow_run_filter + else None + ), + "task_runs": ( + task_run_filter.model_dump(mode="json") if task_run_filter else None + ), + "deployments": ( + deployment_filter.model_dump(mode="json") if deployment_filter else None + ), + "work_pools": ( + work_pool_filter.model_dump(mode="json") if work_pool_filter else None + ), + "work_queues": ( + work_queue_filter.model_dump(mode="json") if work_queue_filter else None + ), + "sort": sort, + "limit": limit, + "offset": offset, + } + + response = self.request("POST", "/flows/filter", json=body) + from prefect.client.schemas.objects import Flow + + return Flow.model_validate_list(response.json()) + + def read_flow_by_name( + self, + flow_name: str, + ) -> "Flow": + """ + Query the Prefect API for a flow by name. + + Args: + flow_name: the name of a flow + + Returns: + a fully hydrated Flow model + """ + response = self.request( + "GET", "/flows/name/{name}", path_params={"name": flow_name} + ) + from prefect.client.schemas.objects import Flow + + return Flow.model_validate(response.json()) + class FlowAsyncClient(BaseAsyncClient): async def create_flow(self, flow: "FlowObject[Any, Any]") -> "UUID": From 34a97e2e70d6201880104c515f8086b2cb5f15b7 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 20:46:05 -0600 Subject: [PATCH 10/19] Update __init__.py --- src/prefect/client/orchestration/__init__.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index e96a022ed634..8ccd288ec738 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -149,13 +149,15 @@ def get_client( *, httpx_settings: Optional[dict[str, Any]] = ..., sync_client: Literal[False] = False, -) -> "PrefectClient": ... +) -> "PrefectClient": + ... @overload def get_client( *, httpx_settings: Optional[dict[str, Any]] = ..., sync_client: Literal[True] = ... -) -> "SyncPrefectClient": ... +) -> "SyncPrefectClient": + ... def get_client( From f9a8bba72866ef55704aed8b2346838a34d357fe Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 22:11:28 -0600 Subject: [PATCH 11/19] init --- src/prefect/client/orchestration/_blocks_documents/__init__.py | 0 src/prefect/client/orchestration/_blocks_documents/client.py | 0 src/prefect/client/orchestration/_blocks_schemas/__init__.py | 0 src/prefect/client/orchestration/_blocks_schemas/client.py | 0 src/prefect/client/orchestration/_blocks_types/__init__.py | 0 src/prefect/client/orchestration/_blocks_types/client.py | 0 6 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 src/prefect/client/orchestration/_blocks_documents/__init__.py create mode 100644 src/prefect/client/orchestration/_blocks_documents/client.py create mode 100644 src/prefect/client/orchestration/_blocks_schemas/__init__.py create mode 100644 src/prefect/client/orchestration/_blocks_schemas/client.py create mode 100644 src/prefect/client/orchestration/_blocks_types/__init__.py create mode 100644 src/prefect/client/orchestration/_blocks_types/client.py diff --git a/src/prefect/client/orchestration/_blocks_documents/__init__.py b/src/prefect/client/orchestration/_blocks_documents/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_blocks_documents/client.py b/src/prefect/client/orchestration/_blocks_documents/client.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_blocks_schemas/__init__.py b/src/prefect/client/orchestration/_blocks_schemas/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_blocks_schemas/client.py b/src/prefect/client/orchestration/_blocks_schemas/client.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_blocks_types/__init__.py b/src/prefect/client/orchestration/_blocks_types/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/prefect/client/orchestration/_blocks_types/client.py b/src/prefect/client/orchestration/_blocks_types/client.py new file mode 100644 index 000000000000..e69de29bb2d1 From 0095b3a4dcfd378fc05c3ef0dac4f0bd9b2ad560 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 22:11:45 -0600 Subject: [PATCH 12/19] jfc lint --- src/prefect/client/orchestration/__init__.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index e96a022ed634..8ccd288ec738 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -149,13 +149,15 @@ def get_client( *, httpx_settings: Optional[dict[str, Any]] = ..., sync_client: Literal[False] = False, -) -> "PrefectClient": ... +) -> "PrefectClient": + ... @overload def get_client( *, httpx_settings: Optional[dict[str, Any]] = ..., sync_client: Literal[True] = ... -) -> "SyncPrefectClient": ... +) -> "SyncPrefectClient": + ... def get_client( From e79f9399bc54a7b849519c5683e9f15024698756 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+AAAZZAM@users.noreply.github.com> Date: Thu, 2 Jan 2025 23:39:27 -0600 Subject: [PATCH 13/19] clients --- src/prefect/client/orchestration/__init__.py | 27 +- .../orchestration/_blocks_documents/client.py | 340 ++++++++++++++++ .../orchestration/_blocks_schemas/client.py | 200 ++++++++++ .../orchestration/_blocks_types/client.py | 376 ++++++++++++++++++ 4 files changed, 941 insertions(+), 2 deletions(-) diff --git a/src/prefect/client/orchestration/__init__.py b/src/prefect/client/orchestration/__init__.py index e96a022ed634..c725a8c2b342 100644 --- a/src/prefect/client/orchestration/__init__.py +++ b/src/prefect/client/orchestration/__init__.py @@ -58,6 +58,21 @@ FlowRunAsyncClient, ) +from prefect.client.orchestration._blocks_documents.client import ( + BlocksDocumentClient, + BlocksDocumentAsyncClient, +) + +from prefect.client.orchestration._blocks_schemas.client import ( + BlocksSchemaClient, + BlocksSchemaAsyncClient, +) + +from prefect.client.orchestration._blocks_types.client import ( + BlocksTypeClient, + BlocksTypeAsyncClient, +) + import prefect import prefect.exceptions import prefect.settings @@ -149,13 +164,15 @@ def get_client( *, httpx_settings: Optional[dict[str, Any]] = ..., sync_client: Literal[False] = False, -) -> "PrefectClient": ... +) -> "PrefectClient": + ... @overload def get_client( *, httpx_settings: Optional[dict[str, Any]] = ..., sync_client: Literal[True] = ... -) -> "SyncPrefectClient": ... +) -> "SyncPrefectClient": + ... def get_client( @@ -249,6 +266,9 @@ class PrefectClient( AutomationAsyncClient, FlowRunAsyncClient, FlowAsyncClient, + BlocksDocumentAsyncClient, + BlocksSchemaAsyncClient, + BlocksTypeAsyncClient, ): """ An asynchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). @@ -1863,6 +1883,9 @@ class SyncPrefectClient( AutomationClient, FlowRunClient, FlowClient, + BlocksDocumentClient, + BlocksSchemaClient, + BlocksTypeClient, ): """ A synchronous client for interacting with the [Prefect REST API](/api-ref/rest-api/). diff --git a/src/prefect/client/orchestration/_blocks_documents/client.py b/src/prefect/client/orchestration/_blocks_documents/client.py index e69de29bb2d1..ba9ce39687e2 100644 --- a/src/prefect/client/orchestration/_blocks_documents/client.py +++ b/src/prefect/client/orchestration/_blocks_documents/client.py @@ -0,0 +1,340 @@ +from __future__ import annotations + +from typing import TYPE_CHECKING + +from httpx import HTTPStatusError + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.exceptions import ObjectAlreadyExists, ObjectNotFound + +if TYPE_CHECKING: + from uuid import UUID + + from prefect.client.schemas.actions import ( + BlockDocumentCreate, + BlockDocumentUpdate, + ) + from prefect.client.schemas.objects import ( + BlockDocument, + ) + + +class BlocksDocumentClient(BaseClient): + def create_block_document( + self, + block_document: "BlockDocument | BlockDocumentCreate", + include_secrets: bool = True, + ) -> "BlockDocument": + """ + Create a block document in the Prefect API. This data is used to configure a + corresponding Block. + + Args: + include_secrets (bool): whether to include secret values + on the stored Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. Note Blocks may not work as expected if + this is set to `False`. + """ + block_document_data = block_document.model_dump( + mode="json", + exclude_unset=True, + exclude={"id", "block_schema", "block_type"}, + context={"include_secrets": include_secrets}, + serialize_as_any=True, + ) + try: + response = self.request( + "POST", + "/block_documents/", + json=block_document_data, + ) + except HTTPStatusError as e: + if e.response.status_code == 409: + raise ObjectAlreadyExists(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate(response.json()) + + def update_block_document( + self, + block_document_id: "UUID", + block_document: "BlockDocumentUpdate", + ) -> None: + """ + Update a block document in the Prefect API. + """ + try: + self.request( + "PATCH", + "/block_documents/{id}", + path_params={"id": block_document_id}, + json=block_document.model_dump( + mode="json", + exclude_unset=True, + include={"data", "merge_existing_data", "block_schema_id"}, + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def delete_block_document(self, block_document_id: "UUID") -> None: + """ + Delete a block document. + """ + try: + self.request( + "DELETE", + "/block_documents/{id}", + path_params={"id": block_document_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def read_block_document( + self, + block_document_id: "UUID", + include_secrets: bool = True, + ) -> "BlockDocument": + """ + Read the block document with the specified ID. + + Args: + block_document_id: the block document id + include_secrets (bool): whether to include secret values + on the Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. These fields are automatically obfuscated + by Pydantic, but users can additionally choose not to receive + their values from the API. Note that any business logic on the + Block may not work if this is `False`. + + Raises: + httpx.RequestError: if the block document was not found for any reason + + Returns: + A block document or None. + """ + + assert ( + block_document_id is not None + ), "Unexpected ID on block document. Was it persisted?" + try: + response = self.request( + "GET", + "/block_documents/{id}", + path_params={"id": block_document_id}, + params=dict(include_secrets=include_secrets), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate(response.json()) + + def read_block_documents( + self, + block_schema_type: "str | None" = None, + offset: "int | None" = None, + limit: "int | None" = None, + include_secrets: bool = True, + ) -> "list[BlockDocument]": + """ + Read block documents + + Args: + block_schema_type: an optional block schema type + offset: an offset + limit: the number of blocks to return + include_secrets (bool): whether to include secret values + on the Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. These fields are automatically obfuscated + by Pydantic, but users can additionally choose not to receive + their values from the API. Note that any business logic on the + Block may not work if this is `False`. + + Returns: + A list of block documents + """ + response = self.request( + "POST", + "/block_documents/filter", + json=dict( + block_schema_type=block_schema_type, + offset=offset, + limit=limit, + include_secrets=include_secrets, + ), + ) + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate_list(response.json()) + + +class BlocksDocumentAsyncClient(BaseAsyncClient): + async def create_block_document( + self, + block_document: "BlockDocument | BlockDocumentCreate", + include_secrets: bool = True, + ) -> "BlockDocument": + """ + Create a block document in the Prefect API. This data is used to configure a + corresponding Block. + + Args: + include_secrets (bool): whether to include secret values + on the stored Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. Note Blocks may not work as expected if + this is set to `False`. + """ + block_document_data = block_document.model_dump( + mode="json", + exclude_unset=True, + exclude={"id", "block_schema", "block_type"}, + context={"include_secrets": include_secrets}, + serialize_as_any=True, + ) + try: + response = await self.request( + "POST", + "/block_documents/", + json=block_document_data, + ) + except HTTPStatusError as e: + if e.response.status_code == 409: + raise ObjectAlreadyExists(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate(response.json()) + + async def update_block_document( + self, + block_document_id: "UUID", + block_document: "BlockDocumentUpdate", + ) -> None: + """ + Update a block document in the Prefect API. + """ + try: + await self.request( + "PATCH", + "/block_documents/{id}", + path_params={"id": block_document_id}, + json=block_document.model_dump( + mode="json", + exclude_unset=True, + include={"data", "merge_existing_data", "block_schema_id"}, + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def delete_block_document(self, block_document_id: "UUID") -> None: + """ + Delete a block document. + """ + try: + await self.request( + "DELETE", + "/block_documents/{id}", + path_params={"id": block_document_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def read_block_document( + self, + block_document_id: "UUID", + include_secrets: bool = True, + ) -> "BlockDocument": + """ + Read the block document with the specified ID. + + Args: + block_document_id: the block document id + include_secrets (bool): whether to include secret values + on the Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. These fields are automatically obfuscated + by Pydantic, but users can additionally choose not to receive + their values from the API. Note that any business logic on the + Block may not work if this is `False`. + + Raises: + httpx.RequestError: if the block document was not found for any reason + + Returns: + A block document or None. + """ + assert ( + block_document_id is not None + ), "Unexpected ID on block document. Was it persisted?" + try: + response = await self.request( + "GET", + "/block_documents/{id}", + path_params={"id": block_document_id}, + params=dict(include_secrets=include_secrets), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate(response.json()) + + async def read_block_documents( + self, + block_schema_type: "str | None" = None, + offset: "int | None" = None, + limit: "int | None" = None, + include_secrets: bool = True, + ) -> "list[BlockDocument]": + """ + Read block documents + + Args: + block_schema_type: an optional block schema type + offset: an offset + limit: the number of blocks to return + include_secrets (bool): whether to include secret values + on the Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. These fields are automatically obfuscated + by Pydantic, but users can additionally choose not to receive + their values from the API. Note that any business logic on the + Block may not work if this is `False`. + + Returns: + A list of block documents + """ + response = await self.request( + "POST", + "/block_documents/filter", + json=dict( + block_schema_type=block_schema_type, + offset=offset, + limit=limit, + include_secrets=include_secrets, + ), + ) + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate_list(response.json()) diff --git a/src/prefect/client/orchestration/_blocks_schemas/client.py b/src/prefect/client/orchestration/_blocks_schemas/client.py index e69de29bb2d1..6d4598b3f8e1 100644 --- a/src/prefect/client/orchestration/_blocks_schemas/client.py +++ b/src/prefect/client/orchestration/_blocks_schemas/client.py @@ -0,0 +1,200 @@ +from __future__ import annotations + +from typing import TYPE_CHECKING + +from httpx import HTTPStatusError + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.exceptions import ObjectAlreadyExists, ObjectNotFound + +if TYPE_CHECKING: + from uuid import UUID + + from prefect.client.schemas.actions import BlockSchemaCreate + from prefect.client.schemas.objects import BlockSchema + + +class BlocksSchemaClient(BaseClient): + def create_block_schema(self, block_schema: "BlockSchemaCreate") -> "BlockSchema": + """ + Create a block schema in the Prefect API. + """ + try: + response = self.request( + "POST", + "/block_schemas/", + json=block_schema.model_dump( + mode="json", + exclude_unset=True, + exclude={"id", "block_type", "checksum"}, + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 409: + raise ObjectAlreadyExists(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockSchema + + return BlockSchema.model_validate(response.json()) + + def read_block_schema_by_checksum( + self, checksum: str, version: str | None = None + ) -> "BlockSchema": + """ + Look up a block schema checksum + """ + try: + response = self.request( + "GET", + "/block_schemas/checksum/{checksum}", + path_params={"checksum": checksum}, + params={"version": version}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockSchema + + return BlockSchema.model_validate(response.json()) + + def read_block_schemas(self) -> "list[BlockSchema]": + """ + Read all block schemas + Raises: + httpx.RequestError: if a valid block schema was not found + + Returns: + A BlockSchema. + """ + response = self.request("POST", "/block_schemas/filter", json={}) + from prefect.client.schemas.objects import BlockSchema + + return BlockSchema.model_validate_list(response.json()) + + def get_most_recent_block_schema_for_block_type( + self, + block_type_id: "UUID", + ) -> "BlockSchema | None": + """ + Fetches the most recent block schema for a specified block type ID. + + Args: + block_type_id: The ID of the block type. + + Raises: + httpx.RequestError: If the request fails for any reason. + + Returns: + The most recent block schema or None. + """ + try: + response = self.request( + "POST", + "/block_schemas/filter", + json={ + "block_schemas": {"block_type_id": {"any_": [str(block_type_id)]}}, + "limit": 1, + }, + ) + except HTTPStatusError: + raise + from prefect.client.schemas.objects import BlockSchema + + return next(iter(BlockSchema.model_validate_list(response.json())), None) + + +class BlocksSchemaAsyncClient(BaseAsyncClient): + async def create_block_schema( + self, block_schema: "BlockSchemaCreate" + ) -> "BlockSchema": + """ + Create a block schema in the Prefect API. + """ + try: + response = await self.request( + "POST", + "/block_schemas/", + json=block_schema.model_dump( + mode="json", + exclude_unset=True, + exclude={"id", "block_type", "checksum"}, + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 409: + raise ObjectAlreadyExists(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockSchema + + return BlockSchema.model_validate(response.json()) + + async def read_block_schema_by_checksum( + self, checksum: str, version: str | None = None + ) -> "BlockSchema": + """ + Look up a block schema checksum + """ + try: + response = await self.request( + "GET", + "/block_schemas/checksum/{checksum}", + path_params={"checksum": checksum}, + params={"version": version}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockSchema + + return BlockSchema.model_validate(response.json()) + + async def read_block_schemas(self) -> "list[BlockSchema]": + """ + Read all block schemas + Raises: + httpx.RequestError: if a valid block schema was not found + + Returns: + A BlockSchema. + """ + response = await self.request("POST", "/block_schemas/filter", json={}) + from prefect.client.schemas.objects import BlockSchema + + return BlockSchema.model_validate_list(response.json()) + + async def get_most_recent_block_schema_for_block_type( + self, + block_type_id: "UUID", + ) -> "BlockSchema | None": + """ + Fetches the most recent block schema for a specified block type ID. + + Args: + block_type_id: The ID of the block type. + + Raises: + httpx.RequestError: If the request fails for any reason. + + Returns: + The most recent block schema or None. + """ + try: + response = await self.request( + "POST", + "/block_schemas/filter", + json={ + "block_schemas": {"block_type_id": {"any_": [str(block_type_id)]}}, + "limit": 1, + }, + ) + except HTTPStatusError: + raise + from prefect.client.schemas.objects import BlockSchema + + return next(iter(BlockSchema.model_validate_list(response.json())), None) diff --git a/src/prefect/client/orchestration/_blocks_types/client.py b/src/prefect/client/orchestration/_blocks_types/client.py index e69de29bb2d1..c37be376b63f 100644 --- a/src/prefect/client/orchestration/_blocks_types/client.py +++ b/src/prefect/client/orchestration/_blocks_types/client.py @@ -0,0 +1,376 @@ +from __future__ import annotations + +from typing import TYPE_CHECKING + +from httpx import HTTPStatusError + +from prefect.client.orchestration.base import BaseAsyncClient, BaseClient +from prefect.exceptions import ObjectAlreadyExists, ObjectNotFound, ProtectedBlockError + +if TYPE_CHECKING: + from uuid import UUID + + from prefect.client.schemas.actions import ( + BlockTypeCreate, + BlockTypeUpdate, + ) + from prefect.client.schemas.objects import ( + BlockDocument, + BlockType, + ) + + +class BlocksTypeClient(BaseClient): + def read_block_documents_by_type( + self, + block_type_slug: str, + offset: "int | None" = None, + limit: "int | None" = None, + include_secrets: bool = True, + ) -> "list[BlockDocument]": + """Retrieve block documents by block type slug. + + Args: + block_type_slug: The block type slug. + offset: an offset + limit: the number of blocks to return + include_secrets: whether to include secret values + + Returns: + A list of block documents + """ + response = self.request( + "GET", + "/block_types/slug/{slug}/block_documents", + path_params={"slug": block_type_slug}, + params=dict( + offset=offset, + limit=limit, + include_secrets=include_secrets, + ), + ) + + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate_list(response.json()) + + def create_block_type(self, block_type: "BlockTypeCreate") -> "BlockType": + """ + Create a block type in the Prefect API. + """ + try: + response = self.request( + "POST", + "/block_types/", + json=block_type.model_dump( + mode="json", exclude_unset=True, exclude={"id"} + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 409: + raise ObjectAlreadyExists(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockType + + return BlockType.model_validate(response.json()) + + def read_block_type_by_slug(self, slug: str) -> "BlockType": + """ + Read a block type by its slug. + """ + try: + response = self.request( + "GET", + "/block_types/slug/{slug}", + path_params={"slug": slug}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockType + + return BlockType.model_validate(response.json()) + + def update_block_type( + self, block_type_id: "UUID", block_type: "BlockTypeUpdate" + ) -> None: + """ + Update a block document in the Prefect API. + """ + try: + self.request( + "PATCH", + "/block_types/{id}", + path_params={"id": block_type_id}, + json=block_type.model_dump( + mode="json", + exclude_unset=True, + include=BlockTypeUpdate.updatable_fields(), + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + def delete_block_type(self, block_type_id: "UUID") -> None: + """ + Delete a block type. + """ + try: + self.request( + "DELETE", + "/block_types/{id}", + path_params={"id": block_type_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + elif ( + e.response.status_code == 403 + and e.response.json()["detail"] + == "protected block types cannot be deleted." + ): + raise ProtectedBlockError( + "Protected block types cannot be deleted." + ) from e + else: + raise + + def read_block_types(self) -> "list[BlockType]": + """ + Read all block types + Raises: + httpx.RequestError: if the block types were not found + + Returns: + List of BlockTypes. + """ + response = self.request("POST", "/block_types/filter", json={}) + from prefect.client.schemas.objects import BlockType + + return BlockType.model_validate_list(response.json()) + + def read_block_document_by_name( + self, + name: str, + block_type_slug: str, + include_secrets: bool = True, + ) -> "BlockDocument": + """ + Read the block document with the specified name that corresponds to a + specific block type name. + + Args: + name: The block document name. + block_type_slug: The block type slug. + include_secrets (bool): whether to include secret values + on the Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. These fields are automatically obfuscated + by Pydantic, but users can additionally choose not to receive + their values from the API. Note that any business logic on the + Block may not work if this is `False`. + + Raises: + httpx.RequestError: if the block document was not found for any reason + + Returns: + A block document or None. + """ + try: + response = self.request( + "GET", + "/block_types/slug/{slug}/block_documents/name/{block_document_name}", + path_params={"slug": block_type_slug, "block_document_name": name}, + params=dict(include_secrets=include_secrets), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate(response.json()) + + +class BlocksTypeAsyncClient(BaseAsyncClient): + async def read_block_documents_by_type( + self, + block_type_slug: str, + offset: "int | None" = None, + limit: "int | None" = None, + include_secrets: bool = True, + ) -> "list[BlockDocument]": + """Retrieve block documents by block type slug. + + Args: + block_type_slug: The block type slug. + offset: an offset + limit: the number of blocks to return + include_secrets: whether to include secret values + + Returns: + A list of block documents + """ + response = await self.request( + "GET", + "/block_types/slug/{slug}/block_documents", + path_params={"slug": block_type_slug}, + params=dict( + offset=offset, + limit=limit, + include_secrets=include_secrets, + ), + ) + + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate_list(response.json()) + + async def create_block_type(self, block_type: "BlockTypeCreate") -> "BlockType": + """ + Create a block type in the Prefect API. + """ + try: + response = await self.request( + "POST", + "/block_types/", + json=block_type.model_dump( + mode="json", exclude_unset=True, exclude={"id"} + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 409: + raise ObjectAlreadyExists(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockType + + return BlockType.model_validate(response.json()) + + async def read_block_type_by_slug(self, slug: str) -> "BlockType": + """ + Read a block type by its slug. + """ + try: + response = await self.request( + "GET", + "/block_types/slug/{slug}", + path_params={"slug": slug}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockType + + return BlockType.model_validate(response.json()) + + async def update_block_type( + self, block_type_id: "UUID", block_type: "BlockTypeUpdate" + ) -> None: + """ + Update a block document in the Prefect API. + """ + try: + await self.request( + "PATCH", + "/block_types/{id}", + path_params={"id": block_type_id}, + json=block_type.model_dump( + mode="json", + exclude_unset=True, + include=BlockTypeUpdate.updatable_fields(), + ), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + + async def delete_block_type(self, block_type_id: "UUID") -> None: + """ + Delete a block type. + """ + try: + await self.request( + "DELETE", + "/block_types/{id}", + path_params={"id": block_type_id}, + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + elif ( + e.response.status_code == 403 + and e.response.json()["detail"] + == "protected block types cannot be deleted." + ): + raise ProtectedBlockError( + "Protected block types cannot be deleted." + ) from e + else: + raise + + async def read_block_types(self) -> "list[BlockType]": + """ + Read all block types + Raises: + httpx.RequestError: if the block types were not found + + Returns: + List of BlockTypes. + """ + response = await self.request("POST", "/block_types/filter", json={}) + from prefect.client.schemas.objects import BlockType + + return BlockType.model_validate_list(response.json()) + + async def read_block_document_by_name( + self, + name: str, + block_type_slug: str, + include_secrets: bool = True, + ) -> "BlockDocument": + """ + Read the block document with the specified name that corresponds to a + specific block type name. + + Args: + name: The block document name. + block_type_slug: The block type slug. + include_secrets (bool): whether to include secret values + on the Block, corresponding to Pydantic's `SecretStr` and + `SecretBytes` fields. These fields are automatically obfuscated + by Pydantic, but users can additionally choose not to receive + their values from the API. Note that any business logic on the + Block may not work if this is `False`. + + Raises: + httpx.RequestError: if the block document was not found for any reason + + Returns: + A block document or None. + """ + try: + response = await self.request( + "GET", + "/block_types/slug/{slug}/block_documents/name/{block_document_name}", + path_params={"slug": block_type_slug, "block_document_name": name}, + params=dict(include_secrets=include_secrets), + ) + except HTTPStatusError as e: + if e.response.status_code == 404: + raise ObjectNotFound(http_exc=e) from e + else: + raise + from prefect.client.schemas.objects import BlockDocument + + return BlockDocument.model_validate(response.json()) From ab65931225ae3caa40ebee88f9ba593f610c1c43 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+aaazzam@users.noreply.github.com> Date: Fri, 3 Jan 2025 14:22:55 -0600 Subject: [PATCH 14/19] Update src/prefect/client/orchestration/_blocks_documents/client.py Co-authored-by: Alex Streed --- src/prefect/client/orchestration/_blocks_documents/client.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/prefect/client/orchestration/_blocks_documents/client.py b/src/prefect/client/orchestration/_blocks_documents/client.py index ba9ce39687e2..4aa030811ae4 100644 --- a/src/prefect/client/orchestration/_blocks_documents/client.py +++ b/src/prefect/client/orchestration/_blocks_documents/client.py @@ -122,9 +122,6 @@ def read_block_document( A block document or None. """ - assert ( - block_document_id is not None - ), "Unexpected ID on block document. Was it persisted?" try: response = self.request( "GET", From 9d47c2ddb632c40265e40eda5b7ae1cbd18ebe58 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+aaazzam@users.noreply.github.com> Date: Fri, 3 Jan 2025 14:23:00 -0600 Subject: [PATCH 15/19] Update src/prefect/client/orchestration/_blocks_documents/client.py Co-authored-by: Alex Streed --- .../client/orchestration/_blocks_documents/client.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/prefect/client/orchestration/_blocks_documents/client.py b/src/prefect/client/orchestration/_blocks_documents/client.py index 4aa030811ae4..064ea6d010f1 100644 --- a/src/prefect/client/orchestration/_blocks_documents/client.py +++ b/src/prefect/client/orchestration/_blocks_documents/client.py @@ -140,9 +140,9 @@ def read_block_document( def read_block_documents( self, - block_schema_type: "str | None" = None, - offset: "int | None" = None, - limit: "int | None" = None, + block_schema_type: str | None = None, + offset: int | None = None, + limit: int | None = None, include_secrets: bool = True, ) -> "list[BlockDocument]": """ From 609c7358fb7b1f4f785c03ee74bd3f2e9290dd1d Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+aaazzam@users.noreply.github.com> Date: Fri, 3 Jan 2025 14:23:05 -0600 Subject: [PATCH 16/19] Update src/prefect/client/orchestration/_blocks_documents/client.py Co-authored-by: Alex Streed --- src/prefect/client/orchestration/_blocks_documents/client.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/prefect/client/orchestration/_blocks_documents/client.py b/src/prefect/client/orchestration/_blocks_documents/client.py index 064ea6d010f1..e8d921c9b1f1 100644 --- a/src/prefect/client/orchestration/_blocks_documents/client.py +++ b/src/prefect/client/orchestration/_blocks_documents/client.py @@ -279,9 +279,6 @@ async def read_block_document( Returns: A block document or None. """ - assert ( - block_document_id is not None - ), "Unexpected ID on block document. Was it persisted?" try: response = await self.request( "GET", From 7e5251db6e8e1871c291a07ede483b639db922b5 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+aaazzam@users.noreply.github.com> Date: Fri, 3 Jan 2025 14:23:11 -0600 Subject: [PATCH 17/19] Update src/prefect/client/orchestration/_blocks_types/client.py Co-authored-by: Alex Streed --- src/prefect/client/orchestration/_blocks_types/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/prefect/client/orchestration/_blocks_types/client.py b/src/prefect/client/orchestration/_blocks_types/client.py index c37be376b63f..4dadcf62408c 100644 --- a/src/prefect/client/orchestration/_blocks_types/client.py +++ b/src/prefect/client/orchestration/_blocks_types/client.py @@ -202,8 +202,8 @@ class BlocksTypeAsyncClient(BaseAsyncClient): async def read_block_documents_by_type( self, block_type_slug: str, - offset: "int | None" = None, - limit: "int | None" = None, + offset: int | None = None, + limit: int | None = None, include_secrets: bool = True, ) -> "list[BlockDocument]": """Retrieve block documents by block type slug. From aa921ab55e9da072857b433247addba4766054f0 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+aaazzam@users.noreply.github.com> Date: Fri, 3 Jan 2025 14:23:17 -0600 Subject: [PATCH 18/19] Update src/prefect/client/orchestration/_blocks_documents/client.py Co-authored-by: Alex Streed --- .../client/orchestration/_blocks_documents/client.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/prefect/client/orchestration/_blocks_documents/client.py b/src/prefect/client/orchestration/_blocks_documents/client.py index e8d921c9b1f1..08ef6db18f90 100644 --- a/src/prefect/client/orchestration/_blocks_documents/client.py +++ b/src/prefect/client/orchestration/_blocks_documents/client.py @@ -297,9 +297,9 @@ async def read_block_document( async def read_block_documents( self, - block_schema_type: "str | None" = None, - offset: "int | None" = None, - limit: "int | None" = None, + block_schema_type: str | None = None, + offset: int | None = None, + limit: int | None = None, include_secrets: bool = True, ) -> "list[BlockDocument]": """ From 771293734db4b9d64e8bc536d9cd8c9d0ff945e7 Mon Sep 17 00:00:00 2001 From: Adam Azzam <33043305+aaazzam@users.noreply.github.com> Date: Fri, 3 Jan 2025 14:23:25 -0600 Subject: [PATCH 19/19] Update src/prefect/client/orchestration/_blocks_types/client.py Co-authored-by: Alex Streed --- src/prefect/client/orchestration/_blocks_types/client.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/prefect/client/orchestration/_blocks_types/client.py b/src/prefect/client/orchestration/_blocks_types/client.py index 4dadcf62408c..921a8d08c68d 100644 --- a/src/prefect/client/orchestration/_blocks_types/client.py +++ b/src/prefect/client/orchestration/_blocks_types/client.py @@ -24,8 +24,8 @@ class BlocksTypeClient(BaseClient): def read_block_documents_by_type( self, block_type_slug: str, - offset: "int | None" = None, - limit: "int | None" = None, + offset: int | None = None, + limit: int | None = None, include_secrets: bool = True, ) -> "list[BlockDocument]": """Retrieve block documents by block type slug.