From 7666cdc2037ea457f76fb7eac0b51d46a0ce5375 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Thu, 2 Oct 2025 12:46:35 -0400 Subject: [PATCH 01/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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/18] 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())