From 7666cdc2037ea457f76fb7eac0b51d46a0ce5375 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Thu, 2 Oct 2025 12:46:35 -0400 Subject: [PATCH 01/25] Sketch standalone activity client APIs --- temporalio/client.py | 660 ++++++++++++++++++++++++++++++++-- temporalio/common.py | 41 +++ temporalio/exceptions.py | 6 + tests/worker/test_workflow.py | 2 +- 4 files changed, 678 insertions(+), 31 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index 6c26d41ef..18feaae8d 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -10,6 +10,7 @@ import inspect import json import re +import typing import uuid import warnings from abc import ABC, abstractmethod @@ -1282,25 +1283,193 @@ async def count_workflows( ) ) + # - TODO: Overloads for no-param, single-param, multi-param + # - TODO: Support sync and async activity functions + async def start_activity( + self, + activity: Callable[..., ReturnType], + *, + args: Sequence[Any], + id: str, + task_queue: str, + # Either schedule_to_close_timeout or start_to_close_timeout must be present + schedule_to_close_timeout: Optional[timedelta] = None, + start_to_close_timeout: Optional[timedelta] = None, + schedule_to_start_timeout: Optional[timedelta] = None, + heartbeat_timeout: Optional[timedelta] = None, + id_reuse_policy: temporalio.common.WorkflowIDReusePolicy = temporalio.common.WorkflowIDReusePolicy.ALLOW_DUPLICATE, + id_conflict_policy: temporalio.common.WorkflowIDConflictPolicy = temporalio.common.WorkflowIDConflictPolicy.FAIL, + retry_policy: Optional[temporalio.common.RetryPolicy] = None, + search_attributes: Optional[ + Union[ + temporalio.common.SearchAttributes, + temporalio.common.TypedSearchAttributes, + ] + ] = None, + static_summary: Optional[str] = None, + static_details: Optional[str] = None, + priority: temporalio.common.Priority = temporalio.common.Priority.default, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> ActivityHandle[ReturnType]: + """Start an activity and return its handle. + + Args: + activity: The activity function to execute. + args: Arguments to pass to the activity. + id: Unique identifier for the activity. Required. + task_queue: Task queue to send the activity to. + schedule_to_close_timeout: Total time allowed for the activity from schedule to completion. + start_to_close_timeout: Time allowed for a single execution attempt. + schedule_to_start_timeout: Time allowed for the activity to sit in the task queue. + heartbeat_timeout: Time between heartbeats before the activity is considered failed. + id_reuse_policy: How to handle reusing activity IDs from closed activities. + id_conflict_policy: How to handle activity ID conflicts with running activities. + retry_policy: Retry policy for the activity. + search_attributes: Search attributes to attach to the activity. + static_summary: A single-line fixed summary for this workflow execution that may appear + in the UI/CLI. This can be in single-line Temporal markdown format. + static_details: General fixed details for this workflow execution that may appear in + UI/CLI. This can be in Temporal markdown format and can span multiple lines. This is + a fixed value on the workflow that cannot be updated. For details that can be + updated, use :py:meth:`temporalio.workflow.get_current_details` within the workflow. + priority: Priority metadata. + rpc_metadata: Headers used on the RPC call. + rpc_timeout: Optional RPC deadline to set for the RPC call. + + Returns: + A handle to the started activity. + """ + # Issues workflowservice StartActivityExecution + raise NotImplementedError + + # Same parameters as start_activity + # (*args **kwargs is just temporary to avoid duplicating parameter lists while they're being designed) + async def execute_activity(self, *args, **kwargs) -> ReturnType: + """ + Start an activity, wait for it to complete, and return its result. + """ + handle = await self.start_activity(*args, **kwargs) + return await handle.result() + + async def list_activities( + self, + query: Optional[str] = None, + *, + page_size: int = 1000, + next_page_token: Optional[bytes] = None, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> ActivityExecutionAsyncIterator: + """List activities. + + Args: + query: A Temporal visibility filter for activities. + page_size: Maximum number of results to return per page. + next_page_token: Token for getting the next page of results. + rpc_metadata: Headers used on the RPC call. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + # Issues a workflowservice ListActivityExecutions call + raise NotImplementedError + + async def count_activities( + self, + query: Optional[str] = None, + *, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> int: + """Count activities matching the query. + + Args: + query: A Temporal visibility filter for activities. + rpc_metadata: Headers used on the RPC call. + rpc_timeout: Optional RPC deadline to set for the RPC call. + + Returns: + Count of activities. + """ + # Issues a workflowservice CountActivityExecutions call + raise NotImplementedError + + @typing.overload + def get_activity_handle( + self, + *, + activity_id: str, + run_id: Optional[str] = None, + ) -> ActivityHandle[Any]: + raise NotImplementedError + + @typing.overload + def get_activity_handle( + self, + *, + activity_id: str, + workflow_id: str, + run_id: Optional[str], + ) -> WorkflowActivityHandle: + raise NotImplementedError + + @typing.overload + def get_activity_handle( + self, + *, + task_token: bytes, + ) -> WorkflowActivityHandle: + raise NotImplementedError + + def get_activity_handle( + self, + *, + activity_id: Optional[str] = None, + workflow_id: Optional[str] = None, + run_id: Optional[str] = None, + task_token: Optional[bytes] = None, + ) -> Union[ActivityHandle[Any], WorkflowActivityHandle]: + """Get a handle to an existing activity. + + Args: + activity_id: The activity ID. + workflow_id: The workflow ID if the activity was started from a workflow. + run_id: The run ID. If not provided, targets the latest run. + task_token: Optional task token for the activity if the activity was + started from a workflow. Cannot be set if any of the id parameters + are set. + + Returns: + A handle to the activity. + """ + raise NotImplementedError + + # Deprecated: get_activity_handle has an equivalent override @overload - def get_async_activity_handle( + def get_workflow_activity_handle( self, *, workflow_id: str, run_id: Optional[str], activity_id: str - ) -> AsyncActivityHandle: + ) -> WorkflowActivityHandle: pass + # Deprecated: get_activity_handle has an equivalent override @overload - def get_async_activity_handle(self, *, task_token: bytes) -> AsyncActivityHandle: + def get_workflow_activity_handle( + self, *, task_token: bytes + ) -> WorkflowActivityHandle: pass - def get_async_activity_handle( + def get_workflow_activity_handle( self, *, workflow_id: Optional[str] = None, run_id: Optional[str] = None, activity_id: Optional[str] = None, task_token: Optional[bytes] = None, - ) -> AsyncActivityHandle: - """Get an async activity handle. + ) -> WorkflowActivityHandle: + """Get a handle to an activity started by a workflow. + + .. warning:: + DEPRECATED: This method is deprecated. + Use :py:meth:`Client.get_activity_handle` instead. Either the workflow_id, run_id, and activity_id can be provided, or a singular task_token can be provided. @@ -1320,13 +1489,13 @@ def get_async_activity_handle( if task_token is not None: if workflow_id is not None or run_id is not None or activity_id is not None: raise ValueError("Task token cannot be present with other IDs") - return AsyncActivityHandle(self, task_token) + return WorkflowActivityHandle(self, task_token) elif workflow_id is not None: if activity_id is None: raise ValueError( "Workflow ID, run ID, and activity ID must all be given together" ) - return AsyncActivityHandle( + return WorkflowActivityHandle( self, AsyncActivityIDReference( workflow_id=workflow_id, run_id=run_id, activity_id=activity_id @@ -1334,6 +1503,9 @@ def get_async_activity_handle( ) raise ValueError("Task token or workflow/run/activity ID must be present") + # Deprecated alias + get_async_activity_handle = get_workflow_activity_handle + async def create_schedule( self, id: str, @@ -2730,28 +2902,190 @@ async def workflow_handle(self) -> WorkflowHandle[SelfType, ReturnType]: return await self._workflow_handle +class ActivityExecutionAsyncIterator: + """Asynchronous iterator for activity execution values. + + Returns either :py:class:`ActivityExecution` (for standalone activities) or + :py:class:`WorkflowActivityExecution` (for activities started by workflows). + """ + + def __aiter__(self) -> ActivityExecutionAsyncIterator: + """Return self as the iterator.""" + return self + + async def __anext__(self) -> Union[ActivityExecution, WorkflowActivityExecution]: + """Return the next execution on this iterator. + + Fetch next page if necessary. + """ + raise NotImplementedError + + +# TODO: this is named ActivityListInfo in our draft proto PR +# https://github.com/temporalio/api/pull/640/files +@dataclass(frozen=True) +class ActivityExecution: + """Info for a standalone activity execution from list response.""" + + activity_id: str + """Activity ID.""" + + run_id: str + """Run ID of the activity.""" + + activity_type: str + """Type name of the activity.""" + + scheduled_time: datetime + """Time the activity was originally scheduled.""" + + close_time: Optional[datetime] + """Time the activity reached a terminal status, if closed.""" + + status: temporalio.common.ActivityExecutionStatus + """Current status of the activity.""" + + search_attributes: temporalio.common.SearchAttributes + """Search attributes from the start request.""" + + task_queue: str + """Task queue the activity was scheduled on.""" + + state_transition_count: int + """Number of state transitions.""" + + execution_duration: Optional[timedelta] + """Duration from scheduled to close time, only populated if closed.""" + + @dataclass(frozen=True) -class AsyncActivityIDReference: - """Reference to an async activity by its qualified ID.""" +class WorkflowActivityExecution: + """Info for a workflow activity execution from list response.""" workflow_id: str + """ID of the workflow that started this activity.""" + + workflow_run_id: Optional[str] + """Run ID of the workflow that started this activity.""" + + activity_id: str + """Activity ID.""" + + activity_type: str + """Type name of the activity.""" + + scheduled_time: datetime + """Time the activity was originally scheduled.""" + + close_time: Optional[datetime] + """Time the activity reached a terminal status, if closed.""" + + task_queue: str + """Task queue the activity was scheduled on.""" + + execution_duration: Optional[timedelta] + """Duration from scheduled to close time, only populated if closed.""" + + +@dataclass(frozen=True) +class ActivityExecutionDescription: + """Detailed information about an activity execution from describe response.""" + + activity_id: str + """Activity ID.""" + + run_id: str + """Run ID of the activity.""" + + activity_type: str + """Type name of the activity.""" + + status: temporalio.common.ActivityExecutionStatus + """Current status of the activity.""" + + run_state: Optional[temporalio.common.PendingActivityState] + """More detailed breakdown if status is RUNNING.""" + + heartbeat_details: Sequence[Any] + """Details from the last heartbeat.""" + + last_heartbeat_time: Optional[datetime] + """Time of the last heartbeat.""" + + last_started_time: Optional[datetime] + """Time the last attempt was started.""" + + attempt: int + """Current attempt number.""" + + maximum_attempts: int + """Maximum number of attempts allowed.""" + + scheduled_time: datetime + """Time the activity was originally scheduled.""" + + expiration_time: datetime + """Scheduled time plus schedule_to_close_timeout.""" + + last_failure: Optional[Exception] + """Failure from the last failed attempt, if any.""" + + last_worker_identity: str + """Identity of the last worker that processed the activity.""" + + current_retry_interval: Optional[timedelta] + """Time until the next retry, if applicable.""" + + last_attempt_complete_time: Optional[datetime] + """Time when the last attempt completed.""" + + next_attempt_schedule_time: Optional[datetime] + """Time when the next attempt will be scheduled.""" + + task_queue: str + """Task queue the activity is scheduled on.""" + + paused: bool + """Whether the activity is paused.""" + + input: Sequence[Any] + """Serialized activity input.""" + + state_transition_count: int + """Number of state transitions.""" + + search_attributes: temporalio.common.SearchAttributes + """Search attributes.""" + + eager_execution_requested: bool + """Whether eager execution was requested.""" + + canceled_reason: Optional[str] + """Reason for cancellation, if cancel was requested.""" + + raw_info: Any + """Raw proto response.""" + + +@dataclass(frozen=True) +class ActivityIDReference: + """Information identifying an activity execution.""" + + workflow_id: Optional[str] run_id: Optional[str] activity_id: str -class AsyncActivityHandle(WithSerializationContext): - """Handle representing an external activity for completion and heartbeat.""" +# Deprecated alias +AsyncActivityIDReference = ActivityIDReference - def __init__( - self, - client: Client, - id_or_token: Union[AsyncActivityIDReference, bytes], - data_converter_override: Optional[DataConverter] = None, - ) -> None: - """Create an async activity handle.""" - self._client = client - self._id_or_token = id_or_token - self._data_converter_override = data_converter_override + +class _BaseActivityHandle(WithSerializationContext): + """Handle representing an activity.""" + + _client: Client + _id_or_token: Union[ActivityIDReference, bytes] + _data_converter_override: Optional[DataConverter] async def heartbeat( self, @@ -2854,6 +3188,126 @@ async def report_cancellation( ), ) + async def pause( + self, + *, + reason: Optional[str] = None, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Pause the activity. + + Args: + reason: Reason for pausing the activity. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + if not isinstance(self._id_or_token, ActivityIDReference): + raise ValueError("Cannot pause activity with task token") + + await self._client.workflow_service.pause_activity( + temporalio.api.workflowservice.v1.PauseActivityRequest( + namespace=self._client.namespace, + execution=temporalio.api.common.v1.WorkflowExecution( + workflow_id=self._id_or_token.workflow_id or "", + run_id=self._id_or_token.run_id or "", + ), + identity=self._client.identity, + id=self._id_or_token.activity_id, + reason=reason or "", + ), + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + + async def unpause( + self, + *, + reset_attempts: bool = False, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Unpause the activity. + + Args: + reset_attempts: Whether to reset the number of attempts. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + if not isinstance(self._id_or_token, ActivityIDReference): + raise ValueError("Cannot unpause activity with task token") + + await self._client.workflow_service.unpause_activity( + temporalio.api.workflowservice.v1.UnpauseActivityRequest( + namespace=self._client.namespace, + execution=temporalio.api.common.v1.WorkflowExecution( + workflow_id=self._id_or_token.workflow_id or "", + run_id=self._id_or_token.run_id or "", + ), + identity=self._client.identity, + id=self._id_or_token.activity_id, + reset_attempts=reset_attempts, + ), + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + + async def reset( + self, + *, + reset_heartbeat: bool = False, + keep_paused: bool = False, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Reset the activity. + + Args: + reset_heartbeat: Whether to reset heartbeat details. + keep_paused: If activity is paused, whether to keep it paused after reset. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + if not isinstance(self._id_or_token, ActivityIDReference): + raise ValueError("Cannot reset activity with task token") + + await self._client.workflow_service.reset_activity( + temporalio.api.workflowservice.v1.ResetActivityRequest( + namespace=self._client.namespace, + execution=temporalio.api.common.v1.WorkflowExecution( + workflow_id=self._id_or_token.workflow_id or "", + run_id=self._id_or_token.run_id or "", + ), + identity=self._client.identity, + id=self._id_or_token.activity_id, + reset_heartbeat=reset_heartbeat, + keep_paused=keep_paused, + ), + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + + +class WorkflowActivityHandle(_BaseActivityHandle): + """Handle representing an activity started by a workflow.""" + + def __init__( + self, + client: Client, + id_or_token: Union[ActivityIDReference, bytes], + data_converter_override: Optional[DataConverter] = None, + ) -> None: + """Create an async activity handle.""" + self._client = client + self._id_or_token = id_or_token + self._data_converter_override = data_converter_override + def with_context(self, context: SerializationContext) -> Self: """Create a new AsyncActivityHandle with a different serialization context. @@ -2878,6 +3332,137 @@ def with_context(self, context: SerializationContext) -> Self: ) +# Deprecated alias +AsyncActivityHandle = WorkflowActivityHandle + + +# TODO: in the future when messages can be sent to activities, we will want the activity handle to +# be generic in the activity type in addition to the return type (as WorkflowHandle), to support +# static type inference for signal/query/update. +class ActivityHandle(Generic[ReturnType], _BaseActivityHandle): + """Handle representing a standalone activity execution.""" + + def __init__( + self, + client: Client, + activity_id_reference: ActivityIDReference, + data_converter_override: Optional[DataConverter] = None, + ) -> None: + """Create an async activity handle.""" + self._client = client + self._id_or_token = activity_id_reference + self._data_converter_override = data_converter_override + + def with_context(self, context: SerializationContext) -> Self: + """Create a new AsyncActivityHandle with a different serialization context. + + Payloads received by the activity will be decoded and deserialized using a data converter + with :py:class:`ActivitySerializationContext` set as context. If you are using a custom data + converter that makes use of this context then you can use this method to supply matching + context data to the data converter used to serialize and encode the outbound payloads. + """ + data_converter = self._client.data_converter.with_context(context) + if data_converter is self._client.data_converter: + return self + cls = type(self) + if cls.__init__ is not ActivityHandle.__init__: + raise TypeError( + "If you have subclassed AsyncActivityHandle and overridden the __init__ method " + "then you must override with_context to return an instance of your class." + ) + return cls( + self._client, + cast(ActivityIDReference, self._id_or_token), + data_converter, + ) + + async def result( + self, + *, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> ReturnType: + """Wait for result of the activity. + + Args: + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for each RPC call. Note, + this is the timeout for each history RPC call not this overall + function. + + Returns: + The result of the activity. + + Raises: + :py:class:`ActivityFailureError`: If the activity completed with a failure. + """ + # Repeatedly issues workflowservice GetActivityResult long-polls. + raise NotImplementedError + + async def cancel( + self, + *, + reason: Optional[str] = None, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Request cancellation of the activity. + + Requesting cancellation of an activity does not automatically transition the activity to + canceled status. If the activity is heartbeating, a :py:class:`exceptions.CancelledError` + exception will be raised when receiving the heartbeat response; if the activity allows this + exception to bubble out, the activity will transition to canceled status. If the activity it + is not heartbeating, this method will have no effect on activity status. + + Args: + reason: Reason for the cancellation. Recorded and available via describe. + rpc_metadata: Headers used on the RPC call. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + raise NotImplementedError + + async def terminate( + self, + *, + reason: Optional[str] = None, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Terminate the activity execution immediately. + + Termination does not reach the worker and the activity code cannot react to it. + A terminated activity may have a running attempt and will be requested to be + canceled by the server when it heartbeats. + + Args: + reason: Reason for the termination. + rpc_metadata: Headers used on the RPC call. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + raise NotImplementedError + + async def describe( + self, + *, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> ActivityExecutionDescription: + """Describe the activity execution. + + Args: + rpc_metadata: Headers used on the RPC call. + rpc_timeout: Optional RPC deadline to set for the RPC call. + + Returns: + Activity execution description. + """ + raise NotImplementedError + + # TODO: + # update_options + + @dataclass class WorkflowExecution: """Info for a single workflow execution run.""" @@ -5305,6 +5890,21 @@ def __init__(self) -> None: super().__init__("Timeout or cancellation waiting for update") +class ActivityFailureError(temporalio.exceptions.TemporalError): + """Error that occurs when a standalone activity is unsuccessful.""" + + def __init__(self, *, cause: BaseException) -> None: + """Create activity failure error.""" + super().__init__("Activity execution failed") + self.__cause__ = cause + + @property + def cause(self) -> BaseException: + """Cause of the activity failure.""" + assert self.__cause__ + return self.__cause__ + + class AsyncActivityCancelledError(temporalio.exceptions.TemporalError): """Error that occurs when async activity attempted heartbeat but was cancelled.""" @@ -5544,7 +6144,7 @@ class StartWorkflowUpdateWithStartInput: class HeartbeatAsyncActivityInput: """Input for :py:meth:`OutboundInterceptor.heartbeat_async_activity`.""" - id_or_token: Union[AsyncActivityIDReference, bytes] + id_or_token: Union[ActivityIDReference, bytes] details: Sequence[Any] rpc_metadata: Mapping[str, Union[str, bytes]] rpc_timeout: Optional[timedelta] @@ -5555,7 +6155,7 @@ class HeartbeatAsyncActivityInput: class CompleteAsyncActivityInput: """Input for :py:meth:`OutboundInterceptor.complete_async_activity`.""" - id_or_token: Union[AsyncActivityIDReference, bytes] + id_or_token: Union[ActivityIDReference, bytes] result: Optional[Any] rpc_metadata: Mapping[str, Union[str, bytes]] rpc_timeout: Optional[timedelta] @@ -5566,7 +6166,7 @@ class CompleteAsyncActivityInput: class FailAsyncActivityInput: """Input for :py:meth:`OutboundInterceptor.fail_async_activity`.""" - id_or_token: Union[AsyncActivityIDReference, bytes] + id_or_token: Union[ActivityIDReference, bytes] error: Exception last_heartbeat_details: Sequence[Any] rpc_metadata: Mapping[str, Union[str, bytes]] @@ -5578,7 +6178,7 @@ class FailAsyncActivityInput: class ReportCancellationAsyncActivityInput: """Input for :py:meth:`OutboundInterceptor.report_cancellation_async_activity`.""" - id_or_token: Union[AsyncActivityIDReference, bytes] + id_or_token: Union[ActivityIDReference, bytes] details: Sequence[Any] rpc_metadata: Mapping[str, Union[str, bytes]] rpc_timeout: Optional[timedelta] @@ -6493,7 +7093,7 @@ async def heartbeat_async_activity( if isinstance(input.id_or_token, AsyncActivityIDReference): resp_by_id = await self._client.workflow_service.record_activity_task_heartbeat_by_id( temporalio.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest( - workflow_id=input.id_or_token.workflow_id, + workflow_id=input.id_or_token.workflow_id or "", run_id=input.id_or_token.run_id or "", activity_id=input.id_or_token.activity_id, namespace=self._client.namespace, @@ -6548,7 +7148,7 @@ async def complete_async_activity(self, input: CompleteAsyncActivityInput) -> No if isinstance(input.id_or_token, AsyncActivityIDReference): await self._client.workflow_service.respond_activity_task_completed_by_id( temporalio.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest( - workflow_id=input.id_or_token.workflow_id, + workflow_id=input.id_or_token.workflow_id or "", run_id=input.id_or_token.run_id or "", activity_id=input.id_or_token.activity_id, namespace=self._client.namespace, @@ -6585,7 +7185,7 @@ async def fail_async_activity(self, input: FailAsyncActivityInput) -> None: if isinstance(input.id_or_token, AsyncActivityIDReference): await self._client.workflow_service.respond_activity_task_failed_by_id( temporalio.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest( - workflow_id=input.id_or_token.workflow_id, + workflow_id=input.id_or_token.workflow_id or "", run_id=input.id_or_token.run_id or "", activity_id=input.id_or_token.activity_id, namespace=self._client.namespace, @@ -6623,7 +7223,7 @@ async def report_cancellation_async_activity( if isinstance(input.id_or_token, AsyncActivityIDReference): await self._client.workflow_service.respond_activity_task_canceled_by_id( temporalio.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest( - workflow_id=input.id_or_token.workflow_id, + workflow_id=input.id_or_token.workflow_id or "", run_id=input.id_or_token.run_id or "", activity_id=input.id_or_token.activity_id, namespace=self._client.namespace, diff --git a/temporalio/common.py b/temporalio/common.py index 844d73f87..9664e9857 100644 --- a/temporalio/common.py +++ b/temporalio/common.py @@ -154,6 +154,47 @@ class WorkflowIDConflictPolicy(IntEnum): ) +class ActivityExecutionStatus(IntEnum): + """Status of a standalone activity execution. + + See :py:class:`temporalio.api.enums.v1.ActivityExecutionStatus`. + """ + + UNSPECIFIED = 0 # ACTIVITY_EXECUTION_STATUS_UNSPECIFIED + RUNNING = 1 # ACTIVITY_EXECUTION_STATUS_RUNNING + COMPLETED = 2 # ACTIVITY_EXECUTION_STATUS_COMPLETED + FAILED = 3 # ACTIVITY_EXECUTION_STATUS_FAILED + CANCELED = 4 # ACTIVITY_EXECUTION_STATUS_CANCELED + TERMINATED = 5 # ACTIVITY_EXECUTION_STATUS_TERMINATED + TIMED_OUT = 6 # ACTIVITY_EXECUTION_STATUS_TIMED_OUT + + +class PendingActivityState(IntEnum): + """State of a pending activity. + + See :py:class:`temporalio.api.enums.v1.PendingActivityState`. + """ + + UNSPECIFIED = int( + temporalio.api.enums.v1.PendingActivityState.PENDING_ACTIVITY_STATE_UNSPECIFIED + ) + SCHEDULED = int( + temporalio.api.enums.v1.PendingActivityState.PENDING_ACTIVITY_STATE_SCHEDULED + ) + STARTED = int( + temporalio.api.enums.v1.PendingActivityState.PENDING_ACTIVITY_STATE_STARTED + ) + CANCEL_REQUESTED = int( + temporalio.api.enums.v1.PendingActivityState.PENDING_ACTIVITY_STATE_CANCEL_REQUESTED + ) + PAUSED = int( + temporalio.api.enums.v1.PendingActivityState.PENDING_ACTIVITY_STATE_PAUSED + ) + PAUSE_REQUESTED = int( + temporalio.api.enums.v1.PendingActivityState.PENDING_ACTIVITY_STATE_PAUSE_REQUESTED + ) + + class QueryRejectCondition(IntEnum): """Whether a query should be rejected in certain conditions. diff --git a/temporalio/exceptions.py b/temporalio/exceptions.py index 74afb7ea7..98031cbf7 100644 --- a/temporalio/exceptions.py +++ b/temporalio/exceptions.py @@ -247,6 +247,10 @@ class RetryState(IntEnum): ) +# TODO: This error class has required history event fields. I propose we retain it as +# workflow-specific and introduce client.ActivityFailureError for an error in a standalone activity. +# We could deprecate this name and introduce WorkflowActivityError as a preferred-going-forwards +# alias. class ActivityError(FailureError): """Error raised on activity failure.""" @@ -362,6 +366,8 @@ def retry_state(self) -> Optional[RetryState]: return self._retry_state +# TODO: This error class has required history event fields. Would we retain it as workflow-specific +# and introduce client.NexusOperationFailureError? See related note on ActivityError above. class NexusOperationError(FailureError): """Error raised on Nexus operation failure inside a Workflow.""" diff --git a/tests/worker/test_workflow.py b/tests/worker/test_workflow.py index a987d1b34..876876f03 100644 --- a/tests/worker/test_workflow.py +++ b/tests/worker/test_workflow.py @@ -8017,7 +8017,7 @@ async def test_external_activity_cancellation_details( # Assert not paused assert not activity_info.paused - external_activity_handle = client.get_async_activity_handle( + external_activity_handle = client.get_workflow_activity_handle( workflow_id=wf_desc.id, run_id=wf_desc.run_id, activity_id=test_activity_id ) From c725d71da8d17e07e7acd0402a3b5107e1487c4c Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Tue, 7 Oct 2025 10:16:00 -0400 Subject: [PATCH 02/25] Partial revert --- temporalio/client.py | 394 +++++++++++++++++----------------- tests/worker/test_workflow.py | 4 +- 2 files changed, 195 insertions(+), 203 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index 18feaae8d..17946e75f 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -10,7 +10,6 @@ import inspect import json import re -import typing import uuid import warnings from abc import ABC, abstractmethod @@ -1393,109 +1392,88 @@ async def count_activities( # Issues a workflowservice CountActivityExecutions call raise NotImplementedError - @typing.overload def get_activity_handle( self, *, activity_id: str, run_id: Optional[str] = None, ) -> ActivityHandle[Any]: - raise NotImplementedError + """Get a handle to an existing activity, as the caller of that activity. - @typing.overload - def get_activity_handle( - self, - *, - activity_id: str, - workflow_id: str, - run_id: Optional[str], - ) -> WorkflowActivityHandle: - raise NotImplementedError - - @typing.overload - def get_activity_handle( - self, - *, - task_token: bytes, - ) -> WorkflowActivityHandle: - raise NotImplementedError - - def get_activity_handle( - self, - *, - activity_id: Optional[str] = None, - workflow_id: Optional[str] = None, - run_id: Optional[str] = None, - task_token: Optional[bytes] = None, - ) -> Union[ActivityHandle[Any], WorkflowActivityHandle]: - """Get a handle to an existing activity. + To get a handle to an activity execution that you control for manual completion and + heartbeating, see :py:meth:`Client.get_async_activity_handle`. Args: activity_id: The activity ID. - workflow_id: The workflow ID if the activity was started from a workflow. - run_id: The run ID. If not provided, targets the latest run. - task_token: Optional task token for the activity if the activity was - started from a workflow. Cannot be set if any of the id parameters - are set. + run_id: The run ID. If not provided, targets the + latest run. Returns: A handle to the activity. """ raise NotImplementedError - # Deprecated: get_activity_handle has an equivalent override @overload - def get_workflow_activity_handle( + def get_async_activity_handle( + self, *, activity_id: str, run_id: Optional[str] + ) -> AsyncActivityHandle: + pass + + @overload + def get_async_activity_handle( self, *, workflow_id: str, run_id: Optional[str], activity_id: str - ) -> WorkflowActivityHandle: + ) -> AsyncActivityHandle: pass - # Deprecated: get_activity_handle has an equivalent override @overload - def get_workflow_activity_handle( - self, *, task_token: bytes - ) -> WorkflowActivityHandle: + def get_async_activity_handle(self, *, task_token: bytes) -> AsyncActivityHandle: pass - def get_workflow_activity_handle( + def get_async_activity_handle( self, *, workflow_id: Optional[str] = None, run_id: Optional[str] = None, activity_id: Optional[str] = None, task_token: Optional[bytes] = None, - ) -> WorkflowActivityHandle: - """Get a handle to an activity started by a workflow. + ) -> AsyncActivityHandle: + """Get a handle to an activity execution that you control, for manual completion and heartbeating. - .. warning:: - DEPRECATED: This method is deprecated. - Use :py:meth:`Client.get_activity_handle` instead. + To get a handle to a standalone activity execution as the caller of that activity, see + :py:meth:`Client.get_activity_handle`. + + This function may be used to get a handle to a standalone activity started by a client, or + an activity started by a workflow. + + To get a handle to an activity started by a workflow, use one of the following two calls: + - Supply ``workflow_id``, ``run_id``, and ``activity_id`` + - Supply the activity ``task_token`` alone + + To get a handle to a standalone activity started by a client, supply ``activity_id`` and + ``run_id`` - Either the workflow_id, run_id, and activity_id can be provided, or a - singular task_token can be provided. Args: - workflow_id: Workflow ID for the activity. Cannot be set if - task_token is set. - run_id: Run ID for the activity. Cannot be set if task_token is set. - activity_id: ID for the activity. Cannot be set if task_token is - set. - task_token: Task token for the activity. Cannot be set if any of the - id parameters are set. + workflow_id: Workflow ID for the activity. + run_id: Run ID for the activity. Cannot be + set if task_token is set. + activity_id: ID for the activity. + activity_id: ID for the activity. + task_token: Task token for the activity. Returns: - A handle that can be used for completion or heartbeat. + A handle that can be used for completion or heartbeating. """ if task_token is not None: if workflow_id is not None or run_id is not None or activity_id is not None: raise ValueError("Task token cannot be present with other IDs") - return WorkflowActivityHandle(self, task_token) + return AsyncActivityHandle(self, task_token) elif workflow_id is not None: if activity_id is None: raise ValueError( "Workflow ID, run ID, and activity ID must all be given together" ) - return WorkflowActivityHandle( + return AsyncActivityHandle( self, AsyncActivityIDReference( workflow_id=workflow_id, run_id=run_id, activity_id=activity_id @@ -1503,9 +1481,6 @@ def get_workflow_activity_handle( ) raise ValueError("Task token or workflow/run/activity ID must be present") - # Deprecated alias - get_async_activity_handle = get_workflow_activity_handle - async def create_schedule( self, id: str, @@ -1791,7 +1766,7 @@ def _data_converter(self) -> temporalio.converter.DataConverter: @property def id(self) -> str: - """ID for the workflow.""" + """ID of the workflow.""" return self._id @property @@ -3080,12 +3055,19 @@ class ActivityIDReference: AsyncActivityIDReference = ActivityIDReference -class _BaseActivityHandle(WithSerializationContext): - """Handle representing an activity.""" +class AsyncActivityHandle(WithSerializationContext): + """Handle representing an external activity for completion and heartbeat.""" - _client: Client - _id_or_token: Union[ActivityIDReference, bytes] - _data_converter_override: Optional[DataConverter] + def __init__( + self, + client: Client, + id_or_token: Union[AsyncActivityIDReference, bytes], + data_converter_override: Optional[DataConverter] = None, + ) -> None: + """Create an async activity handle.""" + self._client = client + self._id_or_token = id_or_token + self._data_converter_override = data_converter_override async def heartbeat( self, @@ -3188,126 +3170,6 @@ async def report_cancellation( ), ) - async def pause( - self, - *, - reason: Optional[str] = None, - rpc_metadata: Mapping[str, Union[str, bytes]] = {}, - rpc_timeout: Optional[timedelta] = None, - ) -> None: - """Pause the activity. - - Args: - reason: Reason for pausing the activity. - rpc_metadata: Headers used on the RPC call. Keys here override - client-level RPC metadata keys. - rpc_timeout: Optional RPC deadline to set for the RPC call. - """ - if not isinstance(self._id_or_token, ActivityIDReference): - raise ValueError("Cannot pause activity with task token") - - await self._client.workflow_service.pause_activity( - temporalio.api.workflowservice.v1.PauseActivityRequest( - namespace=self._client.namespace, - execution=temporalio.api.common.v1.WorkflowExecution( - workflow_id=self._id_or_token.workflow_id or "", - run_id=self._id_or_token.run_id or "", - ), - identity=self._client.identity, - id=self._id_or_token.activity_id, - reason=reason or "", - ), - retry=True, - metadata=rpc_metadata, - timeout=rpc_timeout, - ) - - async def unpause( - self, - *, - reset_attempts: bool = False, - rpc_metadata: Mapping[str, Union[str, bytes]] = {}, - rpc_timeout: Optional[timedelta] = None, - ) -> None: - """Unpause the activity. - - Args: - reset_attempts: Whether to reset the number of attempts. - rpc_metadata: Headers used on the RPC call. Keys here override - client-level RPC metadata keys. - rpc_timeout: Optional RPC deadline to set for the RPC call. - """ - if not isinstance(self._id_or_token, ActivityIDReference): - raise ValueError("Cannot unpause activity with task token") - - await self._client.workflow_service.unpause_activity( - temporalio.api.workflowservice.v1.UnpauseActivityRequest( - namespace=self._client.namespace, - execution=temporalio.api.common.v1.WorkflowExecution( - workflow_id=self._id_or_token.workflow_id or "", - run_id=self._id_or_token.run_id or "", - ), - identity=self._client.identity, - id=self._id_or_token.activity_id, - reset_attempts=reset_attempts, - ), - retry=True, - metadata=rpc_metadata, - timeout=rpc_timeout, - ) - - async def reset( - self, - *, - reset_heartbeat: bool = False, - keep_paused: bool = False, - rpc_metadata: Mapping[str, Union[str, bytes]] = {}, - rpc_timeout: Optional[timedelta] = None, - ) -> None: - """Reset the activity. - - Args: - reset_heartbeat: Whether to reset heartbeat details. - keep_paused: If activity is paused, whether to keep it paused after reset. - rpc_metadata: Headers used on the RPC call. Keys here override - client-level RPC metadata keys. - rpc_timeout: Optional RPC deadline to set for the RPC call. - """ - if not isinstance(self._id_or_token, ActivityIDReference): - raise ValueError("Cannot reset activity with task token") - - await self._client.workflow_service.reset_activity( - temporalio.api.workflowservice.v1.ResetActivityRequest( - namespace=self._client.namespace, - execution=temporalio.api.common.v1.WorkflowExecution( - workflow_id=self._id_or_token.workflow_id or "", - run_id=self._id_or_token.run_id or "", - ), - identity=self._client.identity, - id=self._id_or_token.activity_id, - reset_heartbeat=reset_heartbeat, - keep_paused=keep_paused, - ), - retry=True, - metadata=rpc_metadata, - timeout=rpc_timeout, - ) - - -class WorkflowActivityHandle(_BaseActivityHandle): - """Handle representing an activity started by a workflow.""" - - def __init__( - self, - client: Client, - id_or_token: Union[ActivityIDReference, bytes], - data_converter_override: Optional[DataConverter] = None, - ) -> None: - """Create an async activity handle.""" - self._client = client - self._id_or_token = id_or_token - self._data_converter_override = data_converter_override - def with_context(self, context: SerializationContext) -> Self: """Create a new AsyncActivityHandle with a different serialization context. @@ -3332,29 +3194,38 @@ def with_context(self, context: SerializationContext) -> Self: ) -# Deprecated alias -AsyncActivityHandle = WorkflowActivityHandle - - # TODO: in the future when messages can be sent to activities, we will want the activity handle to # be generic in the activity type in addition to the return type (as WorkflowHandle), to support # static type inference for signal/query/update. -class ActivityHandle(Generic[ReturnType], _BaseActivityHandle): +class ActivityHandle(Generic[ReturnType]): """Handle representing a standalone activity execution.""" def __init__( self, client: Client, - activity_id_reference: ActivityIDReference, + id: str, + *, + run_id: str, data_converter_override: Optional[DataConverter] = None, ) -> None: - """Create an async activity handle.""" + """Create activity handle.""" self._client = client - self._id_or_token = activity_id_reference + self._id = id + self._run_id = run_id self._data_converter_override = data_converter_override + @property + def id(self) -> str: + """ID of the activity.""" + return self._id + + @property + def run_id(self) -> str: + """Run ID of the activity.""" + return self._run_id + def with_context(self, context: SerializationContext) -> Self: - """Create a new AsyncActivityHandle with a different serialization context. + """Create a new ActivityHandle with a different serialization context. Payloads received by the activity will be decoded and deserialized using a data converter with :py:class:`ActivitySerializationContext` set as context. If you are using a custom data @@ -3367,13 +3238,14 @@ def with_context(self, context: SerializationContext) -> Self: cls = type(self) if cls.__init__ is not ActivityHandle.__init__: raise TypeError( - "If you have subclassed AsyncActivityHandle and overridden the __init__ method " + "If you have subclassed ActivityHandle and overridden the __init__ method " "then you must override with_context to return an instance of your class." ) return cls( self._client, - cast(ActivityIDReference, self._id_or_token), - data_converter, + id=self._id, + run_id=self._run_id, + data_converter_override=data_converter, ) async def result( @@ -3462,6 +3334,126 @@ async def describe( # TODO: # update_options + async def pause( + self, + *, + reason: Optional[str] = None, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Pause the activity. + + Args: + reason: Reason for pausing the activity. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + id_ref = ActivityIDReference( + activity_id=self._id, + run_id=self._run_id, + workflow_id=None, + ) + if not isinstance(id_ref, ActivityIDReference): + raise ValueError("Cannot pause activity with task token") + + await self._client.workflow_service.pause_activity( + temporalio.api.workflowservice.v1.PauseActivityRequest( + namespace=self._client.namespace, + execution=temporalio.api.common.v1.WorkflowExecution( + workflow_id=id_ref.workflow_id or "", + run_id=id_ref.run_id or "", + ), + identity=self._client.identity, + id=id_ref.activity_id, + reason=reason or "", + ), + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + + async def unpause( + self, + *, + reset_attempts: bool = False, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Unpause the activity. + + Args: + reset_attempts: Whether to reset the number of attempts. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + id_ref = ActivityIDReference( + activity_id=self._id, + run_id=self._run_id, + workflow_id=None, + ) + if not isinstance(id_ref, ActivityIDReference): + raise ValueError("Cannot unpause activity with task token") + + await self._client.workflow_service.unpause_activity( + temporalio.api.workflowservice.v1.UnpauseActivityRequest( + namespace=self._client.namespace, + execution=temporalio.api.common.v1.WorkflowExecution( + workflow_id=id_ref.workflow_id or "", + run_id=id_ref.run_id or "", + ), + identity=self._client.identity, + id=id_ref.activity_id, + reset_attempts=reset_attempts, + ), + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + + async def reset( + self, + *, + reset_heartbeat: bool = False, + keep_paused: bool = False, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Reset the activity. + + Args: + reset_heartbeat: Whether to reset heartbeat details. + keep_paused: If activity is paused, whether to keep it paused after reset. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for the RPC call. + """ + id_ref = ActivityIDReference( + activity_id=self._id, + run_id=self._run_id, + workflow_id=None, + ) + if not isinstance(id_ref, ActivityIDReference): + raise ValueError("Cannot reset activity with task token") + + await self._client.workflow_service.reset_activity( + temporalio.api.workflowservice.v1.ResetActivityRequest( + namespace=self._client.namespace, + execution=temporalio.api.common.v1.WorkflowExecution( + workflow_id=id_ref.workflow_id or "", + run_id=id_ref.run_id or "", + ), + identity=self._client.identity, + id=id_ref.activity_id, + reset_heartbeat=reset_heartbeat, + keep_paused=keep_paused, + ), + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + @dataclass class WorkflowExecution: diff --git a/tests/worker/test_workflow.py b/tests/worker/test_workflow.py index 876876f03..fd476ca63 100644 --- a/tests/worker/test_workflow.py +++ b/tests/worker/test_workflow.py @@ -8017,7 +8017,7 @@ async def test_external_activity_cancellation_details( # Assert not paused assert not activity_info.paused - external_activity_handle = client.get_workflow_activity_handle( + external_activity_handle = client.get_async_activity_handle( workflow_id=wf_desc.id, run_id=wf_desc.run_id, activity_id=test_activity_id ) @@ -8473,7 +8473,7 @@ async def test_search_attribute_codec(client: Client, env_type: str): result = await client.execute_workflow( SearchAttributeCodecParentWorkflow.run, "Temporal", - id=f"encryption-workflow-id", + id="encryption-workflow-id", task_queue=worker.task_queue, search_attributes=TypedSearchAttributes( [ From b5d2d2f6bb7ce76b63484b2089b710317cb0d534 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 03:33:31 -0400 Subject: [PATCH 03/25] Update core: https://github.com/temporalio/api/pull/640 --- temporalio/bridge/sdk-core | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/temporalio/bridge/sdk-core b/temporalio/bridge/sdk-core index 682d441dd..ca1de1067 160000 --- a/temporalio/bridge/sdk-core +++ b/temporalio/bridge/sdk-core @@ -1 +1 @@ -Subproject commit 682d441dd3b830e1477af3edb7c2330b403c4c33 +Subproject commit ca1de1067cfdae37a3b3bb2a15cc96b147e6d437 From 896fc8773d226b35ce3832ae5263690b8fec73f9 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 04:06:52 -0400 Subject: [PATCH 04/25] Test standalone activity --- tests/test_activity.py | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 tests/test_activity.py diff --git a/tests/test_activity.py b/tests/test_activity.py new file mode 100644 index 000000000..00474c3bc --- /dev/null +++ b/tests/test_activity.py @@ -0,0 +1,33 @@ +import uuid +from datetime import timedelta + +from temporalio import activity +from temporalio.client import Client +from temporalio.common import ActivityExecutionStatus + + +@activity.defn +async def increment(input: int) -> int: + return input + 1 + + +async def test_describe_activity(client: Client): + activity_id = str("test_start_and_describe_activity_id") + task_queue = str(uuid.uuid4()) + + activity_handle = await client.start_activity( + increment, + args=(1,), + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + desc = await activity_handle.describe() + assert desc.activity_id == activity_id + assert desc.run_id == activity_handle.run_id + assert desc.activity_type == "increment" + assert desc.task_queue == task_queue + assert desc.status in [ + ActivityExecutionStatus.RUNNING, + ActivityExecutionStatus.COMPLETED, + ] From 32ff6922d328395e3e3b430636c1f5ac27e1cf47 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 04:20:44 -0400 Subject: [PATCH 05/25] poe gen-protos-docker --- temporalio/api/activity/v1/__init__.py | 10 +- temporalio/api/activity/v1/message_pb2.py | 94 +- temporalio/api/activity/v1/message_pb2.pyi | 524 ++++++- temporalio/api/common/v1/message_pb2.py | 41 +- temporalio/api/common/v1/message_pb2.pyi | 44 +- temporalio/api/deployment/v1/message_pb2.py | 24 +- temporalio/api/deployment/v1/message_pb2.pyi | 10 + temporalio/api/enums/v1/__init__.py | 5 + temporalio/api/enums/v1/activity_pb2.py | 38 + temporalio/api/enums/v1/activity_pb2.pyi | 93 ++ temporalio/api/enums/v1/id_pb2.py | 43 + temporalio/api/enums/v1/id_pb2.pyi | 96 ++ temporalio/api/namespace/v1/message_pb2.py | 36 +- temporalio/api/namespace/v1/message_pb2.pyi | 12 + temporalio/api/workflowservice/v1/__init__.py | 56 + .../v1/request_response_pb2.py | 1308 +++++++++++----- .../v1/request_response_pb2.pyi | 1369 ++++++++++++++++- .../api/workflowservice/v1/service_pb2.py | 56 +- .../workflowservice/v1/service_pb2_grpc.py | 718 ++++++++- .../workflowservice/v1/service_pb2_grpc.pyi | 350 ++++- temporalio/bridge/services_generated.py | 252 +++ temporalio/bridge/src/client_rpc_generated.rs | 107 ++ 22 files changed, 4793 insertions(+), 493 deletions(-) create mode 100644 temporalio/api/enums/v1/activity_pb2.py create mode 100644 temporalio/api/enums/v1/activity_pb2.pyi create mode 100644 temporalio/api/enums/v1/id_pb2.py create mode 100644 temporalio/api/enums/v1/id_pb2.pyi diff --git a/temporalio/api/activity/v1/__init__.py b/temporalio/api/activity/v1/__init__.py index a6e54842f..41de8ac59 100644 --- a/temporalio/api/activity/v1/__init__.py +++ b/temporalio/api/activity/v1/__init__.py @@ -1,5 +1,13 @@ -from .message_pb2 import ActivityOptions +from .message_pb2 import ( + ActivityExecutionInfo, + ActivityListInfo, + ActivityOptions, + OnConflictOptions, +) __all__ = [ + "ActivityExecutionInfo", + "ActivityListInfo", "ActivityOptions", + "OnConflictOptions", ] diff --git a/temporalio/api/activity/v1/message_pb2.py b/temporalio/api/activity/v1/message_pb2.py index 58f1d95cb..de27db684 100644 --- a/temporalio/api/activity/v1/message_pb2.py +++ b/temporalio/api/activity/v1/message_pb2.py @@ -15,20 +15,56 @@ from google.protobuf import duration_pb2 as google_dot_protobuf_dot_duration__pb2 +from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 from temporalio.api.common.v1 import ( message_pb2 as temporal_dot_api_dot_common_dot_v1_dot_message__pb2, ) +from temporalio.api.deployment.v1 import ( + message_pb2 as temporal_dot_api_dot_deployment_dot_v1_dot_message__pb2, +) +from temporalio.api.enums.v1 import ( + activity_pb2 as temporal_dot_api_dot_enums_dot_v1_dot_activity__pb2, +) +from temporalio.api.enums.v1 import ( + workflow_pb2 as temporal_dot_api_dot_enums_dot_v1_dot_workflow__pb2, +) +from temporalio.api.failure.v1 import ( + message_pb2 as temporal_dot_api_dot_failure_dot_v1_dot_message__pb2, +) +from temporalio.api.sdk.v1 import ( + user_metadata_pb2 as temporal_dot_api_dot_sdk_dot_v1_dot_user__metadata__pb2, +) from temporalio.api.taskqueue.v1 import ( message_pb2 as temporal_dot_api_dot_taskqueue_dot_v1_dot_message__pb2, ) DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b"\n&temporal/api/activity/v1/message.proto\x12\x18temporal.api.activity.v1\x1a$temporal/api/common/v1/message.proto\x1a'temporal/api/taskqueue/v1/message.proto\x1a\x1egoogle/protobuf/duration.proto\"\xf3\x02\n\x0f\x41\x63tivityOptions\x12\x38\n\ntask_queue\x18\x01 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12<\n\x19schedule_to_close_timeout\x18\x02 \x01(\x0b\x32\x19.google.protobuf.Duration\x12<\n\x19schedule_to_start_timeout\x18\x03 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x16start_to_close_timeout\x18\x04 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x34\n\x11heartbeat_timeout\x18\x05 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x0cretry_policy\x18\x06 \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicyB\x93\x01\n\x1bio.temporal.api.activity.v1B\x0cMessageProtoP\x01Z'go.temporal.io/api/activity/v1;activity\xaa\x02\x1aTemporalio.Api.Activity.V1\xea\x02\x1dTemporalio::Api::Activity::V1b\x06proto3" + b'\n&temporal/api/activity/v1/message.proto\x12\x18temporal.api.activity.v1\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a$temporal/api/common/v1/message.proto\x1a(temporal/api/deployment/v1/message.proto\x1a$temporal/api/enums/v1/activity.proto\x1a$temporal/api/enums/v1/workflow.proto\x1a%temporal/api/failure/v1/message.proto\x1a\'temporal/api/taskqueue/v1/message.proto\x1a\'temporal/api/sdk/v1/user_metadata.proto"i\n\x11OnConflictOptions\x12\x19\n\x11\x61ttach_request_id\x18\x01 \x01(\x08\x12#\n\x1b\x61ttach_completion_callbacks\x18\x02 \x01(\x08\x12\x14\n\x0c\x61ttach_links\x18\x03 \x01(\x08"\xf3\x02\n\x0f\x41\x63tivityOptions\x12\x38\n\ntask_queue\x18\x01 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12<\n\x19schedule_to_close_timeout\x18\x02 \x01(\x0b\x32\x19.google.protobuf.Duration\x12<\n\x19schedule_to_start_timeout\x18\x03 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x16start_to_close_timeout\x18\x04 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x34\n\x11heartbeat_timeout\x18\x05 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x0cretry_policy\x18\x06 \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy"\xdd\r\n\x15\x41\x63tivityExecutionInfo\x12\x13\n\x0b\x61\x63tivity_id\x18\x01 \x01(\t\x12\x0e\n\x06run_id\x18\x02 \x01(\t\x12;\n\ractivity_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.ActivityType\x12>\n\x06status\x18\x04 \x01(\x0e\x32..temporal.api.enums.v1.ActivityExecutionStatus\x12>\n\trun_state\x18\x05 \x01(\x0e\x32+.temporal.api.enums.v1.PendingActivityState\x12;\n\x11heartbeat_details\x18\x06 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x37\n\x13last_heartbeat_time\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x35\n\x11last_started_time\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07\x61ttempt\x18\t \x01(\x05\x12\x18\n\x10maximum_attempts\x18\n \x01(\x05\x12\x32\n\x0escheduled_time\x18\x0b \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x33\n\x0f\x65xpiration_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x0clast_failure\x18\r \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x1c\n\x14last_worker_identity\x18\x0e \x01(\t\x12\x39\n\x16\x63urrent_retry_interval\x18\x0f \x01(\x0b\x32\x19.google.protobuf.Duration\x12>\n\x1alast_attempt_complete_time\x18\x10 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x1anext_attempt_schedule_time\x18\x11 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12T\n\x17last_deployment_version\x18\x12 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x32\n\x08priority\x18\x13 \x01(\x0b\x32 .temporal.api.common.v1.Priority\x12\x43\n\x10\x61\x63tivity_options\x18\x14 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions\x12/\n\x05input\x18\x15 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x1e\n\x16state_transition_count\x18\x16 \x01(\x03\x12\x43\n\x11search_attributes\x18\x17 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12.\n\x06header\x18\x18 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12!\n\x19\x65\x61ger_execution_requested\x18\x19 \x01(\x08\x12>\n\x14\x63ompletion_callbacks\x18\x1a \x03(\x0b\x32 .temporal.api.common.v1.Callback\x12\x38\n\ruser_metadata\x18\x1b \x01(\x0b\x32!.temporal.api.sdk.v1.UserMetadata\x12+\n\x05links\x18\x1c \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x12\x17\n\x0f\x63\x61nceled_reason\x18\x1d \x01(\t\x12M\n\npause_info\x18\x1e \x01(\x0b\x32\x39.temporal.api.activity.v1.ActivityExecutionInfo.PauseInfo\x1a\xc8\x01\n\tPauseInfo\x12.\n\npause_time\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x06manual\x18\x02 \x01(\x0b\x32@.temporal.api.activity.v1.ActivityExecutionInfo.PauseInfo.ManualH\x00\x1a*\n\x06Manual\x12\x10\n\x08identity\x18\x01 \x01(\t\x12\x0e\n\x06reason\x18\x02 \x01(\tB\x0b\n\tpaused_by"\xf7\x03\n\x10\x41\x63tivityListInfo\x12\x13\n\x0b\x61\x63tivity_id\x18\x01 \x01(\t\x12\x0e\n\x06run_id\x18\x02 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x03 \x01(\t\x12;\n\ractivity_type\x18\x04 \x01(\x0b\x32$.temporal.api.common.v1.ActivityType\x12\x32\n\x0escheduled_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nclose_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12>\n\x06status\x18\x07 \x01(\x0e\x32..temporal.api.enums.v1.ActivityExecutionStatus\x12\x43\n\x11search_attributes\x18\x08 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12\x12\n\ntask_queue\x18\t \x01(\t\x12\x1e\n\x16state_transition_count\x18\n \x01(\x03\x12\x18\n\x10state_size_bytes\x18\x0b \x01(\x03\x12\x35\n\x12\x65xecution_duration\x18\x0c \x01(\x0b\x32\x19.google.protobuf.DurationB\x93\x01\n\x1bio.temporal.api.activity.v1B\x0cMessageProtoP\x01Z\'go.temporal.io/api/activity/v1;activity\xaa\x02\x1aTemporalio.Api.Activity.V1\xea\x02\x1dTemporalio::Api::Activity::V1b\x06proto3' ) +_ONCONFLICTOPTIONS = DESCRIPTOR.message_types_by_name["OnConflictOptions"] _ACTIVITYOPTIONS = DESCRIPTOR.message_types_by_name["ActivityOptions"] +_ACTIVITYEXECUTIONINFO = DESCRIPTOR.message_types_by_name["ActivityExecutionInfo"] +_ACTIVITYEXECUTIONINFO_PAUSEINFO = _ACTIVITYEXECUTIONINFO.nested_types_by_name[ + "PauseInfo" +] +_ACTIVITYEXECUTIONINFO_PAUSEINFO_MANUAL = ( + _ACTIVITYEXECUTIONINFO_PAUSEINFO.nested_types_by_name["Manual"] +) +_ACTIVITYLISTINFO = DESCRIPTOR.message_types_by_name["ActivityListInfo"] +OnConflictOptions = _reflection.GeneratedProtocolMessageType( + "OnConflictOptions", + (_message.Message,), + { + "DESCRIPTOR": _ONCONFLICTOPTIONS, + "__module__": "temporal.api.activity.v1.message_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.activity.v1.OnConflictOptions) + }, +) +_sym_db.RegisterMessage(OnConflictOptions) + ActivityOptions = _reflection.GeneratedProtocolMessageType( "ActivityOptions", (_message.Message,), @@ -40,9 +76,61 @@ ) _sym_db.RegisterMessage(ActivityOptions) +ActivityExecutionInfo = _reflection.GeneratedProtocolMessageType( + "ActivityExecutionInfo", + (_message.Message,), + { + "PauseInfo": _reflection.GeneratedProtocolMessageType( + "PauseInfo", + (_message.Message,), + { + "Manual": _reflection.GeneratedProtocolMessageType( + "Manual", + (_message.Message,), + { + "DESCRIPTOR": _ACTIVITYEXECUTIONINFO_PAUSEINFO_MANUAL, + "__module__": "temporal.api.activity.v1.message_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.activity.v1.ActivityExecutionInfo.PauseInfo.Manual) + }, + ), + "DESCRIPTOR": _ACTIVITYEXECUTIONINFO_PAUSEINFO, + "__module__": "temporal.api.activity.v1.message_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.activity.v1.ActivityExecutionInfo.PauseInfo) + }, + ), + "DESCRIPTOR": _ACTIVITYEXECUTIONINFO, + "__module__": "temporal.api.activity.v1.message_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.activity.v1.ActivityExecutionInfo) + }, +) +_sym_db.RegisterMessage(ActivityExecutionInfo) +_sym_db.RegisterMessage(ActivityExecutionInfo.PauseInfo) +_sym_db.RegisterMessage(ActivityExecutionInfo.PauseInfo.Manual) + +ActivityListInfo = _reflection.GeneratedProtocolMessageType( + "ActivityListInfo", + (_message.Message,), + { + "DESCRIPTOR": _ACTIVITYLISTINFO, + "__module__": "temporal.api.activity.v1.message_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.activity.v1.ActivityListInfo) + }, +) +_sym_db.RegisterMessage(ActivityListInfo) + if _descriptor._USE_C_DESCRIPTORS == False: DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"\n\033io.temporal.api.activity.v1B\014MessageProtoP\001Z'go.temporal.io/api/activity/v1;activity\252\002\032Temporalio.Api.Activity.V1\352\002\035Temporalio::Api::Activity::V1" - _ACTIVITYOPTIONS._serialized_start = 180 - _ACTIVITYOPTIONS._serialized_end = 551 + _ONCONFLICTOPTIONS._serialized_start = 410 + _ONCONFLICTOPTIONS._serialized_end = 515 + _ACTIVITYOPTIONS._serialized_start = 518 + _ACTIVITYOPTIONS._serialized_end = 889 + _ACTIVITYEXECUTIONINFO._serialized_start = 892 + _ACTIVITYEXECUTIONINFO._serialized_end = 2649 + _ACTIVITYEXECUTIONINFO_PAUSEINFO._serialized_start = 2449 + _ACTIVITYEXECUTIONINFO_PAUSEINFO._serialized_end = 2649 + _ACTIVITYEXECUTIONINFO_PAUSEINFO_MANUAL._serialized_start = 2594 + _ACTIVITYEXECUTIONINFO_PAUSEINFO_MANUAL._serialized_end = 2636 + _ACTIVITYLISTINFO._serialized_start = 2652 + _ACTIVITYLISTINFO._serialized_end = 3155 # @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/activity/v1/message_pb2.pyi b/temporalio/api/activity/v1/message_pb2.pyi index 373c5f18f..ff006c2c4 100644 --- a/temporalio/api/activity/v1/message_pb2.pyi +++ b/temporalio/api/activity/v1/message_pb2.pyi @@ -4,13 +4,21 @@ isort:skip_file """ import builtins +import collections.abc import sys import google.protobuf.descriptor import google.protobuf.duration_pb2 +import google.protobuf.internal.containers import google.protobuf.message +import google.protobuf.timestamp_pb2 import temporalio.api.common.v1.message_pb2 +import temporalio.api.deployment.v1.message_pb2 +import temporalio.api.enums.v1.activity_pb2 +import temporalio.api.enums.v1.workflow_pb2 +import temporalio.api.failure.v1.message_pb2 +import temporalio.api.sdk.v1.user_metadata_pb2 import temporalio.api.taskqueue.v1.message_pb2 if sys.version_info >= (3, 8): @@ -20,6 +28,43 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +class OnConflictOptions(google.protobuf.message.Message): + """When StartActivityExecution uses the ID_CONFLICT_POLICY_USE_EXISTING and there is already an existing running + activity, OnConflictOptions defines actions to be taken on the existing running activity, updating its state. + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ATTACH_REQUEST_ID_FIELD_NUMBER: builtins.int + ATTACH_COMPLETION_CALLBACKS_FIELD_NUMBER: builtins.int + ATTACH_LINKS_FIELD_NUMBER: builtins.int + attach_request_id: builtins.bool + """Attaches the request ID to the running workflow.""" + attach_completion_callbacks: builtins.bool + """Attaches the completion callbacks to the running workflow.""" + attach_links: builtins.bool + """Attaches the links to the WorkflowExecutionOptionsUpdatedEvent history event.""" + def __init__( + self, + *, + attach_request_id: builtins.bool = ..., + attach_completion_callbacks: builtins.bool = ..., + attach_links: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "attach_completion_callbacks", + b"attach_completion_callbacks", + "attach_links", + b"attach_links", + "attach_request_id", + b"attach_request_id", + ], + ) -> None: ... + +global___OnConflictOptions = OnConflictOptions + class ActivityOptions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -62,7 +107,8 @@ class ActivityOptions(google.protobuf.message.Message): def heartbeat_timeout(self) -> google.protobuf.duration_pb2.Duration: """Maximum permitted time between successful worker heartbeats.""" @property - def retry_policy(self) -> temporalio.api.common.v1.message_pb2.RetryPolicy: ... + def retry_policy(self) -> temporalio.api.common.v1.message_pb2.RetryPolicy: + """The retry policy for the activity. Will never exceed `schedule_to_close_timeout`.""" def __init__( self, *, @@ -109,3 +155,479 @@ class ActivityOptions(google.protobuf.message.Message): ) -> None: ... global___ActivityOptions = ActivityOptions + +class ActivityExecutionInfo(google.protobuf.message.Message): + """Info for a standalone activity.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class PauseInfo(google.protobuf.message.Message): + """TODO: Move this to a common package?""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Manual(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + IDENTITY_FIELD_NUMBER: builtins.int + REASON_FIELD_NUMBER: builtins.int + identity: builtins.str + """The identity of the actor that paused the activity.""" + reason: builtins.str + """Reason for pausing the activity.""" + def __init__( + self, + *, + identity: builtins.str = ..., + reason: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "identity", b"identity", "reason", b"reason" + ], + ) -> None: ... + + PAUSE_TIME_FIELD_NUMBER: builtins.int + MANUAL_FIELD_NUMBER: builtins.int + @property + def pause_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """The time when the activity was paused.""" + @property + def manual(self) -> global___ActivityExecutionInfo.PauseInfo.Manual: + """The activity was paused by direct API invocation.""" + def __init__( + self, + *, + pause_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + manual: global___ActivityExecutionInfo.PauseInfo.Manual | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "manual", + b"manual", + "pause_time", + b"pause_time", + "paused_by", + b"paused_by", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "manual", + b"manual", + "pause_time", + b"pause_time", + "paused_by", + b"paused_by", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["paused_by", b"paused_by"] + ) -> typing_extensions.Literal["manual"] | None: ... + + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + STATUS_FIELD_NUMBER: builtins.int + RUN_STATE_FIELD_NUMBER: builtins.int + HEARTBEAT_DETAILS_FIELD_NUMBER: builtins.int + LAST_HEARTBEAT_TIME_FIELD_NUMBER: builtins.int + LAST_STARTED_TIME_FIELD_NUMBER: builtins.int + ATTEMPT_FIELD_NUMBER: builtins.int + MAXIMUM_ATTEMPTS_FIELD_NUMBER: builtins.int + SCHEDULED_TIME_FIELD_NUMBER: builtins.int + EXPIRATION_TIME_FIELD_NUMBER: builtins.int + LAST_FAILURE_FIELD_NUMBER: builtins.int + LAST_WORKER_IDENTITY_FIELD_NUMBER: builtins.int + CURRENT_RETRY_INTERVAL_FIELD_NUMBER: builtins.int + LAST_ATTEMPT_COMPLETE_TIME_FIELD_NUMBER: builtins.int + NEXT_ATTEMPT_SCHEDULE_TIME_FIELD_NUMBER: builtins.int + LAST_DEPLOYMENT_VERSION_FIELD_NUMBER: builtins.int + PRIORITY_FIELD_NUMBER: builtins.int + ACTIVITY_OPTIONS_FIELD_NUMBER: builtins.int + INPUT_FIELD_NUMBER: builtins.int + STATE_TRANSITION_COUNT_FIELD_NUMBER: builtins.int + SEARCH_ATTRIBUTES_FIELD_NUMBER: builtins.int + HEADER_FIELD_NUMBER: builtins.int + EAGER_EXECUTION_REQUESTED_FIELD_NUMBER: builtins.int + COMPLETION_CALLBACKS_FIELD_NUMBER: builtins.int + USER_METADATA_FIELD_NUMBER: builtins.int + LINKS_FIELD_NUMBER: builtins.int + CANCELED_REASON_FIELD_NUMBER: builtins.int + PAUSE_INFO_FIELD_NUMBER: builtins.int + activity_id: builtins.str + """Unique identifier of this activity within its namespace along with run ID (below).""" + run_id: builtins.str + @property + def activity_type(self) -> temporalio.api.common.v1.message_pb2.ActivityType: + """The type of the activity, a string that maps to a registered activity on a worker.""" + status: temporalio.api.enums.v1.activity_pb2.ActivityExecutionStatus.ValueType + """A general status for this activity, indicates whether it is currently running or in one of the terminal statuses.""" + run_state: temporalio.api.enums.v1.workflow_pb2.PendingActivityState.ValueType + """More detailed breakdown of ACTIVITY_EXECUTION_STATUS_RUNNING.""" + @property + def heartbeat_details(self) -> temporalio.api.common.v1.message_pb2.Payloads: + """Details provided in the last recorded activity heartbeat.""" + @property + def last_heartbeat_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """Time the last heartbeat was recorded.""" + @property + def last_started_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """Time the last attempt was started.""" + attempt: builtins.int + """The attempt this activity is currently on. + Incremented each time a new attempt is started. + TODO: Confirm if this is on scheduled or started. + """ + maximum_attempts: builtins.int + @property + def scheduled_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """Time the activity was originally scheduled via a StartActivityExecution request.""" + @property + def expiration_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """Scheduled time + schedule to close timeout.""" + @property + def last_failure(self) -> temporalio.api.failure.v1.message_pb2.Failure: + """Failure details from the last failed attempt.""" + last_worker_identity: builtins.str + @property + def current_retry_interval(self) -> google.protobuf.duration_pb2.Duration: + """Time from the last attempt failure to the next activity retry. + If the activity is currently running, this represents the next retry interval in case the attempt fails. + If activity is currently backing off between attempt, this represents the current retry interval. + If there is no next retry allowed, this field will be null. + This interval is typically calculated from the specified retry policy, but may be modified if an activity fails + with a retryable application failure specifying a retry delay. + """ + @property + def last_attempt_complete_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """The time when the last activity attempt completed. If activity has not been completed yet, it will be null.""" + @property + def next_attempt_schedule_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """The time when the next activity attempt will be scheduled. + If activity is currently scheduled or started, this field will be null. + """ + @property + def last_deployment_version( + self, + ) -> temporalio.api.deployment.v1.message_pb2.WorkerDeploymentVersion: + """The Worker Deployment Version this activity was dispatched to most recently. + If nil, the activity has not yet been dispatched or was last dispatched to an unversioned worker. + """ + @property + def priority(self) -> temporalio.api.common.v1.message_pb2.Priority: + """Priority metadata.""" + @property + def activity_options(self) -> global___ActivityOptions: + """Current activity options. May be different from the one used to start the activity.""" + @property + def input(self) -> temporalio.api.common.v1.message_pb2.Payloads: + """Serialized activity input, passed as arguments to the activity function.""" + state_transition_count: builtins.int + """Incremented each time the activity's state is mutated in persistence.""" + @property + def search_attributes( + self, + ) -> temporalio.api.common.v1.message_pb2.SearchAttributes: ... + @property + def header(self) -> temporalio.api.common.v1.message_pb2.Header: ... + eager_execution_requested: builtins.bool + """Whether the activity was started with a request_eager_execution flag set to `true`, indicating that the first + task was delivered inline in the start response, bypassing matching. + """ + @property + def completion_callbacks( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + temporalio.api.common.v1.message_pb2.Callback + ]: + """Callbacks to be called by the server when this activity reaches a terminal status. + Callback addresses must be whitelisted in the server's dynamic configuration. + """ + @property + def user_metadata(self) -> temporalio.api.sdk.v1.user_metadata_pb2.UserMetadata: + """Metadata for use by user interfaces to display the fixed as-of-start summary and details of the activity.""" + @property + def links( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + temporalio.api.common.v1.message_pb2.Link + ]: + """Links to be associated with the activity.""" + canceled_reason: builtins.str + """Set if activity cancelation was requested.""" + @property + def pause_info(self) -> global___ActivityExecutionInfo.PauseInfo: ... + def __init__( + self, + *, + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + activity_type: temporalio.api.common.v1.message_pb2.ActivityType | None = ..., + status: temporalio.api.enums.v1.activity_pb2.ActivityExecutionStatus.ValueType = ..., + run_state: temporalio.api.enums.v1.workflow_pb2.PendingActivityState.ValueType = ..., + heartbeat_details: temporalio.api.common.v1.message_pb2.Payloads | None = ..., + last_heartbeat_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + last_started_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + attempt: builtins.int = ..., + maximum_attempts: builtins.int = ..., + scheduled_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + expiration_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + last_failure: temporalio.api.failure.v1.message_pb2.Failure | None = ..., + last_worker_identity: builtins.str = ..., + current_retry_interval: google.protobuf.duration_pb2.Duration | None = ..., + last_attempt_complete_time: google.protobuf.timestamp_pb2.Timestamp + | None = ..., + next_attempt_schedule_time: google.protobuf.timestamp_pb2.Timestamp + | None = ..., + last_deployment_version: temporalio.api.deployment.v1.message_pb2.WorkerDeploymentVersion + | None = ..., + priority: temporalio.api.common.v1.message_pb2.Priority | None = ..., + activity_options: global___ActivityOptions | None = ..., + input: temporalio.api.common.v1.message_pb2.Payloads | None = ..., + state_transition_count: builtins.int = ..., + search_attributes: temporalio.api.common.v1.message_pb2.SearchAttributes + | None = ..., + header: temporalio.api.common.v1.message_pb2.Header | None = ..., + eager_execution_requested: builtins.bool = ..., + completion_callbacks: collections.abc.Iterable[ + temporalio.api.common.v1.message_pb2.Callback + ] + | None = ..., + user_metadata: temporalio.api.sdk.v1.user_metadata_pb2.UserMetadata + | None = ..., + links: collections.abc.Iterable[temporalio.api.common.v1.message_pb2.Link] + | None = ..., + canceled_reason: builtins.str = ..., + pause_info: global___ActivityExecutionInfo.PauseInfo | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "activity_options", + b"activity_options", + "activity_type", + b"activity_type", + "current_retry_interval", + b"current_retry_interval", + "expiration_time", + b"expiration_time", + "header", + b"header", + "heartbeat_details", + b"heartbeat_details", + "input", + b"input", + "last_attempt_complete_time", + b"last_attempt_complete_time", + "last_deployment_version", + b"last_deployment_version", + "last_failure", + b"last_failure", + "last_heartbeat_time", + b"last_heartbeat_time", + "last_started_time", + b"last_started_time", + "next_attempt_schedule_time", + b"next_attempt_schedule_time", + "pause_info", + b"pause_info", + "priority", + b"priority", + "scheduled_time", + b"scheduled_time", + "search_attributes", + b"search_attributes", + "user_metadata", + b"user_metadata", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "activity_options", + b"activity_options", + "activity_type", + b"activity_type", + "attempt", + b"attempt", + "canceled_reason", + b"canceled_reason", + "completion_callbacks", + b"completion_callbacks", + "current_retry_interval", + b"current_retry_interval", + "eager_execution_requested", + b"eager_execution_requested", + "expiration_time", + b"expiration_time", + "header", + b"header", + "heartbeat_details", + b"heartbeat_details", + "input", + b"input", + "last_attempt_complete_time", + b"last_attempt_complete_time", + "last_deployment_version", + b"last_deployment_version", + "last_failure", + b"last_failure", + "last_heartbeat_time", + b"last_heartbeat_time", + "last_started_time", + b"last_started_time", + "last_worker_identity", + b"last_worker_identity", + "links", + b"links", + "maximum_attempts", + b"maximum_attempts", + "next_attempt_schedule_time", + b"next_attempt_schedule_time", + "pause_info", + b"pause_info", + "priority", + b"priority", + "run_id", + b"run_id", + "run_state", + b"run_state", + "scheduled_time", + b"scheduled_time", + "search_attributes", + b"search_attributes", + "state_transition_count", + b"state_transition_count", + "status", + b"status", + "user_metadata", + b"user_metadata", + ], + ) -> None: ... + +global___ActivityExecutionInfo = ActivityExecutionInfo + +class ActivityListInfo(google.protobuf.message.Message): + """Limited activity information returned in the list response.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + WORKFLOW_ID_FIELD_NUMBER: builtins.int + ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + SCHEDULED_TIME_FIELD_NUMBER: builtins.int + CLOSE_TIME_FIELD_NUMBER: builtins.int + STATUS_FIELD_NUMBER: builtins.int + SEARCH_ATTRIBUTES_FIELD_NUMBER: builtins.int + TASK_QUEUE_FIELD_NUMBER: builtins.int + STATE_TRANSITION_COUNT_FIELD_NUMBER: builtins.int + STATE_SIZE_BYTES_FIELD_NUMBER: builtins.int + EXECUTION_DURATION_FIELD_NUMBER: builtins.int + activity_id: builtins.str + """For standalone activity - a unique identifier of this activity within its namespace along with run ID (below).""" + run_id: builtins.str + """The run ID of the workflow or standalone activity.""" + workflow_id: builtins.str + """Workflow that contains this activity - only present for workflow activity.""" + @property + def activity_type(self) -> temporalio.api.common.v1.message_pb2.ActivityType: + """The type of the activity, a string that maps to a registered activity on a worker.""" + @property + def scheduled_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """Time the activity was originally scheduled via a StartActivityExecution request. + TODO: Workflows call this schedule_time but it's scheduled_time in PendingActivityInfo, what should we choose for + consistency? + """ + @property + def close_time(self) -> google.protobuf.timestamp_pb2.Timestamp: + """If the activity is in a terminal status, this field represents the time the activity transitioned to that status.""" + status: temporalio.api.enums.v1.activity_pb2.ActivityExecutionStatus.ValueType + """Only scheduled and terminal statuses appear here. More detailed information in PendingActivityInfo but not + available in the list response. + """ + @property + def search_attributes( + self, + ) -> temporalio.api.common.v1.message_pb2.SearchAttributes: + """Search attributes from the start request.""" + task_queue: builtins.str + """The task queue this activity was scheduled on when it was originally started, updated on activity options update.""" + state_transition_count: builtins.int + """Updated on terminal status.""" + state_size_bytes: builtins.int + """Updated once on scheduled and once on terminal status.""" + @property + def execution_duration(self) -> google.protobuf.duration_pb2.Duration: + """The difference between close time and scheduled time. + This field is only populated if the activity is closed. + """ + def __init__( + self, + *, + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + workflow_id: builtins.str = ..., + activity_type: temporalio.api.common.v1.message_pb2.ActivityType | None = ..., + scheduled_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + close_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., + status: temporalio.api.enums.v1.activity_pb2.ActivityExecutionStatus.ValueType = ..., + search_attributes: temporalio.api.common.v1.message_pb2.SearchAttributes + | None = ..., + task_queue: builtins.str = ..., + state_transition_count: builtins.int = ..., + state_size_bytes: builtins.int = ..., + execution_duration: google.protobuf.duration_pb2.Duration | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "activity_type", + b"activity_type", + "close_time", + b"close_time", + "execution_duration", + b"execution_duration", + "scheduled_time", + b"scheduled_time", + "search_attributes", + b"search_attributes", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "activity_type", + b"activity_type", + "close_time", + b"close_time", + "execution_duration", + b"execution_duration", + "run_id", + b"run_id", + "scheduled_time", + b"scheduled_time", + "search_attributes", + b"search_attributes", + "state_size_bytes", + b"state_size_bytes", + "state_transition_count", + b"state_transition_count", + "status", + b"status", + "task_queue", + b"task_queue", + "workflow_id", + b"workflow_id", + ], + ) -> None: ... + +global___ActivityListInfo = ActivityListInfo diff --git a/temporalio/api/common/v1/message_pb2.py b/temporalio/api/common/v1/message_pb2.py index a30edcac2..97b6f6720 100644 --- a/temporalio/api/common/v1/message_pb2.py +++ b/temporalio/api/common/v1/message_pb2.py @@ -28,7 +28,7 @@ ) DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n$temporal/api/common/v1/message.proto\x12\x16temporal.api.common.v1\x1a\x1egoogle/protobuf/duration.proto\x1a\x1bgoogle/protobuf/empty.proto\x1a"temporal/api/enums/v1/common.proto\x1a&temporal/api/enums/v1/event_type.proto\x1a!temporal/api/enums/v1/reset.proto"T\n\x08\x44\x61taBlob\x12:\n\rencoding_type\x18\x01 \x01(\x0e\x32#.temporal.api.enums.v1.EncodingType\x12\x0c\n\x04\x64\x61ta\x18\x02 \x01(\x0c"=\n\x08Payloads\x12\x31\n\x08payloads\x18\x01 \x03(\x0b\x32\x1f.temporal.api.common.v1.Payload"\x89\x01\n\x07Payload\x12?\n\x08metadata\x18\x01 \x03(\x0b\x32-.temporal.api.common.v1.Payload.MetadataEntry\x12\x0c\n\x04\x64\x61ta\x18\x02 \x01(\x0c\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\x0c:\x02\x38\x01"\xbe\x01\n\x10SearchAttributes\x12S\n\x0eindexed_fields\x18\x01 \x03(\x0b\x32;.temporal.api.common.v1.SearchAttributes.IndexedFieldsEntry\x1aU\n\x12IndexedFieldsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\x90\x01\n\x04Memo\x12\x38\n\x06\x66ields\x18\x01 \x03(\x0b\x32(.temporal.api.common.v1.Memo.FieldsEntry\x1aN\n\x0b\x46ieldsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\x94\x01\n\x06Header\x12:\n\x06\x66ields\x18\x01 \x03(\x0b\x32*.temporal.api.common.v1.Header.FieldsEntry\x1aN\n\x0b\x46ieldsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"8\n\x11WorkflowExecution\x12\x13\n\x0bworkflow_id\x18\x01 \x01(\t\x12\x0e\n\x06run_id\x18\x02 \x01(\t"\x1c\n\x0cWorkflowType\x12\x0c\n\x04name\x18\x01 \x01(\t"\x1c\n\x0c\x41\x63tivityType\x12\x0c\n\x04name\x18\x01 \x01(\t"\xd1\x01\n\x0bRetryPolicy\x12\x33\n\x10initial_interval\x18\x01 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x1b\n\x13\x62\x61\x63koff_coefficient\x18\x02 \x01(\x01\x12\x33\n\x10maximum_interval\x18\x03 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x18\n\x10maximum_attempts\x18\x04 \x01(\x05\x12!\n\x19non_retryable_error_types\x18\x05 \x03(\t"F\n\x10MeteringMetadata\x12\x32\n*nonfirst_local_activity_execution_attempts\x18\r \x01(\r">\n\x12WorkerVersionStamp\x12\x10\n\x08\x62uild_id\x18\x01 \x01(\t\x12\x16\n\x0euse_versioning\x18\x03 \x01(\x08"e\n\x19WorkerVersionCapabilities\x12\x10\n\x08\x62uild_id\x18\x01 \x01(\t\x12\x16\n\x0euse_versioning\x18\x02 \x01(\x08\x12\x1e\n\x16\x64\x65ployment_series_name\x18\x04 \x01(\t"\xed\x02\n\x0cResetOptions\x12\x35\n\x13\x66irst_workflow_task\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyH\x00\x12\x34\n\x12last_workflow_task\x18\x02 \x01(\x0b\x32\x16.google.protobuf.EmptyH\x00\x12\x1a\n\x10workflow_task_id\x18\x03 \x01(\x03H\x00\x12\x12\n\x08\x62uild_id\x18\x04 \x01(\tH\x00\x12G\n\x12reset_reapply_type\x18\n \x01(\x0e\x32\'.temporal.api.enums.v1.ResetReapplyTypeB\x02\x18\x01\x12\x18\n\x10\x63urrent_run_only\x18\x0b \x01(\x08\x12S\n\x1breset_reapply_exclude_types\x18\x0c \x03(\x0e\x32..temporal.api.enums.v1.ResetReapplyExcludeTypeB\x08\n\x06target"\xe4\x02\n\x08\x43\x61llback\x12\x37\n\x05nexus\x18\x02 \x01(\x0b\x32&.temporal.api.common.v1.Callback.NexusH\x00\x12=\n\x08internal\x18\x03 \x01(\x0b\x32).temporal.api.common.v1.Callback.InternalH\x00\x12+\n\x05links\x18\x64 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x1a\x87\x01\n\x05Nexus\x12\x0b\n\x03url\x18\x01 \x01(\t\x12\x42\n\x06header\x18\x02 \x03(\x0b\x32\x32.temporal.api.common.v1.Callback.Nexus.HeaderEntry\x1a-\n\x0bHeaderEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\x18\n\x08Internal\x12\x0c\n\x04\x64\x61ta\x18\x01 \x01(\x0c\x42\t\n\x07variantJ\x04\x08\x01\x10\x02"\xe9\x04\n\x04Link\x12\x44\n\x0eworkflow_event\x18\x01 \x01(\x0b\x32*.temporal.api.common.v1.Link.WorkflowEventH\x00\x12:\n\tbatch_job\x18\x02 \x01(\x0b\x32%.temporal.api.common.v1.Link.BatchJobH\x00\x1a\xb7\x03\n\rWorkflowEvent\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12N\n\tevent_ref\x18\x64 \x01(\x0b\x32\x39.temporal.api.common.v1.Link.WorkflowEvent.EventReferenceH\x00\x12W\n\x0erequest_id_ref\x18\x65 \x01(\x0b\x32=.temporal.api.common.v1.Link.WorkflowEvent.RequestIdReferenceH\x00\x1aX\n\x0e\x45ventReference\x12\x10\n\x08\x65vent_id\x18\x01 \x01(\x03\x12\x34\n\nevent_type\x18\x02 \x01(\x0e\x32 .temporal.api.enums.v1.EventType\x1a^\n\x12RequestIdReference\x12\x12\n\nrequest_id\x18\x01 \x01(\t\x12\x34\n\nevent_type\x18\x02 \x01(\x0e\x32 .temporal.api.enums.v1.EventTypeB\x0b\n\treference\x1a\x1a\n\x08\x42\x61tchJob\x12\x0e\n\x06job_id\x18\x01 \x01(\tB\t\n\x07variant"O\n\x08Priority\x12\x14\n\x0cpriority_key\x18\x01 \x01(\x05\x12\x14\n\x0c\x66\x61irness_key\x18\x02 \x01(\t\x12\x17\n\x0f\x66\x61irness_weight\x18\x03 \x01(\x02";\n\x0eWorkerSelector\x12\x1d\n\x13worker_instance_key\x18\x01 \x01(\tH\x00\x42\n\n\x08selectorB\x89\x01\n\x19io.temporal.api.common.v1B\x0cMessageProtoP\x01Z#go.temporal.io/api/common/v1;common\xaa\x02\x18Temporalio.Api.Common.V1\xea\x02\x1bTemporalio::Api::Common::V1b\x06proto3' + b'\n$temporal/api/common/v1/message.proto\x12\x16temporal.api.common.v1\x1a\x1egoogle/protobuf/duration.proto\x1a\x1bgoogle/protobuf/empty.proto\x1a"temporal/api/enums/v1/common.proto\x1a&temporal/api/enums/v1/event_type.proto\x1a!temporal/api/enums/v1/reset.proto"T\n\x08\x44\x61taBlob\x12:\n\rencoding_type\x18\x01 \x01(\x0e\x32#.temporal.api.enums.v1.EncodingType\x12\x0c\n\x04\x64\x61ta\x18\x02 \x01(\x0c"=\n\x08Payloads\x12\x31\n\x08payloads\x18\x01 \x03(\x0b\x32\x1f.temporal.api.common.v1.Payload"\x89\x01\n\x07Payload\x12?\n\x08metadata\x18\x01 \x03(\x0b\x32-.temporal.api.common.v1.Payload.MetadataEntry\x12\x0c\n\x04\x64\x61ta\x18\x02 \x01(\x0c\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\x0c:\x02\x38\x01"\xbe\x01\n\x10SearchAttributes\x12S\n\x0eindexed_fields\x18\x01 \x03(\x0b\x32;.temporal.api.common.v1.SearchAttributes.IndexedFieldsEntry\x1aU\n\x12IndexedFieldsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\x90\x01\n\x04Memo\x12\x38\n\x06\x66ields\x18\x01 \x03(\x0b\x32(.temporal.api.common.v1.Memo.FieldsEntry\x1aN\n\x0b\x46ieldsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\x94\x01\n\x06Header\x12:\n\x06\x66ields\x18\x01 \x03(\x0b\x32*.temporal.api.common.v1.Header.FieldsEntry\x1aN\n\x0b\x46ieldsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"8\n\x11WorkflowExecution\x12\x13\n\x0bworkflow_id\x18\x01 \x01(\t\x12\x0e\n\x06run_id\x18\x02 \x01(\t"\x1c\n\x0cWorkflowType\x12\x0c\n\x04name\x18\x01 \x01(\t"\x1c\n\x0c\x41\x63tivityType\x12\x0c\n\x04name\x18\x01 \x01(\t"\xd1\x01\n\x0bRetryPolicy\x12\x33\n\x10initial_interval\x18\x01 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x1b\n\x13\x62\x61\x63koff_coefficient\x18\x02 \x01(\x01\x12\x33\n\x10maximum_interval\x18\x03 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x18\n\x10maximum_attempts\x18\x04 \x01(\x05\x12!\n\x19non_retryable_error_types\x18\x05 \x03(\t"F\n\x10MeteringMetadata\x12\x32\n*nonfirst_local_activity_execution_attempts\x18\r \x01(\r">\n\x12WorkerVersionStamp\x12\x10\n\x08\x62uild_id\x18\x01 \x01(\t\x12\x16\n\x0euse_versioning\x18\x03 \x01(\x08"e\n\x19WorkerVersionCapabilities\x12\x10\n\x08\x62uild_id\x18\x01 \x01(\t\x12\x16\n\x0euse_versioning\x18\x02 \x01(\x08\x12\x1e\n\x16\x64\x65ployment_series_name\x18\x04 \x01(\t"\xed\x02\n\x0cResetOptions\x12\x35\n\x13\x66irst_workflow_task\x18\x01 \x01(\x0b\x32\x16.google.protobuf.EmptyH\x00\x12\x34\n\x12last_workflow_task\x18\x02 \x01(\x0b\x32\x16.google.protobuf.EmptyH\x00\x12\x1a\n\x10workflow_task_id\x18\x03 \x01(\x03H\x00\x12\x12\n\x08\x62uild_id\x18\x04 \x01(\tH\x00\x12G\n\x12reset_reapply_type\x18\n \x01(\x0e\x32\'.temporal.api.enums.v1.ResetReapplyTypeB\x02\x18\x01\x12\x18\n\x10\x63urrent_run_only\x18\x0b \x01(\x08\x12S\n\x1breset_reapply_exclude_types\x18\x0c \x03(\x0e\x32..temporal.api.enums.v1.ResetReapplyExcludeTypeB\x08\n\x06target"\xe4\x02\n\x08\x43\x61llback\x12\x37\n\x05nexus\x18\x02 \x01(\x0b\x32&.temporal.api.common.v1.Callback.NexusH\x00\x12=\n\x08internal\x18\x03 \x01(\x0b\x32).temporal.api.common.v1.Callback.InternalH\x00\x12+\n\x05links\x18\x64 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x1a\x87\x01\n\x05Nexus\x12\x0b\n\x03url\x18\x01 \x01(\t\x12\x42\n\x06header\x18\x02 \x03(\x0b\x32\x32.temporal.api.common.v1.Callback.Nexus.HeaderEntry\x1a-\n\x0bHeaderEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\x18\n\x08Internal\x12\x0c\n\x04\x64\x61ta\x18\x01 \x01(\x0c\x42\t\n\x07variantJ\x04\x08\x01\x10\x02"\xe8\x05\n\x04Link\x12\x44\n\x0eworkflow_event\x18\x01 \x01(\x0b\x32*.temporal.api.common.v1.Link.WorkflowEventH\x00\x12:\n\tbatch_job\x18\x02 \x01(\x0b\x32%.temporal.api.common.v1.Link.BatchJobH\x00\x12\x39\n\x08\x61\x63tivity\x18\x03 \x01(\x0b\x32%.temporal.api.common.v1.Link.ActivityH\x00\x1a\xb7\x03\n\rWorkflowEvent\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12N\n\tevent_ref\x18\x64 \x01(\x0b\x32\x39.temporal.api.common.v1.Link.WorkflowEvent.EventReferenceH\x00\x12W\n\x0erequest_id_ref\x18\x65 \x01(\x0b\x32=.temporal.api.common.v1.Link.WorkflowEvent.RequestIdReferenceH\x00\x1aX\n\x0e\x45ventReference\x12\x10\n\x08\x65vent_id\x18\x01 \x01(\x03\x12\x34\n\nevent_type\x18\x02 \x01(\x0e\x32 .temporal.api.enums.v1.EventType\x1a^\n\x12RequestIdReference\x12\x12\n\nrequest_id\x18\x01 \x01(\t\x12\x34\n\nevent_type\x18\x02 \x01(\x0e\x32 .temporal.api.enums.v1.EventTypeB\x0b\n\treference\x1a\x1a\n\x08\x42\x61tchJob\x12\x0e\n\x06job_id\x18\x01 \x01(\t\x1a\x42\n\x08\x41\x63tivity\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\tB\t\n\x07variant"O\n\x08Priority\x12\x14\n\x0cpriority_key\x18\x01 \x01(\x05\x12\x14\n\x0c\x66\x61irness_key\x18\x02 \x01(\t\x12\x17\n\x0f\x66\x61irness_weight\x18\x03 \x01(\x02";\n\x0eWorkerSelector\x12\x1d\n\x13worker_instance_key\x18\x01 \x01(\tH\x00\x42\n\n\x08selectorB\x89\x01\n\x19io.temporal.api.common.v1B\x0cMessageProtoP\x01Z#go.temporal.io/api/common/v1;common\xaa\x02\x18Temporalio.Api.Common.V1\xea\x02\x1bTemporalio::Api::Common::V1b\x06proto3' ) @@ -67,6 +67,7 @@ "RequestIdReference" ] _LINK_BATCHJOB = _LINK.nested_types_by_name["BatchJob"] +_LINK_ACTIVITY = _LINK.nested_types_by_name["Activity"] _PRIORITY = DESCRIPTOR.message_types_by_name["Priority"] _WORKERSELECTOR = DESCRIPTOR.message_types_by_name["WorkerSelector"] DataBlob = _reflection.GeneratedProtocolMessageType( @@ -344,6 +345,15 @@ # @@protoc_insertion_point(class_scope:temporal.api.common.v1.Link.BatchJob) }, ), + "Activity": _reflection.GeneratedProtocolMessageType( + "Activity", + (_message.Message,), + { + "DESCRIPTOR": _LINK_ACTIVITY, + "__module__": "temporal.api.common.v1.message_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.common.v1.Link.Activity) + }, + ), "DESCRIPTOR": _LINK, "__module__": "temporal.api.common.v1.message_pb2", # @@protoc_insertion_point(class_scope:temporal.api.common.v1.Link) @@ -354,6 +364,7 @@ _sym_db.RegisterMessage(Link.WorkflowEvent.EventReference) _sym_db.RegisterMessage(Link.WorkflowEvent.RequestIdReference) _sym_db.RegisterMessage(Link.BatchJob) +_sym_db.RegisterMessage(Link.Activity) Priority = _reflection.GeneratedProtocolMessageType( "Priority", @@ -437,17 +448,19 @@ _CALLBACK_INTERNAL._serialized_start = 2269 _CALLBACK_INTERNAL._serialized_end = 2293 _LINK._serialized_start = 2313 - _LINK._serialized_end = 2930 - _LINK_WORKFLOWEVENT._serialized_start = 2452 - _LINK_WORKFLOWEVENT._serialized_end = 2891 - _LINK_WORKFLOWEVENT_EVENTREFERENCE._serialized_start = 2694 - _LINK_WORKFLOWEVENT_EVENTREFERENCE._serialized_end = 2782 - _LINK_WORKFLOWEVENT_REQUESTIDREFERENCE._serialized_start = 2784 - _LINK_WORKFLOWEVENT_REQUESTIDREFERENCE._serialized_end = 2878 - _LINK_BATCHJOB._serialized_start = 2893 - _LINK_BATCHJOB._serialized_end = 2919 - _PRIORITY._serialized_start = 2932 - _PRIORITY._serialized_end = 3011 - _WORKERSELECTOR._serialized_start = 3013 - _WORKERSELECTOR._serialized_end = 3072 + _LINK._serialized_end = 3057 + _LINK_WORKFLOWEVENT._serialized_start = 2511 + _LINK_WORKFLOWEVENT._serialized_end = 2950 + _LINK_WORKFLOWEVENT_EVENTREFERENCE._serialized_start = 2753 + _LINK_WORKFLOWEVENT_EVENTREFERENCE._serialized_end = 2841 + _LINK_WORKFLOWEVENT_REQUESTIDREFERENCE._serialized_start = 2843 + _LINK_WORKFLOWEVENT_REQUESTIDREFERENCE._serialized_end = 2937 + _LINK_BATCHJOB._serialized_start = 2952 + _LINK_BATCHJOB._serialized_end = 2978 + _LINK_ACTIVITY._serialized_start = 2980 + _LINK_ACTIVITY._serialized_end = 3046 + _PRIORITY._serialized_start = 3059 + _PRIORITY._serialized_end = 3138 + _WORKERSELECTOR._serialized_start = 3140 + _WORKERSELECTOR._serialized_end = 3199 # @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/common/v1/message_pb2.pyi b/temporalio/api/common/v1/message_pb2.pyi index f94baa802..65a42ae2f 100644 --- a/temporalio/api/common/v1/message_pb2.pyi +++ b/temporalio/api/common/v1/message_pb2.pyi @@ -847,21 +847,57 @@ class Link(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["job_id", b"job_id"] ) -> None: ... + class Activity(google.protobuf.message.Message): + """A link to an activity.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + namespace: builtins.str + activity_id: builtins.str + run_id: builtins.str + def __init__( + self, + *, + namespace: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "namespace", + b"namespace", + "run_id", + b"run_id", + ], + ) -> None: ... + WORKFLOW_EVENT_FIELD_NUMBER: builtins.int BATCH_JOB_FIELD_NUMBER: builtins.int + ACTIVITY_FIELD_NUMBER: builtins.int @property def workflow_event(self) -> global___Link.WorkflowEvent: ... @property def batch_job(self) -> global___Link.BatchJob: ... + @property + def activity(self) -> global___Link.Activity: ... def __init__( self, *, workflow_event: global___Link.WorkflowEvent | None = ..., batch_job: global___Link.BatchJob | None = ..., + activity: global___Link.Activity | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ + "activity", + b"activity", "batch_job", b"batch_job", "variant", @@ -873,6 +909,8 @@ class Link(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ + "activity", + b"activity", "batch_job", b"batch_job", "variant", @@ -883,7 +921,9 @@ class Link(google.protobuf.message.Message): ) -> None: ... def WhichOneof( self, oneof_group: typing_extensions.Literal["variant", b"variant"] - ) -> typing_extensions.Literal["workflow_event", "batch_job"] | None: ... + ) -> ( + typing_extensions.Literal["workflow_event", "batch_job", "activity"] | None + ): ... global___Link = Link @@ -937,7 +977,7 @@ class Priority(google.protobuf.message.Message): configuration, and defaults to 5. If priority is not present (or zero), then the effective priority will be - the default priority, which is is calculated by (min+max)/2. With the + the default priority, which is calculated by (min+max)/2. With the default max of 5, and min of 1, that comes out to 3. """ fairness_key: builtins.str diff --git a/temporalio/api/deployment/v1/message_pb2.py b/temporalio/api/deployment/v1/message_pb2.py index 3a9b0127b..507e594e7 100644 --- a/temporalio/api/deployment/v1/message_pb2.py +++ b/temporalio/api/deployment/v1/message_pb2.py @@ -27,7 +27,7 @@ ) DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n(temporal/api/deployment/v1/message.proto\x12\x1atemporal.api.deployment.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a&temporal/api/enums/v1/deployment.proto\x1a&temporal/api/enums/v1/task_queue.proto\x1a$temporal/api/common/v1/message.proto"\x91\x01\n\x17WorkerDeploymentOptions\x12\x17\n\x0f\x64\x65ployment_name\x18\x01 \x01(\t\x12\x10\n\x08\x62uild_id\x18\x02 \x01(\t\x12K\n\x16worker_versioning_mode\x18\x03 \x01(\x0e\x32+.temporal.api.enums.v1.WorkerVersioningMode"3\n\nDeployment\x12\x13\n\x0bseries_name\x18\x01 \x01(\t\x12\x10\n\x08\x62uild_id\x18\x02 \x01(\t"\x8e\x04\n\x0e\x44\x65ploymentInfo\x12:\n\ndeployment\x18\x01 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x10task_queue_infos\x18\x03 \x03(\x0b\x32\x38.temporal.api.deployment.v1.DeploymentInfo.TaskQueueInfo\x12J\n\x08metadata\x18\x04 \x03(\x0b\x32\x38.temporal.api.deployment.v1.DeploymentInfo.MetadataEntry\x12\x12\n\nis_current\x18\x05 \x01(\x08\x1aP\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01\x1a\x88\x01\n\rTaskQueueInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x32\n\x04type\x18\x02 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12\x35\n\x11\x66irst_poller_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xea\x01\n\x18UpdateDeploymentMetadata\x12_\n\x0eupsert_entries\x18\x01 \x03(\x0b\x32G.temporal.api.deployment.v1.UpdateDeploymentMetadata.UpsertEntriesEntry\x12\x16\n\x0eremove_entries\x18\x02 \x03(\t\x1aU\n\x12UpsertEntriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\x95\x01\n\x12\x44\x65ploymentListInfo\x12:\n\ndeployment\x18\x01 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x12\n\nis_current\x18\x03 \x01(\x08"\x96\x07\n\x1bWorkerDeploymentVersionInfo\x12\x13\n\x07version\x18\x01 \x01(\tB\x02\x18\x01\x12\x44\n\x06status\x18\x0e \x01(\x0e\x32\x34.temporal.api.enums.v1.WorkerDeploymentVersionStatus\x12O\n\x12\x64\x65ployment_version\x18\x0b \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12/\n\x0b\x63reate_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x14routing_changed_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x12\x63urrent_since_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x12ramping_since_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x39\n\x15\x66irst_activation_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12:\n\x16last_deactivation_time\x18\r \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x17\n\x0framp_percentage\x18\x07 \x01(\x02\x12\x66\n\x10task_queue_infos\x18\x08 \x03(\x0b\x32L.temporal.api.deployment.v1.WorkerDeploymentVersionInfo.VersionTaskQueueInfo\x12\x46\n\rdrainage_info\x18\t \x01(\x0b\x32/.temporal.api.deployment.v1.VersionDrainageInfo\x12=\n\x08metadata\x18\n \x01(\x0b\x32+.temporal.api.deployment.v1.VersionMetadata\x1aX\n\x14VersionTaskQueueInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x32\n\x04type\x18\x02 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType"\xc1\x01\n\x13VersionDrainageInfo\x12<\n\x06status\x18\x01 \x01(\x0e\x32,.temporal.api.enums.v1.VersionDrainageStatus\x12\x35\n\x11last_changed_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x35\n\x11last_checked_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xd3\x07\n\x14WorkerDeploymentInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12j\n\x11version_summaries\x18\x02 \x03(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary\x12/\n\x0b\x63reate_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x41\n\x0erouting_config\x18\x04 \x01(\x0b\x32).temporal.api.deployment.v1.RoutingConfig\x12\x1e\n\x16last_modifier_identity\x18\x05 \x01(\t\x1a\xac\x05\n\x1eWorkerDeploymentVersionSummary\x12\x13\n\x07version\x18\x01 \x01(\tB\x02\x18\x01\x12\x44\n\x06status\x18\x0b \x01(\x0e\x32\x34.temporal.api.enums.v1.WorkerDeploymentVersionStatus\x12O\n\x12\x64\x65ployment_version\x18\x04 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x45\n\x0f\x64rainage_status\x18\x03 \x01(\x0e\x32,.temporal.api.enums.v1.VersionDrainageStatus\x12\x46\n\rdrainage_info\x18\x05 \x01(\x0b\x32/.temporal.api.deployment.v1.VersionDrainageInfo\x12\x36\n\x12\x63urrent_since_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x12ramping_since_time\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x37\n\x13routing_update_time\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x39\n\x15\x66irst_activation_time\x18\t \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12:\n\x16last_deactivation_time\x18\n \x01(\x0b\x32\x1a.google.protobuf.Timestamp"D\n\x17WorkerDeploymentVersion\x12\x10\n\x08\x62uild_id\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t"\xad\x01\n\x0fVersionMetadata\x12I\n\x07\x65ntries\x18\x01 \x03(\x0b\x32\x38.temporal.api.deployment.v1.VersionMetadata.EntriesEntry\x1aO\n\x0c\x45ntriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\xf0\x03\n\rRoutingConfig\x12W\n\x1a\x63urrent_deployment_version\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1b\n\x0f\x63urrent_version\x18\x01 \x01(\tB\x02\x18\x01\x12W\n\x1aramping_deployment_version\x18\t \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1b\n\x0framping_version\x18\x02 \x01(\tB\x02\x18\x01\x12"\n\x1aramping_version_percentage\x18\x03 \x01(\x02\x12@\n\x1c\x63urrent_version_changed_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12@\n\x1cramping_version_changed_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12K\n\'ramping_version_percentage_changed_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampB\x9d\x01\n\x1dio.temporal.api.deployment.v1B\x0cMessageProtoP\x01Z+go.temporal.io/api/deployment/v1;deployment\xaa\x02\x1cTemporalio.Api.Deployment.V1\xea\x02\x1fTemporalio::Api::Deployment::V1b\x06proto3' + b'\n(temporal/api/deployment/v1/message.proto\x12\x1atemporal.api.deployment.v1\x1a\x1fgoogle/protobuf/timestamp.proto\x1a&temporal/api/enums/v1/deployment.proto\x1a&temporal/api/enums/v1/task_queue.proto\x1a$temporal/api/common/v1/message.proto"\x91\x01\n\x17WorkerDeploymentOptions\x12\x17\n\x0f\x64\x65ployment_name\x18\x01 \x01(\t\x12\x10\n\x08\x62uild_id\x18\x02 \x01(\t\x12K\n\x16worker_versioning_mode\x18\x03 \x01(\x0e\x32+.temporal.api.enums.v1.WorkerVersioningMode"3\n\nDeployment\x12\x13\n\x0bseries_name\x18\x01 \x01(\t\x12\x10\n\x08\x62uild_id\x18\x02 \x01(\t"\x8e\x04\n\x0e\x44\x65ploymentInfo\x12:\n\ndeployment\x18\x01 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12R\n\x10task_queue_infos\x18\x03 \x03(\x0b\x32\x38.temporal.api.deployment.v1.DeploymentInfo.TaskQueueInfo\x12J\n\x08metadata\x18\x04 \x03(\x0b\x32\x38.temporal.api.deployment.v1.DeploymentInfo.MetadataEntry\x12\x12\n\nis_current\x18\x05 \x01(\x08\x1aP\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01\x1a\x88\x01\n\rTaskQueueInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x32\n\x04type\x18\x02 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12\x35\n\x11\x66irst_poller_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xea\x01\n\x18UpdateDeploymentMetadata\x12_\n\x0eupsert_entries\x18\x01 \x03(\x0b\x32G.temporal.api.deployment.v1.UpdateDeploymentMetadata.UpsertEntriesEntry\x12\x16\n\x0eremove_entries\x18\x02 \x03(\t\x1aU\n\x12UpsertEntriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\x95\x01\n\x12\x44\x65ploymentListInfo\x12:\n\ndeployment\x18\x01 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x12\n\nis_current\x18\x03 \x01(\x08"\x96\x07\n\x1bWorkerDeploymentVersionInfo\x12\x13\n\x07version\x18\x01 \x01(\tB\x02\x18\x01\x12\x44\n\x06status\x18\x0e \x01(\x0e\x32\x34.temporal.api.enums.v1.WorkerDeploymentVersionStatus\x12O\n\x12\x64\x65ployment_version\x18\x0b \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12/\n\x0b\x63reate_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x38\n\x14routing_changed_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x12\x63urrent_since_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x12ramping_since_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x39\n\x15\x66irst_activation_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12:\n\x16last_deactivation_time\x18\r \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x17\n\x0framp_percentage\x18\x07 \x01(\x02\x12\x66\n\x10task_queue_infos\x18\x08 \x03(\x0b\x32L.temporal.api.deployment.v1.WorkerDeploymentVersionInfo.VersionTaskQueueInfo\x12\x46\n\rdrainage_info\x18\t \x01(\x0b\x32/.temporal.api.deployment.v1.VersionDrainageInfo\x12=\n\x08metadata\x18\n \x01(\x0b\x32+.temporal.api.deployment.v1.VersionMetadata\x1aX\n\x14VersionTaskQueueInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x32\n\x04type\x18\x02 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType"\xc1\x01\n\x13VersionDrainageInfo\x12<\n\x06status\x18\x01 \x01(\x0e\x32,.temporal.api.enums.v1.VersionDrainageStatus\x12\x35\n\x11last_changed_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x35\n\x11last_checked_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xed\x07\n\x14WorkerDeploymentInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12j\n\x11version_summaries\x18\x02 \x03(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary\x12/\n\x0b\x63reate_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x41\n\x0erouting_config\x18\x04 \x01(\x0b\x32).temporal.api.deployment.v1.RoutingConfig\x12\x1e\n\x16last_modifier_identity\x18\x05 \x01(\t\x12\x18\n\x10manager_identity\x18\x06 \x01(\t\x1a\xac\x05\n\x1eWorkerDeploymentVersionSummary\x12\x13\n\x07version\x18\x01 \x01(\tB\x02\x18\x01\x12\x44\n\x06status\x18\x0b \x01(\x0e\x32\x34.temporal.api.enums.v1.WorkerDeploymentVersionStatus\x12O\n\x12\x64\x65ployment_version\x18\x04 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x45\n\x0f\x64rainage_status\x18\x03 \x01(\x0e\x32,.temporal.api.enums.v1.VersionDrainageStatus\x12\x46\n\rdrainage_info\x18\x05 \x01(\x0b\x32/.temporal.api.deployment.v1.VersionDrainageInfo\x12\x36\n\x12\x63urrent_since_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x36\n\x12ramping_since_time\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x37\n\x13routing_update_time\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x39\n\x15\x66irst_activation_time\x18\t \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12:\n\x16last_deactivation_time\x18\n \x01(\x0b\x32\x1a.google.protobuf.Timestamp"D\n\x17WorkerDeploymentVersion\x12\x10\n\x08\x62uild_id\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t"\xad\x01\n\x0fVersionMetadata\x12I\n\x07\x65ntries\x18\x01 \x03(\x0b\x32\x38.temporal.api.deployment.v1.VersionMetadata.EntriesEntry\x1aO\n\x0c\x45ntriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"\xf0\x03\n\rRoutingConfig\x12W\n\x1a\x63urrent_deployment_version\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1b\n\x0f\x63urrent_version\x18\x01 \x01(\tB\x02\x18\x01\x12W\n\x1aramping_deployment_version\x18\t \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1b\n\x0framping_version\x18\x02 \x01(\tB\x02\x18\x01\x12"\n\x1aramping_version_percentage\x18\x03 \x01(\x02\x12@\n\x1c\x63urrent_version_changed_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12@\n\x1cramping_version_changed_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12K\n\'ramping_version_percentage_changed_time\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampB\x9d\x01\n\x1dio.temporal.api.deployment.v1B\x0cMessageProtoP\x01Z+go.temporal.io/api/deployment/v1;deployment\xaa\x02\x1cTemporalio.Api.Deployment.V1\xea\x02\x1fTemporalio::Api::Deployment::V1b\x06proto3' ) @@ -283,15 +283,15 @@ _VERSIONDRAINAGEINFO._serialized_start = 2264 _VERSIONDRAINAGEINFO._serialized_end = 2457 _WORKERDEPLOYMENTINFO._serialized_start = 2460 - _WORKERDEPLOYMENTINFO._serialized_end = 3439 - _WORKERDEPLOYMENTINFO_WORKERDEPLOYMENTVERSIONSUMMARY._serialized_start = 2755 - _WORKERDEPLOYMENTINFO_WORKERDEPLOYMENTVERSIONSUMMARY._serialized_end = 3439 - _WORKERDEPLOYMENTVERSION._serialized_start = 3441 - _WORKERDEPLOYMENTVERSION._serialized_end = 3509 - _VERSIONMETADATA._serialized_start = 3512 - _VERSIONMETADATA._serialized_end = 3685 - _VERSIONMETADATA_ENTRIESENTRY._serialized_start = 3606 - _VERSIONMETADATA_ENTRIESENTRY._serialized_end = 3685 - _ROUTINGCONFIG._serialized_start = 3688 - _ROUTINGCONFIG._serialized_end = 4184 + _WORKERDEPLOYMENTINFO._serialized_end = 3465 + _WORKERDEPLOYMENTINFO_WORKERDEPLOYMENTVERSIONSUMMARY._serialized_start = 2781 + _WORKERDEPLOYMENTINFO_WORKERDEPLOYMENTVERSIONSUMMARY._serialized_end = 3465 + _WORKERDEPLOYMENTVERSION._serialized_start = 3467 + _WORKERDEPLOYMENTVERSION._serialized_end = 3535 + _VERSIONMETADATA._serialized_start = 3538 + _VERSIONMETADATA._serialized_end = 3711 + _VERSIONMETADATA_ENTRIESENTRY._serialized_start = 3632 + _VERSIONMETADATA_ENTRIESENTRY._serialized_end = 3711 + _ROUTINGCONFIG._serialized_start = 3714 + _ROUTINGCONFIG._serialized_end = 4210 # @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/deployment/v1/message_pb2.pyi b/temporalio/api/deployment/v1/message_pb2.pyi index 79936effc..5619f33f3 100644 --- a/temporalio/api/deployment/v1/message_pb2.pyi +++ b/temporalio/api/deployment/v1/message_pb2.pyi @@ -716,6 +716,7 @@ class WorkerDeploymentInfo(google.protobuf.message.Message): CREATE_TIME_FIELD_NUMBER: builtins.int ROUTING_CONFIG_FIELD_NUMBER: builtins.int LAST_MODIFIER_IDENTITY_FIELD_NUMBER: builtins.int + MANAGER_IDENTITY_FIELD_NUMBER: builtins.int name: builtins.str """Identifies a Worker Deployment. Must be unique within the namespace.""" @property @@ -739,6 +740,12 @@ class WorkerDeploymentInfo(google.protobuf.message.Message): `identity` value sent by APIs such as `SetWorkerDeploymentCurrentVersion` and `SetWorkerDeploymentRampingVersion`. """ + manager_identity: builtins.str + """Identity of the client that has the exclusive right to make changes to this Worker Deployment. + Empty by default. + If this is set, clients whose identity does not match `manager_identity` will not be able to make changes + to this Worker Deployment. They can either set their own identity as the manager or unset the field to proceed. + """ def __init__( self, *, @@ -750,6 +757,7 @@ class WorkerDeploymentInfo(google.protobuf.message.Message): create_time: google.protobuf.timestamp_pb2.Timestamp | None = ..., routing_config: global___RoutingConfig | None = ..., last_modifier_identity: builtins.str = ..., + manager_identity: builtins.str = ..., ) -> None: ... def HasField( self, @@ -764,6 +772,8 @@ class WorkerDeploymentInfo(google.protobuf.message.Message): b"create_time", "last_modifier_identity", b"last_modifier_identity", + "manager_identity", + b"manager_identity", "name", b"name", "routing_config", diff --git a/temporalio/api/enums/v1/__init__.py b/temporalio/api/enums/v1/__init__.py index 156d02e86..a84f41696 100644 --- a/temporalio/api/enums/v1/__init__.py +++ b/temporalio/api/enums/v1/__init__.py @@ -1,3 +1,4 @@ +from .activity_pb2 import ActivityExecutionStatus from .batch_operation_pb2 import BatchOperationState, BatchOperationType from .command_type_pb2 import CommandType from .common_pb2 import ( @@ -26,6 +27,7 @@ StartChildWorkflowExecutionFailedCause, WorkflowTaskFailedCause, ) +from .id_pb2 import IdConflictPolicy, IdReusePolicy from .namespace_pb2 import ArchivalState, NamespaceState, ReplicationState from .nexus_pb2 import NexusHandlerErrorRetryBehavior from .query_pb2 import QueryRejectCondition, QueryResultType @@ -55,6 +57,7 @@ ) __all__ = [ + "ActivityExecutionStatus", "ApplicationErrorCategory", "ArchivalState", "BatchOperationState", @@ -69,6 +72,8 @@ "EncodingType", "EventType", "HistoryEventFilterType", + "IdConflictPolicy", + "IdReusePolicy", "IndexedValueType", "NamespaceState", "NexusHandlerErrorRetryBehavior", diff --git a/temporalio/api/enums/v1/activity_pb2.py b/temporalio/api/enums/v1/activity_pb2.py new file mode 100644 index 000000000..23aa5aa78 --- /dev/null +++ b/temporalio/api/enums/v1/activity_pb2.py @@ -0,0 +1,38 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: temporal/api/enums/v1/activity.proto +"""Generated protocol buffer code.""" + +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import enum_type_wrapper + +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b"\n$temporal/api/enums/v1/activity.proto\x12\x15temporal.api.enums.v1*\xb5\x02\n\x17\x41\x63tivityExecutionStatus\x12)\n%ACTIVITY_EXECUTION_STATUS_UNSPECIFIED\x10\x00\x12%\n!ACTIVITY_EXECUTION_STATUS_RUNNING\x10\x01\x12'\n#ACTIVITY_EXECUTION_STATUS_COMPLETED\x10\x02\x12$\n ACTIVITY_EXECUTION_STATUS_FAILED\x10\x03\x12&\n\"ACTIVITY_EXECUTION_STATUS_CANCELED\x10\x04\x12(\n$ACTIVITY_EXECUTION_STATUS_TERMINATED\x10\x05\x12'\n#ACTIVITY_EXECUTION_STATUS_TIMED_OUT\x10\x06\x42\x85\x01\n\x18io.temporal.api.enums.v1B\rActivityProtoP\x01Z!go.temporal.io/api/enums/v1;enums\xaa\x02\x17Temporalio.Api.Enums.V1\xea\x02\x1aTemporalio::Api::Enums::V1b\x06proto3" +) + +_ACTIVITYEXECUTIONSTATUS = DESCRIPTOR.enum_types_by_name["ActivityExecutionStatus"] +ActivityExecutionStatus = enum_type_wrapper.EnumTypeWrapper(_ACTIVITYEXECUTIONSTATUS) +ACTIVITY_EXECUTION_STATUS_UNSPECIFIED = 0 +ACTIVITY_EXECUTION_STATUS_RUNNING = 1 +ACTIVITY_EXECUTION_STATUS_COMPLETED = 2 +ACTIVITY_EXECUTION_STATUS_FAILED = 3 +ACTIVITY_EXECUTION_STATUS_CANCELED = 4 +ACTIVITY_EXECUTION_STATUS_TERMINATED = 5 +ACTIVITY_EXECUTION_STATUS_TIMED_OUT = 6 + + +if _descriptor._USE_C_DESCRIPTORS == False: + DESCRIPTOR._options = None + DESCRIPTOR._serialized_options = b"\n\030io.temporal.api.enums.v1B\rActivityProtoP\001Z!go.temporal.io/api/enums/v1;enums\252\002\027Temporalio.Api.Enums.V1\352\002\032Temporalio::Api::Enums::V1" + _ACTIVITYEXECUTIONSTATUS._serialized_start = 64 + _ACTIVITYEXECUTIONSTATUS._serialized_end = 373 +# @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/enums/v1/activity_pb2.pyi b/temporalio/api/enums/v1/activity_pb2.pyi new file mode 100644 index 000000000..d4e0a9e31 --- /dev/null +++ b/temporalio/api/enums/v1/activity_pb2.pyi @@ -0,0 +1,93 @@ +""" +@generated by mypy-protobuf. Do not edit manually! +isort:skip_file +""" + +import builtins +import sys +import typing + +import google.protobuf.descriptor +import google.protobuf.internal.enum_type_wrapper + +if sys.version_info >= (3, 10): + import typing as typing_extensions +else: + import typing_extensions + +DESCRIPTOR: google.protobuf.descriptor.FileDescriptor + +class _ActivityExecutionStatus: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + +class _ActivityExecutionStatusEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + _ActivityExecutionStatus.ValueType + ], + builtins.type, +): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + ACTIVITY_EXECUTION_STATUS_UNSPECIFIED: _ActivityExecutionStatus.ValueType # 0 + ACTIVITY_EXECUTION_STATUS_RUNNING: _ActivityExecutionStatus.ValueType # 1 + """The activity is not in a terminal status. This does not necessarily mean that there is a currently running + attempt. The activity may be backing off between attempts or waiting for a worker to pick it up. + """ + ACTIVITY_EXECUTION_STATUS_COMPLETED: _ActivityExecutionStatus.ValueType # 2 + """The activity completed successfully.""" + ACTIVITY_EXECUTION_STATUS_FAILED: _ActivityExecutionStatus.ValueType # 3 + """The activity completed with failure.""" + ACTIVITY_EXECUTION_STATUS_CANCELED: _ActivityExecutionStatus.ValueType # 4 + """The activity completed as canceled. + Requesting to cancel an activity does not automatically transition the activity to canceled status. If the + activity has a currently running attempt, the activity will only transition to canceled status if the current + attempt is unsuccessful. + TODO: Clarify what happens if there are no more allowed retries after the current attempt. + """ + ACTIVITY_EXECUTION_STATUS_TERMINATED: _ActivityExecutionStatus.ValueType # 5 + """The activity was terminated. Termination does not reach the worker and the activity code cannot react to it. + A terminated activity may have a running attempt and will be requested to be canceled by the server when it + heartbeats. + """ + ACTIVITY_EXECUTION_STATUS_TIMED_OUT: _ActivityExecutionStatus.ValueType # 6 + """The activity has timed out by reaching the specified shedule-to-start or schedule-to-close timeouts. + TODO: Clarify if there are other conditions where the activity can end up in timed out status. + """ + +class ActivityExecutionStatus( + _ActivityExecutionStatus, metaclass=_ActivityExecutionStatusEnumTypeWrapper +): + """Status of a standalone activity. + The status is updated once, when the activity is originally scheduled, and again when the activity reaches a terminal + status. + TODO: Should this be a common execution status? Seems like the other archetypes will share this status. + (-- api-linter: core::0216::synonyms=disabled + aip.dev/not-precedent: Named consistently with WorkflowExecutionStatus. --) + """ + +ACTIVITY_EXECUTION_STATUS_UNSPECIFIED: ActivityExecutionStatus.ValueType # 0 +ACTIVITY_EXECUTION_STATUS_RUNNING: ActivityExecutionStatus.ValueType # 1 +"""The activity is not in a terminal status. This does not necessarily mean that there is a currently running +attempt. The activity may be backing off between attempts or waiting for a worker to pick it up. +""" +ACTIVITY_EXECUTION_STATUS_COMPLETED: ActivityExecutionStatus.ValueType # 2 +"""The activity completed successfully.""" +ACTIVITY_EXECUTION_STATUS_FAILED: ActivityExecutionStatus.ValueType # 3 +"""The activity completed with failure.""" +ACTIVITY_EXECUTION_STATUS_CANCELED: ActivityExecutionStatus.ValueType # 4 +"""The activity completed as canceled. +Requesting to cancel an activity does not automatically transition the activity to canceled status. If the +activity has a currently running attempt, the activity will only transition to canceled status if the current +attempt is unsuccessful. +TODO: Clarify what happens if there are no more allowed retries after the current attempt. +""" +ACTIVITY_EXECUTION_STATUS_TERMINATED: ActivityExecutionStatus.ValueType # 5 +"""The activity was terminated. Termination does not reach the worker and the activity code cannot react to it. +A terminated activity may have a running attempt and will be requested to be canceled by the server when it +heartbeats. +""" +ACTIVITY_EXECUTION_STATUS_TIMED_OUT: ActivityExecutionStatus.ValueType # 6 +"""The activity has timed out by reaching the specified shedule-to-start or schedule-to-close timeouts. +TODO: Clarify if there are other conditions where the activity can end up in timed out status. +""" +global___ActivityExecutionStatus = ActivityExecutionStatus diff --git a/temporalio/api/enums/v1/id_pb2.py b/temporalio/api/enums/v1/id_pb2.py new file mode 100644 index 000000000..974b3d1a9 --- /dev/null +++ b/temporalio/api/enums/v1/id_pb2.py @@ -0,0 +1,43 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: temporal/api/enums/v1/id.proto +"""Generated protocol buffer code.""" + +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import enum_type_wrapper + +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x1etemporal/api/enums/v1/id.proto\x12\x15temporal.api.enums.v1*\xac\x01\n\rIdReusePolicy\x12\x1f\n\x1bID_REUSE_POLICY_UNSPECIFIED\x10\x00\x12#\n\x1fID_REUSE_POLICY_ALLOW_DUPLICATE\x10\x01\x12/\n+ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY\x10\x02\x12$\n ID_REUSE_POLICY_REJECT_DUPLICATE\x10\x03*\xa3\x01\n\x10IdConflictPolicy\x12"\n\x1eID_CONFLICT_POLICY_UNSPECIFIED\x10\x00\x12\x1b\n\x17ID_CONFLICT_POLICY_FAIL\x10\x01\x12#\n\x1fID_CONFLICT_POLICY_USE_EXISTING\x10\x02\x12)\n%ID_CONFLICT_POLICY_TERMINATE_EXISTING\x10\x03\x42\x7f\n\x18io.temporal.api.enums.v1B\x07IdProtoP\x01Z!go.temporal.io/api/enums/v1;enums\xaa\x02\x17Temporalio.Api.Enums.V1\xea\x02\x1aTemporalio::Api::Enums::V1b\x06proto3' +) + +_IDREUSEPOLICY = DESCRIPTOR.enum_types_by_name["IdReusePolicy"] +IdReusePolicy = enum_type_wrapper.EnumTypeWrapper(_IDREUSEPOLICY) +_IDCONFLICTPOLICY = DESCRIPTOR.enum_types_by_name["IdConflictPolicy"] +IdConflictPolicy = enum_type_wrapper.EnumTypeWrapper(_IDCONFLICTPOLICY) +ID_REUSE_POLICY_UNSPECIFIED = 0 +ID_REUSE_POLICY_ALLOW_DUPLICATE = 1 +ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY = 2 +ID_REUSE_POLICY_REJECT_DUPLICATE = 3 +ID_CONFLICT_POLICY_UNSPECIFIED = 0 +ID_CONFLICT_POLICY_FAIL = 1 +ID_CONFLICT_POLICY_USE_EXISTING = 2 +ID_CONFLICT_POLICY_TERMINATE_EXISTING = 3 + + +if _descriptor._USE_C_DESCRIPTORS == False: + DESCRIPTOR._options = None + DESCRIPTOR._serialized_options = b"\n\030io.temporal.api.enums.v1B\007IdProtoP\001Z!go.temporal.io/api/enums/v1;enums\252\002\027Temporalio.Api.Enums.V1\352\002\032Temporalio::Api::Enums::V1" + _IDREUSEPOLICY._serialized_start = 58 + _IDREUSEPOLICY._serialized_end = 230 + _IDCONFLICTPOLICY._serialized_start = 233 + _IDCONFLICTPOLICY._serialized_end = 396 +# @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/enums/v1/id_pb2.pyi b/temporalio/api/enums/v1/id_pb2.pyi new file mode 100644 index 000000000..0fc8add8c --- /dev/null +++ b/temporalio/api/enums/v1/id_pb2.pyi @@ -0,0 +1,96 @@ +""" +@generated by mypy-protobuf. Do not edit manually! +isort:skip_file +""" + +import builtins +import sys +import typing + +import google.protobuf.descriptor +import google.protobuf.internal.enum_type_wrapper + +if sys.version_info >= (3, 10): + import typing as typing_extensions +else: + import typing_extensions + +DESCRIPTOR: google.protobuf.descriptor.FileDescriptor + +class _IdReusePolicy: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + +class _IdReusePolicyEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + _IdReusePolicy.ValueType + ], + builtins.type, +): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + ID_REUSE_POLICY_UNSPECIFIED: _IdReusePolicy.ValueType # 0 + ID_REUSE_POLICY_ALLOW_DUPLICATE: _IdReusePolicy.ValueType # 1 + """Always allow starting an execution using the same entity ID.""" + ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY: _IdReusePolicy.ValueType # 2 + """Allow starting an execution using the same ID, only when the last execution's final state is one of [terminated, + cancelled, timed out, failed]. + """ + ID_REUSE_POLICY_REJECT_DUPLICATE: _IdReusePolicy.ValueType # 3 + """Do not permit re-use of the ID for this execution. Future start requests could potentially change the policy, + allowing re-use of the ID. + """ + +class IdReusePolicy(_IdReusePolicy, metaclass=_IdReusePolicyEnumTypeWrapper): + """Defines whether to allow re-using an ID from a previously *closed* execution. + If the request is denied, the server returns an `ExecutionAlreadyStarted` error. + + See `IdConflictPolicy` for handling ID duplication with a *running* execution. + """ + +ID_REUSE_POLICY_UNSPECIFIED: IdReusePolicy.ValueType # 0 +ID_REUSE_POLICY_ALLOW_DUPLICATE: IdReusePolicy.ValueType # 1 +"""Always allow starting an execution using the same entity ID.""" +ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY: IdReusePolicy.ValueType # 2 +"""Allow starting an execution using the same ID, only when the last execution's final state is one of [terminated, +cancelled, timed out, failed]. +""" +ID_REUSE_POLICY_REJECT_DUPLICATE: IdReusePolicy.ValueType # 3 +"""Do not permit re-use of the ID for this execution. Future start requests could potentially change the policy, +allowing re-use of the ID. +""" +global___IdReusePolicy = IdReusePolicy + +class _IdConflictPolicy: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + +class _IdConflictPolicyEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + _IdConflictPolicy.ValueType + ], + builtins.type, +): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + ID_CONFLICT_POLICY_UNSPECIFIED: _IdConflictPolicy.ValueType # 0 + ID_CONFLICT_POLICY_FAIL: _IdConflictPolicy.ValueType # 1 + """Don't start a new execution; instead return `ExecutionAlreadyStarted` error.""" + ID_CONFLICT_POLICY_USE_EXISTING: _IdConflictPolicy.ValueType # 2 + """Don't start a new execution; instead return a handle for the running execution.""" + ID_CONFLICT_POLICY_TERMINATE_EXISTING: _IdConflictPolicy.ValueType # 3 + """Terminate the running execution before starting a new one.""" + +class IdConflictPolicy(_IdConflictPolicy, metaclass=_IdConflictPolicyEnumTypeWrapper): + """Defines what to do when trying to start an execution with the same ID as a *running* execution. + Note that it is *never* valid to have two actively running instances of the same execution ID. + + See `IdReusePolicy` for handling execution ID duplication with a *closed* execution. + """ + +ID_CONFLICT_POLICY_UNSPECIFIED: IdConflictPolicy.ValueType # 0 +ID_CONFLICT_POLICY_FAIL: IdConflictPolicy.ValueType # 1 +"""Don't start a new execution; instead return `ExecutionAlreadyStarted` error.""" +ID_CONFLICT_POLICY_USE_EXISTING: IdConflictPolicy.ValueType # 2 +"""Don't start a new execution; instead return a handle for the running execution.""" +ID_CONFLICT_POLICY_TERMINATE_EXISTING: IdConflictPolicy.ValueType # 3 +"""Terminate the running execution before starting a new one.""" +global___IdConflictPolicy = IdConflictPolicy diff --git a/temporalio/api/namespace/v1/message_pb2.py b/temporalio/api/namespace/v1/message_pb2.py index 06b0fd956..51367c8f2 100644 --- a/temporalio/api/namespace/v1/message_pb2.py +++ b/temporalio/api/namespace/v1/message_pb2.py @@ -22,7 +22,7 @@ ) DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\'temporal/api/namespace/v1/message.proto\x12\x19temporal.api.namespace.v1\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a%temporal/api/enums/v1/namespace.proto"\xba\x03\n\rNamespaceInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x34\n\x05state\x18\x02 \x01(\x0e\x32%.temporal.api.enums.v1.NamespaceState\x12\x13\n\x0b\x64\x65scription\x18\x03 \x01(\t\x12\x13\n\x0bowner_email\x18\x04 \x01(\t\x12@\n\x04\x64\x61ta\x18\x05 \x03(\x0b\x32\x32.temporal.api.namespace.v1.NamespaceInfo.DataEntry\x12\n\n\x02id\x18\x06 \x01(\t\x12K\n\x0c\x63\x61pabilities\x18\x07 \x01(\x0b\x32\x35.temporal.api.namespace.v1.NamespaceInfo.Capabilities\x12\x1a\n\x12supports_schedules\x18\x64 \x01(\x08\x1a+\n\tDataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1aW\n\x0c\x43\x61pabilities\x12\x1c\n\x14\x65\x61ger_workflow_start\x18\x01 \x01(\x08\x12\x13\n\x0bsync_update\x18\x02 \x01(\x08\x12\x14\n\x0c\x61sync_update\x18\x03 \x01(\x08"\x9e\x04\n\x0fNamespaceConfig\x12\x43\n workflow_execution_retention_ttl\x18\x01 \x01(\x0b\x32\x19.google.protobuf.Duration\x12<\n\x0c\x62\x61\x64_binaries\x18\x02 \x01(\x0b\x32&.temporal.api.namespace.v1.BadBinaries\x12\x44\n\x16history_archival_state\x18\x03 \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1c\n\x14history_archival_uri\x18\x04 \x01(\t\x12G\n\x19visibility_archival_state\x18\x05 \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1f\n\x17visibility_archival_uri\x18\x06 \x01(\t\x12u\n\x1f\x63ustom_search_attribute_aliases\x18\x07 \x03(\x0b\x32L.temporal.api.namespace.v1.NamespaceConfig.CustomSearchAttributeAliasesEntry\x1a\x43\n!CustomSearchAttributeAliasesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xb0\x01\n\x0b\x42\x61\x64\x42inaries\x12\x46\n\x08\x62inaries\x18\x01 \x03(\x0b\x32\x34.temporal.api.namespace.v1.BadBinaries.BinariesEntry\x1aY\n\rBinariesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.temporal.api.namespace.v1.BadBinaryInfo:\x02\x38\x01"b\n\rBadBinaryInfo\x12\x0e\n\x06reason\x18\x01 \x01(\t\x12\x10\n\x08operator\x18\x02 \x01(\t\x12/\n\x0b\x63reate_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xea\x01\n\x13UpdateNamespaceInfo\x12\x13\n\x0b\x64\x65scription\x18\x01 \x01(\t\x12\x13\n\x0bowner_email\x18\x02 \x01(\t\x12\x46\n\x04\x64\x61ta\x18\x03 \x03(\x0b\x32\x38.temporal.api.namespace.v1.UpdateNamespaceInfo.DataEntry\x12\x34\n\x05state\x18\x04 \x01(\x0e\x32%.temporal.api.enums.v1.NamespaceState\x1a+\n\tDataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"*\n\x0fNamespaceFilter\x12\x17\n\x0finclude_deleted\x18\x01 \x01(\x08\x42\x98\x01\n\x1cio.temporal.api.namespace.v1B\x0cMessageProtoP\x01Z)go.temporal.io/api/namespace/v1;namespace\xaa\x02\x1bTemporalio.Api.Namespace.V1\xea\x02\x1eTemporalio::Api::Namespace::V1b\x06proto3' + b'\n\'temporal/api/namespace/v1/message.proto\x12\x19temporal.api.namespace.v1\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a%temporal/api/enums/v1/namespace.proto"\x82\x04\n\rNamespaceInfo\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x34\n\x05state\x18\x02 \x01(\x0e\x32%.temporal.api.enums.v1.NamespaceState\x12\x13\n\x0b\x64\x65scription\x18\x03 \x01(\t\x12\x13\n\x0bowner_email\x18\x04 \x01(\t\x12@\n\x04\x64\x61ta\x18\x05 \x03(\x0b\x32\x32.temporal.api.namespace.v1.NamespaceInfo.DataEntry\x12\n\n\x02id\x18\x06 \x01(\t\x12K\n\x0c\x63\x61pabilities\x18\x07 \x01(\x0b\x32\x35.temporal.api.namespace.v1.NamespaceInfo.Capabilities\x12\x1a\n\x12supports_schedules\x18\x64 \x01(\x08\x1a+\n\tDataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\x9e\x01\n\x0c\x43\x61pabilities\x12\x1c\n\x14\x65\x61ger_workflow_start\x18\x01 \x01(\x08\x12\x13\n\x0bsync_update\x18\x02 \x01(\x08\x12\x14\n\x0c\x61sync_update\x18\x03 \x01(\x08\x12\x19\n\x11worker_heartbeats\x18\x04 \x01(\x08\x12*\n"reported_problems_search_attribute\x18\x05 \x01(\x08"\x9e\x04\n\x0fNamespaceConfig\x12\x43\n workflow_execution_retention_ttl\x18\x01 \x01(\x0b\x32\x19.google.protobuf.Duration\x12<\n\x0c\x62\x61\x64_binaries\x18\x02 \x01(\x0b\x32&.temporal.api.namespace.v1.BadBinaries\x12\x44\n\x16history_archival_state\x18\x03 \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1c\n\x14history_archival_uri\x18\x04 \x01(\t\x12G\n\x19visibility_archival_state\x18\x05 \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1f\n\x17visibility_archival_uri\x18\x06 \x01(\t\x12u\n\x1f\x63ustom_search_attribute_aliases\x18\x07 \x03(\x0b\x32L.temporal.api.namespace.v1.NamespaceConfig.CustomSearchAttributeAliasesEntry\x1a\x43\n!CustomSearchAttributeAliasesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xb0\x01\n\x0b\x42\x61\x64\x42inaries\x12\x46\n\x08\x62inaries\x18\x01 \x03(\x0b\x32\x34.temporal.api.namespace.v1.BadBinaries.BinariesEntry\x1aY\n\rBinariesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.temporal.api.namespace.v1.BadBinaryInfo:\x02\x38\x01"b\n\rBadBinaryInfo\x12\x0e\n\x06reason\x18\x01 \x01(\t\x12\x10\n\x08operator\x18\x02 \x01(\t\x12/\n\x0b\x63reate_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xea\x01\n\x13UpdateNamespaceInfo\x12\x13\n\x0b\x64\x65scription\x18\x01 \x01(\t\x12\x13\n\x0bowner_email\x18\x02 \x01(\t\x12\x46\n\x04\x64\x61ta\x18\x03 \x03(\x0b\x32\x38.temporal.api.namespace.v1.UpdateNamespaceInfo.DataEntry\x12\x34\n\x05state\x18\x04 \x01(\x0e\x32%.temporal.api.enums.v1.NamespaceState\x1a+\n\tDataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"*\n\x0fNamespaceFilter\x12\x17\n\x0finclude_deleted\x18\x01 \x01(\x08\x42\x98\x01\n\x1cio.temporal.api.namespace.v1B\x0cMessageProtoP\x01Z)go.temporal.io/api/namespace/v1;namespace\xaa\x02\x1bTemporalio.Api.Namespace.V1\xea\x02\x1eTemporalio::Api::Namespace::V1b\x06proto3' ) @@ -167,25 +167,25 @@ _UPDATENAMESPACEINFO_DATAENTRY._options = None _UPDATENAMESPACEINFO_DATAENTRY._serialized_options = b"8\001" _NAMESPACEINFO._serialized_start = 175 - _NAMESPACEINFO._serialized_end = 617 + _NAMESPACEINFO._serialized_end = 689 _NAMESPACEINFO_DATAENTRY._serialized_start = 485 _NAMESPACEINFO_DATAENTRY._serialized_end = 528 - _NAMESPACEINFO_CAPABILITIES._serialized_start = 530 - _NAMESPACEINFO_CAPABILITIES._serialized_end = 617 - _NAMESPACECONFIG._serialized_start = 620 - _NAMESPACECONFIG._serialized_end = 1162 - _NAMESPACECONFIG_CUSTOMSEARCHATTRIBUTEALIASESENTRY._serialized_start = 1095 - _NAMESPACECONFIG_CUSTOMSEARCHATTRIBUTEALIASESENTRY._serialized_end = 1162 - _BADBINARIES._serialized_start = 1165 - _BADBINARIES._serialized_end = 1341 - _BADBINARIES_BINARIESENTRY._serialized_start = 1252 - _BADBINARIES_BINARIESENTRY._serialized_end = 1341 - _BADBINARYINFO._serialized_start = 1343 - _BADBINARYINFO._serialized_end = 1441 - _UPDATENAMESPACEINFO._serialized_start = 1444 - _UPDATENAMESPACEINFO._serialized_end = 1678 + _NAMESPACEINFO_CAPABILITIES._serialized_start = 531 + _NAMESPACEINFO_CAPABILITIES._serialized_end = 689 + _NAMESPACECONFIG._serialized_start = 692 + _NAMESPACECONFIG._serialized_end = 1234 + _NAMESPACECONFIG_CUSTOMSEARCHATTRIBUTEALIASESENTRY._serialized_start = 1167 + _NAMESPACECONFIG_CUSTOMSEARCHATTRIBUTEALIASESENTRY._serialized_end = 1234 + _BADBINARIES._serialized_start = 1237 + _BADBINARIES._serialized_end = 1413 + _BADBINARIES_BINARIESENTRY._serialized_start = 1324 + _BADBINARIES_BINARIESENTRY._serialized_end = 1413 + _BADBINARYINFO._serialized_start = 1415 + _BADBINARYINFO._serialized_end = 1513 + _UPDATENAMESPACEINFO._serialized_start = 1516 + _UPDATENAMESPACEINFO._serialized_end = 1750 _UPDATENAMESPACEINFO_DATAENTRY._serialized_start = 485 _UPDATENAMESPACEINFO_DATAENTRY._serialized_end = 528 - _NAMESPACEFILTER._serialized_start = 1680 - _NAMESPACEFILTER._serialized_end = 1722 + _NAMESPACEFILTER._serialized_start = 1752 + _NAMESPACEFILTER._serialized_end = 1794 # @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/namespace/v1/message_pb2.pyi b/temporalio/api/namespace/v1/message_pb2.pyi index 831972b79..14f23f5ed 100644 --- a/temporalio/api/namespace/v1/message_pb2.pyi +++ b/temporalio/api/namespace/v1/message_pb2.pyi @@ -51,18 +51,26 @@ class NamespaceInfo(google.protobuf.message.Message): EAGER_WORKFLOW_START_FIELD_NUMBER: builtins.int SYNC_UPDATE_FIELD_NUMBER: builtins.int ASYNC_UPDATE_FIELD_NUMBER: builtins.int + WORKER_HEARTBEATS_FIELD_NUMBER: builtins.int + REPORTED_PROBLEMS_SEARCH_ATTRIBUTE_FIELD_NUMBER: builtins.int eager_workflow_start: builtins.bool """True if the namespace supports eager workflow start.""" sync_update: builtins.bool """True if the namespace supports sync update""" async_update: builtins.bool """True if the namespace supports async update""" + worker_heartbeats: builtins.bool + """True if the namespace supports worker heartbeats""" + reported_problems_search_attribute: builtins.bool + """True if the namespace supports reported problems search attribute""" def __init__( self, *, eager_workflow_start: builtins.bool = ..., sync_update: builtins.bool = ..., async_update: builtins.bool = ..., + worker_heartbeats: builtins.bool = ..., + reported_problems_search_attribute: builtins.bool = ..., ) -> None: ... def ClearField( self, @@ -71,8 +79,12 @@ class NamespaceInfo(google.protobuf.message.Message): b"async_update", "eager_workflow_start", b"eager_workflow_start", + "reported_problems_search_attribute", + b"reported_problems_search_attribute", "sync_update", b"sync_update", + "worker_heartbeats", + b"worker_heartbeats", ], ) -> None: ... diff --git a/temporalio/api/workflowservice/v1/__init__.py b/temporalio/api/workflowservice/v1/__init__.py index 8f82668cb..ba0eb8b75 100644 --- a/temporalio/api/workflowservice/v1/__init__.py +++ b/temporalio/api/workflowservice/v1/__init__.py @@ -1,10 +1,14 @@ from .request_response_pb2 import ( + CountActivityExecutionsRequest, + CountActivityExecutionsResponse, CountWorkflowExecutionsRequest, CountWorkflowExecutionsResponse, CreateScheduleRequest, CreateScheduleResponse, CreateWorkflowRuleRequest, CreateWorkflowRuleResponse, + DeleteActivityExecutionRequest, + DeleteActivityExecutionResponse, DeleteScheduleRequest, DeleteScheduleResponse, DeleteWorkerDeploymentRequest, @@ -17,6 +21,8 @@ DeleteWorkflowRuleResponse, DeprecateNamespaceRequest, DeprecateNamespaceResponse, + DescribeActivityExecutionRequest, + DescribeActivityExecutionResponse, DescribeBatchOperationRequest, DescribeBatchOperationResponse, DescribeDeploymentRequest, @@ -31,6 +37,8 @@ DescribeWorkerDeploymentResponse, DescribeWorkerDeploymentVersionRequest, DescribeWorkerDeploymentVersionResponse, + DescribeWorkerRequest, + DescribeWorkerResponse, DescribeWorkflowExecutionRequest, DescribeWorkflowExecutionResponse, DescribeWorkflowRuleRequest, @@ -39,6 +47,8 @@ ExecuteMultiOperationResponse, FetchWorkerConfigRequest, FetchWorkerConfigResponse, + GetActivityExecutionResultRequest, + GetActivityExecutionResultResponse, GetClusterInfoRequest, GetClusterInfoResponse, GetCurrentDeploymentRequest, @@ -59,6 +69,8 @@ GetWorkflowExecutionHistoryResponse, GetWorkflowExecutionHistoryReverseRequest, GetWorkflowExecutionHistoryReverseResponse, + ListActivityExecutionsRequest, + ListActivityExecutionsResponse, ListArchivedWorkflowExecutionsRequest, ListArchivedWorkflowExecutionsResponse, ListBatchOperationsRequest, @@ -87,6 +99,8 @@ ListWorkflowRulesResponse, PatchScheduleRequest, PatchScheduleResponse, + PauseActivityExecutionRequest, + PauseActivityExecutionResponse, PauseActivityRequest, PauseActivityResponse, PollActivityTaskQueueRequest, @@ -107,8 +121,12 @@ RecordWorkerHeartbeatResponse, RegisterNamespaceRequest, RegisterNamespaceResponse, + RequestCancelActivityExecutionRequest, + RequestCancelActivityExecutionResponse, RequestCancelWorkflowExecutionRequest, RequestCancelWorkflowExecutionResponse, + ResetActivityExecutionRequest, + ResetActivityExecutionResponse, ResetActivityRequest, ResetActivityResponse, ResetStickyTaskQueueRequest, @@ -143,6 +161,8 @@ SetCurrentDeploymentResponse, SetWorkerDeploymentCurrentVersionRequest, SetWorkerDeploymentCurrentVersionResponse, + SetWorkerDeploymentManagerRequest, + SetWorkerDeploymentManagerResponse, SetWorkerDeploymentRampingVersionRequest, SetWorkerDeploymentRampingVersionResponse, ShutdownWorkerRequest, @@ -151,18 +171,26 @@ SignalWithStartWorkflowExecutionResponse, SignalWorkflowExecutionRequest, SignalWorkflowExecutionResponse, + StartActivityExecutionRequest, + StartActivityExecutionResponse, StartBatchOperationRequest, StartBatchOperationResponse, StartWorkflowExecutionRequest, StartWorkflowExecutionResponse, StopBatchOperationRequest, StopBatchOperationResponse, + TerminateActivityExecutionRequest, + TerminateActivityExecutionResponse, TerminateWorkflowExecutionRequest, TerminateWorkflowExecutionResponse, TriggerWorkflowRuleRequest, TriggerWorkflowRuleResponse, + UnpauseActivityExecutionRequest, + UnpauseActivityExecutionResponse, UnpauseActivityRequest, UnpauseActivityResponse, + UpdateActivityExecutionOptionsRequest, + UpdateActivityExecutionOptionsResponse, UpdateActivityOptionsRequest, UpdateActivityOptionsResponse, UpdateNamespaceRequest, @@ -186,12 +214,16 @@ ) __all__ = [ + "CountActivityExecutionsRequest", + "CountActivityExecutionsResponse", "CountWorkflowExecutionsRequest", "CountWorkflowExecutionsResponse", "CreateScheduleRequest", "CreateScheduleResponse", "CreateWorkflowRuleRequest", "CreateWorkflowRuleResponse", + "DeleteActivityExecutionRequest", + "DeleteActivityExecutionResponse", "DeleteScheduleRequest", "DeleteScheduleResponse", "DeleteWorkerDeploymentRequest", @@ -204,6 +236,8 @@ "DeleteWorkflowRuleResponse", "DeprecateNamespaceRequest", "DeprecateNamespaceResponse", + "DescribeActivityExecutionRequest", + "DescribeActivityExecutionResponse", "DescribeBatchOperationRequest", "DescribeBatchOperationResponse", "DescribeDeploymentRequest", @@ -218,6 +252,8 @@ "DescribeWorkerDeploymentResponse", "DescribeWorkerDeploymentVersionRequest", "DescribeWorkerDeploymentVersionResponse", + "DescribeWorkerRequest", + "DescribeWorkerResponse", "DescribeWorkflowExecutionRequest", "DescribeWorkflowExecutionResponse", "DescribeWorkflowRuleRequest", @@ -226,6 +262,8 @@ "ExecuteMultiOperationResponse", "FetchWorkerConfigRequest", "FetchWorkerConfigResponse", + "GetActivityExecutionResultRequest", + "GetActivityExecutionResultResponse", "GetClusterInfoRequest", "GetClusterInfoResponse", "GetCurrentDeploymentRequest", @@ -246,6 +284,8 @@ "GetWorkflowExecutionHistoryResponse", "GetWorkflowExecutionHistoryReverseRequest", "GetWorkflowExecutionHistoryReverseResponse", + "ListActivityExecutionsRequest", + "ListActivityExecutionsResponse", "ListArchivedWorkflowExecutionsRequest", "ListArchivedWorkflowExecutionsResponse", "ListBatchOperationsRequest", @@ -274,6 +314,8 @@ "ListWorkflowRulesResponse", "PatchScheduleRequest", "PatchScheduleResponse", + "PauseActivityExecutionRequest", + "PauseActivityExecutionResponse", "PauseActivityRequest", "PauseActivityResponse", "PollActivityTaskQueueRequest", @@ -294,8 +336,12 @@ "RecordWorkerHeartbeatResponse", "RegisterNamespaceRequest", "RegisterNamespaceResponse", + "RequestCancelActivityExecutionRequest", + "RequestCancelActivityExecutionResponse", "RequestCancelWorkflowExecutionRequest", "RequestCancelWorkflowExecutionResponse", + "ResetActivityExecutionRequest", + "ResetActivityExecutionResponse", "ResetActivityRequest", "ResetActivityResponse", "ResetStickyTaskQueueRequest", @@ -330,6 +376,8 @@ "SetCurrentDeploymentResponse", "SetWorkerDeploymentCurrentVersionRequest", "SetWorkerDeploymentCurrentVersionResponse", + "SetWorkerDeploymentManagerRequest", + "SetWorkerDeploymentManagerResponse", "SetWorkerDeploymentRampingVersionRequest", "SetWorkerDeploymentRampingVersionResponse", "ShutdownWorkerRequest", @@ -338,18 +386,26 @@ "SignalWithStartWorkflowExecutionResponse", "SignalWorkflowExecutionRequest", "SignalWorkflowExecutionResponse", + "StartActivityExecutionRequest", + "StartActivityExecutionResponse", "StartBatchOperationRequest", "StartBatchOperationResponse", "StartWorkflowExecutionRequest", "StartWorkflowExecutionResponse", "StopBatchOperationRequest", "StopBatchOperationResponse", + "TerminateActivityExecutionRequest", + "TerminateActivityExecutionResponse", "TerminateWorkflowExecutionRequest", "TerminateWorkflowExecutionResponse", "TriggerWorkflowRuleRequest", "TriggerWorkflowRuleResponse", + "UnpauseActivityExecutionRequest", + "UnpauseActivityExecutionResponse", "UnpauseActivityRequest", "UnpauseActivityResponse", + "UpdateActivityExecutionOptionsRequest", + "UpdateActivityExecutionOptionsResponse", "UpdateActivityOptionsRequest", "UpdateActivityOptionsResponse", "UpdateNamespaceRequest", diff --git a/temporalio/api/workflowservice/v1/request_response_pb2.py b/temporalio/api/workflowservice/v1/request_response_pb2.py index 5fd33bb8f..02bf5c008 100644 --- a/temporalio/api/workflowservice/v1/request_response_pb2.py +++ b/temporalio/api/workflowservice/v1/request_response_pb2.py @@ -45,6 +45,9 @@ from temporalio.api.enums.v1 import ( failed_cause_pb2 as temporal_dot_api_dot_enums_dot_v1_dot_failed__cause__pb2, ) +from temporalio.api.enums.v1 import ( + id_pb2 as temporal_dot_api_dot_enums_dot_v1_dot_id__pb2, +) from temporalio.api.enums.v1 import ( namespace_pb2 as temporal_dot_api_dot_enums_dot_v1_dot_namespace__pb2, ) @@ -119,7 +122,7 @@ ) DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n6temporal/api/workflowservice/v1/request_response.proto\x12\x1ftemporal.api.workflowservice.v1\x1a+temporal/api/enums/v1/batch_operation.proto\x1a"temporal/api/enums/v1/common.proto\x1a$temporal/api/enums/v1/workflow.proto\x1a%temporal/api/enums/v1/namespace.proto\x1a(temporal/api/enums/v1/failed_cause.proto\x1a!temporal/api/enums/v1/query.proto\x1a!temporal/api/enums/v1/reset.proto\x1a&temporal/api/enums/v1/task_queue.proto\x1a&temporal/api/enums/v1/deployment.proto\x1a"temporal/api/enums/v1/update.proto\x1a&temporal/api/activity/v1/message.proto\x1a$temporal/api/common/v1/message.proto\x1a%temporal/api/history/v1/message.proto\x1a&temporal/api/workflow/v1/message.proto\x1a%temporal/api/command/v1/message.proto\x1a(temporal/api/deployment/v1/message.proto\x1a%temporal/api/failure/v1/message.proto\x1a$temporal/api/filter/v1/message.proto\x1a&temporal/api/protocol/v1/message.proto\x1a\'temporal/api/namespace/v1/message.proto\x1a#temporal/api/query/v1/message.proto\x1a)temporal/api/replication/v1/message.proto\x1a#temporal/api/rules/v1/message.proto\x1a\'temporal/api/sdk/v1/worker_config.proto\x1a&temporal/api/schedule/v1/message.proto\x1a\'temporal/api/taskqueue/v1/message.proto\x1a$temporal/api/update/v1/message.proto\x1a%temporal/api/version/v1/message.proto\x1a#temporal/api/batch/v1/message.proto\x1a\x30temporal/api/sdk/v1/task_complete_metadata.proto\x1a\'temporal/api/sdk/v1/user_metadata.proto\x1a#temporal/api/nexus/v1/message.proto\x1a$temporal/api/worker/v1/message.proto\x1a\x1egoogle/protobuf/duration.proto\x1a google/protobuf/field_mask.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x88\x05\n\x18RegisterNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x13\n\x0bowner_email\x18\x03 \x01(\t\x12\x46\n#workflow_execution_retention_period\x18\x04 \x01(\x0b\x32\x19.google.protobuf.Duration\x12G\n\x08\x63lusters\x18\x05 \x03(\x0b\x32\x35.temporal.api.replication.v1.ClusterReplicationConfig\x12\x1b\n\x13\x61\x63tive_cluster_name\x18\x06 \x01(\t\x12Q\n\x04\x64\x61ta\x18\x07 \x03(\x0b\x32\x43.temporal.api.workflowservice.v1.RegisterNamespaceRequest.DataEntry\x12\x16\n\x0esecurity_token\x18\x08 \x01(\t\x12\x1b\n\x13is_global_namespace\x18\t \x01(\x08\x12\x44\n\x16history_archival_state\x18\n \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1c\n\x14history_archival_uri\x18\x0b \x01(\t\x12G\n\x19visibility_archival_state\x18\x0c \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1f\n\x17visibility_archival_uri\x18\r \x01(\t\x1a+\n\tDataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\x1b\n\x19RegisterNamespaceResponse"\x89\x01\n\x15ListNamespacesRequest\x12\x11\n\tpage_size\x18\x01 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c\x12\x44\n\x10namespace_filter\x18\x03 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceFilter"\x81\x01\n\x16ListNamespacesResponse\x12N\n\nnamespaces\x18\x01 \x03(\x0b\x32:.temporal.api.workflowservice.v1.DescribeNamespaceResponse\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"9\n\x18\x44\x65scribeNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\n\n\x02id\x18\x02 \x01(\t"\xec\x02\n\x19\x44\x65scribeNamespaceResponse\x12@\n\x0enamespace_info\x18\x01 \x01(\x0b\x32(.temporal.api.namespace.v1.NamespaceInfo\x12:\n\x06\x63onfig\x18\x02 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceConfig\x12S\n\x12replication_config\x18\x03 \x01(\x0b\x32\x37.temporal.api.replication.v1.NamespaceReplicationConfig\x12\x18\n\x10\x66\x61ilover_version\x18\x04 \x01(\x03\x12\x1b\n\x13is_global_namespace\x18\x05 \x01(\x08\x12\x45\n\x10\x66\x61ilover_history\x18\x06 \x03(\x0b\x32+.temporal.api.replication.v1.FailoverStatus"\xcf\x02\n\x16UpdateNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x43\n\x0bupdate_info\x18\x02 \x01(\x0b\x32..temporal.api.namespace.v1.UpdateNamespaceInfo\x12:\n\x06\x63onfig\x18\x03 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceConfig\x12S\n\x12replication_config\x18\x04 \x01(\x0b\x32\x37.temporal.api.replication.v1.NamespaceReplicationConfig\x12\x16\n\x0esecurity_token\x18\x05 \x01(\t\x12\x19\n\x11\x64\x65lete_bad_binary\x18\x06 \x01(\t\x12\x19\n\x11promote_namespace\x18\x07 \x01(\x08"\xa3\x02\n\x17UpdateNamespaceResponse\x12@\n\x0enamespace_info\x18\x01 \x01(\x0b\x32(.temporal.api.namespace.v1.NamespaceInfo\x12:\n\x06\x63onfig\x18\x02 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceConfig\x12S\n\x12replication_config\x18\x03 \x01(\x0b\x32\x37.temporal.api.replication.v1.NamespaceReplicationConfig\x12\x18\n\x10\x66\x61ilover_version\x18\x04 \x01(\x03\x12\x1b\n\x13is_global_namespace\x18\x05 \x01(\x08"F\n\x19\x44\x65precateNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x16\n\x0esecurity_token\x18\x02 \x01(\t"\x1c\n\x1a\x44\x65precateNamespaceResponse"\xa9\x0b\n\x1dStartWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12\x38\n\ntask_queue\x18\x04 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12/\n\x05input\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12=\n\x1aworkflow_execution_timeout\x18\x06 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x37\n\x14workflow_run_timeout\x18\x07 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x38\n\x15workflow_task_timeout\x18\x08 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x10\n\x08identity\x18\t \x01(\t\x12\x12\n\nrequest_id\x18\n \x01(\t\x12N\n\x18workflow_id_reuse_policy\x18\x0b \x01(\x0e\x32,.temporal.api.enums.v1.WorkflowIdReusePolicy\x12T\n\x1bworkflow_id_conflict_policy\x18\x16 \x01(\x0e\x32/.temporal.api.enums.v1.WorkflowIdConflictPolicy\x12\x39\n\x0cretry_policy\x18\x0c \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy\x12\x15\n\rcron_schedule\x18\r \x01(\t\x12*\n\x04memo\x18\x0e \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x0f \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12.\n\x06header\x18\x10 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12\x1f\n\x17request_eager_execution\x18\x11 \x01(\x08\x12;\n\x11\x63ontinued_failure\x18\x12 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12@\n\x16last_completion_result\x18\x13 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x37\n\x14workflow_start_delay\x18\x14 \x01(\x0b\x32\x19.google.protobuf.Duration\x12>\n\x14\x63ompletion_callbacks\x18\x15 \x03(\x0b\x32 .temporal.api.common.v1.Callback\x12\x38\n\ruser_metadata\x18\x17 \x01(\x0b\x32!.temporal.api.sdk.v1.UserMetadata\x12+\n\x05links\x18\x18 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x12I\n\x13versioning_override\x18\x19 \x01(\x0b\x32,.temporal.api.workflow.v1.VersioningOverride\x12H\n\x13on_conflict_options\x18\x1a \x01(\x0b\x32+.temporal.api.workflow.v1.OnConflictOptions\x12\x32\n\x08priority\x18\x1b \x01(\x0b\x32 .temporal.api.common.v1.Priority"\x8a\x02\n\x1eStartWorkflowExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t\x12\x0f\n\x07started\x18\x03 \x01(\x08\x12>\n\x06status\x18\x05 \x01(\x0e\x32..temporal.api.enums.v1.WorkflowExecutionStatus\x12[\n\x13\x65\x61ger_workflow_task\x18\x02 \x01(\x0b\x32>.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse\x12*\n\x04link\x18\x04 \x01(\x0b\x32\x1c.temporal.api.common.v1.Link"\xaa\x02\n"GetWorkflowExecutionHistoryRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x19\n\x11maximum_page_size\x18\x03 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x04 \x01(\x0c\x12\x16\n\x0ewait_new_event\x18\x05 \x01(\x08\x12P\n\x19history_event_filter_type\x18\x06 \x01(\x0e\x32-.temporal.api.enums.v1.HistoryEventFilterType\x12\x15\n\rskip_archival\x18\x07 \x01(\x08"\xba\x01\n#GetWorkflowExecutionHistoryResponse\x12\x31\n\x07history\x18\x01 \x01(\x0b\x32 .temporal.api.history.v1.History\x12\x35\n\x0braw_history\x18\x02 \x03(\x0b\x32 .temporal.api.common.v1.DataBlob\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x10\n\x08\x61rchived\x18\x04 \x01(\x08"\xb0\x01\n)GetWorkflowExecutionHistoryReverseRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x19\n\x11maximum_page_size\x18\x03 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x04 \x01(\x0c"x\n*GetWorkflowExecutionHistoryReverseResponse\x12\x31\n\x07history\x18\x01 \x01(\x0b\x32 .temporal.api.history.v1.History\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c"\x8a\x03\n\x1cPollWorkflowTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x1b\n\x0f\x62inary_checksum\x18\x04 \x01(\tB\x02\x18\x01\x12Z\n\x1bworker_version_capabilities\x18\x05 \x01(\x0b\x32\x31.temporal.api.common.v1.WorkerVersionCapabilitiesB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x06 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x12\x41\n\x10worker_heartbeat\x18\x07 \x01(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\x91\x07\n\x1dPollWorkflowTaskQueueResponse\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12!\n\x19previous_started_event_id\x18\x04 \x01(\x03\x12\x18\n\x10started_event_id\x18\x05 \x01(\x03\x12\x0f\n\x07\x61ttempt\x18\x06 \x01(\x05\x12\x1a\n\x12\x62\x61\x63klog_count_hint\x18\x07 \x01(\x03\x12\x31\n\x07history\x18\x08 \x01(\x0b\x32 .temporal.api.history.v1.History\x12\x17\n\x0fnext_page_token\x18\t \x01(\x0c\x12\x33\n\x05query\x18\n \x01(\x0b\x32$.temporal.api.query.v1.WorkflowQuery\x12K\n\x1dworkflow_execution_task_queue\x18\x0b \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12\x32\n\x0escheduled_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x30\n\x0cstarted_time\x18\r \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\\\n\x07queries\x18\x0e \x03(\x0b\x32K.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.QueriesEntry\x12\x33\n\x08messages\x18\x0f \x03(\x0b\x32!.temporal.api.protocol.v1.Message\x12Q\n\x17poller_scaling_decision\x18\x10 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.PollerScalingDecision\x1aT\n\x0cQueriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x33\n\x05value\x18\x02 \x01(\x0b\x32$.temporal.api.query.v1.WorkflowQuery:\x02\x38\x01"\xb5\t\n#RespondWorkflowTaskCompletedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x32\n\x08\x63ommands\x18\x02 \x03(\x0b\x32 .temporal.api.command.v1.Command\x12\x10\n\x08identity\x18\x03 \x01(\t\x12O\n\x11sticky_attributes\x18\x04 \x01(\x0b\x32\x34.temporal.api.taskqueue.v1.StickyExecutionAttributes\x12 \n\x18return_new_workflow_task\x18\x05 \x01(\x08\x12&\n\x1e\x66orce_create_new_workflow_task\x18\x06 \x01(\x08\x12\x1b\n\x0f\x62inary_checksum\x18\x07 \x01(\tB\x02\x18\x01\x12m\n\rquery_results\x18\x08 \x03(\x0b\x32V.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.QueryResultsEntry\x12\x11\n\tnamespace\x18\t \x01(\t\x12L\n\x14worker_version_stamp\x18\n \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12\x33\n\x08messages\x18\x0b \x03(\x0b\x32!.temporal.api.protocol.v1.Message\x12H\n\x0csdk_metadata\x18\x0c \x01(\x0b\x32\x32.temporal.api.sdk.v1.WorkflowTaskCompletedMetadata\x12\x43\n\x11metering_metadata\x18\r \x01(\x0b\x32(.temporal.api.common.v1.MeteringMetadata\x12g\n\x0c\x63\x61pabilities\x18\x0e \x01(\x0b\x32Q.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.Capabilities\x12>\n\ndeployment\x18\x0f \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12\x46\n\x13versioning_behavior\x18\x10 \x01(\x0e\x32).temporal.api.enums.v1.VersioningBehavior\x12O\n\x12\x64\x65ployment_options\x18\x11 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x1a_\n\x11QueryResultsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.temporal.api.query.v1.WorkflowQueryResult:\x02\x38\x01\x1a\x45\n\x0c\x43\x61pabilities\x12\x35\n-discard_speculative_workflow_task_with_events\x18\x01 \x01(\x08"\xf5\x01\n$RespondWorkflowTaskCompletedResponse\x12U\n\rworkflow_task\x18\x01 \x01(\x0b\x32>.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse\x12V\n\x0e\x61\x63tivity_tasks\x18\x02 \x03(\x0b\x32>.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse\x12\x1e\n\x16reset_history_event_id\x18\x03 \x01(\x03"\xf8\x03\n RespondWorkflowTaskFailedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12=\n\x05\x63\x61use\x18\x02 \x01(\x0e\x32..temporal.api.enums.v1.WorkflowTaskFailedCause\x12\x31\n\x07\x66\x61ilure\x18\x03 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x10\n\x08identity\x18\x04 \x01(\t\x12\x1b\n\x0f\x62inary_checksum\x18\x05 \x01(\tB\x02\x18\x01\x12\x11\n\tnamespace\x18\x06 \x01(\t\x12\x33\n\x08messages\x18\x07 \x03(\x0b\x32!.temporal.api.protocol.v1.Message\x12\x46\n\x0eworker_version\x18\x08 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\t \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\n \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"#\n!RespondWorkflowTaskFailedResponse"\xb8\x03\n\x1cPollActivityTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12\x10\n\x08identity\x18\x03 \x01(\t\x12I\n\x13task_queue_metadata\x18\x04 \x01(\x0b\x32,.temporal.api.taskqueue.v1.TaskQueueMetadata\x12Z\n\x1bworker_version_capabilities\x18\x05 \x01(\x0b\x32\x31.temporal.api.common.v1.WorkerVersionCapabilitiesB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x06 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x12\x41\n\x10worker_heartbeat\x18\x07 \x01(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\xef\x07\n\x1dPollActivityTaskQueueResponse\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x1a\n\x12workflow_namespace\x18\x02 \x01(\t\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12\x45\n\x12workflow_execution\x18\x04 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12;\n\ractivity_type\x18\x05 \x01(\x0b\x32$.temporal.api.common.v1.ActivityType\x12\x13\n\x0b\x61\x63tivity_id\x18\x06 \x01(\t\x12.\n\x06header\x18\x07 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12/\n\x05input\x18\x08 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12;\n\x11heartbeat_details\x18\t \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x32\n\x0escheduled_time\x18\n \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x42\n\x1e\x63urrent_attempt_scheduled_time\x18\x0b \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x30\n\x0cstarted_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07\x61ttempt\x18\r \x01(\x05\x12<\n\x19schedule_to_close_timeout\x18\x0e \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x16start_to_close_timeout\x18\x0f \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x34\n\x11heartbeat_timeout\x18\x10 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x0cretry_policy\x18\x11 \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy\x12Q\n\x17poller_scaling_decision\x18\x12 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.PollerScalingDecision\x12\x32\n\x08priority\x18\x13 \x01(\x0b\x32 .temporal.api.common.v1.Priority"\x90\x01\n"RecordActivityTaskHeartbeatRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x31\n\x07\x64\x65tails\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t"p\n#RecordActivityTaskHeartbeatResponse\x12\x18\n\x10\x63\x61ncel_requested\x18\x01 \x01(\x08\x12\x17\n\x0f\x61\x63tivity_paused\x18\x02 \x01(\x08\x12\x16\n\x0e\x61\x63tivity_reset\x18\x03 \x01(\x08"\xba\x01\n&RecordActivityTaskHeartbeatByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x31\n\x07\x64\x65tails\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x06 \x01(\t"t\n\'RecordActivityTaskHeartbeatByIdResponse\x12\x18\n\x10\x63\x61ncel_requested\x18\x01 \x01(\x08\x12\x17\n\x0f\x61\x63tivity_paused\x18\x02 \x01(\x08\x12\x16\n\x0e\x61\x63tivity_reset\x18\x03 \x01(\x08"\xe9\x02\n#RespondActivityTaskCompletedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x30\n\x06result\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t\x12\x46\n\x0eworker_version\x18\x05 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\x06 \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"&\n$RespondActivityTaskCompletedResponse"\xba\x01\n\'RespondActivityTaskCompletedByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x30\n\x06result\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x06 \x01(\t"*\n(RespondActivityTaskCompletedByIdResponse"\xa9\x03\n RespondActivityTaskFailedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x31\n\x07\x66\x61ilure\x18\x02 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t\x12@\n\x16last_heartbeat_details\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x46\n\x0eworker_version\x18\x06 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\x07 \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x08 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"W\n!RespondActivityTaskFailedResponse\x12\x32\n\x08\x66\x61ilures\x18\x01 \x03(\x0b\x32 .temporal.api.failure.v1.Failure"\xfa\x01\n$RespondActivityTaskFailedByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x31\n\x07\x66\x61ilure\x18\x05 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x10\n\x08identity\x18\x06 \x01(\t\x12@\n\x16last_heartbeat_details\x18\x07 \x01(\x0b\x32 .temporal.api.common.v1.Payloads"[\n%RespondActivityTaskFailedByIdResponse\x12\x32\n\x08\x66\x61ilures\x18\x01 \x03(\x0b\x32 .temporal.api.failure.v1.Failure"\xe9\x02\n"RespondActivityTaskCanceledRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x31\n\x07\x64\x65tails\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t\x12\x46\n\x0eworker_version\x18\x05 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\x06 \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"%\n#RespondActivityTaskCanceledResponse"\x8b\x02\n&RespondActivityTaskCanceledByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x31\n\x07\x64\x65tails\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x06 \x01(\t\x12O\n\x12\x64\x65ployment_options\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions")\n\'RespondActivityTaskCanceledByIdResponse"\x84\x02\n%RequestCancelWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x12\n\nrequest_id\x18\x04 \x01(\t\x12\x1e\n\x16\x66irst_execution_run_id\x18\x05 \x01(\t\x12\x0e\n\x06reason\x18\x06 \x01(\t\x12+\n\x05links\x18\x07 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link"(\n&RequestCancelWorkflowExecutionResponse"\xde\x02\n\x1eSignalWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x13\n\x0bsignal_name\x18\x03 \x01(\t\x12/\n\x05input\x18\x04 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x12\n\nrequest_id\x18\x06 \x01(\t\x12\x13\n\x07\x63ontrol\x18\x07 \x01(\tB\x02\x18\x01\x12.\n\x06header\x18\x08 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12+\n\x05links\x18\n \x03(\x0b\x32\x1c.temporal.api.common.v1.LinkJ\x04\x08\t\x10\n"!\n\x1fSignalWorkflowExecutionResponse"\xf1\t\n\'SignalWithStartWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12\x38\n\ntask_queue\x18\x04 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12/\n\x05input\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12=\n\x1aworkflow_execution_timeout\x18\x06 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x37\n\x14workflow_run_timeout\x18\x07 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x38\n\x15workflow_task_timeout\x18\x08 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x10\n\x08identity\x18\t \x01(\t\x12\x12\n\nrequest_id\x18\n \x01(\t\x12N\n\x18workflow_id_reuse_policy\x18\x0b \x01(\x0e\x32,.temporal.api.enums.v1.WorkflowIdReusePolicy\x12T\n\x1bworkflow_id_conflict_policy\x18\x16 \x01(\x0e\x32/.temporal.api.enums.v1.WorkflowIdConflictPolicy\x12\x13\n\x0bsignal_name\x18\x0c \x01(\t\x12\x36\n\x0csignal_input\x18\r \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x13\n\x07\x63ontrol\x18\x0e \x01(\tB\x02\x18\x01\x12\x39\n\x0cretry_policy\x18\x0f \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy\x12\x15\n\rcron_schedule\x18\x10 \x01(\t\x12*\n\x04memo\x18\x11 \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x12 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12.\n\x06header\x18\x13 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12\x37\n\x14workflow_start_delay\x18\x14 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x38\n\ruser_metadata\x18\x17 \x01(\x0b\x32!.temporal.api.sdk.v1.UserMetadata\x12+\n\x05links\x18\x18 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x12I\n\x13versioning_override\x18\x19 \x01(\x0b\x32,.temporal.api.workflow.v1.VersioningOverride\x12\x32\n\x08priority\x18\x1a \x01(\x0b\x32 .temporal.api.common.v1.PriorityJ\x04\x08\x15\x10\x16"K\n(SignalWithStartWorkflowExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t\x12\x0f\n\x07started\x18\x02 \x01(\x08"\xc1\x03\n\x1dResetWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12%\n\x1dworkflow_task_finish_event_id\x18\x04 \x01(\x03\x12\x12\n\nrequest_id\x18\x05 \x01(\t\x12G\n\x12reset_reapply_type\x18\x06 \x01(\x0e\x32\'.temporal.api.enums.v1.ResetReapplyTypeB\x02\x18\x01\x12S\n\x1breset_reapply_exclude_types\x18\x07 \x03(\x0e\x32..temporal.api.enums.v1.ResetReapplyExcludeType\x12K\n\x15post_reset_operations\x18\x08 \x03(\x0b\x32,.temporal.api.workflow.v1.PostResetOperation\x12\x10\n\x08identity\x18\t \x01(\t"0\n\x1eResetWorkflowExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t"\x9f\x02\n!TerminateWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x31\n\x07\x64\x65tails\x18\x04 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x1e\n\x16\x66irst_execution_run_id\x18\x06 \x01(\t\x12+\n\x05links\x18\x07 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link"$\n"TerminateWorkflowExecutionResponse"z\n\x1e\x44\x65leteWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution"!\n\x1f\x44\x65leteWorkflowExecutionResponse"\xc9\x02\n!ListOpenWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11maximum_page_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x42\n\x11start_time_filter\x18\x04 \x01(\x0b\x32\'.temporal.api.filter.v1.StartTimeFilter\x12K\n\x10\x65xecution_filter\x18\x05 \x01(\x0b\x32/.temporal.api.filter.v1.WorkflowExecutionFilterH\x00\x12\x41\n\x0btype_filter\x18\x06 \x01(\x0b\x32*.temporal.api.filter.v1.WorkflowTypeFilterH\x00\x42\t\n\x07\x66ilters"\x82\x01\n"ListOpenWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\x8a\x03\n#ListClosedWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11maximum_page_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x42\n\x11start_time_filter\x18\x04 \x01(\x0b\x32\'.temporal.api.filter.v1.StartTimeFilter\x12K\n\x10\x65xecution_filter\x18\x05 \x01(\x0b\x32/.temporal.api.filter.v1.WorkflowExecutionFilterH\x00\x12\x41\n\x0btype_filter\x18\x06 \x01(\x0b\x32*.temporal.api.filter.v1.WorkflowTypeFilterH\x00\x12=\n\rstatus_filter\x18\x07 \x01(\x0b\x32$.temporal.api.filter.v1.StatusFilterH\x00\x42\t\n\x07\x66ilters"\x84\x01\n$ListClosedWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"m\n\x1dListWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"~\n\x1eListWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"u\n%ListArchivedWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"\x86\x01\n&ListArchivedWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"m\n\x1dScanWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"~\n\x1eScanWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"B\n\x1e\x43ountWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\r\n\x05query\x18\x02 \x01(\t"\xed\x01\n\x1f\x43ountWorkflowExecutionsResponse\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\x12\x61\n\x06groups\x18\x02 \x03(\x0b\x32Q.temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse.AggregationGroup\x1aX\n\x10\x41ggregationGroup\x12\x35\n\x0cgroup_values\x18\x01 \x03(\x0b\x32\x1f.temporal.api.common.v1.Payload\x12\r\n\x05\x63ount\x18\x02 \x01(\x03"\x1c\n\x1aGetSearchAttributesRequest"\xc9\x01\n\x1bGetSearchAttributesResponse\x12T\n\x04keys\x18\x01 \x03(\x0b\x32\x46.temporal.api.workflowservice.v1.GetSearchAttributesResponse.KeysEntry\x1aT\n\tKeysEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x36\n\x05value\x18\x02 \x01(\x0e\x32\'.temporal.api.enums.v1.IndexedValueType:\x02\x38\x01"\xd0\x02\n RespondQueryTaskCompletedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12>\n\x0e\x63ompleted_type\x18\x02 \x01(\x0e\x32&.temporal.api.enums.v1.QueryResultType\x12\x36\n\x0cquery_result\x18\x03 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x15\n\rerror_message\x18\x04 \x01(\t\x12\x11\n\tnamespace\x18\x06 \x01(\t\x12\x31\n\x07\x66\x61ilure\x18\x07 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12=\n\x05\x63\x61use\x18\x08 \x01(\x0e\x32..temporal.api.enums.v1.WorkflowTaskFailedCauseJ\x04\x08\x05\x10\x06"#\n!RespondQueryTaskCompletedResponse"n\n\x1bResetStickyTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution"\x1e\n\x1cResetStickyTaskQueueResponse"\xaa\x01\n\x15ShutdownWorkerRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11sticky_task_queue\x18\x02 \x01(\t\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0e\n\x06reason\x18\x04 \x01(\t\x12\x41\n\x10worker_heartbeat\x18\x05 \x01(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\x18\n\x16ShutdownWorkerResponse"\xe9\x01\n\x14QueryWorkflowRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x33\n\x05query\x18\x03 \x01(\x0b\x32$.temporal.api.query.v1.WorkflowQuery\x12K\n\x16query_reject_condition\x18\x04 \x01(\x0e\x32+.temporal.api.enums.v1.QueryRejectCondition"\x8d\x01\n\x15QueryWorkflowResponse\x12\x36\n\x0cquery_result\x18\x01 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12<\n\x0equery_rejected\x18\x02 \x01(\x0b\x32$.temporal.api.query.v1.QueryRejected"s\n DescribeWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution"\x99\x05\n!DescribeWorkflowExecutionResponse\x12K\n\x10\x65xecution_config\x18\x01 \x01(\x0b\x32\x31.temporal.api.workflow.v1.WorkflowExecutionConfig\x12P\n\x17workflow_execution_info\x18\x02 \x01(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12I\n\x12pending_activities\x18\x03 \x03(\x0b\x32-.temporal.api.workflow.v1.PendingActivityInfo\x12M\n\x10pending_children\x18\x04 \x03(\x0b\x32\x33.temporal.api.workflow.v1.PendingChildExecutionInfo\x12P\n\x15pending_workflow_task\x18\x05 \x01(\x0b\x32\x31.temporal.api.workflow.v1.PendingWorkflowTaskInfo\x12\x39\n\tcallbacks\x18\x06 \x03(\x0b\x32&.temporal.api.workflow.v1.CallbackInfo\x12U\n\x18pending_nexus_operations\x18\x07 \x03(\x0b\x32\x33.temporal.api.workflow.v1.PendingNexusOperationInfo\x12W\n\x16workflow_extended_info\x18\x08 \x01(\x0b\x32\x37.temporal.api.workflow.v1.WorkflowExecutionExtendedInfo"\x90\x04\n\x18\x44\x65scribeTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12=\n\x0ftask_queue_type\x18\x03 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12\x14\n\x0creport_stats\x18\x08 \x01(\x08\x12\x15\n\rreport_config\x18\x0b \x01(\x08\x12%\n\x19include_task_queue_status\x18\x04 \x01(\x08\x42\x02\x18\x01\x12\x42\n\x08\x61pi_mode\x18\x05 \x01(\x0e\x32,.temporal.api.enums.v1.DescribeTaskQueueModeB\x02\x18\x01\x12J\n\x08versions\x18\x06 \x01(\x0b\x32\x34.temporal.api.taskqueue.v1.TaskQueueVersionSelectionB\x02\x18\x01\x12\x42\n\x10task_queue_types\x18\x07 \x03(\x0e\x32$.temporal.api.enums.v1.TaskQueueTypeB\x02\x18\x01\x12\x1a\n\x0ereport_pollers\x18\t \x01(\x08\x42\x02\x18\x01\x12$\n\x18report_task_reachability\x18\n \x01(\x08\x42\x02\x18\x01"\xec\x07\n\x19\x44\x65scribeTaskQueueResponse\x12\x36\n\x07pollers\x18\x01 \x03(\x0b\x32%.temporal.api.taskqueue.v1.PollerInfo\x12\x38\n\x05stats\x18\x05 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats\x12q\n\x15stats_by_priority_key\x18\x08 \x03(\x0b\x32R.temporal.api.workflowservice.v1.DescribeTaskQueueResponse.StatsByPriorityKeyEntry\x12K\n\x0fversioning_info\x18\x04 \x01(\x0b\x32\x32.temporal.api.taskqueue.v1.TaskQueueVersioningInfo\x12:\n\x06\x63onfig\x18\x06 \x01(\x0b\x32*.temporal.api.taskqueue.v1.TaskQueueConfig\x12k\n\x14\x65\x66\x66\x65\x63tive_rate_limit\x18\x07 \x01(\x0b\x32M.temporal.api.workflowservice.v1.DescribeTaskQueueResponse.EffectiveRateLimit\x12I\n\x11task_queue_status\x18\x02 \x01(\x0b\x32*.temporal.api.taskqueue.v1.TaskQueueStatusB\x02\x18\x01\x12g\n\rversions_info\x18\x03 \x03(\x0b\x32L.temporal.api.workflowservice.v1.DescribeTaskQueueResponse.VersionsInfoEntryB\x02\x18\x01\x1a\x64\n\x17StatsByPriorityKeyEntry\x12\x0b\n\x03key\x18\x01 \x01(\x05\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats:\x02\x38\x01\x1at\n\x12\x45\x66\x66\x65\x63tiveRateLimit\x12\x1b\n\x13requests_per_second\x18\x01 \x01(\x02\x12\x41\n\x11rate_limit_source\x18\x02 \x01(\x0e\x32&.temporal.api.enums.v1.RateLimitSource\x1a\x64\n\x11VersionsInfoEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12>\n\x05value\x18\x02 \x01(\x0b\x32/.temporal.api.taskqueue.v1.TaskQueueVersionInfo:\x02\x38\x01"\x17\n\x15GetClusterInfoRequest"\x8b\x03\n\x16GetClusterInfoResponse\x12h\n\x11supported_clients\x18\x01 \x03(\x0b\x32M.temporal.api.workflowservice.v1.GetClusterInfoResponse.SupportedClientsEntry\x12\x16\n\x0eserver_version\x18\x02 \x01(\t\x12\x12\n\ncluster_id\x18\x03 \x01(\t\x12:\n\x0cversion_info\x18\x04 \x01(\x0b\x32$.temporal.api.version.v1.VersionInfo\x12\x14\n\x0c\x63luster_name\x18\x05 \x01(\t\x12\x1b\n\x13history_shard_count\x18\x06 \x01(\x05\x12\x19\n\x11persistence_store\x18\x07 \x01(\t\x12\x18\n\x10visibility_store\x18\x08 \x01(\t\x1a\x37\n\x15SupportedClientsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\x16\n\x14GetSystemInfoRequest"\xf4\x03\n\x15GetSystemInfoResponse\x12\x16\n\x0eserver_version\x18\x01 \x01(\t\x12Y\n\x0c\x63\x61pabilities\x18\x02 \x01(\x0b\x32\x43.temporal.api.workflowservice.v1.GetSystemInfoResponse.Capabilities\x1a\xe7\x02\n\x0c\x43\x61pabilities\x12\x1f\n\x17signal_and_query_header\x18\x01 \x01(\x08\x12&\n\x1einternal_error_differentiation\x18\x02 \x01(\x08\x12*\n"activity_failure_include_heartbeat\x18\x03 \x01(\x08\x12\x1a\n\x12supports_schedules\x18\x04 \x01(\x08\x12"\n\x1a\x65ncoded_failure_attributes\x18\x05 \x01(\x08\x12!\n\x19\x62uild_id_based_versioning\x18\x06 \x01(\x08\x12\x13\n\x0bupsert_memo\x18\x07 \x01(\x08\x12\x1c\n\x14\x65\x61ger_workflow_start\x18\x08 \x01(\x08\x12\x14\n\x0csdk_metadata\x18\t \x01(\x08\x12\'\n\x1f\x63ount_group_by_execution_status\x18\n \x01(\x08\x12\r\n\x05nexus\x18\x0b \x01(\x08"m\n\x1eListTaskQueuePartitionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue"\xdf\x01\n\x1fListTaskQueuePartitionsResponse\x12]\n\x1e\x61\x63tivity_task_queue_partitions\x18\x01 \x03(\x0b\x32\x35.temporal.api.taskqueue.v1.TaskQueuePartitionMetadata\x12]\n\x1eworkflow_task_queue_partitions\x18\x02 \x03(\x0b\x32\x35.temporal.api.taskqueue.v1.TaskQueuePartitionMetadata"\xcc\x02\n\x15\x43reateScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x34\n\x08schedule\x18\x03 \x01(\x0b\x32".temporal.api.schedule.v1.Schedule\x12>\n\rinitial_patch\x18\x04 \x01(\x0b\x32\'.temporal.api.schedule.v1.SchedulePatch\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x12\n\nrequest_id\x18\x06 \x01(\t\x12*\n\x04memo\x18\x07 \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x08 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes"0\n\x16\x43reateScheduleResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c"A\n\x17\x44\x65scribeScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t"\x8f\x02\n\x18\x44\x65scribeScheduleResponse\x12\x34\n\x08schedule\x18\x01 \x01(\x0b\x32".temporal.api.schedule.v1.Schedule\x12\x34\n\x04info\x18\x02 \x01(\x0b\x32&.temporal.api.schedule.v1.ScheduleInfo\x12*\n\x04memo\x18\x03 \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x04 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12\x16\n\x0e\x63onflict_token\x18\x05 \x01(\x0c"\xf8\x01\n\x15UpdateScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x34\n\x08schedule\x18\x03 \x01(\x0b\x32".temporal.api.schedule.v1.Schedule\x12\x16\n\x0e\x63onflict_token\x18\x04 \x01(\x0c\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x12\n\nrequest_id\x18\x06 \x01(\t\x12\x43\n\x11search_attributes\x18\x07 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes"\x18\n\x16UpdateScheduleResponse"\x9c\x01\n\x14PatchScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x36\n\x05patch\x18\x03 \x01(\x0b\x32\'.temporal.api.schedule.v1.SchedulePatch\x12\x10\n\x08identity\x18\x04 \x01(\t\x12\x12\n\nrequest_id\x18\x05 \x01(\t"\x17\n\x15PatchScheduleResponse"\xa8\x01\n ListScheduleMatchingTimesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12.\n\nstart_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12,\n\x08\x65nd_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"S\n!ListScheduleMatchingTimesResponse\x12.\n\nstart_time\x18\x01 \x03(\x0b\x32\x1a.google.protobuf.Timestamp"Q\n\x15\x44\x65leteScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x10\n\x08identity\x18\x03 \x01(\t"\x18\n\x16\x44\x65leteScheduleResponse"l\n\x14ListSchedulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11maximum_page_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"p\n\x15ListSchedulesResponse\x12>\n\tschedules\x18\x01 \x03(\x0b\x32+.temporal.api.schedule.v1.ScheduleListEntry\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\x86\x05\n\'UpdateWorkerBuildIdCompatibilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t\x12-\n#add_new_build_id_in_new_default_set\x18\x03 \x01(\tH\x00\x12\x87\x01\n\x1b\x61\x64\x64_new_compatible_build_id\x18\x04 \x01(\x0b\x32`.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.AddNewCompatibleVersionH\x00\x12!\n\x17promote_set_by_build_id\x18\x05 \x01(\tH\x00\x12%\n\x1bpromote_build_id_within_set\x18\x06 \x01(\tH\x00\x12h\n\nmerge_sets\x18\x07 \x01(\x0b\x32R.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.MergeSetsH\x00\x1ao\n\x17\x41\x64\x64NewCompatibleVersion\x12\x14\n\x0cnew_build_id\x18\x01 \x01(\t\x12$\n\x1c\x65xisting_compatible_build_id\x18\x02 \x01(\t\x12\x18\n\x10make_set_default\x18\x03 \x01(\x08\x1aI\n\tMergeSets\x12\x1c\n\x14primary_set_build_id\x18\x01 \x01(\t\x12\x1e\n\x16secondary_set_build_id\x18\x02 \x01(\tB\x0b\n\toperation"@\n(UpdateWorkerBuildIdCompatibilityResponseJ\x04\x08\x01\x10\x02R\x0eversion_set_id"_\n$GetWorkerBuildIdCompatibilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t\x12\x10\n\x08max_sets\x18\x03 \x01(\x05"t\n%GetWorkerBuildIdCompatibilityResponse\x12K\n\x12major_version_sets\x18\x01 \x03(\x0b\x32/.temporal.api.taskqueue.v1.CompatibleVersionSet"\xb5\r\n"UpdateWorkerVersioningRulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t\x12\x16\n\x0e\x63onflict_token\x18\x03 \x01(\x0c\x12\x81\x01\n\x16insert_assignment_rule\x18\x04 \x01(\x0b\x32_.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.InsertBuildIdAssignmentRuleH\x00\x12\x83\x01\n\x17replace_assignment_rule\x18\x05 \x01(\x0b\x32`.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceBuildIdAssignmentRuleH\x00\x12\x81\x01\n\x16\x64\x65lete_assignment_rule\x18\x06 \x01(\x0b\x32_.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteBuildIdAssignmentRuleH\x00\x12\x8c\x01\n\x1c\x61\x64\x64_compatible_redirect_rule\x18\x07 \x01(\x0b\x32\x64.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.AddCompatibleBuildIdRedirectRuleH\x00\x12\x94\x01\n replace_compatible_redirect_rule\x18\x08 \x01(\x0b\x32h.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceCompatibleBuildIdRedirectRuleH\x00\x12\x92\x01\n\x1f\x64\x65lete_compatible_redirect_rule\x18\t \x01(\x0b\x32g.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteCompatibleBuildIdRedirectRuleH\x00\x12l\n\x0f\x63ommit_build_id\x18\n \x01(\x0b\x32Q.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.CommitBuildIdH\x00\x1aq\n\x1bInsertBuildIdAssignmentRule\x12\x12\n\nrule_index\x18\x01 \x01(\x05\x12>\n\x04rule\x18\x02 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.BuildIdAssignmentRule\x1a\x81\x01\n\x1cReplaceBuildIdAssignmentRule\x12\x12\n\nrule_index\x18\x01 \x01(\x05\x12>\n\x04rule\x18\x02 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.BuildIdAssignmentRule\x12\r\n\x05\x66orce\x18\x03 \x01(\x08\x1a@\n\x1b\x44\x65leteBuildIdAssignmentRule\x12\x12\n\nrule_index\x18\x01 \x01(\x05\x12\r\n\x05\x66orce\x18\x02 \x01(\x08\x1aj\n AddCompatibleBuildIdRedirectRule\x12\x46\n\x04rule\x18\x01 \x01(\x0b\x32\x38.temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule\x1an\n$ReplaceCompatibleBuildIdRedirectRule\x12\x46\n\x04rule\x18\x01 \x01(\x0b\x32\x38.temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule\x1a>\n#DeleteCompatibleBuildIdRedirectRule\x12\x17\n\x0fsource_build_id\x18\x01 \x01(\t\x1a\x37\n\rCommitBuildId\x12\x17\n\x0ftarget_build_id\x18\x01 \x01(\t\x12\r\n\x05\x66orce\x18\x02 \x01(\x08\x42\x0b\n\toperation"\xfc\x01\n#UpdateWorkerVersioningRulesResponse\x12U\n\x10\x61ssignment_rules\x18\x01 \x03(\x0b\x32;.temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule\x12\x66\n\x19\x63ompatible_redirect_rules\x18\x02 \x03(\x0b\x32\x43.temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule\x12\x16\n\x0e\x63onflict_token\x18\x03 \x01(\x0c"H\n\x1fGetWorkerVersioningRulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t"\xf9\x01\n GetWorkerVersioningRulesResponse\x12U\n\x10\x61ssignment_rules\x18\x01 \x03(\x0b\x32;.temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule\x12\x66\n\x19\x63ompatible_redirect_rules\x18\x02 \x03(\x0b\x32\x43.temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule\x12\x16\n\x0e\x63onflict_token\x18\x03 \x01(\x0c"\x9c\x01\n GetWorkerTaskReachabilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tbuild_ids\x18\x02 \x03(\t\x12\x13\n\x0btask_queues\x18\x03 \x03(\t\x12=\n\x0creachability\x18\x04 \x01(\x0e\x32\'.temporal.api.enums.v1.TaskReachability"r\n!GetWorkerTaskReachabilityResponse\x12M\n\x15\x62uild_id_reachability\x18\x01 \x03(\x0b\x32..temporal.api.taskqueue.v1.BuildIdReachability"\x85\x02\n\x1eUpdateWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x1e\n\x16\x66irst_execution_run_id\x18\x03 \x01(\t\x12\x37\n\x0bwait_policy\x18\x04 \x01(\x0b\x32".temporal.api.update.v1.WaitPolicy\x12\x30\n\x07request\x18\x05 \x01(\x0b\x32\x1f.temporal.api.update.v1.Request"\xd7\x01\n\x1fUpdateWorkflowExecutionResponse\x12\x35\n\nupdate_ref\x18\x01 \x01(\x0b\x32!.temporal.api.update.v1.UpdateRef\x12\x30\n\x07outcome\x18\x02 \x01(\x0b\x32\x1f.temporal.api.update.v1.Outcome\x12K\n\x05stage\x18\x03 \x01(\x0e\x32<.temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage"\xf4\x07\n\x1aStartBatchOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x18\n\x10visibility_query\x18\x02 \x01(\t\x12\x0e\n\x06job_id\x18\x03 \x01(\t\x12\x0e\n\x06reason\x18\x04 \x01(\t\x12=\n\nexecutions\x18\x05 \x03(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12!\n\x19max_operations_per_second\x18\x06 \x01(\x02\x12Q\n\x15termination_operation\x18\n \x01(\x0b\x32\x30.temporal.api.batch.v1.BatchOperationTerminationH\x00\x12G\n\x10signal_operation\x18\x0b \x01(\x0b\x32+.temporal.api.batch.v1.BatchOperationSignalH\x00\x12S\n\x16\x63\x61ncellation_operation\x18\x0c \x01(\x0b\x32\x31.temporal.api.batch.v1.BatchOperationCancellationH\x00\x12K\n\x12\x64\x65letion_operation\x18\r \x01(\x0b\x32-.temporal.api.batch.v1.BatchOperationDeletionH\x00\x12\x45\n\x0freset_operation\x18\x0e \x01(\x0b\x32*.temporal.api.batch.v1.BatchOperationResetH\x00\x12p\n!update_workflow_options_operation\x18\x0f \x01(\x0b\x32\x43.temporal.api.batch.v1.BatchOperationUpdateWorkflowExecutionOptionsH\x00\x12^\n\x1cunpause_activities_operation\x18\x10 \x01(\x0b\x32\x36.temporal.api.batch.v1.BatchOperationUnpauseActivitiesH\x00\x12Z\n\x1areset_activities_operation\x18\x11 \x01(\x0b\x32\x34.temporal.api.batch.v1.BatchOperationResetActivitiesH\x00\x12g\n!update_activity_options_operation\x18\x12 \x01(\x0b\x32:.temporal.api.batch.v1.BatchOperationUpdateActivityOptionsH\x00\x42\x0b\n\toperation"\x1d\n\x1bStartBatchOperationResponse"`\n\x19StopBatchOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0e\n\x06job_id\x18\x02 \x01(\t\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x10\n\x08identity\x18\x04 \x01(\t"\x1c\n\x1aStopBatchOperationResponse"B\n\x1d\x44\x65scribeBatchOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0e\n\x06job_id\x18\x02 \x01(\t"\x92\x03\n\x1e\x44\x65scribeBatchOperationResponse\x12\x41\n\x0eoperation_type\x18\x01 \x01(\x0e\x32).temporal.api.enums.v1.BatchOperationType\x12\x0e\n\x06job_id\x18\x02 \x01(\t\x12\x39\n\x05state\x18\x03 \x01(\x0e\x32*.temporal.api.enums.v1.BatchOperationState\x12.\n\nstart_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nclose_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x1d\n\x15total_operation_count\x18\x06 \x01(\x03\x12 \n\x18\x63omplete_operation_count\x18\x07 \x01(\x03\x12\x1f\n\x17\x66\x61ilure_operation_count\x18\x08 \x01(\x03\x12\x10\n\x08identity\x18\t \x01(\t\x12\x0e\n\x06reason\x18\n \x01(\t"[\n\x1aListBatchOperationsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c"y\n\x1bListBatchOperationsResponse\x12\x41\n\x0eoperation_info\x18\x01 \x03(\x0b\x32).temporal.api.batch.v1.BatchOperationInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\xb9\x01\n"PollWorkflowExecutionUpdateRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x35\n\nupdate_ref\x18\x02 \x01(\x0b\x32!.temporal.api.update.v1.UpdateRef\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x37\n\x0bwait_policy\x18\x04 \x01(\x0b\x32".temporal.api.update.v1.WaitPolicy"\xdb\x01\n#PollWorkflowExecutionUpdateResponse\x12\x30\n\x07outcome\x18\x01 \x01(\x0b\x32\x1f.temporal.api.update.v1.Outcome\x12K\n\x05stage\x18\x02 \x01(\x0e\x32<.temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage\x12\x35\n\nupdate_ref\x18\x03 \x01(\x0b\x32!.temporal.api.update.v1.UpdateRef"\xea\x02\n\x19PollNexusTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x38\n\ntask_queue\x18\x03 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12Z\n\x1bworker_version_capabilities\x18\x04 \x01(\x0b\x32\x31.temporal.api.common.v1.WorkerVersionCapabilitiesB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x06 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x12\x41\n\x10worker_heartbeat\x18\x07 \x03(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\xb4\x01\n\x1aPollNexusTaskQueueResponse\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12/\n\x07request\x18\x02 \x01(\x0b\x32\x1e.temporal.api.nexus.v1.Request\x12Q\n\x17poller_scaling_decision\x18\x03 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.PollerScalingDecision"\x8e\x01\n RespondNexusTaskCompletedRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\ntask_token\x18\x03 \x01(\x0c\x12\x31\n\x08response\x18\x04 \x01(\x0b\x32\x1f.temporal.api.nexus.v1.Response"#\n!RespondNexusTaskCompletedResponse"\x8c\x01\n\x1dRespondNexusTaskFailedRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\ntask_token\x18\x03 \x01(\x0c\x12\x32\n\x05\x65rror\x18\x04 \x01(\x0b\x32#.temporal.api.nexus.v1.HandlerError" \n\x1eRespondNexusTaskFailedResponse"\xdf\x02\n\x1c\x45xecuteMultiOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12[\n\noperations\x18\x02 \x03(\x0b\x32G.temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.Operation\x1a\xce\x01\n\tOperation\x12X\n\x0estart_workflow\x18\x01 \x01(\x0b\x32>.temporal.api.workflowservice.v1.StartWorkflowExecutionRequestH\x00\x12Z\n\x0fupdate_workflow\x18\x02 \x01(\x0b\x32?.temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequestH\x00\x42\x0b\n\toperation"\xcc\x02\n\x1d\x45xecuteMultiOperationResponse\x12Z\n\tresponses\x18\x01 \x03(\x0b\x32G.temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.Response\x1a\xce\x01\n\x08Response\x12Y\n\x0estart_workflow\x18\x01 \x01(\x0b\x32?.temporal.api.workflowservice.v1.StartWorkflowExecutionResponseH\x00\x12[\n\x0fupdate_workflow\x18\x02 \x01(\x0b\x32@.temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponseH\x00\x42\n\n\x08response"\xd0\x02\n\x1cUpdateActivityOptionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x43\n\x10\x61\x63tivity_options\x18\x04 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions\x12/\n\x0bupdate_mask\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.FieldMask\x12\x0c\n\x02id\x18\x06 \x01(\tH\x00\x12\x0e\n\x04type\x18\x07 \x01(\tH\x00\x12\x13\n\tmatch_all\x18\t \x01(\x08H\x00\x12\x18\n\x10restore_original\x18\x08 \x01(\x08\x42\n\n\x08\x61\x63tivity"d\n\x1dUpdateActivityOptionsResponse\x12\x43\n\x10\x61\x63tivity_options\x18\x01 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions"\xb3\x01\n\x14PauseActivityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x0e\n\x04type\x18\x05 \x01(\tH\x00\x12\x0e\n\x06reason\x18\x06 \x01(\tB\n\n\x08\x61\x63tivity"\x17\n\x15PauseActivityResponse"\x98\x02\n\x16UnpauseActivityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x0e\n\x04type\x18\x05 \x01(\tH\x00\x12\x15\n\x0bunpause_all\x18\x06 \x01(\x08H\x00\x12\x16\n\x0ereset_attempts\x18\x07 \x01(\x08\x12\x17\n\x0freset_heartbeat\x18\x08 \x01(\x08\x12)\n\x06jitter\x18\t \x01(\x0b\x32\x19.google.protobuf.DurationB\n\n\x08\x61\x63tivity"\x19\n\x17UnpauseActivityResponse"\xb3\x02\n\x14ResetActivityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x0e\n\x04type\x18\x05 \x01(\tH\x00\x12\x13\n\tmatch_all\x18\n \x01(\x08H\x00\x12\x17\n\x0freset_heartbeat\x18\x06 \x01(\x08\x12\x13\n\x0bkeep_paused\x18\x07 \x01(\x08\x12)\n\x06jitter\x18\x08 \x01(\x0b\x32\x19.google.protobuf.Duration\x12 \n\x18restore_original_options\x18\t \x01(\x08\x42\n\n\x08\x61\x63tivity"\x17\n\x15ResetActivityResponse"\x8a\x02\n%UpdateWorkflowExecutionOptionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12V\n\x1aworkflow_execution_options\x18\x03 \x01(\x0b\x32\x32.temporal.api.workflow.v1.WorkflowExecutionOptions\x12/\n\x0bupdate_mask\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.FieldMask"\x80\x01\n&UpdateWorkflowExecutionOptionsResponse\x12V\n\x1aworkflow_execution_options\x18\x01 \x01(\x0b\x32\x32.temporal.api.workflow.v1.WorkflowExecutionOptions"j\n\x19\x44\x65scribeDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12:\n\ndeployment\x18\x02 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment"a\n\x1a\x44\x65scribeDeploymentResponse\x12\x43\n\x0f\x64\x65ployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo"\xc2\x01\n&DescribeWorkerDeploymentVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x07version\x18\x02 \x01(\tB\x02\x18\x01\x12O\n\x12\x64\x65ployment_version\x18\x03 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1f\n\x17report_task_queue_stats\x18\x04 \x01(\x08"\x8c\x05\n\'DescribeWorkerDeploymentVersionResponse\x12_\n\x1eworker_deployment_version_info\x18\x01 \x01(\x0b\x32\x37.temporal.api.deployment.v1.WorkerDeploymentVersionInfo\x12v\n\x13version_task_queues\x18\x02 \x03(\x0b\x32Y.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionResponse.VersionTaskQueue\x1a\x87\x03\n\x10VersionTaskQueue\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x32\n\x04type\x18\x02 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12\x38\n\x05stats\x18\x03 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats\x12\x90\x01\n\x15stats_by_priority_key\x18\x04 \x03(\x0b\x32q.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionResponse.VersionTaskQueue.StatsByPriorityKeyEntry\x1a\x64\n\x17StatsByPriorityKeyEntry\x12\x0b\n\x03key\x18\x01 \x01(\x05\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats:\x02\x38\x01"M\n\x1f\x44\x65scribeWorkerDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t"\x8c\x01\n DescribeWorkerDeploymentResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12P\n\x16worker_deployment_info\x18\x02 \x01(\x0b\x32\x30.temporal.api.deployment.v1.WorkerDeploymentInfo"l\n\x16ListDeploymentsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x13\n\x0bseries_name\x18\x04 \x01(\t"w\n\x17ListDeploymentsResponse\x12\x17\n\x0fnext_page_token\x18\x01 \x01(\x0c\x12\x43\n\x0b\x64\x65ployments\x18\x02 \x03(\x0b\x32..temporal.api.deployment.v1.DeploymentListInfo"\xcd\x01\n\x1bSetCurrentDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12:\n\ndeployment\x18\x02 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment\x12\x10\n\x08identity\x18\x03 \x01(\t\x12M\n\x0fupdate_metadata\x18\x04 \x01(\x0b\x32\x34.temporal.api.deployment.v1.UpdateDeploymentMetadata"\xb9\x01\n\x1cSetCurrentDeploymentResponse\x12K\n\x17\x63urrent_deployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo\x12L\n\x18previous_deployment_info\x18\x02 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo"\xcb\x01\n(SetWorkerDeploymentCurrentVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x13\n\x07version\x18\x03 \x01(\tB\x02\x18\x01\x12\x10\n\x08\x62uild_id\x18\x07 \x01(\t\x12\x16\n\x0e\x63onflict_token\x18\x04 \x01(\x0c\x12\x10\n\x08identity\x18\x05 \x01(\t\x12"\n\x1aignore_missing_task_queues\x18\x06 \x01(\x08"\xbb\x01\n)SetWorkerDeploymentCurrentVersionResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12\x1c\n\x10previous_version\x18\x02 \x01(\tB\x02\x18\x01\x12X\n\x1bprevious_deployment_version\x18\x03 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion"\xdf\x01\n(SetWorkerDeploymentRampingVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x13\n\x07version\x18\x03 \x01(\tB\x02\x18\x01\x12\x10\n\x08\x62uild_id\x18\x08 \x01(\t\x12\x12\n\npercentage\x18\x04 \x01(\x02\x12\x16\n\x0e\x63onflict_token\x18\x05 \x01(\x0c\x12\x10\n\x08identity\x18\x06 \x01(\t\x12"\n\x1aignore_missing_task_queues\x18\x07 \x01(\x08"\xd8\x01\n)SetWorkerDeploymentRampingVersionResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12\x1c\n\x10previous_version\x18\x02 \x01(\tB\x02\x18\x01\x12X\n\x1bprevious_deployment_version\x18\x04 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1b\n\x13previous_percentage\x18\x03 \x01(\x02"]\n\x1cListWorkerDeploymentsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c"\x9f\x05\n\x1dListWorkerDeploymentsResponse\x12\x17\n\x0fnext_page_token\x18\x01 \x01(\x0c\x12r\n\x12worker_deployments\x18\x02 \x03(\x0b\x32V.temporal.api.workflowservice.v1.ListWorkerDeploymentsResponse.WorkerDeploymentSummary\x1a\xf0\x03\n\x17WorkerDeploymentSummary\x12\x0c\n\x04name\x18\x01 \x01(\t\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x41\n\x0erouting_config\x18\x03 \x01(\x0b\x32).temporal.api.deployment.v1.RoutingConfig\x12o\n\x16latest_version_summary\x18\x04 \x01(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary\x12p\n\x17\x63urrent_version_summary\x18\x05 \x01(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary\x12p\n\x17ramping_version_summary\x18\x06 \x01(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary"\xc8\x01\n$DeleteWorkerDeploymentVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x07version\x18\x02 \x01(\tB\x02\x18\x01\x12O\n\x12\x64\x65ployment_version\x18\x05 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x15\n\rskip_drainage\x18\x03 \x01(\x08\x12\x10\n\x08identity\x18\x04 \x01(\t"\'\n%DeleteWorkerDeploymentVersionResponse"]\n\x1d\x44\x65leteWorkerDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x10\n\x08identity\x18\x03 \x01(\t" \n\x1e\x44\x65leteWorkerDeploymentResponse"\xa2\x03\n,UpdateWorkerDeploymentVersionMetadataRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x07version\x18\x02 \x01(\tB\x02\x18\x01\x12O\n\x12\x64\x65ployment_version\x18\x05 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12x\n\x0eupsert_entries\x18\x03 \x03(\x0b\x32`.temporal.api.workflowservice.v1.UpdateWorkerDeploymentVersionMetadataRequest.UpsertEntriesEntry\x12\x16\n\x0eremove_entries\x18\x04 \x03(\t\x12\x10\n\x08identity\x18\x06 \x01(\t\x1aU\n\x12UpsertEntriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"n\n-UpdateWorkerDeploymentVersionMetadataResponse\x12=\n\x08metadata\x18\x01 \x01(\x0b\x32+.temporal.api.deployment.v1.VersionMetadata"E\n\x1bGetCurrentDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bseries_name\x18\x02 \x01(\t"k\n\x1cGetCurrentDeploymentResponse\x12K\n\x17\x63urrent_deployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo"q\n GetDeploymentReachabilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12:\n\ndeployment\x18\x02 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment"\xe3\x01\n!GetDeploymentReachabilityResponse\x12\x43\n\x0f\x64\x65ployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo\x12\x43\n\x0creachability\x18\x02 \x01(\x0e\x32-.temporal.api.enums.v1.DeploymentReachability\x12\x34\n\x10last_update_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xb4\x01\n\x19\x43reateWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x35\n\x04spec\x18\x02 \x01(\x0b\x32\'.temporal.api.rules.v1.WorkflowRuleSpec\x12\x12\n\nforce_scan\x18\x03 \x01(\x08\x12\x12\n\nrequest_id\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x06 \x01(\t"_\n\x1a\x43reateWorkflowRuleResponse\x12\x31\n\x04rule\x18\x01 \x01(\x0b\x32#.temporal.api.rules.v1.WorkflowRule\x12\x0e\n\x06job_id\x18\x02 \x01(\t"A\n\x1b\x44\x65scribeWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0f\n\x07rule_id\x18\x02 \x01(\t"Q\n\x1c\x44\x65scribeWorkflowRuleResponse\x12\x31\n\x04rule\x18\x01 \x01(\x0b\x32#.temporal.api.rules.v1.WorkflowRule"?\n\x19\x44\x65leteWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0f\n\x07rule_id\x18\x02 \x01(\t"\x1c\n\x1a\x44\x65leteWorkflowRuleResponse"F\n\x18ListWorkflowRulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"h\n\x19ListWorkflowRulesResponse\x12\x32\n\x05rules\x18\x01 \x03(\x0b\x32#.temporal.api.rules.v1.WorkflowRule\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\xce\x01\n\x1aTriggerWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x37\n\x04spec\x18\x05 \x01(\x0b\x32\'.temporal.api.rules.v1.WorkflowRuleSpecH\x00\x12\x10\n\x08identity\x18\x03 \x01(\tB\x06\n\x04rule".\n\x1bTriggerWorkflowRuleResponse\x12\x0f\n\x07\x61pplied\x18\x01 \x01(\x08"\x86\x01\n\x1cRecordWorkerHeartbeatRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x41\n\x10worker_heartbeat\x18\x03 \x03(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\x1f\n\x1dRecordWorkerHeartbeatResponse"b\n\x12ListWorkersRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"h\n\x13ListWorkersResponse\x12\x38\n\x0cworkers_info\x18\x01 \x03(\x0b\x32".temporal.api.worker.v1.WorkerInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\xe2\x03\n\x1cUpdateTaskQueueConfigRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\ntask_queue\x18\x03 \x01(\t\x12=\n\x0ftask_queue_type\x18\x04 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12n\n\x17update_queue_rate_limit\x18\x05 \x01(\x0b\x32M.temporal.api.workflowservice.v1.UpdateTaskQueueConfigRequest.RateLimitUpdate\x12}\n&update_fairness_key_rate_limit_default\x18\x06 \x01(\x0b\x32M.temporal.api.workflowservice.v1.UpdateTaskQueueConfigRequest.RateLimitUpdate\x1a[\n\x0fRateLimitUpdate\x12\x38\n\nrate_limit\x18\x01 \x01(\x0b\x32$.temporal.api.taskqueue.v1.RateLimit\x12\x0e\n\x06reason\x18\x02 \x01(\t"[\n\x1dUpdateTaskQueueConfigResponse\x12:\n\x06\x63onfig\x18\x01 \x01(\x0b\x32*.temporal.api.taskqueue.v1.TaskQueueConfig"\x89\x01\n\x18\x46\x65tchWorkerConfigRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x38\n\x08selector\x18\x06 \x01(\x0b\x32&.temporal.api.common.v1.WorkerSelector"U\n\x19\x46\x65tchWorkerConfigResponse\x12\x38\n\rworker_config\x18\x01 \x01(\x0b\x32!.temporal.api.sdk.v1.WorkerConfig"\xf5\x01\n\x19UpdateWorkerConfigRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x38\n\rworker_config\x18\x04 \x01(\x0b\x32!.temporal.api.sdk.v1.WorkerConfig\x12/\n\x0bupdate_mask\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.FieldMask\x12\x38\n\x08selector\x18\x06 \x01(\x0b\x32&.temporal.api.common.v1.WorkerSelector"d\n\x1aUpdateWorkerConfigResponse\x12:\n\rworker_config\x18\x01 \x01(\x0b\x32!.temporal.api.sdk.v1.WorkerConfigH\x00\x42\n\n\x08responseB\xbe\x01\n"io.temporal.api.workflowservice.v1B\x14RequestResponseProtoP\x01Z5go.temporal.io/api/workflowservice/v1;workflowservice\xaa\x02!Temporalio.Api.WorkflowService.V1\xea\x02$Temporalio::Api::WorkflowService::V1b\x06proto3' + b'\n6temporal/api/workflowservice/v1/request_response.proto\x12\x1ftemporal.api.workflowservice.v1\x1a+temporal/api/enums/v1/batch_operation.proto\x1a"temporal/api/enums/v1/common.proto\x1a$temporal/api/enums/v1/workflow.proto\x1a%temporal/api/enums/v1/namespace.proto\x1a(temporal/api/enums/v1/failed_cause.proto\x1a!temporal/api/enums/v1/query.proto\x1a!temporal/api/enums/v1/reset.proto\x1a&temporal/api/enums/v1/task_queue.proto\x1a&temporal/api/enums/v1/deployment.proto\x1a"temporal/api/enums/v1/update.proto\x1a\x1etemporal/api/enums/v1/id.proto\x1a&temporal/api/activity/v1/message.proto\x1a$temporal/api/common/v1/message.proto\x1a%temporal/api/history/v1/message.proto\x1a&temporal/api/workflow/v1/message.proto\x1a%temporal/api/command/v1/message.proto\x1a(temporal/api/deployment/v1/message.proto\x1a%temporal/api/failure/v1/message.proto\x1a$temporal/api/filter/v1/message.proto\x1a&temporal/api/protocol/v1/message.proto\x1a\'temporal/api/namespace/v1/message.proto\x1a#temporal/api/query/v1/message.proto\x1a)temporal/api/replication/v1/message.proto\x1a#temporal/api/rules/v1/message.proto\x1a\'temporal/api/sdk/v1/worker_config.proto\x1a&temporal/api/schedule/v1/message.proto\x1a\'temporal/api/taskqueue/v1/message.proto\x1a$temporal/api/update/v1/message.proto\x1a%temporal/api/version/v1/message.proto\x1a#temporal/api/batch/v1/message.proto\x1a\x30temporal/api/sdk/v1/task_complete_metadata.proto\x1a\'temporal/api/sdk/v1/user_metadata.proto\x1a#temporal/api/nexus/v1/message.proto\x1a$temporal/api/worker/v1/message.proto\x1a\x1egoogle/protobuf/duration.proto\x1a google/protobuf/field_mask.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x88\x05\n\x18RegisterNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x13\n\x0bowner_email\x18\x03 \x01(\t\x12\x46\n#workflow_execution_retention_period\x18\x04 \x01(\x0b\x32\x19.google.protobuf.Duration\x12G\n\x08\x63lusters\x18\x05 \x03(\x0b\x32\x35.temporal.api.replication.v1.ClusterReplicationConfig\x12\x1b\n\x13\x61\x63tive_cluster_name\x18\x06 \x01(\t\x12Q\n\x04\x64\x61ta\x18\x07 \x03(\x0b\x32\x43.temporal.api.workflowservice.v1.RegisterNamespaceRequest.DataEntry\x12\x16\n\x0esecurity_token\x18\x08 \x01(\t\x12\x1b\n\x13is_global_namespace\x18\t \x01(\x08\x12\x44\n\x16history_archival_state\x18\n \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1c\n\x14history_archival_uri\x18\x0b \x01(\t\x12G\n\x19visibility_archival_state\x18\x0c \x01(\x0e\x32$.temporal.api.enums.v1.ArchivalState\x12\x1f\n\x17visibility_archival_uri\x18\r \x01(\t\x1a+\n\tDataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\x1b\n\x19RegisterNamespaceResponse"\x89\x01\n\x15ListNamespacesRequest\x12\x11\n\tpage_size\x18\x01 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c\x12\x44\n\x10namespace_filter\x18\x03 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceFilter"\x81\x01\n\x16ListNamespacesResponse\x12N\n\nnamespaces\x18\x01 \x03(\x0b\x32:.temporal.api.workflowservice.v1.DescribeNamespaceResponse\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"9\n\x18\x44\x65scribeNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\n\n\x02id\x18\x02 \x01(\t"\xec\x02\n\x19\x44\x65scribeNamespaceResponse\x12@\n\x0enamespace_info\x18\x01 \x01(\x0b\x32(.temporal.api.namespace.v1.NamespaceInfo\x12:\n\x06\x63onfig\x18\x02 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceConfig\x12S\n\x12replication_config\x18\x03 \x01(\x0b\x32\x37.temporal.api.replication.v1.NamespaceReplicationConfig\x12\x18\n\x10\x66\x61ilover_version\x18\x04 \x01(\x03\x12\x1b\n\x13is_global_namespace\x18\x05 \x01(\x08\x12\x45\n\x10\x66\x61ilover_history\x18\x06 \x03(\x0b\x32+.temporal.api.replication.v1.FailoverStatus"\xcf\x02\n\x16UpdateNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x43\n\x0bupdate_info\x18\x02 \x01(\x0b\x32..temporal.api.namespace.v1.UpdateNamespaceInfo\x12:\n\x06\x63onfig\x18\x03 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceConfig\x12S\n\x12replication_config\x18\x04 \x01(\x0b\x32\x37.temporal.api.replication.v1.NamespaceReplicationConfig\x12\x16\n\x0esecurity_token\x18\x05 \x01(\t\x12\x19\n\x11\x64\x65lete_bad_binary\x18\x06 \x01(\t\x12\x19\n\x11promote_namespace\x18\x07 \x01(\x08"\xa3\x02\n\x17UpdateNamespaceResponse\x12@\n\x0enamespace_info\x18\x01 \x01(\x0b\x32(.temporal.api.namespace.v1.NamespaceInfo\x12:\n\x06\x63onfig\x18\x02 \x01(\x0b\x32*.temporal.api.namespace.v1.NamespaceConfig\x12S\n\x12replication_config\x18\x03 \x01(\x0b\x32\x37.temporal.api.replication.v1.NamespaceReplicationConfig\x12\x18\n\x10\x66\x61ilover_version\x18\x04 \x01(\x03\x12\x1b\n\x13is_global_namespace\x18\x05 \x01(\x08"F\n\x19\x44\x65precateNamespaceRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x16\n\x0esecurity_token\x18\x02 \x01(\t"\x1c\n\x1a\x44\x65precateNamespaceResponse"\xa9\x0b\n\x1dStartWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12\x38\n\ntask_queue\x18\x04 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12/\n\x05input\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12=\n\x1aworkflow_execution_timeout\x18\x06 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x37\n\x14workflow_run_timeout\x18\x07 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x38\n\x15workflow_task_timeout\x18\x08 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x10\n\x08identity\x18\t \x01(\t\x12\x12\n\nrequest_id\x18\n \x01(\t\x12N\n\x18workflow_id_reuse_policy\x18\x0b \x01(\x0e\x32,.temporal.api.enums.v1.WorkflowIdReusePolicy\x12T\n\x1bworkflow_id_conflict_policy\x18\x16 \x01(\x0e\x32/.temporal.api.enums.v1.WorkflowIdConflictPolicy\x12\x39\n\x0cretry_policy\x18\x0c \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy\x12\x15\n\rcron_schedule\x18\r \x01(\t\x12*\n\x04memo\x18\x0e \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x0f \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12.\n\x06header\x18\x10 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12\x1f\n\x17request_eager_execution\x18\x11 \x01(\x08\x12;\n\x11\x63ontinued_failure\x18\x12 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12@\n\x16last_completion_result\x18\x13 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x37\n\x14workflow_start_delay\x18\x14 \x01(\x0b\x32\x19.google.protobuf.Duration\x12>\n\x14\x63ompletion_callbacks\x18\x15 \x03(\x0b\x32 .temporal.api.common.v1.Callback\x12\x38\n\ruser_metadata\x18\x17 \x01(\x0b\x32!.temporal.api.sdk.v1.UserMetadata\x12+\n\x05links\x18\x18 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x12I\n\x13versioning_override\x18\x19 \x01(\x0b\x32,.temporal.api.workflow.v1.VersioningOverride\x12H\n\x13on_conflict_options\x18\x1a \x01(\x0b\x32+.temporal.api.workflow.v1.OnConflictOptions\x12\x32\n\x08priority\x18\x1b \x01(\x0b\x32 .temporal.api.common.v1.Priority"\x8a\x02\n\x1eStartWorkflowExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t\x12\x0f\n\x07started\x18\x03 \x01(\x08\x12>\n\x06status\x18\x05 \x01(\x0e\x32..temporal.api.enums.v1.WorkflowExecutionStatus\x12[\n\x13\x65\x61ger_workflow_task\x18\x02 \x01(\x0b\x32>.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse\x12*\n\x04link\x18\x04 \x01(\x0b\x32\x1c.temporal.api.common.v1.Link"\xaa\x02\n"GetWorkflowExecutionHistoryRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x19\n\x11maximum_page_size\x18\x03 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x04 \x01(\x0c\x12\x16\n\x0ewait_new_event\x18\x05 \x01(\x08\x12P\n\x19history_event_filter_type\x18\x06 \x01(\x0e\x32-.temporal.api.enums.v1.HistoryEventFilterType\x12\x15\n\rskip_archival\x18\x07 \x01(\x08"\xba\x01\n#GetWorkflowExecutionHistoryResponse\x12\x31\n\x07history\x18\x01 \x01(\x0b\x32 .temporal.api.history.v1.History\x12\x35\n\x0braw_history\x18\x02 \x03(\x0b\x32 .temporal.api.common.v1.DataBlob\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x10\n\x08\x61rchived\x18\x04 \x01(\x08"\xb0\x01\n)GetWorkflowExecutionHistoryReverseRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x19\n\x11maximum_page_size\x18\x03 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x04 \x01(\x0c"x\n*GetWorkflowExecutionHistoryReverseResponse\x12\x31\n\x07history\x18\x01 \x01(\x0b\x32 .temporal.api.history.v1.History\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c"\x8a\x03\n\x1cPollWorkflowTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x1b\n\x0f\x62inary_checksum\x18\x04 \x01(\tB\x02\x18\x01\x12Z\n\x1bworker_version_capabilities\x18\x05 \x01(\x0b\x32\x31.temporal.api.common.v1.WorkerVersionCapabilitiesB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x06 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x12\x41\n\x10worker_heartbeat\x18\x07 \x01(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\x91\x07\n\x1dPollWorkflowTaskQueueResponse\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12!\n\x19previous_started_event_id\x18\x04 \x01(\x03\x12\x18\n\x10started_event_id\x18\x05 \x01(\x03\x12\x0f\n\x07\x61ttempt\x18\x06 \x01(\x05\x12\x1a\n\x12\x62\x61\x63klog_count_hint\x18\x07 \x01(\x03\x12\x31\n\x07history\x18\x08 \x01(\x0b\x32 .temporal.api.history.v1.History\x12\x17\n\x0fnext_page_token\x18\t \x01(\x0c\x12\x33\n\x05query\x18\n \x01(\x0b\x32$.temporal.api.query.v1.WorkflowQuery\x12K\n\x1dworkflow_execution_task_queue\x18\x0b \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12\x32\n\x0escheduled_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x30\n\x0cstarted_time\x18\r \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\\\n\x07queries\x18\x0e \x03(\x0b\x32K.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.QueriesEntry\x12\x33\n\x08messages\x18\x0f \x03(\x0b\x32!.temporal.api.protocol.v1.Message\x12Q\n\x17poller_scaling_decision\x18\x10 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.PollerScalingDecision\x1aT\n\x0cQueriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x33\n\x05value\x18\x02 \x01(\x0b\x32$.temporal.api.query.v1.WorkflowQuery:\x02\x38\x01"\xb5\t\n#RespondWorkflowTaskCompletedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x32\n\x08\x63ommands\x18\x02 \x03(\x0b\x32 .temporal.api.command.v1.Command\x12\x10\n\x08identity\x18\x03 \x01(\t\x12O\n\x11sticky_attributes\x18\x04 \x01(\x0b\x32\x34.temporal.api.taskqueue.v1.StickyExecutionAttributes\x12 \n\x18return_new_workflow_task\x18\x05 \x01(\x08\x12&\n\x1e\x66orce_create_new_workflow_task\x18\x06 \x01(\x08\x12\x1b\n\x0f\x62inary_checksum\x18\x07 \x01(\tB\x02\x18\x01\x12m\n\rquery_results\x18\x08 \x03(\x0b\x32V.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.QueryResultsEntry\x12\x11\n\tnamespace\x18\t \x01(\t\x12L\n\x14worker_version_stamp\x18\n \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12\x33\n\x08messages\x18\x0b \x03(\x0b\x32!.temporal.api.protocol.v1.Message\x12H\n\x0csdk_metadata\x18\x0c \x01(\x0b\x32\x32.temporal.api.sdk.v1.WorkflowTaskCompletedMetadata\x12\x43\n\x11metering_metadata\x18\r \x01(\x0b\x32(.temporal.api.common.v1.MeteringMetadata\x12g\n\x0c\x63\x61pabilities\x18\x0e \x01(\x0b\x32Q.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.Capabilities\x12>\n\ndeployment\x18\x0f \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12\x46\n\x13versioning_behavior\x18\x10 \x01(\x0e\x32).temporal.api.enums.v1.VersioningBehavior\x12O\n\x12\x64\x65ployment_options\x18\x11 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x1a_\n\x11QueryResultsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.temporal.api.query.v1.WorkflowQueryResult:\x02\x38\x01\x1a\x45\n\x0c\x43\x61pabilities\x12\x35\n-discard_speculative_workflow_task_with_events\x18\x01 \x01(\x08"\xf5\x01\n$RespondWorkflowTaskCompletedResponse\x12U\n\rworkflow_task\x18\x01 \x01(\x0b\x32>.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse\x12V\n\x0e\x61\x63tivity_tasks\x18\x02 \x03(\x0b\x32>.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse\x12\x1e\n\x16reset_history_event_id\x18\x03 \x01(\x03"\xf8\x03\n RespondWorkflowTaskFailedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12=\n\x05\x63\x61use\x18\x02 \x01(\x0e\x32..temporal.api.enums.v1.WorkflowTaskFailedCause\x12\x31\n\x07\x66\x61ilure\x18\x03 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x10\n\x08identity\x18\x04 \x01(\t\x12\x1b\n\x0f\x62inary_checksum\x18\x05 \x01(\tB\x02\x18\x01\x12\x11\n\tnamespace\x18\x06 \x01(\t\x12\x33\n\x08messages\x18\x07 \x03(\x0b\x32!.temporal.api.protocol.v1.Message\x12\x46\n\x0eworker_version\x18\x08 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\t \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\n \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"#\n!RespondWorkflowTaskFailedResponse"\xb8\x03\n\x1cPollActivityTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12\x10\n\x08identity\x18\x03 \x01(\t\x12I\n\x13task_queue_metadata\x18\x04 \x01(\x0b\x32,.temporal.api.taskqueue.v1.TaskQueueMetadata\x12Z\n\x1bworker_version_capabilities\x18\x05 \x01(\x0b\x32\x31.temporal.api.common.v1.WorkerVersionCapabilitiesB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x06 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x12\x41\n\x10worker_heartbeat\x18\x07 \x01(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\xef\x07\n\x1dPollActivityTaskQueueResponse\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x1a\n\x12workflow_namespace\x18\x02 \x01(\t\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12\x45\n\x12workflow_execution\x18\x04 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12;\n\ractivity_type\x18\x05 \x01(\x0b\x32$.temporal.api.common.v1.ActivityType\x12\x13\n\x0b\x61\x63tivity_id\x18\x06 \x01(\t\x12.\n\x06header\x18\x07 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12/\n\x05input\x18\x08 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12;\n\x11heartbeat_details\x18\t \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x32\n\x0escheduled_time\x18\n \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x42\n\x1e\x63urrent_attempt_scheduled_time\x18\x0b \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x30\n\x0cstarted_time\x18\x0c \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07\x61ttempt\x18\r \x01(\x05\x12<\n\x19schedule_to_close_timeout\x18\x0e \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x16start_to_close_timeout\x18\x0f \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x34\n\x11heartbeat_timeout\x18\x10 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x39\n\x0cretry_policy\x18\x11 \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy\x12Q\n\x17poller_scaling_decision\x18\x12 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.PollerScalingDecision\x12\x32\n\x08priority\x18\x13 \x01(\x0b\x32 .temporal.api.common.v1.Priority"\x90\x01\n"RecordActivityTaskHeartbeatRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x31\n\x07\x64\x65tails\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t"p\n#RecordActivityTaskHeartbeatResponse\x12\x18\n\x10\x63\x61ncel_requested\x18\x01 \x01(\x08\x12\x17\n\x0f\x61\x63tivity_paused\x18\x02 \x01(\x08\x12\x16\n\x0e\x61\x63tivity_reset\x18\x03 \x01(\x08"\xba\x01\n&RecordActivityTaskHeartbeatByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x31\n\x07\x64\x65tails\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x06 \x01(\t"t\n\'RecordActivityTaskHeartbeatByIdResponse\x12\x18\n\x10\x63\x61ncel_requested\x18\x01 \x01(\x08\x12\x17\n\x0f\x61\x63tivity_paused\x18\x02 \x01(\x08\x12\x16\n\x0e\x61\x63tivity_reset\x18\x03 \x01(\x08"\xe9\x02\n#RespondActivityTaskCompletedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x30\n\x06result\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t\x12\x46\n\x0eworker_version\x18\x05 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\x06 \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"&\n$RespondActivityTaskCompletedResponse"\xba\x01\n\'RespondActivityTaskCompletedByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x30\n\x06result\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x06 \x01(\t"*\n(RespondActivityTaskCompletedByIdResponse"\xa9\x03\n RespondActivityTaskFailedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x31\n\x07\x66\x61ilure\x18\x02 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t\x12@\n\x16last_heartbeat_details\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x46\n\x0eworker_version\x18\x06 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\x07 \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x08 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"W\n!RespondActivityTaskFailedResponse\x12\x32\n\x08\x66\x61ilures\x18\x01 \x03(\x0b\x32 .temporal.api.failure.v1.Failure"\xfa\x01\n$RespondActivityTaskFailedByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x31\n\x07\x66\x61ilure\x18\x05 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12\x10\n\x08identity\x18\x06 \x01(\t\x12@\n\x16last_heartbeat_details\x18\x07 \x01(\x0b\x32 .temporal.api.common.v1.Payloads"[\n%RespondActivityTaskFailedByIdResponse\x12\x32\n\x08\x66\x61ilures\x18\x01 \x03(\x0b\x32 .temporal.api.failure.v1.Failure"\xe9\x02\n"RespondActivityTaskCanceledRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12\x31\n\x07\x64\x65tails\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x11\n\tnamespace\x18\x04 \x01(\t\x12\x46\n\x0eworker_version\x18\x05 \x01(\x0b\x32*.temporal.api.common.v1.WorkerVersionStampB\x02\x18\x01\x12>\n\ndeployment\x18\x06 \x01(\x0b\x32&.temporal.api.deployment.v1.DeploymentB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions"%\n#RespondActivityTaskCanceledResponse"\x8b\x02\n&RespondActivityTaskCanceledByIdRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12\x31\n\x07\x64\x65tails\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x06 \x01(\t\x12O\n\x12\x64\x65ployment_options\x18\x07 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions")\n\'RespondActivityTaskCanceledByIdResponse"\x84\x02\n%RequestCancelWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x12\n\nrequest_id\x18\x04 \x01(\t\x12\x1e\n\x16\x66irst_execution_run_id\x18\x05 \x01(\t\x12\x0e\n\x06reason\x18\x06 \x01(\t\x12+\n\x05links\x18\x07 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link"(\n&RequestCancelWorkflowExecutionResponse"\xde\x02\n\x1eSignalWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x13\n\x0bsignal_name\x18\x03 \x01(\t\x12/\n\x05input\x18\x04 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x12\n\nrequest_id\x18\x06 \x01(\t\x12\x13\n\x07\x63ontrol\x18\x07 \x01(\tB\x02\x18\x01\x12.\n\x06header\x18\x08 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12+\n\x05links\x18\n \x03(\x0b\x32\x1c.temporal.api.common.v1.LinkJ\x04\x08\t\x10\n"!\n\x1fSignalWorkflowExecutionResponse"\xf1\t\n\'SignalWithStartWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12;\n\rworkflow_type\x18\x03 \x01(\x0b\x32$.temporal.api.common.v1.WorkflowType\x12\x38\n\ntask_queue\x18\x04 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12/\n\x05input\x18\x05 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12=\n\x1aworkflow_execution_timeout\x18\x06 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x37\n\x14workflow_run_timeout\x18\x07 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x38\n\x15workflow_task_timeout\x18\x08 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x10\n\x08identity\x18\t \x01(\t\x12\x12\n\nrequest_id\x18\n \x01(\t\x12N\n\x18workflow_id_reuse_policy\x18\x0b \x01(\x0e\x32,.temporal.api.enums.v1.WorkflowIdReusePolicy\x12T\n\x1bworkflow_id_conflict_policy\x18\x16 \x01(\x0e\x32/.temporal.api.enums.v1.WorkflowIdConflictPolicy\x12\x13\n\x0bsignal_name\x18\x0c \x01(\t\x12\x36\n\x0csignal_input\x18\r \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x13\n\x07\x63ontrol\x18\x0e \x01(\tB\x02\x18\x01\x12\x39\n\x0cretry_policy\x18\x0f \x01(\x0b\x32#.temporal.api.common.v1.RetryPolicy\x12\x15\n\rcron_schedule\x18\x10 \x01(\t\x12*\n\x04memo\x18\x11 \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x12 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12.\n\x06header\x18\x13 \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12\x37\n\x14workflow_start_delay\x18\x14 \x01(\x0b\x32\x19.google.protobuf.Duration\x12\x38\n\ruser_metadata\x18\x17 \x01(\x0b\x32!.temporal.api.sdk.v1.UserMetadata\x12+\n\x05links\x18\x18 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x12I\n\x13versioning_override\x18\x19 \x01(\x0b\x32,.temporal.api.workflow.v1.VersioningOverride\x12\x32\n\x08priority\x18\x1a \x01(\x0b\x32 .temporal.api.common.v1.PriorityJ\x04\x08\x15\x10\x16"K\n(SignalWithStartWorkflowExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t\x12\x0f\n\x07started\x18\x02 \x01(\x08"\xc1\x03\n\x1dResetWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12%\n\x1dworkflow_task_finish_event_id\x18\x04 \x01(\x03\x12\x12\n\nrequest_id\x18\x05 \x01(\t\x12G\n\x12reset_reapply_type\x18\x06 \x01(\x0e\x32\'.temporal.api.enums.v1.ResetReapplyTypeB\x02\x18\x01\x12S\n\x1breset_reapply_exclude_types\x18\x07 \x03(\x0e\x32..temporal.api.enums.v1.ResetReapplyExcludeType\x12K\n\x15post_reset_operations\x18\x08 \x03(\x0b\x32,.temporal.api.workflow.v1.PostResetOperation\x12\x10\n\x08identity\x18\t \x01(\t"0\n\x1eResetWorkflowExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t"\x9f\x02\n!TerminateWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x31\n\x07\x64\x65tails\x18\x04 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x1e\n\x16\x66irst_execution_run_id\x18\x06 \x01(\t\x12+\n\x05links\x18\x07 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link"$\n"TerminateWorkflowExecutionResponse"z\n\x1e\x44\x65leteWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution"!\n\x1f\x44\x65leteWorkflowExecutionResponse"\xc9\x02\n!ListOpenWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11maximum_page_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x42\n\x11start_time_filter\x18\x04 \x01(\x0b\x32\'.temporal.api.filter.v1.StartTimeFilter\x12K\n\x10\x65xecution_filter\x18\x05 \x01(\x0b\x32/.temporal.api.filter.v1.WorkflowExecutionFilterH\x00\x12\x41\n\x0btype_filter\x18\x06 \x01(\x0b\x32*.temporal.api.filter.v1.WorkflowTypeFilterH\x00\x42\t\n\x07\x66ilters"\x82\x01\n"ListOpenWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\x8a\x03\n#ListClosedWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11maximum_page_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x42\n\x11start_time_filter\x18\x04 \x01(\x0b\x32\'.temporal.api.filter.v1.StartTimeFilter\x12K\n\x10\x65xecution_filter\x18\x05 \x01(\x0b\x32/.temporal.api.filter.v1.WorkflowExecutionFilterH\x00\x12\x41\n\x0btype_filter\x18\x06 \x01(\x0b\x32*.temporal.api.filter.v1.WorkflowTypeFilterH\x00\x12=\n\rstatus_filter\x18\x07 \x01(\x0b\x32$.temporal.api.filter.v1.StatusFilterH\x00\x42\t\n\x07\x66ilters"\x84\x01\n$ListClosedWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"m\n\x1dListWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"~\n\x1eListWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"u\n%ListArchivedWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"\x86\x01\n&ListArchivedWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"m\n\x1dScanWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"~\n\x1eScanWorkflowExecutionsResponse\x12\x43\n\nexecutions\x18\x01 \x03(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"B\n\x1e\x43ountWorkflowExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\r\n\x05query\x18\x02 \x01(\t"\xed\x01\n\x1f\x43ountWorkflowExecutionsResponse\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\x12\x61\n\x06groups\x18\x02 \x03(\x0b\x32Q.temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse.AggregationGroup\x1aX\n\x10\x41ggregationGroup\x12\x35\n\x0cgroup_values\x18\x01 \x03(\x0b\x32\x1f.temporal.api.common.v1.Payload\x12\r\n\x05\x63ount\x18\x02 \x01(\x03"\x1c\n\x1aGetSearchAttributesRequest"\xc9\x01\n\x1bGetSearchAttributesResponse\x12T\n\x04keys\x18\x01 \x03(\x0b\x32\x46.temporal.api.workflowservice.v1.GetSearchAttributesResponse.KeysEntry\x1aT\n\tKeysEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x36\n\x05value\x18\x02 \x01(\x0e\x32\'.temporal.api.enums.v1.IndexedValueType:\x02\x38\x01"\xd0\x02\n RespondQueryTaskCompletedRequest\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12>\n\x0e\x63ompleted_type\x18\x02 \x01(\x0e\x32&.temporal.api.enums.v1.QueryResultType\x12\x36\n\x0cquery_result\x18\x03 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12\x15\n\rerror_message\x18\x04 \x01(\t\x12\x11\n\tnamespace\x18\x06 \x01(\t\x12\x31\n\x07\x66\x61ilure\x18\x07 \x01(\x0b\x32 .temporal.api.failure.v1.Failure\x12=\n\x05\x63\x61use\x18\x08 \x01(\x0e\x32..temporal.api.enums.v1.WorkflowTaskFailedCauseJ\x04\x08\x05\x10\x06"#\n!RespondQueryTaskCompletedResponse"n\n\x1bResetStickyTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution"\x1e\n\x1cResetStickyTaskQueueResponse"\xaa\x01\n\x15ShutdownWorkerRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11sticky_task_queue\x18\x02 \x01(\t\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0e\n\x06reason\x18\x04 \x01(\t\x12\x41\n\x10worker_heartbeat\x18\x05 \x01(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\x18\n\x16ShutdownWorkerResponse"\xe9\x01\n\x14QueryWorkflowRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x33\n\x05query\x18\x03 \x01(\x0b\x32$.temporal.api.query.v1.WorkflowQuery\x12K\n\x16query_reject_condition\x18\x04 \x01(\x0e\x32+.temporal.api.enums.v1.QueryRejectCondition"\x8d\x01\n\x15QueryWorkflowResponse\x12\x36\n\x0cquery_result\x18\x01 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12<\n\x0equery_rejected\x18\x02 \x01(\x0b\x32$.temporal.api.query.v1.QueryRejected"s\n DescribeWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution"\x99\x05\n!DescribeWorkflowExecutionResponse\x12K\n\x10\x65xecution_config\x18\x01 \x01(\x0b\x32\x31.temporal.api.workflow.v1.WorkflowExecutionConfig\x12P\n\x17workflow_execution_info\x18\x02 \x01(\x0b\x32/.temporal.api.workflow.v1.WorkflowExecutionInfo\x12I\n\x12pending_activities\x18\x03 \x03(\x0b\x32-.temporal.api.workflow.v1.PendingActivityInfo\x12M\n\x10pending_children\x18\x04 \x03(\x0b\x32\x33.temporal.api.workflow.v1.PendingChildExecutionInfo\x12P\n\x15pending_workflow_task\x18\x05 \x01(\x0b\x32\x31.temporal.api.workflow.v1.PendingWorkflowTaskInfo\x12\x39\n\tcallbacks\x18\x06 \x03(\x0b\x32&.temporal.api.workflow.v1.CallbackInfo\x12U\n\x18pending_nexus_operations\x18\x07 \x03(\x0b\x32\x33.temporal.api.workflow.v1.PendingNexusOperationInfo\x12W\n\x16workflow_extended_info\x18\x08 \x01(\x0b\x32\x37.temporal.api.workflow.v1.WorkflowExecutionExtendedInfo"\x90\x04\n\x18\x44\x65scribeTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12=\n\x0ftask_queue_type\x18\x03 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12\x14\n\x0creport_stats\x18\x08 \x01(\x08\x12\x15\n\rreport_config\x18\x0b \x01(\x08\x12%\n\x19include_task_queue_status\x18\x04 \x01(\x08\x42\x02\x18\x01\x12\x42\n\x08\x61pi_mode\x18\x05 \x01(\x0e\x32,.temporal.api.enums.v1.DescribeTaskQueueModeB\x02\x18\x01\x12J\n\x08versions\x18\x06 \x01(\x0b\x32\x34.temporal.api.taskqueue.v1.TaskQueueVersionSelectionB\x02\x18\x01\x12\x42\n\x10task_queue_types\x18\x07 \x03(\x0e\x32$.temporal.api.enums.v1.TaskQueueTypeB\x02\x18\x01\x12\x1a\n\x0ereport_pollers\x18\t \x01(\x08\x42\x02\x18\x01\x12$\n\x18report_task_reachability\x18\n \x01(\x08\x42\x02\x18\x01"\xec\x07\n\x19\x44\x65scribeTaskQueueResponse\x12\x36\n\x07pollers\x18\x01 \x03(\x0b\x32%.temporal.api.taskqueue.v1.PollerInfo\x12\x38\n\x05stats\x18\x05 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats\x12q\n\x15stats_by_priority_key\x18\x08 \x03(\x0b\x32R.temporal.api.workflowservice.v1.DescribeTaskQueueResponse.StatsByPriorityKeyEntry\x12K\n\x0fversioning_info\x18\x04 \x01(\x0b\x32\x32.temporal.api.taskqueue.v1.TaskQueueVersioningInfo\x12:\n\x06\x63onfig\x18\x06 \x01(\x0b\x32*.temporal.api.taskqueue.v1.TaskQueueConfig\x12k\n\x14\x65\x66\x66\x65\x63tive_rate_limit\x18\x07 \x01(\x0b\x32M.temporal.api.workflowservice.v1.DescribeTaskQueueResponse.EffectiveRateLimit\x12I\n\x11task_queue_status\x18\x02 \x01(\x0b\x32*.temporal.api.taskqueue.v1.TaskQueueStatusB\x02\x18\x01\x12g\n\rversions_info\x18\x03 \x03(\x0b\x32L.temporal.api.workflowservice.v1.DescribeTaskQueueResponse.VersionsInfoEntryB\x02\x18\x01\x1a\x64\n\x17StatsByPriorityKeyEntry\x12\x0b\n\x03key\x18\x01 \x01(\x05\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats:\x02\x38\x01\x1at\n\x12\x45\x66\x66\x65\x63tiveRateLimit\x12\x1b\n\x13requests_per_second\x18\x01 \x01(\x02\x12\x41\n\x11rate_limit_source\x18\x02 \x01(\x0e\x32&.temporal.api.enums.v1.RateLimitSource\x1a\x64\n\x11VersionsInfoEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12>\n\x05value\x18\x02 \x01(\x0b\x32/.temporal.api.taskqueue.v1.TaskQueueVersionInfo:\x02\x38\x01"\x17\n\x15GetClusterInfoRequest"\xd1\x03\n\x16GetClusterInfoResponse\x12h\n\x11supported_clients\x18\x01 \x03(\x0b\x32M.temporal.api.workflowservice.v1.GetClusterInfoResponse.SupportedClientsEntry\x12\x16\n\x0eserver_version\x18\x02 \x01(\t\x12\x12\n\ncluster_id\x18\x03 \x01(\t\x12:\n\x0cversion_info\x18\x04 \x01(\x0b\x32$.temporal.api.version.v1.VersionInfo\x12\x14\n\x0c\x63luster_name\x18\x05 \x01(\t\x12\x1b\n\x13history_shard_count\x18\x06 \x01(\x05\x12\x19\n\x11persistence_store\x18\x07 \x01(\t\x12\x18\n\x10visibility_store\x18\x08 \x01(\t\x12 \n\x18initial_failover_version\x18\t \x01(\x03\x12"\n\x1a\x66\x61ilover_version_increment\x18\n \x01(\x03\x1a\x37\n\x15SupportedClientsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\x16\n\x14GetSystemInfoRequest"\xf4\x03\n\x15GetSystemInfoResponse\x12\x16\n\x0eserver_version\x18\x01 \x01(\t\x12Y\n\x0c\x63\x61pabilities\x18\x02 \x01(\x0b\x32\x43.temporal.api.workflowservice.v1.GetSystemInfoResponse.Capabilities\x1a\xe7\x02\n\x0c\x43\x61pabilities\x12\x1f\n\x17signal_and_query_header\x18\x01 \x01(\x08\x12&\n\x1einternal_error_differentiation\x18\x02 \x01(\x08\x12*\n"activity_failure_include_heartbeat\x18\x03 \x01(\x08\x12\x1a\n\x12supports_schedules\x18\x04 \x01(\x08\x12"\n\x1a\x65ncoded_failure_attributes\x18\x05 \x01(\x08\x12!\n\x19\x62uild_id_based_versioning\x18\x06 \x01(\x08\x12\x13\n\x0bupsert_memo\x18\x07 \x01(\x08\x12\x1c\n\x14\x65\x61ger_workflow_start\x18\x08 \x01(\x08\x12\x14\n\x0csdk_metadata\x18\t \x01(\x08\x12\'\n\x1f\x63ount_group_by_execution_status\x18\n \x01(\x08\x12\r\n\x05nexus\x18\x0b \x01(\x08"m\n\x1eListTaskQueuePartitionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x38\n\ntask_queue\x18\x02 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue"\xdf\x01\n\x1fListTaskQueuePartitionsResponse\x12]\n\x1e\x61\x63tivity_task_queue_partitions\x18\x01 \x03(\x0b\x32\x35.temporal.api.taskqueue.v1.TaskQueuePartitionMetadata\x12]\n\x1eworkflow_task_queue_partitions\x18\x02 \x03(\x0b\x32\x35.temporal.api.taskqueue.v1.TaskQueuePartitionMetadata"\xcc\x02\n\x15\x43reateScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x34\n\x08schedule\x18\x03 \x01(\x0b\x32".temporal.api.schedule.v1.Schedule\x12>\n\rinitial_patch\x18\x04 \x01(\x0b\x32\'.temporal.api.schedule.v1.SchedulePatch\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x12\n\nrequest_id\x18\x06 \x01(\t\x12*\n\x04memo\x18\x07 \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x08 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes"0\n\x16\x43reateScheduleResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c"A\n\x17\x44\x65scribeScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t"\x8f\x02\n\x18\x44\x65scribeScheduleResponse\x12\x34\n\x08schedule\x18\x01 \x01(\x0b\x32".temporal.api.schedule.v1.Schedule\x12\x34\n\x04info\x18\x02 \x01(\x0b\x32&.temporal.api.schedule.v1.ScheduleInfo\x12*\n\x04memo\x18\x03 \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x04 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12\x16\n\x0e\x63onflict_token\x18\x05 \x01(\x0c"\xf8\x01\n\x15UpdateScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x34\n\x08schedule\x18\x03 \x01(\x0b\x32".temporal.api.schedule.v1.Schedule\x12\x16\n\x0e\x63onflict_token\x18\x04 \x01(\x0c\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x12\n\nrequest_id\x18\x06 \x01(\t\x12\x43\n\x11search_attributes\x18\x07 \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes"\x18\n\x16UpdateScheduleResponse"\x9c\x01\n\x14PatchScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x36\n\x05patch\x18\x03 \x01(\x0b\x32\'.temporal.api.schedule.v1.SchedulePatch\x12\x10\n\x08identity\x18\x04 \x01(\t\x12\x12\n\nrequest_id\x18\x05 \x01(\t"\x17\n\x15PatchScheduleResponse"\xa8\x01\n ListScheduleMatchingTimesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12.\n\nstart_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12,\n\x08\x65nd_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"S\n!ListScheduleMatchingTimesResponse\x12.\n\nstart_time\x18\x01 \x03(\x0b\x32\x1a.google.protobuf.Timestamp"Q\n\x15\x44\x65leteScheduleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bschedule_id\x18\x02 \x01(\t\x12\x10\n\x08identity\x18\x03 \x01(\t"\x18\n\x16\x44\x65leteScheduleResponse"l\n\x14ListSchedulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x19\n\x11maximum_page_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"p\n\x15ListSchedulesResponse\x12>\n\tschedules\x18\x01 \x03(\x0b\x32+.temporal.api.schedule.v1.ScheduleListEntry\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\x86\x05\n\'UpdateWorkerBuildIdCompatibilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t\x12-\n#add_new_build_id_in_new_default_set\x18\x03 \x01(\tH\x00\x12\x87\x01\n\x1b\x61\x64\x64_new_compatible_build_id\x18\x04 \x01(\x0b\x32`.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.AddNewCompatibleVersionH\x00\x12!\n\x17promote_set_by_build_id\x18\x05 \x01(\tH\x00\x12%\n\x1bpromote_build_id_within_set\x18\x06 \x01(\tH\x00\x12h\n\nmerge_sets\x18\x07 \x01(\x0b\x32R.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.MergeSetsH\x00\x1ao\n\x17\x41\x64\x64NewCompatibleVersion\x12\x14\n\x0cnew_build_id\x18\x01 \x01(\t\x12$\n\x1c\x65xisting_compatible_build_id\x18\x02 \x01(\t\x12\x18\n\x10make_set_default\x18\x03 \x01(\x08\x1aI\n\tMergeSets\x12\x1c\n\x14primary_set_build_id\x18\x01 \x01(\t\x12\x1e\n\x16secondary_set_build_id\x18\x02 \x01(\tB\x0b\n\toperation"@\n(UpdateWorkerBuildIdCompatibilityResponseJ\x04\x08\x01\x10\x02R\x0eversion_set_id"_\n$GetWorkerBuildIdCompatibilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t\x12\x10\n\x08max_sets\x18\x03 \x01(\x05"t\n%GetWorkerBuildIdCompatibilityResponse\x12K\n\x12major_version_sets\x18\x01 \x03(\x0b\x32/.temporal.api.taskqueue.v1.CompatibleVersionSet"\xb5\r\n"UpdateWorkerVersioningRulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t\x12\x16\n\x0e\x63onflict_token\x18\x03 \x01(\x0c\x12\x81\x01\n\x16insert_assignment_rule\x18\x04 \x01(\x0b\x32_.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.InsertBuildIdAssignmentRuleH\x00\x12\x83\x01\n\x17replace_assignment_rule\x18\x05 \x01(\x0b\x32`.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceBuildIdAssignmentRuleH\x00\x12\x81\x01\n\x16\x64\x65lete_assignment_rule\x18\x06 \x01(\x0b\x32_.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteBuildIdAssignmentRuleH\x00\x12\x8c\x01\n\x1c\x61\x64\x64_compatible_redirect_rule\x18\x07 \x01(\x0b\x32\x64.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.AddCompatibleBuildIdRedirectRuleH\x00\x12\x94\x01\n replace_compatible_redirect_rule\x18\x08 \x01(\x0b\x32h.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceCompatibleBuildIdRedirectRuleH\x00\x12\x92\x01\n\x1f\x64\x65lete_compatible_redirect_rule\x18\t \x01(\x0b\x32g.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteCompatibleBuildIdRedirectRuleH\x00\x12l\n\x0f\x63ommit_build_id\x18\n \x01(\x0b\x32Q.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.CommitBuildIdH\x00\x1aq\n\x1bInsertBuildIdAssignmentRule\x12\x12\n\nrule_index\x18\x01 \x01(\x05\x12>\n\x04rule\x18\x02 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.BuildIdAssignmentRule\x1a\x81\x01\n\x1cReplaceBuildIdAssignmentRule\x12\x12\n\nrule_index\x18\x01 \x01(\x05\x12>\n\x04rule\x18\x02 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.BuildIdAssignmentRule\x12\r\n\x05\x66orce\x18\x03 \x01(\x08\x1a@\n\x1b\x44\x65leteBuildIdAssignmentRule\x12\x12\n\nrule_index\x18\x01 \x01(\x05\x12\r\n\x05\x66orce\x18\x02 \x01(\x08\x1aj\n AddCompatibleBuildIdRedirectRule\x12\x46\n\x04rule\x18\x01 \x01(\x0b\x32\x38.temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule\x1an\n$ReplaceCompatibleBuildIdRedirectRule\x12\x46\n\x04rule\x18\x01 \x01(\x0b\x32\x38.temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule\x1a>\n#DeleteCompatibleBuildIdRedirectRule\x12\x17\n\x0fsource_build_id\x18\x01 \x01(\t\x1a\x37\n\rCommitBuildId\x12\x17\n\x0ftarget_build_id\x18\x01 \x01(\t\x12\r\n\x05\x66orce\x18\x02 \x01(\x08\x42\x0b\n\toperation"\xfc\x01\n#UpdateWorkerVersioningRulesResponse\x12U\n\x10\x61ssignment_rules\x18\x01 \x03(\x0b\x32;.temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule\x12\x66\n\x19\x63ompatible_redirect_rules\x18\x02 \x03(\x0b\x32\x43.temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule\x12\x16\n\x0e\x63onflict_token\x18\x03 \x01(\x0c"H\n\x1fGetWorkerVersioningRulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x12\n\ntask_queue\x18\x02 \x01(\t"\xf9\x01\n GetWorkerVersioningRulesResponse\x12U\n\x10\x61ssignment_rules\x18\x01 \x03(\x0b\x32;.temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule\x12\x66\n\x19\x63ompatible_redirect_rules\x18\x02 \x03(\x0b\x32\x43.temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule\x12\x16\n\x0e\x63onflict_token\x18\x03 \x01(\x0c"\x9c\x01\n GetWorkerTaskReachabilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tbuild_ids\x18\x02 \x03(\t\x12\x13\n\x0btask_queues\x18\x03 \x03(\t\x12=\n\x0creachability\x18\x04 \x01(\x0e\x32\'.temporal.api.enums.v1.TaskReachability"r\n!GetWorkerTaskReachabilityResponse\x12M\n\x15\x62uild_id_reachability\x18\x01 \x03(\x0b\x32..temporal.api.taskqueue.v1.BuildIdReachability"\x85\x02\n\x1eUpdateWorkflowExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x1e\n\x16\x66irst_execution_run_id\x18\x03 \x01(\t\x12\x37\n\x0bwait_policy\x18\x04 \x01(\x0b\x32".temporal.api.update.v1.WaitPolicy\x12\x30\n\x07request\x18\x05 \x01(\x0b\x32\x1f.temporal.api.update.v1.Request"\xd7\x01\n\x1fUpdateWorkflowExecutionResponse\x12\x35\n\nupdate_ref\x18\x01 \x01(\x0b\x32!.temporal.api.update.v1.UpdateRef\x12\x30\n\x07outcome\x18\x02 \x01(\x0b\x32\x1f.temporal.api.update.v1.Outcome\x12K\n\x05stage\x18\x03 \x01(\x0e\x32<.temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage"\xf4\x07\n\x1aStartBatchOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x18\n\x10visibility_query\x18\x02 \x01(\t\x12\x0e\n\x06job_id\x18\x03 \x01(\t\x12\x0e\n\x06reason\x18\x04 \x01(\t\x12=\n\nexecutions\x18\x05 \x03(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12!\n\x19max_operations_per_second\x18\x06 \x01(\x02\x12Q\n\x15termination_operation\x18\n \x01(\x0b\x32\x30.temporal.api.batch.v1.BatchOperationTerminationH\x00\x12G\n\x10signal_operation\x18\x0b \x01(\x0b\x32+.temporal.api.batch.v1.BatchOperationSignalH\x00\x12S\n\x16\x63\x61ncellation_operation\x18\x0c \x01(\x0b\x32\x31.temporal.api.batch.v1.BatchOperationCancellationH\x00\x12K\n\x12\x64\x65letion_operation\x18\r \x01(\x0b\x32-.temporal.api.batch.v1.BatchOperationDeletionH\x00\x12\x45\n\x0freset_operation\x18\x0e \x01(\x0b\x32*.temporal.api.batch.v1.BatchOperationResetH\x00\x12p\n!update_workflow_options_operation\x18\x0f \x01(\x0b\x32\x43.temporal.api.batch.v1.BatchOperationUpdateWorkflowExecutionOptionsH\x00\x12^\n\x1cunpause_activities_operation\x18\x10 \x01(\x0b\x32\x36.temporal.api.batch.v1.BatchOperationUnpauseActivitiesH\x00\x12Z\n\x1areset_activities_operation\x18\x11 \x01(\x0b\x32\x34.temporal.api.batch.v1.BatchOperationResetActivitiesH\x00\x12g\n!update_activity_options_operation\x18\x12 \x01(\x0b\x32:.temporal.api.batch.v1.BatchOperationUpdateActivityOptionsH\x00\x42\x0b\n\toperation"\x1d\n\x1bStartBatchOperationResponse"`\n\x19StopBatchOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0e\n\x06job_id\x18\x02 \x01(\t\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x10\n\x08identity\x18\x04 \x01(\t"\x1c\n\x1aStopBatchOperationResponse"B\n\x1d\x44\x65scribeBatchOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0e\n\x06job_id\x18\x02 \x01(\t"\x92\x03\n\x1e\x44\x65scribeBatchOperationResponse\x12\x41\n\x0eoperation_type\x18\x01 \x01(\x0e\x32).temporal.api.enums.v1.BatchOperationType\x12\x0e\n\x06job_id\x18\x02 \x01(\t\x12\x39\n\x05state\x18\x03 \x01(\x0e\x32*.temporal.api.enums.v1.BatchOperationState\x12.\n\nstart_time\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nclose_time\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x1d\n\x15total_operation_count\x18\x06 \x01(\x03\x12 \n\x18\x63omplete_operation_count\x18\x07 \x01(\x03\x12\x1f\n\x17\x66\x61ilure_operation_count\x18\x08 \x01(\x03\x12\x10\n\x08identity\x18\t \x01(\t\x12\x0e\n\x06reason\x18\n \x01(\t"[\n\x1aListBatchOperationsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c"y\n\x1bListBatchOperationsResponse\x12\x41\n\x0eoperation_info\x18\x01 \x03(\x0b\x32).temporal.api.batch.v1.BatchOperationInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\xb9\x01\n"PollWorkflowExecutionUpdateRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x35\n\nupdate_ref\x18\x02 \x01(\x0b\x32!.temporal.api.update.v1.UpdateRef\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x37\n\x0bwait_policy\x18\x04 \x01(\x0b\x32".temporal.api.update.v1.WaitPolicy"\xdb\x01\n#PollWorkflowExecutionUpdateResponse\x12\x30\n\x07outcome\x18\x01 \x01(\x0b\x32\x1f.temporal.api.update.v1.Outcome\x12K\n\x05stage\x18\x02 \x01(\x0e\x32<.temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage\x12\x35\n\nupdate_ref\x18\x03 \x01(\x0b\x32!.temporal.api.update.v1.UpdateRef"\xea\x02\n\x19PollNexusTaskQueueRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x38\n\ntask_queue\x18\x03 \x01(\x0b\x32$.temporal.api.taskqueue.v1.TaskQueue\x12Z\n\x1bworker_version_capabilities\x18\x04 \x01(\x0b\x32\x31.temporal.api.common.v1.WorkerVersionCapabilitiesB\x02\x18\x01\x12O\n\x12\x64\x65ployment_options\x18\x06 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentOptions\x12\x41\n\x10worker_heartbeat\x18\x07 \x03(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\xb4\x01\n\x1aPollNexusTaskQueueResponse\x12\x12\n\ntask_token\x18\x01 \x01(\x0c\x12/\n\x07request\x18\x02 \x01(\x0b\x32\x1e.temporal.api.nexus.v1.Request\x12Q\n\x17poller_scaling_decision\x18\x03 \x01(\x0b\x32\x30.temporal.api.taskqueue.v1.PollerScalingDecision"\x8e\x01\n RespondNexusTaskCompletedRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\ntask_token\x18\x03 \x01(\x0c\x12\x31\n\x08response\x18\x04 \x01(\x0b\x32\x1f.temporal.api.nexus.v1.Response"#\n!RespondNexusTaskCompletedResponse"\x8c\x01\n\x1dRespondNexusTaskFailedRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\ntask_token\x18\x03 \x01(\x0c\x12\x32\n\x05\x65rror\x18\x04 \x01(\x0b\x32#.temporal.api.nexus.v1.HandlerError" \n\x1eRespondNexusTaskFailedResponse"\xdf\x02\n\x1c\x45xecuteMultiOperationRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12[\n\noperations\x18\x02 \x03(\x0b\x32G.temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.Operation\x1a\xce\x01\n\tOperation\x12X\n\x0estart_workflow\x18\x01 \x01(\x0b\x32>.temporal.api.workflowservice.v1.StartWorkflowExecutionRequestH\x00\x12Z\n\x0fupdate_workflow\x18\x02 \x01(\x0b\x32?.temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequestH\x00\x42\x0b\n\toperation"\xcc\x02\n\x1d\x45xecuteMultiOperationResponse\x12Z\n\tresponses\x18\x01 \x03(\x0b\x32G.temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.Response\x1a\xce\x01\n\x08Response\x12Y\n\x0estart_workflow\x18\x01 \x01(\x0b\x32?.temporal.api.workflowservice.v1.StartWorkflowExecutionResponseH\x00\x12[\n\x0fupdate_workflow\x18\x02 \x01(\x0b\x32@.temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponseH\x00\x42\n\n\x08response"\xd0\x02\n\x1cUpdateActivityOptionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x43\n\x10\x61\x63tivity_options\x18\x04 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions\x12/\n\x0bupdate_mask\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.FieldMask\x12\x0c\n\x02id\x18\x06 \x01(\tH\x00\x12\x0e\n\x04type\x18\x07 \x01(\tH\x00\x12\x13\n\tmatch_all\x18\t \x01(\x08H\x00\x12\x18\n\x10restore_original\x18\x08 \x01(\x08\x42\n\n\x08\x61\x63tivity"d\n\x1dUpdateActivityOptionsResponse\x12\x43\n\x10\x61\x63tivity_options\x18\x01 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions"\xb3\x01\n\x14PauseActivityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x0e\n\x04type\x18\x05 \x01(\tH\x00\x12\x0e\n\x06reason\x18\x06 \x01(\tB\n\n\x08\x61\x63tivity"\x17\n\x15PauseActivityResponse"\x98\x02\n\x16UnpauseActivityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x0e\n\x04type\x18\x05 \x01(\tH\x00\x12\x15\n\x0bunpause_all\x18\x06 \x01(\x08H\x00\x12\x16\n\x0ereset_attempts\x18\x07 \x01(\x08\x12\x17\n\x0freset_heartbeat\x18\x08 \x01(\x08\x12)\n\x06jitter\x18\t \x01(\x0b\x32\x19.google.protobuf.DurationB\n\n\x08\x61\x63tivity"\x19\n\x17UnpauseActivityResponse"\xb3\x02\n\x14ResetActivityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x10\n\x08identity\x18\x03 \x01(\t\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x0e\n\x04type\x18\x05 \x01(\tH\x00\x12\x13\n\tmatch_all\x18\n \x01(\x08H\x00\x12\x17\n\x0freset_heartbeat\x18\x06 \x01(\x08\x12\x13\n\x0bkeep_paused\x18\x07 \x01(\x08\x12)\n\x06jitter\x18\x08 \x01(\x0b\x32\x19.google.protobuf.Duration\x12 \n\x18restore_original_options\x18\t \x01(\x08\x42\n\n\x08\x61\x63tivity"\x17\n\x15ResetActivityResponse"\x8a\x02\n%UpdateWorkflowExecutionOptionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x45\n\x12workflow_execution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12V\n\x1aworkflow_execution_options\x18\x03 \x01(\x0b\x32\x32.temporal.api.workflow.v1.WorkflowExecutionOptions\x12/\n\x0bupdate_mask\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.FieldMask"\x80\x01\n&UpdateWorkflowExecutionOptionsResponse\x12V\n\x1aworkflow_execution_options\x18\x01 \x01(\x0b\x32\x32.temporal.api.workflow.v1.WorkflowExecutionOptions"j\n\x19\x44\x65scribeDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12:\n\ndeployment\x18\x02 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment"a\n\x1a\x44\x65scribeDeploymentResponse\x12\x43\n\x0f\x64\x65ployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo"\xc2\x01\n&DescribeWorkerDeploymentVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x07version\x18\x02 \x01(\tB\x02\x18\x01\x12O\n\x12\x64\x65ployment_version\x18\x03 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1f\n\x17report_task_queue_stats\x18\x04 \x01(\x08"\x8c\x05\n\'DescribeWorkerDeploymentVersionResponse\x12_\n\x1eworker_deployment_version_info\x18\x01 \x01(\x0b\x32\x37.temporal.api.deployment.v1.WorkerDeploymentVersionInfo\x12v\n\x13version_task_queues\x18\x02 \x03(\x0b\x32Y.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionResponse.VersionTaskQueue\x1a\x87\x03\n\x10VersionTaskQueue\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x32\n\x04type\x18\x02 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12\x38\n\x05stats\x18\x03 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats\x12\x90\x01\n\x15stats_by_priority_key\x18\x04 \x03(\x0b\x32q.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionResponse.VersionTaskQueue.StatsByPriorityKeyEntry\x1a\x64\n\x17StatsByPriorityKeyEntry\x12\x0b\n\x03key\x18\x01 \x01(\x05\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).temporal.api.taskqueue.v1.TaskQueueStats:\x02\x38\x01"M\n\x1f\x44\x65scribeWorkerDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t"\x8c\x01\n DescribeWorkerDeploymentResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12P\n\x16worker_deployment_info\x18\x02 \x01(\x0b\x32\x30.temporal.api.deployment.v1.WorkerDeploymentInfo"l\n\x16ListDeploymentsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\x13\n\x0bseries_name\x18\x04 \x01(\t"w\n\x17ListDeploymentsResponse\x12\x17\n\x0fnext_page_token\x18\x01 \x01(\x0c\x12\x43\n\x0b\x64\x65ployments\x18\x02 \x03(\x0b\x32..temporal.api.deployment.v1.DeploymentListInfo"\xcd\x01\n\x1bSetCurrentDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12:\n\ndeployment\x18\x02 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment\x12\x10\n\x08identity\x18\x03 \x01(\t\x12M\n\x0fupdate_metadata\x18\x04 \x01(\x0b\x32\x34.temporal.api.deployment.v1.UpdateDeploymentMetadata"\xb9\x01\n\x1cSetCurrentDeploymentResponse\x12K\n\x17\x63urrent_deployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo\x12L\n\x18previous_deployment_info\x18\x02 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo"\xe5\x01\n(SetWorkerDeploymentCurrentVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x13\n\x07version\x18\x03 \x01(\tB\x02\x18\x01\x12\x10\n\x08\x62uild_id\x18\x07 \x01(\t\x12\x16\n\x0e\x63onflict_token\x18\x04 \x01(\x0c\x12\x10\n\x08identity\x18\x05 \x01(\t\x12"\n\x1aignore_missing_task_queues\x18\x06 \x01(\x08\x12\x18\n\x10\x61llow_no_pollers\x18\t \x01(\x08"\xbb\x01\n)SetWorkerDeploymentCurrentVersionResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12\x1c\n\x10previous_version\x18\x02 \x01(\tB\x02\x18\x01\x12X\n\x1bprevious_deployment_version\x18\x03 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion"\xf9\x01\n(SetWorkerDeploymentRampingVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x13\n\x07version\x18\x03 \x01(\tB\x02\x18\x01\x12\x10\n\x08\x62uild_id\x18\x08 \x01(\t\x12\x12\n\npercentage\x18\x04 \x01(\x02\x12\x16\n\x0e\x63onflict_token\x18\x05 \x01(\x0c\x12\x10\n\x08identity\x18\x06 \x01(\t\x12"\n\x1aignore_missing_task_queues\x18\x07 \x01(\x08\x12\x18\n\x10\x61llow_no_pollers\x18\n \x01(\x08"\xd8\x01\n)SetWorkerDeploymentRampingVersionResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12\x1c\n\x10previous_version\x18\x02 \x01(\tB\x02\x18\x01\x12X\n\x1bprevious_deployment_version\x18\x04 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x1b\n\x13previous_percentage\x18\x03 \x01(\x02"]\n\x1cListWorkerDeploymentsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c"\x9f\x05\n\x1dListWorkerDeploymentsResponse\x12\x17\n\x0fnext_page_token\x18\x01 \x01(\x0c\x12r\n\x12worker_deployments\x18\x02 \x03(\x0b\x32V.temporal.api.workflowservice.v1.ListWorkerDeploymentsResponse.WorkerDeploymentSummary\x1a\xf0\x03\n\x17WorkerDeploymentSummary\x12\x0c\n\x04name\x18\x01 \x01(\t\x12/\n\x0b\x63reate_time\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x41\n\x0erouting_config\x18\x03 \x01(\x0b\x32).temporal.api.deployment.v1.RoutingConfig\x12o\n\x16latest_version_summary\x18\x04 \x01(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary\x12p\n\x17\x63urrent_version_summary\x18\x05 \x01(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary\x12p\n\x17ramping_version_summary\x18\x06 \x01(\x0b\x32O.temporal.api.deployment.v1.WorkerDeploymentInfo.WorkerDeploymentVersionSummary"\xc8\x01\n$DeleteWorkerDeploymentVersionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x07version\x18\x02 \x01(\tB\x02\x18\x01\x12O\n\x12\x64\x65ployment_version\x18\x05 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12\x15\n\rskip_drainage\x18\x03 \x01(\x08\x12\x10\n\x08identity\x18\x04 \x01(\t"\'\n%DeleteWorkerDeploymentVersionResponse"]\n\x1d\x44\x65leteWorkerDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x10\n\x08identity\x18\x03 \x01(\t" \n\x1e\x44\x65leteWorkerDeploymentResponse"\xa2\x03\n,UpdateWorkerDeploymentVersionMetadataRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x07version\x18\x02 \x01(\tB\x02\x18\x01\x12O\n\x12\x64\x65ployment_version\x18\x05 \x01(\x0b\x32\x33.temporal.api.deployment.v1.WorkerDeploymentVersion\x12x\n\x0eupsert_entries\x18\x03 \x03(\x0b\x32`.temporal.api.workflowservice.v1.UpdateWorkerDeploymentVersionMetadataRequest.UpsertEntriesEntry\x12\x16\n\x0eremove_entries\x18\x04 \x03(\t\x12\x10\n\x08identity\x18\x06 \x01(\t\x1aU\n\x12UpsertEntriesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12.\n\x05value\x18\x02 \x01(\x0b\x32\x1f.temporal.api.common.v1.Payload:\x02\x38\x01"n\n-UpdateWorkerDeploymentVersionMetadataResponse\x12=\n\x08metadata\x18\x01 \x01(\x0b\x32+.temporal.api.deployment.v1.VersionMetadata"\xbd\x01\n!SetWorkerDeploymentManagerRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0f\x64\x65ployment_name\x18\x02 \x01(\t\x12\x1a\n\x10manager_identity\x18\x03 \x01(\tH\x00\x12\x0e\n\x04self\x18\x04 \x01(\x08H\x00\x12\x16\n\x0e\x63onflict_token\x18\x05 \x01(\x0c\x12\x10\n\x08identity\x18\x06 \x01(\tB\x16\n\x14new_manager_identity"_\n"SetWorkerDeploymentManagerResponse\x12\x16\n\x0e\x63onflict_token\x18\x01 \x01(\x0c\x12!\n\x19previous_manager_identity\x18\x02 \x01(\t"E\n\x1bGetCurrentDeploymentRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bseries_name\x18\x02 \x01(\t"k\n\x1cGetCurrentDeploymentResponse\x12K\n\x17\x63urrent_deployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo"q\n GetDeploymentReachabilityRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12:\n\ndeployment\x18\x02 \x01(\x0b\x32&.temporal.api.deployment.v1.Deployment"\xe3\x01\n!GetDeploymentReachabilityResponse\x12\x43\n\x0f\x64\x65ployment_info\x18\x01 \x01(\x0b\x32*.temporal.api.deployment.v1.DeploymentInfo\x12\x43\n\x0creachability\x18\x02 \x01(\x0e\x32-.temporal.api.enums.v1.DeploymentReachability\x12\x34\n\x10last_update_time\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp"\xb4\x01\n\x19\x43reateWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x35\n\x04spec\x18\x02 \x01(\x0b\x32\'.temporal.api.rules.v1.WorkflowRuleSpec\x12\x12\n\nforce_scan\x18\x03 \x01(\x08\x12\x12\n\nrequest_id\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x06 \x01(\t"_\n\x1a\x43reateWorkflowRuleResponse\x12\x31\n\x04rule\x18\x01 \x01(\x0b\x32#.temporal.api.rules.v1.WorkflowRule\x12\x0e\n\x06job_id\x18\x02 \x01(\t"A\n\x1b\x44\x65scribeWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0f\n\x07rule_id\x18\x02 \x01(\t"Q\n\x1c\x44\x65scribeWorkflowRuleResponse\x12\x31\n\x04rule\x18\x01 \x01(\x0b\x32#.temporal.api.rules.v1.WorkflowRule"?\n\x19\x44\x65leteWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x0f\n\x07rule_id\x18\x02 \x01(\t"\x1c\n\x1a\x44\x65leteWorkflowRuleResponse"F\n\x18ListWorkflowRulesRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"h\n\x19ListWorkflowRulesResponse\x12\x32\n\x05rules\x18\x01 \x03(\x0b\x32#.temporal.api.rules.v1.WorkflowRule\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\xce\x01\n\x1aTriggerWorkflowRuleRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12<\n\texecution\x18\x02 \x01(\x0b\x32).temporal.api.common.v1.WorkflowExecution\x12\x0c\n\x02id\x18\x04 \x01(\tH\x00\x12\x37\n\x04spec\x18\x05 \x01(\x0b\x32\'.temporal.api.rules.v1.WorkflowRuleSpecH\x00\x12\x10\n\x08identity\x18\x03 \x01(\tB\x06\n\x04rule".\n\x1bTriggerWorkflowRuleResponse\x12\x0f\n\x07\x61pplied\x18\x01 \x01(\x08"\x86\x01\n\x1cRecordWorkerHeartbeatRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x41\n\x10worker_heartbeat\x18\x03 \x03(\x0b\x32\'.temporal.api.worker.v1.WorkerHeartbeat"\x1f\n\x1dRecordWorkerHeartbeatResponse"b\n\x12ListWorkersRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"h\n\x13ListWorkersResponse\x12\x38\n\x0cworkers_info\x18\x01 \x03(\x0b\x32".temporal.api.worker.v1.WorkerInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"\xe2\x03\n\x1cUpdateTaskQueueConfigRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\ntask_queue\x18\x03 \x01(\t\x12=\n\x0ftask_queue_type\x18\x04 \x01(\x0e\x32$.temporal.api.enums.v1.TaskQueueType\x12n\n\x17update_queue_rate_limit\x18\x05 \x01(\x0b\x32M.temporal.api.workflowservice.v1.UpdateTaskQueueConfigRequest.RateLimitUpdate\x12}\n&update_fairness_key_rate_limit_default\x18\x06 \x01(\x0b\x32M.temporal.api.workflowservice.v1.UpdateTaskQueueConfigRequest.RateLimitUpdate\x1a[\n\x0fRateLimitUpdate\x12\x38\n\nrate_limit\x18\x01 \x01(\x0b\x32$.temporal.api.taskqueue.v1.RateLimit\x12\x0e\n\x06reason\x18\x02 \x01(\t"[\n\x1dUpdateTaskQueueConfigResponse\x12:\n\x06\x63onfig\x18\x01 \x01(\x0b\x32*.temporal.api.taskqueue.v1.TaskQueueConfig"\x89\x01\n\x18\x46\x65tchWorkerConfigRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x38\n\x08selector\x18\x06 \x01(\x0b\x32&.temporal.api.common.v1.WorkerSelector"U\n\x19\x46\x65tchWorkerConfigResponse\x12\x38\n\rworker_config\x18\x01 \x01(\x0b\x32!.temporal.api.sdk.v1.WorkerConfig"\xf5\x01\n\x19UpdateWorkerConfigRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x38\n\rworker_config\x18\x04 \x01(\x0b\x32!.temporal.api.sdk.v1.WorkerConfig\x12/\n\x0bupdate_mask\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.FieldMask\x12\x38\n\x08selector\x18\x06 \x01(\x0b\x32&.temporal.api.common.v1.WorkerSelector"d\n\x1aUpdateWorkerConfigResponse\x12:\n\rworker_config\x18\x01 \x01(\x0b\x32!.temporal.api.sdk.v1.WorkerConfigH\x00\x42\n\n\x08response"G\n\x15\x44\x65scribeWorkerRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x1b\n\x13worker_instance_key\x18\x02 \x01(\t"Q\n\x16\x44\x65scribeWorkerResponse\x12\x37\n\x0bworker_info\x18\x01 \x01(\x0b\x32".temporal.api.worker.v1.WorkerInfo"\x82\x07\n\x1dStartActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x10\n\x08identity\x18\x02 \x01(\t\x12\x12\n\nrequest_id\x18\x03 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x04 \x01(\t\x12;\n\ractivity_type\x18\x05 \x01(\x0b\x32$.temporal.api.common.v1.ActivityType\x12:\n\x07options\x18\x06 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions\x12/\n\x05input\x18\x07 \x01(\x0b\x32 .temporal.api.common.v1.Payloads\x12=\n\x0fid_reuse_policy\x18\x08 \x01(\x0e\x32$.temporal.api.enums.v1.IdReusePolicy\x12\x43\n\x12id_conflict_policy\x18\t \x01(\x0e\x32\'.temporal.api.enums.v1.IdConflictPolicy\x12*\n\x04memo\x18\n \x01(\x0b\x32\x1c.temporal.api.common.v1.Memo\x12\x43\n\x11search_attributes\x18\x0b \x01(\x0b\x32(.temporal.api.common.v1.SearchAttributes\x12.\n\x06header\x18\x0c \x01(\x0b\x32\x1e.temporal.api.common.v1.Header\x12\x1f\n\x17request_eager_execution\x18\r \x01(\x08\x12>\n\x14\x63ompletion_callbacks\x18\x0e \x03(\x0b\x32 .temporal.api.common.v1.Callback\x12\x38\n\ruser_metadata\x18\x0f \x01(\x0b\x32!.temporal.api.sdk.v1.UserMetadata\x12+\n\x05links\x18\x10 \x03(\x0b\x32\x1c.temporal.api.common.v1.Link\x12H\n\x13on_conflict_options\x18\x11 \x01(\x0b\x32+.temporal.api.activity.v1.OnConflictOptions\x12\x32\n\x08priority\x18\x12 \x01(\x0b\x32 .temporal.api.common.v1.Priority"\xc1\x01\n\x1eStartActivityExecutionResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t\x12\x0f\n\x07started\x18\x02 \x01(\x08\x12R\n\neager_task\x18\x03 \x01(\x0b\x32>.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse\x12*\n\x04link\x18\x04 \x01(\x0b\x32\x1c.temporal.api.common.v1.Link"\x8a\x01\n DescribeActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x15\n\rinclude_input\x18\x04 \x01(\x08\x12\x17\n\x0flong_poll_token\x18\x05 \x01(\x0c"{\n!DescribeActivityExecutionResponse\x12=\n\x04info\x18\x01 \x01(\x0b\x32/.temporal.api.activity.v1.ActivityExecutionInfo\x12\x17\n\x0flong_poll_token\x18\x02 \x01(\x0c"m\n\x1dListActivityExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x11\n\tpage_size\x18\x02 \x01(\x05\x12\x17\n\x0fnext_page_token\x18\x03 \x01(\x0c\x12\r\n\x05query\x18\x04 \x01(\t"y\n\x1eListActivityExecutionsResponse\x12>\n\nexecutions\x18\x01 \x03(\x0b\x32*.temporal.api.activity.v1.ActivityListInfo\x12\x17\n\x0fnext_page_token\x18\x02 \x01(\x0c"B\n\x1e\x43ountActivityExecutionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\r\n\x05query\x18\x02 \x01(\t"\xed\x01\n\x1f\x43ountActivityExecutionsResponse\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\x12\x61\n\x06groups\x18\x02 \x03(\x0b\x32Q.temporal.api.workflowservice.v1.CountActivityExecutionsResponse.AggregationGroup\x1aX\n\x10\x41ggregationGroup\x12\x35\n\x0cgroup_values\x18\x01 \x03(\x0b\x32\x1f.temporal.api.common.v1.Payload\x12\r\n\x05\x63ount\x18\x02 \x01(\x03"i\n!GetActivityExecutionResultRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x0c\n\x04wait\x18\x04 \x01(\x08"\xa8\x01\n"GetActivityExecutionResultResponse\x12\x0e\n\x06run_id\x18\x01 \x01(\t\x12\x32\n\x06result\x18\x02 \x01(\x0b\x32 .temporal.api.common.v1.PayloadsH\x00\x12\x33\n\x07\x66\x61ilure\x18\x03 \x01(\x0b\x32 .temporal.api.failure.v1.FailureH\x00\x42\t\n\x07outcome"\x95\x01\n%RequestCancelActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x10\n\x08identity\x18\x04 \x01(\t\x12\x12\n\nrequest_id\x18\x05 \x01(\t\x12\x0e\n\x06reason\x18\x06 \x01(\t"(\n&RequestCancelActivityExecutionResponse"}\n!TerminateActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t\x12\x0e\n\x06reason\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t"$\n"TerminateActivityExecutionResponse"X\n\x1e\x44\x65leteActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x02 \x01(\t\x12\x0e\n\x06run_id\x18\x03 \x01(\t"!\n\x1f\x44\x65leteActivityExecutionResponse"\xd7\x02\n%UpdateActivityExecutionOptionsRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x03 \x01(\t\x12\x0e\n\x06run_id\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x1e\n\x16workflow_activity_type\x18\x06 \x01(\t\x12\x1f\n\x17\x61ll_workflow_activities\x18\x07 \x01(\x08\x12\x43\n\x10\x61\x63tivity_options\x18\x08 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions\x12/\n\x0bupdate_mask\x18\t \x01(\x0b\x32\x1a.google.protobuf.FieldMask\x12\x18\n\x10restore_original\x18\n \x01(\x08"m\n&UpdateActivityExecutionOptionsResponse\x12\x43\n\x10\x61\x63tivity_options\x18\x01 \x01(\x0b\x32).temporal.api.activity.v1.ActivityOptions"\xae\x01\n\x1dPauseActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x03 \x01(\t\x12\x0e\n\x06run_id\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x1e\n\x16workflow_activity_type\x18\x06 \x01(\t\x12\x0e\n\x06reason\x18\x07 \x01(\t" \n\x1ePauseActivityExecutionResponse"\x9d\x02\n\x1fUnpauseActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x03 \x01(\t\x12\x0e\n\x06run_id\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x1e\n\x16workflow_activity_type\x18\x06 \x01(\t\x12\x1f\n\x17\x61ll_workflow_activities\x18\x07 \x01(\x08\x12\x16\n\x0ereset_attempts\x18\x08 \x01(\x08\x12\x17\n\x0freset_heartbeat\x18\t \x01(\x08\x12)\n\x06jitter\x18\n \x01(\x0b\x32\x19.google.protobuf.Duration""\n UnpauseActivityExecutionResponse"\xba\x02\n\x1dResetActivityExecutionRequest\x12\x11\n\tnamespace\x18\x01 \x01(\t\x12\x13\n\x0bworkflow_id\x18\x02 \x01(\t\x12\x13\n\x0b\x61\x63tivity_id\x18\x03 \x01(\t\x12\x0e\n\x06run_id\x18\x04 \x01(\t\x12\x10\n\x08identity\x18\x05 \x01(\t\x12\x1e\n\x16workflow_activity_type\x18\x06 \x01(\t\x12\x1f\n\x17\x61ll_workflow_activities\x18\x07 \x01(\x08\x12\x17\n\x0freset_heartbeat\x18\x08 \x01(\x08\x12\x13\n\x0bkeep_paused\x18\t \x01(\x08\x12)\n\x06jitter\x18\n \x01(\x0b\x32\x19.google.protobuf.Duration\x12 \n\x18restore_original_options\x18\x0b \x01(\x08" \n\x1eResetActivityExecutionResponseB\xbe\x01\n"io.temporal.api.workflowservice.v1B\x14RequestResponseProtoP\x01Z5go.temporal.io/api/workflowservice/v1;workflowservice\xaa\x02!Temporalio.Api.WorkflowService.V1\xea\x02$Temporalio::Api::WorkflowService::V1b\x06proto3' ) @@ -629,6 +632,12 @@ _UPDATEWORKERDEPLOYMENTVERSIONMETADATARESPONSE = DESCRIPTOR.message_types_by_name[ "UpdateWorkerDeploymentVersionMetadataResponse" ] +_SETWORKERDEPLOYMENTMANAGERREQUEST = DESCRIPTOR.message_types_by_name[ + "SetWorkerDeploymentManagerRequest" +] +_SETWORKERDEPLOYMENTMANAGERRESPONSE = DESCRIPTOR.message_types_by_name[ + "SetWorkerDeploymentManagerResponse" +] _GETCURRENTDEPLOYMENTREQUEST = DESCRIPTOR.message_types_by_name[ "GetCurrentDeploymentRequest" ] @@ -696,6 +705,83 @@ _UPDATEWORKERCONFIGRESPONSE = DESCRIPTOR.message_types_by_name[ "UpdateWorkerConfigResponse" ] +_DESCRIBEWORKERREQUEST = DESCRIPTOR.message_types_by_name["DescribeWorkerRequest"] +_DESCRIBEWORKERRESPONSE = DESCRIPTOR.message_types_by_name["DescribeWorkerResponse"] +_STARTACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "StartActivityExecutionRequest" +] +_STARTACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "StartActivityExecutionResponse" +] +_DESCRIBEACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "DescribeActivityExecutionRequest" +] +_DESCRIBEACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "DescribeActivityExecutionResponse" +] +_LISTACTIVITYEXECUTIONSREQUEST = DESCRIPTOR.message_types_by_name[ + "ListActivityExecutionsRequest" +] +_LISTACTIVITYEXECUTIONSRESPONSE = DESCRIPTOR.message_types_by_name[ + "ListActivityExecutionsResponse" +] +_COUNTACTIVITYEXECUTIONSREQUEST = DESCRIPTOR.message_types_by_name[ + "CountActivityExecutionsRequest" +] +_COUNTACTIVITYEXECUTIONSRESPONSE = DESCRIPTOR.message_types_by_name[ + "CountActivityExecutionsResponse" +] +_COUNTACTIVITYEXECUTIONSRESPONSE_AGGREGATIONGROUP = ( + _COUNTACTIVITYEXECUTIONSRESPONSE.nested_types_by_name["AggregationGroup"] +) +_GETACTIVITYEXECUTIONRESULTREQUEST = DESCRIPTOR.message_types_by_name[ + "GetActivityExecutionResultRequest" +] +_GETACTIVITYEXECUTIONRESULTRESPONSE = DESCRIPTOR.message_types_by_name[ + "GetActivityExecutionResultResponse" +] +_REQUESTCANCELACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "RequestCancelActivityExecutionRequest" +] +_REQUESTCANCELACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "RequestCancelActivityExecutionResponse" +] +_TERMINATEACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "TerminateActivityExecutionRequest" +] +_TERMINATEACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "TerminateActivityExecutionResponse" +] +_DELETEACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "DeleteActivityExecutionRequest" +] +_DELETEACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "DeleteActivityExecutionResponse" +] +_UPDATEACTIVITYEXECUTIONOPTIONSREQUEST = DESCRIPTOR.message_types_by_name[ + "UpdateActivityExecutionOptionsRequest" +] +_UPDATEACTIVITYEXECUTIONOPTIONSRESPONSE = DESCRIPTOR.message_types_by_name[ + "UpdateActivityExecutionOptionsResponse" +] +_PAUSEACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "PauseActivityExecutionRequest" +] +_PAUSEACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "PauseActivityExecutionResponse" +] +_UNPAUSEACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "UnpauseActivityExecutionRequest" +] +_UNPAUSEACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "UnpauseActivityExecutionResponse" +] +_RESETACTIVITYEXECUTIONREQUEST = DESCRIPTOR.message_types_by_name[ + "ResetActivityExecutionRequest" +] +_RESETACTIVITYEXECUTIONRESPONSE = DESCRIPTOR.message_types_by_name[ + "ResetActivityExecutionResponse" +] RegisterNamespaceRequest = _reflection.GeneratedProtocolMessageType( "RegisterNamespaceRequest", (_message.Message,), @@ -2726,6 +2812,28 @@ ) _sym_db.RegisterMessage(UpdateWorkerDeploymentVersionMetadataResponse) +SetWorkerDeploymentManagerRequest = _reflection.GeneratedProtocolMessageType( + "SetWorkerDeploymentManagerRequest", + (_message.Message,), + { + "DESCRIPTOR": _SETWORKERDEPLOYMENTMANAGERREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.SetWorkerDeploymentManagerRequest) + }, +) +_sym_db.RegisterMessage(SetWorkerDeploymentManagerRequest) + +SetWorkerDeploymentManagerResponse = _reflection.GeneratedProtocolMessageType( + "SetWorkerDeploymentManagerResponse", + (_message.Message,), + { + "DESCRIPTOR": _SETWORKERDEPLOYMENTMANAGERRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.SetWorkerDeploymentManagerResponse) + }, +) +_sym_db.RegisterMessage(SetWorkerDeploymentManagerResponse) + GetCurrentDeploymentRequest = _reflection.GeneratedProtocolMessageType( "GetCurrentDeploymentRequest", (_message.Message,), @@ -3000,6 +3108,302 @@ ) _sym_db.RegisterMessage(UpdateWorkerConfigResponse) +DescribeWorkerRequest = _reflection.GeneratedProtocolMessageType( + "DescribeWorkerRequest", + (_message.Message,), + { + "DESCRIPTOR": _DESCRIBEWORKERREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.DescribeWorkerRequest) + }, +) +_sym_db.RegisterMessage(DescribeWorkerRequest) + +DescribeWorkerResponse = _reflection.GeneratedProtocolMessageType( + "DescribeWorkerResponse", + (_message.Message,), + { + "DESCRIPTOR": _DESCRIBEWORKERRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.DescribeWorkerResponse) + }, +) +_sym_db.RegisterMessage(DescribeWorkerResponse) + +StartActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "StartActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _STARTACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.StartActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(StartActivityExecutionRequest) + +StartActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "StartActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _STARTACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.StartActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(StartActivityExecutionResponse) + +DescribeActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "DescribeActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _DESCRIBEACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.DescribeActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(DescribeActivityExecutionRequest) + +DescribeActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "DescribeActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _DESCRIBEACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.DescribeActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(DescribeActivityExecutionResponse) + +ListActivityExecutionsRequest = _reflection.GeneratedProtocolMessageType( + "ListActivityExecutionsRequest", + (_message.Message,), + { + "DESCRIPTOR": _LISTACTIVITYEXECUTIONSREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.ListActivityExecutionsRequest) + }, +) +_sym_db.RegisterMessage(ListActivityExecutionsRequest) + +ListActivityExecutionsResponse = _reflection.GeneratedProtocolMessageType( + "ListActivityExecutionsResponse", + (_message.Message,), + { + "DESCRIPTOR": _LISTACTIVITYEXECUTIONSRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.ListActivityExecutionsResponse) + }, +) +_sym_db.RegisterMessage(ListActivityExecutionsResponse) + +CountActivityExecutionsRequest = _reflection.GeneratedProtocolMessageType( + "CountActivityExecutionsRequest", + (_message.Message,), + { + "DESCRIPTOR": _COUNTACTIVITYEXECUTIONSREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.CountActivityExecutionsRequest) + }, +) +_sym_db.RegisterMessage(CountActivityExecutionsRequest) + +CountActivityExecutionsResponse = _reflection.GeneratedProtocolMessageType( + "CountActivityExecutionsResponse", + (_message.Message,), + { + "AggregationGroup": _reflection.GeneratedProtocolMessageType( + "AggregationGroup", + (_message.Message,), + { + "DESCRIPTOR": _COUNTACTIVITYEXECUTIONSRESPONSE_AGGREGATIONGROUP, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.CountActivityExecutionsResponse.AggregationGroup) + }, + ), + "DESCRIPTOR": _COUNTACTIVITYEXECUTIONSRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.CountActivityExecutionsResponse) + }, +) +_sym_db.RegisterMessage(CountActivityExecutionsResponse) +_sym_db.RegisterMessage(CountActivityExecutionsResponse.AggregationGroup) + +GetActivityExecutionResultRequest = _reflection.GeneratedProtocolMessageType( + "GetActivityExecutionResultRequest", + (_message.Message,), + { + "DESCRIPTOR": _GETACTIVITYEXECUTIONRESULTREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.GetActivityExecutionResultRequest) + }, +) +_sym_db.RegisterMessage(GetActivityExecutionResultRequest) + +GetActivityExecutionResultResponse = _reflection.GeneratedProtocolMessageType( + "GetActivityExecutionResultResponse", + (_message.Message,), + { + "DESCRIPTOR": _GETACTIVITYEXECUTIONRESULTRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.GetActivityExecutionResultResponse) + }, +) +_sym_db.RegisterMessage(GetActivityExecutionResultResponse) + +RequestCancelActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "RequestCancelActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _REQUESTCANCELACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.RequestCancelActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(RequestCancelActivityExecutionRequest) + +RequestCancelActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "RequestCancelActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _REQUESTCANCELACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.RequestCancelActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(RequestCancelActivityExecutionResponse) + +TerminateActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "TerminateActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _TERMINATEACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.TerminateActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(TerminateActivityExecutionRequest) + +TerminateActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "TerminateActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _TERMINATEACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.TerminateActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(TerminateActivityExecutionResponse) + +DeleteActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "DeleteActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _DELETEACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.DeleteActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(DeleteActivityExecutionRequest) + +DeleteActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "DeleteActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _DELETEACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.DeleteActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(DeleteActivityExecutionResponse) + +UpdateActivityExecutionOptionsRequest = _reflection.GeneratedProtocolMessageType( + "UpdateActivityExecutionOptionsRequest", + (_message.Message,), + { + "DESCRIPTOR": _UPDATEACTIVITYEXECUTIONOPTIONSREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.UpdateActivityExecutionOptionsRequest) + }, +) +_sym_db.RegisterMessage(UpdateActivityExecutionOptionsRequest) + +UpdateActivityExecutionOptionsResponse = _reflection.GeneratedProtocolMessageType( + "UpdateActivityExecutionOptionsResponse", + (_message.Message,), + { + "DESCRIPTOR": _UPDATEACTIVITYEXECUTIONOPTIONSRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.UpdateActivityExecutionOptionsResponse) + }, +) +_sym_db.RegisterMessage(UpdateActivityExecutionOptionsResponse) + +PauseActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "PauseActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _PAUSEACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.PauseActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(PauseActivityExecutionRequest) + +PauseActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "PauseActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _PAUSEACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.PauseActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(PauseActivityExecutionResponse) + +UnpauseActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "UnpauseActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _UNPAUSEACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.UnpauseActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(UnpauseActivityExecutionRequest) + +UnpauseActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "UnpauseActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _UNPAUSEACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.UnpauseActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(UnpauseActivityExecutionResponse) + +ResetActivityExecutionRequest = _reflection.GeneratedProtocolMessageType( + "ResetActivityExecutionRequest", + (_message.Message,), + { + "DESCRIPTOR": _RESETACTIVITYEXECUTIONREQUEST, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.ResetActivityExecutionRequest) + }, +) +_sym_db.RegisterMessage(ResetActivityExecutionRequest) + +ResetActivityExecutionResponse = _reflection.GeneratedProtocolMessageType( + "ResetActivityExecutionResponse", + (_message.Message,), + { + "DESCRIPTOR": _RESETACTIVITYEXECUTIONRESPONSE, + "__module__": "temporal.api.workflowservice.v1.request_response_pb2", + # @@protoc_insertion_point(class_scope:temporal.api.workflowservice.v1.ResetActivityExecutionResponse) + }, +) +_sym_db.RegisterMessage(ResetActivityExecutionResponse) + if _descriptor._USE_C_DESCRIPTORS == False: DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b'\n"io.temporal.api.workflowservice.v1B\024RequestResponseProtoP\001Z5go.temporal.io/api/workflowservice/v1;workflowservice\252\002!Temporalio.Api.WorkflowService.V1\352\002$Temporalio::Api::WorkflowService::V1' @@ -3179,436 +3583,494 @@ _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST.fields_by_name[ "version" ]._serialized_options = b"\030\001" - _REGISTERNAMESPACEREQUEST._serialized_start = 1492 - _REGISTERNAMESPACEREQUEST._serialized_end = 2140 - _REGISTERNAMESPACEREQUEST_DATAENTRY._serialized_start = 2097 - _REGISTERNAMESPACEREQUEST_DATAENTRY._serialized_end = 2140 - _REGISTERNAMESPACERESPONSE._serialized_start = 2142 - _REGISTERNAMESPACERESPONSE._serialized_end = 2169 - _LISTNAMESPACESREQUEST._serialized_start = 2172 - _LISTNAMESPACESREQUEST._serialized_end = 2309 - _LISTNAMESPACESRESPONSE._serialized_start = 2312 - _LISTNAMESPACESRESPONSE._serialized_end = 2441 - _DESCRIBENAMESPACEREQUEST._serialized_start = 2443 - _DESCRIBENAMESPACEREQUEST._serialized_end = 2500 - _DESCRIBENAMESPACERESPONSE._serialized_start = 2503 - _DESCRIBENAMESPACERESPONSE._serialized_end = 2867 - _UPDATENAMESPACEREQUEST._serialized_start = 2870 - _UPDATENAMESPACEREQUEST._serialized_end = 3205 - _UPDATENAMESPACERESPONSE._serialized_start = 3208 - _UPDATENAMESPACERESPONSE._serialized_end = 3499 - _DEPRECATENAMESPACEREQUEST._serialized_start = 3501 - _DEPRECATENAMESPACEREQUEST._serialized_end = 3571 - _DEPRECATENAMESPACERESPONSE._serialized_start = 3573 - _DEPRECATENAMESPACERESPONSE._serialized_end = 3601 - _STARTWORKFLOWEXECUTIONREQUEST._serialized_start = 3604 - _STARTWORKFLOWEXECUTIONREQUEST._serialized_end = 5053 - _STARTWORKFLOWEXECUTIONRESPONSE._serialized_start = 5056 - _STARTWORKFLOWEXECUTIONRESPONSE._serialized_end = 5322 - _GETWORKFLOWEXECUTIONHISTORYREQUEST._serialized_start = 5325 - _GETWORKFLOWEXECUTIONHISTORYREQUEST._serialized_end = 5623 - _GETWORKFLOWEXECUTIONHISTORYRESPONSE._serialized_start = 5626 - _GETWORKFLOWEXECUTIONHISTORYRESPONSE._serialized_end = 5812 - _GETWORKFLOWEXECUTIONHISTORYREVERSEREQUEST._serialized_start = 5815 - _GETWORKFLOWEXECUTIONHISTORYREVERSEREQUEST._serialized_end = 5991 - _GETWORKFLOWEXECUTIONHISTORYREVERSERESPONSE._serialized_start = 5993 - _GETWORKFLOWEXECUTIONHISTORYREVERSERESPONSE._serialized_end = 6113 - _POLLWORKFLOWTASKQUEUEREQUEST._serialized_start = 6116 - _POLLWORKFLOWTASKQUEUEREQUEST._serialized_end = 6510 - _POLLWORKFLOWTASKQUEUERESPONSE._serialized_start = 6513 - _POLLWORKFLOWTASKQUEUERESPONSE._serialized_end = 7426 - _POLLWORKFLOWTASKQUEUERESPONSE_QUERIESENTRY._serialized_start = 7342 - _POLLWORKFLOWTASKQUEUERESPONSE_QUERIESENTRY._serialized_end = 7426 - _RESPONDWORKFLOWTASKCOMPLETEDREQUEST._serialized_start = 7429 - _RESPONDWORKFLOWTASKCOMPLETEDREQUEST._serialized_end = 8634 - _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_QUERYRESULTSENTRY._serialized_start = 8468 - _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_QUERYRESULTSENTRY._serialized_end = 8563 - _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_CAPABILITIES._serialized_start = 8565 - _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_CAPABILITIES._serialized_end = 8634 - _RESPONDWORKFLOWTASKCOMPLETEDRESPONSE._serialized_start = 8637 - _RESPONDWORKFLOWTASKCOMPLETEDRESPONSE._serialized_end = 8882 - _RESPONDWORKFLOWTASKFAILEDREQUEST._serialized_start = 8885 - _RESPONDWORKFLOWTASKFAILEDREQUEST._serialized_end = 9389 - _RESPONDWORKFLOWTASKFAILEDRESPONSE._serialized_start = 9391 - _RESPONDWORKFLOWTASKFAILEDRESPONSE._serialized_end = 9426 - _POLLACTIVITYTASKQUEUEREQUEST._serialized_start = 9429 - _POLLACTIVITYTASKQUEUEREQUEST._serialized_end = 9869 - _POLLACTIVITYTASKQUEUERESPONSE._serialized_start = 9872 - _POLLACTIVITYTASKQUEUERESPONSE._serialized_end = 10879 - _RECORDACTIVITYTASKHEARTBEATREQUEST._serialized_start = 10882 - _RECORDACTIVITYTASKHEARTBEATREQUEST._serialized_end = 11026 - _RECORDACTIVITYTASKHEARTBEATRESPONSE._serialized_start = 11028 - _RECORDACTIVITYTASKHEARTBEATRESPONSE._serialized_end = 11140 - _RECORDACTIVITYTASKHEARTBEATBYIDREQUEST._serialized_start = 11143 - _RECORDACTIVITYTASKHEARTBEATBYIDREQUEST._serialized_end = 11329 - _RECORDACTIVITYTASKHEARTBEATBYIDRESPONSE._serialized_start = 11331 - _RECORDACTIVITYTASKHEARTBEATBYIDRESPONSE._serialized_end = 11447 - _RESPONDACTIVITYTASKCOMPLETEDREQUEST._serialized_start = 11450 - _RESPONDACTIVITYTASKCOMPLETEDREQUEST._serialized_end = 11811 - _RESPONDACTIVITYTASKCOMPLETEDRESPONSE._serialized_start = 11813 - _RESPONDACTIVITYTASKCOMPLETEDRESPONSE._serialized_end = 11851 - _RESPONDACTIVITYTASKCOMPLETEDBYIDREQUEST._serialized_start = 11854 - _RESPONDACTIVITYTASKCOMPLETEDBYIDREQUEST._serialized_end = 12040 - _RESPONDACTIVITYTASKCOMPLETEDBYIDRESPONSE._serialized_start = 12042 - _RESPONDACTIVITYTASKCOMPLETEDBYIDRESPONSE._serialized_end = 12084 - _RESPONDACTIVITYTASKFAILEDREQUEST._serialized_start = 12087 - _RESPONDACTIVITYTASKFAILEDREQUEST._serialized_end = 12512 - _RESPONDACTIVITYTASKFAILEDRESPONSE._serialized_start = 12514 - _RESPONDACTIVITYTASKFAILEDRESPONSE._serialized_end = 12601 - _RESPONDACTIVITYTASKFAILEDBYIDREQUEST._serialized_start = 12604 - _RESPONDACTIVITYTASKFAILEDBYIDREQUEST._serialized_end = 12854 - _RESPONDACTIVITYTASKFAILEDBYIDRESPONSE._serialized_start = 12856 - _RESPONDACTIVITYTASKFAILEDBYIDRESPONSE._serialized_end = 12947 - _RESPONDACTIVITYTASKCANCELEDREQUEST._serialized_start = 12950 - _RESPONDACTIVITYTASKCANCELEDREQUEST._serialized_end = 13311 - _RESPONDACTIVITYTASKCANCELEDRESPONSE._serialized_start = 13313 - _RESPONDACTIVITYTASKCANCELEDRESPONSE._serialized_end = 13350 - _RESPONDACTIVITYTASKCANCELEDBYIDREQUEST._serialized_start = 13353 - _RESPONDACTIVITYTASKCANCELEDBYIDREQUEST._serialized_end = 13620 - _RESPONDACTIVITYTASKCANCELEDBYIDRESPONSE._serialized_start = 13622 - _RESPONDACTIVITYTASKCANCELEDBYIDRESPONSE._serialized_end = 13663 - _REQUESTCANCELWORKFLOWEXECUTIONREQUEST._serialized_start = 13666 - _REQUESTCANCELWORKFLOWEXECUTIONREQUEST._serialized_end = 13926 - _REQUESTCANCELWORKFLOWEXECUTIONRESPONSE._serialized_start = 13928 - _REQUESTCANCELWORKFLOWEXECUTIONRESPONSE._serialized_end = 13968 - _SIGNALWORKFLOWEXECUTIONREQUEST._serialized_start = 13971 - _SIGNALWORKFLOWEXECUTIONREQUEST._serialized_end = 14321 - _SIGNALWORKFLOWEXECUTIONRESPONSE._serialized_start = 14323 - _SIGNALWORKFLOWEXECUTIONRESPONSE._serialized_end = 14356 - _SIGNALWITHSTARTWORKFLOWEXECUTIONREQUEST._serialized_start = 14359 - _SIGNALWITHSTARTWORKFLOWEXECUTIONREQUEST._serialized_end = 15624 - _SIGNALWITHSTARTWORKFLOWEXECUTIONRESPONSE._serialized_start = 15626 - _SIGNALWITHSTARTWORKFLOWEXECUTIONRESPONSE._serialized_end = 15701 - _RESETWORKFLOWEXECUTIONREQUEST._serialized_start = 15704 - _RESETWORKFLOWEXECUTIONREQUEST._serialized_end = 16153 - _RESETWORKFLOWEXECUTIONRESPONSE._serialized_start = 16155 - _RESETWORKFLOWEXECUTIONRESPONSE._serialized_end = 16203 - _TERMINATEWORKFLOWEXECUTIONREQUEST._serialized_start = 16206 - _TERMINATEWORKFLOWEXECUTIONREQUEST._serialized_end = 16493 - _TERMINATEWORKFLOWEXECUTIONRESPONSE._serialized_start = 16495 - _TERMINATEWORKFLOWEXECUTIONRESPONSE._serialized_end = 16531 - _DELETEWORKFLOWEXECUTIONREQUEST._serialized_start = 16533 - _DELETEWORKFLOWEXECUTIONREQUEST._serialized_end = 16655 - _DELETEWORKFLOWEXECUTIONRESPONSE._serialized_start = 16657 - _DELETEWORKFLOWEXECUTIONRESPONSE._serialized_end = 16690 - _LISTOPENWORKFLOWEXECUTIONSREQUEST._serialized_start = 16693 - _LISTOPENWORKFLOWEXECUTIONSREQUEST._serialized_end = 17022 - _LISTOPENWORKFLOWEXECUTIONSRESPONSE._serialized_start = 17025 - _LISTOPENWORKFLOWEXECUTIONSRESPONSE._serialized_end = 17155 - _LISTCLOSEDWORKFLOWEXECUTIONSREQUEST._serialized_start = 17158 - _LISTCLOSEDWORKFLOWEXECUTIONSREQUEST._serialized_end = 17552 - _LISTCLOSEDWORKFLOWEXECUTIONSRESPONSE._serialized_start = 17555 - _LISTCLOSEDWORKFLOWEXECUTIONSRESPONSE._serialized_end = 17687 - _LISTWORKFLOWEXECUTIONSREQUEST._serialized_start = 17689 - _LISTWORKFLOWEXECUTIONSREQUEST._serialized_end = 17798 - _LISTWORKFLOWEXECUTIONSRESPONSE._serialized_start = 17800 - _LISTWORKFLOWEXECUTIONSRESPONSE._serialized_end = 17926 - _LISTARCHIVEDWORKFLOWEXECUTIONSREQUEST._serialized_start = 17928 - _LISTARCHIVEDWORKFLOWEXECUTIONSREQUEST._serialized_end = 18045 - _LISTARCHIVEDWORKFLOWEXECUTIONSRESPONSE._serialized_start = 18048 - _LISTARCHIVEDWORKFLOWEXECUTIONSRESPONSE._serialized_end = 18182 - _SCANWORKFLOWEXECUTIONSREQUEST._serialized_start = 18184 - _SCANWORKFLOWEXECUTIONSREQUEST._serialized_end = 18293 - _SCANWORKFLOWEXECUTIONSRESPONSE._serialized_start = 18295 - _SCANWORKFLOWEXECUTIONSRESPONSE._serialized_end = 18421 - _COUNTWORKFLOWEXECUTIONSREQUEST._serialized_start = 18423 - _COUNTWORKFLOWEXECUTIONSREQUEST._serialized_end = 18489 - _COUNTWORKFLOWEXECUTIONSRESPONSE._serialized_start = 18492 - _COUNTWORKFLOWEXECUTIONSRESPONSE._serialized_end = 18729 - _COUNTWORKFLOWEXECUTIONSRESPONSE_AGGREGATIONGROUP._serialized_start = 18641 - _COUNTWORKFLOWEXECUTIONSRESPONSE_AGGREGATIONGROUP._serialized_end = 18729 - _GETSEARCHATTRIBUTESREQUEST._serialized_start = 18731 - _GETSEARCHATTRIBUTESREQUEST._serialized_end = 18759 - _GETSEARCHATTRIBUTESRESPONSE._serialized_start = 18762 - _GETSEARCHATTRIBUTESRESPONSE._serialized_end = 18963 - _GETSEARCHATTRIBUTESRESPONSE_KEYSENTRY._serialized_start = 18879 - _GETSEARCHATTRIBUTESRESPONSE_KEYSENTRY._serialized_end = 18963 - _RESPONDQUERYTASKCOMPLETEDREQUEST._serialized_start = 18966 - _RESPONDQUERYTASKCOMPLETEDREQUEST._serialized_end = 19302 - _RESPONDQUERYTASKCOMPLETEDRESPONSE._serialized_start = 19304 - _RESPONDQUERYTASKCOMPLETEDRESPONSE._serialized_end = 19339 - _RESETSTICKYTASKQUEUEREQUEST._serialized_start = 19341 - _RESETSTICKYTASKQUEUEREQUEST._serialized_end = 19451 - _RESETSTICKYTASKQUEUERESPONSE._serialized_start = 19453 - _RESETSTICKYTASKQUEUERESPONSE._serialized_end = 19483 - _SHUTDOWNWORKERREQUEST._serialized_start = 19486 - _SHUTDOWNWORKERREQUEST._serialized_end = 19656 - _SHUTDOWNWORKERRESPONSE._serialized_start = 19658 - _SHUTDOWNWORKERRESPONSE._serialized_end = 19682 - _QUERYWORKFLOWREQUEST._serialized_start = 19685 - _QUERYWORKFLOWREQUEST._serialized_end = 19918 - _QUERYWORKFLOWRESPONSE._serialized_start = 19921 - _QUERYWORKFLOWRESPONSE._serialized_end = 20062 - _DESCRIBEWORKFLOWEXECUTIONREQUEST._serialized_start = 20064 - _DESCRIBEWORKFLOWEXECUTIONREQUEST._serialized_end = 20179 - _DESCRIBEWORKFLOWEXECUTIONRESPONSE._serialized_start = 20182 - _DESCRIBEWORKFLOWEXECUTIONRESPONSE._serialized_end = 20847 - _DESCRIBETASKQUEUEREQUEST._serialized_start = 20850 - _DESCRIBETASKQUEUEREQUEST._serialized_end = 21378 - _DESCRIBETASKQUEUERESPONSE._serialized_start = 21381 - _DESCRIBETASKQUEUERESPONSE._serialized_end = 22385 - _DESCRIBETASKQUEUERESPONSE_STATSBYPRIORITYKEYENTRY._serialized_start = 22065 - _DESCRIBETASKQUEUERESPONSE_STATSBYPRIORITYKEYENTRY._serialized_end = 22165 - _DESCRIBETASKQUEUERESPONSE_EFFECTIVERATELIMIT._serialized_start = 22167 - _DESCRIBETASKQUEUERESPONSE_EFFECTIVERATELIMIT._serialized_end = 22283 - _DESCRIBETASKQUEUERESPONSE_VERSIONSINFOENTRY._serialized_start = 22285 - _DESCRIBETASKQUEUERESPONSE_VERSIONSINFOENTRY._serialized_end = 22385 - _GETCLUSTERINFOREQUEST._serialized_start = 22387 - _GETCLUSTERINFOREQUEST._serialized_end = 22410 - _GETCLUSTERINFORESPONSE._serialized_start = 22413 - _GETCLUSTERINFORESPONSE._serialized_end = 22808 - _GETCLUSTERINFORESPONSE_SUPPORTEDCLIENTSENTRY._serialized_start = 22753 - _GETCLUSTERINFORESPONSE_SUPPORTEDCLIENTSENTRY._serialized_end = 22808 - _GETSYSTEMINFOREQUEST._serialized_start = 22810 - _GETSYSTEMINFOREQUEST._serialized_end = 22832 - _GETSYSTEMINFORESPONSE._serialized_start = 22835 - _GETSYSTEMINFORESPONSE._serialized_end = 23335 - _GETSYSTEMINFORESPONSE_CAPABILITIES._serialized_start = 22976 - _GETSYSTEMINFORESPONSE_CAPABILITIES._serialized_end = 23335 - _LISTTASKQUEUEPARTITIONSREQUEST._serialized_start = 23337 - _LISTTASKQUEUEPARTITIONSREQUEST._serialized_end = 23446 - _LISTTASKQUEUEPARTITIONSRESPONSE._serialized_start = 23449 - _LISTTASKQUEUEPARTITIONSRESPONSE._serialized_end = 23672 - _CREATESCHEDULEREQUEST._serialized_start = 23675 - _CREATESCHEDULEREQUEST._serialized_end = 24007 - _CREATESCHEDULERESPONSE._serialized_start = 24009 - _CREATESCHEDULERESPONSE._serialized_end = 24057 - _DESCRIBESCHEDULEREQUEST._serialized_start = 24059 - _DESCRIBESCHEDULEREQUEST._serialized_end = 24124 - _DESCRIBESCHEDULERESPONSE._serialized_start = 24127 - _DESCRIBESCHEDULERESPONSE._serialized_end = 24398 - _UPDATESCHEDULEREQUEST._serialized_start = 24401 - _UPDATESCHEDULEREQUEST._serialized_end = 24649 - _UPDATESCHEDULERESPONSE._serialized_start = 24651 - _UPDATESCHEDULERESPONSE._serialized_end = 24675 - _PATCHSCHEDULEREQUEST._serialized_start = 24678 - _PATCHSCHEDULEREQUEST._serialized_end = 24834 - _PATCHSCHEDULERESPONSE._serialized_start = 24836 - _PATCHSCHEDULERESPONSE._serialized_end = 24859 - _LISTSCHEDULEMATCHINGTIMESREQUEST._serialized_start = 24862 - _LISTSCHEDULEMATCHINGTIMESREQUEST._serialized_end = 25030 - _LISTSCHEDULEMATCHINGTIMESRESPONSE._serialized_start = 25032 - _LISTSCHEDULEMATCHINGTIMESRESPONSE._serialized_end = 25115 - _DELETESCHEDULEREQUEST._serialized_start = 25117 - _DELETESCHEDULEREQUEST._serialized_end = 25198 - _DELETESCHEDULERESPONSE._serialized_start = 25200 - _DELETESCHEDULERESPONSE._serialized_end = 25224 - _LISTSCHEDULESREQUEST._serialized_start = 25226 - _LISTSCHEDULESREQUEST._serialized_end = 25334 - _LISTSCHEDULESRESPONSE._serialized_start = 25336 - _LISTSCHEDULESRESPONSE._serialized_end = 25448 - _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_start = 25451 - _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_end = 26097 - _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_ADDNEWCOMPATIBLEVERSION._serialized_start = 25898 + _REGISTERNAMESPACEREQUEST._serialized_start = 1524 + _REGISTERNAMESPACEREQUEST._serialized_end = 2172 + _REGISTERNAMESPACEREQUEST_DATAENTRY._serialized_start = 2129 + _REGISTERNAMESPACEREQUEST_DATAENTRY._serialized_end = 2172 + _REGISTERNAMESPACERESPONSE._serialized_start = 2174 + _REGISTERNAMESPACERESPONSE._serialized_end = 2201 + _LISTNAMESPACESREQUEST._serialized_start = 2204 + _LISTNAMESPACESREQUEST._serialized_end = 2341 + _LISTNAMESPACESRESPONSE._serialized_start = 2344 + _LISTNAMESPACESRESPONSE._serialized_end = 2473 + _DESCRIBENAMESPACEREQUEST._serialized_start = 2475 + _DESCRIBENAMESPACEREQUEST._serialized_end = 2532 + _DESCRIBENAMESPACERESPONSE._serialized_start = 2535 + _DESCRIBENAMESPACERESPONSE._serialized_end = 2899 + _UPDATENAMESPACEREQUEST._serialized_start = 2902 + _UPDATENAMESPACEREQUEST._serialized_end = 3237 + _UPDATENAMESPACERESPONSE._serialized_start = 3240 + _UPDATENAMESPACERESPONSE._serialized_end = 3531 + _DEPRECATENAMESPACEREQUEST._serialized_start = 3533 + _DEPRECATENAMESPACEREQUEST._serialized_end = 3603 + _DEPRECATENAMESPACERESPONSE._serialized_start = 3605 + _DEPRECATENAMESPACERESPONSE._serialized_end = 3633 + _STARTWORKFLOWEXECUTIONREQUEST._serialized_start = 3636 + _STARTWORKFLOWEXECUTIONREQUEST._serialized_end = 5085 + _STARTWORKFLOWEXECUTIONRESPONSE._serialized_start = 5088 + _STARTWORKFLOWEXECUTIONRESPONSE._serialized_end = 5354 + _GETWORKFLOWEXECUTIONHISTORYREQUEST._serialized_start = 5357 + _GETWORKFLOWEXECUTIONHISTORYREQUEST._serialized_end = 5655 + _GETWORKFLOWEXECUTIONHISTORYRESPONSE._serialized_start = 5658 + _GETWORKFLOWEXECUTIONHISTORYRESPONSE._serialized_end = 5844 + _GETWORKFLOWEXECUTIONHISTORYREVERSEREQUEST._serialized_start = 5847 + _GETWORKFLOWEXECUTIONHISTORYREVERSEREQUEST._serialized_end = 6023 + _GETWORKFLOWEXECUTIONHISTORYREVERSERESPONSE._serialized_start = 6025 + _GETWORKFLOWEXECUTIONHISTORYREVERSERESPONSE._serialized_end = 6145 + _POLLWORKFLOWTASKQUEUEREQUEST._serialized_start = 6148 + _POLLWORKFLOWTASKQUEUEREQUEST._serialized_end = 6542 + _POLLWORKFLOWTASKQUEUERESPONSE._serialized_start = 6545 + _POLLWORKFLOWTASKQUEUERESPONSE._serialized_end = 7458 + _POLLWORKFLOWTASKQUEUERESPONSE_QUERIESENTRY._serialized_start = 7374 + _POLLWORKFLOWTASKQUEUERESPONSE_QUERIESENTRY._serialized_end = 7458 + _RESPONDWORKFLOWTASKCOMPLETEDREQUEST._serialized_start = 7461 + _RESPONDWORKFLOWTASKCOMPLETEDREQUEST._serialized_end = 8666 + _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_QUERYRESULTSENTRY._serialized_start = 8500 + _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_QUERYRESULTSENTRY._serialized_end = 8595 + _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_CAPABILITIES._serialized_start = 8597 + _RESPONDWORKFLOWTASKCOMPLETEDREQUEST_CAPABILITIES._serialized_end = 8666 + _RESPONDWORKFLOWTASKCOMPLETEDRESPONSE._serialized_start = 8669 + _RESPONDWORKFLOWTASKCOMPLETEDRESPONSE._serialized_end = 8914 + _RESPONDWORKFLOWTASKFAILEDREQUEST._serialized_start = 8917 + _RESPONDWORKFLOWTASKFAILEDREQUEST._serialized_end = 9421 + _RESPONDWORKFLOWTASKFAILEDRESPONSE._serialized_start = 9423 + _RESPONDWORKFLOWTASKFAILEDRESPONSE._serialized_end = 9458 + _POLLACTIVITYTASKQUEUEREQUEST._serialized_start = 9461 + _POLLACTIVITYTASKQUEUEREQUEST._serialized_end = 9901 + _POLLACTIVITYTASKQUEUERESPONSE._serialized_start = 9904 + _POLLACTIVITYTASKQUEUERESPONSE._serialized_end = 10911 + _RECORDACTIVITYTASKHEARTBEATREQUEST._serialized_start = 10914 + _RECORDACTIVITYTASKHEARTBEATREQUEST._serialized_end = 11058 + _RECORDACTIVITYTASKHEARTBEATRESPONSE._serialized_start = 11060 + _RECORDACTIVITYTASKHEARTBEATRESPONSE._serialized_end = 11172 + _RECORDACTIVITYTASKHEARTBEATBYIDREQUEST._serialized_start = 11175 + _RECORDACTIVITYTASKHEARTBEATBYIDREQUEST._serialized_end = 11361 + _RECORDACTIVITYTASKHEARTBEATBYIDRESPONSE._serialized_start = 11363 + _RECORDACTIVITYTASKHEARTBEATBYIDRESPONSE._serialized_end = 11479 + _RESPONDACTIVITYTASKCOMPLETEDREQUEST._serialized_start = 11482 + _RESPONDACTIVITYTASKCOMPLETEDREQUEST._serialized_end = 11843 + _RESPONDACTIVITYTASKCOMPLETEDRESPONSE._serialized_start = 11845 + _RESPONDACTIVITYTASKCOMPLETEDRESPONSE._serialized_end = 11883 + _RESPONDACTIVITYTASKCOMPLETEDBYIDREQUEST._serialized_start = 11886 + _RESPONDACTIVITYTASKCOMPLETEDBYIDREQUEST._serialized_end = 12072 + _RESPONDACTIVITYTASKCOMPLETEDBYIDRESPONSE._serialized_start = 12074 + _RESPONDACTIVITYTASKCOMPLETEDBYIDRESPONSE._serialized_end = 12116 + _RESPONDACTIVITYTASKFAILEDREQUEST._serialized_start = 12119 + _RESPONDACTIVITYTASKFAILEDREQUEST._serialized_end = 12544 + _RESPONDACTIVITYTASKFAILEDRESPONSE._serialized_start = 12546 + _RESPONDACTIVITYTASKFAILEDRESPONSE._serialized_end = 12633 + _RESPONDACTIVITYTASKFAILEDBYIDREQUEST._serialized_start = 12636 + _RESPONDACTIVITYTASKFAILEDBYIDREQUEST._serialized_end = 12886 + _RESPONDACTIVITYTASKFAILEDBYIDRESPONSE._serialized_start = 12888 + _RESPONDACTIVITYTASKFAILEDBYIDRESPONSE._serialized_end = 12979 + _RESPONDACTIVITYTASKCANCELEDREQUEST._serialized_start = 12982 + _RESPONDACTIVITYTASKCANCELEDREQUEST._serialized_end = 13343 + _RESPONDACTIVITYTASKCANCELEDRESPONSE._serialized_start = 13345 + _RESPONDACTIVITYTASKCANCELEDRESPONSE._serialized_end = 13382 + _RESPONDACTIVITYTASKCANCELEDBYIDREQUEST._serialized_start = 13385 + _RESPONDACTIVITYTASKCANCELEDBYIDREQUEST._serialized_end = 13652 + _RESPONDACTIVITYTASKCANCELEDBYIDRESPONSE._serialized_start = 13654 + _RESPONDACTIVITYTASKCANCELEDBYIDRESPONSE._serialized_end = 13695 + _REQUESTCANCELWORKFLOWEXECUTIONREQUEST._serialized_start = 13698 + _REQUESTCANCELWORKFLOWEXECUTIONREQUEST._serialized_end = 13958 + _REQUESTCANCELWORKFLOWEXECUTIONRESPONSE._serialized_start = 13960 + _REQUESTCANCELWORKFLOWEXECUTIONRESPONSE._serialized_end = 14000 + _SIGNALWORKFLOWEXECUTIONREQUEST._serialized_start = 14003 + _SIGNALWORKFLOWEXECUTIONREQUEST._serialized_end = 14353 + _SIGNALWORKFLOWEXECUTIONRESPONSE._serialized_start = 14355 + _SIGNALWORKFLOWEXECUTIONRESPONSE._serialized_end = 14388 + _SIGNALWITHSTARTWORKFLOWEXECUTIONREQUEST._serialized_start = 14391 + _SIGNALWITHSTARTWORKFLOWEXECUTIONREQUEST._serialized_end = 15656 + _SIGNALWITHSTARTWORKFLOWEXECUTIONRESPONSE._serialized_start = 15658 + _SIGNALWITHSTARTWORKFLOWEXECUTIONRESPONSE._serialized_end = 15733 + _RESETWORKFLOWEXECUTIONREQUEST._serialized_start = 15736 + _RESETWORKFLOWEXECUTIONREQUEST._serialized_end = 16185 + _RESETWORKFLOWEXECUTIONRESPONSE._serialized_start = 16187 + _RESETWORKFLOWEXECUTIONRESPONSE._serialized_end = 16235 + _TERMINATEWORKFLOWEXECUTIONREQUEST._serialized_start = 16238 + _TERMINATEWORKFLOWEXECUTIONREQUEST._serialized_end = 16525 + _TERMINATEWORKFLOWEXECUTIONRESPONSE._serialized_start = 16527 + _TERMINATEWORKFLOWEXECUTIONRESPONSE._serialized_end = 16563 + _DELETEWORKFLOWEXECUTIONREQUEST._serialized_start = 16565 + _DELETEWORKFLOWEXECUTIONREQUEST._serialized_end = 16687 + _DELETEWORKFLOWEXECUTIONRESPONSE._serialized_start = 16689 + _DELETEWORKFLOWEXECUTIONRESPONSE._serialized_end = 16722 + _LISTOPENWORKFLOWEXECUTIONSREQUEST._serialized_start = 16725 + _LISTOPENWORKFLOWEXECUTIONSREQUEST._serialized_end = 17054 + _LISTOPENWORKFLOWEXECUTIONSRESPONSE._serialized_start = 17057 + _LISTOPENWORKFLOWEXECUTIONSRESPONSE._serialized_end = 17187 + _LISTCLOSEDWORKFLOWEXECUTIONSREQUEST._serialized_start = 17190 + _LISTCLOSEDWORKFLOWEXECUTIONSREQUEST._serialized_end = 17584 + _LISTCLOSEDWORKFLOWEXECUTIONSRESPONSE._serialized_start = 17587 + _LISTCLOSEDWORKFLOWEXECUTIONSRESPONSE._serialized_end = 17719 + _LISTWORKFLOWEXECUTIONSREQUEST._serialized_start = 17721 + _LISTWORKFLOWEXECUTIONSREQUEST._serialized_end = 17830 + _LISTWORKFLOWEXECUTIONSRESPONSE._serialized_start = 17832 + _LISTWORKFLOWEXECUTIONSRESPONSE._serialized_end = 17958 + _LISTARCHIVEDWORKFLOWEXECUTIONSREQUEST._serialized_start = 17960 + _LISTARCHIVEDWORKFLOWEXECUTIONSREQUEST._serialized_end = 18077 + _LISTARCHIVEDWORKFLOWEXECUTIONSRESPONSE._serialized_start = 18080 + _LISTARCHIVEDWORKFLOWEXECUTIONSRESPONSE._serialized_end = 18214 + _SCANWORKFLOWEXECUTIONSREQUEST._serialized_start = 18216 + _SCANWORKFLOWEXECUTIONSREQUEST._serialized_end = 18325 + _SCANWORKFLOWEXECUTIONSRESPONSE._serialized_start = 18327 + _SCANWORKFLOWEXECUTIONSRESPONSE._serialized_end = 18453 + _COUNTWORKFLOWEXECUTIONSREQUEST._serialized_start = 18455 + _COUNTWORKFLOWEXECUTIONSREQUEST._serialized_end = 18521 + _COUNTWORKFLOWEXECUTIONSRESPONSE._serialized_start = 18524 + _COUNTWORKFLOWEXECUTIONSRESPONSE._serialized_end = 18761 + _COUNTWORKFLOWEXECUTIONSRESPONSE_AGGREGATIONGROUP._serialized_start = 18673 + _COUNTWORKFLOWEXECUTIONSRESPONSE_AGGREGATIONGROUP._serialized_end = 18761 + _GETSEARCHATTRIBUTESREQUEST._serialized_start = 18763 + _GETSEARCHATTRIBUTESREQUEST._serialized_end = 18791 + _GETSEARCHATTRIBUTESRESPONSE._serialized_start = 18794 + _GETSEARCHATTRIBUTESRESPONSE._serialized_end = 18995 + _GETSEARCHATTRIBUTESRESPONSE_KEYSENTRY._serialized_start = 18911 + _GETSEARCHATTRIBUTESRESPONSE_KEYSENTRY._serialized_end = 18995 + _RESPONDQUERYTASKCOMPLETEDREQUEST._serialized_start = 18998 + _RESPONDQUERYTASKCOMPLETEDREQUEST._serialized_end = 19334 + _RESPONDQUERYTASKCOMPLETEDRESPONSE._serialized_start = 19336 + _RESPONDQUERYTASKCOMPLETEDRESPONSE._serialized_end = 19371 + _RESETSTICKYTASKQUEUEREQUEST._serialized_start = 19373 + _RESETSTICKYTASKQUEUEREQUEST._serialized_end = 19483 + _RESETSTICKYTASKQUEUERESPONSE._serialized_start = 19485 + _RESETSTICKYTASKQUEUERESPONSE._serialized_end = 19515 + _SHUTDOWNWORKERREQUEST._serialized_start = 19518 + _SHUTDOWNWORKERREQUEST._serialized_end = 19688 + _SHUTDOWNWORKERRESPONSE._serialized_start = 19690 + _SHUTDOWNWORKERRESPONSE._serialized_end = 19714 + _QUERYWORKFLOWREQUEST._serialized_start = 19717 + _QUERYWORKFLOWREQUEST._serialized_end = 19950 + _QUERYWORKFLOWRESPONSE._serialized_start = 19953 + _QUERYWORKFLOWRESPONSE._serialized_end = 20094 + _DESCRIBEWORKFLOWEXECUTIONREQUEST._serialized_start = 20096 + _DESCRIBEWORKFLOWEXECUTIONREQUEST._serialized_end = 20211 + _DESCRIBEWORKFLOWEXECUTIONRESPONSE._serialized_start = 20214 + _DESCRIBEWORKFLOWEXECUTIONRESPONSE._serialized_end = 20879 + _DESCRIBETASKQUEUEREQUEST._serialized_start = 20882 + _DESCRIBETASKQUEUEREQUEST._serialized_end = 21410 + _DESCRIBETASKQUEUERESPONSE._serialized_start = 21413 + _DESCRIBETASKQUEUERESPONSE._serialized_end = 22417 + _DESCRIBETASKQUEUERESPONSE_STATSBYPRIORITYKEYENTRY._serialized_start = 22097 + _DESCRIBETASKQUEUERESPONSE_STATSBYPRIORITYKEYENTRY._serialized_end = 22197 + _DESCRIBETASKQUEUERESPONSE_EFFECTIVERATELIMIT._serialized_start = 22199 + _DESCRIBETASKQUEUERESPONSE_EFFECTIVERATELIMIT._serialized_end = 22315 + _DESCRIBETASKQUEUERESPONSE_VERSIONSINFOENTRY._serialized_start = 22317 + _DESCRIBETASKQUEUERESPONSE_VERSIONSINFOENTRY._serialized_end = 22417 + _GETCLUSTERINFOREQUEST._serialized_start = 22419 + _GETCLUSTERINFOREQUEST._serialized_end = 22442 + _GETCLUSTERINFORESPONSE._serialized_start = 22445 + _GETCLUSTERINFORESPONSE._serialized_end = 22910 + _GETCLUSTERINFORESPONSE_SUPPORTEDCLIENTSENTRY._serialized_start = 22855 + _GETCLUSTERINFORESPONSE_SUPPORTEDCLIENTSENTRY._serialized_end = 22910 + _GETSYSTEMINFOREQUEST._serialized_start = 22912 + _GETSYSTEMINFOREQUEST._serialized_end = 22934 + _GETSYSTEMINFORESPONSE._serialized_start = 22937 + _GETSYSTEMINFORESPONSE._serialized_end = 23437 + _GETSYSTEMINFORESPONSE_CAPABILITIES._serialized_start = 23078 + _GETSYSTEMINFORESPONSE_CAPABILITIES._serialized_end = 23437 + _LISTTASKQUEUEPARTITIONSREQUEST._serialized_start = 23439 + _LISTTASKQUEUEPARTITIONSREQUEST._serialized_end = 23548 + _LISTTASKQUEUEPARTITIONSRESPONSE._serialized_start = 23551 + _LISTTASKQUEUEPARTITIONSRESPONSE._serialized_end = 23774 + _CREATESCHEDULEREQUEST._serialized_start = 23777 + _CREATESCHEDULEREQUEST._serialized_end = 24109 + _CREATESCHEDULERESPONSE._serialized_start = 24111 + _CREATESCHEDULERESPONSE._serialized_end = 24159 + _DESCRIBESCHEDULEREQUEST._serialized_start = 24161 + _DESCRIBESCHEDULEREQUEST._serialized_end = 24226 + _DESCRIBESCHEDULERESPONSE._serialized_start = 24229 + _DESCRIBESCHEDULERESPONSE._serialized_end = 24500 + _UPDATESCHEDULEREQUEST._serialized_start = 24503 + _UPDATESCHEDULEREQUEST._serialized_end = 24751 + _UPDATESCHEDULERESPONSE._serialized_start = 24753 + _UPDATESCHEDULERESPONSE._serialized_end = 24777 + _PATCHSCHEDULEREQUEST._serialized_start = 24780 + _PATCHSCHEDULEREQUEST._serialized_end = 24936 + _PATCHSCHEDULERESPONSE._serialized_start = 24938 + _PATCHSCHEDULERESPONSE._serialized_end = 24961 + _LISTSCHEDULEMATCHINGTIMESREQUEST._serialized_start = 24964 + _LISTSCHEDULEMATCHINGTIMESREQUEST._serialized_end = 25132 + _LISTSCHEDULEMATCHINGTIMESRESPONSE._serialized_start = 25134 + _LISTSCHEDULEMATCHINGTIMESRESPONSE._serialized_end = 25217 + _DELETESCHEDULEREQUEST._serialized_start = 25219 + _DELETESCHEDULEREQUEST._serialized_end = 25300 + _DELETESCHEDULERESPONSE._serialized_start = 25302 + _DELETESCHEDULERESPONSE._serialized_end = 25326 + _LISTSCHEDULESREQUEST._serialized_start = 25328 + _LISTSCHEDULESREQUEST._serialized_end = 25436 + _LISTSCHEDULESRESPONSE._serialized_start = 25438 + _LISTSCHEDULESRESPONSE._serialized_end = 25550 + _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_start = 25553 + _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_end = 26199 + _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_ADDNEWCOMPATIBLEVERSION._serialized_start = 26000 _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_ADDNEWCOMPATIBLEVERSION._serialized_end = ( - 26009 + 26111 ) - _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_MERGESETS._serialized_start = 26011 - _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_MERGESETS._serialized_end = 26084 - _UPDATEWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_start = 26099 - _UPDATEWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_end = 26163 - _GETWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_start = 26165 - _GETWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_end = 26260 - _GETWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_start = 26262 - _GETWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_end = 26378 - _UPDATEWORKERVERSIONINGRULESREQUEST._serialized_start = 26381 - _UPDATEWORKERVERSIONINGRULESREQUEST._serialized_end = 28098 - _UPDATEWORKERVERSIONINGRULESREQUEST_INSERTBUILDIDASSIGNMENTRULE._serialized_start = 27433 + _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_MERGESETS._serialized_start = 26113 + _UPDATEWORKERBUILDIDCOMPATIBILITYREQUEST_MERGESETS._serialized_end = 26186 + _UPDATEWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_start = 26201 + _UPDATEWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_end = 26265 + _GETWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_start = 26267 + _GETWORKERBUILDIDCOMPATIBILITYREQUEST._serialized_end = 26362 + _GETWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_start = 26364 + _GETWORKERBUILDIDCOMPATIBILITYRESPONSE._serialized_end = 26480 + _UPDATEWORKERVERSIONINGRULESREQUEST._serialized_start = 26483 + _UPDATEWORKERVERSIONINGRULESREQUEST._serialized_end = 28200 + _UPDATEWORKERVERSIONINGRULESREQUEST_INSERTBUILDIDASSIGNMENTRULE._serialized_start = 27535 _UPDATEWORKERVERSIONINGRULESREQUEST_INSERTBUILDIDASSIGNMENTRULE._serialized_end = ( - 27546 + 27648 ) - _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACEBUILDIDASSIGNMENTRULE._serialized_start = 27549 + _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACEBUILDIDASSIGNMENTRULE._serialized_start = 27651 _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACEBUILDIDASSIGNMENTRULE._serialized_end = ( - 27678 + 27780 ) - _UPDATEWORKERVERSIONINGRULESREQUEST_DELETEBUILDIDASSIGNMENTRULE._serialized_start = 27680 + _UPDATEWORKERVERSIONINGRULESREQUEST_DELETEBUILDIDASSIGNMENTRULE._serialized_start = 27782 _UPDATEWORKERVERSIONINGRULESREQUEST_DELETEBUILDIDASSIGNMENTRULE._serialized_end = ( - 27744 + 27846 ) - _UPDATEWORKERVERSIONINGRULESREQUEST_ADDCOMPATIBLEBUILDIDREDIRECTRULE._serialized_start = 27746 - _UPDATEWORKERVERSIONINGRULESREQUEST_ADDCOMPATIBLEBUILDIDREDIRECTRULE._serialized_end = 27852 - _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACECOMPATIBLEBUILDIDREDIRECTRULE._serialized_start = 27854 - _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACECOMPATIBLEBUILDIDREDIRECTRULE._serialized_end = 27964 - _UPDATEWORKERVERSIONINGRULESREQUEST_DELETECOMPATIBLEBUILDIDREDIRECTRULE._serialized_start = 27966 - _UPDATEWORKERVERSIONINGRULESREQUEST_DELETECOMPATIBLEBUILDIDREDIRECTRULE._serialized_end = 28028 - _UPDATEWORKERVERSIONINGRULESREQUEST_COMMITBUILDID._serialized_start = 28030 - _UPDATEWORKERVERSIONINGRULESREQUEST_COMMITBUILDID._serialized_end = 28085 - _UPDATEWORKERVERSIONINGRULESRESPONSE._serialized_start = 28101 - _UPDATEWORKERVERSIONINGRULESRESPONSE._serialized_end = 28353 - _GETWORKERVERSIONINGRULESREQUEST._serialized_start = 28355 - _GETWORKERVERSIONINGRULESREQUEST._serialized_end = 28427 - _GETWORKERVERSIONINGRULESRESPONSE._serialized_start = 28430 - _GETWORKERVERSIONINGRULESRESPONSE._serialized_end = 28679 - _GETWORKERTASKREACHABILITYREQUEST._serialized_start = 28682 - _GETWORKERTASKREACHABILITYREQUEST._serialized_end = 28838 - _GETWORKERTASKREACHABILITYRESPONSE._serialized_start = 28840 - _GETWORKERTASKREACHABILITYRESPONSE._serialized_end = 28954 - _UPDATEWORKFLOWEXECUTIONREQUEST._serialized_start = 28957 - _UPDATEWORKFLOWEXECUTIONREQUEST._serialized_end = 29218 - _UPDATEWORKFLOWEXECUTIONRESPONSE._serialized_start = 29221 - _UPDATEWORKFLOWEXECUTIONRESPONSE._serialized_end = 29436 - _STARTBATCHOPERATIONREQUEST._serialized_start = 29439 - _STARTBATCHOPERATIONREQUEST._serialized_end = 30451 - _STARTBATCHOPERATIONRESPONSE._serialized_start = 30453 - _STARTBATCHOPERATIONRESPONSE._serialized_end = 30482 - _STOPBATCHOPERATIONREQUEST._serialized_start = 30484 - _STOPBATCHOPERATIONREQUEST._serialized_end = 30580 - _STOPBATCHOPERATIONRESPONSE._serialized_start = 30582 - _STOPBATCHOPERATIONRESPONSE._serialized_end = 30610 - _DESCRIBEBATCHOPERATIONREQUEST._serialized_start = 30612 - _DESCRIBEBATCHOPERATIONREQUEST._serialized_end = 30678 - _DESCRIBEBATCHOPERATIONRESPONSE._serialized_start = 30681 - _DESCRIBEBATCHOPERATIONRESPONSE._serialized_end = 31083 - _LISTBATCHOPERATIONSREQUEST._serialized_start = 31085 - _LISTBATCHOPERATIONSREQUEST._serialized_end = 31176 - _LISTBATCHOPERATIONSRESPONSE._serialized_start = 31178 - _LISTBATCHOPERATIONSRESPONSE._serialized_end = 31299 - _POLLWORKFLOWEXECUTIONUPDATEREQUEST._serialized_start = 31302 - _POLLWORKFLOWEXECUTIONUPDATEREQUEST._serialized_end = 31487 - _POLLWORKFLOWEXECUTIONUPDATERESPONSE._serialized_start = 31490 - _POLLWORKFLOWEXECUTIONUPDATERESPONSE._serialized_end = 31709 - _POLLNEXUSTASKQUEUEREQUEST._serialized_start = 31712 - _POLLNEXUSTASKQUEUEREQUEST._serialized_end = 32074 - _POLLNEXUSTASKQUEUERESPONSE._serialized_start = 32077 - _POLLNEXUSTASKQUEUERESPONSE._serialized_end = 32257 - _RESPONDNEXUSTASKCOMPLETEDREQUEST._serialized_start = 32260 - _RESPONDNEXUSTASKCOMPLETEDREQUEST._serialized_end = 32402 - _RESPONDNEXUSTASKCOMPLETEDRESPONSE._serialized_start = 32404 - _RESPONDNEXUSTASKCOMPLETEDRESPONSE._serialized_end = 32439 - _RESPONDNEXUSTASKFAILEDREQUEST._serialized_start = 32442 - _RESPONDNEXUSTASKFAILEDREQUEST._serialized_end = 32582 - _RESPONDNEXUSTASKFAILEDRESPONSE._serialized_start = 32584 - _RESPONDNEXUSTASKFAILEDRESPONSE._serialized_end = 32616 - _EXECUTEMULTIOPERATIONREQUEST._serialized_start = 32619 - _EXECUTEMULTIOPERATIONREQUEST._serialized_end = 32970 - _EXECUTEMULTIOPERATIONREQUEST_OPERATION._serialized_start = 32764 - _EXECUTEMULTIOPERATIONREQUEST_OPERATION._serialized_end = 32970 - _EXECUTEMULTIOPERATIONRESPONSE._serialized_start = 32973 - _EXECUTEMULTIOPERATIONRESPONSE._serialized_end = 33305 - _EXECUTEMULTIOPERATIONRESPONSE_RESPONSE._serialized_start = 33099 - _EXECUTEMULTIOPERATIONRESPONSE_RESPONSE._serialized_end = 33305 - _UPDATEACTIVITYOPTIONSREQUEST._serialized_start = 33308 - _UPDATEACTIVITYOPTIONSREQUEST._serialized_end = 33644 - _UPDATEACTIVITYOPTIONSRESPONSE._serialized_start = 33646 - _UPDATEACTIVITYOPTIONSRESPONSE._serialized_end = 33746 - _PAUSEACTIVITYREQUEST._serialized_start = 33749 - _PAUSEACTIVITYREQUEST._serialized_end = 33928 - _PAUSEACTIVITYRESPONSE._serialized_start = 33930 - _PAUSEACTIVITYRESPONSE._serialized_end = 33953 - _UNPAUSEACTIVITYREQUEST._serialized_start = 33956 - _UNPAUSEACTIVITYREQUEST._serialized_end = 34236 - _UNPAUSEACTIVITYRESPONSE._serialized_start = 34238 - _UNPAUSEACTIVITYRESPONSE._serialized_end = 34263 - _RESETACTIVITYREQUEST._serialized_start = 34266 - _RESETACTIVITYREQUEST._serialized_end = 34573 - _RESETACTIVITYRESPONSE._serialized_start = 34575 - _RESETACTIVITYRESPONSE._serialized_end = 34598 - _UPDATEWORKFLOWEXECUTIONOPTIONSREQUEST._serialized_start = 34601 - _UPDATEWORKFLOWEXECUTIONOPTIONSREQUEST._serialized_end = 34867 - _UPDATEWORKFLOWEXECUTIONOPTIONSRESPONSE._serialized_start = 34870 - _UPDATEWORKFLOWEXECUTIONOPTIONSRESPONSE._serialized_end = 34998 - _DESCRIBEDEPLOYMENTREQUEST._serialized_start = 35000 - _DESCRIBEDEPLOYMENTREQUEST._serialized_end = 35106 - _DESCRIBEDEPLOYMENTRESPONSE._serialized_start = 35108 - _DESCRIBEDEPLOYMENTRESPONSE._serialized_end = 35205 - _DESCRIBEWORKERDEPLOYMENTVERSIONREQUEST._serialized_start = 35208 - _DESCRIBEWORKERDEPLOYMENTVERSIONREQUEST._serialized_end = 35402 - _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_start = 35405 - _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_end = 36057 - _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE._serialized_start = 35666 - _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE._serialized_end = 36057 - _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE_STATSBYPRIORITYKEYENTRY._serialized_start = 22065 - _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE_STATSBYPRIORITYKEYENTRY._serialized_end = 22165 - _DESCRIBEWORKERDEPLOYMENTREQUEST._serialized_start = 36059 - _DESCRIBEWORKERDEPLOYMENTREQUEST._serialized_end = 36136 - _DESCRIBEWORKERDEPLOYMENTRESPONSE._serialized_start = 36139 - _DESCRIBEWORKERDEPLOYMENTRESPONSE._serialized_end = 36279 - _LISTDEPLOYMENTSREQUEST._serialized_start = 36281 - _LISTDEPLOYMENTSREQUEST._serialized_end = 36389 - _LISTDEPLOYMENTSRESPONSE._serialized_start = 36391 - _LISTDEPLOYMENTSRESPONSE._serialized_end = 36510 - _SETCURRENTDEPLOYMENTREQUEST._serialized_start = 36513 - _SETCURRENTDEPLOYMENTREQUEST._serialized_end = 36718 - _SETCURRENTDEPLOYMENTRESPONSE._serialized_start = 36721 - _SETCURRENTDEPLOYMENTRESPONSE._serialized_end = 36906 - _SETWORKERDEPLOYMENTCURRENTVERSIONREQUEST._serialized_start = 36909 - _SETWORKERDEPLOYMENTCURRENTVERSIONREQUEST._serialized_end = 37112 - _SETWORKERDEPLOYMENTCURRENTVERSIONRESPONSE._serialized_start = 37115 - _SETWORKERDEPLOYMENTCURRENTVERSIONRESPONSE._serialized_end = 37302 - _SETWORKERDEPLOYMENTRAMPINGVERSIONREQUEST._serialized_start = 37305 - _SETWORKERDEPLOYMENTRAMPINGVERSIONREQUEST._serialized_end = 37528 - _SETWORKERDEPLOYMENTRAMPINGVERSIONRESPONSE._serialized_start = 37531 - _SETWORKERDEPLOYMENTRAMPINGVERSIONRESPONSE._serialized_end = 37747 - _LISTWORKERDEPLOYMENTSREQUEST._serialized_start = 37749 - _LISTWORKERDEPLOYMENTSREQUEST._serialized_end = 37842 - _LISTWORKERDEPLOYMENTSRESPONSE._serialized_start = 37845 - _LISTWORKERDEPLOYMENTSRESPONSE._serialized_end = 38516 - _LISTWORKERDEPLOYMENTSRESPONSE_WORKERDEPLOYMENTSUMMARY._serialized_start = 38020 - _LISTWORKERDEPLOYMENTSRESPONSE_WORKERDEPLOYMENTSUMMARY._serialized_end = 38516 - _DELETEWORKERDEPLOYMENTVERSIONREQUEST._serialized_start = 38519 - _DELETEWORKERDEPLOYMENTVERSIONREQUEST._serialized_end = 38719 - _DELETEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_start = 38721 - _DELETEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_end = 38760 - _DELETEWORKERDEPLOYMENTREQUEST._serialized_start = 38762 - _DELETEWORKERDEPLOYMENTREQUEST._serialized_end = 38855 - _DELETEWORKERDEPLOYMENTRESPONSE._serialized_start = 38857 - _DELETEWORKERDEPLOYMENTRESPONSE._serialized_end = 38889 - _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST._serialized_start = 38892 - _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST._serialized_end = 39310 - _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST_UPSERTENTRIESENTRY._serialized_start = 39225 + _UPDATEWORKERVERSIONINGRULESREQUEST_ADDCOMPATIBLEBUILDIDREDIRECTRULE._serialized_start = 27848 + _UPDATEWORKERVERSIONINGRULESREQUEST_ADDCOMPATIBLEBUILDIDREDIRECTRULE._serialized_end = 27954 + _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACECOMPATIBLEBUILDIDREDIRECTRULE._serialized_start = 27956 + _UPDATEWORKERVERSIONINGRULESREQUEST_REPLACECOMPATIBLEBUILDIDREDIRECTRULE._serialized_end = 28066 + _UPDATEWORKERVERSIONINGRULESREQUEST_DELETECOMPATIBLEBUILDIDREDIRECTRULE._serialized_start = 28068 + _UPDATEWORKERVERSIONINGRULESREQUEST_DELETECOMPATIBLEBUILDIDREDIRECTRULE._serialized_end = 28130 + _UPDATEWORKERVERSIONINGRULESREQUEST_COMMITBUILDID._serialized_start = 28132 + _UPDATEWORKERVERSIONINGRULESREQUEST_COMMITBUILDID._serialized_end = 28187 + _UPDATEWORKERVERSIONINGRULESRESPONSE._serialized_start = 28203 + _UPDATEWORKERVERSIONINGRULESRESPONSE._serialized_end = 28455 + _GETWORKERVERSIONINGRULESREQUEST._serialized_start = 28457 + _GETWORKERVERSIONINGRULESREQUEST._serialized_end = 28529 + _GETWORKERVERSIONINGRULESRESPONSE._serialized_start = 28532 + _GETWORKERVERSIONINGRULESRESPONSE._serialized_end = 28781 + _GETWORKERTASKREACHABILITYREQUEST._serialized_start = 28784 + _GETWORKERTASKREACHABILITYREQUEST._serialized_end = 28940 + _GETWORKERTASKREACHABILITYRESPONSE._serialized_start = 28942 + _GETWORKERTASKREACHABILITYRESPONSE._serialized_end = 29056 + _UPDATEWORKFLOWEXECUTIONREQUEST._serialized_start = 29059 + _UPDATEWORKFLOWEXECUTIONREQUEST._serialized_end = 29320 + _UPDATEWORKFLOWEXECUTIONRESPONSE._serialized_start = 29323 + _UPDATEWORKFLOWEXECUTIONRESPONSE._serialized_end = 29538 + _STARTBATCHOPERATIONREQUEST._serialized_start = 29541 + _STARTBATCHOPERATIONREQUEST._serialized_end = 30553 + _STARTBATCHOPERATIONRESPONSE._serialized_start = 30555 + _STARTBATCHOPERATIONRESPONSE._serialized_end = 30584 + _STOPBATCHOPERATIONREQUEST._serialized_start = 30586 + _STOPBATCHOPERATIONREQUEST._serialized_end = 30682 + _STOPBATCHOPERATIONRESPONSE._serialized_start = 30684 + _STOPBATCHOPERATIONRESPONSE._serialized_end = 30712 + _DESCRIBEBATCHOPERATIONREQUEST._serialized_start = 30714 + _DESCRIBEBATCHOPERATIONREQUEST._serialized_end = 30780 + _DESCRIBEBATCHOPERATIONRESPONSE._serialized_start = 30783 + _DESCRIBEBATCHOPERATIONRESPONSE._serialized_end = 31185 + _LISTBATCHOPERATIONSREQUEST._serialized_start = 31187 + _LISTBATCHOPERATIONSREQUEST._serialized_end = 31278 + _LISTBATCHOPERATIONSRESPONSE._serialized_start = 31280 + _LISTBATCHOPERATIONSRESPONSE._serialized_end = 31401 + _POLLWORKFLOWEXECUTIONUPDATEREQUEST._serialized_start = 31404 + _POLLWORKFLOWEXECUTIONUPDATEREQUEST._serialized_end = 31589 + _POLLWORKFLOWEXECUTIONUPDATERESPONSE._serialized_start = 31592 + _POLLWORKFLOWEXECUTIONUPDATERESPONSE._serialized_end = 31811 + _POLLNEXUSTASKQUEUEREQUEST._serialized_start = 31814 + _POLLNEXUSTASKQUEUEREQUEST._serialized_end = 32176 + _POLLNEXUSTASKQUEUERESPONSE._serialized_start = 32179 + _POLLNEXUSTASKQUEUERESPONSE._serialized_end = 32359 + _RESPONDNEXUSTASKCOMPLETEDREQUEST._serialized_start = 32362 + _RESPONDNEXUSTASKCOMPLETEDREQUEST._serialized_end = 32504 + _RESPONDNEXUSTASKCOMPLETEDRESPONSE._serialized_start = 32506 + _RESPONDNEXUSTASKCOMPLETEDRESPONSE._serialized_end = 32541 + _RESPONDNEXUSTASKFAILEDREQUEST._serialized_start = 32544 + _RESPONDNEXUSTASKFAILEDREQUEST._serialized_end = 32684 + _RESPONDNEXUSTASKFAILEDRESPONSE._serialized_start = 32686 + _RESPONDNEXUSTASKFAILEDRESPONSE._serialized_end = 32718 + _EXECUTEMULTIOPERATIONREQUEST._serialized_start = 32721 + _EXECUTEMULTIOPERATIONREQUEST._serialized_end = 33072 + _EXECUTEMULTIOPERATIONREQUEST_OPERATION._serialized_start = 32866 + _EXECUTEMULTIOPERATIONREQUEST_OPERATION._serialized_end = 33072 + _EXECUTEMULTIOPERATIONRESPONSE._serialized_start = 33075 + _EXECUTEMULTIOPERATIONRESPONSE._serialized_end = 33407 + _EXECUTEMULTIOPERATIONRESPONSE_RESPONSE._serialized_start = 33201 + _EXECUTEMULTIOPERATIONRESPONSE_RESPONSE._serialized_end = 33407 + _UPDATEACTIVITYOPTIONSREQUEST._serialized_start = 33410 + _UPDATEACTIVITYOPTIONSREQUEST._serialized_end = 33746 + _UPDATEACTIVITYOPTIONSRESPONSE._serialized_start = 33748 + _UPDATEACTIVITYOPTIONSRESPONSE._serialized_end = 33848 + _PAUSEACTIVITYREQUEST._serialized_start = 33851 + _PAUSEACTIVITYREQUEST._serialized_end = 34030 + _PAUSEACTIVITYRESPONSE._serialized_start = 34032 + _PAUSEACTIVITYRESPONSE._serialized_end = 34055 + _UNPAUSEACTIVITYREQUEST._serialized_start = 34058 + _UNPAUSEACTIVITYREQUEST._serialized_end = 34338 + _UNPAUSEACTIVITYRESPONSE._serialized_start = 34340 + _UNPAUSEACTIVITYRESPONSE._serialized_end = 34365 + _RESETACTIVITYREQUEST._serialized_start = 34368 + _RESETACTIVITYREQUEST._serialized_end = 34675 + _RESETACTIVITYRESPONSE._serialized_start = 34677 + _RESETACTIVITYRESPONSE._serialized_end = 34700 + _UPDATEWORKFLOWEXECUTIONOPTIONSREQUEST._serialized_start = 34703 + _UPDATEWORKFLOWEXECUTIONOPTIONSREQUEST._serialized_end = 34969 + _UPDATEWORKFLOWEXECUTIONOPTIONSRESPONSE._serialized_start = 34972 + _UPDATEWORKFLOWEXECUTIONOPTIONSRESPONSE._serialized_end = 35100 + _DESCRIBEDEPLOYMENTREQUEST._serialized_start = 35102 + _DESCRIBEDEPLOYMENTREQUEST._serialized_end = 35208 + _DESCRIBEDEPLOYMENTRESPONSE._serialized_start = 35210 + _DESCRIBEDEPLOYMENTRESPONSE._serialized_end = 35307 + _DESCRIBEWORKERDEPLOYMENTVERSIONREQUEST._serialized_start = 35310 + _DESCRIBEWORKERDEPLOYMENTVERSIONREQUEST._serialized_end = 35504 + _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_start = 35507 + _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_end = 36159 + _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE._serialized_start = 35768 + _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE._serialized_end = 36159 + _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE_STATSBYPRIORITYKEYENTRY._serialized_start = 22097 + _DESCRIBEWORKERDEPLOYMENTVERSIONRESPONSE_VERSIONTASKQUEUE_STATSBYPRIORITYKEYENTRY._serialized_end = 22197 + _DESCRIBEWORKERDEPLOYMENTREQUEST._serialized_start = 36161 + _DESCRIBEWORKERDEPLOYMENTREQUEST._serialized_end = 36238 + _DESCRIBEWORKERDEPLOYMENTRESPONSE._serialized_start = 36241 + _DESCRIBEWORKERDEPLOYMENTRESPONSE._serialized_end = 36381 + _LISTDEPLOYMENTSREQUEST._serialized_start = 36383 + _LISTDEPLOYMENTSREQUEST._serialized_end = 36491 + _LISTDEPLOYMENTSRESPONSE._serialized_start = 36493 + _LISTDEPLOYMENTSRESPONSE._serialized_end = 36612 + _SETCURRENTDEPLOYMENTREQUEST._serialized_start = 36615 + _SETCURRENTDEPLOYMENTREQUEST._serialized_end = 36820 + _SETCURRENTDEPLOYMENTRESPONSE._serialized_start = 36823 + _SETCURRENTDEPLOYMENTRESPONSE._serialized_end = 37008 + _SETWORKERDEPLOYMENTCURRENTVERSIONREQUEST._serialized_start = 37011 + _SETWORKERDEPLOYMENTCURRENTVERSIONREQUEST._serialized_end = 37240 + _SETWORKERDEPLOYMENTCURRENTVERSIONRESPONSE._serialized_start = 37243 + _SETWORKERDEPLOYMENTCURRENTVERSIONRESPONSE._serialized_end = 37430 + _SETWORKERDEPLOYMENTRAMPINGVERSIONREQUEST._serialized_start = 37433 + _SETWORKERDEPLOYMENTRAMPINGVERSIONREQUEST._serialized_end = 37682 + _SETWORKERDEPLOYMENTRAMPINGVERSIONRESPONSE._serialized_start = 37685 + _SETWORKERDEPLOYMENTRAMPINGVERSIONRESPONSE._serialized_end = 37901 + _LISTWORKERDEPLOYMENTSREQUEST._serialized_start = 37903 + _LISTWORKERDEPLOYMENTSREQUEST._serialized_end = 37996 + _LISTWORKERDEPLOYMENTSRESPONSE._serialized_start = 37999 + _LISTWORKERDEPLOYMENTSRESPONSE._serialized_end = 38670 + _LISTWORKERDEPLOYMENTSRESPONSE_WORKERDEPLOYMENTSUMMARY._serialized_start = 38174 + _LISTWORKERDEPLOYMENTSRESPONSE_WORKERDEPLOYMENTSUMMARY._serialized_end = 38670 + _DELETEWORKERDEPLOYMENTVERSIONREQUEST._serialized_start = 38673 + _DELETEWORKERDEPLOYMENTVERSIONREQUEST._serialized_end = 38873 + _DELETEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_start = 38875 + _DELETEWORKERDEPLOYMENTVERSIONRESPONSE._serialized_end = 38914 + _DELETEWORKERDEPLOYMENTREQUEST._serialized_start = 38916 + _DELETEWORKERDEPLOYMENTREQUEST._serialized_end = 39009 + _DELETEWORKERDEPLOYMENTRESPONSE._serialized_start = 39011 + _DELETEWORKERDEPLOYMENTRESPONSE._serialized_end = 39043 + _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST._serialized_start = 39046 + _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST._serialized_end = 39464 + _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST_UPSERTENTRIESENTRY._serialized_start = 39379 _UPDATEWORKERDEPLOYMENTVERSIONMETADATAREQUEST_UPSERTENTRIESENTRY._serialized_end = ( - 39310 + 39464 ) - _UPDATEWORKERDEPLOYMENTVERSIONMETADATARESPONSE._serialized_start = 39312 - _UPDATEWORKERDEPLOYMENTVERSIONMETADATARESPONSE._serialized_end = 39422 - _GETCURRENTDEPLOYMENTREQUEST._serialized_start = 39424 - _GETCURRENTDEPLOYMENTREQUEST._serialized_end = 39493 - _GETCURRENTDEPLOYMENTRESPONSE._serialized_start = 39495 - _GETCURRENTDEPLOYMENTRESPONSE._serialized_end = 39602 - _GETDEPLOYMENTREACHABILITYREQUEST._serialized_start = 39604 - _GETDEPLOYMENTREACHABILITYREQUEST._serialized_end = 39717 - _GETDEPLOYMENTREACHABILITYRESPONSE._serialized_start = 39720 - _GETDEPLOYMENTREACHABILITYRESPONSE._serialized_end = 39947 - _CREATEWORKFLOWRULEREQUEST._serialized_start = 39950 - _CREATEWORKFLOWRULEREQUEST._serialized_end = 40130 - _CREATEWORKFLOWRULERESPONSE._serialized_start = 40132 - _CREATEWORKFLOWRULERESPONSE._serialized_end = 40227 - _DESCRIBEWORKFLOWRULEREQUEST._serialized_start = 40229 - _DESCRIBEWORKFLOWRULEREQUEST._serialized_end = 40294 - _DESCRIBEWORKFLOWRULERESPONSE._serialized_start = 40296 - _DESCRIBEWORKFLOWRULERESPONSE._serialized_end = 40377 - _DELETEWORKFLOWRULEREQUEST._serialized_start = 40379 - _DELETEWORKFLOWRULEREQUEST._serialized_end = 40442 - _DELETEWORKFLOWRULERESPONSE._serialized_start = 40444 - _DELETEWORKFLOWRULERESPONSE._serialized_end = 40472 - _LISTWORKFLOWRULESREQUEST._serialized_start = 40474 - _LISTWORKFLOWRULESREQUEST._serialized_end = 40544 - _LISTWORKFLOWRULESRESPONSE._serialized_start = 40546 - _LISTWORKFLOWRULESRESPONSE._serialized_end = 40650 - _TRIGGERWORKFLOWRULEREQUEST._serialized_start = 40653 - _TRIGGERWORKFLOWRULEREQUEST._serialized_end = 40859 - _TRIGGERWORKFLOWRULERESPONSE._serialized_start = 40861 - _TRIGGERWORKFLOWRULERESPONSE._serialized_end = 40907 - _RECORDWORKERHEARTBEATREQUEST._serialized_start = 40910 - _RECORDWORKERHEARTBEATREQUEST._serialized_end = 41044 - _RECORDWORKERHEARTBEATRESPONSE._serialized_start = 41046 - _RECORDWORKERHEARTBEATRESPONSE._serialized_end = 41077 - _LISTWORKERSREQUEST._serialized_start = 41079 - _LISTWORKERSREQUEST._serialized_end = 41177 - _LISTWORKERSRESPONSE._serialized_start = 41179 - _LISTWORKERSRESPONSE._serialized_end = 41283 - _UPDATETASKQUEUECONFIGREQUEST._serialized_start = 41286 - _UPDATETASKQUEUECONFIGREQUEST._serialized_end = 41768 - _UPDATETASKQUEUECONFIGREQUEST_RATELIMITUPDATE._serialized_start = 41677 - _UPDATETASKQUEUECONFIGREQUEST_RATELIMITUPDATE._serialized_end = 41768 - _UPDATETASKQUEUECONFIGRESPONSE._serialized_start = 41770 - _UPDATETASKQUEUECONFIGRESPONSE._serialized_end = 41861 - _FETCHWORKERCONFIGREQUEST._serialized_start = 41864 - _FETCHWORKERCONFIGREQUEST._serialized_end = 42001 - _FETCHWORKERCONFIGRESPONSE._serialized_start = 42003 - _FETCHWORKERCONFIGRESPONSE._serialized_end = 42088 - _UPDATEWORKERCONFIGREQUEST._serialized_start = 42091 - _UPDATEWORKERCONFIGREQUEST._serialized_end = 42336 - _UPDATEWORKERCONFIGRESPONSE._serialized_start = 42338 - _UPDATEWORKERCONFIGRESPONSE._serialized_end = 42438 + _UPDATEWORKERDEPLOYMENTVERSIONMETADATARESPONSE._serialized_start = 39466 + _UPDATEWORKERDEPLOYMENTVERSIONMETADATARESPONSE._serialized_end = 39576 + _SETWORKERDEPLOYMENTMANAGERREQUEST._serialized_start = 39579 + _SETWORKERDEPLOYMENTMANAGERREQUEST._serialized_end = 39768 + _SETWORKERDEPLOYMENTMANAGERRESPONSE._serialized_start = 39770 + _SETWORKERDEPLOYMENTMANAGERRESPONSE._serialized_end = 39865 + _GETCURRENTDEPLOYMENTREQUEST._serialized_start = 39867 + _GETCURRENTDEPLOYMENTREQUEST._serialized_end = 39936 + _GETCURRENTDEPLOYMENTRESPONSE._serialized_start = 39938 + _GETCURRENTDEPLOYMENTRESPONSE._serialized_end = 40045 + _GETDEPLOYMENTREACHABILITYREQUEST._serialized_start = 40047 + _GETDEPLOYMENTREACHABILITYREQUEST._serialized_end = 40160 + _GETDEPLOYMENTREACHABILITYRESPONSE._serialized_start = 40163 + _GETDEPLOYMENTREACHABILITYRESPONSE._serialized_end = 40390 + _CREATEWORKFLOWRULEREQUEST._serialized_start = 40393 + _CREATEWORKFLOWRULEREQUEST._serialized_end = 40573 + _CREATEWORKFLOWRULERESPONSE._serialized_start = 40575 + _CREATEWORKFLOWRULERESPONSE._serialized_end = 40670 + _DESCRIBEWORKFLOWRULEREQUEST._serialized_start = 40672 + _DESCRIBEWORKFLOWRULEREQUEST._serialized_end = 40737 + _DESCRIBEWORKFLOWRULERESPONSE._serialized_start = 40739 + _DESCRIBEWORKFLOWRULERESPONSE._serialized_end = 40820 + _DELETEWORKFLOWRULEREQUEST._serialized_start = 40822 + _DELETEWORKFLOWRULEREQUEST._serialized_end = 40885 + _DELETEWORKFLOWRULERESPONSE._serialized_start = 40887 + _DELETEWORKFLOWRULERESPONSE._serialized_end = 40915 + _LISTWORKFLOWRULESREQUEST._serialized_start = 40917 + _LISTWORKFLOWRULESREQUEST._serialized_end = 40987 + _LISTWORKFLOWRULESRESPONSE._serialized_start = 40989 + _LISTWORKFLOWRULESRESPONSE._serialized_end = 41093 + _TRIGGERWORKFLOWRULEREQUEST._serialized_start = 41096 + _TRIGGERWORKFLOWRULEREQUEST._serialized_end = 41302 + _TRIGGERWORKFLOWRULERESPONSE._serialized_start = 41304 + _TRIGGERWORKFLOWRULERESPONSE._serialized_end = 41350 + _RECORDWORKERHEARTBEATREQUEST._serialized_start = 41353 + _RECORDWORKERHEARTBEATREQUEST._serialized_end = 41487 + _RECORDWORKERHEARTBEATRESPONSE._serialized_start = 41489 + _RECORDWORKERHEARTBEATRESPONSE._serialized_end = 41520 + _LISTWORKERSREQUEST._serialized_start = 41522 + _LISTWORKERSREQUEST._serialized_end = 41620 + _LISTWORKERSRESPONSE._serialized_start = 41622 + _LISTWORKERSRESPONSE._serialized_end = 41726 + _UPDATETASKQUEUECONFIGREQUEST._serialized_start = 41729 + _UPDATETASKQUEUECONFIGREQUEST._serialized_end = 42211 + _UPDATETASKQUEUECONFIGREQUEST_RATELIMITUPDATE._serialized_start = 42120 + _UPDATETASKQUEUECONFIGREQUEST_RATELIMITUPDATE._serialized_end = 42211 + _UPDATETASKQUEUECONFIGRESPONSE._serialized_start = 42213 + _UPDATETASKQUEUECONFIGRESPONSE._serialized_end = 42304 + _FETCHWORKERCONFIGREQUEST._serialized_start = 42307 + _FETCHWORKERCONFIGREQUEST._serialized_end = 42444 + _FETCHWORKERCONFIGRESPONSE._serialized_start = 42446 + _FETCHWORKERCONFIGRESPONSE._serialized_end = 42531 + _UPDATEWORKERCONFIGREQUEST._serialized_start = 42534 + _UPDATEWORKERCONFIGREQUEST._serialized_end = 42779 + _UPDATEWORKERCONFIGRESPONSE._serialized_start = 42781 + _UPDATEWORKERCONFIGRESPONSE._serialized_end = 42881 + _DESCRIBEWORKERREQUEST._serialized_start = 42883 + _DESCRIBEWORKERREQUEST._serialized_end = 42954 + _DESCRIBEWORKERRESPONSE._serialized_start = 42956 + _DESCRIBEWORKERRESPONSE._serialized_end = 43037 + _STARTACTIVITYEXECUTIONREQUEST._serialized_start = 43040 + _STARTACTIVITYEXECUTIONREQUEST._serialized_end = 43938 + _STARTACTIVITYEXECUTIONRESPONSE._serialized_start = 43941 + _STARTACTIVITYEXECUTIONRESPONSE._serialized_end = 44134 + _DESCRIBEACTIVITYEXECUTIONREQUEST._serialized_start = 44137 + _DESCRIBEACTIVITYEXECUTIONREQUEST._serialized_end = 44275 + _DESCRIBEACTIVITYEXECUTIONRESPONSE._serialized_start = 44277 + _DESCRIBEACTIVITYEXECUTIONRESPONSE._serialized_end = 44400 + _LISTACTIVITYEXECUTIONSREQUEST._serialized_start = 44402 + _LISTACTIVITYEXECUTIONSREQUEST._serialized_end = 44511 + _LISTACTIVITYEXECUTIONSRESPONSE._serialized_start = 44513 + _LISTACTIVITYEXECUTIONSRESPONSE._serialized_end = 44634 + _COUNTACTIVITYEXECUTIONSREQUEST._serialized_start = 44636 + _COUNTACTIVITYEXECUTIONSREQUEST._serialized_end = 44702 + _COUNTACTIVITYEXECUTIONSRESPONSE._serialized_start = 44705 + _COUNTACTIVITYEXECUTIONSRESPONSE._serialized_end = 44942 + _COUNTACTIVITYEXECUTIONSRESPONSE_AGGREGATIONGROUP._serialized_start = 18673 + _COUNTACTIVITYEXECUTIONSRESPONSE_AGGREGATIONGROUP._serialized_end = 18761 + _GETACTIVITYEXECUTIONRESULTREQUEST._serialized_start = 44944 + _GETACTIVITYEXECUTIONRESULTREQUEST._serialized_end = 45049 + _GETACTIVITYEXECUTIONRESULTRESPONSE._serialized_start = 45052 + _GETACTIVITYEXECUTIONRESULTRESPONSE._serialized_end = 45220 + _REQUESTCANCELACTIVITYEXECUTIONREQUEST._serialized_start = 45223 + _REQUESTCANCELACTIVITYEXECUTIONREQUEST._serialized_end = 45372 + _REQUESTCANCELACTIVITYEXECUTIONRESPONSE._serialized_start = 45374 + _REQUESTCANCELACTIVITYEXECUTIONRESPONSE._serialized_end = 45414 + _TERMINATEACTIVITYEXECUTIONREQUEST._serialized_start = 45416 + _TERMINATEACTIVITYEXECUTIONREQUEST._serialized_end = 45541 + _TERMINATEACTIVITYEXECUTIONRESPONSE._serialized_start = 45543 + _TERMINATEACTIVITYEXECUTIONRESPONSE._serialized_end = 45579 + _DELETEACTIVITYEXECUTIONREQUEST._serialized_start = 45581 + _DELETEACTIVITYEXECUTIONREQUEST._serialized_end = 45669 + _DELETEACTIVITYEXECUTIONRESPONSE._serialized_start = 45671 + _DELETEACTIVITYEXECUTIONRESPONSE._serialized_end = 45704 + _UPDATEACTIVITYEXECUTIONOPTIONSREQUEST._serialized_start = 45707 + _UPDATEACTIVITYEXECUTIONOPTIONSREQUEST._serialized_end = 46050 + _UPDATEACTIVITYEXECUTIONOPTIONSRESPONSE._serialized_start = 46052 + _UPDATEACTIVITYEXECUTIONOPTIONSRESPONSE._serialized_end = 46161 + _PAUSEACTIVITYEXECUTIONREQUEST._serialized_start = 46164 + _PAUSEACTIVITYEXECUTIONREQUEST._serialized_end = 46338 + _PAUSEACTIVITYEXECUTIONRESPONSE._serialized_start = 46340 + _PAUSEACTIVITYEXECUTIONRESPONSE._serialized_end = 46372 + _UNPAUSEACTIVITYEXECUTIONREQUEST._serialized_start = 46375 + _UNPAUSEACTIVITYEXECUTIONREQUEST._serialized_end = 46660 + _UNPAUSEACTIVITYEXECUTIONRESPONSE._serialized_start = 46662 + _UNPAUSEACTIVITYEXECUTIONRESPONSE._serialized_end = 46696 + _RESETACTIVITYEXECUTIONREQUEST._serialized_start = 46699 + _RESETACTIVITYEXECUTIONREQUEST._serialized_end = 47013 + _RESETACTIVITYEXECUTIONRESPONSE._serialized_start = 47015 + _RESETACTIVITYEXECUTIONRESPONSE._serialized_end = 47047 # @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/workflowservice/v1/request_response_pb2.pyi b/temporalio/api/workflowservice/v1/request_response_pb2.pyi index a0be16462..e69aaf4fc 100644 --- a/temporalio/api/workflowservice/v1/request_response_pb2.pyi +++ b/temporalio/api/workflowservice/v1/request_response_pb2.pyi @@ -23,6 +23,7 @@ import temporalio.api.enums.v1.batch_operation_pb2 import temporalio.api.enums.v1.common_pb2 import temporalio.api.enums.v1.deployment_pb2 import temporalio.api.enums.v1.failed_cause_pb2 +import temporalio.api.enums.v1.id_pb2 import temporalio.api.enums.v1.namespace_pb2 import temporalio.api.enums.v1.query_pb2 import temporalio.api.enums.v1.reset_pb2 @@ -1901,6 +1902,7 @@ class PollActivityTaskQueueResponse(google.protobuf.message.Message): """The autogenerated or user specified identifier of this activity. Can be used to complete the activity via `RespondActivityTaskCompletedById`. May be re-used as long as the last usage has resolved, but unique IDs for every activity invocation is a good idea. + Note that only a workflow activity ID may be autogenerated. """ @property def header(self) -> temporalio.api.common.v1.message_pb2.Header: @@ -2155,9 +2157,11 @@ class RecordActivityTaskHeartbeatByIdRequest(google.protobuf.message.Message): namespace: builtins.str """Namespace of the workflow which scheduled this activity""" workflow_id: builtins.str - """Id of the workflow which scheduled this activity""" + """Id of the workflow which scheduled this activity, leave empty to target a standalone activity""" run_id: builtins.str - """Run Id of the workflow which scheduled this activity""" + """For a workflow activity - the run ID of the workflow which scheduled this activity. + For a standalone activity - the run ID of the activity. + """ activity_id: builtins.str """Id of the activity we're heartbeating""" @property @@ -2342,9 +2346,11 @@ class RespondActivityTaskCompletedByIdRequest(google.protobuf.message.Message): namespace: builtins.str """Namespace of the workflow which scheduled this activity""" workflow_id: builtins.str - """Id of the workflow which scheduled this activity""" + """Id of the workflow which scheduled this activity, leave empty to target a standalone activity""" run_id: builtins.str - """Run Id of the workflow which scheduled this activity""" + """For a workflow activity - the run ID of the workflow which scheduled this activity. + For a standalone activity - the run ID of the activity. + """ activity_id: builtins.str """Id of the activity to complete""" @property @@ -2532,9 +2538,11 @@ class RespondActivityTaskFailedByIdRequest(google.protobuf.message.Message): namespace: builtins.str """Namespace of the workflow which scheduled this activity""" workflow_id: builtins.str - """Id of the workflow which scheduled this activity""" + """Id of the workflow which scheduled this activity, leave empty to target a standalone activity""" run_id: builtins.str - """Run Id of the workflow which scheduled this activity""" + """For a workflow activity - the run ID of the workflow which scheduled this activity. + For a standalone activity - the run ID of the activity. + """ activity_id: builtins.str """Id of the activity to fail""" @property @@ -2718,9 +2726,11 @@ class RespondActivityTaskCanceledByIdRequest(google.protobuf.message.Message): namespace: builtins.str """Namespace of the workflow which scheduled this activity""" workflow_id: builtins.str - """Id of the workflow which scheduled this activity""" + """Id of the workflow which scheduled this activity, leave empty to target a standalone activity""" run_id: builtins.str - """Run Id of the workflow which scheduled this activity""" + """For a workflow activity - the run ID of the workflow which scheduled this activity. + For a standalone activity - the run ID of the activity. + """ activity_id: builtins.str """Id of the activity to confirm is cancelled""" @property @@ -4820,6 +4830,8 @@ class GetClusterInfoResponse(google.protobuf.message.Message): HISTORY_SHARD_COUNT_FIELD_NUMBER: builtins.int PERSISTENCE_STORE_FIELD_NUMBER: builtins.int VISIBILITY_STORE_FIELD_NUMBER: builtins.int + INITIAL_FAILOVER_VERSION_FIELD_NUMBER: builtins.int + FAILOVER_VERSION_INCREMENT_FIELD_NUMBER: builtins.int @property def supported_clients( self, @@ -4835,6 +4847,8 @@ class GetClusterInfoResponse(google.protobuf.message.Message): history_shard_count: builtins.int persistence_store: builtins.str visibility_store: builtins.str + initial_failover_version: builtins.int + failover_version_increment: builtins.int def __init__( self, *, @@ -4847,6 +4861,8 @@ class GetClusterInfoResponse(google.protobuf.message.Message): history_shard_count: builtins.int = ..., persistence_store: builtins.str = ..., visibility_store: builtins.str = ..., + initial_failover_version: builtins.int = ..., + failover_version_increment: builtins.int = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal["version_info", b"version_info"] @@ -4858,8 +4874,12 @@ class GetClusterInfoResponse(google.protobuf.message.Message): b"cluster_id", "cluster_name", b"cluster_name", + "failover_version_increment", + b"failover_version_increment", "history_shard_count", b"history_shard_count", + "initial_failover_version", + b"initial_failover_version", "persistence_store", b"persistence_store", "server_version", @@ -7491,7 +7511,9 @@ class ExecuteMultiOperationResponse(google.protobuf.message.Message): global___ExecuteMultiOperationResponse = ExecuteMultiOperationResponse class UpdateActivityOptionsRequest(google.protobuf.message.Message): - """NOTE: keep in sync with temporalio.api.batch.v1.BatchOperationUpdateActivityOptions""" + """NOTE: keep in sync with temporalio.api.batch.v1.BatchOperationUpdateActivityOptions + Deprecated. Use UpdateActivityExecutionOptionsRequest. + """ DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -7597,6 +7619,8 @@ class UpdateActivityOptionsRequest(google.protobuf.message.Message): global___UpdateActivityOptionsRequest = UpdateActivityOptionsRequest class UpdateActivityOptionsResponse(google.protobuf.message.Message): + """Deprecated. See UpdateActivityExecutionOptionsResponse.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor ACTIVITY_OPTIONS_FIELD_NUMBER: builtins.int @@ -7623,6 +7647,8 @@ class UpdateActivityOptionsResponse(google.protobuf.message.Message): global___UpdateActivityOptionsResponse = UpdateActivityOptionsResponse class PauseActivityRequest(google.protobuf.message.Message): + """Deprecated. See PauseActivityExecutionRequest.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor NAMESPACE_FIELD_NUMBER: builtins.int @@ -7641,7 +7667,9 @@ class PauseActivityRequest(google.protobuf.message.Message): id: builtins.str """Only the activity with this ID will be paused.""" type: builtins.str - """Pause all running activities of this type.""" + """Pause all running activities of this type. + Note: Experimental - the behavior of pause by activity type might change in a future release. + """ reason: builtins.str """Reason to pause the activity.""" def __init__( @@ -7693,6 +7721,8 @@ class PauseActivityRequest(google.protobuf.message.Message): global___PauseActivityRequest = PauseActivityRequest class PauseActivityResponse(google.protobuf.message.Message): + """Deprecated. See PauseActivityExecutionResponse.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor def __init__( @@ -7702,6 +7732,8 @@ class PauseActivityResponse(google.protobuf.message.Message): global___PauseActivityResponse = PauseActivityResponse class UnpauseActivityRequest(google.protobuf.message.Message): + """Deprecated. See UnpauseActivityExecutionRequest.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor NAMESPACE_FIELD_NUMBER: builtins.int @@ -7795,6 +7827,8 @@ class UnpauseActivityRequest(google.protobuf.message.Message): global___UnpauseActivityRequest = UnpauseActivityRequest class UnpauseActivityResponse(google.protobuf.message.Message): + """Deprecated. See UnpauseActivityExecutionResponse.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor def __init__( @@ -7804,7 +7838,9 @@ class UnpauseActivityResponse(google.protobuf.message.Message): global___UnpauseActivityResponse = UnpauseActivityResponse class ResetActivityRequest(google.protobuf.message.Message): - """NOTE: keep in sync with temporalio.api.batch.v1.BatchOperationResetActivities""" + """NOTE: keep in sync with temporalio.api.batch.v1.BatchOperationResetActivities + Deprecated. See ResetActivityExecutionRequest. + """ DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -7912,6 +7948,8 @@ class ResetActivityRequest(google.protobuf.message.Message): global___ResetActivityRequest = ResetActivityRequest class ResetActivityResponse(google.protobuf.message.Message): + """Deprecated. See ResetActivityExecutionResponse.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor def __init__( @@ -8487,6 +8525,7 @@ class SetWorkerDeploymentCurrentVersionRequest(google.protobuf.message.Message): CONFLICT_TOKEN_FIELD_NUMBER: builtins.int IDENTITY_FIELD_NUMBER: builtins.int IGNORE_MISSING_TASK_QUEUES_FIELD_NUMBER: builtins.int + ALLOW_NO_POLLERS_FIELD_NUMBER: builtins.int namespace: builtins.str deployment_name: builtins.str version: builtins.str @@ -8519,6 +8558,11 @@ class SetWorkerDeploymentCurrentVersionRequest(google.protobuf.message.Message): pollers have not reached to the server yet. Only set this if you expect those pollers to never arrive. """ + allow_no_pollers: builtins.bool + """Optional. By default this request will be rejected if no pollers have been seen for the proposed + Current Version, in order to protect users from routing tasks to pollers that do not exist, leading + to possible timeouts. Pass `true` here to bypass this protection. + """ def __init__( self, *, @@ -8529,10 +8573,13 @@ class SetWorkerDeploymentCurrentVersionRequest(google.protobuf.message.Message): conflict_token: builtins.bytes = ..., identity: builtins.str = ..., ignore_missing_task_queues: builtins.bool = ..., + allow_no_pollers: builtins.bool = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ + "allow_no_pollers", + b"allow_no_pollers", "build_id", b"build_id", "conflict_token", @@ -8615,6 +8662,7 @@ class SetWorkerDeploymentRampingVersionRequest(google.protobuf.message.Message): CONFLICT_TOKEN_FIELD_NUMBER: builtins.int IDENTITY_FIELD_NUMBER: builtins.int IGNORE_MISSING_TASK_QUEUES_FIELD_NUMBER: builtins.int + ALLOW_NO_POLLERS_FIELD_NUMBER: builtins.int namespace: builtins.str deployment_name: builtins.str version: builtins.str @@ -8652,6 +8700,11 @@ class SetWorkerDeploymentRampingVersionRequest(google.protobuf.message.Message): that the percentage changes. Also note that the check is against the deployment's Current Version, not the previous Ramping Version. """ + allow_no_pollers: builtins.bool + """Optional. By default this request will be rejected if no pollers have been seen for the proposed + Current Version, in order to protect users from routing tasks to pollers that do not exist, leading + to possible timeouts. Pass `true` here to bypass this protection. + """ def __init__( self, *, @@ -8663,10 +8716,13 @@ class SetWorkerDeploymentRampingVersionRequest(google.protobuf.message.Message): conflict_token: builtins.bytes = ..., identity: builtins.str = ..., ignore_missing_task_queues: builtins.bool = ..., + allow_no_pollers: builtins.bool = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ + "allow_no_pollers", + b"allow_no_pollers", "build_id", b"build_id", "conflict_token", @@ -9126,6 +9182,113 @@ global___UpdateWorkerDeploymentVersionMetadataResponse = ( UpdateWorkerDeploymentVersionMetadataResponse ) +class SetWorkerDeploymentManagerRequest(google.protobuf.message.Message): + """Update the ManagerIdentity of a Worker Deployment.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + DEPLOYMENT_NAME_FIELD_NUMBER: builtins.int + MANAGER_IDENTITY_FIELD_NUMBER: builtins.int + SELF_FIELD_NUMBER: builtins.int + CONFLICT_TOKEN_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + namespace: builtins.str + deployment_name: builtins.str + manager_identity: builtins.str + """Arbitrary value for `manager_identity`. + Empty will unset the field. + """ + self: builtins.bool + """True will set `manager_identity` to `identity`.""" + conflict_token: builtins.bytes + """Optional. This can be the value of conflict_token from a Describe, or another Worker + Deployment API. Passing a non-nil conflict token will cause this request to fail if the + Deployment's configuration has been modified between the API call that generated the + token and this one. + """ + identity: builtins.str + """Required. The identity of the client who initiated this request.""" + def __init__( + # pyright: reportSelfClsParameterName=false + self_, + *, + namespace: builtins.str = ..., + deployment_name: builtins.str = ..., + manager_identity: builtins.str = ..., + self: builtins.bool = ..., + conflict_token: builtins.bytes = ..., + identity: builtins.str = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "manager_identity", + b"manager_identity", + "new_manager_identity", + b"new_manager_identity", + "self", + b"self", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "conflict_token", + b"conflict_token", + "deployment_name", + b"deployment_name", + "identity", + b"identity", + "manager_identity", + b"manager_identity", + "namespace", + b"namespace", + "new_manager_identity", + b"new_manager_identity", + "self", + b"self", + ], + ) -> None: ... + def WhichOneof( + self, + oneof_group: typing_extensions.Literal[ + "new_manager_identity", b"new_manager_identity" + ], + ) -> typing_extensions.Literal["manager_identity", "self"] | None: ... + +global___SetWorkerDeploymentManagerRequest = SetWorkerDeploymentManagerRequest + +class SetWorkerDeploymentManagerResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + CONFLICT_TOKEN_FIELD_NUMBER: builtins.int + PREVIOUS_MANAGER_IDENTITY_FIELD_NUMBER: builtins.int + conflict_token: builtins.bytes + """This value is returned so that it can be optionally passed to APIs + that write to the Worker Deployment state to ensure that the state + did not change between this API call and a future write. + """ + previous_manager_identity: builtins.str + """What the `manager_identity` field was before this change.""" + def __init__( + self, + *, + conflict_token: builtins.bytes = ..., + previous_manager_identity: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "conflict_token", + b"conflict_token", + "previous_manager_identity", + b"previous_manager_identity", + ], + ) -> None: ... + +global___SetWorkerDeploymentManagerResponse = SetWorkerDeploymentManagerResponse + class GetCurrentDeploymentRequest(google.protobuf.message.Message): """Returns the Current Deployment of a deployment series. [cleanup-wv-pre-release] Pre-release deployment APIs, clean up later @@ -9959,3 +10122,1187 @@ class UpdateWorkerConfigResponse(google.protobuf.message.Message): ) -> typing_extensions.Literal["worker_config"] | None: ... global___UpdateWorkerConfigResponse = UpdateWorkerConfigResponse + +class DescribeWorkerRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + WORKER_INSTANCE_KEY_FIELD_NUMBER: builtins.int + namespace: builtins.str + """Namespace this worker belongs to.""" + worker_instance_key: builtins.str + """Worker instance key to describe.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + worker_instance_key: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "namespace", b"namespace", "worker_instance_key", b"worker_instance_key" + ], + ) -> None: ... + +global___DescribeWorkerRequest = DescribeWorkerRequest + +class DescribeWorkerResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + WORKER_INFO_FIELD_NUMBER: builtins.int + @property + def worker_info(self) -> temporalio.api.worker.v1.message_pb2.WorkerInfo: ... + def __init__( + self, + *, + worker_info: temporalio.api.worker.v1.message_pb2.WorkerInfo | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["worker_info", b"worker_info"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["worker_info", b"worker_info"] + ) -> None: ... + +global___DescribeWorkerResponse = DescribeWorkerResponse + +class StartActivityExecutionRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + REQUEST_ID_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + OPTIONS_FIELD_NUMBER: builtins.int + INPUT_FIELD_NUMBER: builtins.int + ID_REUSE_POLICY_FIELD_NUMBER: builtins.int + ID_CONFLICT_POLICY_FIELD_NUMBER: builtins.int + MEMO_FIELD_NUMBER: builtins.int + SEARCH_ATTRIBUTES_FIELD_NUMBER: builtins.int + HEADER_FIELD_NUMBER: builtins.int + REQUEST_EAGER_EXECUTION_FIELD_NUMBER: builtins.int + COMPLETION_CALLBACKS_FIELD_NUMBER: builtins.int + USER_METADATA_FIELD_NUMBER: builtins.int + LINKS_FIELD_NUMBER: builtins.int + ON_CONFLICT_OPTIONS_FIELD_NUMBER: builtins.int + PRIORITY_FIELD_NUMBER: builtins.int + namespace: builtins.str + identity: builtins.str + """The identity of the client who initiated this request""" + request_id: builtins.str + """A unique identifier for this start request. Typically UUIDv4.""" + activity_id: builtins.str + @property + def activity_type(self) -> temporalio.api.common.v1.message_pb2.ActivityType: ... + @property + def options(self) -> temporalio.api.activity.v1.message_pb2.ActivityOptions: ... + @property + def input(self) -> temporalio.api.common.v1.message_pb2.Payloads: + """Serialized arguments to the activity. These are passed as arguments to the activity function.""" + id_reuse_policy: temporalio.api.enums.v1.id_pb2.IdReusePolicy.ValueType + """Defines whether to allow re-using the activity id from a previously *closed* activity. + The default policy is ID_REUSE_POLICY_ALLOW_DUPLICATE. + """ + id_conflict_policy: temporalio.api.enums.v1.id_pb2.IdConflictPolicy.ValueType + """Defines how to resolve an activity id conflict with a *running* activity. + The default policy is ID_CONFLICT_POLICY_FAIL. + """ + @property + def memo(self) -> temporalio.api.common.v1.message_pb2.Memo: + """Arbitrary structured data that can be attached to the activity execution and made available via the list and + describe APIs. + """ + @property + def search_attributes( + self, + ) -> temporalio.api.common.v1.message_pb2.SearchAttributes: + """Search attributes for indexing.""" + @property + def header(self) -> temporalio.api.common.v1.message_pb2.Header: + """Header for context propagation and tracing purposes.""" + request_eager_execution: builtins.bool + """Request to get the first activity task inline in the response bypassing matching service and worker polling. + If set to `true` the caller is expected to have a worker available and capable of processing the task. + The returned task will be marked as started and is expected to be completed by the specified + `schedule_to_close_timeout`. + """ + @property + def completion_callbacks( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + temporalio.api.common.v1.message_pb2.Callback + ]: + """Callbacks to be called by the server when this activity reaches a terminal status. + Callback addresses must be whitelisted in the server's dynamic configuration. + """ + @property + def user_metadata(self) -> temporalio.api.sdk.v1.user_metadata_pb2.UserMetadata: + """Metadata for use by user interfaces to display the fixed as-of-start summary and details of the activity.""" + @property + def links( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + temporalio.api.common.v1.message_pb2.Link + ]: + """Links to be associated with the activity.""" + @property + def on_conflict_options( + self, + ) -> temporalio.api.activity.v1.message_pb2.OnConflictOptions: + """Defines actions to be done to the existing running activity when ID_CONFLICT_POLICY_USE_EXISTING is used. If not + set or empty, it won't do anything to the existing running activity. + """ + @property + def priority(self) -> temporalio.api.common.v1.message_pb2.Priority: + """Priority metadata""" + def __init__( + self, + *, + namespace: builtins.str = ..., + identity: builtins.str = ..., + request_id: builtins.str = ..., + activity_id: builtins.str = ..., + activity_type: temporalio.api.common.v1.message_pb2.ActivityType | None = ..., + options: temporalio.api.activity.v1.message_pb2.ActivityOptions | None = ..., + input: temporalio.api.common.v1.message_pb2.Payloads | None = ..., + id_reuse_policy: temporalio.api.enums.v1.id_pb2.IdReusePolicy.ValueType = ..., + id_conflict_policy: temporalio.api.enums.v1.id_pb2.IdConflictPolicy.ValueType = ..., + memo: temporalio.api.common.v1.message_pb2.Memo | None = ..., + search_attributes: temporalio.api.common.v1.message_pb2.SearchAttributes + | None = ..., + header: temporalio.api.common.v1.message_pb2.Header | None = ..., + request_eager_execution: builtins.bool = ..., + completion_callbacks: collections.abc.Iterable[ + temporalio.api.common.v1.message_pb2.Callback + ] + | None = ..., + user_metadata: temporalio.api.sdk.v1.user_metadata_pb2.UserMetadata + | None = ..., + links: collections.abc.Iterable[temporalio.api.common.v1.message_pb2.Link] + | None = ..., + on_conflict_options: temporalio.api.activity.v1.message_pb2.OnConflictOptions + | None = ..., + priority: temporalio.api.common.v1.message_pb2.Priority | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "activity_type", + b"activity_type", + "header", + b"header", + "input", + b"input", + "memo", + b"memo", + "on_conflict_options", + b"on_conflict_options", + "options", + b"options", + "priority", + b"priority", + "search_attributes", + b"search_attributes", + "user_metadata", + b"user_metadata", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "activity_type", + b"activity_type", + "completion_callbacks", + b"completion_callbacks", + "header", + b"header", + "id_conflict_policy", + b"id_conflict_policy", + "id_reuse_policy", + b"id_reuse_policy", + "identity", + b"identity", + "input", + b"input", + "links", + b"links", + "memo", + b"memo", + "namespace", + b"namespace", + "on_conflict_options", + b"on_conflict_options", + "options", + b"options", + "priority", + b"priority", + "request_eager_execution", + b"request_eager_execution", + "request_id", + b"request_id", + "search_attributes", + b"search_attributes", + "user_metadata", + b"user_metadata", + ], + ) -> None: ... + +global___StartActivityExecutionRequest = StartActivityExecutionRequest + +class StartActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + RUN_ID_FIELD_NUMBER: builtins.int + STARTED_FIELD_NUMBER: builtins.int + EAGER_TASK_FIELD_NUMBER: builtins.int + LINK_FIELD_NUMBER: builtins.int + run_id: builtins.str + """The run ID of the activity that was started - or used (via ID_CONFLICT_POLICY_USE_EXISTING).""" + started: builtins.bool + """If true, a new activity was started.""" + @property + def eager_task(self) -> global___PollActivityTaskQueueResponse: + """When `request_eager_execution` is set on the `StartActivityExecutionRequest`, the server will return the first + activity task to be eagerly executed. + The caller is expected to have a worker available to process the task. + """ + @property + def link(self) -> temporalio.api.common.v1.message_pb2.Link: + """Link to the workflow event.""" + def __init__( + self, + *, + run_id: builtins.str = ..., + started: builtins.bool = ..., + eager_task: global___PollActivityTaskQueueResponse | None = ..., + link: temporalio.api.common.v1.message_pb2.Link | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "eager_task", b"eager_task", "link", b"link" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "eager_task", + b"eager_task", + "link", + b"link", + "run_id", + b"run_id", + "started", + b"started", + ], + ) -> None: ... + +global___StartActivityExecutionResponse = StartActivityExecutionResponse + +class DescribeActivityExecutionRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + INCLUDE_INPUT_FIELD_NUMBER: builtins.int + LONG_POLL_TOKEN_FIELD_NUMBER: builtins.int + namespace: builtins.str + activity_id: builtins.str + run_id: builtins.str + """Activity run ID, targets the latest run if run_id is empty.""" + include_input: builtins.bool + """If true, the activity input is returned in the response.""" + long_poll_token: builtins.bytes + """If not empty, turns this request into a long poll that is unblocked when the activity state changes from the time + the token was returned. + This token is returned as part of the `DescribeActivityExecutionResponse`. + """ + def __init__( + self, + *, + namespace: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + include_input: builtins.bool = ..., + long_poll_token: builtins.bytes = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "include_input", + b"include_input", + "long_poll_token", + b"long_poll_token", + "namespace", + b"namespace", + "run_id", + b"run_id", + ], + ) -> None: ... + +global___DescribeActivityExecutionRequest = DescribeActivityExecutionRequest + +class DescribeActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + INFO_FIELD_NUMBER: builtins.int + LONG_POLL_TOKEN_FIELD_NUMBER: builtins.int + @property + def info(self) -> temporalio.api.activity.v1.message_pb2.ActivityExecutionInfo: ... + long_poll_token: builtins.bytes + """A token that can be passed in via a subsequent `DescribeActivityExecutionRequest` to long poll on the activity + state as it makes progress. + """ + def __init__( + self, + *, + info: temporalio.api.activity.v1.message_pb2.ActivityExecutionInfo | None = ..., + long_poll_token: builtins.bytes = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["info", b"info"] + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "info", b"info", "long_poll_token", b"long_poll_token" + ], + ) -> None: ... + +global___DescribeActivityExecutionResponse = DescribeActivityExecutionResponse + +class ListActivityExecutionsRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + PAGE_SIZE_FIELD_NUMBER: builtins.int + NEXT_PAGE_TOKEN_FIELD_NUMBER: builtins.int + QUERY_FIELD_NUMBER: builtins.int + namespace: builtins.str + page_size: builtins.int + """Max number of executions to return per page.""" + next_page_token: builtins.bytes + """Token returned in ListActivityExecutionsResponse.""" + query: builtins.str + """Visibility query, see https://docs.temporal.io/list-filter for the syntax.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + page_size: builtins.int = ..., + next_page_token: builtins.bytes = ..., + query: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "namespace", + b"namespace", + "next_page_token", + b"next_page_token", + "page_size", + b"page_size", + "query", + b"query", + ], + ) -> None: ... + +global___ListActivityExecutionsRequest = ListActivityExecutionsRequest + +class ListActivityExecutionsResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + EXECUTIONS_FIELD_NUMBER: builtins.int + NEXT_PAGE_TOKEN_FIELD_NUMBER: builtins.int + @property + def executions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + temporalio.api.activity.v1.message_pb2.ActivityListInfo + ]: ... + next_page_token: builtins.bytes + """Token to use to fetch the next page. If empty, there is no next page.""" + def __init__( + self, + *, + executions: collections.abc.Iterable[ + temporalio.api.activity.v1.message_pb2.ActivityListInfo + ] + | None = ..., + next_page_token: builtins.bytes = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "executions", b"executions", "next_page_token", b"next_page_token" + ], + ) -> None: ... + +global___ListActivityExecutionsResponse = ListActivityExecutionsResponse + +class CountActivityExecutionsRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + QUERY_FIELD_NUMBER: builtins.int + namespace: builtins.str + query: builtins.str + """Visibility query, see https://docs.temporal.io/list-filter for the syntax.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + query: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "namespace", b"namespace", "query", b"query" + ], + ) -> None: ... + +global___CountActivityExecutionsRequest = CountActivityExecutionsRequest + +class CountActivityExecutionsResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class AggregationGroup(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + GROUP_VALUES_FIELD_NUMBER: builtins.int + COUNT_FIELD_NUMBER: builtins.int + @property + def group_values( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + temporalio.api.common.v1.message_pb2.Payload + ]: ... + count: builtins.int + def __init__( + self, + *, + group_values: collections.abc.Iterable[ + temporalio.api.common.v1.message_pb2.Payload + ] + | None = ..., + count: builtins.int = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "count", b"count", "group_values", b"group_values" + ], + ) -> None: ... + + COUNT_FIELD_NUMBER: builtins.int + GROUPS_FIELD_NUMBER: builtins.int + count: builtins.int + """If `query` is not grouping by any field, the count is an approximate number + of activities that match the query. + If `query` is grouping by a field, the count is simply the sum of the counts + of the groups returned in the response. This number can be smaller than the + total number of activities matching the query. + """ + @property + def groups( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___CountActivityExecutionsResponse.AggregationGroup + ]: + """Contains the groups if the request is grouping by a field. + The list might not be complete, and the counts of each group is approximate. + """ + def __init__( + self, + *, + count: builtins.int = ..., + groups: collections.abc.Iterable[ + global___CountActivityExecutionsResponse.AggregationGroup + ] + | None = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal["count", b"count", "groups", b"groups"], + ) -> None: ... + +global___CountActivityExecutionsResponse = CountActivityExecutionsResponse + +class GetActivityExecutionResultRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + WAIT_FIELD_NUMBER: builtins.int + namespace: builtins.str + activity_id: builtins.str + run_id: builtins.str + """Activity run ID, targets the latest run if run_id is empty.""" + wait: builtins.bool + """If set, turns this request into a long poll that is unblocked when the activity reaches a terminal status. + The wait duration is capped by the request's context deadline or by the maximum enforced long poll interval + allowed by the server. + """ + def __init__( + self, + *, + namespace: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + wait: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "namespace", + b"namespace", + "run_id", + b"run_id", + "wait", + b"wait", + ], + ) -> None: ... + +global___GetActivityExecutionResultRequest = GetActivityExecutionResultRequest + +class GetActivityExecutionResultResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + RUN_ID_FIELD_NUMBER: builtins.int + RESULT_FIELD_NUMBER: builtins.int + FAILURE_FIELD_NUMBER: builtins.int + run_id: builtins.str + """The run ID of the completed activity, may be used in case a run ID was not specified in the request.""" + @property + def result(self) -> temporalio.api.common.v1.message_pb2.Payloads: + """The result if the activity completed successfully.""" + @property + def failure(self) -> temporalio.api.failure.v1.message_pb2.Failure: + """The failure if the activity completed unsuccessfully.""" + def __init__( + self, + *, + run_id: builtins.str = ..., + result: temporalio.api.common.v1.message_pb2.Payloads | None = ..., + failure: temporalio.api.failure.v1.message_pb2.Failure | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "failure", b"failure", "outcome", b"outcome", "result", b"result" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "failure", + b"failure", + "outcome", + b"outcome", + "result", + b"result", + "run_id", + b"run_id", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["outcome", b"outcome"] + ) -> typing_extensions.Literal["result", "failure"] | None: ... + +global___GetActivityExecutionResultResponse = GetActivityExecutionResultResponse + +class RequestCancelActivityExecutionRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + REQUEST_ID_FIELD_NUMBER: builtins.int + REASON_FIELD_NUMBER: builtins.int + namespace: builtins.str + activity_id: builtins.str + run_id: builtins.str + """Activity run ID, targets the latest run if run_id is empty.""" + identity: builtins.str + """The identity of the worker/client.""" + request_id: builtins.str + """Used to de-dupe cancellation requests.""" + reason: builtins.str + """Reason for requesting the cancellation, recorded and available via the DescribeActivityExecution API. + Not propagated to a worker if an activity attempt is currently running. + """ + def __init__( + self, + *, + namespace: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + identity: builtins.str = ..., + request_id: builtins.str = ..., + reason: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "identity", + b"identity", + "namespace", + b"namespace", + "reason", + b"reason", + "request_id", + b"request_id", + "run_id", + b"run_id", + ], + ) -> None: ... + +global___RequestCancelActivityExecutionRequest = RequestCancelActivityExecutionRequest + +class RequestCancelActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___RequestCancelActivityExecutionResponse = RequestCancelActivityExecutionResponse + +class TerminateActivityExecutionRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + REASON_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + namespace: builtins.str + activity_id: builtins.str + run_id: builtins.str + """Activity run ID, targets the latest run if run_id is empty.""" + reason: builtins.str + """Reason for requesting the termination, recorded in in the activity's result failure outcome.""" + identity: builtins.str + """The identity of the worker/client.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + reason: builtins.str = ..., + identity: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "identity", + b"identity", + "namespace", + b"namespace", + "reason", + b"reason", + "run_id", + b"run_id", + ], + ) -> None: ... + +global___TerminateActivityExecutionRequest = TerminateActivityExecutionRequest + +class TerminateActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___TerminateActivityExecutionResponse = TerminateActivityExecutionResponse + +class DeleteActivityExecutionRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + namespace: builtins.str + activity_id: builtins.str + run_id: builtins.str + """Activity run ID, targets the latest run if run_id is empty.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "namespace", + b"namespace", + "run_id", + b"run_id", + ], + ) -> None: ... + +global___DeleteActivityExecutionRequest = DeleteActivityExecutionRequest + +class DeleteActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___DeleteActivityExecutionResponse = DeleteActivityExecutionResponse + +class UpdateActivityExecutionOptionsRequest(google.protobuf.message.Message): + """TODO: update batch + NOTE: keep in sync with temporalio.api.batch.v1.BatchOperationUpdateActivityOptions + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + WORKFLOW_ID_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + WORKFLOW_ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + ALL_WORKFLOW_ACTIVITIES_FIELD_NUMBER: builtins.int + ACTIVITY_OPTIONS_FIELD_NUMBER: builtins.int + UPDATE_MASK_FIELD_NUMBER: builtins.int + RESTORE_ORIGINAL_FIELD_NUMBER: builtins.int + namespace: builtins.str + """Namespace of the workflow which scheduled this activity""" + workflow_id: builtins.str + """If provided, update options for a workflow activity (or activities) for the given workflow ID. If empty, target a + standalone activity. + """ + activity_id: builtins.str + """Update options for an activity with this ID. Must be provided for a standalone activity. + Mutually exclusive with workflow_activity_type and all_workflow_activities. + """ + run_id: builtins.str + """Run ID of the workflow or standalone activity.""" + identity: builtins.str + """The identity of the client who initiated this request""" + workflow_activity_type: builtins.str + """Update all pending workflow activities of this type. + Only available if workflow_id is provided. + Mutually exclusive with activity_id and all_workflow_activities. + + Note: Experimental - the behavior of updating by activity type may change or be removed in a future release. + """ + all_workflow_activities: builtins.bool + """Update all pending workflow activities. + Only available if workflow_id is provided. + Mutually exclusive with activity_id and workflow_activity_type. + + Note: Experimental - the behavior of updating all activities may change or be removed in a future release. + """ + @property + def activity_options( + self, + ) -> temporalio.api.activity.v1.message_pb2.ActivityOptions: + """Activity options. Partial updates are accepted and controlled by update_mask + Mutually exclusive with restore_original. + """ + @property + def update_mask(self) -> google.protobuf.field_mask_pb2.FieldMask: + """Controls which fields from `activity_options` will be applied""" + restore_original: builtins.bool + """If set, the activity options will be restored to the defaults. + Default options are then options activity was originally created with. + For workflow activities the original options are restored from first ActivityTaskScheduled event. + Mutually exclusive with activity_options. + """ + def __init__( + self, + *, + namespace: builtins.str = ..., + workflow_id: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + identity: builtins.str = ..., + workflow_activity_type: builtins.str = ..., + all_workflow_activities: builtins.bool = ..., + activity_options: temporalio.api.activity.v1.message_pb2.ActivityOptions + | None = ..., + update_mask: google.protobuf.field_mask_pb2.FieldMask | None = ..., + restore_original: builtins.bool = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "activity_options", b"activity_options", "update_mask", b"update_mask" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "activity_options", + b"activity_options", + "all_workflow_activities", + b"all_workflow_activities", + "identity", + b"identity", + "namespace", + b"namespace", + "restore_original", + b"restore_original", + "run_id", + b"run_id", + "update_mask", + b"update_mask", + "workflow_activity_type", + b"workflow_activity_type", + "workflow_id", + b"workflow_id", + ], + ) -> None: ... + +global___UpdateActivityExecutionOptionsRequest = UpdateActivityExecutionOptionsRequest + +class UpdateActivityExecutionOptionsResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ACTIVITY_OPTIONS_FIELD_NUMBER: builtins.int + @property + def activity_options( + self, + ) -> temporalio.api.activity.v1.message_pb2.ActivityOptions: + """Activity options after an update""" + def __init__( + self, + *, + activity_options: temporalio.api.activity.v1.message_pb2.ActivityOptions + | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal["activity_options", b"activity_options"], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal["activity_options", b"activity_options"], + ) -> None: ... + +global___UpdateActivityExecutionOptionsResponse = UpdateActivityExecutionOptionsResponse + +class PauseActivityExecutionRequest(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + WORKFLOW_ID_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + WORKFLOW_ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + REASON_FIELD_NUMBER: builtins.int + namespace: builtins.str + """Namespace of the workflow which scheduled this activity.""" + workflow_id: builtins.str + """If provided, pause a workflow activity (or activities) for the given workflow ID. + If empty, target a standalone activity. + """ + activity_id: builtins.str + """Pause an activity with this ID. Must be provided for a standalone activity. + Mutually exclusive with workflow_activity_type. + """ + run_id: builtins.str + """Run ID of the workflow or standalone activity.""" + identity: builtins.str + """The identity of the client who initiated this request.""" + workflow_activity_type: builtins.str + """Pause all pending activities of this type. + Only available if workflow_id is provided. + Mutually exclusive with activity_id. + + Note: Experimental - the behavior of pausing by activity type might change or be removed in a future release. + """ + reason: builtins.str + """Reason to pause the activity.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + workflow_id: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + identity: builtins.str = ..., + workflow_activity_type: builtins.str = ..., + reason: builtins.str = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "identity", + b"identity", + "namespace", + b"namespace", + "reason", + b"reason", + "run_id", + b"run_id", + "workflow_activity_type", + b"workflow_activity_type", + "workflow_id", + b"workflow_id", + ], + ) -> None: ... + +global___PauseActivityExecutionRequest = PauseActivityExecutionRequest + +class PauseActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___PauseActivityExecutionResponse = PauseActivityExecutionResponse + +class UnpauseActivityExecutionRequest(google.protobuf.message.Message): + """TODO: update batch""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + WORKFLOW_ID_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + WORKFLOW_ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + ALL_WORKFLOW_ACTIVITIES_FIELD_NUMBER: builtins.int + RESET_ATTEMPTS_FIELD_NUMBER: builtins.int + RESET_HEARTBEAT_FIELD_NUMBER: builtins.int + JITTER_FIELD_NUMBER: builtins.int + namespace: builtins.str + """Namespace of the workflow which scheduled this activity.""" + workflow_id: builtins.str + """If provided, unpause a workflow activity (or activities) for the given workflow ID. + If empty, target a standalone activity. + """ + activity_id: builtins.str + """Unpause an activity with this ID. Must be provided for a standalone activity. + Mutually exclusive with workflow_activity_type and all_workflow_activities. + """ + run_id: builtins.str + """Run ID of the workflow or standalone activity.""" + identity: builtins.str + """The identity of the client who initiated this request.""" + workflow_activity_type: builtins.str + """Unpause all currently paused workflow activities of this type. + Only available if workflow_id is provided. + Mutually exclusive with activity_id and all_workflow_activities. + + Note: Experimental - the behavior of unpausing by activity type may change or be removed in a future release. + """ + all_workflow_activities: builtins.bool + """Unpause all paused workflow activities. + Only available if workflow_id is provided. + Mutually exclusive with activity_id and workflow_activity_type. + + Note: Experimental - the behavior of unpausing all activities may change or be removed in a future release. + """ + reset_attempts: builtins.bool + """Providing this flag will also reset the number of attempts.""" + reset_heartbeat: builtins.bool + """Providing this flag will also reset the heartbeat details.""" + @property + def jitter(self) -> google.protobuf.duration_pb2.Duration: + """If set, the activity will start at a random time within the specified jitter duration.""" + def __init__( + self, + *, + namespace: builtins.str = ..., + workflow_id: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + identity: builtins.str = ..., + workflow_activity_type: builtins.str = ..., + all_workflow_activities: builtins.bool = ..., + reset_attempts: builtins.bool = ..., + reset_heartbeat: builtins.bool = ..., + jitter: google.protobuf.duration_pb2.Duration | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["jitter", b"jitter"] + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "all_workflow_activities", + b"all_workflow_activities", + "identity", + b"identity", + "jitter", + b"jitter", + "namespace", + b"namespace", + "reset_attempts", + b"reset_attempts", + "reset_heartbeat", + b"reset_heartbeat", + "run_id", + b"run_id", + "workflow_activity_type", + b"workflow_activity_type", + "workflow_id", + b"workflow_id", + ], + ) -> None: ... + +global___UnpauseActivityExecutionRequest = UnpauseActivityExecutionRequest + +class UnpauseActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___UnpauseActivityExecutionResponse = UnpauseActivityExecutionResponse + +class ResetActivityExecutionRequest(google.protobuf.message.Message): + """TODO: update batch + NOTE: keep in sync with temporalio.api.batch.v1.BatchOperationResetActivities + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NAMESPACE_FIELD_NUMBER: builtins.int + WORKFLOW_ID_FIELD_NUMBER: builtins.int + ACTIVITY_ID_FIELD_NUMBER: builtins.int + RUN_ID_FIELD_NUMBER: builtins.int + IDENTITY_FIELD_NUMBER: builtins.int + WORKFLOW_ACTIVITY_TYPE_FIELD_NUMBER: builtins.int + ALL_WORKFLOW_ACTIVITIES_FIELD_NUMBER: builtins.int + RESET_HEARTBEAT_FIELD_NUMBER: builtins.int + KEEP_PAUSED_FIELD_NUMBER: builtins.int + JITTER_FIELD_NUMBER: builtins.int + RESTORE_ORIGINAL_OPTIONS_FIELD_NUMBER: builtins.int + namespace: builtins.str + """Namespace of the workflow which scheduled this activity.""" + workflow_id: builtins.str + """If provided, reset a workflow activity (or activities) for the given workflow ID. + If empty, target a standalone activity. + """ + activity_id: builtins.str + """Reset an activity with this ID. Must be provided for a standalone activity. + Mutually exclusive with workflow_activity_type and all_workflow_activities. + """ + run_id: builtins.str + """Run ID of the workflow or standalone activity.""" + identity: builtins.str + """The identity of the client who initiated this request.""" + workflow_activity_type: builtins.str + """Reset all pending workflow activities of this type. + Only available if workflow_id is provided. + Mutually exclusive with activity_id and all_workflow_activities. + + Note: Experimental - the behavior of resetting by activity type may change or be removed in a future release. + """ + all_workflow_activities: builtins.bool + """Reset all pending workflow activities. + Only available if workflow_id is provided. + Mutually exclusive with activity_id and workflow_activity_type. + + Note: Experimental - the behavior of resetting all activities may change or be removed in a future release. + """ + reset_heartbeat: builtins.bool + """Indicates that activity should reset heartbeat details. + This flag will be applied only to the new instance of the activity. + """ + keep_paused: builtins.bool + """If activity is paused, it will remain paused after reset""" + @property + def jitter(self) -> google.protobuf.duration_pb2.Duration: + """If set, and activity is in backoff, the activity will start at a random time within the specified jitter duration. + (unless it is paused and keep_paused is set) + """ + restore_original_options: builtins.bool + """If set, the activity options will be restored to the defaults. + Default options are then options activity was originally created with. + For workflow activities the original options are restored from first ActivityTaskScheduled event. + """ + def __init__( + self, + *, + namespace: builtins.str = ..., + workflow_id: builtins.str = ..., + activity_id: builtins.str = ..., + run_id: builtins.str = ..., + identity: builtins.str = ..., + workflow_activity_type: builtins.str = ..., + all_workflow_activities: builtins.bool = ..., + reset_heartbeat: builtins.bool = ..., + keep_paused: builtins.bool = ..., + jitter: google.protobuf.duration_pb2.Duration | None = ..., + restore_original_options: builtins.bool = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["jitter", b"jitter"] + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "activity_id", + b"activity_id", + "all_workflow_activities", + b"all_workflow_activities", + "identity", + b"identity", + "jitter", + b"jitter", + "keep_paused", + b"keep_paused", + "namespace", + b"namespace", + "reset_heartbeat", + b"reset_heartbeat", + "restore_original_options", + b"restore_original_options", + "run_id", + b"run_id", + "workflow_activity_type", + b"workflow_activity_type", + "workflow_id", + b"workflow_id", + ], + ) -> None: ... + +global___ResetActivityExecutionRequest = ResetActivityExecutionRequest + +class ResetActivityExecutionResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + +global___ResetActivityExecutionResponse = ResetActivityExecutionResponse diff --git a/temporalio/api/workflowservice/v1/service_pb2.py b/temporalio/api/workflowservice/v1/service_pb2.py index 5cc1619ef..7b83723ca 100644 --- a/temporalio/api/workflowservice/v1/service_pb2.py +++ b/temporalio/api/workflowservice/v1/service_pb2.py @@ -21,7 +21,7 @@ ) DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n-temporal/api/workflowservice/v1/service.proto\x12\x1ftemporal.api.workflowservice.v1\x1a\x36temporal/api/workflowservice/v1/request_response.proto\x1a\x1cgoogle/api/annotations.proto2\x8f\xbd\x01\n\x0fWorkflowService\x12\xc3\x01\n\x11RegisterNamespace\x12\x39.temporal.api.workflowservice.v1.RegisterNamespaceRequest\x1a:.temporal.api.workflowservice.v1.RegisterNamespaceResponse"7\x82\xd3\xe4\x93\x02\x31"\x13/cluster/namespaces:\x01*Z\x17"\x12/api/v1/namespaces:\x01*\x12\xd5\x01\n\x11\x44\x65scribeNamespace\x12\x39.temporal.api.workflowservice.v1.DescribeNamespaceRequest\x1a:.temporal.api.workflowservice.v1.DescribeNamespaceResponse"I\x82\xd3\xe4\x93\x02\x43\x12\x1f/cluster/namespaces/{namespace}Z \x12\x1e/api/v1/namespaces/{namespace}\x12\xb4\x01\n\x0eListNamespaces\x12\x36.temporal.api.workflowservice.v1.ListNamespacesRequest\x1a\x37.temporal.api.workflowservice.v1.ListNamespacesResponse"1\x82\xd3\xe4\x93\x02+\x12\x13/cluster/namespacesZ\x14\x12\x12/api/v1/namespaces\x12\xe3\x01\n\x0fUpdateNamespace\x12\x37.temporal.api.workflowservice.v1.UpdateNamespaceRequest\x1a\x38.temporal.api.workflowservice.v1.UpdateNamespaceResponse"]\x82\xd3\xe4\x93\x02W"&/cluster/namespaces/{namespace}/update:\x01*Z*"%/api/v1/namespaces/{namespace}/update:\x01*\x12\x8f\x01\n\x12\x44\x65precateNamespace\x12:.temporal.api.workflowservice.v1.DeprecateNamespaceRequest\x1a;.temporal.api.workflowservice.v1.DeprecateNamespaceResponse"\x00\x12\x92\x02\n\x16StartWorkflowExecution\x12>.temporal.api.workflowservice.v1.StartWorkflowExecutionRequest\x1a?.temporal.api.workflowservice.v1.StartWorkflowExecutionResponse"w\x82\xd3\xe4\x93\x02q"//namespaces/{namespace}/workflows/{workflow_id}:\x01*Z;"6/api/v1/namespaces/{namespace}/workflows/{workflow_id}:\x01*\x12\xa5\x02\n\x15\x45xecuteMultiOperation\x12=.temporal.api.workflowservice.v1.ExecuteMultiOperationRequest\x1a>.temporal.api.workflowservice.v1.ExecuteMultiOperationResponse"\x8c\x01\x82\xd3\xe4\x93\x02\x85\x01"9/namespaces/{namespace}/workflows/execute-multi-operation:\x01*ZE"@/api/v1/namespaces/{namespace}/workflows/execute-multi-operation:\x01*\x12\xc1\x02\n\x1bGetWorkflowExecutionHistory\x12\x43.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest\x1a\x44.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse"\x96\x01\x82\xd3\xe4\x93\x02\x8f\x01\x12\x41/namespaces/{namespace}/workflows/{execution.workflow_id}/historyZJ\x12H/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history\x12\xe6\x02\n"GetWorkflowExecutionHistoryReverse\x12J.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseRequest\x1aK.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseResponse"\xa6\x01\x82\xd3\xe4\x93\x02\x9f\x01\x12I/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverseZR\x12P/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverse\x12\x98\x01\n\x15PollWorkflowTaskQueue\x12=.temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest\x1a>.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse"\x00\x12\xad\x01\n\x1cRespondWorkflowTaskCompleted\x12\x44.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest\x1a\x45.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse"\x00\x12\xa4\x01\n\x19RespondWorkflowTaskFailed\x12\x41.temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondWorkflowTaskFailedResponse"\x00\x12\x98\x01\n\x15PollActivityTaskQueue\x12=.temporal.api.workflowservice.v1.PollActivityTaskQueueRequest\x1a>.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse"\x00\x12\x9b\x02\n\x1bRecordActivityTaskHeartbeat\x12\x43.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatRequest\x1a\x44.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatResponse"q\x82\xd3\xe4\x93\x02k",/namespaces/{namespace}/activities/heartbeat:\x01*Z8"3/api/v1/namespaces/{namespace}/activities/heartbeat:\x01*\x12\xb3\x02\n\x1fRecordActivityTaskHeartbeatById\x12G.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest\x1aH.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdResponse"}\x82\xd3\xe4\x93\x02w"2/namespaces/{namespace}/activities/heartbeat-by-id:\x01*Z>"9/api/v1/namespaces/{namespace}/activities/heartbeat-by-id:\x01*\x12\x9c\x02\n\x1cRespondActivityTaskCompleted\x12\x44.temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest\x1a\x45.temporal.api.workflowservice.v1.RespondActivityTaskCompletedResponse"o\x82\xd3\xe4\x93\x02i"+/namespaces/{namespace}/activities/complete:\x01*Z7"2/api/v1/namespaces/{namespace}/activities/complete:\x01*\x12\xb4\x02\n RespondActivityTaskCompletedById\x12H.temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest\x1aI.temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdResponse"{\x82\xd3\xe4\x93\x02u"1/namespaces/{namespace}/activities/complete-by-id:\x01*Z="8/api/v1/namespaces/{namespace}/activities/complete-by-id:\x01*\x12\x8b\x02\n\x19RespondActivityTaskFailed\x12\x41.temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondActivityTaskFailedResponse"g\x82\xd3\xe4\x93\x02\x61"\'/namespaces/{namespace}/activities/fail:\x01*Z3"./api/v1/namespaces/{namespace}/activities/fail:\x01*\x12\xa3\x02\n\x1dRespondActivityTaskFailedById\x12\x45.temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest\x1a\x46.temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdResponse"s\x82\xd3\xe4\x93\x02m"-/namespaces/{namespace}/activities/fail-by-id:\x01*Z9"4/api/v1/namespaces/{namespace}/activities/fail-by-id:\x01*\x12\x95\x02\n\x1bRespondActivityTaskCanceled\x12\x43.temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest\x1a\x44.temporal.api.workflowservice.v1.RespondActivityTaskCanceledResponse"k\x82\xd3\xe4\x93\x02\x65")/namespaces/{namespace}/activities/cancel:\x01*Z5"0/api/v1/namespaces/{namespace}/activities/cancel:\x01*\x12\xad\x02\n\x1fRespondActivityTaskCanceledById\x12G.temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest\x1aH.temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdResponse"w\x82\xd3\xe4\x93\x02q"//namespaces/{namespace}/activities/cancel-by-id:\x01*Z;"6/api/v1/namespaces/{namespace}/activities/cancel-by-id:\x01*\x12\xe0\x02\n\x1eRequestCancelWorkflowExecution\x12\x46.temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest\x1aG.temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionResponse"\xac\x01\x82\xd3\xe4\x93\x02\xa5\x01"I/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel:\x01*ZU"P/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel:\x01*\x12\xe7\x02\n\x17SignalWorkflowExecution\x12?.temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest\x1a@.temporal.api.workflowservice.v1.SignalWorkflowExecutionResponse"\xc8\x01\x82\xd3\xe4\x93\x02\xc1\x01"W/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}:\x01*Zc"^/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}:\x01*\x12\xf2\x02\n SignalWithStartWorkflowExecution\x12H.temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest\x1aI.temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionResponse"\xb8\x01\x82\xd3\xe4\x93\x02\xb1\x01"O/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}:\x01*Z["V/api/v1/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}:\x01*\x12\xc6\x02\n\x16ResetWorkflowExecution\x12>.temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest\x1a?.temporal.api.workflowservice.v1.ResetWorkflowExecutionResponse"\xaa\x01\x82\xd3\xe4\x93\x02\xa3\x01"H/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset:\x01*ZT"O/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset:\x01*\x12\xda\x02\n\x1aTerminateWorkflowExecution\x12\x42.temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest\x1a\x43.temporal.api.workflowservice.v1.TerminateWorkflowExecutionResponse"\xb2\x01\x82\xd3\xe4\x93\x02\xab\x01"L/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate:\x01*ZX"S/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate:\x01*\x12\x9e\x01\n\x17\x44\x65leteWorkflowExecution\x12?.temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest\x1a@.temporal.api.workflowservice.v1.DeleteWorkflowExecutionResponse"\x00\x12\xa7\x01\n\x1aListOpenWorkflowExecutions\x12\x42.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest\x1a\x43.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse"\x00\x12\xad\x01\n\x1cListClosedWorkflowExecutions\x12\x44.temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest\x1a\x45.temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse"\x00\x12\xf0\x01\n\x16ListWorkflowExecutions\x12>.temporal.api.workflowservice.v1.ListWorkflowExecutionsRequest\x1a?.temporal.api.workflowservice.v1.ListWorkflowExecutionsResponse"U\x82\xd3\xe4\x93\x02O\x12!/namespaces/{namespace}/workflowsZ*\x12(/api/v1/namespaces/{namespace}/workflows\x12\x9a\x02\n\x1eListArchivedWorkflowExecutions\x12\x46.temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsRequest\x1aG.temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsResponse"g\x82\xd3\xe4\x93\x02\x61\x12*/namespaces/{namespace}/archived-workflowsZ3\x12\x31/api/v1/namespaces/{namespace}/archived-workflows\x12\x9b\x01\n\x16ScanWorkflowExecutions\x12>.temporal.api.workflowservice.v1.ScanWorkflowExecutionsRequest\x1a?.temporal.api.workflowservice.v1.ScanWorkflowExecutionsResponse"\x00\x12\xfd\x01\n\x17\x43ountWorkflowExecutions\x12?.temporal.api.workflowservice.v1.CountWorkflowExecutionsRequest\x1a@.temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse"_\x82\xd3\xe4\x93\x02Y\x12&/namespaces/{namespace}/workflow-countZ/\x12-/api/v1/namespaces/{namespace}/workflow-count\x12\x92\x01\n\x13GetSearchAttributes\x12;.temporal.api.workflowservice.v1.GetSearchAttributesRequest\x1a<.temporal.api.workflowservice.v1.GetSearchAttributesResponse"\x00\x12\xa4\x01\n\x19RespondQueryTaskCompleted\x12\x41.temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondQueryTaskCompletedResponse"\x00\x12\x95\x01\n\x14ResetStickyTaskQueue\x12<.temporal.api.workflowservice.v1.ResetStickyTaskQueueRequest\x1a=.temporal.api.workflowservice.v1.ResetStickyTaskQueueResponse"\x00\x12\x83\x01\n\x0eShutdownWorker\x12\x36.temporal.api.workflowservice.v1.ShutdownWorkerRequest\x1a\x37.temporal.api.workflowservice.v1.ShutdownWorkerResponse"\x00\x12\xbf\x02\n\rQueryWorkflow\x12\x35.temporal.api.workflowservice.v1.QueryWorkflowRequest\x1a\x36.temporal.api.workflowservice.v1.QueryWorkflowResponse"\xbe\x01\x82\xd3\xe4\x93\x02\xb7\x01"R/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}:\x01*Z^"Y/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}:\x01*\x12\xaa\x02\n\x19\x44\x65scribeWorkflowExecution\x12\x41.temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest\x1a\x42.temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse"\x85\x01\x82\xd3\xe4\x93\x02\x7f\x12\x39/namespaces/{namespace}/workflows/{execution.workflow_id}ZB\x12@/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}\x12\x89\x02\n\x11\x44\x65scribeTaskQueue\x12\x39.temporal.api.workflowservice.v1.DescribeTaskQueueRequest\x1a:.temporal.api.workflowservice.v1.DescribeTaskQueueResponse"}\x82\xd3\xe4\x93\x02w\x12\x35/namespaces/{namespace}/task-queues/{task_queue.name}Z>\x12/namespaces/{namespace}/schedules/{schedule_id}/matching-timesZG\x12\x45/api/v1/namespaces/{namespace}/schedules/{schedule_id}/matching-times\x12\xf4\x01\n\x0e\x44\x65leteSchedule\x12\x36.temporal.api.workflowservice.v1.DeleteScheduleRequest\x1a\x37.temporal.api.workflowservice.v1.DeleteScheduleResponse"q\x82\xd3\xe4\x93\x02k*//namespaces/{namespace}/schedules/{schedule_id}Z8*6/api/v1/namespaces/{namespace}/schedules/{schedule_id}\x12\xd5\x01\n\rListSchedules\x12\x35.temporal.api.workflowservice.v1.ListSchedulesRequest\x1a\x36.temporal.api.workflowservice.v1.ListSchedulesResponse"U\x82\xd3\xe4\x93\x02O\x12!/namespaces/{namespace}/schedulesZ*\x12(/api/v1/namespaces/{namespace}/schedules\x12\xb9\x01\n UpdateWorkerBuildIdCompatibility\x12H.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest\x1aI.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityResponse"\x00\x12\xe1\x02\n\x1dGetWorkerBuildIdCompatibility\x12\x45.temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityRequest\x1a\x46.temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityResponse"\xb0\x01\x82\xd3\xe4\x93\x02\xa9\x01\x12N/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibilityZW\x12U/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibility\x12\xaa\x01\n\x1bUpdateWorkerVersioningRules\x12\x43.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest\x1a\x44.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse"\x00\x12\xc6\x02\n\x18GetWorkerVersioningRules\x12@.temporal.api.workflowservice.v1.GetWorkerVersioningRulesRequest\x1a\x41.temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse"\xa4\x01\x82\xd3\xe4\x93\x02\x9d\x01\x12H/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rulesZQ\x12O/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rules\x12\x97\x02\n\x19GetWorkerTaskReachability\x12\x41.temporal.api.workflowservice.v1.GetWorkerTaskReachabilityRequest\x1a\x42.temporal.api.workflowservice.v1.GetWorkerTaskReachabilityResponse"s\x82\xd3\xe4\x93\x02m\x12\x30/namespaces/{namespace}/worker-task-reachabilityZ9\x12\x37/api/v1/namespaces/{namespace}/worker-task-reachability\x12\xc8\x02\n\x12\x44\x65scribeDeployment\x12:.temporal.api.workflowservice.v1.DescribeDeploymentRequest\x1a;.temporal.api.workflowservice.v1.DescribeDeploymentResponse"\xb8\x01\x82\xd3\xe4\x93\x02\xb1\x01\x12R/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}Z[\x12Y/api/v1/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}\x12\xb5\x03\n\x1f\x44\x65scribeWorkerDeploymentVersion\x12G.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionRequest\x1aH.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionResponse"\xfe\x01\x82\xd3\xe4\x93\x02\xf7\x01\x12u/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}Z~\x12|/api/v1/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}\x12\xdf\x01\n\x0fListDeployments\x12\x37.temporal.api.workflowservice.v1.ListDeploymentsRequest\x1a\x38.temporal.api.workflowservice.v1.ListDeploymentsResponse"Y\x82\xd3\xe4\x93\x02S\x12#/namespaces/{namespace}/deploymentsZ,\x12*/api/v1/namespaces/{namespace}/deployments\x12\xf7\x02\n\x19GetDeploymentReachability\x12\x41.temporal.api.workflowservice.v1.GetDeploymentReachabilityRequest\x1a\x42.temporal.api.workflowservice.v1.GetDeploymentReachabilityResponse"\xd2\x01\x82\xd3\xe4\x93\x02\xcb\x01\x12_/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}/reachabilityZh\x12\x66/api/v1/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}/reachability\x12\x99\x02\n\x14GetCurrentDeployment\x12<.temporal.api.workflowservice.v1.GetCurrentDeploymentRequest\x1a=.temporal.api.workflowservice.v1.GetCurrentDeploymentResponse"\x83\x01\x82\xd3\xe4\x93\x02}\x12\x38/namespaces/{namespace}/current-deployment/{series_name}ZA\x12?/api/v1/namespaces/{namespace}/current-deployment/{series_name}\x12\xb6\x02\n\x14SetCurrentDeployment\x12<.temporal.api.workflowservice.v1.SetCurrentDeploymentRequest\x1a=.temporal.api.workflowservice.v1.SetCurrentDeploymentResponse"\xa0\x01\x82\xd3\xe4\x93\x02\x99\x01"C/namespaces/{namespace}/current-deployment/{deployment.series_name}:\x01*ZO"J/api/v1/namespaces/{namespace}/current-deployment/{deployment.series_name}:\x01*\x12\xf7\x02\n!SetWorkerDeploymentCurrentVersion\x12I.temporal.api.workflowservice.v1.SetWorkerDeploymentCurrentVersionRequest\x1aJ.temporal.api.workflowservice.v1.SetWorkerDeploymentCurrentVersionResponse"\xba\x01\x82\xd3\xe4\x93\x02\xb3\x01"P/namespaces/{namespace}/worker-deployments/{deployment_name}/set-current-version:\x01*Z\\"W/api/v1/namespaces/{namespace}/worker-deployments/{deployment_name}/set-current-version:\x01*\x12\xae\x02\n\x18\x44\x65scribeWorkerDeployment\x12@.temporal.api.workflowservice.v1.DescribeWorkerDeploymentRequest\x1a\x41.temporal.api.workflowservice.v1.DescribeWorkerDeploymentResponse"\x8c\x01\x82\xd3\xe4\x93\x02\x85\x01\x12.temporal.api.workflowservice.v1.DeleteWorkerDeploymentRequest\x1a?.temporal.api.workflowservice.v1.DeleteWorkerDeploymentResponse"\x8c\x01\x82\xd3\xe4\x93\x02\x85\x01*.temporal.api.workflowservice.v1.ListWorkerDeploymentsResponse"g\x82\xd3\xe4\x93\x02\x61\x12*/namespaces/{namespace}/worker-deploymentsZ3\x12\x31/api/v1/namespaces/{namespace}/worker-deployments\x12\xf0\x03\n%UpdateWorkerDeploymentVersionMetadata\x12M.temporal.api.workflowservice.v1.UpdateWorkerDeploymentVersionMetadataRequest\x1aN.temporal.api.workflowservice.v1.UpdateWorkerDeploymentVersionMetadataResponse"\xa7\x02\x82\xd3\xe4\x93\x02\xa0\x02"\x85\x01/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}/update-metadata:\x01*Z\x92\x01"\x8c\x01/api/v1/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}/update-metadata:\x01*\x12\xf5\x02\n\x17UpdateWorkflowExecution\x12?.temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest\x1a@.temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse"\xd6\x01\x82\xd3\xe4\x93\x02\xcf\x01"^/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}:\x01*Zj"e/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}:\x01*\x12\xaa\x01\n\x1bPollWorkflowExecutionUpdate\x12\x43.temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest\x1a\x44.temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse"\x00\x12\x8d\x02\n\x13StartBatchOperation\x12;.temporal.api.workflowservice.v1.StartBatchOperationRequest\x1a<.temporal.api.workflowservice.v1.StartBatchOperationResponse"{\x82\xd3\xe4\x93\x02u"1/namespaces/{namespace}/batch-operations/{job_id}:\x01*Z="8/api/v1/namespaces/{namespace}/batch-operations/{job_id}:\x01*\x12\x95\x02\n\x12StopBatchOperation\x12:.temporal.api.workflowservice.v1.StopBatchOperationRequest\x1a;.temporal.api.workflowservice.v1.StopBatchOperationResponse"\x85\x01\x82\xd3\xe4\x93\x02\x7f"6/namespaces/{namespace}/batch-operations/{job_id}/stop:\x01*ZB"=/api/v1/namespaces/{namespace}/batch-operations/{job_id}/stop:\x01*\x12\x90\x02\n\x16\x44\x65scribeBatchOperation\x12>.temporal.api.workflowservice.v1.DescribeBatchOperationRequest\x1a?.temporal.api.workflowservice.v1.DescribeBatchOperationResponse"u\x82\xd3\xe4\x93\x02o\x12\x31/namespaces/{namespace}/batch-operations/{job_id}Z:\x12\x38/api/v1/namespaces/{namespace}/batch-operations/{job_id}\x12\xf5\x01\n\x13ListBatchOperations\x12;.temporal.api.workflowservice.v1.ListBatchOperationsRequest\x1a<.temporal.api.workflowservice.v1.ListBatchOperationsResponse"c\x82\xd3\xe4\x93\x02]\x12(/namespaces/{namespace}/batch-operationsZ1\x12//api/v1/namespaces/{namespace}/batch-operations\x12\x8f\x01\n\x12PollNexusTaskQueue\x12:.temporal.api.workflowservice.v1.PollNexusTaskQueueRequest\x1a;.temporal.api.workflowservice.v1.PollNexusTaskQueueResponse"\x00\x12\xa4\x01\n\x19RespondNexusTaskCompleted\x12\x41.temporal.api.workflowservice.v1.RespondNexusTaskCompletedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondNexusTaskCompletedResponse"\x00\x12\x9b\x01\n\x16RespondNexusTaskFailed\x12>.temporal.api.workflowservice.v1.RespondNexusTaskFailedRequest\x1a?.temporal.api.workflowservice.v1.RespondNexusTaskFailedResponse"\x00\x12\x93\x02\n\x15UpdateActivityOptions\x12=.temporal.api.workflowservice.v1.UpdateActivityOptionsRequest\x1a>.temporal.api.workflowservice.v1.UpdateActivityOptionsResponse"{\x82\xd3\xe4\x93\x02u"1/namespaces/{namespace}/activities/update-options:\x01*Z="8/api/v1/namespaces/{namespace}/activities/update-options:\x01*\x12\xf0\x02\n\x1eUpdateWorkflowExecutionOptions\x12\x46.temporal.api.workflowservice.v1.UpdateWorkflowExecutionOptionsRequest\x1aG.temporal.api.workflowservice.v1.UpdateWorkflowExecutionOptionsResponse"\xbc\x01\x82\xd3\xe4\x93\x02\xb5\x01"Q/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update-options:\x01*Z]"X/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update-options:\x01*\x12\xe9\x01\n\rPauseActivity\x12\x35.temporal.api.workflowservice.v1.PauseActivityRequest\x1a\x36.temporal.api.workflowservice.v1.PauseActivityResponse"i\x82\xd3\xe4\x93\x02\x63"(/namespaces/{namespace}/activities/pause:\x01*Z4"//api/v1/namespaces/{namespace}/activities/pause:\x01*\x12\xf3\x01\n\x0fUnpauseActivity\x12\x37.temporal.api.workflowservice.v1.UnpauseActivityRequest\x1a\x38.temporal.api.workflowservice.v1.UnpauseActivityResponse"m\x82\xd3\xe4\x93\x02g"*/namespaces/{namespace}/activities/unpause:\x01*Z6"1/api/v1/namespaces/{namespace}/activities/unpause:\x01*\x12\xe9\x01\n\rResetActivity\x12\x35.temporal.api.workflowservice.v1.ResetActivityRequest\x1a\x36.temporal.api.workflowservice.v1.ResetActivityResponse"i\x82\xd3\xe4\x93\x02\x63"(/namespaces/{namespace}/activities/reset:\x01*Z4"//api/v1/namespaces/{namespace}/activities/reset:\x01*\x12\xf4\x01\n\x12\x43reateWorkflowRule\x12:.temporal.api.workflowservice.v1.CreateWorkflowRuleRequest\x1a;.temporal.api.workflowservice.v1.CreateWorkflowRuleResponse"e\x82\xd3\xe4\x93\x02_"&/namespaces/{namespace}/workflow-rules:\x01*Z2"-/api/v1/namespaces/{namespace}/workflow-rules:\x01*\x12\x88\x02\n\x14\x44\x65scribeWorkflowRule\x12<.temporal.api.workflowservice.v1.DescribeWorkflowRuleRequest\x1a=.temporal.api.workflowservice.v1.DescribeWorkflowRuleResponse"s\x82\xd3\xe4\x93\x02m\x12\x30/namespaces/{namespace}/workflow-rules/{rule_id}Z9\x12\x37/api/v1/namespaces/{namespace}/workflow-rules/{rule_id}\x12\x82\x02\n\x12\x44\x65leteWorkflowRule\x12:.temporal.api.workflowservice.v1.DeleteWorkflowRuleRequest\x1a;.temporal.api.workflowservice.v1.DeleteWorkflowRuleResponse"s\x82\xd3\xe4\x93\x02m*0/namespaces/{namespace}/workflow-rules/{rule_id}Z9*7/api/v1/namespaces/{namespace}/workflow-rules/{rule_id}\x12\xeb\x01\n\x11ListWorkflowRules\x12\x39.temporal.api.workflowservice.v1.ListWorkflowRulesRequest\x1a:.temporal.api.workflowservice.v1.ListWorkflowRulesResponse"_\x82\xd3\xe4\x93\x02Y\x12&/namespaces/{namespace}/workflow-rulesZ/\x12-/api/v1/namespaces/{namespace}/workflow-rules\x12\xb9\x02\n\x13TriggerWorkflowRule\x12;.temporal.api.workflowservice.v1.TriggerWorkflowRuleRequest\x1a<.temporal.api.workflowservice.v1.TriggerWorkflowRuleResponse"\xa6\x01\x82\xd3\xe4\x93\x02\x9f\x01"F/namespaces/{namespace}/workflows/{execution.workflow_id}/trigger-rule:\x01*ZR"M/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/trigger-rule:\x01*\x12\x83\x02\n\x15RecordWorkerHeartbeat\x12=.temporal.api.workflowservice.v1.RecordWorkerHeartbeatRequest\x1a>.temporal.api.workflowservice.v1.RecordWorkerHeartbeatResponse"k\x82\xd3\xe4\x93\x02\x65")/namespaces/{namespace}/workers/heartbeat:\x01*Z5"0/api/v1/namespaces/{namespace}/workers/heartbeat:\x01*\x12\xcb\x01\n\x0bListWorkers\x12\x33.temporal.api.workflowservice.v1.ListWorkersRequest\x1a\x34.temporal.api.workflowservice.v1.ListWorkersResponse"Q\x82\xd3\xe4\x93\x02K\x12\x1f/namespaces/{namespace}/workersZ(\x12&/api/v1/namespaces/{namespace}/workers\x12\xaf\x02\n\x15UpdateTaskQueueConfig\x12=.temporal.api.workflowservice.v1.UpdateTaskQueueConfigRequest\x1a>.temporal.api.workflowservice.v1.UpdateTaskQueueConfigResponse"\x96\x01\x82\xd3\xe4\x93\x02\x8f\x01">/namespaces/{namespace}/task-queues/{task_queue}/update-config:\x01*ZJ"E/api/v1/namespaces/{namespace}/task-queues/{task_queue}/update-config:\x01*\x12\xfd\x01\n\x11\x46\x65tchWorkerConfig\x12\x39.temporal.api.workflowservice.v1.FetchWorkerConfigRequest\x1a:.temporal.api.workflowservice.v1.FetchWorkerConfigResponse"q\x82\xd3\xe4\x93\x02k",/namespaces/{namespace}/workers/fetch-config:\x01*Z8"3/api/v1/namespaces/{namespace}/workers/fetch-config:\x01*\x12\x82\x02\n\x12UpdateWorkerConfig\x12:.temporal.api.workflowservice.v1.UpdateWorkerConfigRequest\x1a;.temporal.api.workflowservice.v1.UpdateWorkerConfigResponse"s\x82\xd3\xe4\x93\x02m"-/namespaces/{namespace}/workers/update-config:\x01*Z9"4/api/v1/namespaces/{namespace}/workers/update-config:\x01*B\xb6\x01\n"io.temporal.api.workflowservice.v1B\x0cServiceProtoP\x01Z5go.temporal.io/api/workflowservice/v1;workflowservice\xaa\x02!Temporalio.Api.WorkflowService.V1\xea\x02$Temporalio::Api::WorkflowService::V1b\x06proto3' + b'\n-temporal/api/workflowservice/v1/service.proto\x12\x1ftemporal.api.workflowservice.v1\x1a\x36temporal/api/workflowservice/v1/request_response.proto\x1a\x1cgoogle/api/annotations.proto2\xa8\xe1\x01\n\x0fWorkflowService\x12\xc3\x01\n\x11RegisterNamespace\x12\x39.temporal.api.workflowservice.v1.RegisterNamespaceRequest\x1a:.temporal.api.workflowservice.v1.RegisterNamespaceResponse"7\x82\xd3\xe4\x93\x02\x31"\x13/cluster/namespaces:\x01*Z\x17"\x12/api/v1/namespaces:\x01*\x12\xd5\x01\n\x11\x44\x65scribeNamespace\x12\x39.temporal.api.workflowservice.v1.DescribeNamespaceRequest\x1a:.temporal.api.workflowservice.v1.DescribeNamespaceResponse"I\x82\xd3\xe4\x93\x02\x43\x12\x1f/cluster/namespaces/{namespace}Z \x12\x1e/api/v1/namespaces/{namespace}\x12\xb4\x01\n\x0eListNamespaces\x12\x36.temporal.api.workflowservice.v1.ListNamespacesRequest\x1a\x37.temporal.api.workflowservice.v1.ListNamespacesResponse"1\x82\xd3\xe4\x93\x02+\x12\x13/cluster/namespacesZ\x14\x12\x12/api/v1/namespaces\x12\xe3\x01\n\x0fUpdateNamespace\x12\x37.temporal.api.workflowservice.v1.UpdateNamespaceRequest\x1a\x38.temporal.api.workflowservice.v1.UpdateNamespaceResponse"]\x82\xd3\xe4\x93\x02W"&/cluster/namespaces/{namespace}/update:\x01*Z*"%/api/v1/namespaces/{namespace}/update:\x01*\x12\x8f\x01\n\x12\x44\x65precateNamespace\x12:.temporal.api.workflowservice.v1.DeprecateNamespaceRequest\x1a;.temporal.api.workflowservice.v1.DeprecateNamespaceResponse"\x00\x12\x92\x02\n\x16StartWorkflowExecution\x12>.temporal.api.workflowservice.v1.StartWorkflowExecutionRequest\x1a?.temporal.api.workflowservice.v1.StartWorkflowExecutionResponse"w\x82\xd3\xe4\x93\x02q"//namespaces/{namespace}/workflows/{workflow_id}:\x01*Z;"6/api/v1/namespaces/{namespace}/workflows/{workflow_id}:\x01*\x12\xa5\x02\n\x15\x45xecuteMultiOperation\x12=.temporal.api.workflowservice.v1.ExecuteMultiOperationRequest\x1a>.temporal.api.workflowservice.v1.ExecuteMultiOperationResponse"\x8c\x01\x82\xd3\xe4\x93\x02\x85\x01"9/namespaces/{namespace}/workflows/execute-multi-operation:\x01*ZE"@/api/v1/namespaces/{namespace}/workflows/execute-multi-operation:\x01*\x12\xc1\x02\n\x1bGetWorkflowExecutionHistory\x12\x43.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest\x1a\x44.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse"\x96\x01\x82\xd3\xe4\x93\x02\x8f\x01\x12\x41/namespaces/{namespace}/workflows/{execution.workflow_id}/historyZJ\x12H/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history\x12\xe6\x02\n"GetWorkflowExecutionHistoryReverse\x12J.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseRequest\x1aK.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseResponse"\xa6\x01\x82\xd3\xe4\x93\x02\x9f\x01\x12I/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverseZR\x12P/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverse\x12\x98\x01\n\x15PollWorkflowTaskQueue\x12=.temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest\x1a>.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse"\x00\x12\xad\x01\n\x1cRespondWorkflowTaskCompleted\x12\x44.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest\x1a\x45.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse"\x00\x12\xa4\x01\n\x19RespondWorkflowTaskFailed\x12\x41.temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondWorkflowTaskFailedResponse"\x00\x12\x98\x01\n\x15PollActivityTaskQueue\x12=.temporal.api.workflowservice.v1.PollActivityTaskQueueRequest\x1a>.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse"\x00\x12\x9b\x02\n\x1bRecordActivityTaskHeartbeat\x12\x43.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatRequest\x1a\x44.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatResponse"q\x82\xd3\xe4\x93\x02k",/namespaces/{namespace}/activities/heartbeat:\x01*Z8"3/api/v1/namespaces/{namespace}/activities/heartbeat:\x01*\x12\xb3\x02\n\x1fRecordActivityTaskHeartbeatById\x12G.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest\x1aH.temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdResponse"}\x82\xd3\xe4\x93\x02w"2/namespaces/{namespace}/activities/heartbeat-by-id:\x01*Z>"9/api/v1/namespaces/{namespace}/activities/heartbeat-by-id:\x01*\x12\x9c\x02\n\x1cRespondActivityTaskCompleted\x12\x44.temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest\x1a\x45.temporal.api.workflowservice.v1.RespondActivityTaskCompletedResponse"o\x82\xd3\xe4\x93\x02i"+/namespaces/{namespace}/activities/complete:\x01*Z7"2/api/v1/namespaces/{namespace}/activities/complete:\x01*\x12\xb4\x02\n RespondActivityTaskCompletedById\x12H.temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest\x1aI.temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdResponse"{\x82\xd3\xe4\x93\x02u"1/namespaces/{namespace}/activities/complete-by-id:\x01*Z="8/api/v1/namespaces/{namespace}/activities/complete-by-id:\x01*\x12\x8b\x02\n\x19RespondActivityTaskFailed\x12\x41.temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondActivityTaskFailedResponse"g\x82\xd3\xe4\x93\x02\x61"\'/namespaces/{namespace}/activities/fail:\x01*Z3"./api/v1/namespaces/{namespace}/activities/fail:\x01*\x12\xa3\x02\n\x1dRespondActivityTaskFailedById\x12\x45.temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest\x1a\x46.temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdResponse"s\x82\xd3\xe4\x93\x02m"-/namespaces/{namespace}/activities/fail-by-id:\x01*Z9"4/api/v1/namespaces/{namespace}/activities/fail-by-id:\x01*\x12\x95\x02\n\x1bRespondActivityTaskCanceled\x12\x43.temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest\x1a\x44.temporal.api.workflowservice.v1.RespondActivityTaskCanceledResponse"k\x82\xd3\xe4\x93\x02\x65")/namespaces/{namespace}/activities/cancel:\x01*Z5"0/api/v1/namespaces/{namespace}/activities/cancel:\x01*\x12\xad\x02\n\x1fRespondActivityTaskCanceledById\x12G.temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest\x1aH.temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdResponse"w\x82\xd3\xe4\x93\x02q"//namespaces/{namespace}/activities/cancel-by-id:\x01*Z;"6/api/v1/namespaces/{namespace}/activities/cancel-by-id:\x01*\x12\xe0\x02\n\x1eRequestCancelWorkflowExecution\x12\x46.temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest\x1aG.temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionResponse"\xac\x01\x82\xd3\xe4\x93\x02\xa5\x01"I/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel:\x01*ZU"P/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel:\x01*\x12\xe7\x02\n\x17SignalWorkflowExecution\x12?.temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest\x1a@.temporal.api.workflowservice.v1.SignalWorkflowExecutionResponse"\xc8\x01\x82\xd3\xe4\x93\x02\xc1\x01"W/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}:\x01*Zc"^/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}:\x01*\x12\xf2\x02\n SignalWithStartWorkflowExecution\x12H.temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest\x1aI.temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionResponse"\xb8\x01\x82\xd3\xe4\x93\x02\xb1\x01"O/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}:\x01*Z["V/api/v1/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}:\x01*\x12\xc6\x02\n\x16ResetWorkflowExecution\x12>.temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest\x1a?.temporal.api.workflowservice.v1.ResetWorkflowExecutionResponse"\xaa\x01\x82\xd3\xe4\x93\x02\xa3\x01"H/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset:\x01*ZT"O/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset:\x01*\x12\xda\x02\n\x1aTerminateWorkflowExecution\x12\x42.temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest\x1a\x43.temporal.api.workflowservice.v1.TerminateWorkflowExecutionResponse"\xb2\x01\x82\xd3\xe4\x93\x02\xab\x01"L/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate:\x01*ZX"S/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate:\x01*\x12\x9e\x01\n\x17\x44\x65leteWorkflowExecution\x12?.temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest\x1a@.temporal.api.workflowservice.v1.DeleteWorkflowExecutionResponse"\x00\x12\xa7\x01\n\x1aListOpenWorkflowExecutions\x12\x42.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest\x1a\x43.temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse"\x00\x12\xad\x01\n\x1cListClosedWorkflowExecutions\x12\x44.temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest\x1a\x45.temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse"\x00\x12\xf0\x01\n\x16ListWorkflowExecutions\x12>.temporal.api.workflowservice.v1.ListWorkflowExecutionsRequest\x1a?.temporal.api.workflowservice.v1.ListWorkflowExecutionsResponse"U\x82\xd3\xe4\x93\x02O\x12!/namespaces/{namespace}/workflowsZ*\x12(/api/v1/namespaces/{namespace}/workflows\x12\x9a\x02\n\x1eListArchivedWorkflowExecutions\x12\x46.temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsRequest\x1aG.temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsResponse"g\x82\xd3\xe4\x93\x02\x61\x12*/namespaces/{namespace}/archived-workflowsZ3\x12\x31/api/v1/namespaces/{namespace}/archived-workflows\x12\x9b\x01\n\x16ScanWorkflowExecutions\x12>.temporal.api.workflowservice.v1.ScanWorkflowExecutionsRequest\x1a?.temporal.api.workflowservice.v1.ScanWorkflowExecutionsResponse"\x00\x12\xfd\x01\n\x17\x43ountWorkflowExecutions\x12?.temporal.api.workflowservice.v1.CountWorkflowExecutionsRequest\x1a@.temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse"_\x82\xd3\xe4\x93\x02Y\x12&/namespaces/{namespace}/workflow-countZ/\x12-/api/v1/namespaces/{namespace}/workflow-count\x12\x92\x01\n\x13GetSearchAttributes\x12;.temporal.api.workflowservice.v1.GetSearchAttributesRequest\x1a<.temporal.api.workflowservice.v1.GetSearchAttributesResponse"\x00\x12\xa4\x01\n\x19RespondQueryTaskCompleted\x12\x41.temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondQueryTaskCompletedResponse"\x00\x12\x95\x01\n\x14ResetStickyTaskQueue\x12<.temporal.api.workflowservice.v1.ResetStickyTaskQueueRequest\x1a=.temporal.api.workflowservice.v1.ResetStickyTaskQueueResponse"\x00\x12\x83\x01\n\x0eShutdownWorker\x12\x36.temporal.api.workflowservice.v1.ShutdownWorkerRequest\x1a\x37.temporal.api.workflowservice.v1.ShutdownWorkerResponse"\x00\x12\xbf\x02\n\rQueryWorkflow\x12\x35.temporal.api.workflowservice.v1.QueryWorkflowRequest\x1a\x36.temporal.api.workflowservice.v1.QueryWorkflowResponse"\xbe\x01\x82\xd3\xe4\x93\x02\xb7\x01"R/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}:\x01*Z^"Y/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}:\x01*\x12\xaa\x02\n\x19\x44\x65scribeWorkflowExecution\x12\x41.temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest\x1a\x42.temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse"\x85\x01\x82\xd3\xe4\x93\x02\x7f\x12\x39/namespaces/{namespace}/workflows/{execution.workflow_id}ZB\x12@/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}\x12\x89\x02\n\x11\x44\x65scribeTaskQueue\x12\x39.temporal.api.workflowservice.v1.DescribeTaskQueueRequest\x1a:.temporal.api.workflowservice.v1.DescribeTaskQueueResponse"}\x82\xd3\xe4\x93\x02w\x12\x35/namespaces/{namespace}/task-queues/{task_queue.name}Z>\x12/namespaces/{namespace}/schedules/{schedule_id}/matching-timesZG\x12\x45/api/v1/namespaces/{namespace}/schedules/{schedule_id}/matching-times\x12\xf4\x01\n\x0e\x44\x65leteSchedule\x12\x36.temporal.api.workflowservice.v1.DeleteScheduleRequest\x1a\x37.temporal.api.workflowservice.v1.DeleteScheduleResponse"q\x82\xd3\xe4\x93\x02k*//namespaces/{namespace}/schedules/{schedule_id}Z8*6/api/v1/namespaces/{namespace}/schedules/{schedule_id}\x12\xd5\x01\n\rListSchedules\x12\x35.temporal.api.workflowservice.v1.ListSchedulesRequest\x1a\x36.temporal.api.workflowservice.v1.ListSchedulesResponse"U\x82\xd3\xe4\x93\x02O\x12!/namespaces/{namespace}/schedulesZ*\x12(/api/v1/namespaces/{namespace}/schedules\x12\xb9\x01\n UpdateWorkerBuildIdCompatibility\x12H.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest\x1aI.temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityResponse"\x00\x12\xe1\x02\n\x1dGetWorkerBuildIdCompatibility\x12\x45.temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityRequest\x1a\x46.temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityResponse"\xb0\x01\x82\xd3\xe4\x93\x02\xa9\x01\x12N/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibilityZW\x12U/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibility\x12\xaa\x01\n\x1bUpdateWorkerVersioningRules\x12\x43.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest\x1a\x44.temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse"\x00\x12\xc6\x02\n\x18GetWorkerVersioningRules\x12@.temporal.api.workflowservice.v1.GetWorkerVersioningRulesRequest\x1a\x41.temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse"\xa4\x01\x82\xd3\xe4\x93\x02\x9d\x01\x12H/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rulesZQ\x12O/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rules\x12\x97\x02\n\x19GetWorkerTaskReachability\x12\x41.temporal.api.workflowservice.v1.GetWorkerTaskReachabilityRequest\x1a\x42.temporal.api.workflowservice.v1.GetWorkerTaskReachabilityResponse"s\x82\xd3\xe4\x93\x02m\x12\x30/namespaces/{namespace}/worker-task-reachabilityZ9\x12\x37/api/v1/namespaces/{namespace}/worker-task-reachability\x12\xc8\x02\n\x12\x44\x65scribeDeployment\x12:.temporal.api.workflowservice.v1.DescribeDeploymentRequest\x1a;.temporal.api.workflowservice.v1.DescribeDeploymentResponse"\xb8\x01\x82\xd3\xe4\x93\x02\xb1\x01\x12R/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}Z[\x12Y/api/v1/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}\x12\xb5\x03\n\x1f\x44\x65scribeWorkerDeploymentVersion\x12G.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionRequest\x1aH.temporal.api.workflowservice.v1.DescribeWorkerDeploymentVersionResponse"\xfe\x01\x82\xd3\xe4\x93\x02\xf7\x01\x12u/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}Z~\x12|/api/v1/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}\x12\xdf\x01\n\x0fListDeployments\x12\x37.temporal.api.workflowservice.v1.ListDeploymentsRequest\x1a\x38.temporal.api.workflowservice.v1.ListDeploymentsResponse"Y\x82\xd3\xe4\x93\x02S\x12#/namespaces/{namespace}/deploymentsZ,\x12*/api/v1/namespaces/{namespace}/deployments\x12\xf7\x02\n\x19GetDeploymentReachability\x12\x41.temporal.api.workflowservice.v1.GetDeploymentReachabilityRequest\x1a\x42.temporal.api.workflowservice.v1.GetDeploymentReachabilityResponse"\xd2\x01\x82\xd3\xe4\x93\x02\xcb\x01\x12_/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}/reachabilityZh\x12\x66/api/v1/namespaces/{namespace}/deployments/{deployment.series_name}/{deployment.build_id}/reachability\x12\x99\x02\n\x14GetCurrentDeployment\x12<.temporal.api.workflowservice.v1.GetCurrentDeploymentRequest\x1a=.temporal.api.workflowservice.v1.GetCurrentDeploymentResponse"\x83\x01\x82\xd3\xe4\x93\x02}\x12\x38/namespaces/{namespace}/current-deployment/{series_name}ZA\x12?/api/v1/namespaces/{namespace}/current-deployment/{series_name}\x12\xb6\x02\n\x14SetCurrentDeployment\x12<.temporal.api.workflowservice.v1.SetCurrentDeploymentRequest\x1a=.temporal.api.workflowservice.v1.SetCurrentDeploymentResponse"\xa0\x01\x82\xd3\xe4\x93\x02\x99\x01"C/namespaces/{namespace}/current-deployment/{deployment.series_name}:\x01*ZO"J/api/v1/namespaces/{namespace}/current-deployment/{deployment.series_name}:\x01*\x12\xf7\x02\n!SetWorkerDeploymentCurrentVersion\x12I.temporal.api.workflowservice.v1.SetWorkerDeploymentCurrentVersionRequest\x1aJ.temporal.api.workflowservice.v1.SetWorkerDeploymentCurrentVersionResponse"\xba\x01\x82\xd3\xe4\x93\x02\xb3\x01"P/namespaces/{namespace}/worker-deployments/{deployment_name}/set-current-version:\x01*Z\\"W/api/v1/namespaces/{namespace}/worker-deployments/{deployment_name}/set-current-version:\x01*\x12\xae\x02\n\x18\x44\x65scribeWorkerDeployment\x12@.temporal.api.workflowservice.v1.DescribeWorkerDeploymentRequest\x1a\x41.temporal.api.workflowservice.v1.DescribeWorkerDeploymentResponse"\x8c\x01\x82\xd3\xe4\x93\x02\x85\x01\x12.temporal.api.workflowservice.v1.DeleteWorkerDeploymentRequest\x1a?.temporal.api.workflowservice.v1.DeleteWorkerDeploymentResponse"\x8c\x01\x82\xd3\xe4\x93\x02\x85\x01*.temporal.api.workflowservice.v1.ListWorkerDeploymentsResponse"g\x82\xd3\xe4\x93\x02\x61\x12*/namespaces/{namespace}/worker-deploymentsZ3\x12\x31/api/v1/namespaces/{namespace}/worker-deployments\x12\xf0\x03\n%UpdateWorkerDeploymentVersionMetadata\x12M.temporal.api.workflowservice.v1.UpdateWorkerDeploymentVersionMetadataRequest\x1aN.temporal.api.workflowservice.v1.UpdateWorkerDeploymentVersionMetadataResponse"\xa7\x02\x82\xd3\xe4\x93\x02\xa0\x02"\x85\x01/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}/update-metadata:\x01*Z\x92\x01"\x8c\x01/api/v1/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}/update-metadata:\x01*\x12\xd2\x02\n\x1aSetWorkerDeploymentManager\x12\x42.temporal.api.workflowservice.v1.SetWorkerDeploymentManagerRequest\x1a\x43.temporal.api.workflowservice.v1.SetWorkerDeploymentManagerResponse"\xaa\x01\x82\xd3\xe4\x93\x02\xa3\x01"H/namespaces/{namespace}/worker-deployments/{deployment_name}/set-manager:\x01*ZT"O/api/v1/namespaces/{namespace}/worker-deployments/{deployment_name}/set-manager:\x01*\x12\xf5\x02\n\x17UpdateWorkflowExecution\x12?.temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest\x1a@.temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse"\xd6\x01\x82\xd3\xe4\x93\x02\xcf\x01"^/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}:\x01*Zj"e/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}:\x01*\x12\xaa\x01\n\x1bPollWorkflowExecutionUpdate\x12\x43.temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest\x1a\x44.temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse"\x00\x12\x8d\x02\n\x13StartBatchOperation\x12;.temporal.api.workflowservice.v1.StartBatchOperationRequest\x1a<.temporal.api.workflowservice.v1.StartBatchOperationResponse"{\x82\xd3\xe4\x93\x02u"1/namespaces/{namespace}/batch-operations/{job_id}:\x01*Z="8/api/v1/namespaces/{namespace}/batch-operations/{job_id}:\x01*\x12\x95\x02\n\x12StopBatchOperation\x12:.temporal.api.workflowservice.v1.StopBatchOperationRequest\x1a;.temporal.api.workflowservice.v1.StopBatchOperationResponse"\x85\x01\x82\xd3\xe4\x93\x02\x7f"6/namespaces/{namespace}/batch-operations/{job_id}/stop:\x01*ZB"=/api/v1/namespaces/{namespace}/batch-operations/{job_id}/stop:\x01*\x12\x90\x02\n\x16\x44\x65scribeBatchOperation\x12>.temporal.api.workflowservice.v1.DescribeBatchOperationRequest\x1a?.temporal.api.workflowservice.v1.DescribeBatchOperationResponse"u\x82\xd3\xe4\x93\x02o\x12\x31/namespaces/{namespace}/batch-operations/{job_id}Z:\x12\x38/api/v1/namespaces/{namespace}/batch-operations/{job_id}\x12\xf5\x01\n\x13ListBatchOperations\x12;.temporal.api.workflowservice.v1.ListBatchOperationsRequest\x1a<.temporal.api.workflowservice.v1.ListBatchOperationsResponse"c\x82\xd3\xe4\x93\x02]\x12(/namespaces/{namespace}/batch-operationsZ1\x12//api/v1/namespaces/{namespace}/batch-operations\x12\x8f\x01\n\x12PollNexusTaskQueue\x12:.temporal.api.workflowservice.v1.PollNexusTaskQueueRequest\x1a;.temporal.api.workflowservice.v1.PollNexusTaskQueueResponse"\x00\x12\xa4\x01\n\x19RespondNexusTaskCompleted\x12\x41.temporal.api.workflowservice.v1.RespondNexusTaskCompletedRequest\x1a\x42.temporal.api.workflowservice.v1.RespondNexusTaskCompletedResponse"\x00\x12\x9b\x01\n\x16RespondNexusTaskFailed\x12>.temporal.api.workflowservice.v1.RespondNexusTaskFailedRequest\x1a?.temporal.api.workflowservice.v1.RespondNexusTaskFailedResponse"\x00\x12\xf7\x03\n\x1eUpdateActivityExecutionOptions\x12\x46.temporal.api.workflowservice.v1.UpdateActivityExecutionOptionsRequest\x1aG.temporal.api.workflowservice.v1.UpdateActivityExecutionOptionsResponse"\xc3\x02\x82\xd3\xe4\x93\x02\xbc\x02"3/namespaces/activities/{activity_id}/update-options:\x01*Z?":/api/v1/namespaces/activities/{activity_id}/update-options:\x01*Z\\"W/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/update-options:\x01*Zc"^/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/update-options:\x01*\x12\x93\x02\n\x15UpdateActivityOptions\x12=.temporal.api.workflowservice.v1.UpdateActivityOptionsRequest\x1a>.temporal.api.workflowservice.v1.UpdateActivityOptionsResponse"{\x82\xd3\xe4\x93\x02u"1/namespaces/{namespace}/activities/update-options:\x01*Z="8/api/v1/namespaces/{namespace}/activities/update-options:\x01*\x12\xf0\x02\n\x1eUpdateWorkflowExecutionOptions\x12\x46.temporal.api.workflowservice.v1.UpdateWorkflowExecutionOptionsRequest\x1aG.temporal.api.workflowservice.v1.UpdateWorkflowExecutionOptionsResponse"\xbc\x01\x82\xd3\xe4\x93\x02\xb5\x01"Q/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update-options:\x01*Z]"X/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update-options:\x01*\x12\xbb\x03\n\x16PauseActivityExecution\x12>.temporal.api.workflowservice.v1.PauseActivityExecutionRequest\x1a?.temporal.api.workflowservice.v1.PauseActivityExecutionResponse"\x9f\x02\x82\xd3\xe4\x93\x02\x98\x02"*/namespaces/activities/{activity_id}/pause:\x01*Z6"1/api/v1/namespaces/activities/{activity_id}/pause:\x01*ZS"N/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/pause:\x01*ZZ"U/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/pause:\x01*\x12\xe9\x01\n\rPauseActivity\x12\x35.temporal.api.workflowservice.v1.PauseActivityRequest\x1a\x36.temporal.api.workflowservice.v1.PauseActivityResponse"i\x82\xd3\xe4\x93\x02\x63"(/namespaces/{namespace}/activities/pause:\x01*Z4"//api/v1/namespaces/{namespace}/activities/pause:\x01*\x12\xc9\x03\n\x18UnpauseActivityExecution\x12@.temporal.api.workflowservice.v1.UnpauseActivityExecutionRequest\x1a\x41.temporal.api.workflowservice.v1.UnpauseActivityExecutionResponse"\xa7\x02\x82\xd3\xe4\x93\x02\xa0\x02",/namespaces/activities/{activity_id}/unpause:\x01*Z8"3/api/v1/namespaces/activities/{activity_id}/unpause:\x01*ZU"P/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/unpause:\x01*Z\\"W/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/unpause:\x01*\x12\xf3\x01\n\x0fUnpauseActivity\x12\x37.temporal.api.workflowservice.v1.UnpauseActivityRequest\x1a\x38.temporal.api.workflowservice.v1.UnpauseActivityResponse"m\x82\xd3\xe4\x93\x02g"*/namespaces/{namespace}/activities/unpause:\x01*Z6"1/api/v1/namespaces/{namespace}/activities/unpause:\x01*\x12\xbb\x03\n\x16ResetActivityExecution\x12>.temporal.api.workflowservice.v1.ResetActivityExecutionRequest\x1a?.temporal.api.workflowservice.v1.ResetActivityExecutionResponse"\x9f\x02\x82\xd3\xe4\x93\x02\x98\x02"*/namespaces/activities/{activity_id}/reset:\x01*Z6"1/api/v1/namespaces/activities/{activity_id}/reset:\x01*ZS"N/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/reset:\x01*ZZ"U/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/reset:\x01*\x12\xe9\x01\n\rResetActivity\x12\x35.temporal.api.workflowservice.v1.ResetActivityRequest\x1a\x36.temporal.api.workflowservice.v1.ResetActivityResponse"i\x82\xd3\xe4\x93\x02\x63"(/namespaces/{namespace}/activities/reset:\x01*Z4"//api/v1/namespaces/{namespace}/activities/reset:\x01*\x12\xf4\x01\n\x12\x43reateWorkflowRule\x12:.temporal.api.workflowservice.v1.CreateWorkflowRuleRequest\x1a;.temporal.api.workflowservice.v1.CreateWorkflowRuleResponse"e\x82\xd3\xe4\x93\x02_"&/namespaces/{namespace}/workflow-rules:\x01*Z2"-/api/v1/namespaces/{namespace}/workflow-rules:\x01*\x12\x88\x02\n\x14\x44\x65scribeWorkflowRule\x12<.temporal.api.workflowservice.v1.DescribeWorkflowRuleRequest\x1a=.temporal.api.workflowservice.v1.DescribeWorkflowRuleResponse"s\x82\xd3\xe4\x93\x02m\x12\x30/namespaces/{namespace}/workflow-rules/{rule_id}Z9\x12\x37/api/v1/namespaces/{namespace}/workflow-rules/{rule_id}\x12\x82\x02\n\x12\x44\x65leteWorkflowRule\x12:.temporal.api.workflowservice.v1.DeleteWorkflowRuleRequest\x1a;.temporal.api.workflowservice.v1.DeleteWorkflowRuleResponse"s\x82\xd3\xe4\x93\x02m*0/namespaces/{namespace}/workflow-rules/{rule_id}Z9*7/api/v1/namespaces/{namespace}/workflow-rules/{rule_id}\x12\xeb\x01\n\x11ListWorkflowRules\x12\x39.temporal.api.workflowservice.v1.ListWorkflowRulesRequest\x1a:.temporal.api.workflowservice.v1.ListWorkflowRulesResponse"_\x82\xd3\xe4\x93\x02Y\x12&/namespaces/{namespace}/workflow-rulesZ/\x12-/api/v1/namespaces/{namespace}/workflow-rules\x12\xb9\x02\n\x13TriggerWorkflowRule\x12;.temporal.api.workflowservice.v1.TriggerWorkflowRuleRequest\x1a<.temporal.api.workflowservice.v1.TriggerWorkflowRuleResponse"\xa6\x01\x82\xd3\xe4\x93\x02\x9f\x01"F/namespaces/{namespace}/workflows/{execution.workflow_id}/trigger-rule:\x01*ZR"M/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/trigger-rule:\x01*\x12\x83\x02\n\x15RecordWorkerHeartbeat\x12=.temporal.api.workflowservice.v1.RecordWorkerHeartbeatRequest\x1a>.temporal.api.workflowservice.v1.RecordWorkerHeartbeatResponse"k\x82\xd3\xe4\x93\x02\x65")/namespaces/{namespace}/workers/heartbeat:\x01*Z5"0/api/v1/namespaces/{namespace}/workers/heartbeat:\x01*\x12\xcb\x01\n\x0bListWorkers\x12\x33.temporal.api.workflowservice.v1.ListWorkersRequest\x1a\x34.temporal.api.workflowservice.v1.ListWorkersResponse"Q\x82\xd3\xe4\x93\x02K\x12\x1f/namespaces/{namespace}/workersZ(\x12&/api/v1/namespaces/{namespace}/workers\x12\xaf\x02\n\x15UpdateTaskQueueConfig\x12=.temporal.api.workflowservice.v1.UpdateTaskQueueConfigRequest\x1a>.temporal.api.workflowservice.v1.UpdateTaskQueueConfigResponse"\x96\x01\x82\xd3\xe4\x93\x02\x8f\x01">/namespaces/{namespace}/task-queues/{task_queue}/update-config:\x01*ZJ"E/api/v1/namespaces/{namespace}/task-queues/{task_queue}/update-config:\x01*\x12\xfd\x01\n\x11\x46\x65tchWorkerConfig\x12\x39.temporal.api.workflowservice.v1.FetchWorkerConfigRequest\x1a:.temporal.api.workflowservice.v1.FetchWorkerConfigResponse"q\x82\xd3\xe4\x93\x02k",/namespaces/{namespace}/workers/fetch-config:\x01*Z8"3/api/v1/namespaces/{namespace}/workers/fetch-config:\x01*\x12\x82\x02\n\x12UpdateWorkerConfig\x12:.temporal.api.workflowservice.v1.UpdateWorkerConfigRequest\x1a;.temporal.api.workflowservice.v1.UpdateWorkerConfigResponse"s\x82\xd3\xe4\x93\x02m"-/namespaces/{namespace}/workers/update-config:\x01*Z9"4/api/v1/namespaces/{namespace}/workers/update-config:\x01*\x12\x94\x02\n\x0e\x44\x65scribeWorker\x12\x36.temporal.api.workflowservice.v1.DescribeWorkerRequest\x1a\x37.temporal.api.workflowservice.v1.DescribeWorkerResponse"\x90\x01\x82\xd3\xe4\x93\x02\x89\x01\x12>/namespaces/{namespace}/workers/describe/{worker_instance_key}ZG\x12\x45/api/v1/namespaces/{namespace}/workers/describe/{worker_instance_key}\x12\x94\x02\n\x16StartActivityExecution\x12>.temporal.api.workflowservice.v1.StartActivityExecutionRequest\x1a?.temporal.api.workflowservice.v1.StartActivityExecutionResponse"y\x82\xd3\xe4\x93\x02s"0/namespaces/{namespace}/activities/{activity_id}:\x01*Z<"7/api/v1/namespaces/{namespace}/activities/{activity_id}:\x01*\x12\x97\x02\n\x19\x44\x65scribeActivityExecution\x12\x41.temporal.api.workflowservice.v1.DescribeActivityExecutionRequest\x1a\x42.temporal.api.workflowservice.v1.DescribeActivityExecutionResponse"s\x82\xd3\xe4\x93\x02m\x12\x30/namespaces/{namespace}/activities/{activity_id}Z9\x12\x37/api/v1/namespaces/{namespace}/activities/{activity_id}\x12\xf2\x01\n\x16ListActivityExecutions\x12>.temporal.api.workflowservice.v1.ListActivityExecutionsRequest\x1a?.temporal.api.workflowservice.v1.ListActivityExecutionsResponse"W\x82\xd3\xe4\x93\x02Q\x12"/namespaces/{namespace}/activitiesZ+\x12)/api/v1/namespaces/{namespace}/activities\x12\xfd\x01\n\x17\x43ountActivityExecutions\x12?.temporal.api.workflowservice.v1.CountActivityExecutionsRequest\x1a@.temporal.api.workflowservice.v1.CountActivityExecutionsResponse"_\x82\xd3\xe4\x93\x02Y\x12&/namespaces/{namespace}/activity-countZ/\x12-/api/v1/namespaces/{namespace}/activity-count\x12\xa9\x02\n\x1aGetActivityExecutionResult\x12\x42.temporal.api.workflowservice.v1.GetActivityExecutionResultRequest\x1a\x43.temporal.api.workflowservice.v1.GetActivityExecutionResultResponse"\x81\x01\x82\xd3\xe4\x93\x02{\x12\x37/namespaces/{namespace}/activities/{activity_id}/resultZ@\x12>/api/v1/namespaces/{namespace}/activities/{activity_id}/result\x12\xbc\x02\n\x1eRequestCancelActivityExecution\x12\x46.temporal.api.workflowservice.v1.RequestCancelActivityExecutionRequest\x1aG.temporal.api.workflowservice.v1.RequestCancelActivityExecutionResponse"\x88\x01\x82\xd3\xe4\x93\x02\x81\x01"7/namespaces/{namespace}/activities/{activity_id}/cancel:\x01*ZC">/api/v1/namespaces/{namespace}/activities/{activity_id}/cancel:\x01*\x12\xb6\x02\n\x1aTerminateActivityExecution\x12\x42.temporal.api.workflowservice.v1.TerminateActivityExecutionRequest\x1a\x43.temporal.api.workflowservice.v1.TerminateActivityExecutionResponse"\x8e\x01\x82\xd3\xe4\x93\x02\x87\x01":/namespaces/{namespace}/activities/{activity_id}/terminate:\x01*ZF"A/api/v1/namespaces/{namespace}/activities/{activity_id}/terminate:\x01*\x12\x9e\x01\n\x17\x44\x65leteActivityExecution\x12?.temporal.api.workflowservice.v1.DeleteActivityExecutionRequest\x1a@.temporal.api.workflowservice.v1.DeleteActivityExecutionResponse"\x00\x42\xb6\x01\n"io.temporal.api.workflowservice.v1B\x0cServiceProtoP\x01Z5go.temporal.io/api/workflowservice/v1;workflowservice\xaa\x02!Temporalio.Api.WorkflowService.V1\xea\x02$Temporalio::Api::WorkflowService::V1b\x06proto3' ) @@ -249,6 +249,10 @@ _WORKFLOWSERVICE.methods_by_name[ "UpdateWorkerDeploymentVersionMetadata" ]._serialized_options = b'\202\323\344\223\002\240\002"\205\001/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}/update-metadata:\001*Z\222\001"\214\001/api/v1/namespaces/{namespace}/worker-deployment-versions/{deployment_version.deployment_name}/{deployment_version.build_id}/update-metadata:\001*' + _WORKFLOWSERVICE.methods_by_name["SetWorkerDeploymentManager"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "SetWorkerDeploymentManager" + ]._serialized_options = b'\202\323\344\223\002\243\001"H/namespaces/{namespace}/worker-deployments/{deployment_name}/set-manager:\001*ZT"O/api/v1/namespaces/{namespace}/worker-deployments/{deployment_name}/set-manager:\001*' _WORKFLOWSERVICE.methods_by_name["UpdateWorkflowExecution"]._options = None _WORKFLOWSERVICE.methods_by_name[ "UpdateWorkflowExecution" @@ -269,6 +273,10 @@ _WORKFLOWSERVICE.methods_by_name[ "ListBatchOperations" ]._serialized_options = b"\202\323\344\223\002]\022(/namespaces/{namespace}/batch-operationsZ1\022//api/v1/namespaces/{namespace}/batch-operations" + _WORKFLOWSERVICE.methods_by_name["UpdateActivityExecutionOptions"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "UpdateActivityExecutionOptions" + ]._serialized_options = b'\202\323\344\223\002\274\002"3/namespaces/activities/{activity_id}/update-options:\001*Z?":/api/v1/namespaces/activities/{activity_id}/update-options:\001*Z\\"W/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/update-options:\001*Zc"^/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/update-options:\001*' _WORKFLOWSERVICE.methods_by_name["UpdateActivityOptions"]._options = None _WORKFLOWSERVICE.methods_by_name[ "UpdateActivityOptions" @@ -277,14 +285,26 @@ _WORKFLOWSERVICE.methods_by_name[ "UpdateWorkflowExecutionOptions" ]._serialized_options = b'\202\323\344\223\002\265\001"Q/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update-options:\001*Z]"X/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update-options:\001*' + _WORKFLOWSERVICE.methods_by_name["PauseActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "PauseActivityExecution" + ]._serialized_options = b'\202\323\344\223\002\230\002"*/namespaces/activities/{activity_id}/pause:\001*Z6"1/api/v1/namespaces/activities/{activity_id}/pause:\001*ZS"N/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/pause:\001*ZZ"U/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/pause:\001*' _WORKFLOWSERVICE.methods_by_name["PauseActivity"]._options = None _WORKFLOWSERVICE.methods_by_name[ "PauseActivity" ]._serialized_options = b'\202\323\344\223\002c"(/namespaces/{namespace}/activities/pause:\001*Z4"//api/v1/namespaces/{namespace}/activities/pause:\001*' + _WORKFLOWSERVICE.methods_by_name["UnpauseActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "UnpauseActivityExecution" + ]._serialized_options = b'\202\323\344\223\002\240\002",/namespaces/activities/{activity_id}/unpause:\001*Z8"3/api/v1/namespaces/activities/{activity_id}/unpause:\001*ZU"P/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/unpause:\001*Z\\"W/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/unpause:\001*' _WORKFLOWSERVICE.methods_by_name["UnpauseActivity"]._options = None _WORKFLOWSERVICE.methods_by_name[ "UnpauseActivity" ]._serialized_options = b'\202\323\344\223\002g"*/namespaces/{namespace}/activities/unpause:\001*Z6"1/api/v1/namespaces/{namespace}/activities/unpause:\001*' + _WORKFLOWSERVICE.methods_by_name["ResetActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "ResetActivityExecution" + ]._serialized_options = b'\202\323\344\223\002\230\002"*/namespaces/activities/{activity_id}/reset:\001*Z6"1/api/v1/namespaces/activities/{activity_id}/reset:\001*ZS"N/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/reset:\001*ZZ"U/api/v1/namespaces/{namespace}/workflows/{workflow_id}/activities/{activity_id}/reset:\001*' _WORKFLOWSERVICE.methods_by_name["ResetActivity"]._options = None _WORKFLOWSERVICE.methods_by_name[ "ResetActivity" @@ -329,6 +349,38 @@ _WORKFLOWSERVICE.methods_by_name[ "UpdateWorkerConfig" ]._serialized_options = b'\202\323\344\223\002m"-/namespaces/{namespace}/workers/update-config:\001*Z9"4/api/v1/namespaces/{namespace}/workers/update-config:\001*' + _WORKFLOWSERVICE.methods_by_name["DescribeWorker"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "DescribeWorker" + ]._serialized_options = b"\202\323\344\223\002\211\001\022>/namespaces/{namespace}/workers/describe/{worker_instance_key}ZG\022E/api/v1/namespaces/{namespace}/workers/describe/{worker_instance_key}" + _WORKFLOWSERVICE.methods_by_name["StartActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "StartActivityExecution" + ]._serialized_options = b'\202\323\344\223\002s"0/namespaces/{namespace}/activities/{activity_id}:\001*Z<"7/api/v1/namespaces/{namespace}/activities/{activity_id}:\001*' + _WORKFLOWSERVICE.methods_by_name["DescribeActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "DescribeActivityExecution" + ]._serialized_options = b"\202\323\344\223\002m\0220/namespaces/{namespace}/activities/{activity_id}Z9\0227/api/v1/namespaces/{namespace}/activities/{activity_id}" + _WORKFLOWSERVICE.methods_by_name["ListActivityExecutions"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "ListActivityExecutions" + ]._serialized_options = b'\202\323\344\223\002Q\022"/namespaces/{namespace}/activitiesZ+\022)/api/v1/namespaces/{namespace}/activities' + _WORKFLOWSERVICE.methods_by_name["CountActivityExecutions"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "CountActivityExecutions" + ]._serialized_options = b"\202\323\344\223\002Y\022&/namespaces/{namespace}/activity-countZ/\022-/api/v1/namespaces/{namespace}/activity-count" + _WORKFLOWSERVICE.methods_by_name["GetActivityExecutionResult"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "GetActivityExecutionResult" + ]._serialized_options = b"\202\323\344\223\002{\0227/namespaces/{namespace}/activities/{activity_id}/resultZ@\022>/api/v1/namespaces/{namespace}/activities/{activity_id}/result" + _WORKFLOWSERVICE.methods_by_name["RequestCancelActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "RequestCancelActivityExecution" + ]._serialized_options = b'\202\323\344\223\002\201\001"7/namespaces/{namespace}/activities/{activity_id}/cancel:\001*ZC">/api/v1/namespaces/{namespace}/activities/{activity_id}/cancel:\001*' + _WORKFLOWSERVICE.methods_by_name["TerminateActivityExecution"]._options = None + _WORKFLOWSERVICE.methods_by_name[ + "TerminateActivityExecution" + ]._serialized_options = b'\202\323\344\223\002\207\001":/namespaces/{namespace}/activities/{activity_id}/terminate:\001*ZF"A/api/v1/namespaces/{namespace}/activities/{activity_id}/terminate:\001*' _WORKFLOWSERVICE._serialized_start = 170 - _WORKFLOWSERVICE._serialized_end = 24377 + _WORKFLOWSERVICE._serialized_end = 29010 # @@protoc_insertion_point(module_scope) diff --git a/temporalio/api/workflowservice/v1/service_pb2_grpc.py b/temporalio/api/workflowservice/v1/service_pb2_grpc.py index 770781e6d..8dfd495a8 100644 --- a/temporalio/api/workflowservice/v1/service_pb2_grpc.py +++ b/temporalio/api/workflowservice/v1/service_pb2_grpc.py @@ -368,6 +368,11 @@ def __init__(self, channel): request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerDeploymentVersionMetadataRequest.SerializeToString, response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerDeploymentVersionMetadataResponse.FromString, ) + self.SetWorkerDeploymentManager = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/SetWorkerDeploymentManager", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.SetWorkerDeploymentManagerRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.SetWorkerDeploymentManagerResponse.FromString, + ) self.UpdateWorkflowExecution = channel.unary_unary( "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkflowExecution", request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkflowExecutionRequest.SerializeToString, @@ -413,6 +418,11 @@ def __init__(self, channel): request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RespondNexusTaskFailedRequest.SerializeToString, response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RespondNexusTaskFailedResponse.FromString, ) + self.UpdateActivityExecutionOptions = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/UpdateActivityExecutionOptions", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityExecutionOptionsRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityExecutionOptionsResponse.FromString, + ) self.UpdateActivityOptions = channel.unary_unary( "/temporal.api.workflowservice.v1.WorkflowService/UpdateActivityOptions", request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityOptionsRequest.SerializeToString, @@ -423,16 +433,31 @@ def __init__(self, channel): request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkflowExecutionOptionsRequest.SerializeToString, response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkflowExecutionOptionsResponse.FromString, ) + self.PauseActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/PauseActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityExecutionResponse.FromString, + ) self.PauseActivity = channel.unary_unary( "/temporal.api.workflowservice.v1.WorkflowService/PauseActivity", request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityRequest.SerializeToString, response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityResponse.FromString, ) + self.UnpauseActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/UnpauseActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityExecutionResponse.FromString, + ) self.UnpauseActivity = channel.unary_unary( "/temporal.api.workflowservice.v1.WorkflowService/UnpauseActivity", request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityRequest.SerializeToString, response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityResponse.FromString, ) + self.ResetActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/ResetActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityExecutionResponse.FromString, + ) self.ResetActivity = channel.unary_unary( "/temporal.api.workflowservice.v1.WorkflowService/ResetActivity", request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityRequest.SerializeToString, @@ -488,6 +513,51 @@ def __init__(self, channel): request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerConfigRequest.SerializeToString, response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerConfigResponse.FromString, ) + self.DescribeWorker = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorker", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeWorkerRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeWorkerResponse.FromString, + ) + self.StartActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/StartActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.StartActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.StartActivityExecutionResponse.FromString, + ) + self.DescribeActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/DescribeActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeActivityExecutionResponse.FromString, + ) + self.ListActivityExecutions = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/ListActivityExecutions", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ListActivityExecutionsRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ListActivityExecutionsResponse.FromString, + ) + self.CountActivityExecutions = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/CountActivityExecutions", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.CountActivityExecutionsRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.CountActivityExecutionsResponse.FromString, + ) + self.GetActivityExecutionResult = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/GetActivityExecutionResult", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.GetActivityExecutionResultRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.GetActivityExecutionResultResponse.FromString, + ) + self.RequestCancelActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RequestCancelActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RequestCancelActivityExecutionResponse.FromString, + ) + self.TerminateActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/TerminateActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.TerminateActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.TerminateActivityExecutionResponse.FromString, + ) + self.DeleteActivityExecution = channel.unary_unary( + "/temporal.api.workflowservice.v1.WorkflowService/DeleteActivityExecution", + request_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DeleteActivityExecutionRequest.SerializeToString, + response_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DeleteActivityExecutionResponse.FromString, + ) class WorkflowServiceServicer(object): @@ -857,7 +927,8 @@ def ListArchivedWorkflowExecutions(self, request, context): raise NotImplementedError("Method not implemented!") def ScanWorkflowExecutions(self, request, context): - """ScanWorkflowExecutions is a visibility API to list large amount of workflow executions in a specific namespace without order. + """ScanWorkflowExecutions _was_ a visibility API to list large amount of workflow executions in a specific namespace without order. + It has since been deprecated in favor of `ListWorkflowExecutions` and rewritten to use `ListWorkflowExecutions` internally. Deprecated: Replaced with `ListWorkflowExecutions`. (-- api-linter: core::0127::http-annotation=disabled @@ -1233,6 +1304,14 @@ def UpdateWorkerDeploymentVersionMetadata(self, request, context): context.set_details("Method not implemented!") raise NotImplementedError("Method not implemented!") + def SetWorkerDeploymentManager(self, request, context): + """Set/unset the ManagerIdentity of a Worker Deployment. + Experimental. This API might significantly change or be removed in a future release. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + def UpdateWorkflowExecution(self, request, context): """Invokes the specified Update function on user Workflow code.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) @@ -1303,9 +1382,18 @@ def RespondNexusTaskFailed(self, request, context): context.set_details("Method not implemented!") raise NotImplementedError("Method not implemented!") + def UpdateActivityExecutionOptions(self, request, context): + """UpdateActivityExecutionOptions is called by the client to update the options of an activity by its ID or type. + If there are multiple pending activities of the provided type - all of them will be updated. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + def UpdateActivityOptions(self, request, context): """UpdateActivityOptions is called by the client to update the options of an activity by its ID or type. If there are multiple pending activities of the provided type - all of them will be updated. + Deprecated. See UpdateActivityExecutionOptions. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) context.set_details("Method not implemented!") @@ -1317,6 +1405,28 @@ def UpdateWorkflowExecutionOptions(self, request, context): context.set_details("Method not implemented!") raise NotImplementedError("Method not implemented!") + def PauseActivityExecution(self, request, context): + """PauseActivityExecution pauses the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be paused + + Pausing an activity means: + - If the activity is currently waiting for a retry or is running and subsequently fails, + it will not be rescheduled until it is unpaused. + - If the activity is already paused, calling this method will have no effect. + - If the activity is running and finishes successfully, the activity will be completed. + - If the activity is running and finishes with failure: + * if there is no retry left - the activity will be completed. + * if there are more retries left - the activity will be paused. + For long-running activities: + - activities in paused state will send a cancellation with "activity_paused" set to 'true' in response to 'RecordActivityTaskHeartbeat'. + - The activity should respond to the cancellation accordingly. + + Returns a `NotFound` error if there is no pending activity with the provided ID or type + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + def PauseActivity(self, request, context): """PauseActivity pauses the execution of an activity specified by its ID or type. If there are multiple pending activities of the provided type - all of them will be paused @@ -1333,6 +1443,26 @@ def PauseActivity(self, request, context): - activities in paused state will send a cancellation with "activity_paused" set to 'true' in response to 'RecordActivityTaskHeartbeat'. - The activity should respond to the cancellation accordingly. + Returns a `NotFound` error if there is no pending activity with the provided ID or type + Deprecated. See PauseActivityExecution. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def UnpauseActivityExecution(self, request, context): + """UnpauseActivityExecution unpauses the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be unpaused. + + If activity is not paused, this call will have no effect. + If the activity was paused while waiting for retry, it will be scheduled immediately (* see 'jitter' flag). + Once the activity is unpaused, all timeout timers will be regenerated. + + Flags: + 'jitter': the activity will be scheduled at a random time within the jitter duration. + 'reset_attempts': the number of attempts will be reset. + 'reset_heartbeat': the activity heartbeat timer and heartbeats will be reset. + Returns a `NotFound` error if there is no pending activity with the provided ID or type """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) @@ -1353,6 +1483,30 @@ def UnpauseActivity(self, request, context): 'reset_heartbeat': the activity heartbeat timer and heartbeats will be reset. Returns a `NotFound` error if there is no pending activity with the provided ID or type + Deprecated. See UnpauseActivityExecution. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def ResetActivityExecution(self, request, context): + """ResetActivityExecution resets the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be reset. + + Resetting an activity means: + * number of attempts will be reset to 0. + * activity timeouts will be reset. + * if the activity is waiting for retry, and it is not paused or 'keep_paused' is not provided: + it will be scheduled immediately (* see 'jitter' flag), + + Flags: + + 'jitter': the activity will be scheduled at a random time within the jitter duration. + If the activity currently paused it will be unpaused, unless 'keep_paused' flag is provided. + 'reset_heartbeats': the activity heartbeat timer and heartbeats will be reset. + 'keep_paused': if the activity is paused, it will remain paused. + + Returns a `NotFound` error if there is no pending activity with the provided ID or type. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) context.set_details("Method not implemented!") @@ -1376,6 +1530,7 @@ def ResetActivity(self, request, context): 'keep_paused': if the activity is paused, it will remain paused. Returns a `NotFound` error if there is no pending activity with the provided ID or type. + Deprecated. See ResetActivityExecution. """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) context.set_details("Method not implemented!") @@ -1459,6 +1614,91 @@ def UpdateWorkerConfig(self, request, context): context.set_details("Method not implemented!") raise NotImplementedError("Method not implemented!") + def DescribeWorker(self, request, context): + """DescribeWorker returns information about the specified worker.""" + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def StartActivityExecution(self, request, context): + """StartActivityExecution starts a new activity execution. + + Returns an `ExecutionAlreadyStarted` error if an instance already exists with same activity ID in this namespace + unless permitted by the specified ID conflict policy. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def DescribeActivityExecution(self, request, context): + """DescribeActivityExecution returns information about the specified activity execution. + Pass in a long_poll_token to turn this request into a long poll that gets unblocked when the activity makes + progress. + In case the activity has not made progress by the time the long poll request times out, an empty response is + returned and the caller may issue an identical DescribeActivityExecution request to continue polling. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def ListActivityExecutions(self, request, context): + """ListActivityExecutions is a visibility API to list activity executions in a specific namespace.""" + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def CountActivityExecutions(self, request, context): + """CountActivityExecutions is a visibility API to count of activity executions in a specific namespace.""" + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def GetActivityExecutionResult(self, request, context): + """GetActivityExecutionResult returns the activity result if it is in a terminal status or (optionally) wait for it + to reach one. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def RequestCancelActivityExecution(self, request, context): + """RequestCancelActivityExecution requests cancellation of an activity execution. + + Requesting to cancel an activity does not automatically transition the activity to canceled status. If the + activity has a currently running attempt, the activity will only transition to canceled status if the current + attempt is unsuccessful. + TODO: Clarify what happens if there are no more allowed retries after the current attempt. + + It returns success if the requested activity is already closed. + TODO: This ^^ is copied from RequestCancelWorkflowExecution, do we want to preserve this behavior? + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def TerminateActivityExecution(self, request, context): + """TerminateActivityExecution terminates an existing activity execution immediately. + + Termination does not reach the worker and the activity code cannot react to it. A terminated activity may have a + running attempt and will be requested to be canceled by the server when it heartbeats. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + + def DeleteActivityExecution(self, request, context): + """DeleteActivityExecution asynchronously deletes a specific activity execution (when + ActivityExecution.run_id is provided) or the latest activity execution (when + ActivityExecution.run_id is not provided). If the activity EXecution is running, it will be + terminated before deletion. + + (-- api-linter: core::0127::http-annotation=disabled + aip.dev/not-precedent: Activity deletion not exposed to HTTP, users should use cancel or terminate. --) + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + def add_WorkflowServiceServicer_to_server(servicer, server): rpc_method_handlers = { @@ -1802,6 +2042,11 @@ def add_WorkflowServiceServicer_to_server(servicer, server): request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerDeploymentVersionMetadataRequest.FromString, response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerDeploymentVersionMetadataResponse.SerializeToString, ), + "SetWorkerDeploymentManager": grpc.unary_unary_rpc_method_handler( + servicer.SetWorkerDeploymentManager, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.SetWorkerDeploymentManagerRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.SetWorkerDeploymentManagerResponse.SerializeToString, + ), "UpdateWorkflowExecution": grpc.unary_unary_rpc_method_handler( servicer.UpdateWorkflowExecution, request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkflowExecutionRequest.FromString, @@ -1847,6 +2092,11 @@ def add_WorkflowServiceServicer_to_server(servicer, server): request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RespondNexusTaskFailedRequest.FromString, response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RespondNexusTaskFailedResponse.SerializeToString, ), + "UpdateActivityExecutionOptions": grpc.unary_unary_rpc_method_handler( + servicer.UpdateActivityExecutionOptions, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityExecutionOptionsRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityExecutionOptionsResponse.SerializeToString, + ), "UpdateActivityOptions": grpc.unary_unary_rpc_method_handler( servicer.UpdateActivityOptions, request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityOptionsRequest.FromString, @@ -1857,16 +2107,31 @@ def add_WorkflowServiceServicer_to_server(servicer, server): request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkflowExecutionOptionsRequest.FromString, response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkflowExecutionOptionsResponse.SerializeToString, ), + "PauseActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.PauseActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityExecutionResponse.SerializeToString, + ), "PauseActivity": grpc.unary_unary_rpc_method_handler( servicer.PauseActivity, request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityRequest.FromString, response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityResponse.SerializeToString, ), + "UnpauseActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.UnpauseActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityExecutionResponse.SerializeToString, + ), "UnpauseActivity": grpc.unary_unary_rpc_method_handler( servicer.UnpauseActivity, request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityRequest.FromString, response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityResponse.SerializeToString, ), + "ResetActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.ResetActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityExecutionResponse.SerializeToString, + ), "ResetActivity": grpc.unary_unary_rpc_method_handler( servicer.ResetActivity, request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityRequest.FromString, @@ -1922,6 +2187,51 @@ def add_WorkflowServiceServicer_to_server(servicer, server): request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerConfigRequest.FromString, response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateWorkerConfigResponse.SerializeToString, ), + "DescribeWorker": grpc.unary_unary_rpc_method_handler( + servicer.DescribeWorker, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeWorkerRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeWorkerResponse.SerializeToString, + ), + "StartActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.StartActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.StartActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.StartActivityExecutionResponse.SerializeToString, + ), + "DescribeActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.DescribeActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeActivityExecutionResponse.SerializeToString, + ), + "ListActivityExecutions": grpc.unary_unary_rpc_method_handler( + servicer.ListActivityExecutions, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ListActivityExecutionsRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ListActivityExecutionsResponse.SerializeToString, + ), + "CountActivityExecutions": grpc.unary_unary_rpc_method_handler( + servicer.CountActivityExecutions, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.CountActivityExecutionsRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.CountActivityExecutionsResponse.SerializeToString, + ), + "GetActivityExecutionResult": grpc.unary_unary_rpc_method_handler( + servicer.GetActivityExecutionResult, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.GetActivityExecutionResultRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.GetActivityExecutionResultResponse.SerializeToString, + ), + "RequestCancelActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.RequestCancelActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RequestCancelActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RequestCancelActivityExecutionResponse.SerializeToString, + ), + "TerminateActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.TerminateActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.TerminateActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.TerminateActivityExecutionResponse.SerializeToString, + ), + "DeleteActivityExecution": grpc.unary_unary_rpc_method_handler( + servicer.DeleteActivityExecution, + request_deserializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DeleteActivityExecutionRequest.FromString, + response_serializer=temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DeleteActivityExecutionResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( "temporal.api.workflowservice.v1.WorkflowService", rpc_method_handlers @@ -3916,6 +4226,35 @@ def UpdateWorkerDeploymentVersionMetadata( metadata, ) + @staticmethod + def SetWorkerDeploymentManager( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/SetWorkerDeploymentManager", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.SetWorkerDeploymentManagerRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.SetWorkerDeploymentManagerResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + @staticmethod def UpdateWorkflowExecution( request, @@ -4177,6 +4516,35 @@ def RespondNexusTaskFailed( metadata, ) + @staticmethod + def UpdateActivityExecutionOptions( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/UpdateActivityExecutionOptions", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityExecutionOptionsRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UpdateActivityExecutionOptionsResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + @staticmethod def UpdateActivityOptions( request, @@ -4235,6 +4603,35 @@ def UpdateWorkflowExecutionOptions( metadata, ) + @staticmethod + def PauseActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/PauseActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.PauseActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + @staticmethod def PauseActivity( request, @@ -4264,6 +4661,35 @@ def PauseActivity( metadata, ) + @staticmethod + def UnpauseActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/UnpauseActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.UnpauseActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + @staticmethod def UnpauseActivity( request, @@ -4293,6 +4719,35 @@ def UnpauseActivity( metadata, ) + @staticmethod + def ResetActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/ResetActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ResetActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + @staticmethod def ResetActivity( request, @@ -4611,3 +5066,264 @@ def UpdateWorkerConfig( timeout, metadata, ) + + @staticmethod + def DescribeWorker( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorker", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeWorkerRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeWorkerResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def StartActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/StartActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.StartActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.StartActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def DescribeActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/DescribeActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DescribeActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def ListActivityExecutions( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/ListActivityExecutions", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ListActivityExecutionsRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.ListActivityExecutionsResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def CountActivityExecutions( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/CountActivityExecutions", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.CountActivityExecutionsRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.CountActivityExecutionsResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def GetActivityExecutionResult( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/GetActivityExecutionResult", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.GetActivityExecutionResultRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.GetActivityExecutionResultResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def RequestCancelActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RequestCancelActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.RequestCancelActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def TerminateActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/TerminateActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.TerminateActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.TerminateActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) + + @staticmethod + def DeleteActivityExecution( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/temporal.api.workflowservice.v1.WorkflowService/DeleteActivityExecution", + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DeleteActivityExecutionRequest.SerializeToString, + temporal_dot_api_dot_workflowservice_dot_v1_dot_request__response__pb2.DeleteActivityExecutionResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/temporalio/api/workflowservice/v1/service_pb2_grpc.pyi b/temporalio/api/workflowservice/v1/service_pb2_grpc.pyi index 6da9c7db3..63436bc3e 100644 --- a/temporalio/api/workflowservice/v1/service_pb2_grpc.pyi +++ b/temporalio/api/workflowservice/v1/service_pb2_grpc.pyi @@ -100,8 +100,8 @@ class WorkflowServiceStub: temporalio.api.workflowservice.v1.request_response_pb2.GetWorkflowExecutionHistoryReverseRequest, temporalio.api.workflowservice.v1.request_response_pb2.GetWorkflowExecutionHistoryReverseResponse, ] - """GetWorkflowExecutionHistoryReverse returns the history of specified workflow execution in reverse - order (starting from last event). Fails with`NotFound` if the specified workflow execution is + """GetWorkflowExecutionHistoryReverse returns the history of specified workflow execution in reverse + order (starting from last event). Fails with`NotFound` if the specified workflow execution is unknown to the service. """ PollWorkflowTaskQueue: grpc.UnaryUnaryMultiCallable[ @@ -349,7 +349,8 @@ class WorkflowServiceStub: temporalio.api.workflowservice.v1.request_response_pb2.ScanWorkflowExecutionsRequest, temporalio.api.workflowservice.v1.request_response_pb2.ScanWorkflowExecutionsResponse, ] - """ScanWorkflowExecutions is a visibility API to list large amount of workflow executions in a specific namespace without order. + """ScanWorkflowExecutions _was_ a visibility API to list large amount of workflow executions in a specific namespace without order. + It has since been deprecated in favor of `ListWorkflowExecutions` and rewritten to use `ListWorkflowExecutions` internally. Deprecated: Replaced with `ListWorkflowExecutions`. (-- api-linter: core::0127::http-annotation=disabled @@ -502,7 +503,7 @@ class WorkflowServiceStub: members are compatible with one another. A single build id may be mapped to multiple task queues using this API for cases where a single process hosts - multiple workers. + multiple workers. To query which workers can be retired, use the `GetWorkerTaskReachability` API. @@ -685,6 +686,13 @@ class WorkflowServiceStub: """Updates the user-given metadata attached to a Worker Deployment Version. Experimental. This API might significantly change or be removed in a future release. """ + SetWorkerDeploymentManager: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.SetWorkerDeploymentManagerRequest, + temporalio.api.workflowservice.v1.request_response_pb2.SetWorkerDeploymentManagerResponse, + ] + """Set/unset the ManagerIdentity of a Worker Deployment. + Experimental. This API might significantly change or be removed in a future release. + """ UpdateWorkflowExecution: grpc.UnaryUnaryMultiCallable[ temporalio.api.workflowservice.v1.request_response_pb2.UpdateWorkflowExecutionRequest, temporalio.api.workflowservice.v1.request_response_pb2.UpdateWorkflowExecutionResponse, @@ -746,18 +754,47 @@ class WorkflowServiceStub: (-- api-linter: core::0127::http-annotation=disabled aip.dev/not-precedent: We do not expose worker API to HTTP. --) """ + UpdateActivityExecutionOptions: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityExecutionOptionsRequest, + temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityExecutionOptionsResponse, + ] + """UpdateActivityExecutionOptions is called by the client to update the options of an activity by its ID or type. + If there are multiple pending activities of the provided type - all of them will be updated. + """ UpdateActivityOptions: grpc.UnaryUnaryMultiCallable[ temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityOptionsRequest, temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityOptionsResponse, ] """UpdateActivityOptions is called by the client to update the options of an activity by its ID or type. If there are multiple pending activities of the provided type - all of them will be updated. + Deprecated. See UpdateActivityExecutionOptions. """ UpdateWorkflowExecutionOptions: grpc.UnaryUnaryMultiCallable[ temporalio.api.workflowservice.v1.request_response_pb2.UpdateWorkflowExecutionOptionsRequest, temporalio.api.workflowservice.v1.request_response_pb2.UpdateWorkflowExecutionOptionsResponse, ] """UpdateWorkflowExecutionOptions partially updates the WorkflowExecutionOptions of an existing workflow execution.""" + PauseActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityExecutionResponse, + ] + """PauseActivityExecution pauses the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be paused + + Pausing an activity means: + - If the activity is currently waiting for a retry or is running and subsequently fails, + it will not be rescheduled until it is unpaused. + - If the activity is already paused, calling this method will have no effect. + - If the activity is running and finishes successfully, the activity will be completed. + - If the activity is running and finishes with failure: + * if there is no retry left - the activity will be completed. + * if there are more retries left - the activity will be paused. + For long-running activities: + - activities in paused state will send a cancellation with "activity_paused" set to 'true' in response to 'RecordActivityTaskHeartbeat'. + - The activity should respond to the cancellation accordingly. + + Returns a `NotFound` error if there is no pending activity with the provided ID or type + """ PauseActivity: grpc.UnaryUnaryMultiCallable[ temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityRequest, temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityResponse, @@ -777,6 +814,25 @@ class WorkflowServiceStub: - activities in paused state will send a cancellation with "activity_paused" set to 'true' in response to 'RecordActivityTaskHeartbeat'. - The activity should respond to the cancellation accordingly. + Returns a `NotFound` error if there is no pending activity with the provided ID or type + Deprecated. See PauseActivityExecution. + """ + UnpauseActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.UnpauseActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.UnpauseActivityExecutionResponse, + ] + """UnpauseActivityExecution unpauses the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be unpaused. + + If activity is not paused, this call will have no effect. + If the activity was paused while waiting for retry, it will be scheduled immediately (* see 'jitter' flag). + Once the activity is unpaused, all timeout timers will be regenerated. + + Flags: + 'jitter': the activity will be scheduled at a random time within the jitter duration. + 'reset_attempts': the number of attempts will be reset. + 'reset_heartbeat': the activity heartbeat timer and heartbeats will be reset. + Returns a `NotFound` error if there is no pending activity with the provided ID or type """ UnpauseActivity: grpc.UnaryUnaryMultiCallable[ @@ -796,6 +852,29 @@ class WorkflowServiceStub: 'reset_heartbeat': the activity heartbeat timer and heartbeats will be reset. Returns a `NotFound` error if there is no pending activity with the provided ID or type + Deprecated. See UnpauseActivityExecution. + """ + ResetActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.ResetActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.ResetActivityExecutionResponse, + ] + """ResetActivityExecution resets the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be reset. + + Resetting an activity means: + * number of attempts will be reset to 0. + * activity timeouts will be reset. + * if the activity is waiting for retry, and it is not paused or 'keep_paused' is not provided: + it will be scheduled immediately (* see 'jitter' flag), + + Flags: + + 'jitter': the activity will be scheduled at a random time within the jitter duration. + If the activity currently paused it will be unpaused, unless 'keep_paused' flag is provided. + 'reset_heartbeats': the activity heartbeat timer and heartbeats will be reset. + 'keep_paused': if the activity is paused, it will remain paused. + + Returns a `NotFound` error if there is no pending activity with the provided ID or type. """ ResetActivity: grpc.UnaryUnaryMultiCallable[ temporalio.api.workflowservice.v1.request_response_pb2.ResetActivityRequest, @@ -818,6 +897,7 @@ class WorkflowServiceStub: 'keep_paused': if the activity is paused, it will remain paused. Returns a `NotFound` error if there is no pending activity with the provided ID or type. + Deprecated. See ResetActivityExecution. """ CreateWorkflowRule: grpc.UnaryUnaryMultiCallable[ temporalio.api.workflowservice.v1.request_response_pb2.CreateWorkflowRuleRequest, @@ -887,6 +967,82 @@ class WorkflowServiceStub: Can be used to partially update the worker configuration. Can be used to update the configuration of multiple workers. """ + DescribeWorker: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.DescribeWorkerRequest, + temporalio.api.workflowservice.v1.request_response_pb2.DescribeWorkerResponse, + ] + """DescribeWorker returns information about the specified worker.""" + StartActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.StartActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.StartActivityExecutionResponse, + ] + """StartActivityExecution starts a new activity execution. + + Returns an `ExecutionAlreadyStarted` error if an instance already exists with same activity ID in this namespace + unless permitted by the specified ID conflict policy. + """ + DescribeActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.DescribeActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.DescribeActivityExecutionResponse, + ] + """DescribeActivityExecution returns information about the specified activity execution. + Pass in a long_poll_token to turn this request into a long poll that gets unblocked when the activity makes + progress. + In case the activity has not made progress by the time the long poll request times out, an empty response is + returned and the caller may issue an identical DescribeActivityExecution request to continue polling. + """ + ListActivityExecutions: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.ListActivityExecutionsRequest, + temporalio.api.workflowservice.v1.request_response_pb2.ListActivityExecutionsResponse, + ] + """ListActivityExecutions is a visibility API to list activity executions in a specific namespace.""" + CountActivityExecutions: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.CountActivityExecutionsRequest, + temporalio.api.workflowservice.v1.request_response_pb2.CountActivityExecutionsResponse, + ] + """CountActivityExecutions is a visibility API to count of activity executions in a specific namespace.""" + GetActivityExecutionResult: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.GetActivityExecutionResultRequest, + temporalio.api.workflowservice.v1.request_response_pb2.GetActivityExecutionResultResponse, + ] + """GetActivityExecutionResult returns the activity result if it is in a terminal status or (optionally) wait for it + to reach one. + """ + RequestCancelActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.RequestCancelActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.RequestCancelActivityExecutionResponse, + ] + """RequestCancelActivityExecution requests cancellation of an activity execution. + + Requesting to cancel an activity does not automatically transition the activity to canceled status. If the + activity has a currently running attempt, the activity will only transition to canceled status if the current + attempt is unsuccessful. + TODO: Clarify what happens if there are no more allowed retries after the current attempt. + + It returns success if the requested activity is already closed. + TODO: This ^^ is copied from RequestCancelWorkflowExecution, do we want to preserve this behavior? + """ + TerminateActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.TerminateActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.TerminateActivityExecutionResponse, + ] + """TerminateActivityExecution terminates an existing activity execution immediately. + + Termination does not reach the worker and the activity code cannot react to it. A terminated activity may have a + running attempt and will be requested to be canceled by the server when it heartbeats. + """ + DeleteActivityExecution: grpc.UnaryUnaryMultiCallable[ + temporalio.api.workflowservice.v1.request_response_pb2.DeleteActivityExecutionRequest, + temporalio.api.workflowservice.v1.request_response_pb2.DeleteActivityExecutionResponse, + ] + """DeleteActivityExecution asynchronously deletes a specific activity execution (when + ActivityExecution.run_id is provided) or the latest activity execution (when + ActivityExecution.run_id is not provided). If the activity EXecution is running, it will be + terminated before deletion. + + (-- api-linter: core::0127::http-annotation=disabled + aip.dev/not-precedent: Activity deletion not exposed to HTTP, users should use cancel or terminate. --) + """ class WorkflowServiceServicer(metaclass=abc.ABCMeta): """WorkflowService API defines how Temporal SDKs and other clients interact with the Temporal server @@ -1295,7 +1451,8 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): request: temporalio.api.workflowservice.v1.request_response_pb2.ScanWorkflowExecutionsRequest, context: grpc.ServicerContext, ) -> temporalio.api.workflowservice.v1.request_response_pb2.ScanWorkflowExecutionsResponse: - """ScanWorkflowExecutions is a visibility API to list large amount of workflow executions in a specific namespace without order. + """ScanWorkflowExecutions _was_ a visibility API to list large amount of workflow executions in a specific namespace without order. + It has since been deprecated in favor of `ListWorkflowExecutions` and rewritten to use `ListWorkflowExecutions` internally. Deprecated: Replaced with `ListWorkflowExecutions`. (-- api-linter: core::0127::http-annotation=disabled @@ -1708,6 +1865,15 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): Experimental. This API might significantly change or be removed in a future release. """ @abc.abstractmethod + def SetWorkerDeploymentManager( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.SetWorkerDeploymentManagerRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.SetWorkerDeploymentManagerResponse: + """Set/unset the ManagerIdentity of a Worker Deployment. + Experimental. This API might significantly change or be removed in a future release. + """ + @abc.abstractmethod def UpdateWorkflowExecution( self, request: temporalio.api.workflowservice.v1.request_response_pb2.UpdateWorkflowExecutionRequest, @@ -1787,6 +1953,15 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): aip.dev/not-precedent: We do not expose worker API to HTTP. --) """ @abc.abstractmethod + def UpdateActivityExecutionOptions( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityExecutionOptionsRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityExecutionOptionsResponse: + """UpdateActivityExecutionOptions is called by the client to update the options of an activity by its ID or type. + If there are multiple pending activities of the provided type - all of them will be updated. + """ + @abc.abstractmethod def UpdateActivityOptions( self, request: temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityOptionsRequest, @@ -1794,6 +1969,7 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): ) -> temporalio.api.workflowservice.v1.request_response_pb2.UpdateActivityOptionsResponse: """UpdateActivityOptions is called by the client to update the options of an activity by its ID or type. If there are multiple pending activities of the provided type - all of them will be updated. + Deprecated. See UpdateActivityExecutionOptions. """ @abc.abstractmethod def UpdateWorkflowExecutionOptions( @@ -1803,6 +1979,29 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): ) -> temporalio.api.workflowservice.v1.request_response_pb2.UpdateWorkflowExecutionOptionsResponse: """UpdateWorkflowExecutionOptions partially updates the WorkflowExecutionOptions of an existing workflow execution.""" @abc.abstractmethod + def PauseActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityExecutionResponse: + """PauseActivityExecution pauses the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be paused + + Pausing an activity means: + - If the activity is currently waiting for a retry or is running and subsequently fails, + it will not be rescheduled until it is unpaused. + - If the activity is already paused, calling this method will have no effect. + - If the activity is running and finishes successfully, the activity will be completed. + - If the activity is running and finishes with failure: + * if there is no retry left - the activity will be completed. + * if there are more retries left - the activity will be paused. + For long-running activities: + - activities in paused state will send a cancellation with "activity_paused" set to 'true' in response to 'RecordActivityTaskHeartbeat'. + - The activity should respond to the cancellation accordingly. + + Returns a `NotFound` error if there is no pending activity with the provided ID or type + """ + @abc.abstractmethod def PauseActivity( self, request: temporalio.api.workflowservice.v1.request_response_pb2.PauseActivityRequest, @@ -1823,6 +2022,27 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): - activities in paused state will send a cancellation with "activity_paused" set to 'true' in response to 'RecordActivityTaskHeartbeat'. - The activity should respond to the cancellation accordingly. + Returns a `NotFound` error if there is no pending activity with the provided ID or type + Deprecated. See PauseActivityExecution. + """ + @abc.abstractmethod + def UnpauseActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.UnpauseActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.UnpauseActivityExecutionResponse: + """UnpauseActivityExecution unpauses the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be unpaused. + + If activity is not paused, this call will have no effect. + If the activity was paused while waiting for retry, it will be scheduled immediately (* see 'jitter' flag). + Once the activity is unpaused, all timeout timers will be regenerated. + + Flags: + 'jitter': the activity will be scheduled at a random time within the jitter duration. + 'reset_attempts': the number of attempts will be reset. + 'reset_heartbeat': the activity heartbeat timer and heartbeats will be reset. + Returns a `NotFound` error if there is no pending activity with the provided ID or type """ @abc.abstractmethod @@ -1844,6 +2064,31 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): 'reset_heartbeat': the activity heartbeat timer and heartbeats will be reset. Returns a `NotFound` error if there is no pending activity with the provided ID or type + Deprecated. See UnpauseActivityExecution. + """ + @abc.abstractmethod + def ResetActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.ResetActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.ResetActivityExecutionResponse: + """ResetActivityExecution resets the execution of an activity specified by its ID or type. + If there are multiple pending activities of the provided type - all of them will be reset. + + Resetting an activity means: + * number of attempts will be reset to 0. + * activity timeouts will be reset. + * if the activity is waiting for retry, and it is not paused or 'keep_paused' is not provided: + it will be scheduled immediately (* see 'jitter' flag), + + Flags: + + 'jitter': the activity will be scheduled at a random time within the jitter duration. + If the activity currently paused it will be unpaused, unless 'keep_paused' flag is provided. + 'reset_heartbeats': the activity heartbeat timer and heartbeats will be reset. + 'keep_paused': if the activity is paused, it will remain paused. + + Returns a `NotFound` error if there is no pending activity with the provided ID or type. """ @abc.abstractmethod def ResetActivity( @@ -1868,6 +2113,7 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): 'keep_paused': if the activity is paused, it will remain paused. Returns a `NotFound` error if there is no pending activity with the provided ID or type. + Deprecated. See ResetActivityExecution. """ @abc.abstractmethod def CreateWorkflowRule( @@ -1961,6 +2207,100 @@ class WorkflowServiceServicer(metaclass=abc.ABCMeta): Can be used to partially update the worker configuration. Can be used to update the configuration of multiple workers. """ + @abc.abstractmethod + def DescribeWorker( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.DescribeWorkerRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.DescribeWorkerResponse: + """DescribeWorker returns information about the specified worker.""" + @abc.abstractmethod + def StartActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.StartActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.StartActivityExecutionResponse: + """StartActivityExecution starts a new activity execution. + + Returns an `ExecutionAlreadyStarted` error if an instance already exists with same activity ID in this namespace + unless permitted by the specified ID conflict policy. + """ + @abc.abstractmethod + def DescribeActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.DescribeActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.DescribeActivityExecutionResponse: + """DescribeActivityExecution returns information about the specified activity execution. + Pass in a long_poll_token to turn this request into a long poll that gets unblocked when the activity makes + progress. + In case the activity has not made progress by the time the long poll request times out, an empty response is + returned and the caller may issue an identical DescribeActivityExecution request to continue polling. + """ + @abc.abstractmethod + def ListActivityExecutions( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.ListActivityExecutionsRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.ListActivityExecutionsResponse: + """ListActivityExecutions is a visibility API to list activity executions in a specific namespace.""" + @abc.abstractmethod + def CountActivityExecutions( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.CountActivityExecutionsRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.CountActivityExecutionsResponse: + """CountActivityExecutions is a visibility API to count of activity executions in a specific namespace.""" + @abc.abstractmethod + def GetActivityExecutionResult( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.GetActivityExecutionResultRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.GetActivityExecutionResultResponse: + """GetActivityExecutionResult returns the activity result if it is in a terminal status or (optionally) wait for it + to reach one. + """ + @abc.abstractmethod + def RequestCancelActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.RequestCancelActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.RequestCancelActivityExecutionResponse: + """RequestCancelActivityExecution requests cancellation of an activity execution. + + Requesting to cancel an activity does not automatically transition the activity to canceled status. If the + activity has a currently running attempt, the activity will only transition to canceled status if the current + attempt is unsuccessful. + TODO: Clarify what happens if there are no more allowed retries after the current attempt. + + It returns success if the requested activity is already closed. + TODO: This ^^ is copied from RequestCancelWorkflowExecution, do we want to preserve this behavior? + """ + @abc.abstractmethod + def TerminateActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.TerminateActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.TerminateActivityExecutionResponse: + """TerminateActivityExecution terminates an existing activity execution immediately. + + Termination does not reach the worker and the activity code cannot react to it. A terminated activity may have a + running attempt and will be requested to be canceled by the server when it heartbeats. + """ + @abc.abstractmethod + def DeleteActivityExecution( + self, + request: temporalio.api.workflowservice.v1.request_response_pb2.DeleteActivityExecutionRequest, + context: grpc.ServicerContext, + ) -> temporalio.api.workflowservice.v1.request_response_pb2.DeleteActivityExecutionResponse: + """DeleteActivityExecution asynchronously deletes a specific activity execution (when + ActivityExecution.run_id is provided) or the latest activity execution (when + ActivityExecution.run_id is not provided). If the activity EXecution is running, it will be + terminated before deletion. + + (-- api-linter: core::0127::http-annotation=disabled + aip.dev/not-precedent: Activity deletion not exposed to HTTP, users should use cancel or terminate. --) + """ def add_WorkflowServiceServicer_to_server( servicer: WorkflowServiceServicer, server: grpc.Server diff --git a/temporalio/bridge/services_generated.py b/temporalio/bridge/services_generated.py index b9aa25d75..5b33c66bc 100644 --- a/temporalio/bridge/services_generated.py +++ b/temporalio/bridge/services_generated.py @@ -26,6 +26,24 @@ def __init__(self, client: ServiceClient): self._client = client self._service = "workflow" + async def count_activity_executions( + self, + req: temporalio.api.workflowservice.v1.CountActivityExecutionsRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.CountActivityExecutionsResponse: + """Invokes the WorkflowService.count_activity_executions rpc method.""" + return await self._client._rpc_call( + rpc="count_activity_executions", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.CountActivityExecutionsResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def count_workflow_executions( self, req: temporalio.api.workflowservice.v1.CountWorkflowExecutionsRequest, @@ -80,6 +98,24 @@ async def create_workflow_rule( timeout=timeout, ) + async def delete_activity_execution( + self, + req: temporalio.api.workflowservice.v1.DeleteActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.DeleteActivityExecutionResponse: + """Invokes the WorkflowService.delete_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="delete_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.DeleteActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def delete_schedule( self, req: temporalio.api.workflowservice.v1.DeleteScheduleRequest, @@ -188,6 +224,24 @@ async def deprecate_namespace( timeout=timeout, ) + async def describe_activity_execution( + self, + req: temporalio.api.workflowservice.v1.DescribeActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.DescribeActivityExecutionResponse: + """Invokes the WorkflowService.describe_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="describe_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.DescribeActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def describe_batch_operation( self, req: temporalio.api.workflowservice.v1.DescribeBatchOperationRequest, @@ -278,6 +332,24 @@ async def describe_task_queue( timeout=timeout, ) + async def describe_worker( + self, + req: temporalio.api.workflowservice.v1.DescribeWorkerRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.DescribeWorkerResponse: + """Invokes the WorkflowService.describe_worker rpc method.""" + return await self._client._rpc_call( + rpc="describe_worker", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.DescribeWorkerResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def describe_worker_deployment( self, req: temporalio.api.workflowservice.v1.DescribeWorkerDeploymentRequest, @@ -386,6 +458,24 @@ async def fetch_worker_config( timeout=timeout, ) + async def get_activity_execution_result( + self, + req: temporalio.api.workflowservice.v1.GetActivityExecutionResultRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.GetActivityExecutionResultResponse: + """Invokes the WorkflowService.get_activity_execution_result rpc method.""" + return await self._client._rpc_call( + rpc="get_activity_execution_result", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.GetActivityExecutionResultResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def get_cluster_info( self, req: temporalio.api.workflowservice.v1.GetClusterInfoRequest, @@ -566,6 +656,24 @@ async def get_workflow_execution_history_reverse( timeout=timeout, ) + async def list_activity_executions( + self, + req: temporalio.api.workflowservice.v1.ListActivityExecutionsRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.ListActivityExecutionsResponse: + """Invokes the WorkflowService.list_activity_executions rpc method.""" + return await self._client._rpc_call( + rpc="list_activity_executions", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.ListActivityExecutionsResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def list_archived_workflow_executions( self, req: temporalio.api.workflowservice.v1.ListArchivedWorkflowExecutionsRequest, @@ -836,6 +944,24 @@ async def pause_activity( timeout=timeout, ) + async def pause_activity_execution( + self, + req: temporalio.api.workflowservice.v1.PauseActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.PauseActivityExecutionResponse: + """Invokes the WorkflowService.pause_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="pause_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.PauseActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def poll_activity_task_queue( self, req: temporalio.api.workflowservice.v1.PollActivityTaskQueueRequest, @@ -998,6 +1124,24 @@ async def register_namespace( timeout=timeout, ) + async def request_cancel_activity_execution( + self, + req: temporalio.api.workflowservice.v1.RequestCancelActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.RequestCancelActivityExecutionResponse: + """Invokes the WorkflowService.request_cancel_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="request_cancel_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.RequestCancelActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def request_cancel_workflow_execution( self, req: temporalio.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest, @@ -1034,6 +1178,24 @@ async def reset_activity( timeout=timeout, ) + async def reset_activity_execution( + self, + req: temporalio.api.workflowservice.v1.ResetActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.ResetActivityExecutionResponse: + """Invokes the WorkflowService.reset_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="reset_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.ResetActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def reset_sticky_task_queue( self, req: temporalio.api.workflowservice.v1.ResetStickyTaskQueueRequest, @@ -1322,6 +1484,24 @@ async def set_worker_deployment_current_version( timeout=timeout, ) + async def set_worker_deployment_manager( + self, + req: temporalio.api.workflowservice.v1.SetWorkerDeploymentManagerRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.SetWorkerDeploymentManagerResponse: + """Invokes the WorkflowService.set_worker_deployment_manager rpc method.""" + return await self._client._rpc_call( + rpc="set_worker_deployment_manager", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.SetWorkerDeploymentManagerResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def set_worker_deployment_ramping_version( self, req: temporalio.api.workflowservice.v1.SetWorkerDeploymentRampingVersionRequest, @@ -1394,6 +1574,24 @@ async def signal_workflow_execution( timeout=timeout, ) + async def start_activity_execution( + self, + req: temporalio.api.workflowservice.v1.StartActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.StartActivityExecutionResponse: + """Invokes the WorkflowService.start_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="start_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.StartActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def start_batch_operation( self, req: temporalio.api.workflowservice.v1.StartBatchOperationRequest, @@ -1448,6 +1646,24 @@ async def stop_batch_operation( timeout=timeout, ) + async def terminate_activity_execution( + self, + req: temporalio.api.workflowservice.v1.TerminateActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.TerminateActivityExecutionResponse: + """Invokes the WorkflowService.terminate_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="terminate_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.TerminateActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def terminate_workflow_execution( self, req: temporalio.api.workflowservice.v1.TerminateWorkflowExecutionRequest, @@ -1502,6 +1718,42 @@ async def unpause_activity( timeout=timeout, ) + async def unpause_activity_execution( + self, + req: temporalio.api.workflowservice.v1.UnpauseActivityExecutionRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.UnpauseActivityExecutionResponse: + """Invokes the WorkflowService.unpause_activity_execution rpc method.""" + return await self._client._rpc_call( + rpc="unpause_activity_execution", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.UnpauseActivityExecutionResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + + async def update_activity_execution_options( + self, + req: temporalio.api.workflowservice.v1.UpdateActivityExecutionOptionsRequest, + retry: bool = False, + metadata: Mapping[str, Union[str, bytes]] = {}, + timeout: Optional[timedelta] = None, + ) -> temporalio.api.workflowservice.v1.UpdateActivityExecutionOptionsResponse: + """Invokes the WorkflowService.update_activity_execution_options rpc method.""" + return await self._client._rpc_call( + rpc="update_activity_execution_options", + req=req, + service=self._service, + resp_type=temporalio.api.workflowservice.v1.UpdateActivityExecutionOptionsResponse, + retry=retry, + metadata=metadata, + timeout=timeout, + ) + async def update_activity_options( self, req: temporalio.api.workflowservice.v1.UpdateActivityOptionsRequest, diff --git a/temporalio/bridge/src/client_rpc_generated.rs b/temporalio/bridge/src/client_rpc_generated.rs index 659f5d8cf..aa999691d 100644 --- a/temporalio/bridge/src/client_rpc_generated.rs +++ b/temporalio/bridge/src/client_rpc_generated.rs @@ -19,6 +19,14 @@ impl ClientRef { let mut retry_client = self.retry_client.clone(); self.runtime.future_into_py(py, async move { let bytes = match call.rpc.as_str() { + "count_activity_executions" => { + rpc_call!( + retry_client, + call, + WorkflowService, + count_activity_executions + ) + } "count_workflow_executions" => { rpc_call!( retry_client, @@ -33,6 +41,14 @@ impl ClientRef { "create_workflow_rule" => { rpc_call!(retry_client, call, WorkflowService, create_workflow_rule) } + "delete_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + delete_activity_execution + ) + } "delete_schedule" => { rpc_call!(retry_client, call, WorkflowService, delete_schedule) } @@ -66,6 +82,14 @@ impl ClientRef { "deprecate_namespace" => { rpc_call!(retry_client, call, WorkflowService, deprecate_namespace) } + "describe_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + describe_activity_execution + ) + } "describe_batch_operation" => { rpc_call!( retry_client, @@ -86,6 +110,9 @@ impl ClientRef { "describe_task_queue" => { rpc_call!(retry_client, call, WorkflowService, describe_task_queue) } + "describe_worker" => { + rpc_call!(retry_client, call, WorkflowService, describe_worker) + } "describe_worker_deployment" => { rpc_call!( retry_client, @@ -119,6 +146,14 @@ impl ClientRef { "fetch_worker_config" => { rpc_call!(retry_client, call, WorkflowService, fetch_worker_config) } + "get_activity_execution_result" => { + rpc_call!( + retry_client, + call, + WorkflowService, + get_activity_execution_result + ) + } "get_cluster_info" => { rpc_call!(retry_client, call, WorkflowService, get_cluster_info) } @@ -179,6 +214,14 @@ impl ClientRef { get_workflow_execution_history_reverse ) } + "list_activity_executions" => { + rpc_call!( + retry_client, + call, + WorkflowService, + list_activity_executions + ) + } "list_archived_workflow_executions" => { rpc_call!( retry_client, @@ -254,6 +297,14 @@ impl ClientRef { "pause_activity" => { rpc_call!(retry_client, call, WorkflowService, pause_activity) } + "pause_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + pause_activity_execution + ) + } "poll_activity_task_queue" => { rpc_call!( retry_client, @@ -306,6 +357,14 @@ impl ClientRef { "register_namespace" => { rpc_call!(retry_client, call, WorkflowService, register_namespace) } + "request_cancel_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + request_cancel_activity_execution + ) + } "request_cancel_workflow_execution" => { rpc_call!( retry_client, @@ -317,6 +376,14 @@ impl ClientRef { "reset_activity" => { rpc_call!(retry_client, call, WorkflowService, reset_activity) } + "reset_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + reset_activity_execution + ) + } "reset_sticky_task_queue" => { rpc_call!(retry_client, call, WorkflowService, reset_sticky_task_queue) } @@ -435,6 +502,14 @@ impl ClientRef { set_worker_deployment_current_version ) } + "set_worker_deployment_manager" => { + rpc_call!( + retry_client, + call, + WorkflowService, + set_worker_deployment_manager + ) + } "set_worker_deployment_ramping_version" => { rpc_call!( retry_client, @@ -462,6 +537,14 @@ impl ClientRef { signal_workflow_execution ) } + "start_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + start_activity_execution + ) + } "start_batch_operation" => { rpc_call!(retry_client, call, WorkflowService, start_batch_operation) } @@ -476,6 +559,14 @@ impl ClientRef { "stop_batch_operation" => { rpc_call!(retry_client, call, WorkflowService, stop_batch_operation) } + "terminate_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + terminate_activity_execution + ) + } "terminate_workflow_execution" => { rpc_call!( retry_client, @@ -490,6 +581,22 @@ impl ClientRef { "unpause_activity" => { rpc_call!(retry_client, call, WorkflowService, unpause_activity) } + "unpause_activity_execution" => { + rpc_call!( + retry_client, + call, + WorkflowService, + unpause_activity_execution + ) + } + "update_activity_execution_options" => { + rpc_call!( + retry_client, + call, + WorkflowService, + update_activity_execution_options + ) + } "update_activity_options" => { rpc_call!(retry_client, call, WorkflowService, update_activity_options) } From c78fee7cd0742c9f6acb1682ef828bbced1b66f7 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 06:18:48 -0400 Subject: [PATCH 06/25] Implement list/count activities --- temporalio/client.py | 543 +++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 522 insertions(+), 21 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index 17946e75f..5322b07c7 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -43,6 +43,7 @@ from google.protobuf.internal.containers import MessageMap from typing_extensions import Concatenate, Required, Self, TypedDict +import temporalio.api.activity.v1 import temporalio.api.common.v1 import temporalio.api.enums.v1 import temporalio.api.errordetails.v1 @@ -1351,10 +1352,11 @@ async def execute_activity(self, *args, **kwargs) -> ReturnType: handle = await self.start_activity(*args, **kwargs) return await handle.result() - async def list_activities( + def list_activities( self, query: Optional[str] = None, *, + limit: Optional[int] = None, page_size: int = 1000, next_page_token: Optional[bytes] = None, rpc_metadata: Mapping[str, Union[str, bytes]] = {}, @@ -1362,15 +1364,36 @@ async def list_activities( ) -> ActivityExecutionAsyncIterator: """List activities. + This does not make a request until the first iteration is attempted. + Therefore any errors will not occur until then. + Args: - query: A Temporal visibility filter for activities. - page_size: Maximum number of results to return per page. - next_page_token: Token for getting the next page of results. - rpc_metadata: Headers used on the RPC call. - rpc_timeout: Optional RPC deadline to set for the RPC call. + query: A Temporal visibility list filter for activities. + limit: Maximum number of activities to return. If unset, all + activities are returned. Only applies if using the + returned :py:class:`ActivityExecutionAsyncIterator` + as an async iterator. + page_size: Maximum number of results for each page. + next_page_token: A previously obtained next page token if doing + pagination. Usually not needed as the iterator automatically + starts from the beginning. + rpc_metadata: Headers used on each RPC call. Keys here override + client-level RPC metadata keys. + rpc_timeout: Optional RPC deadline to set for each RPC call. + + Returns: + An async iterator that can be used with ``async for``. """ - # Issues a workflowservice ListActivityExecutions call - raise NotImplementedError + return self._impl.list_activities( + ListActivitiesInput( + query=query, + page_size=page_size, + next_page_token=next_page_token, + rpc_metadata=rpc_metadata, + rpc_timeout=rpc_timeout, + limit=limit, + ) + ) async def count_activities( self, @@ -1378,19 +1401,23 @@ async def count_activities( *, rpc_metadata: Mapping[str, Union[str, bytes]] = {}, rpc_timeout: Optional[timedelta] = None, - ) -> int: + ) -> ActivityExecutionCount: """Count activities matching the query. Args: query: A Temporal visibility filter for activities. - rpc_metadata: Headers used on the RPC call. + rpc_metadata: Headers used on the RPC call. Keys here override + client-level RPC metadata keys. rpc_timeout: Optional RPC deadline to set for the RPC call. Returns: Count of activities. """ - # Issues a workflowservice CountActivityExecutions call - raise NotImplementedError + return await self._impl.count_activities( + CountActivitiesInput( + query=query, rpc_metadata=rpc_metadata, rpc_timeout=rpc_timeout + ) + ) def get_activity_handle( self, @@ -2880,20 +2907,119 @@ async def workflow_handle(self) -> WorkflowHandle[SelfType, ReturnType]: class ActivityExecutionAsyncIterator: """Asynchronous iterator for activity execution values. - Returns either :py:class:`ActivityExecution` (for standalone activities) or - :py:class:`WorkflowActivityExecution` (for activities started by workflows). + Each item yielded by the iterator is either a :py:class:`ActivityExecution` (i.e. a standalone + activity) or a :py:class:`WorkflowActivityExecution` (i.e. an activity started by a workflow). + + You should typically use ``async for`` on this iterator and not call any of its methods. """ + # TODO(dan): do we want to use the "standalone" explanatory qualifier in docstrings? + + def __init__( + self, + client: Client, + input: ListActivitiesInput, + ) -> None: + """Create an asynchronous iterator for the given input. + + Users should not create this directly, but rather use + :py:meth:`Client.list_activities`. + """ + self._client = client + self._input = input + self._next_page_token = input.next_page_token + self._current_page: Optional[ + Sequence[Union[ActivityExecution, WorkflowActivityExecution]] + ] = None + self._current_page_index = 0 + self._limit = input.limit + self._yielded = 0 + + @property + def current_page_index(self) -> int: + """Index of the entry in the current page that will be returned from + the next :py:meth:`__anext__` call. + """ + return self._current_page_index + + @property + def current_page( + self, + ) -> Optional[Sequence[Union[ActivityExecution, WorkflowActivityExecution]]]: + """Current page, if it has been fetched yet.""" + return self._current_page + + @property + def next_page_token(self) -> Optional[bytes]: + """Token for the next page request if any.""" + return self._next_page_token + + async def fetch_next_page(self, *, page_size: Optional[int] = None) -> None: + """Fetch the next page of results. + + Args: + page_size: Override the page size this iterator was originally + created with. + """ + page_size = page_size or self._input.page_size + if self._limit is not None and self._limit - self._yielded < page_size: + page_size = self._limit - self._yielded + + resp = await self._client.workflow_service.list_activity_executions( + temporalio.api.workflowservice.v1.ListActivityExecutionsRequest( + namespace=self._client.namespace, + page_size=page_size, + next_page_token=self._next_page_token or b"", + query=self._input.query or "", + ), + retry=True, + metadata=self._input.rpc_metadata, + timeout=self._input.rpc_timeout, + ) + + self._current_page = [ + WorkflowActivityExecution._from_raw_info( + v, self._client.namespace, self._client.data_converter + ) + if v.workflow_id + else ActivityExecution._from_raw_info( + v, self._client.namespace, self._client.data_converter + ) + for v in resp.executions + ] + self._current_page_index = 0 + self._next_page_token = resp.next_page_token or None + def __aiter__(self) -> ActivityExecutionAsyncIterator: """Return self as the iterator.""" return self + # This is a direct copy of WorkflowExecutionAsyncIterator.__anext__ async def __anext__(self) -> Union[ActivityExecution, WorkflowActivityExecution]: - """Return the next execution on this iterator. - - Fetch next page if necessary. + """Get the next execution on this iterator, fetching next page if + necessary. """ - raise NotImplementedError + if self._limit is not None and self._yielded >= self._limit: + raise StopAsyncIteration + while True: + # No page? fetch and continue + if self._current_page is None: + await self.fetch_next_page() + continue + # No more left in page? + if self._current_page_index >= len(self._current_page): + # If there is a next page token, try to get another page and try + # again + if self._next_page_token is not None: + await self.fetch_next_page() + continue + # No more pages means we're done + raise StopAsyncIteration + # Get current, increment page index, and return + ret = self._current_page[self._current_page_index] + self._current_page_index += 1 + self._yielded += 1 + return ret # TODO: this is named ActivityListInfo in our draft proto PR @@ -2932,6 +3058,51 @@ class ActivityExecution: execution_duration: Optional[timedelta] """Duration from scheduled to close time, only populated if closed.""" + raw_info: temporalio.api.activity.v1.ActivityListInfo + """Underlying protobuf info.""" + + @classmethod + def _from_raw_info( + cls, + info: temporalio.api.activity.v1.ActivityListInfo, + namespace: str, + converter: temporalio.converter.DataConverter, + ) -> Self: + """Create from raw proto activity list info.""" + return cls( + activity_id=info.activity_id, + run_id=info.run_id, + activity_type=( + info.activity_type.name if info.HasField("activity_type") else "" + ), + scheduled_time=( + info.scheduled_time.ToDatetime().replace(tzinfo=timezone.utc) + if info.HasField("scheduled_time") + else datetime.min + ), + close_time=( + info.close_time.ToDatetime().replace(tzinfo=timezone.utc) + if info.HasField("close_time") + else None + ), + status=( + temporalio.common.ActivityExecutionStatus(info.status) + if info.status + else temporalio.common.ActivityExecutionStatus.RUNNING + ), + search_attributes=temporalio.converter.decode_search_attributes( + info.search_attributes + ), + task_queue=info.task_queue, + state_transition_count=info.state_transition_count, + execution_duration=( + info.execution_duration.ToTimedelta() + if info.HasField("execution_duration") + else None + ), + raw_info=info, + ) + @dataclass(frozen=True) class WorkflowActivityExecution: @@ -2961,6 +3132,61 @@ class WorkflowActivityExecution: execution_duration: Optional[timedelta] """Duration from scheduled to close time, only populated if closed.""" + raw_info: temporalio.api.activity.v1.ActivityListInfo + """Underlying protobuf info.""" + + @classmethod + def _from_raw_info( + cls, + info: temporalio.api.activity.v1.ActivityListInfo, + namespace: str, + converter: temporalio.converter.DataConverter, + ) -> Self: + """Create from raw proto activity list info.""" + # For workflow activities, we expect workflow_id to be set + return cls( + workflow_id=info.workflow_id, + workflow_run_id=None, # Not provided in list response + activity_id=info.activity_id, + activity_type=info.activity_type.name + if info.HasField("activity_type") + else "", + scheduled_time=( + info.scheduled_time.ToDatetime().replace(tzinfo=timezone.utc) + if info.HasField("scheduled_time") + else datetime.min + ), + close_time=( + info.close_time.ToDatetime().replace(tzinfo=timezone.utc) + if info.HasField("close_time") + else None + ), + task_queue=info.task_queue, + execution_duration=( + info.execution_duration.ToTimedelta() + if info.HasField("execution_duration") + else None + ), + raw_info=info, + ) + + +@dataclass(frozen=True) +class ActivityExecutionCount: + """Representation of a count from a count activities call.""" + + count: int + """Total count matching the filter, if any.""" + + @staticmethod + def _from_raw( + resp: temporalio.api.workflowservice.v1.CountActivityExecutionsResponse, + ) -> ActivityExecutionCount: + """Create from raw proto response.""" + return ActivityExecutionCount( + count=resp.count, + ) + @dataclass(frozen=True) class ActivityExecutionDescription: @@ -3041,6 +3267,101 @@ class ActivityExecutionDescription: raw_info: Any """Raw proto response.""" + @classmethod + async def _from_raw_info( + cls, + info: temporalio.api.activity.v1.ActivityExecutionInfo, + data_converter: temporalio.converter.DataConverter, + ) -> Self: + """Create from raw proto activity info.""" + return cls( + activity_id=info.activity_id, + run_id=info.run_id, + activity_type=( + info.activity_type.name if info.HasField("activity_type") else "" + ), + status=( + temporalio.common.ActivityExecutionStatus(info.status) + if info.status + else temporalio.common.ActivityExecutionStatus.RUNNING + ), + run_state=( + temporalio.common.PendingActivityState(info.run_state) + if info.run_state + else None + ), + heartbeat_details=( + await data_converter.decode(info.heartbeat_details.payloads) + if info.HasField("heartbeat_details") + else [] + ), + last_heartbeat_time=( + info.last_heartbeat_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("last_heartbeat_time") + else None + ), + last_started_time=( + info.last_started_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("last_started_time") + else None + ), + attempt=info.attempt, + maximum_attempts=info.maximum_attempts, + scheduled_time=( + info.scheduled_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("scheduled_time") + else datetime.min + ), + expiration_time=( + info.expiration_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("expiration_time") + else datetime.min + ), + last_failure=( + cast( + Optional[Exception], + await data_converter.decode_failure(info.last_failure), + ) + if info.HasField("last_failure") + else None + ), + last_worker_identity=info.last_worker_identity, + current_retry_interval=( + info.current_retry_interval.ToTimedelta() + if info.HasField("current_retry_interval") + else None + ), + last_attempt_complete_time=( + info.last_attempt_complete_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("last_attempt_complete_time") + else None + ), + next_attempt_schedule_time=( + info.next_attempt_schedule_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("next_attempt_schedule_time") + else None + ), + task_queue=( + info.activity_options.task_queue.name + if info.HasField("activity_options") + and info.activity_options.HasField("task_queue") + else "" + ), + paused=info.HasField("pause_info"), + input=( + await data_converter.decode(info.input.payloads) + if info.HasField("input") + else [] + ), + state_transition_count=info.state_transition_count, + search_attributes=temporalio.converter.decode_search_attributes( + info.search_attributes + ), + eager_execution_requested=info.eager_execution_requested, + canceled_reason=info.canceled_reason, + raw_info=info, + ) + @dataclass(frozen=True) class ActivityIDReference: @@ -3292,7 +3613,15 @@ async def cancel( rpc_metadata: Headers used on the RPC call. rpc_timeout: Optional RPC deadline to set for the RPC call. """ - raise NotImplementedError + await self._client._impl.cancel_activity( + CancelActivityInput( + activity_id=self._id, + run_id=self._run_id, + reason=reason, + rpc_metadata=rpc_metadata, + rpc_timeout=rpc_timeout, + ) + ) async def terminate( self, @@ -3312,7 +3641,15 @@ async def terminate( rpc_metadata: Headers used on the RPC call. rpc_timeout: Optional RPC deadline to set for the RPC call. """ - raise NotImplementedError + await self._client._impl.terminate_activity( + TerminateActivityInput( + activity_id=self._id, + run_id=self._run_id, + reason=reason, + rpc_metadata=rpc_metadata, + rpc_timeout=rpc_timeout, + ) + ) async def describe( self, @@ -3329,7 +3666,14 @@ async def describe( Returns: Activity execution description. """ - raise NotImplementedError + return await self._client._impl.describe_activity( + DescribeActivityInput( + activity_id=self._id, + run_id=self._run_id, + rpc_metadata=rpc_metadata, + rpc_timeout=rpc_timeout, + ) + ) # TODO: # update_options @@ -6053,6 +6397,59 @@ class TerminateWorkflowInput: rpc_timeout: Optional[timedelta] +@dataclass +class CancelActivityInput: + """Input for :py:meth:`OutboundInterceptor.cancel_activity`.""" + + activity_id: str + run_id: str + reason: Optional[str] + rpc_metadata: Mapping[str, Union[str, bytes]] + rpc_timeout: Optional[timedelta] + + +@dataclass +class TerminateActivityInput: + """Input for :py:meth:`OutboundInterceptor.terminate_activity`.""" + + activity_id: str + run_id: str + reason: Optional[str] + rpc_metadata: Mapping[str, Union[str, bytes]] + rpc_timeout: Optional[timedelta] + + +@dataclass +class DescribeActivityInput: + """Input for :py:meth:`OutboundInterceptor.describe_activity`.""" + + activity_id: str + run_id: str + rpc_metadata: Mapping[str, Union[str, bytes]] + rpc_timeout: Optional[timedelta] + + +@dataclass +class ListActivitiesInput: + """Input for :py:meth:`OutboundInterceptor.list_activities`.""" + + query: Optional[str] + page_size: int + next_page_token: Optional[bytes] + rpc_metadata: Mapping[str, Union[str, bytes]] + rpc_timeout: Optional[timedelta] + limit: Optional[int] + + +@dataclass +class CountActivitiesInput: + """Input for :py:meth:`OutboundInterceptor.count_activities`.""" + + query: Optional[str] + rpc_metadata: Mapping[str, Union[str, bytes]] + rpc_timeout: Optional[timedelta] + + @dataclass class StartWorkflowUpdateInput: """Input for :py:meth:`OutboundInterceptor.start_workflow_update`.""" @@ -6391,6 +6788,34 @@ async def terminate_workflow(self, input: TerminateWorkflowInput) -> None: """Called for every :py:meth:`WorkflowHandle.terminate` call.""" await self.next.terminate_workflow(input) + ### Activity calls + + async def cancel_activity(self, input: CancelActivityInput) -> None: + """Called for every :py:meth:`ActivityHandle.cancel` call.""" + await self.next.cancel_activity(input) + + async def terminate_activity(self, input: TerminateActivityInput) -> None: + """Called for every :py:meth:`ActivityHandle.terminate` call.""" + await self.next.terminate_activity(input) + + async def describe_activity( + self, input: DescribeActivityInput + ) -> ActivityExecutionDescription: + """Called for every :py:meth:`ActivityHandle.describe` call.""" + return await self.next.describe_activity(input) + + def list_activities( + self, input: ListActivitiesInput + ) -> ActivityExecutionAsyncIterator: + """Called for every :py:meth:`Client.list_activities` call.""" + return self.next.list_activities(input) + + async def count_activities( + self, input: CountActivitiesInput + ) -> ActivityExecutionCount: + """Called for every :py:meth:`Client.count_activities` call.""" + return await self.next.count_activities(input) + async def start_workflow_update( self, input: StartWorkflowUpdateInput ) -> WorkflowUpdateHandle[Any]: @@ -6842,6 +7267,82 @@ async def terminate_workflow(self, input: TerminateWorkflowInput) -> None: req, retry=True, metadata=input.rpc_metadata, timeout=input.rpc_timeout ) + async def cancel_activity(self, input: CancelActivityInput) -> None: + """Cancel a standalone activity.""" + await self._client.workflow_service.request_cancel_activity_execution( + temporalio.api.workflowservice.v1.RequestCancelActivityExecutionRequest( + namespace=self._client.namespace, + activity_id=input.activity_id, + run_id=input.run_id, + identity=self._client.identity, + request_id=str(uuid.uuid4()), + reason=input.reason or "", + ), + retry=True, + metadata=input.rpc_metadata, + timeout=input.rpc_timeout, + ) + + async def terminate_activity(self, input: TerminateActivityInput) -> None: + """Terminate a standalone activity.""" + await self._client.workflow_service.terminate_activity_execution( + temporalio.api.workflowservice.v1.TerminateActivityExecutionRequest( + namespace=self._client.namespace, + activity_id=input.activity_id, + run_id=input.run_id, + reason=input.reason or "", + identity=self._client.identity, + ), + retry=True, + metadata=input.rpc_metadata, + timeout=input.rpc_timeout, + ) + + async def describe_activity( + self, input: DescribeActivityInput + ) -> ActivityExecutionDescription: + """Describe a standalone activity.""" + resp = await self._client.workflow_service.describe_activity_execution( + temporalio.api.workflowservice.v1.DescribeActivityExecutionRequest( + namespace=self._client.namespace, + activity_id=input.activity_id, + run_id=input.run_id, + include_input=True, + ), + retry=True, + metadata=input.rpc_metadata, + timeout=input.rpc_timeout, + ) + return await ActivityExecutionDescription._from_raw_info( + resp.info, + self._client.data_converter.with_context( + WorkflowSerializationContext( + namespace=self._client.namespace, + workflow_id=input.activity_id, # Using activity_id as workflow_id for standalone activities + ) + ), + ) + + def list_activities( + self, input: ListActivitiesInput + ) -> ActivityExecutionAsyncIterator: + return ActivityExecutionAsyncIterator(self._client, input) + + async def count_activities( + self, input: CountActivitiesInput + ) -> ActivityExecutionCount: + return ActivityExecutionCount._from_raw( + await self._client.workflow_service.count_activity_executions( + temporalio.api.workflowservice.v1.CountActivityExecutionsRequest( + namespace=self._client.namespace, + query=input.query or "", + ), + retry=True, + metadata=input.rpc_metadata, + timeout=input.rpc_timeout, + ) + ) + async def start_workflow_update( self, input: StartWorkflowUpdateInput ) -> WorkflowUpdateHandle[Any]: From e6f6ca24b31f0d7504060164bbb0220e9d6d88df Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 08:05:06 -0400 Subject: [PATCH 07/25] alphabetical order --- temporalio/client.py | 275 +++++++++++++++++++++---------------------- 1 file changed, 137 insertions(+), 138 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index 5322b07c7..0e7a892ba 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -3031,35 +3031,35 @@ class ActivityExecution: activity_id: str """Activity ID.""" - run_id: str - """Run ID of the activity.""" - activity_type: str """Type name of the activity.""" - scheduled_time: datetime - """Time the activity was originally scheduled.""" - close_time: Optional[datetime] """Time the activity reached a terminal status, if closed.""" - status: temporalio.common.ActivityExecutionStatus - """Current status of the activity.""" + execution_duration: Optional[timedelta] + """Duration from scheduled to close time, only populated if closed.""" + + raw_info: temporalio.api.activity.v1.ActivityListInfo + """Underlying protobuf info.""" + + run_id: str + """Run ID of the activity.""" + + scheduled_time: datetime + """Time the activity was originally scheduled.""" search_attributes: temporalio.common.SearchAttributes """Search attributes from the start request.""" - task_queue: str - """Task queue the activity was scheduled on.""" - state_transition_count: int """Number of state transitions.""" - execution_duration: Optional[timedelta] - """Duration from scheduled to close time, only populated if closed.""" + status: temporalio.common.ActivityExecutionStatus + """Current status of the activity.""" - raw_info: temporalio.api.activity.v1.ActivityListInfo - """Underlying protobuf info.""" + task_queue: str + """Task queue the activity was scheduled on.""" @classmethod def _from_raw_info( @@ -3071,36 +3071,36 @@ def _from_raw_info( """Create from raw proto activity list info.""" return cls( activity_id=info.activity_id, - run_id=info.run_id, activity_type=( info.activity_type.name if info.HasField("activity_type") else "" ), + close_time=( + info.close_time.ToDatetime().replace(tzinfo=timezone.utc) + if info.HasField("close_time") + else None + ), + execution_duration=( + info.execution_duration.ToTimedelta() + if info.HasField("execution_duration") + else None + ), + raw_info=info, + run_id=info.run_id, scheduled_time=( info.scheduled_time.ToDatetime().replace(tzinfo=timezone.utc) if info.HasField("scheduled_time") else datetime.min ), - close_time=( - info.close_time.ToDatetime().replace(tzinfo=timezone.utc) - if info.HasField("close_time") - else None + search_attributes=temporalio.converter.decode_search_attributes( + info.search_attributes ), + state_transition_count=info.state_transition_count, status=( temporalio.common.ActivityExecutionStatus(info.status) if info.status else temporalio.common.ActivityExecutionStatus.RUNNING ), - search_attributes=temporalio.converter.decode_search_attributes( - info.search_attributes - ), task_queue=info.task_queue, - state_transition_count=info.state_transition_count, - execution_duration=( - info.execution_duration.ToTimedelta() - if info.HasField("execution_duration") - else None - ), - raw_info=info, ) @@ -3108,33 +3108,33 @@ def _from_raw_info( class WorkflowActivityExecution: """Info for a workflow activity execution from list response.""" - workflow_id: str - """ID of the workflow that started this activity.""" - - workflow_run_id: Optional[str] - """Run ID of the workflow that started this activity.""" - activity_id: str """Activity ID.""" activity_type: str """Type name of the activity.""" - scheduled_time: datetime - """Time the activity was originally scheduled.""" - close_time: Optional[datetime] """Time the activity reached a terminal status, if closed.""" - task_queue: str - """Task queue the activity was scheduled on.""" - execution_duration: Optional[timedelta] """Duration from scheduled to close time, only populated if closed.""" raw_info: temporalio.api.activity.v1.ActivityListInfo """Underlying protobuf info.""" + scheduled_time: datetime + """Time the activity was originally scheduled.""" + + task_queue: str + """Task queue the activity was scheduled on.""" + + workflow_id: str + """ID of the workflow that started this activity.""" + + workflow_run_id: Optional[str] + """Run ID of the workflow that started this activity.""" + @classmethod def _from_raw_info( cls, @@ -3145,29 +3145,29 @@ def _from_raw_info( """Create from raw proto activity list info.""" # For workflow activities, we expect workflow_id to be set return cls( - workflow_id=info.workflow_id, - workflow_run_id=None, # Not provided in list response activity_id=info.activity_id, activity_type=info.activity_type.name if info.HasField("activity_type") else "", - scheduled_time=( - info.scheduled_time.ToDatetime().replace(tzinfo=timezone.utc) - if info.HasField("scheduled_time") - else datetime.min - ), close_time=( info.close_time.ToDatetime().replace(tzinfo=timezone.utc) if info.HasField("close_time") else None ), - task_queue=info.task_queue, execution_duration=( info.execution_duration.ToTimedelta() if info.HasField("execution_duration") else None ), raw_info=info, + scheduled_time=( + info.scheduled_time.ToDatetime().replace(tzinfo=timezone.utc) + if info.HasField("scheduled_time") + else datetime.min + ), + task_queue=info.task_queue, + workflow_id=info.workflow_id, + workflow_run_id=None, # Not provided in list response ) @@ -3195,77 +3195,76 @@ class ActivityExecutionDescription: activity_id: str """Activity ID.""" - run_id: str - """Run ID of the activity.""" - activity_type: str """Type name of the activity.""" - status: temporalio.common.ActivityExecutionStatus - """Current status of the activity.""" - - run_state: Optional[temporalio.common.PendingActivityState] - """More detailed breakdown if status is RUNNING.""" - - heartbeat_details: Sequence[Any] - """Details from the last heartbeat.""" - - last_heartbeat_time: Optional[datetime] - """Time of the last heartbeat.""" - - last_started_time: Optional[datetime] - """Time the last attempt was started.""" - attempt: int """Current attempt number.""" - maximum_attempts: int - """Maximum number of attempts allowed.""" + canceled_reason: Optional[str] + """Reason for cancellation, if cancel was requested.""" - scheduled_time: datetime - """Time the activity was originally scheduled.""" + current_retry_interval: Optional[timedelta] + """Time until the next retry, if applicable.""" + eager_execution_requested: bool + """Whether eager execution was requested.""" expiration_time: datetime """Scheduled time plus schedule_to_close_timeout.""" + heartbeat_details: Sequence[Any] + """Details from the last heartbeat.""" + + input: Sequence[Any] + """Serialized activity input.""" + + last_attempt_complete_time: Optional[datetime] + """Time when the last attempt completed.""" + last_failure: Optional[Exception] """Failure from the last failed attempt, if any.""" + last_heartbeat_time: Optional[datetime] + """Time of the last heartbeat.""" + + last_started_time: Optional[datetime] + """Time the last attempt was started.""" + last_worker_identity: str """Identity of the last worker that processed the activity.""" - current_retry_interval: Optional[timedelta] - """Time until the next retry, if applicable.""" - - last_attempt_complete_time: Optional[datetime] - """Time when the last attempt completed.""" + maximum_attempts: int + """Maximum number of attempts allowed.""" next_attempt_schedule_time: Optional[datetime] """Time when the next attempt will be scheduled.""" - task_queue: str - """Task queue the activity is scheduled on.""" - paused: bool """Whether the activity is paused.""" - input: Sequence[Any] - """Serialized activity input.""" + raw_info: Any + """Raw proto response.""" - state_transition_count: int - """Number of state transitions.""" + run_id: str + """Run ID of the activity.""" + + run_state: Optional[temporalio.common.PendingActivityState] + """More detailed breakdown if status is RUNNING.""" + + scheduled_time: datetime + """Time the activity was originally scheduled.""" search_attributes: temporalio.common.SearchAttributes """Search attributes.""" - eager_execution_requested: bool - """Whether eager execution was requested.""" + state_transition_count: int + """Number of state transitions.""" - canceled_reason: Optional[str] - """Reason for cancellation, if cancel was requested.""" + status: temporalio.common.ActivityExecutionStatus + """Current status of the activity.""" - raw_info: Any - """Raw proto response.""" + task_queue: str + """Task queue the activity is scheduled on.""" @classmethod async def _from_raw_info( @@ -3276,47 +3275,37 @@ async def _from_raw_info( """Create from raw proto activity info.""" return cls( activity_id=info.activity_id, - run_id=info.run_id, activity_type=( info.activity_type.name if info.HasField("activity_type") else "" ), - status=( - temporalio.common.ActivityExecutionStatus(info.status) - if info.status - else temporalio.common.ActivityExecutionStatus.RUNNING - ), - run_state=( - temporalio.common.PendingActivityState(info.run_state) - if info.run_state + attempt=info.attempt, + canceled_reason=info.canceled_reason, + current_retry_interval=( + info.current_retry_interval.ToTimedelta() + if info.HasField("current_retry_interval") else None ), + eager_execution_requested=info.eager_execution_requested, + expiration_time=( + info.expiration_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("expiration_time") + else datetime.min + ), heartbeat_details=( await data_converter.decode(info.heartbeat_details.payloads) if info.HasField("heartbeat_details") else [] ), - last_heartbeat_time=( - info.last_heartbeat_time.ToDatetime(tzinfo=timezone.utc) - if info.HasField("last_heartbeat_time") - else None + input=( + await data_converter.decode(info.input.payloads) + if info.HasField("input") + else [] ), - last_started_time=( - info.last_started_time.ToDatetime(tzinfo=timezone.utc) - if info.HasField("last_started_time") + last_attempt_complete_time=( + info.last_attempt_complete_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("last_attempt_complete_time") else None ), - attempt=info.attempt, - maximum_attempts=info.maximum_attempts, - scheduled_time=( - info.scheduled_time.ToDatetime(tzinfo=timezone.utc) - if info.HasField("scheduled_time") - else datetime.min - ), - expiration_time=( - info.expiration_time.ToDatetime(tzinfo=timezone.utc) - if info.HasField("expiration_time") - else datetime.min - ), last_failure=( cast( Optional[Exception], @@ -3325,41 +3314,51 @@ async def _from_raw_info( if info.HasField("last_failure") else None ), - last_worker_identity=info.last_worker_identity, - current_retry_interval=( - info.current_retry_interval.ToTimedelta() - if info.HasField("current_retry_interval") + last_heartbeat_time=( + info.last_heartbeat_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("last_heartbeat_time") else None ), - last_attempt_complete_time=( - info.last_attempt_complete_time.ToDatetime(tzinfo=timezone.utc) - if info.HasField("last_attempt_complete_time") + last_started_time=( + info.last_started_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("last_started_time") else None ), + last_worker_identity=info.last_worker_identity, + maximum_attempts=info.maximum_attempts, next_attempt_schedule_time=( info.next_attempt_schedule_time.ToDatetime(tzinfo=timezone.utc) if info.HasField("next_attempt_schedule_time") else None ), + paused=info.HasField("pause_info"), + raw_info=info, + run_id=info.run_id, + run_state=( + temporalio.common.PendingActivityState(info.run_state) + if info.run_state + else None + ), + scheduled_time=( + info.scheduled_time.ToDatetime(tzinfo=timezone.utc) + if info.HasField("scheduled_time") + else datetime.min + ), + search_attributes=temporalio.converter.decode_search_attributes( + info.search_attributes + ), + state_transition_count=info.state_transition_count, + status=( + temporalio.common.ActivityExecutionStatus(info.status) + if info.status + else temporalio.common.ActivityExecutionStatus.RUNNING + ), task_queue=( info.activity_options.task_queue.name if info.HasField("activity_options") and info.activity_options.HasField("task_queue") else "" ), - paused=info.HasField("pause_info"), - input=( - await data_converter.decode(info.input.payloads) - if info.HasField("input") - else [] - ), - state_transition_count=info.state_transition_count, - search_attributes=temporalio.converter.decode_search_attributes( - info.search_attributes - ), - eager_execution_requested=info.eager_execution_requested, - canceled_reason=info.canceled_reason, - raw_info=info, ) From 51f7da8a87e613dc09bae06f4d4d44130143aabd Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 09:26:45 -0400 Subject: [PATCH 08/25] ActivityHandle.result --- temporalio/client.py | 91 ++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 84 insertions(+), 7 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index 0e7a892ba..e1727335d 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -3526,13 +3526,21 @@ def __init__( id: str, *, run_id: str, + result_type: Optional[Type] = None, data_converter_override: Optional[DataConverter] = None, ) -> None: """Create activity handle.""" self._client = client self._id = id self._run_id = run_id + self._result_type = result_type self._data_converter_override = data_converter_override + self._known_outcome: Optional[ + Union[ + temporalio.api.common.v1.Payloads, + temporalio.api.failure.v1.Failure, + ] + ] = None @property def id(self) -> str: @@ -3565,6 +3573,7 @@ def with_context(self, context: SerializationContext) -> Self: self._client, id=self._id, run_id=self._run_id, + result_type=self._result_type, data_converter_override=data_converter, ) @@ -3576,21 +3585,89 @@ async def result( ) -> ReturnType: """Wait for result of the activity. + The result may already be known if this method has been called before, + in which case no network call is made. Otherwise the result will be + polled for until it is available. + Args: rpc_metadata: Headers used on the RPC call. Keys here override client-level RPC metadata keys. - rpc_timeout: Optional RPC deadline to set for each RPC call. Note, - this is the timeout for each history RPC call not this overall - function. + rpc_timeout: Optional RPC deadline to set for each RPC call. Note: + this is the timeout for each RPC call while polling, not a + timeout for the function as a whole. If an individual RPC + times out, it will be retried until the result is available. Returns: The result of the activity. Raises: - :py:class:`ActivityFailureError`: If the activity completed with a failure. + ActivityFailureError: If the activity completed with a failure. + RPCError: Activity result could not be fetched for some reason. """ - # Repeatedly issues workflowservice GetActivityResult long-polls. - raise NotImplementedError + await self._poll_until_outcome( + rpc_metadata=rpc_metadata, rpc_timeout=rpc_timeout + ) + data_converter = self._data_converter_override or self._client.data_converter + assert self._known_outcome + if isinstance(self._known_outcome, temporalio.api.failure.v1.Failure): + raise ActivityFailedError( + cause=await data_converter.decode_failure(self._known_outcome), + ) + payloads = self._known_outcome + if not payloads.payloads: + # E.g. a void workflow function in another language may not set any payloads. + return None # type: ignore + type_hints = [self._result_type] if self._result_type else None + results = await data_converter.decode(payloads.payloads, type_hints) + if not results: + # Following workflow/update/query result processing. Technically not necessary since + # from_payloads is documented to always return non-empty + return None # type: ignore + elif len(results) > 1: + warnings.warn(f"Expected single activity result, got {len(results)}") + return results[0] + + async def _poll_until_outcome( + self, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> None: + """Poll for activity result until it's available.""" + if self._known_outcome: + return + + req = temporalio.api.workflowservice.v1.GetActivityExecutionResultRequest( + namespace=self._client.namespace, + activity_id=self._id, + run_id=self._run_id, + wait=True, # Enable long polling + ) + + # Continue polling as long as we have no outcome + while True: + try: + res = await self._client.workflow_service.get_activity_execution_result( + req, + retry=True, + metadata=rpc_metadata, + timeout=rpc_timeout, + ) + if res.HasField("result"): + self._known_outcome = res.result + return + elif res.HasField("failure"): + self._known_outcome = res.failure + return + except RPCError as err: + if err.status == RPCStatusCode.DEADLINE_EXCEEDED: + # Deadline exceeded is expected with long polling; retry + continue + elif err.status == RPCStatusCode.CANCELLED: + raise asyncio.CancelledError() from err + else: + raise + except asyncio.CancelledError: + raise async def cancel( self, @@ -6225,7 +6302,7 @@ def __init__(self) -> None: super().__init__("Timeout or cancellation waiting for update") -class ActivityFailureError(temporalio.exceptions.TemporalError): +class ActivityFailedError(temporalio.exceptions.TemporalError): """Error that occurs when a standalone activity is unsuccessful.""" def __init__(self, *, cause: BaseException) -> None: From 7781c2a12a59751ca105b23d7a5f6900797f7373 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 12:27:16 -0400 Subject: [PATCH 09/25] Implement start_activity --- temporalio/activity.py | 14 ++ temporalio/client.py | 259 ++++++++++++++++++++++-- temporalio/common.py | 36 ++++ temporalio/converter.py | 15 +- temporalio/worker/_activity.py | 2 + temporalio/worker/_workflow_instance.py | 3 + tests/test_serialization_context.py | 9 +- 7 files changed, 315 insertions(+), 23 deletions(-) diff --git a/temporalio/activity.py b/temporalio/activity.py index d726b9ef2..68072a8bb 100644 --- a/temporalio/activity.py +++ b/temporalio/activity.py @@ -588,6 +588,20 @@ def must_from_callable(fn: Callable) -> _Definition: f"Activity {fn_name} missing attributes, was it decorated with @activity.defn?" ) + @classmethod + def get_name_and_result_type( + cls, name_or_run_fn: Union[str, Callable[..., Any]] + ) -> Tuple[str, Optional[Type]]: + if isinstance(name_or_run_fn, str): + return name_or_run_fn, None + elif callable(name_or_run_fn): + defn = cls.must_from_callable(name_or_run_fn) + if not defn.name: + raise ValueError(f"Activity {name_or_run_fn} definition has no name") + return defn.name, defn.ret_type + else: + raise TypeError("Activity must be a string or callable") + @staticmethod def _apply_to_callable( fn: Callable, diff --git a/temporalio/client.py b/temporalio/client.py index e1727335d..b2a2042b2 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -43,6 +43,7 @@ from google.protobuf.internal.containers import MessageMap from typing_extensions import Concatenate, Required, Self, TypedDict +import temporalio.activity import temporalio.api.activity.v1 import temporalio.api.common.v1 import temporalio.api.enums.v1 @@ -65,6 +66,7 @@ import temporalio.workflow from temporalio.activity import ActivityCancellationDetails from temporalio.converter import ( + ActivitySerializationContext, DataConverter, SerializationContext, WithSerializationContext, @@ -1287,18 +1289,19 @@ async def count_workflows( # - TODO: Support sync and async activity functions async def start_activity( self, - activity: Callable[..., ReturnType], + activity: Union[str, Callable[..., Awaitable[ReturnType]]], *, - args: Sequence[Any], + args: Sequence[Any] = [], id: str, task_queue: str, + result_type: Optional[Type] = None, # Either schedule_to_close_timeout or start_to_close_timeout must be present schedule_to_close_timeout: Optional[timedelta] = None, start_to_close_timeout: Optional[timedelta] = None, schedule_to_start_timeout: Optional[timedelta] = None, heartbeat_timeout: Optional[timedelta] = None, - id_reuse_policy: temporalio.common.WorkflowIDReusePolicy = temporalio.common.WorkflowIDReusePolicy.ALLOW_DUPLICATE, - id_conflict_policy: temporalio.common.WorkflowIDConflictPolicy = temporalio.common.WorkflowIDConflictPolicy.FAIL, + id_reuse_policy: temporalio.common.IdReusePolicy = temporalio.common.IdReusePolicy.ALLOW_DUPLICATE, + id_conflict_policy: temporalio.common.IdConflictPolicy = temporalio.common.IdConflictPolicy.FAIL, retry_policy: Optional[temporalio.common.RetryPolicy] = None, search_attributes: Optional[ Union[ @@ -1315,41 +1318,118 @@ async def start_activity( """Start an activity and return its handle. Args: - activity: The activity function to execute. + activity: String name or callable activity function to execute. args: Arguments to pass to the activity. id: Unique identifier for the activity. Required. task_queue: Task queue to send the activity to. + result_type: For string name activities, optional type to deserialize result into. schedule_to_close_timeout: Total time allowed for the activity from schedule to completion. start_to_close_timeout: Time allowed for a single execution attempt. schedule_to_start_timeout: Time allowed for the activity to sit in the task queue. heartbeat_timeout: Time between heartbeats before the activity is considered failed. id_reuse_policy: How to handle reusing activity IDs from closed activities. + Default is ALLOW_DUPLICATE. id_conflict_policy: How to handle activity ID conflicts with running activities. + Default is FAIL. retry_policy: Retry policy for the activity. - search_attributes: Search attributes to attach to the activity. - static_summary: A single-line fixed summary for this workflow execution that may appear + search_attributes: Search attributes for the activity. + static_summary: A single-line fixed summary for this activity that may appear in the UI/CLI. This can be in single-line Temporal markdown format. - static_details: General fixed details for this workflow execution that may appear in - UI/CLI. This can be in Temporal markdown format and can span multiple lines. This is - a fixed value on the workflow that cannot be updated. For details that can be - updated, use :py:meth:`temporalio.workflow.get_current_details` within the workflow. - priority: Priority metadata. + static_details: General fixed details for this activity that may appear in + UI/CLI. This can be in Temporal markdown format and can span multiple lines. + priority: Priority of the activity execution. rpc_metadata: Headers used on the RPC call. rpc_timeout: Optional RPC deadline to set for the RPC call. Returns: A handle to the started activity. """ - # Issues workflowservice StartActivityExecution - raise NotImplementedError + name, result_type_from_type_annotation = ( + temporalio.activity._Definition.get_name_and_result_type(activity) + ) + return await self._impl.start_activity( + StartActivityInput( + activity_type=name, + args=args, + id=id, + task_queue=task_queue, + ret_type=result_type or result_type_from_type_annotation, + schedule_to_close_timeout=schedule_to_close_timeout, + start_to_close_timeout=start_to_close_timeout, + schedule_to_start_timeout=schedule_to_start_timeout, + heartbeat_timeout=heartbeat_timeout, + id_reuse_policy=id_reuse_policy, + id_conflict_policy=id_conflict_policy, + retry_policy=retry_policy, + search_attributes=search_attributes, + static_summary=static_summary, + static_details=static_details, + headers={}, + rpc_metadata=rpc_metadata, + rpc_timeout=rpc_timeout, + priority=priority, + ) + ) - # Same parameters as start_activity - # (*args **kwargs is just temporary to avoid duplicating parameter lists while they're being designed) - async def execute_activity(self, *args, **kwargs) -> ReturnType: - """ - Start an activity, wait for it to complete, and return its result. + async def execute_activity( + self, + activity: Union[str, Callable[..., Awaitable[ReturnType]]], + *, + args: Sequence[Any] = [], + id: str, + task_queue: str, + result_type: Optional[Type] = None, + # Either schedule_to_close_timeout or start_to_close_timeout must be present + schedule_to_close_timeout: Optional[timedelta] = None, + start_to_close_timeout: Optional[timedelta] = None, + schedule_to_start_timeout: Optional[timedelta] = None, + heartbeat_timeout: Optional[timedelta] = None, + id_reuse_policy: temporalio.common.IdReusePolicy = temporalio.common.IdReusePolicy.ALLOW_DUPLICATE, + id_conflict_policy: temporalio.common.IdConflictPolicy = temporalio.common.IdConflictPolicy.FAIL, + retry_policy: Optional[temporalio.common.RetryPolicy] = None, + search_attributes: Optional[ + Union[ + temporalio.common.SearchAttributes, + temporalio.common.TypedSearchAttributes, + ] + ] = None, + static_summary: Optional[str] = None, + static_details: Optional[str] = None, + priority: temporalio.common.Priority = temporalio.common.Priority.default, + rpc_metadata: Mapping[str, Union[str, bytes]] = {}, + rpc_timeout: Optional[timedelta] = None, + ) -> ReturnType: + """Start an activity, wait for it to complete, and return its result. + + This is a convenience method that combines :py:meth:`start_activity` and + :py:meth:`ActivityHandle.result`. + + Returns: + The result of the activity. + + Raises: + ActivityFailedError: If the activity completed with a failure. """ - handle = await self.start_activity(*args, **kwargs) + handle = await self.start_activity( + activity, + args=args, + id=id, + task_queue=task_queue, + result_type=result_type, + schedule_to_close_timeout=schedule_to_close_timeout, + start_to_close_timeout=start_to_close_timeout, + schedule_to_start_timeout=schedule_to_start_timeout, + heartbeat_timeout=heartbeat_timeout, + id_reuse_policy=id_reuse_policy, + id_conflict_policy=id_conflict_policy, + retry_policy=retry_policy, + search_attributes=search_attributes, + static_summary=static_summary, + static_details=static_details, + priority=priority, + rpc_metadata=rpc_metadata, + rpc_timeout=rpc_timeout, + ) return await handle.result() def list_activities( @@ -1456,6 +1536,7 @@ def get_async_activity_handle( def get_async_activity_handle(self, *, task_token: bytes) -> AsyncActivityHandle: pass + # TODO(dan): add typed API get_async_activity_handle_for? def get_async_activity_handle( self, *, @@ -6473,6 +6554,36 @@ class TerminateWorkflowInput: rpc_timeout: Optional[timedelta] +@dataclass +class StartActivityInput: + """Input for :py:meth:`OutboundInterceptor.start_activity`.""" + + activity_type: str + args: Sequence[Any] + id: str + task_queue: str + ret_type: Optional[Type] + schedule_to_close_timeout: Optional[timedelta] + start_to_close_timeout: Optional[timedelta] + schedule_to_start_timeout: Optional[timedelta] + heartbeat_timeout: Optional[timedelta] + id_reuse_policy: temporalio.common.IdReusePolicy + id_conflict_policy: temporalio.common.IdConflictPolicy + retry_policy: Optional[temporalio.common.RetryPolicy] + priority: temporalio.common.Priority + search_attributes: Optional[ + Union[ + temporalio.common.SearchAttributes, + temporalio.common.TypedSearchAttributes, + ] + ] + static_summary: Optional[str] + static_details: Optional[str] + headers: Mapping[str, temporalio.api.common.v1.Payload] + rpc_metadata: Mapping[str, Union[str, bytes]] + rpc_timeout: Optional[timedelta] + + @dataclass class CancelActivityInput: """Input for :py:meth:`OutboundInterceptor.cancel_activity`.""" @@ -6866,6 +6977,10 @@ async def terminate_workflow(self, input: TerminateWorkflowInput) -> None: ### Activity calls + async def start_activity(self, input: StartActivityInput) -> ActivityHandle[Any]: + """Called for every :py:meth:`Client.start_activity` call.""" + return await self.next.start_activity(input) + async def cancel_activity(self, input: CancelActivityInput) -> None: """Called for every :py:meth:`ActivityHandle.cancel` call.""" await self.next.cancel_activity(input) @@ -7343,6 +7458,110 @@ async def terminate_workflow(self, input: TerminateWorkflowInput) -> None: req, retry=True, metadata=input.rpc_metadata, timeout=input.rpc_timeout ) + async def start_activity(self, input: StartActivityInput) -> ActivityHandle[Any]: + """Start an activity and return a handle to it.""" + if not (input.start_to_close_timeout or input.schedule_to_close_timeout): + raise ValueError( + "Activity must have start_to_close_timeout or schedule_to_close_timeout" + ) + req = await self._build_start_activity_execution_request(input) + + # TODO(dan): any counterpart of WorkflowExecutionAlreadyStartedFailure? + # If RPCError with err.status == RPCStatusCode.ALREADY_EXISTS + + resp = await self._client.workflow_service.start_activity_execution( + req, + retry=True, + metadata=input.rpc_metadata, + timeout=input.rpc_timeout, + ) + return ActivityHandle( + self._client, + id=input.id, + run_id=resp.run_id, + result_type=input.ret_type, + ) + + async def _build_start_activity_execution_request( + self, input: StartActivityInput + ) -> temporalio.api.workflowservice.v1.StartActivityExecutionRequest: + """Build StartActivityExecutionRequest from input.""" + data_converter = self._client.data_converter.with_context( + ActivitySerializationContext( + namespace=self._client.namespace, + activity_id=input.id, + activity_type=input.activity_type, + activity_task_queue=input.task_queue, + is_local=False, + workflow_id=None, + workflow_type=None, + ) + ) + + req = temporalio.api.workflowservice.v1.StartActivityExecutionRequest( + namespace=self._client.namespace, + identity=self._client.identity, + activity_id=input.id, + activity_type=temporalio.api.common.v1.ActivityType( + name=input.activity_type + ), + id_reuse_policy=cast( + "temporalio.api.enums.v1.IdReusePolicy.ValueType", + int(input.id_reuse_policy), + ), + id_conflict_policy=cast( + "temporalio.api.enums.v1.IdConflictPolicy.ValueType", + int(input.id_conflict_policy), + ), + ) + + # Build ActivityOptions + options = temporalio.api.activity.v1.ActivityOptions( + task_queue=temporalio.api.taskqueue.v1.TaskQueue(name=input.task_queue), + ) + if input.schedule_to_close_timeout is not None: + options.schedule_to_close_timeout.FromTimedelta( + input.schedule_to_close_timeout + ) + if input.start_to_close_timeout is not None: + options.start_to_close_timeout.FromTimedelta(input.start_to_close_timeout) + if input.schedule_to_start_timeout is not None: + options.schedule_to_start_timeout.FromTimedelta( + input.schedule_to_start_timeout + ) + if input.heartbeat_timeout is not None: + options.heartbeat_timeout.FromTimedelta(input.heartbeat_timeout) + if input.retry_policy is not None: + input.retry_policy.apply_to_proto(options.retry_policy) + req.options.CopyFrom(options) + + # Set input payloads + if input.args: + req.input.payloads.extend(await data_converter.encode(input.args)) + + # Set search attributes + if input.search_attributes is not None: + temporalio.converter.encode_search_attributes( + input.search_attributes, req.search_attributes + ) + + # Set user metadata + metadata = await _encode_user_metadata( + data_converter, input.static_summary, input.static_details + ) + if metadata is not None: + req.user_metadata.CopyFrom(metadata) + + # Set headers + if input.headers is not None: + await self._apply_headers(input.headers, req.header.fields) + + # Set priority + if input.priority is not None: + req.priority.CopyFrom(input.priority._to_proto()) + + return req + async def cancel_activity(self, input: CancelActivityInput) -> None: """Cancel a standalone activity.""" await self._client.workflow_service.request_cancel_activity_execution( diff --git a/temporalio/common.py b/temporalio/common.py index 9664e9857..ac201cab5 100644 --- a/temporalio/common.py +++ b/temporalio/common.py @@ -154,6 +154,42 @@ class WorkflowIDConflictPolicy(IntEnum): ) +class IdReusePolicy(IntEnum): + """How already-closed entity IDs are handled on start. + + See :py:class:`temporalio.api.enums.v1.IdReusePolicy`. + """ + + UNSPECIFIED = int(temporalio.api.enums.v1.IdReusePolicy.ID_REUSE_POLICY_UNSPECIFIED) + ALLOW_DUPLICATE = int( + temporalio.api.enums.v1.IdReusePolicy.ID_REUSE_POLICY_ALLOW_DUPLICATE + ) + ALLOW_DUPLICATE_FAILED_ONLY = int( + temporalio.api.enums.v1.IdReusePolicy.ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY + ) + REJECT_DUPLICATE = int( + temporalio.api.enums.v1.IdReusePolicy.ID_REUSE_POLICY_REJECT_DUPLICATE + ) + + +class IdConflictPolicy(IntEnum): + """How already-running entity IDs are handled on start. + + See :py:class:`temporalio.api.enums.v1.IdConflictPolicy`. + """ + + UNSPECIFIED = int( + temporalio.api.enums.v1.IdConflictPolicy.ID_CONFLICT_POLICY_UNSPECIFIED + ) + FAIL = int(temporalio.api.enums.v1.IdConflictPolicy.ID_CONFLICT_POLICY_FAIL) + USE_EXISTING = int( + temporalio.api.enums.v1.IdConflictPolicy.ID_CONFLICT_POLICY_USE_EXISTING + ) + TERMINATE_EXISTING = int( + temporalio.api.enums.v1.IdConflictPolicy.ID_CONFLICT_POLICY_TERMINATE_EXISTING + ) + + class ActivityExecutionStatus(IntEnum): """Status of a standalone activity execution. diff --git a/temporalio/converter.py b/temporalio/converter.py index 29eb35566..1a9b38d2b 100644 --- a/temporalio/converter.py +++ b/temporalio/converter.py @@ -97,7 +97,8 @@ class BaseWorkflowSerializationContext(SerializationContext): """Base serialization context shared by workflow and activity serialization contexts.""" namespace: str - workflow_id: str + workflow_id: Optional[str] + """Workflow ID.""" @dataclass(frozen=True) @@ -134,10 +135,20 @@ class ActivitySerializationContext(BaseWorkflowSerializationContext): is_local: Whether the activity is a local activity. """ - workflow_type: str + workflow_type: Optional[str] + """Workflow type.""" + activity_type: str + """Activity type.""" + + activity_id: Optional[str] + """Activity ID.""" + activity_task_queue: str + """Activity task queue.""" + is_local: bool + """Whether the activity is a local activity.""" class WithSerializationContext(ABC): diff --git a/temporalio/worker/_activity.py b/temporalio/worker/_activity.py index 44bfb6910..9391f99b2 100644 --- a/temporalio/worker/_activity.py +++ b/temporalio/worker/_activity.py @@ -259,6 +259,7 @@ async def _heartbeat_async( workflow_id=activity.info.workflow_id, workflow_type=activity.info.workflow_type, activity_type=activity.info.activity_type, + activity_id=activity.info.activity_id, activity_task_queue=self._task_queue, is_local=activity.info.is_local, ) @@ -311,6 +312,7 @@ async def _handle_start_activity_task( workflow_id=start.workflow_execution.workflow_id, workflow_type=start.workflow_type, activity_type=start.activity_type, + activity_id=start.activity_id, activity_task_queue=self._task_queue, is_local=start.is_local, ) diff --git a/temporalio/worker/_workflow_instance.py b/temporalio/worker/_workflow_instance.py index 44eb443ff..79059715a 100644 --- a/temporalio/worker/_workflow_instance.py +++ b/temporalio/worker/_workflow_instance.py @@ -791,6 +791,7 @@ def _apply_resolve_activity( workflow_id=self._info.workflow_id, workflow_type=self._info.workflow_type, activity_type=handle._input.activity, + activity_id=handle._input.activity_id, activity_task_queue=( handle._input.task_queue or self._info.task_queue if isinstance(handle._input, StartActivityInput) @@ -2130,6 +2131,7 @@ def get_serialization_context( workflow_id=self._info.workflow_id, workflow_type=self._info.workflow_type, activity_type=activity_handle._input.activity, + activity_id=activity_handle._input.activity_id, activity_task_queue=( activity_handle._input.task_queue if isinstance(activity_handle._input, StartActivityInput) @@ -2924,6 +2926,7 @@ def __init__( workflow_id=self._instance._info.workflow_id, workflow_type=self._instance._info.workflow_type, activity_type=self._input.activity, + activity_id=self._input.activity_id, activity_task_queue=( self._input.task_queue or self._instance._info.task_queue if isinstance(self._input, StartActivityInput) diff --git a/tests/test_serialization_context.py b/tests/test_serialization_context.py index ee7be8684..9475e29ad 100644 --- a/tests/test_serialization_context.py +++ b/tests/test_serialization_context.py @@ -231,6 +231,7 @@ async def test_payload_conversion_calls_follow_expected_sequence_and_contexts( workflow_id=workflow_id, workflow_type=PayloadConversionWorkflow.__name__, activity_type=passthrough_activity.__name__, + activity_id=None, activity_task_queue=task_queue, is_local=False, ) @@ -370,6 +371,7 @@ async def test_heartbeat_details_payload_conversion(client: Client): workflow_id=workflow_id, workflow_type=HeartbeatDetailsSerializationContextTestWorkflow.__name__, activity_type=activity_with_heartbeat_details.__name__, + activity_id=None, activity_task_queue=task_queue, is_local=False, ) @@ -459,6 +461,7 @@ async def test_local_activity_payload_conversion(client: Client): workflow_id=workflow_id, workflow_type=LocalActivityWorkflow.__name__, activity_type=local_activity.__name__, + activity_id=None, activity_task_queue=task_queue, is_local=True, ) @@ -572,6 +575,7 @@ async def test_async_activity_completion_payload_conversion( workflow_id=workflow_id, workflow_type=AsyncActivityCompletionSerializationContextTestWorkflow.__name__, activity_type=async_activity.__name__, + activity_id=None, activity_task_queue=task_queue, is_local=False, ) @@ -644,6 +648,7 @@ def test_subclassed_async_activity_handle(client: Client): workflow_id="workflow-id", workflow_type="workflow-type", activity_type="activity-type", + activity_id=None, activity_task_queue="activity-task-queue", is_local=False, ) @@ -1062,7 +1067,7 @@ async def run(self) -> Never: raise Exception("Unreachable") -test_traces: dict[str, list[TraceItem]] = defaultdict(list) +test_traces: dict[Optional[str], list[TraceItem]] = defaultdict(list) class FailureConverterWithContext(DefaultFailureConverter, WithSerializationContext): @@ -1154,6 +1159,7 @@ async def test_failure_converter_with_context(client: Client): workflow_id=workflow_id, workflow_type=FailureConverterTestWorkflow.__name__, activity_type=failing_activity.__name__, + activity_id=None, activity_task_queue=task_queue, is_local=False, ) @@ -1360,6 +1366,7 @@ async def test_local_activity_codec_with_context(client: Client): workflow_id=workflow_id, workflow_type=LocalActivityCodecTestWorkflow.__name__, activity_type=codec_test_local_activity.__name__, + activity_id=None, activity_task_queue=task_queue, is_local=True, ) From ecbaed9f06dda3a4f42983c796bc103e2b49fb51 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 8 Oct 2025 17:18:04 -0400 Subject: [PATCH 10/25] Update sdk-core --- temporalio/bridge/sdk-core | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/temporalio/bridge/sdk-core b/temporalio/bridge/sdk-core index ca1de1067..52d6adc58 160000 --- a/temporalio/bridge/sdk-core +++ b/temporalio/bridge/sdk-core @@ -1 +1 @@ -Subproject commit ca1de1067cfdae37a3b3bb2a15cc96b147e6d437 +Subproject commit 52d6adc5855e0d3100f73b56c39206abbd79b736 From 4a4a731fe46976202f6f7f302b5d1cb0da46ddbd Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Thu, 9 Oct 2025 11:29:21 -0400 Subject: [PATCH 11/25] Server isn't returning RunID yet --- tests/test_activity.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 00474c3bc..1f96f78bb 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -24,10 +24,8 @@ async def test_describe_activity(client: Client): ) desc = await activity_handle.describe() assert desc.activity_id == activity_id - assert desc.run_id == activity_handle.run_id + # TODO: server not returning run ID yet + # assert desc.run_id == activity_handle.run_id assert desc.activity_type == "increment" assert desc.task_queue == task_queue - assert desc.status in [ - ActivityExecutionStatus.RUNNING, - ActivityExecutionStatus.COMPLETED, - ] + assert desc.status == ActivityExecutionStatus.RUNNING From 32f5abbe074f7de9971e26376245c179a328754c Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 03:19:36 -0400 Subject: [PATCH 12/25] Add activity IDs to test since they are now set in context --- tests/test_serialization_context.py | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/tests/test_serialization_context.py b/tests/test_serialization_context.py index 9475e29ad..d9c10046b 100644 --- a/tests/test_serialization_context.py +++ b/tests/test_serialization_context.py @@ -179,6 +179,7 @@ async def run(self, data: TraceData) -> TraceData: data, start_to_close_timeout=timedelta(seconds=10), heartbeat_timeout=timedelta(seconds=2), + activity_id="activity-id", ) data = await workflow.execute_child_workflow( EchoWorkflow.run, data, id=f"{workflow.info().workflow_id}_child" @@ -231,7 +232,7 @@ async def test_payload_conversion_calls_follow_expected_sequence_and_contexts( workflow_id=workflow_id, workflow_type=PayloadConversionWorkflow.__name__, activity_type=passthrough_activity.__name__, - activity_id=None, + activity_id="activity-id", activity_task_queue=task_queue, is_local=False, ) @@ -329,6 +330,7 @@ async def run(self) -> TraceData: initial_interval=timedelta(milliseconds=100), maximum_attempts=2, ), + activity_id="activity-id", ) @@ -371,7 +373,7 @@ async def test_heartbeat_details_payload_conversion(client: Client): workflow_id=workflow_id, workflow_type=HeartbeatDetailsSerializationContextTestWorkflow.__name__, activity_type=activity_with_heartbeat_details.__name__, - activity_id=None, + activity_id="activity-id", activity_task_queue=task_queue, is_local=False, ) @@ -421,6 +423,7 @@ async def run(self, data: TraceData) -> TraceData: local_activity, data, start_to_close_timeout=timedelta(seconds=10), + activity_id="activity-id", ) @@ -461,7 +464,7 @@ async def test_local_activity_payload_conversion(client: Client): workflow_id=workflow_id, workflow_type=LocalActivityWorkflow.__name__, activity_type=local_activity.__name__, - activity_id=None, + activity_id="activity-id", activity_task_queue=task_queue, is_local=True, ) @@ -575,7 +578,7 @@ async def test_async_activity_completion_payload_conversion( workflow_id=workflow_id, workflow_type=AsyncActivityCompletionSerializationContextTestWorkflow.__name__, activity_type=async_activity.__name__, - activity_id=None, + activity_id="async-activity-id", activity_task_queue=task_queue, is_local=False, ) @@ -648,7 +651,7 @@ def test_subclassed_async_activity_handle(client: Client): workflow_id="workflow-id", workflow_type="workflow-type", activity_type="activity-type", - activity_id=None, + activity_id="activity-id", activity_task_queue="activity-task-queue", is_local=False, ) @@ -1063,6 +1066,7 @@ async def run(self) -> Never: failing_activity, start_to_close_timeout=timedelta(seconds=10), retry_policy=RetryPolicy(maximum_attempts=1), + activity_id="activity-id", ) raise Exception("Unreachable") @@ -1159,7 +1163,7 @@ async def test_failure_converter_with_context(client: Client): workflow_id=workflow_id, workflow_type=FailureConverterTestWorkflow.__name__, activity_type=failing_activity.__name__, - activity_id=None, + activity_id="activity-id", activity_task_queue=task_queue, is_local=False, ) @@ -1328,6 +1332,7 @@ async def run(self, data: str) -> str: codec_test_local_activity, data, start_to_close_timeout=timedelta(seconds=10), + activity_id="activity-id", ) @@ -1366,7 +1371,7 @@ async def test_local_activity_codec_with_context(client: Client): workflow_id=workflow_id, workflow_type=LocalActivityCodecTestWorkflow.__name__, activity_type=codec_test_local_activity.__name__, - activity_id=None, + activity_id="activity-id", activity_task_queue=task_queue, is_local=True, ) @@ -1600,6 +1605,7 @@ async def run(self, data: str) -> str: payload_encryption_activity, "outbound", start_to_close_timeout=timedelta(seconds=10), + activity_id="activity-id", ), workflow.execute_child_workflow( PayloadEncryptionChildWorkflow.run, From d28ae5b619ade89076f432c56dd89294d04b7542 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 03:28:08 -0400 Subject: [PATCH 13/25] Eliminate base serialization context class --- temporalio/converter.py | 48 +++++++++++++++-------------- tests/test_serialization_context.py | 9 +++--- 2 files changed, 30 insertions(+), 27 deletions(-) diff --git a/temporalio/converter.py b/temporalio/converter.py index 1a9b38d2b..ae592dd64 100644 --- a/temporalio/converter.py +++ b/temporalio/converter.py @@ -93,16 +93,7 @@ class SerializationContext(ABC): @dataclass(frozen=True) -class BaseWorkflowSerializationContext(SerializationContext): - """Base serialization context shared by workflow and activity serialization contexts.""" - - namespace: str - workflow_id: Optional[str] - """Workflow ID.""" - - -@dataclass(frozen=True) -class WorkflowSerializationContext(BaseWorkflowSerializationContext): +class WorkflowSerializationContext(SerializationContext): """Serialization context for workflows. See :py:class:`SerializationContext` for more details. @@ -115,40 +106,51 @@ class WorkflowSerializationContext(BaseWorkflowSerializationContext): when the workflow is created by the schedule. """ - pass + namespace: str + """Namespace.""" + + workflow_id: Optional[str] + """Workflow ID.""" @dataclass(frozen=True) -class ActivitySerializationContext(BaseWorkflowSerializationContext): +class ActivitySerializationContext(SerializationContext): """Serialization context for activities. See :py:class:`SerializationContext` for more details. Attributes: namespace: Workflow/activity namespace. - workflow_id: Workflow ID. Note, when creating/describing schedules, + activity_id: Activity ID. Optional if this is an activity started from a workflow. + activity_type: Activity type. + activity_task_queue: Activity task queue. + workflow_id: Workflow ID. Only set if this is an activity started from a workflow. Note, when creating/describing schedules, this may be the workflow ID prefix as configured, not the final workflow ID when the workflow is created by the schedule. - workflow_type: Workflow Type. - activity_type: Activity Type. - activity_task_queue: Activity task queue. - is_local: Whether the activity is a local activity. + workflow_type: Workflow Type. Only set if this is an activity started from a workflow. + is_local: Whether the activity is a local activity. False if this is a standalone activity started directly by a client. """ - workflow_type: Optional[str] - """Workflow type.""" + namespace: str + """Namespace.""" + + activity_id: Optional[str] + """Activity ID. Optional if this is an activity started from a workflow.""" activity_type: str """Activity type.""" - activity_id: Optional[str] - """Activity ID.""" - activity_task_queue: str """Activity task queue.""" + workflow_id: Optional[str] + """Workflow ID if this is an activity started from a workflow.""" + + workflow_type: Optional[str] + """Workflow type if this is an activity started from a workflow.""" + is_local: bool - """Whether the activity is a local activity.""" + """Whether the activity is a local activity started from a workflow.""" class WithSerializationContext(ABC): diff --git a/tests/test_serialization_context.py b/tests/test_serialization_context.py index d9c10046b..02a697c71 100644 --- a/tests/test_serialization_context.py +++ b/tests/test_serialization_context.py @@ -510,7 +510,7 @@ async def test_local_activity_payload_conversion(client: Client): @workflow.defn -class EventWorkflow: +class WaitForSignalWorkflow: # Like a global asyncio.Event() def __init__(self) -> None: @@ -527,10 +527,11 @@ def signal(self) -> None: @activity.defn async def async_activity() -> TraceData: + # Notify test that the activity has started and is ready to be completed manually await ( activity.client() .get_workflow_handle("activity-started-wf-id") - .signal(EventWorkflow.signal) + .signal(WaitForSignalWorkflow.signal) ) activity.raise_complete_async() @@ -564,7 +565,7 @@ async def test_async_activity_completion_payload_conversion( task_queue=task_queue, workflows=[ AsyncActivityCompletionSerializationContextTestWorkflow, - EventWorkflow, + WaitForSignalWorkflow, ], activities=[async_activity], workflow_runner=UnsandboxedWorkflowRunner(), # so that we can use isinstance @@ -584,7 +585,7 @@ async def test_async_activity_completion_payload_conversion( ) act_started_wf_handle = await client.start_workflow( - EventWorkflow.run, + WaitForSignalWorkflow.run, id="activity-started-wf-id", task_queue=task_queue, ) From 741603fbf7ea4192d184067b811fc344e22161eb Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 03:37:31 -0400 Subject: [PATCH 14/25] Extend tests --- tests/test_activity.py | 234 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 230 insertions(+), 4 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 1f96f78bb..043863148 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -1,9 +1,14 @@ +import asyncio import uuid from datetime import timedelta -from temporalio import activity -from temporalio.client import Client +import pytest + +from temporalio import activity, workflow +from temporalio.client import ActivityFailedError, Client from temporalio.common import ActivityExecutionStatus +from temporalio.exceptions import ApplicationError, CancelledError +from temporalio.worker import Worker @activity.defn @@ -11,8 +16,9 @@ async def increment(input: int) -> int: return input + 1 -async def test_describe_activity(client: Client): - activity_id = str("test_start_and_describe_activity_id") +@pytest.mark.skip("StartActivityExecution not implemented in server") +async def test_start_activity_and_describe_activity(client: Client): + activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) activity_handle = await client.start_activity( @@ -29,3 +35,223 @@ async def test_describe_activity(client: Client): assert desc.activity_type == "increment" assert desc.task_queue == task_queue assert desc.status == ActivityExecutionStatus.RUNNING + + +@pytest.mark.skip("GetActivityExecutionResult not implemented in server") +async def test_start_activity_and_get_result(client: Client): + activity_id = str(uuid.uuid4()) + task_queue = str(uuid.uuid4()) + + activity_handle = await client.start_activity( + increment, + args=(1,), + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + result_via_execute_activity = client.execute_activity( + increment, + args=(1,), + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + + async with Worker( + client, + task_queue=task_queue, + activities=[increment], + ): + assert await activity_handle.result() == 2 + assert await result_via_execute_activity == 2 + + +@activity.defn +async def async_activity() -> int: + # Notify test that the activity has started and is ready to be completed manually + await ( + activity.client() + .get_workflow_handle("activity-started-wf-id") + .signal(WaitForSignalWorkflow.signal) + ) + activity.raise_complete_async() + + +@pytest.mark.skip( + "RespondActivityTaskCompleted not implemented in server for standalone activity" +) +async def test_manual_completion(client: Client): + activity_id = str(uuid.uuid4()) + task_queue = str(uuid.uuid4()) + + activity_handle = await client.start_activity( + async_activity, + args=(), # TODO: overloads + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + + async with Worker( + client, + task_queue=task_queue, + activities=[async_activity], + workflows=[WaitForSignalWorkflow], + ): + # Wait for activity to start + await client.execute_workflow( + WaitForSignalWorkflow.run, + id="activity-started-wf-id", + task_queue=task_queue, + ) + # Complete activity manually + async_activity_handle = client.get_async_activity_handle( + activity_id=activity_id, + run_id=activity_handle.run_id, + ) + await async_activity_handle.complete(7) + assert await activity_handle.result() == 7 + + +@pytest.mark.skip( + "RespondActivityTaskCanceled not implemented in server for standalone activity" +) +async def test_manual_cancellation(client: Client): + activity_id = str(uuid.uuid4()) + task_queue = str(uuid.uuid4()) + + activity_handle = await client.start_activity( + async_activity, + args=(), # TODO: overloads + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + + async with Worker( + client, + task_queue=task_queue, + activities=[async_activity], + workflows=[WaitForSignalWorkflow], + ): + await client.execute_workflow( + WaitForSignalWorkflow.run, + id="activity-started-wf-id", + task_queue=task_queue, + ) + async_activity_handle = client.get_async_activity_handle( + activity_id=activity_id, + run_id=activity_handle.run_id, + ) + await async_activity_handle.report_cancellation("Test cancellation") + with pytest.raises(ActivityFailedError) as err: + await activity_handle.result() + assert isinstance(err.value.cause, CancelledError) + assert str(err.value.cause) == "Test cancellation" + + +@pytest.mark.skip( + "RespondActivityTaskFailed not implemented in server for standalone activity" +) +async def test_manual_fail(client: Client): + activity_id = str(uuid.uuid4()) + task_queue = str(uuid.uuid4()) + + activity_handle = await client.start_activity( + async_activity, + args=(), # TODO: overloads + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + async with Worker( + client, + task_queue=task_queue, + activities=[async_activity], + workflows=[WaitForSignalWorkflow], + ): + await client.execute_workflow( + WaitForSignalWorkflow.run, + id="activity-started-wf-id", + task_queue=task_queue, + ) + async_activity_handle = client.get_async_activity_handle( + activity_id=activity_id, + run_id=activity_handle.run_id, + ) + await async_activity_handle.fail(Exception("Test failure")) + with pytest.raises(ActivityFailedError) as err: + await activity_handle.result() + assert isinstance(err.value.cause, ApplicationError) + assert str(err.value.cause) == "Test failure" + + +@activity.defn +async def activity_for_testing_heartbeat() -> str: + wait_for_heartbeat_wf_handle = await activity.client().start_workflow( + WaitForSignalWorkflow.run, + id="test-has-sent-heartbeat-wf-id", + task_queue=activity.info().task_queue, + ) + info = activity.info() + if info.attempt == 1: + # Wait for test to notify that it has sent heartbeat + await wait_for_heartbeat_wf_handle.result() + raise Exception("Intentional error to force retry") + elif info.attempt == 2: + [heartbeat_data] = info.heartbeat_details + assert isinstance(heartbeat_data, str) + return heartbeat_data + else: + raise AssertionError(f"Unexpected attempt number: {info.attempt}") + + +@pytest.mark.skip( + "RecordActivityTaskHeartbeat not implemented in server for standalone activity" +) +async def test_manual_heartbeat(client: Client): + activity_id = str(uuid.uuid4()) + task_queue = str(uuid.uuid4()) + + activity_handle = await client.start_activity( + activity_for_testing_heartbeat, + args=(), # TODO: overloads + id=activity_id, + task_queue=task_queue, + start_to_close_timeout=timedelta(seconds=5), + ) + async with Worker( + client, + task_queue=task_queue, + activities=[activity_for_testing_heartbeat], + workflows=[WaitForSignalWorkflow], + ): + async_activity_handle = client.get_async_activity_handle( + activity_id=activity_id, + run_id=activity_handle.run_id, + ) + await async_activity_handle.heartbeat("Test heartbeat details") + await client.get_workflow_handle_for( + WaitForSignalWorkflow.run, + workflow_id="test-has-sent-heartbeat-wf-id", + ).signal(WaitForSignalWorkflow.signal) + assert await activity_handle.result() == "Test heartbeat details" + + +# Utilities + + +@workflow.defn +class WaitForSignalWorkflow: + # Like a global asyncio.Event() + + def __init__(self) -> None: + self.signal_received = asyncio.Event() + + @workflow.run + async def run(self) -> None: + await self.signal_received.wait() + + @workflow.signal + def signal(self) -> None: + self.signal_received.set() From 1e75e382aa056a34509aee45bc1cedc5b109da61 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 07:20:52 -0400 Subject: [PATCH 15/25] Don't hardcode workflow IDs --- tests/test_activity.py | 46 ++++++++++++++++++++++++++++-------------- 1 file changed, 31 insertions(+), 15 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 043863148..44b359778 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -1,5 +1,6 @@ import asyncio import uuid +from dataclasses import dataclass from datetime import timedelta import pytest @@ -66,27 +67,33 @@ async def test_start_activity_and_get_result(client: Client): assert await result_via_execute_activity == 2 +@dataclass +class ActivityInput: + wait_for_signal_workflow_id: str + + @activity.defn -async def async_activity() -> int: +async def async_activity(input: ActivityInput) -> int: # Notify test that the activity has started and is ready to be completed manually await ( activity.client() - .get_workflow_handle("activity-started-wf-id") + .get_workflow_handle(input.wait_for_signal_workflow_id) .signal(WaitForSignalWorkflow.signal) ) activity.raise_complete_async() -@pytest.mark.skip( - "RespondActivityTaskCompleted not implemented in server for standalone activity" -) +@pytest.mark.skip("StartActivityExecution not implemented in server") async def test_manual_completion(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) + wait_for_signal_workflow_id = str(uuid.uuid4()) activity_handle = await client.start_activity( async_activity, - args=(), # TODO: overloads + args=( + ActivityInput(wait_for_signal_workflow_id=wait_for_signal_workflow_id), + ), # TODO: overloads id=activity_id, task_queue=task_queue, start_to_close_timeout=timedelta(seconds=5), @@ -101,7 +108,7 @@ async def test_manual_completion(client: Client): # Wait for activity to start await client.execute_workflow( WaitForSignalWorkflow.run, - id="activity-started-wf-id", + id=wait_for_signal_workflow_id, task_queue=task_queue, ) # Complete activity manually @@ -119,10 +126,13 @@ async def test_manual_completion(client: Client): async def test_manual_cancellation(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) + wait_for_signal_workflow_id = str(uuid.uuid4()) activity_handle = await client.start_activity( async_activity, - args=(), # TODO: overloads + args=( + ActivityInput(wait_for_signal_workflow_id=wait_for_signal_workflow_id), + ), # TODO: overloads id=activity_id, task_queue=task_queue, start_to_close_timeout=timedelta(seconds=5), @@ -136,7 +146,7 @@ async def test_manual_cancellation(client: Client): ): await client.execute_workflow( WaitForSignalWorkflow.run, - id="activity-started-wf-id", + id=wait_for_signal_workflow_id, task_queue=task_queue, ) async_activity_handle = client.get_async_activity_handle( @@ -156,10 +166,13 @@ async def test_manual_cancellation(client: Client): async def test_manual_fail(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) + wait_for_signal_workflow_id = str(uuid.uuid4()) activity_handle = await client.start_activity( async_activity, - args=(), # TODO: overloads + args=( + ActivityInput(wait_for_signal_workflow_id=wait_for_signal_workflow_id), + ), # TODO: overloads id=activity_id, task_queue=task_queue, start_to_close_timeout=timedelta(seconds=5), @@ -172,7 +185,7 @@ async def test_manual_fail(client: Client): ): await client.execute_workflow( WaitForSignalWorkflow.run, - id="activity-started-wf-id", + id=wait_for_signal_workflow_id, task_queue=task_queue, ) async_activity_handle = client.get_async_activity_handle( @@ -187,10 +200,10 @@ async def test_manual_fail(client: Client): @activity.defn -async def activity_for_testing_heartbeat() -> str: +async def activity_for_testing_heartbeat(input: ActivityInput) -> str: wait_for_heartbeat_wf_handle = await activity.client().start_workflow( WaitForSignalWorkflow.run, - id="test-has-sent-heartbeat-wf-id", + id=input.wait_for_signal_workflow_id, task_queue=activity.info().task_queue, ) info = activity.info() @@ -212,10 +225,13 @@ async def activity_for_testing_heartbeat() -> str: async def test_manual_heartbeat(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) + wait_for_signal_workflow_id = str(uuid.uuid4()) activity_handle = await client.start_activity( activity_for_testing_heartbeat, - args=(), # TODO: overloads + args=( + ActivityInput(wait_for_signal_workflow_id=wait_for_signal_workflow_id), + ), # TODO: overloads id=activity_id, task_queue=task_queue, start_to_close_timeout=timedelta(seconds=5), @@ -233,7 +249,7 @@ async def test_manual_heartbeat(client: Client): await async_activity_handle.heartbeat("Test heartbeat details") await client.get_workflow_handle_for( WaitForSignalWorkflow.run, - workflow_id="test-has-sent-heartbeat-wf-id", + workflow_id=wait_for_signal_workflow_id, ).signal(WaitForSignalWorkflow.signal) assert await activity_handle.result() == "Test heartbeat details" From e0697327130aeb3d894744d0c35542ae50a8450d Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 09:14:00 -0400 Subject: [PATCH 16/25] Fix AsyncActivityHandle --- temporalio/client.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index b2a2042b2..13bcffe0d 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -1566,7 +1566,6 @@ def get_async_activity_handle( run_id: Run ID for the activity. Cannot be set if task_token is set. activity_id: ID for the activity. - activity_id: ID for the activity. task_token: Task token for the activity. Returns: @@ -1587,7 +1586,18 @@ def get_async_activity_handle( workflow_id=workflow_id, run_id=run_id, activity_id=activity_id ), ) - raise ValueError("Task token or workflow/run/activity ID must be present") + elif activity_id is not None: + return AsyncActivityHandle( + self, + AsyncActivityIDReference( + activity_id=activity_id, + run_id=run_id, + workflow_id=None, + ), + ) + raise ValueError( + "Require task token, or workflow_id & run_id & activity_id, or activity_id & run_id" + ) async def create_schedule( self, From e37c625d3918c2df8b6d559813f1a6aca9ff6d53 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 09:15:29 -0400 Subject: [PATCH 17/25] TEMP: Alter manual activity tests to not require worker --- tests/test_activity.py | 45 +++++++++++++++++++++++++----------------- 1 file changed, 27 insertions(+), 18 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 44b359778..26aa3a3d4 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -11,6 +11,8 @@ from temporalio.exceptions import ApplicationError, CancelledError from temporalio.worker import Worker +STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED = False + @activity.defn async def increment(input: int) -> int: @@ -105,12 +107,13 @@ async def test_manual_completion(client: Client): activities=[async_activity], workflows=[WaitForSignalWorkflow], ): - # Wait for activity to start - await client.execute_workflow( - WaitForSignalWorkflow.run, - id=wait_for_signal_workflow_id, - task_queue=task_queue, - ) + if STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED: + # Wait for activity to start + await client.execute_workflow( + WaitForSignalWorkflow.run, + id=wait_for_signal_workflow_id, + task_queue=task_queue, + ) # Complete activity manually async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, @@ -144,11 +147,14 @@ async def test_manual_cancellation(client: Client): activities=[async_activity], workflows=[WaitForSignalWorkflow], ): - await client.execute_workflow( - WaitForSignalWorkflow.run, - id=wait_for_signal_workflow_id, - task_queue=task_queue, - ) + if STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED: + # Wait for activity to start + await client.execute_workflow( + WaitForSignalWorkflow.run, + id=wait_for_signal_workflow_id, + task_queue=task_queue, + ) + # Cancel activity manually async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, run_id=activity_handle.run_id, @@ -183,11 +189,14 @@ async def test_manual_fail(client: Client): activities=[async_activity], workflows=[WaitForSignalWorkflow], ): - await client.execute_workflow( - WaitForSignalWorkflow.run, - id=wait_for_signal_workflow_id, - task_queue=task_queue, - ) + if STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED: + # Wait for activity to start + await client.execute_workflow( + WaitForSignalWorkflow.run, + id=wait_for_signal_workflow_id, + task_queue=task_queue, + ) + # Fail activity manually async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, run_id=activity_handle.run_id, @@ -242,13 +251,13 @@ async def test_manual_heartbeat(client: Client): activities=[activity_for_testing_heartbeat], workflows=[WaitForSignalWorkflow], ): + # Send heartbeat manually async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, run_id=activity_handle.run_id, ) await async_activity_handle.heartbeat("Test heartbeat details") - await client.get_workflow_handle_for( - WaitForSignalWorkflow.run, + await client.get_workflow_handle( workflow_id=wait_for_signal_workflow_id, ).signal(WaitForSignalWorkflow.signal) assert await activity_handle.result() == "Test heartbeat details" From 269d532241493912a0f083711c407cd34c885f3e Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 10 Oct 2025 09:18:46 -0400 Subject: [PATCH 18/25] Skip tests globally --- tests/test_activity.py | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 26aa3a3d4..866938a05 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -13,13 +13,14 @@ STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED = False +pytest.skip("Standalone activity is not implemented in server", allow_module_level=True) + @activity.defn async def increment(input: int) -> int: return input + 1 -@pytest.mark.skip("StartActivityExecution not implemented in server") async def test_start_activity_and_describe_activity(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) @@ -40,7 +41,6 @@ async def test_start_activity_and_describe_activity(client: Client): assert desc.status == ActivityExecutionStatus.RUNNING -@pytest.mark.skip("GetActivityExecutionResult not implemented in server") async def test_start_activity_and_get_result(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) @@ -85,7 +85,6 @@ async def async_activity(input: ActivityInput) -> int: activity.raise_complete_async() -@pytest.mark.skip("StartActivityExecution not implemented in server") async def test_manual_completion(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) @@ -123,9 +122,6 @@ async def test_manual_completion(client: Client): assert await activity_handle.result() == 7 -@pytest.mark.skip( - "RespondActivityTaskCanceled not implemented in server for standalone activity" -) async def test_manual_cancellation(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) @@ -166,9 +162,6 @@ async def test_manual_cancellation(client: Client): assert str(err.value.cause) == "Test cancellation" -@pytest.mark.skip( - "RespondActivityTaskFailed not implemented in server for standalone activity" -) async def test_manual_fail(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) @@ -228,9 +221,6 @@ async def activity_for_testing_heartbeat(input: ActivityInput) -> str: raise AssertionError(f"Unexpected attempt number: {info.attempt}") -@pytest.mark.skip( - "RecordActivityTaskHeartbeat not implemented in server for standalone activity" -) async def test_manual_heartbeat(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) From 4d355d14296dcf96a0b8d2e625e3bcfab16038a1 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Sat, 11 Oct 2025 00:23:39 -0400 Subject: [PATCH 19/25] Revert "TEMP: Alter manual activity tests to not require worker" This reverts commit 7127c0fe5296d6df4c174f2792447c98934a95ce. --- tests/test_activity.py | 47 ++++++++++++++++-------------------------- 1 file changed, 18 insertions(+), 29 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 866938a05..7bfa45e42 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -11,10 +11,6 @@ from temporalio.exceptions import ApplicationError, CancelledError from temporalio.worker import Worker -STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED = False - -pytest.skip("Standalone activity is not implemented in server", allow_module_level=True) - @activity.defn async def increment(input: int) -> int: @@ -106,13 +102,12 @@ async def test_manual_completion(client: Client): activities=[async_activity], workflows=[WaitForSignalWorkflow], ): - if STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED: - # Wait for activity to start - await client.execute_workflow( - WaitForSignalWorkflow.run, - id=wait_for_signal_workflow_id, - task_queue=task_queue, - ) + # Wait for activity to start + await client.execute_workflow( + WaitForSignalWorkflow.run, + id=wait_for_signal_workflow_id, + task_queue=task_queue, + ) # Complete activity manually async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, @@ -143,14 +138,11 @@ async def test_manual_cancellation(client: Client): activities=[async_activity], workflows=[WaitForSignalWorkflow], ): - if STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED: - # Wait for activity to start - await client.execute_workflow( - WaitForSignalWorkflow.run, - id=wait_for_signal_workflow_id, - task_queue=task_queue, - ) - # Cancel activity manually + await client.execute_workflow( + WaitForSignalWorkflow.run, + id=wait_for_signal_workflow_id, + task_queue=task_queue, + ) async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, run_id=activity_handle.run_id, @@ -182,14 +174,11 @@ async def test_manual_fail(client: Client): activities=[async_activity], workflows=[WaitForSignalWorkflow], ): - if STANDALONE_ACTIVITY_MATCHING_TASKS_IMPLEMENTED: - # Wait for activity to start - await client.execute_workflow( - WaitForSignalWorkflow.run, - id=wait_for_signal_workflow_id, - task_queue=task_queue, - ) - # Fail activity manually + await client.execute_workflow( + WaitForSignalWorkflow.run, + id=wait_for_signal_workflow_id, + task_queue=task_queue, + ) async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, run_id=activity_handle.run_id, @@ -241,13 +230,13 @@ async def test_manual_heartbeat(client: Client): activities=[activity_for_testing_heartbeat], workflows=[WaitForSignalWorkflow], ): - # Send heartbeat manually async_activity_handle = client.get_async_activity_handle( activity_id=activity_id, run_id=activity_handle.run_id, ) await async_activity_handle.heartbeat("Test heartbeat details") - await client.get_workflow_handle( + await client.get_workflow_handle_for( + WaitForSignalWorkflow.run, workflow_id=wait_for_signal_workflow_id, ).signal(WaitForSignalWorkflow.signal) assert await activity_handle.result() == "Test heartbeat details" From 976532d7b19b6fe63230fc74bee2ce2b6f16643f Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Mon, 13 Oct 2025 08:07:52 -0400 Subject: [PATCH 20/25] Assert describe returns new state after transition --- tests/test_activity.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 7bfa45e42..7584ebfa2 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -116,6 +116,9 @@ async def test_manual_completion(client: Client): await async_activity_handle.complete(7) assert await activity_handle.result() == 7 + desc = await activity_handle.describe() + assert desc.status == ActivityExecutionStatus.COMPLETED + async def test_manual_cancellation(client: Client): activity_id = str(uuid.uuid4()) @@ -153,8 +156,11 @@ async def test_manual_cancellation(client: Client): assert isinstance(err.value.cause, CancelledError) assert str(err.value.cause) == "Test cancellation" + desc = await activity_handle.describe() + assert desc.status == ActivityExecutionStatus.CANCELED + -async def test_manual_fail(client: Client): +async def test_manual_failure(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) wait_for_signal_workflow_id = str(uuid.uuid4()) @@ -189,6 +195,9 @@ async def test_manual_fail(client: Client): assert isinstance(err.value.cause, ApplicationError) assert str(err.value.cause) == "Test failure" + desc = await activity_handle.describe() + assert desc.status == ActivityExecutionStatus.FAILED + @activity.defn async def activity_for_testing_heartbeat(input: ActivityInput) -> str: From 92faa7f82c9bf7ae224ec14c7118fa27e094def1 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Mon, 13 Oct 2025 08:50:38 -0400 Subject: [PATCH 21/25] Fix cancellation and failure tests --- tests/test_activity.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 7584ebfa2..c313597eb 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -154,7 +154,7 @@ async def test_manual_cancellation(client: Client): with pytest.raises(ActivityFailedError) as err: await activity_handle.result() assert isinstance(err.value.cause, CancelledError) - assert str(err.value.cause) == "Test cancellation" + assert list(err.value.cause.details) == ["Test cancellation"] desc = await activity_handle.describe() assert desc.status == ActivityExecutionStatus.CANCELED @@ -189,7 +189,7 @@ async def test_manual_failure(client: Client): activity_id=activity_id, run_id=activity_handle.run_id, ) - await async_activity_handle.fail(Exception("Test failure")) + await async_activity_handle.fail(ApplicationError("Test failure")) with pytest.raises(ActivityFailedError) as err: await activity_handle.result() assert isinstance(err.value.cause, ApplicationError) From ea402d9229386285f544fff24388329bcb14ec4b Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Mon, 13 Oct 2025 10:22:26 -0400 Subject: [PATCH 22/25] Cleanup --- tests/test_activity.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index c313597eb..65c8743d2 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -17,7 +17,7 @@ async def increment(input: int) -> int: return input + 1 -async def test_start_activity_and_describe_activity(client: Client): +async def test_describe(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) @@ -37,7 +37,7 @@ async def test_start_activity_and_describe_activity(client: Client): assert desc.status == ActivityExecutionStatus.RUNNING -async def test_start_activity_and_get_result(client: Client): +async def test_get_result(client: Client): activity_id = str(uuid.uuid4()) task_queue = str(uuid.uuid4()) From af9f80a7e88988549b79173858508ce36d920858 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Mon, 13 Oct 2025 11:03:39 -0400 Subject: [PATCH 23/25] Add experimental notices --- temporalio/client.py | 150 +++++++++++++++++++++++++++++++++++++------ temporalio/common.py | 12 ++++ 2 files changed, 143 insertions(+), 19 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index 13bcffe0d..eaac6bc05 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -1317,6 +1317,9 @@ async def start_activity( ) -> ActivityHandle[ReturnType]: """Start an activity and return its handle. + .. warning:: + This API is experimental. + Args: activity: String name or callable activity function to execute. args: Arguments to pass to the activity. @@ -1401,6 +1404,9 @@ async def execute_activity( ) -> ReturnType: """Start an activity, wait for it to complete, and return its result. + .. warning:: + This API is experimental. + This is a convenience method that combines :py:meth:`start_activity` and :py:meth:`ActivityHandle.result`. @@ -1444,6 +1450,9 @@ def list_activities( ) -> ActivityExecutionAsyncIterator: """List activities. + .. warning:: + This API is experimental. + This does not make a request until the first iteration is attempted. Therefore any errors will not occur until then. @@ -1484,6 +1493,9 @@ async def count_activities( ) -> ActivityExecutionCount: """Count activities matching the query. + .. warning:: + This API is experimental. + Args: query: A Temporal visibility filter for activities. rpc_metadata: Headers used on the RPC call. Keys here override @@ -1507,6 +1519,9 @@ def get_activity_handle( ) -> ActivityHandle[Any]: """Get a handle to an existing activity, as the caller of that activity. + .. warning:: + This API is experimental. + To get a handle to an activity execution that you control for manual completion and heartbeating, see :py:meth:`Client.get_async_activity_handle`. @@ -3117,7 +3132,11 @@ async def __anext__(self) -> Union[ActivityExecution, WorkflowActivityExecution] # https://github.com/temporalio/api/pull/640/files @dataclass(frozen=True) class ActivityExecution: - """Info for a standalone activity execution from list response.""" + """Info for a standalone activity execution from list response. + + .. warning:: + This API is experimental. + """ activity_id: str """Activity ID.""" @@ -3197,7 +3216,11 @@ def _from_raw_info( @dataclass(frozen=True) class WorkflowActivityExecution: - """Info for a workflow activity execution from list response.""" + """Info for a workflow activity execution from list response. + + .. warning:: + This API is experimental. + """ activity_id: str """Activity ID.""" @@ -3264,7 +3287,11 @@ def _from_raw_info( @dataclass(frozen=True) class ActivityExecutionCount: - """Representation of a count from a count activities call.""" + """Representation of a count from a count activities call. + + .. warning:: + This API is experimental. + """ count: int """Total count matching the filter, if any.""" @@ -3281,7 +3308,11 @@ def _from_raw( @dataclass(frozen=True) class ActivityExecutionDescription: - """Detailed information about an activity execution from describe response.""" + """Detailed information about an activity execution from describe response. + + .. warning:: + This API is experimental. + """ activity_id: str """Activity ID.""" @@ -3455,7 +3486,11 @@ async def _from_raw_info( @dataclass(frozen=True) class ActivityIDReference: - """Information identifying an activity execution.""" + """Information identifying an activity execution. + + .. warning:: + This API is experimental. + """ workflow_id: Optional[str] run_id: Optional[str] @@ -3609,7 +3644,11 @@ def with_context(self, context: SerializationContext) -> Self: # be generic in the activity type in addition to the return type (as WorkflowHandle), to support # static type inference for signal/query/update. class ActivityHandle(Generic[ReturnType]): - """Handle representing a standalone activity execution.""" + """Handle representing a standalone activity execution. + + .. warning:: + This API is experimental. + """ def __init__( self, @@ -3676,6 +3715,9 @@ async def result( ) -> ReturnType: """Wait for result of the activity. + .. warning:: + This API is experimental. + The result may already be known if this method has been called before, in which case no network call is made. Otherwise the result will be polled for until it is available. @@ -3769,6 +3811,9 @@ async def cancel( ) -> None: """Request cancellation of the activity. + .. warning:: + This API is experimental. + Requesting cancellation of an activity does not automatically transition the activity to canceled status. If the activity is heartbeating, a :py:class:`exceptions.CancelledError` exception will be raised when receiving the heartbeat response; if the activity allows this @@ -3799,6 +3844,9 @@ async def terminate( ) -> None: """Terminate the activity execution immediately. + .. warning:: + This API is experimental. + Termination does not reach the worker and the activity code cannot react to it. A terminated activity may have a running attempt and will be requested to be canceled by the server when it heartbeats. @@ -3826,6 +3874,9 @@ async def describe( ) -> ActivityExecutionDescription: """Describe the activity execution. + .. warning:: + This API is experimental. + Args: rpc_metadata: Headers used on the RPC call. rpc_timeout: Optional RPC deadline to set for the RPC call. @@ -3854,6 +3905,9 @@ async def pause( ) -> None: """Pause the activity. + .. warning:: + This API is experimental. + Args: reason: Reason for pausing the activity. rpc_metadata: Headers used on the RPC call. Keys here override @@ -3893,6 +3947,9 @@ async def unpause( ) -> None: """Unpause the activity. + .. warning:: + This API is experimental. + Args: reset_attempts: Whether to reset the number of attempts. rpc_metadata: Headers used on the RPC call. Keys here override @@ -3933,6 +3990,9 @@ async def reset( ) -> None: """Reset the activity. + .. warning:: + This API is experimental. + Args: reset_heartbeat: Whether to reset heartbeat details. keep_paused: If activity is paused, whether to keep it paused after reset. @@ -6394,7 +6454,11 @@ def __init__(self) -> None: class ActivityFailedError(temporalio.exceptions.TemporalError): - """Error that occurs when a standalone activity is unsuccessful.""" + """Error that occurs when a standalone activity is unsuccessful. + + .. warning:: + This API is experimental. + """ def __init__(self, *, cause: BaseException) -> None: """Create activity failure error.""" @@ -6566,7 +6630,11 @@ class TerminateWorkflowInput: @dataclass class StartActivityInput: - """Input for :py:meth:`OutboundInterceptor.start_activity`.""" + """Input for :py:meth:`OutboundInterceptor.start_activity`. + + .. warning:: + This API is experimental. + """ activity_type: str args: Sequence[Any] @@ -6596,7 +6664,11 @@ class StartActivityInput: @dataclass class CancelActivityInput: - """Input for :py:meth:`OutboundInterceptor.cancel_activity`.""" + """Input for :py:meth:`OutboundInterceptor.cancel_activity`. + + .. warning:: + This API is experimental. + """ activity_id: str run_id: str @@ -6607,7 +6679,11 @@ class CancelActivityInput: @dataclass class TerminateActivityInput: - """Input for :py:meth:`OutboundInterceptor.terminate_activity`.""" + """Input for :py:meth:`OutboundInterceptor.terminate_activity`. + + .. warning:: + This API is experimental. + """ activity_id: str run_id: str @@ -6618,7 +6694,11 @@ class TerminateActivityInput: @dataclass class DescribeActivityInput: - """Input for :py:meth:`OutboundInterceptor.describe_activity`.""" + """Input for :py:meth:`OutboundInterceptor.describe_activity`. + + .. warning:: + This API is experimental. + """ activity_id: str run_id: str @@ -6628,7 +6708,11 @@ class DescribeActivityInput: @dataclass class ListActivitiesInput: - """Input for :py:meth:`OutboundInterceptor.list_activities`.""" + """Input for :py:meth:`OutboundInterceptor.list_activities`. + + .. warning:: + This API is experimental. + """ query: Optional[str] page_size: int @@ -6640,7 +6724,11 @@ class ListActivitiesInput: @dataclass class CountActivitiesInput: - """Input for :py:meth:`OutboundInterceptor.count_activities`.""" + """Input for :py:meth:`OutboundInterceptor.count_activities`. + + .. warning:: + This API is experimental. + """ query: Optional[str] rpc_metadata: Mapping[str, Union[str, bytes]] @@ -6988,33 +7076,57 @@ async def terminate_workflow(self, input: TerminateWorkflowInput) -> None: ### Activity calls async def start_activity(self, input: StartActivityInput) -> ActivityHandle[Any]: - """Called for every :py:meth:`Client.start_activity` call.""" + """Called for every :py:meth:`Client.start_activity` call. + + .. warning:: + This API is experimental. + """ return await self.next.start_activity(input) async def cancel_activity(self, input: CancelActivityInput) -> None: - """Called for every :py:meth:`ActivityHandle.cancel` call.""" + """Called for every :py:meth:`ActivityHandle.cancel` call. + + .. warning:: + This API is experimental. + """ await self.next.cancel_activity(input) async def terminate_activity(self, input: TerminateActivityInput) -> None: - """Called for every :py:meth:`ActivityHandle.terminate` call.""" + """Called for every :py:meth:`ActivityHandle.terminate` call. + + .. warning:: + This API is experimental. + """ await self.next.terminate_activity(input) async def describe_activity( self, input: DescribeActivityInput ) -> ActivityExecutionDescription: - """Called for every :py:meth:`ActivityHandle.describe` call.""" + """Called for every :py:meth:`ActivityHandle.describe` call. + + .. warning:: + This API is experimental. + """ return await self.next.describe_activity(input) def list_activities( self, input: ListActivitiesInput ) -> ActivityExecutionAsyncIterator: - """Called for every :py:meth:`Client.list_activities` call.""" + """Called for every :py:meth:`Client.list_activities` call. + + .. warning:: + This API is experimental. + """ return self.next.list_activities(input) async def count_activities( self, input: CountActivitiesInput ) -> ActivityExecutionCount: - """Called for every :py:meth:`Client.count_activities` call.""" + """Called for every :py:meth:`Client.count_activities` call. + + .. warning:: + This API is experimental. + """ return await self.next.count_activities(input) async def start_workflow_update( diff --git a/temporalio/common.py b/temporalio/common.py index ac201cab5..cf30b7247 100644 --- a/temporalio/common.py +++ b/temporalio/common.py @@ -157,6 +157,9 @@ class WorkflowIDConflictPolicy(IntEnum): class IdReusePolicy(IntEnum): """How already-closed entity IDs are handled on start. + .. warning:: + This API is experimental. + See :py:class:`temporalio.api.enums.v1.IdReusePolicy`. """ @@ -175,6 +178,9 @@ class IdReusePolicy(IntEnum): class IdConflictPolicy(IntEnum): """How already-running entity IDs are handled on start. + .. warning:: + This API is experimental. + See :py:class:`temporalio.api.enums.v1.IdConflictPolicy`. """ @@ -193,6 +199,9 @@ class IdConflictPolicy(IntEnum): class ActivityExecutionStatus(IntEnum): """Status of a standalone activity execution. + .. warning:: + This API is experimental. + See :py:class:`temporalio.api.enums.v1.ActivityExecutionStatus`. """ @@ -208,6 +217,9 @@ class ActivityExecutionStatus(IntEnum): class PendingActivityState(IntEnum): """State of a pending activity. + .. warning:: + This API is experimental. + See :py:class:`temporalio.api.enums.v1.PendingActivityState`. """ From 48d5ddc6d945c84d2007000a2bf2988717660139 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Mon, 13 Oct 2025 12:15:09 -0400 Subject: [PATCH 24/25] Workaround mypy bug --- tests/test_activity.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/test_activity.py b/tests/test_activity.py index 65c8743d2..ad217dc21 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -244,8 +244,7 @@ async def test_manual_heartbeat(client: Client): run_id=activity_handle.run_id, ) await async_activity_handle.heartbeat("Test heartbeat details") - await client.get_workflow_handle_for( - WaitForSignalWorkflow.run, + await client.get_workflow_handle( workflow_id=wait_for_signal_workflow_id, ).signal(WaitForSignalWorkflow.signal) assert await activity_handle.result() == "Test heartbeat details" From d71fe318bd5fa66d8ec4cbba43f102063d28838e Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Mon, 13 Oct 2025 12:29:33 -0400 Subject: [PATCH 25/25] Skip Standalone Activity tests in CI --- temporalio/client.py | 2 -- tests/test_activity.py | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/temporalio/client.py b/temporalio/client.py index eaac6bc05..8d8e36c80 100644 --- a/temporalio/client.py +++ b/temporalio/client.py @@ -3128,8 +3128,6 @@ async def __anext__(self) -> Union[ActivityExecution, WorkflowActivityExecution] return ret -# TODO: this is named ActivityListInfo in our draft proto PR -# https://github.com/temporalio/api/pull/640/files @dataclass(frozen=True) class ActivityExecution: """Info for a standalone activity execution from list response. diff --git a/tests/test_activity.py b/tests/test_activity.py index ad217dc21..0160ac20f 100644 --- a/tests/test_activity.py +++ b/tests/test_activity.py @@ -11,6 +11,8 @@ from temporalio.exceptions import ApplicationError, CancelledError from temporalio.worker import Worker +pytest.skip(reason="Standalone Activity server support is under development") + @activity.defn async def increment(input: int) -> int: