From 07076fa228e32b17093c1becc69a68e76a659da4 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Wed, 15 Dec 2021 14:39:37 -0800 Subject: [PATCH 01/12] Extract DeleteWorkflowExecution to the historyEngine --- common/metrics/defs.go | 2 + service/history/consts/const.go | 2 + service/history/historyEngine.go | 196 +++++++++++++++++- service/history/shard/context_impl.go | 4 +- service/history/timerQueueTaskExecutorBase.go | 180 +++------------- .../history/workflowTaskHandlerCallbacks.go | 2 +- 6 files changed, 225 insertions(+), 161 deletions(-) diff --git a/common/metrics/defs.go b/common/metrics/defs.go index e7ec665229b..8bd1d1fa0ca 100644 --- a/common/metrics/defs.go +++ b/common/metrics/defs.go @@ -1091,6 +1091,8 @@ const ( HistoryQueryWorkflowScope // HistoryProcessDeleteHistoryEventScope tracks ProcessDeleteHistoryEvent processing calls HistoryProcessDeleteHistoryEventScope + // HistoryDeleteWorkflowExecutionScope tracks DeleteWorkflowExecutions API calls + HistoryDeleteWorkflowExecutionScope // WorkflowCompletionStatsScope tracks workflow completion updates WorkflowCompletionStatsScope // ArchiverClientScope is scope used by all metrics emitted by archiver.Client diff --git a/service/history/consts/const.go b/service/history/consts/const.go index 16708f8580c..6edf37c6834 100644 --- a/service/history/consts/const.go +++ b/service/history/consts/const.go @@ -54,6 +54,8 @@ var ( ErrActivityTaskNotCancelRequested = serviceerror.NewInvalidArgument("unable to mark activity as canceled without activity being request canceled first") // ErrWorkflowCompleted is the error to indicate workflow execution already completed ErrWorkflowCompleted = serviceerror.NewNotFound("workflow execution already completed") + // ErrWorkflowIsRunning is the error to indicate workflow is still running + ErrWorkflowIsRunning = serviceerror.NewInvalidArgument("workflow is running") // ErrWorkflowExecutionNotFound is the error to indicate workflow execution does not exist ErrWorkflowExecutionNotFound = serviceerror.NewNotFound("workflow execution not found") // ErrWorkflowParent is the error to parent execution is given and mismatch diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 4090ad98445..bac2482ab79 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -27,6 +27,7 @@ package history import ( "bytes" "context" + "errors" "fmt" "sync" "sync/atomic" @@ -120,7 +121,6 @@ type ( rawMatchingClient matchingservice.MatchingServiceClient replicationDLQHandler replicationDLQHandler searchAttributesValidator *searchattribute.Validator - searchAttributesMapper searchattribute.Mapper } ) @@ -217,8 +217,6 @@ func NewEngineWithShardContext( config.SearchAttributesTotalSizeLimit, ) - historyEngImpl.searchAttributesMapper = shard.GetSearchAttributesMapper() - historyEngImpl.workflowTaskHandler = newWorkflowTaskHandlerCallback(historyEngImpl) historyEngImpl.replicationDLQHandler = newLazyReplicationDLQHandler(shard) @@ -514,7 +512,7 @@ func (e *historyEngineImpl) StartWorkflowExecution( return nil, err } - err = searchattribute.SubstituteAliases(e.searchAttributesMapper, request.GetSearchAttributes(), namespace.String()) + err = searchattribute.SubstituteAliases(e.shard.GetSearchAttributesMapper(), request.GetSearchAttributes(), namespace.String()) if err != nil { return nil, err } @@ -2012,7 +2010,7 @@ func (e *historyEngineImpl) SignalWithStartWorkflowExecution( return nil, err } - err = searchattribute.SubstituteAliases(e.searchAttributesMapper, request.GetSearchAttributes(), namespace.String()) + err = searchattribute.SubstituteAliases(e.shard.GetSearchAttributesMapper(), request.GetSearchAttributes(), namespace.String()) if err != nil { return nil, err } @@ -2240,6 +2238,194 @@ func (e *historyEngineImpl) TerminateWorkflowExecution( }) } +func (e *historyEngineImpl) DeleteWorkflowExecution( + ctx context.Context, + namespaceID namespace.ID, + we commonpb.WorkflowExecution, + archiveIfEnabled bool, +) (retError error) { + + weCtx, release, err := e.historyCache.GetOrCreateWorkflowExecution( + ctx, + namespaceID, + we, + workflow.CallerTypeAPI, + ) + if err != nil { + return err + } + defer func() { release(retError) }() + + mutableState, err := weCtx.LoadWorkflowExecution() + if err != nil { + return err + } + + return e.deleteWorkflowExecutionInternal( + namespaceID, + we, + weCtx, + mutableState, + mutableState.GetCurrentVersion(), + archiveIfEnabled, + e.metricsClient.Scope(metrics.HistoryDeleteWorkflowExecutionScope), + ) +} + +func (e *historyEngineImpl) DeleteWorkflowExecutionFromTimerTask( + namespaceID namespace.ID, + we commonpb.WorkflowExecution, + weCtx workflow.Context, + ms workflow.MutableState, + timerTaskVersion int64, +) (retError error) { + + err := e.deleteWorkflowExecutionInternal( + namespaceID, + we, + weCtx, + ms, + timerTaskVersion, + true, + e.metricsClient.Scope(metrics.HistoryProcessDeleteHistoryEventScope), + ) + + if err != nil && errors.Is(err, consts.ErrWorkflowIsRunning) { + // If workflow is running then just ignore DeleteHistoryEventTask timer task. + // This should almost never happen because DeleteHistoryEventTask is created only for closed workflows. + // But cross DC replication can resurrect workflow and therefore DeleteHistoryEventTask should be ignored. + return nil + } + + return err +} + +func (e *historyEngineImpl) deleteWorkflowExecutionInternal( + namespaceID namespace.ID, + we commonpb.WorkflowExecution, + weCtx workflow.Context, + ms workflow.MutableState, + deleteVisibilityTaskVersion int64, + archiveIfEnabled bool, + scope metrics.Scope, +) error { + + if ms.IsWorkflowExecutionRunning() { + return consts.ErrWorkflowIsRunning + } + + currentBranchToken, err := ms.GetCurrentBranchToken() + if err != nil { + return err + } + + shouldDeleteHistory := true + if archiveIfEnabled { + shouldDeleteHistory, err = e.archiveWorkflowIfEnabled(namespaceID, we, currentBranchToken, weCtx, ms, scope) + if err != nil { + return err + } + } + + if !shouldDeleteHistory { + // currentBranchToken == nil means don't delete history. + currentBranchToken = nil + } + if err := e.shard.DeleteWorkflowExecution( + definition.WorkflowKey{ + NamespaceID: namespaceID.String(), + WorkflowID: we.GetWorkflowId(), + RunID: we.GetRunId(), + }, + currentBranchToken, + deleteVisibilityTaskVersion, + ); err != nil { + return err + } + + // Clear workflow execution context here to prevent further readers to get stale copy of non-exiting workflow execution. + weCtx.Clear() + + scope.IncCounter(metrics.WorkflowCleanupDeleteCount) + return nil +} + +func (e *historyEngineImpl) archiveWorkflowIfEnabled( + namespaceID namespace.ID, + workflowExecution commonpb.WorkflowExecution, + currentBranchToken []byte, + weCtx workflow.Context, + mutableState workflow.MutableState, + scope metrics.Scope, +) (bool, error) { + + namespaceRegistryEntry, err := e.shard.GetNamespaceRegistry().GetNamespaceByID(namespaceID) + if err != nil { + return false, err + } + clusterConfiguredForHistoryArchival := e.shard.GetArchivalMetadata().GetHistoryConfig().ClusterConfiguredForArchival() + namespaceConfiguredForHistoryArchival := namespaceRegistryEntry.HistoryArchivalState().State == enumspb.ARCHIVAL_STATE_ENABLED + archiveHistory := clusterConfiguredForHistoryArchival && namespaceConfiguredForHistoryArchival + + // TODO: @ycyang once archival backfill is in place cluster:paused && namespace:enabled should be a nop rather than a delete + if !archiveHistory { + return true, nil + } + + closeFailoverVersion, err := mutableState.GetLastWriteVersion() + if err != nil { + return false, err + } + + req := &archiver.ClientRequest{ + ArchiveRequest: &archiver.ArchiveRequest{ + NamespaceID: namespaceID.String(), + WorkflowID: workflowExecution.GetWorkflowId(), + RunID: workflowExecution.GetRunId(), + Namespace: namespaceRegistryEntry.Name().String(), + ShardID: e.shard.GetShardID(), + Targets: []archiver.ArchivalTarget{archiver.ArchiveTargetHistory}, + HistoryURI: namespaceRegistryEntry.HistoryArchivalState().URI, + NextEventID: mutableState.GetNextEventID(), + BranchToken: currentBranchToken, + CloseFailoverVersion: closeFailoverVersion, + }, + CallerService: common.HistoryServiceName, + AttemptArchiveInline: false, // archive in workflow by default + } + executionStats, err := weCtx.LoadExecutionStats() + if err == nil && executionStats.HistorySize < int64(e.config.TimerProcessorHistoryArchivalSizeLimit()) { + req.AttemptArchiveInline = true + } + + saTypeMap, err := e.shard.GetSearchAttributesProvider().GetSearchAttributes(e.config.DefaultVisibilityIndexName, false) + if err != nil { + return false, err + } + // Setting search attributes types here because archival client needs to stringify them, + // and it might not have access to typeMap (i.e. type needs to be embedded). + searchattribute.ApplyTypeMap(req.ArchiveRequest.SearchAttributes, saTypeMap) + + ctx, cancel := context.WithTimeout(context.Background(), e.config.TimerProcessorArchivalTimeLimit()) + defer cancel() + resp, err := e.archivalClient.Archive(ctx, req) + if err != nil { + return false, err + } + + var deleteHistory bool + if resp.HistoryArchivedInline { + scope.IncCounter(metrics.WorkflowCleanupDeleteHistoryInlineCount) + deleteHistory = true + } else { + scope.IncCounter(metrics.WorkflowCleanupArchiveCount) + // Don't delete workflow history if it wasn't achieve inline because archival workflow will need it. + deleteHistory = false + } + + return deleteHistory, nil +} + // RecordChildExecutionCompleted records the completion of child execution into parent execution history func (e *historyEngineImpl) RecordChildExecutionCompleted( ctx context.Context, diff --git a/service/history/shard/context_impl.go b/service/history/shard/context_impl.go index a2553a83212..d07fd4d9f0e 100644 --- a/service/history/shard/context_impl.go +++ b/service/history/shard/context_impl.go @@ -784,7 +784,7 @@ func (s *ContextImpl) AppendHistoryEvents( func (s *ContextImpl) DeleteWorkflowExecution( key definition.WorkflowKey, branchToken []byte, - version int64, + deleteVisibilityTaskVersion int64, ) error { if err := s.errorByState(); err != nil { return err @@ -855,7 +855,7 @@ func (s *ContextImpl) DeleteWorkflowExecution( // TaskID is set by addTasksLocked WorkflowKey: key, VisibilityTimestamp: s.timeSource.Now(), - Version: version, + Version: deleteVisibilityTaskVersion, }}, } err = s.addTasksLocked(addTasksRequest, namespaceEntry) diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index 1c0a2cc0c78..344f5d2eebe 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -28,31 +28,24 @@ import ( "context" commonpb "go.temporal.io/api/common/v1" - enumspb "go.temporal.io/api/enums/v1" - "go.temporal.io/server/common" - "go.temporal.io/server/common/definition" "go.temporal.io/server/common/log" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" - "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" - - "go.temporal.io/server/service/worker/archiver" ) type ( timerQueueTaskExecutorBase struct { - shard shard.Context - historyService *historyEngineImpl - cache workflow.Cache - logger log.Logger - metricsClient metrics.Client - config *configs.Config - searchAttributesProvider searchattribute.Provider + shard shard.Context + historyEngine *historyEngineImpl + cache workflow.Cache + logger log.Logger + metricsClient metrics.Client + config *configs.Config } ) @@ -64,13 +57,12 @@ func newTimerQueueTaskExecutorBase( config *configs.Config, ) *timerQueueTaskExecutorBase { return &timerQueueTaskExecutorBase{ - shard: shard, - historyService: historyEngine, - cache: historyEngine.historyCache, - logger: logger, - metricsClient: metricsClient, - config: config, - searchAttributesProvider: shard.GetSearchAttributesProvider(), + shard: shard, + historyEngine: historyEngine, + cache: historyEngine.historyCache, + logger: logger, + metricsClient: metricsClient, + config: config, } } @@ -78,16 +70,19 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( ctx context.Context, task *tasks.DeleteHistoryEventTask, ) (retError error) { - var cancel context.CancelFunc - ctx, cancel = context.WithTimeout(ctx, taskTimeout) + ctx, cancel := context.WithTimeout(ctx, taskTimeout) defer cancel() - namespaceID, execution := t.getNamespaceIDAndWorkflowExecution(task) + workflowExecution := commonpb.WorkflowExecution{ + WorkflowId: task.GetWorkflowID(), + RunId: task.GetRunID(), + } + weContext, release, err := t.cache.GetOrCreateWorkflowExecution( ctx, - namespaceID, - execution, + namespace.ID(task.GetNamespaceID()), + workflowExecution, workflow.CallerTypeTask, ) if err != nil { @@ -99,9 +94,6 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( if err != nil { return err } - if mutableState == nil || mutableState.IsWorkflowExecutionRunning() { - return nil - } lastWriteVersion, err := mutableState.GetLastWriteVersion() if err != nil { @@ -112,131 +104,13 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( return err } - namespaceRegistryEntry, err := t.shard.GetNamespaceRegistry().GetNamespaceByID(namespace.ID(task.NamespaceID)) - if err != nil { - return err - } - clusterConfiguredForHistoryArchival := t.shard.GetArchivalMetadata().GetHistoryConfig().ClusterConfiguredForArchival() - namespaceConfiguredForHistoryArchival := namespaceRegistryEntry.HistoryArchivalState().State == enumspb.ARCHIVAL_STATE_ENABLED - archiveHistory := clusterConfiguredForHistoryArchival && namespaceConfiguredForHistoryArchival - - // TODO: @ycyang once archival backfill is in place cluster:paused && namespace:enabled should be a nop rather than a delete - if archiveHistory { - t.metricsClient.IncCounter(metrics.HistoryProcessDeleteHistoryEventScope, metrics.WorkflowCleanupArchiveCount) - return t.archiveWorkflow(task, weContext, mutableState, namespaceRegistryEntry) - } - - t.metricsClient.IncCounter(metrics.HistoryProcessDeleteHistoryEventScope, metrics.WorkflowCleanupDeleteCount) - return t.deleteWorkflow(task, weContext, mutableState) -} - -func (t *timerQueueTaskExecutorBase) deleteWorkflow( - task *tasks.DeleteHistoryEventTask, - workflowContext workflow.Context, - msBuilder workflow.MutableState, -) error { - branchToken, err := msBuilder.GetCurrentBranchToken() - if err != nil { - return err - } - - if err := t.shard.DeleteWorkflowExecution( - definition.WorkflowKey{ - NamespaceID: task.NamespaceID, - WorkflowID: task.WorkflowID, - RunID: task.RunID, - }, - branchToken, - task.Version, - ); err != nil { - return err - } - - // calling clear here to force accesses of mutable state to read database - // if this is not called then callers will get mutable state even though its been removed from database - workflowContext.Clear() - return nil -} - -func (t *timerQueueTaskExecutorBase) archiveWorkflow( - task *tasks.DeleteHistoryEventTask, - workflowContext workflow.Context, - msBuilder workflow.MutableState, - namespaceRegistryEntry *namespace.Namespace, -) error { - branchToken, err := msBuilder.GetCurrentBranchToken() - if err != nil { - return err - } - closeFailoverVersion, err := msBuilder.GetLastWriteVersion() - if err != nil { - return err - } - - req := &archiver.ClientRequest{ - ArchiveRequest: &archiver.ArchiveRequest{ - NamespaceID: task.NamespaceID, - WorkflowID: task.WorkflowID, - RunID: task.RunID, - Namespace: namespaceRegistryEntry.Name().String(), - ShardID: t.shard.GetShardID(), - Targets: []archiver.ArchivalTarget{archiver.ArchiveTargetHistory}, - HistoryURI: namespaceRegistryEntry.HistoryArchivalState().URI, - NextEventID: msBuilder.GetNextEventID(), - BranchToken: branchToken, - CloseFailoverVersion: closeFailoverVersion, - }, - CallerService: common.HistoryServiceName, - AttemptArchiveInline: false, // archive in workflow by default - } - executionStats, err := workflowContext.LoadExecutionStats() - if err == nil && executionStats.HistorySize < int64(t.config.TimerProcessorHistoryArchivalSizeLimit()) { - req.AttemptArchiveInline = true - } - - saTypeMap, err := t.searchAttributesProvider.GetSearchAttributes(t.config.DefaultVisibilityIndexName, false) - if err != nil { - return err - } - // Setting search attributes types here because archival client needs to stringify them - // and it might not have access to typeMap (i.e. type needs to be embedded). - searchattribute.ApplyTypeMap(req.ArchiveRequest.SearchAttributes, saTypeMap) - - ctx, cancel := context.WithTimeout(context.Background(), t.config.TimerProcessorArchivalTimeLimit()) - defer cancel() - resp, err := t.historyService.archivalClient.Archive(ctx, req) - if err != nil { - return err - } - - // delete visibility record here regardless if it's been archived inline or not - // since the entire record is included as part of the archive request. - - // delete workflow history if history archival is not needed or history has been archived inline - if resp.HistoryArchivedInline { - // branchToken was retrieved above - t.metricsClient.IncCounter(metrics.HistoryProcessDeleteHistoryEventScope, metrics.WorkflowCleanupDeleteHistoryInlineCount) - } else { - // branchToken == nil means don't delete history - branchToken = nil - } - - if err := t.shard.DeleteWorkflowExecution( - definition.WorkflowKey{ - NamespaceID: task.NamespaceID, - WorkflowID: task.WorkflowID, - RunID: task.RunID, - }, - branchToken, - task.Version, - ); err != nil { - return err - } - - // calling clear here to force accesses of mutable state to read database - // if this is not called then callers will get mutable state even though its been removed from database - workflowContext.Clear() - return nil + return t.historyEngine.DeleteWorkflowExecutionFromTimerTask( + namespace.ID(task.GetNamespaceID()), + workflowExecution, + weContext, + mutableState, + task.GetVersion(), + ) } func (t *timerQueueTaskExecutorBase) getNamespaceIDAndWorkflowExecution( diff --git a/service/history/workflowTaskHandlerCallbacks.go b/service/history/workflowTaskHandlerCallbacks.go index cf69fcd59ad..70c3fb0fc53 100644 --- a/service/history/workflowTaskHandlerCallbacks.go +++ b/service/history/workflowTaskHandlerCallbacks.go @@ -107,7 +107,7 @@ func newWorkflowTaskHandlerCallback(historyEngine *historyEngineImpl) *workflowT historyEngine.config, historyEngine.searchAttributesValidator, ), - searchAttributesMapper: historyEngine.searchAttributesMapper, + searchAttributesMapper: historyEngine.shard.GetSearchAttributesMapper(), } } From 006a262a25ffee74ed5822cdf40574fbb44df2f0 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Thu, 16 Dec 2021 11:57:11 -0800 Subject: [PATCH 02/12] Extract DeleteManager --- service/history/historyEngine.go | 243 +++-------------- service/history/historyEngine2_test.go | 29 +- service/history/shard/context_testutil.go | 1 + service/history/shard/engine.go | 1 + service/history/timerQueueActiveProcessor.go | 6 +- .../history/timerQueueActiveTaskExecutor.go | 6 +- .../timerQueueActiveTaskExecutor_test.go | 28 +- service/history/timerQueueStandbyProcessor.go | 3 +- .../history/timerQueueStandbyTaskExecutor.go | 6 +- .../timerQueueStandbyTaskExecutor_test.go | 28 +- service/history/timerQueueTaskExecutorBase.go | 11 +- .../timerQueueTaskExecutorBase_test.go | 189 +++++-------- service/history/workflow/cache.go | 2 + service/history/workflow/cache_mock.go | 93 +++++++ service/history/workflow/delete_manager.go | 243 +++++++++++++++++ .../history/workflow/delete_manager_mock.go | 89 ++++++ .../history/workflow/delete_manager_test.go | 254 ++++++++++++++++++ 17 files changed, 844 insertions(+), 388 deletions(-) create mode 100644 service/history/workflow/cache_mock.go create mode 100644 service/history/workflow/delete_manager.go create mode 100644 service/history/workflow/delete_manager_mock.go create mode 100644 service/history/workflow/delete_manager_test.go diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index bac2482ab79..04889a4bb73 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -27,7 +27,6 @@ package history import ( "bytes" "context" - "errors" "fmt" "sync" "sync/atomic" @@ -121,6 +120,7 @@ type ( rawMatchingClient matchingservice.MatchingServiceClient replicationDLQHandler replicationDLQHandler searchAttributesValidator *searchattribute.Validator + workflowDeleteManager workflow.DeleteManager } ) @@ -145,33 +145,44 @@ func NewEngineWithShardContext( logger := shard.GetLogger() executionManager := shard.GetExecutionManager() historyCache := newCacheFn(shard) + + archivalClient := archiver.NewClient( + shard.GetMetricsClient(), + logger, + publicClient, + shard.GetConfig().NumArchiveSystemWorkflows, + shard.GetConfig().ArchiveRequestRPS, + archiverProvider, + ) + + workflowDeleteManager := workflow.NewDeleteManager( + shard, + historyCache, + config, + archivalClient, + ) + historyEngImpl := &historyEngineImpl{ - status: common.DaemonStatusInitialized, - currentClusterName: currentClusterName, - shard: shard, - clusterMetadata: shard.GetClusterMetadata(), - timeSource: shard.GetTimeSource(), - executionManager: executionManager, - tokenSerializer: common.NewProtoTaskTokenSerializer(), - historyCache: historyCache, - logger: log.With(logger, tag.ComponentHistoryEngine), - throttledLogger: log.With(shard.GetThrottledLogger(), tag.ComponentHistoryEngine), - metricsClient: shard.GetMetricsClient(), - eventNotifier: eventNotifier, - config: config, - archivalClient: archiver.NewClient( - shard.GetMetricsClient(), - logger, - publicClient, - shard.GetConfig().NumArchiveSystemWorkflows, - shard.GetConfig().ArchiveRequestRPS, - archiverProvider, - ), + status: common.DaemonStatusInitialized, + currentClusterName: currentClusterName, + shard: shard, + clusterMetadata: shard.GetClusterMetadata(), + timeSource: shard.GetTimeSource(), + executionManager: executionManager, + tokenSerializer: common.NewProtoTaskTokenSerializer(), + historyCache: historyCache, + logger: log.With(logger, tag.ComponentHistoryEngine), + throttledLogger: log.With(shard.GetThrottledLogger(), tag.ComponentHistoryEngine), + metricsClient: shard.GetMetricsClient(), + eventNotifier: eventNotifier, + config: config, + archivalClient: archivalClient, publicClient: publicClient, matchingClient: matchingClient, rawMatchingClient: rawMatchingClient, replicationTaskProcessors: make(map[string]ReplicationTaskProcessor), replicationTaskFetchers: replicationTaskFetchers, + workflowDeleteManager: workflowDeleteManager, } historyEngImpl.txProcessor = newTransferQueueProcessor(shard, historyEngImpl, @@ -2238,194 +2249,6 @@ func (e *historyEngineImpl) TerminateWorkflowExecution( }) } -func (e *historyEngineImpl) DeleteWorkflowExecution( - ctx context.Context, - namespaceID namespace.ID, - we commonpb.WorkflowExecution, - archiveIfEnabled bool, -) (retError error) { - - weCtx, release, err := e.historyCache.GetOrCreateWorkflowExecution( - ctx, - namespaceID, - we, - workflow.CallerTypeAPI, - ) - if err != nil { - return err - } - defer func() { release(retError) }() - - mutableState, err := weCtx.LoadWorkflowExecution() - if err != nil { - return err - } - - return e.deleteWorkflowExecutionInternal( - namespaceID, - we, - weCtx, - mutableState, - mutableState.GetCurrentVersion(), - archiveIfEnabled, - e.metricsClient.Scope(metrics.HistoryDeleteWorkflowExecutionScope), - ) -} - -func (e *historyEngineImpl) DeleteWorkflowExecutionFromTimerTask( - namespaceID namespace.ID, - we commonpb.WorkflowExecution, - weCtx workflow.Context, - ms workflow.MutableState, - timerTaskVersion int64, -) (retError error) { - - err := e.deleteWorkflowExecutionInternal( - namespaceID, - we, - weCtx, - ms, - timerTaskVersion, - true, - e.metricsClient.Scope(metrics.HistoryProcessDeleteHistoryEventScope), - ) - - if err != nil && errors.Is(err, consts.ErrWorkflowIsRunning) { - // If workflow is running then just ignore DeleteHistoryEventTask timer task. - // This should almost never happen because DeleteHistoryEventTask is created only for closed workflows. - // But cross DC replication can resurrect workflow and therefore DeleteHistoryEventTask should be ignored. - return nil - } - - return err -} - -func (e *historyEngineImpl) deleteWorkflowExecutionInternal( - namespaceID namespace.ID, - we commonpb.WorkflowExecution, - weCtx workflow.Context, - ms workflow.MutableState, - deleteVisibilityTaskVersion int64, - archiveIfEnabled bool, - scope metrics.Scope, -) error { - - if ms.IsWorkflowExecutionRunning() { - return consts.ErrWorkflowIsRunning - } - - currentBranchToken, err := ms.GetCurrentBranchToken() - if err != nil { - return err - } - - shouldDeleteHistory := true - if archiveIfEnabled { - shouldDeleteHistory, err = e.archiveWorkflowIfEnabled(namespaceID, we, currentBranchToken, weCtx, ms, scope) - if err != nil { - return err - } - } - - if !shouldDeleteHistory { - // currentBranchToken == nil means don't delete history. - currentBranchToken = nil - } - if err := e.shard.DeleteWorkflowExecution( - definition.WorkflowKey{ - NamespaceID: namespaceID.String(), - WorkflowID: we.GetWorkflowId(), - RunID: we.GetRunId(), - }, - currentBranchToken, - deleteVisibilityTaskVersion, - ); err != nil { - return err - } - - // Clear workflow execution context here to prevent further readers to get stale copy of non-exiting workflow execution. - weCtx.Clear() - - scope.IncCounter(metrics.WorkflowCleanupDeleteCount) - return nil -} - -func (e *historyEngineImpl) archiveWorkflowIfEnabled( - namespaceID namespace.ID, - workflowExecution commonpb.WorkflowExecution, - currentBranchToken []byte, - weCtx workflow.Context, - mutableState workflow.MutableState, - scope metrics.Scope, -) (bool, error) { - - namespaceRegistryEntry, err := e.shard.GetNamespaceRegistry().GetNamespaceByID(namespaceID) - if err != nil { - return false, err - } - clusterConfiguredForHistoryArchival := e.shard.GetArchivalMetadata().GetHistoryConfig().ClusterConfiguredForArchival() - namespaceConfiguredForHistoryArchival := namespaceRegistryEntry.HistoryArchivalState().State == enumspb.ARCHIVAL_STATE_ENABLED - archiveHistory := clusterConfiguredForHistoryArchival && namespaceConfiguredForHistoryArchival - - // TODO: @ycyang once archival backfill is in place cluster:paused && namespace:enabled should be a nop rather than a delete - if !archiveHistory { - return true, nil - } - - closeFailoverVersion, err := mutableState.GetLastWriteVersion() - if err != nil { - return false, err - } - - req := &archiver.ClientRequest{ - ArchiveRequest: &archiver.ArchiveRequest{ - NamespaceID: namespaceID.String(), - WorkflowID: workflowExecution.GetWorkflowId(), - RunID: workflowExecution.GetRunId(), - Namespace: namespaceRegistryEntry.Name().String(), - ShardID: e.shard.GetShardID(), - Targets: []archiver.ArchivalTarget{archiver.ArchiveTargetHistory}, - HistoryURI: namespaceRegistryEntry.HistoryArchivalState().URI, - NextEventID: mutableState.GetNextEventID(), - BranchToken: currentBranchToken, - CloseFailoverVersion: closeFailoverVersion, - }, - CallerService: common.HistoryServiceName, - AttemptArchiveInline: false, // archive in workflow by default - } - executionStats, err := weCtx.LoadExecutionStats() - if err == nil && executionStats.HistorySize < int64(e.config.TimerProcessorHistoryArchivalSizeLimit()) { - req.AttemptArchiveInline = true - } - - saTypeMap, err := e.shard.GetSearchAttributesProvider().GetSearchAttributes(e.config.DefaultVisibilityIndexName, false) - if err != nil { - return false, err - } - // Setting search attributes types here because archival client needs to stringify them, - // and it might not have access to typeMap (i.e. type needs to be embedded). - searchattribute.ApplyTypeMap(req.ArchiveRequest.SearchAttributes, saTypeMap) - - ctx, cancel := context.WithTimeout(context.Background(), e.config.TimerProcessorArchivalTimeLimit()) - defer cancel() - resp, err := e.archivalClient.Archive(ctx, req) - if err != nil { - return false, err - } - - var deleteHistory bool - if resp.HistoryArchivedInline { - scope.IncCounter(metrics.WorkflowCleanupDeleteHistoryInlineCount) - deleteHistory = true - } else { - scope.IncCounter(metrics.WorkflowCleanupArchiveCount) - // Don't delete workflow history if it wasn't achieve inline because archival workflow will need it. - deleteHistory = false - } - - return deleteHistory, nil -} - // RecordChildExecutionCompleted records the completion of child execution into parent execution history func (e *historyEngineImpl) RecordChildExecutionCompleted( ctx context.Context, diff --git a/service/history/historyEngine2_test.go b/service/history/historyEngine2_test.go index 5343223c594..3495ad73729 100644 --- a/service/history/historyEngine2_test.go +++ b/service/history/historyEngine2_test.go @@ -143,21 +143,20 @@ func (s *engine2Suite) SetupTest() { historyCache := workflow.NewCache(s.mockShard) h := &historyEngineImpl{ - currentClusterName: s.mockShard.GetClusterMetadata().GetCurrentClusterName(), - shard: s.mockShard, - clusterMetadata: s.mockClusterMetadata, - executionManager: s.mockExecutionMgr, - historyCache: historyCache, - logger: s.logger, - throttledLogger: s.logger, - metricsClient: metrics.NewNoopMetricsClient(), - tokenSerializer: common.NewProtoTaskTokenSerializer(), - config: s.config, - timeSource: s.mockShard.GetTimeSource(), - eventNotifier: events.NewNotifier(clock.NewRealTimeSource(), metrics.NewNoopMetricsClient(), func(namespace.ID, string) int32 { return 1 }), - txProcessor: s.mockTxProcessor, - timerProcessor: s.mockTimerProcessor, - searchAttributesMapper: s.mockShard.Resource.SearchAttributesMapper, + currentClusterName: s.mockShard.GetClusterMetadata().GetCurrentClusterName(), + shard: s.mockShard, + clusterMetadata: s.mockClusterMetadata, + executionManager: s.mockExecutionMgr, + historyCache: historyCache, + logger: s.logger, + throttledLogger: s.logger, + metricsClient: metrics.NewNoopMetricsClient(), + tokenSerializer: common.NewProtoTaskTokenSerializer(), + config: s.config, + timeSource: s.mockShard.GetTimeSource(), + eventNotifier: events.NewNotifier(clock.NewRealTimeSource(), metrics.NewNoopMetricsClient(), func(namespace.ID, string) int32 { return 1 }), + txProcessor: s.mockTxProcessor, + timerProcessor: s.mockTimerProcessor, searchAttributesValidator: searchattribute.NewValidator( searchattribute.NewTestProvider(), s.mockShard.Resource.SearchAttributesMapper, diff --git a/service/history/shard/context_testutil.go b/service/history/shard/context_testutil.go index 0b74994b64b..e87a4a44232 100644 --- a/service/history/shard/context_testutil.go +++ b/service/history/shard/context_testutil.go @@ -94,6 +94,7 @@ func NewTestContext( persistenceShardManager: resource.GetShardManager(), clientBean: resource.GetClientBean(), saProvider: resource.GetSearchAttributesProvider(), + saMapper: resource.GetSearchAttributesMapper(), historyClient: resource.GetHistoryClient(), archivalMetadata: resource.GetArchivalMetadata(), } diff --git a/service/history/shard/engine.go b/service/history/shard/engine.go index 9148fc1383e..4b01c0455f1 100644 --- a/service/history/shard/engine.go +++ b/service/history/shard/engine.go @@ -32,6 +32,7 @@ import ( commonpb "go.temporal.io/api/common/v1" historypb "go.temporal.io/api/history/v1" + "go.temporal.io/server/api/historyservice/v1" replicationspb "go.temporal.io/server/api/replication/v1" "go.temporal.io/server/common" diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go index 3b83d15b50c..c6691800689 100644 --- a/service/history/timerQueueActiveProcessor.go +++ b/service/history/timerQueueActiveProcessor.go @@ -97,7 +97,8 @@ func newTimerQueueActiveProcessor( } processor.taskExecutor = newTimerQueueActiveTaskExecutor( shard, - historyService, + historyService.workflowDeleteManager, + historyService.historyCache, processor, logger, historyService.metricsClient, @@ -189,7 +190,8 @@ func newTimerQueueFailoverProcessor( } processor.taskExecutor = newTimerQueueActiveTaskExecutor( shard, - historyService, + historyService.workflowDeleteManager, + historyService.historyCache, processor, logger, historyService.metricsClient, diff --git a/service/history/timerQueueActiveTaskExecutor.go b/service/history/timerQueueActiveTaskExecutor.go index c447ea8d5f2..2bf7d2d55b6 100644 --- a/service/history/timerQueueActiveTaskExecutor.go +++ b/service/history/timerQueueActiveTaskExecutor.go @@ -62,7 +62,8 @@ type ( func newTimerQueueActiveTaskExecutor( shard shard.Context, - historyService *historyEngineImpl, + workflowDeleteManager workflow.DeleteManager, + cache workflow.Cache, queueProcessor *timerQueueActiveProcessorImpl, logger log.Logger, metricsClient metrics.Client, @@ -72,7 +73,8 @@ func newTimerQueueActiveTaskExecutor( return &timerQueueActiveTaskExecutor{ timerQueueTaskExecutorBase: newTimerQueueTaskExecutorBase( shard, - historyService, + workflowDeleteManager, + cache, logger, metricsClient, config, diff --git a/service/history/timerQueueActiveTaskExecutor_test.go b/service/history/timerQueueActiveTaskExecutor_test.go index fbb14f86b2a..a7497268fb8 100644 --- a/service/history/timerQueueActiveTaskExecutor_test.go +++ b/service/history/timerQueueActiveTaskExecutor_test.go @@ -74,6 +74,7 @@ type ( mockClusterMetadata *cluster.MockMetadata mockHistoryEngine *historyEngineImpl + mockDeleteManager *workflow.MockDeleteManager mockExecutionMgr *persistence.MockExecutionManager logger log.Logger @@ -147,25 +148,28 @@ func (s *timerQueueActiveTaskExecutorSuite) SetupTest() { s.logger = s.mockShard.GetLogger() historyCache := workflow.NewCache(s.mockShard) + s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) h := &historyEngineImpl{ - currentClusterName: s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName(), - shard: s.mockShard, - clusterMetadata: s.mockClusterMetadata, - executionManager: s.mockExecutionMgr, - historyCache: historyCache, - logger: s.logger, - tokenSerializer: common.NewProtoTaskTokenSerializer(), - metricsClient: s.mockShard.GetMetricsClient(), - eventNotifier: events.NewNotifier(clock.NewRealTimeSource(), metrics.NewNoopMetricsClient(), func(namespace.ID, string) int32 { return 1 }), - txProcessor: s.mockTxProcessor, - timerProcessor: s.mockTimerProcessor, + currentClusterName: s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName(), + shard: s.mockShard, + clusterMetadata: s.mockClusterMetadata, + executionManager: s.mockExecutionMgr, + historyCache: historyCache, + logger: s.logger, + tokenSerializer: common.NewProtoTaskTokenSerializer(), + metricsClient: s.mockShard.GetMetricsClient(), + eventNotifier: events.NewNotifier(clock.NewRealTimeSource(), metrics.NewNoopMetricsClient(), func(namespace.ID, string) int32 { return 1 }), + txProcessor: s.mockTxProcessor, + timerProcessor: s.mockTimerProcessor, + workflowDeleteManager: s.mockDeleteManager, } s.mockShard.SetEngineForTesting(h) s.mockHistoryEngine = h s.timerQueueActiveTaskExecutor = newTimerQueueActiveTaskExecutor( s.mockShard, - h, + s.mockDeleteManager, + historyCache, newTimerQueueActiveProcessor( s.mockShard, h, diff --git a/service/history/timerQueueStandbyProcessor.go b/service/history/timerQueueStandbyProcessor.go index b1ece2eefa6..0a85ffb3df8 100644 --- a/service/history/timerQueueStandbyProcessor.go +++ b/service/history/timerQueueStandbyProcessor.go @@ -97,7 +97,8 @@ func newTimerQueueStandbyProcessor( timerGate: timerGate, taskExecutor: newTimerQueueStandbyTaskExecutor( shard, - historyService, + historyService.workflowDeleteManager, + historyService.historyCache, nDCHistoryResender, logger, historyService.metricsClient, diff --git a/service/history/timerQueueStandbyTaskExecutor.go b/service/history/timerQueueStandbyTaskExecutor.go index 948d8a4918c..c466bc97b35 100644 --- a/service/history/timerQueueStandbyTaskExecutor.go +++ b/service/history/timerQueueStandbyTaskExecutor.go @@ -60,7 +60,8 @@ type ( func newTimerQueueStandbyTaskExecutor( shard shard.Context, - historyService *historyEngineImpl, + deleteManager workflow.DeleteManager, + cache workflow.Cache, nDCHistoryResender xdc.NDCHistoryResender, logger log.Logger, metricsClient metrics.Client, @@ -71,7 +72,8 @@ func newTimerQueueStandbyTaskExecutor( return &timerQueueStandbyTaskExecutor{ timerQueueTaskExecutorBase: newTimerQueueTaskExecutorBase( shard, - historyService, + deleteManager, + cache, logger, metricsClient, config, diff --git a/service/history/timerQueueStandbyTaskExecutor_test.go b/service/history/timerQueueStandbyTaskExecutor_test.go index 01ddf81a666..c2dc7ec31e1 100644 --- a/service/history/timerQueueStandbyTaskExecutor_test.go +++ b/service/history/timerQueueStandbyTaskExecutor_test.go @@ -78,6 +78,7 @@ type ( mockClusterMetadata *cluster.MockMetadata mockAdminClient *adminservicemock.MockAdminServiceClient mockNDCHistoryResender *xdc.MockNDCHistoryResender + mockDeleteManager *workflow.MockDeleteManager logger log.Logger namespaceID namespace.ID @@ -162,18 +163,20 @@ func (s *timerQueueStandbyTaskExecutorSuite) SetupTest() { s.logger = s.mockShard.GetLogger() historyCache := workflow.NewCache(s.mockShard) + s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) h := &historyEngineImpl{ - currentClusterName: s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName(), - shard: s.mockShard, - clusterMetadata: s.mockClusterMetadata, - executionManager: s.mockExecutionMgr, - historyCache: historyCache, - logger: s.logger, - tokenSerializer: common.NewProtoTaskTokenSerializer(), - metricsClient: s.mockShard.GetMetricsClient(), - eventNotifier: events.NewNotifier(s.timeSource, metrics.NewNoopMetricsClient(), func(namespace.ID, string) int32 { return 1 }), - txProcessor: s.mockTxProcessor, - timerProcessor: s.mockTimerProcessor, + currentClusterName: s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName(), + shard: s.mockShard, + clusterMetadata: s.mockClusterMetadata, + executionManager: s.mockExecutionMgr, + historyCache: historyCache, + logger: s.logger, + tokenSerializer: common.NewProtoTaskTokenSerializer(), + metricsClient: s.mockShard.GetMetricsClient(), + eventNotifier: events.NewNotifier(s.timeSource, metrics.NewNoopMetricsClient(), func(namespace.ID, string) int32 { return 1 }), + txProcessor: s.mockTxProcessor, + timerProcessor: s.mockTimerProcessor, + workflowDeleteManager: s.mockDeleteManager, } s.mockShard.SetEngineForTesting(h) s.mockBean = client.NewMockBean(s.controller) @@ -181,7 +184,8 @@ func (s *timerQueueStandbyTaskExecutorSuite) SetupTest() { s.timerQueueStandbyTaskExecutor = newTimerQueueStandbyTaskExecutor( s.mockShard, - h, + s.mockDeleteManager, + historyCache, s.mockNDCHistoryResender, s.logger, s.mockShard.GetMetricsClient(), diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index 344f5d2eebe..4c82e78acaa 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -41,7 +41,7 @@ import ( type ( timerQueueTaskExecutorBase struct { shard shard.Context - historyEngine *historyEngineImpl + deleteManager workflow.DeleteManager cache workflow.Cache logger log.Logger metricsClient metrics.Client @@ -51,15 +51,16 @@ type ( func newTimerQueueTaskExecutorBase( shard shard.Context, - historyEngine *historyEngineImpl, + deleteManager workflow.DeleteManager, + cache workflow.Cache, logger log.Logger, metricsClient metrics.Client, config *configs.Config, ) *timerQueueTaskExecutorBase { return &timerQueueTaskExecutorBase{ shard: shard, - historyEngine: historyEngine, - cache: historyEngine.historyCache, + deleteManager: deleteManager, + cache: cache, logger: logger, metricsClient: metricsClient, config: config, @@ -104,7 +105,7 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( return err } - return t.historyEngine.DeleteWorkflowExecutionFromTimerTask( + return t.deleteManager.DeleteWorkflowExecutionFromTimerTask( namespace.ID(task.GetNamespaceID()), workflowExecution, weContext, diff --git a/service/history/timerQueueTaskExecutorBase_test.go b/service/history/timerQueueTaskExecutorBase_test.go index cb68d3dc717..95a16b9c954 100644 --- a/service/history/timerQueueTaskExecutorBase_test.go +++ b/service/history/timerQueueTaskExecutorBase_test.go @@ -25,27 +25,24 @@ package history import ( - "errors" + "context" "testing" "time" "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/serviceerror" "go.temporal.io/server/common/definition" - "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" persistencespb "go.temporal.io/server/api/persistence/v1" - "go.temporal.io/server/common" - "go.temporal.io/server/common/cluster" - "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" - "go.temporal.io/server/service/worker/archiver" ) type ( @@ -53,18 +50,11 @@ type ( suite.Suite *require.Assertions - controller *gomock.Controller - mockShard *shard.ContextTest - mockEngine *shard.MockEngine - mockWorkflowExecutionContext *workflow.MockContext - mockMutableState *workflow.MockMutableState - - mockExecutionManager *persistence.MockExecutionManager - mockArchivalClient *archiver.MockClient - mockNamespaceCache *namespace.MockRegistry - mockClusterMetadata *cluster.MockMetadata - mockSearchAttributesProvider *searchattribute.MockProvider + controller *gomock.Controller + mockDeleteManager *workflow.MockDeleteManager + mockCache *workflow.MockCache + testShardContext *shard.ContextTest timerQueueTaskExecutorBase *timerQueueTaskExecutorBase } ) @@ -86,11 +76,11 @@ func (s *timerQueueTaskExecutorBaseSuite) SetupTest() { s.Assertions = require.New(s.T()) s.controller = gomock.NewController(s.T()) - s.mockWorkflowExecutionContext = workflow.NewMockContext(s.controller) - s.mockMutableState = workflow.NewMockMutableState(s.controller) + s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) + s.mockCache = workflow.NewMockCache(s.controller) config := tests.NewDynamicConfig() - s.mockShard = shard.NewTestContext( + s.testShardContext = shard.NewTestContext( s.controller, &persistence.ShardInfoWithFailover{ ShardInfo: &persistencespb.ShardInfo{ @@ -100,30 +90,13 @@ func (s *timerQueueTaskExecutorBaseSuite) SetupTest() { }}, config, ) - s.mockEngine = shard.NewMockEngine(s.controller) - s.mockShard.SetEngineForTesting(s.mockEngine) - - s.mockExecutionManager = s.mockShard.Resource.ExecutionMgr - s.mockArchivalClient = archiver.NewMockClient(s.controller) - s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache - s.mockClusterMetadata = s.mockShard.Resource.ClusterMetadata - s.mockSearchAttributesProvider = s.mockShard.Resource.SearchAttributesProvider - - logger := s.mockShard.GetLogger() - - h := &historyEngineImpl{ - shard: s.mockShard, - logger: logger, - metricsClient: s.mockShard.GetMetricsClient(), - archivalClient: s.mockArchivalClient, - clusterMetadata: s.mockClusterMetadata, - } s.timerQueueTaskExecutorBase = newTimerQueueTaskExecutorBase( - s.mockShard, - h, - logger, - s.mockShard.GetMetricsClient(), + s.testShardContext, + s.mockDeleteManager, + s.mockCache, + s.testShardContext.GetLogger(), + s.testShardContext.GetMetricsClient(), config, ) } @@ -132,7 +105,7 @@ func (s *timerQueueTaskExecutorBaseSuite) TearDownTest() { s.controller.Finish() } -func (s *timerQueueTaskExecutorBaseSuite) TestDeleteWorkflow_NoErr() { +func (s *timerQueueTaskExecutorBaseSuite) Test_executeDeleteHistoryEventTask_NoErr() { task := &tasks.DeleteHistoryEventTask{ WorkflowKey: definition.NewWorkflowKey( tests.NamespaceID.String(), @@ -143,70 +116,37 @@ func (s *timerQueueTaskExecutorBaseSuite) TestDeleteWorkflow_NoErr() { TaskID: 12345, VisibilityTimestamp: time.Now().UTC(), } + we := commonpb.WorkflowExecution{ + WorkflowId: tests.WorkflowID, + RunId: tests.RunID, + } - s.mockWorkflowExecutionContext.EXPECT().Clear() + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) - s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionManager.EXPECT().AddTasks(gomock.Any()).Return(nil) - s.mockEngine.EXPECT().NotifyNewTransferTasks(gomock.Any(), gomock.Any()) - s.mockEngine.EXPECT().NotifyNewTimerTasks(gomock.Any(), gomock.Any()) - s.mockEngine.EXPECT().NotifyNewVisibilityTasks(gomock.Any()) - s.mockEngine.EXPECT().NotifyNewReplicationTasks(gomock.Any()) + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, workflow.NoopReleaseFn, nil) - s.mockExecutionManager.EXPECT().DeleteCurrentWorkflowExecution(gomock.Any()).Return(nil) - s.mockExecutionManager.EXPECT().DeleteWorkflowExecution(gomock.Any()).Return(nil) - s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any()).Return(nil) - s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) + mockWeCtx.EXPECT().LoadWorkflowExecution().Return(mockMutableState, nil) + mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) + mockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{}) + mockMutableState.EXPECT().GetNextEventID().Return(int64(2)) + s.testShardContext.Resource.ClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(false) - err := s.timerQueueTaskExecutorBase.deleteWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState) - s.NoError(err) -} - -func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_NoErr_InlineArchivalFailed() { - task := &tasks.DeleteHistoryEventTask{ - WorkflowKey: definition.NewWorkflowKey( - tests.NamespaceID.String(), - tests.WorkflowID, - tests.RunID, - ), - Version: 123, - TaskID: 12345, - VisibilityTimestamp: time.Now().UTC(), - } + s.mockDeleteManager.EXPECT().DeleteWorkflowExecutionFromTimerTask( + tests.NamespaceID, + we, + mockWeCtx, + mockMutableState, + int64(123), + ).Return(nil) - s.mockWorkflowExecutionContext.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ - HistorySize: 1024, - }, nil) - s.mockWorkflowExecutionContext.EXPECT().Clear() - s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) - s.mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1234), nil) - s.mockMutableState.EXPECT().GetNextEventID().Return(int64(101)) - - s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionManager.EXPECT().AddTasks(gomock.Any()).Return(nil) - s.mockEngine.EXPECT().NotifyNewTransferTasks(gomock.Any(), gomock.Any()) - s.mockEngine.EXPECT().NotifyNewTimerTasks(gomock.Any(), gomock.Any()) - s.mockEngine.EXPECT().NotifyNewVisibilityTasks(gomock.Any()) - s.mockEngine.EXPECT().NotifyNewReplicationTasks(gomock.Any()) - - s.mockExecutionManager.EXPECT().DeleteCurrentWorkflowExecution(gomock.Any()).Return(nil) - s.mockExecutionManager.EXPECT().DeleteWorkflowExecution(gomock.Any()).Return(nil) - - s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: true}). - Return(&archiver.ClientResponse{ - HistoryArchivedInline: false, - }, nil) - - s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false) - - namespaceRegistryEntry := namespace.NewNamespaceForTest(&persistencespb.NamespaceInfo{}, &persistencespb.NamespaceConfig{}, false, nil, 0) - err := s.timerQueueTaskExecutorBase.archiveWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState, namespaceRegistryEntry) + err := s.timerQueueTaskExecutorBase.executeDeleteHistoryEventTask( + context.Background(), + task) s.NoError(err) } -func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_SendSignalErr() { +func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_DeleteFailed() { task := &tasks.DeleteHistoryEventTask{ WorkflowKey: definition.NewWorkflowKey( tests.NamespaceID.String(), @@ -217,37 +157,32 @@ func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_SendSignalErr() { TaskID: 12345, VisibilityTimestamp: time.Now().UTC(), } + we := commonpb.WorkflowExecution{ + WorkflowId: tests.WorkflowID, + RunId: tests.RunID, + } - s.mockWorkflowExecutionContext.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ - HistorySize: 1024 * 1024 * 1024, - }, nil) + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) - s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) - s.mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1234), nil) - s.mockMutableState.EXPECT().GetNextEventID().Return(int64(101)) + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, workflow.NoopReleaseFn, nil) - s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: false}). - Return(nil, errors.New("failed to send signal")) - s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false) + mockWeCtx.EXPECT().LoadWorkflowExecution().Return(mockMutableState, nil) + mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) + mockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{}) + mockMutableState.EXPECT().GetNextEventID().Return(int64(2)) + s.testShardContext.Resource.ClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(false) - namespaceRegistryEntry := namespace.NewNamespaceForTest(&persistencespb.NamespaceInfo{}, &persistencespb.NamespaceConfig{}, false, nil, 0) - err := s.timerQueueTaskExecutorBase.archiveWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState, namespaceRegistryEntry) - s.Error(err) -} + s.mockDeleteManager.EXPECT().DeleteWorkflowExecutionFromTimerTask( + tests.NamespaceID, + we, + mockWeCtx, + mockMutableState, + int64(123), + ).Return(serviceerror.NewInternal("test error")) -type ( - archiverClientRequestMatcher struct { - inline bool - } -) - -func (m archiverClientRequestMatcher) Matches(x interface{}) bool { - req := x.(*archiver.ClientRequest) - return req.CallerService == common.HistoryServiceName && - req.AttemptArchiveInline == m.inline && - req.ArchiveRequest.Targets[0] == archiver.ArchiveTargetHistory -} - -func (m archiverClientRequestMatcher) String() string { - return "archiverClientRequestMatcher" + err := s.timerQueueTaskExecutorBase.executeDeleteHistoryEventTask( + context.Background(), + task) + s.Error(err) } diff --git a/service/history/workflow/cache.go b/service/history/workflow/cache.go index 78c7b019920..5b5e9c54080 100644 --- a/service/history/workflow/cache.go +++ b/service/history/workflow/cache.go @@ -22,6 +22,8 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. +//go:generate mockgen -copyright_file ../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination cache_mock.go + package workflow import ( diff --git a/service/history/workflow/cache_mock.go b/service/history/workflow/cache_mock.go new file mode 100644 index 00000000000..a7ab56d1854 --- /dev/null +++ b/service/history/workflow/cache_mock.go @@ -0,0 +1,93 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by MockGen. DO NOT EDIT. +// Source: cache.go + +// Package workflow is a generated GoMock package. +package workflow + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + v1 "go.temporal.io/api/common/v1" + namespace "go.temporal.io/server/common/namespace" +) + +// MockCache is a mock of Cache interface. +type MockCache struct { + ctrl *gomock.Controller + recorder *MockCacheMockRecorder +} + +// MockCacheMockRecorder is the mock recorder for MockCache. +type MockCacheMockRecorder struct { + mock *MockCache +} + +// NewMockCache creates a new mock instance. +func NewMockCache(ctrl *gomock.Controller) *MockCache { + mock := &MockCache{ctrl: ctrl} + mock.recorder = &MockCacheMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockCache) EXPECT() *MockCacheMockRecorder { + return m.recorder +} + +// GetOrCreateCurrentWorkflowExecution mocks base method. +func (m *MockCache) GetOrCreateCurrentWorkflowExecution(ctx context.Context, namespaceID namespace.ID, workflowID string) (Context, ReleaseCacheFunc, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetOrCreateCurrentWorkflowExecution", ctx, namespaceID, workflowID) + ret0, _ := ret[0].(Context) + ret1, _ := ret[1].(ReleaseCacheFunc) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// GetOrCreateCurrentWorkflowExecution indicates an expected call of GetOrCreateCurrentWorkflowExecution. +func (mr *MockCacheMockRecorder) GetOrCreateCurrentWorkflowExecution(ctx, namespaceID, workflowID interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOrCreateCurrentWorkflowExecution", reflect.TypeOf((*MockCache)(nil).GetOrCreateCurrentWorkflowExecution), ctx, namespaceID, workflowID) +} + +// GetOrCreateWorkflowExecution mocks base method. +func (m *MockCache) GetOrCreateWorkflowExecution(ctx context.Context, namespaceID namespace.ID, execution v1.WorkflowExecution, caller CallerType) (Context, ReleaseCacheFunc, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetOrCreateWorkflowExecution", ctx, namespaceID, execution, caller) + ret0, _ := ret[0].(Context) + ret1, _ := ret[1].(ReleaseCacheFunc) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// GetOrCreateWorkflowExecution indicates an expected call of GetOrCreateWorkflowExecution. +func (mr *MockCacheMockRecorder) GetOrCreateWorkflowExecution(ctx, namespaceID, execution, caller interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOrCreateWorkflowExecution", reflect.TypeOf((*MockCache)(nil).GetOrCreateWorkflowExecution), ctx, namespaceID, execution, caller) +} diff --git a/service/history/workflow/delete_manager.go b/service/history/workflow/delete_manager.go new file mode 100644 index 00000000000..258862797be --- /dev/null +++ b/service/history/workflow/delete_manager.go @@ -0,0 +1,243 @@ +//go:generate mockgen -copyright_file ../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination delete_manager_mock.go + +package workflow + +import ( + "context" + "errors" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + + "go.temporal.io/server/common" + "go.temporal.io/server/common/definition" + "go.temporal.io/server/common/metrics" + "go.temporal.io/server/common/namespace" + "go.temporal.io/server/common/searchattribute" + "go.temporal.io/server/service/history/configs" + "go.temporal.io/server/service/history/consts" + "go.temporal.io/server/service/history/shard" + "go.temporal.io/server/service/worker/archiver" +) + +type ( + DeleteManager interface { + DeleteWorkflowExecution(ctx context.Context, namespaceID namespace.ID, we commonpb.WorkflowExecution, archiveIfEnabled bool) (retError error) + DeleteWorkflowExecutionFromTimerTask(namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64) (retError error) + } + + DeleteManagerImpl struct { + shard shard.Context + historyCache Cache + config *configs.Config + metricsClient metrics.Client + archivalClient archiver.Client + } +) + +var _ DeleteManager = (*DeleteManagerImpl)(nil) + +func NewDeleteManager( + shard shard.Context, + cache Cache, + config *configs.Config, + archiverClient archiver.Client, +) *DeleteManagerImpl { + deleteManager := &DeleteManagerImpl{ + shard: shard, + historyCache: cache, + metricsClient: shard.GetMetricsClient(), + config: config, + archivalClient: archiverClient, + } + + return deleteManager +} + +func (e *DeleteManagerImpl) DeleteWorkflowExecution( + ctx context.Context, + namespaceID namespace.ID, + we commonpb.WorkflowExecution, + archiveIfEnabled bool, +) (retError error) { + + weCtx, release, err := e.historyCache.GetOrCreateWorkflowExecution( + ctx, + namespaceID, + we, + CallerTypeAPI, + ) + if err != nil { + return err + } + defer func() { release(retError) }() + + mutableState, err := weCtx.LoadWorkflowExecution() + if err != nil { + return err + } + + return e.deleteWorkflowExecutionInternal( + namespaceID, + we, + weCtx, + mutableState, + mutableState.GetCurrentVersion(), + archiveIfEnabled, + e.metricsClient.Scope(metrics.HistoryDeleteWorkflowExecutionScope), + ) +} + +func (e *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( + namespaceID namespace.ID, + we commonpb.WorkflowExecution, + weCtx Context, + ms MutableState, + timerTaskVersion int64, +) (retError error) { + + err := e.deleteWorkflowExecutionInternal( + namespaceID, + we, + weCtx, + ms, + timerTaskVersion, + true, + e.metricsClient.Scope(metrics.HistoryProcessDeleteHistoryEventScope), + ) + + if err != nil && errors.Is(err, consts.ErrWorkflowIsRunning) { + // If workflow is running then just ignore DeleteHistoryEventTask timer task. + // This should almost never happen because DeleteHistoryEventTask is created only for closed workflows. + // But cross DC replication can resurrect workflow and therefore DeleteHistoryEventTask should be ignored. + return nil + } + + return err +} + +func (e *DeleteManagerImpl) deleteWorkflowExecutionInternal( + namespaceID namespace.ID, + we commonpb.WorkflowExecution, + weCtx Context, + ms MutableState, + deleteVisibilityTaskVersion int64, + archiveIfEnabled bool, + scope metrics.Scope, +) error { + + if ms.IsWorkflowExecutionRunning() { + return consts.ErrWorkflowIsRunning + } + + currentBranchToken, err := ms.GetCurrentBranchToken() + if err != nil { + return err + } + + shouldDeleteHistory := true + if archiveIfEnabled { + shouldDeleteHistory, err = e.archiveWorkflowIfEnabled(namespaceID, we, currentBranchToken, weCtx, ms, scope) + if err != nil { + return err + } + } + + if !shouldDeleteHistory { + // currentBranchToken == nil means don't delete history. + currentBranchToken = nil + } + if err := e.shard.DeleteWorkflowExecution( + definition.WorkflowKey{ + NamespaceID: namespaceID.String(), + WorkflowID: we.GetWorkflowId(), + RunID: we.GetRunId(), + }, + currentBranchToken, + deleteVisibilityTaskVersion, + ); err != nil { + return err + } + + // Clear workflow execution context here to prevent further readers to get stale copy of non-exiting workflow execution. + weCtx.Clear() + + scope.IncCounter(metrics.WorkflowCleanupDeleteCount) + return nil +} + +func (e *DeleteManagerImpl) archiveWorkflowIfEnabled( + namespaceID namespace.ID, + workflowExecution commonpb.WorkflowExecution, + currentBranchToken []byte, + weCtx Context, + mutableState MutableState, + scope metrics.Scope, +) (bool, error) { + + namespaceRegistryEntry, err := e.shard.GetNamespaceRegistry().GetNamespaceByID(namespaceID) + if err != nil { + return false, err + } + clusterConfiguredForHistoryArchival := e.shard.GetArchivalMetadata().GetHistoryConfig().ClusterConfiguredForArchival() + namespaceConfiguredForHistoryArchival := namespaceRegistryEntry.HistoryArchivalState().State == enumspb.ARCHIVAL_STATE_ENABLED + archiveHistory := clusterConfiguredForHistoryArchival && namespaceConfiguredForHistoryArchival + + // TODO: @ycyang once archival backfill is in place cluster:paused && namespace:enabled should be a nop rather than a delete + if !archiveHistory { + return true, nil + } + + closeFailoverVersion, err := mutableState.GetLastWriteVersion() + if err != nil { + return false, err + } + + req := &archiver.ClientRequest{ + ArchiveRequest: &archiver.ArchiveRequest{ + NamespaceID: namespaceID.String(), + WorkflowID: workflowExecution.GetWorkflowId(), + RunID: workflowExecution.GetRunId(), + Namespace: namespaceRegistryEntry.Name().String(), + ShardID: e.shard.GetShardID(), + Targets: []archiver.ArchivalTarget{archiver.ArchiveTargetHistory}, + HistoryURI: namespaceRegistryEntry.HistoryArchivalState().URI, + NextEventID: mutableState.GetNextEventID(), + BranchToken: currentBranchToken, + CloseFailoverVersion: closeFailoverVersion, + }, + CallerService: common.HistoryServiceName, + AttemptArchiveInline: false, // archive in workflow by default + } + executionStats, err := weCtx.LoadExecutionStats() + if err == nil && executionStats.HistorySize < int64(e.config.TimerProcessorHistoryArchivalSizeLimit()) { + req.AttemptArchiveInline = true + } + + saTypeMap, err := e.shard.GetSearchAttributesProvider().GetSearchAttributes(e.config.DefaultVisibilityIndexName, false) + if err != nil { + return false, err + } + // Setting search attributes types here because archival client needs to stringify them, + // and it might not have access to typeMap (i.e. type needs to be embedded). + searchattribute.ApplyTypeMap(req.ArchiveRequest.SearchAttributes, saTypeMap) + + ctx, cancel := context.WithTimeout(context.Background(), e.config.TimerProcessorArchivalTimeLimit()) + defer cancel() + resp, err := e.archivalClient.Archive(ctx, req) + if err != nil { + return false, err + } + + var deleteHistory bool + if resp.HistoryArchivedInline { + scope.IncCounter(metrics.WorkflowCleanupDeleteHistoryInlineCount) + deleteHistory = true + } else { + scope.IncCounter(metrics.WorkflowCleanupArchiveCount) + // Don't delete workflow history if it wasn't achieve inline because archival workflow will need it. + deleteHistory = false + } + + return deleteHistory, nil +} diff --git a/service/history/workflow/delete_manager_mock.go b/service/history/workflow/delete_manager_mock.go new file mode 100644 index 00000000000..a1754390d8d --- /dev/null +++ b/service/history/workflow/delete_manager_mock.go @@ -0,0 +1,89 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by MockGen. DO NOT EDIT. +// Source: delete_manager.go + +// Package workflow is a generated GoMock package. +package workflow + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + v1 "go.temporal.io/api/common/v1" + namespace "go.temporal.io/server/common/namespace" +) + +// MockDeleteManager is a mock of DeleteManager interface. +type MockDeleteManager struct { + ctrl *gomock.Controller + recorder *MockDeleteManagerMockRecorder +} + +// MockDeleteManagerMockRecorder is the mock recorder for MockDeleteManager. +type MockDeleteManagerMockRecorder struct { + mock *MockDeleteManager +} + +// NewMockDeleteManager creates a new mock instance. +func NewMockDeleteManager(ctrl *gomock.Controller) *MockDeleteManager { + mock := &MockDeleteManager{ctrl: ctrl} + mock.recorder = &MockDeleteManagerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockDeleteManager) EXPECT() *MockDeleteManagerMockRecorder { + return m.recorder +} + +// DeleteWorkflowExecution mocks base method. +func (m *MockDeleteManager) DeleteWorkflowExecution(ctx context.Context, namespaceID namespace.ID, we v1.WorkflowExecution, archiveIfEnabled bool) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteWorkflowExecution", ctx, namespaceID, we, archiveIfEnabled) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. +func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecution(ctx, namespaceID, we, archiveIfEnabled interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockDeleteManager)(nil).DeleteWorkflowExecution), ctx, namespaceID, we, archiveIfEnabled) +} + +// DeleteWorkflowExecutionFromTimerTask mocks base method. +func (m *MockDeleteManager) DeleteWorkflowExecutionFromTimerTask(namespaceID namespace.ID, we v1.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteWorkflowExecutionFromTimerTask", namespaceID, we, weCtx, ms, timerTaskVersion) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteWorkflowExecutionFromTimerTask indicates an expected call of DeleteWorkflowExecutionFromTimerTask. +func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecutionFromTimerTask(namespaceID, we, weCtx, ms, timerTaskVersion interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecutionFromTimerTask", reflect.TypeOf((*MockDeleteManager)(nil).DeleteWorkflowExecutionFromTimerTask), namespaceID, we, weCtx, ms, timerTaskVersion) +} diff --git a/service/history/workflow/delete_manager_test.go b/service/history/workflow/delete_manager_test.go new file mode 100644 index 00000000000..726da49f44e --- /dev/null +++ b/service/history/workflow/delete_manager_test.go @@ -0,0 +1,254 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package workflow + +// +// import ( +// "errors" +// "testing" +// "time" +// +// "github.com/golang/mock/gomock" +// "github.com/stretchr/testify/require" +// "github.com/stretchr/testify/suite" +// +// "go.temporal.io/server/common/definition" +// "go.temporal.io/server/common/searchattribute" +// "go.temporal.io/server/service/history/tasks" +// "go.temporal.io/server/service/history/workflow" +// +// persistencespb "go.temporal.io/server/api/persistence/v1" +// "go.temporal.io/server/common" +// "go.temporal.io/server/common/cluster" +// "go.temporal.io/server/common/namespace" +// "go.temporal.io/server/common/persistence" +// "go.temporal.io/server/service/history/shard" +// "go.temporal.io/server/service/history/tests" +// "go.temporal.io/server/service/worker/archiver" +// ) +// +// type ( +// historyEngineDeleteWorkflowSuite struct { +// suite.Suite +// *require.Assertions +// +// controller *gomock.Controller +// mockShard *shard.ContextTest +// mockEngine *shard.MockEngine +// mockWorkflowExecutionContext *workflow.MockContext +// mockMutableState *workflow.MockMutableState +// +// mockExecutionManager *persistence.MockExecutionManager +// mockArchivalClient *archiver.MockClient +// mockNamespaceCache *namespace.MockRegistry +// mockClusterMetadata *cluster.MockMetadata +// mockSearchAttributesProvider *searchattribute.MockProvider +// +// timerQueueTaskExecutorBase *timerQueueTaskExecutorBase +// } +// ) +// +// func TestHistoryEngineDeleteWorkflowSuite(t *testing.T) { +// s := new(timerQueueTaskExecutorBaseSuite) +// suite.Run(t, s) +// } +// +// func (s *historyEngineDeleteWorkflowSuite) SetupSuite() { +// +// } +// +// func (s *historyEngineDeleteWorkflowSuite) TearDownSuite() { +// +// } +// +// func (s *historyEngineDeleteWorkflowSuite) SetupTest() { +// s.Assertions = require.New(s.T()) +// +// s.controller = gomock.NewController(s.T()) +// s.mockWorkflowExecutionContext = workflow.NewMockContext(s.controller) +// s.mockMutableState = workflow.NewMockMutableState(s.controller) +// +// config := tests.NewDynamicConfig() +// s.mockShard = shard.NewTestContext( +// s.controller, +// &persistence.ShardInfoWithFailover{ +// ShardInfo: &persistencespb.ShardInfo{ +// ShardId: 0, +// RangeId: 1, +// TransferAckLevel: 0, +// }}, +// config, +// ) +// s.mockEngine = shard.NewMockEngine(s.controller) +// s.mockShard.SetEngineForTesting(s.mockEngine) +// +// s.mockExecutionManager = s.mockShard.Resource.ExecutionMgr +// s.mockArchivalClient = archiver.NewMockClient(s.controller) +// s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache +// s.mockClusterMetadata = s.mockShard.Resource.ClusterMetadata +// s.mockSearchAttributesProvider = s.mockShard.Resource.SearchAttributesProvider +// +// logger := s.mockShard.GetLogger() +// +// h := &historyEngineImpl{ +// shard: s.mockShard, +// logger: logger, +// metricsClient: s.mockShard.GetMetricsClient(), +// archivalClient: s.mockArchivalClient, +// clusterMetadata: s.mockClusterMetadata, +// } +// +// s.timerQueueTaskExecutorBase = newTimerQueueTaskExecutorBase( +// s.mockShard, +// h, +// logger, +// s.mockShard.GetMetricsClient(), +// config, +// ) +// } +// +// func (s *historyEngineDeleteWorkflowSuite) TearDownTest() { +// s.controller.Finish() +// } +// +// func (s *historyEngineDeleteWorkflowSuite) TestDeleteWorkflow_NoErr() { +// task := &tasks.DeleteHistoryEventTask{ +// WorkflowKey: definition.NewWorkflowKey( +// tests.NamespaceID.String(), +// tests.WorkflowID, +// tests.RunID, +// ), +// Version: 123, +// TaskID: 12345, +// VisibilityTimestamp: time.Now().UTC(), +// } +// +// s.mockWorkflowExecutionContext.EXPECT().Clear() +// +// s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() +// s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() +// s.mockExecutionManager.EXPECT().AddTasks(gomock.Any()).Return(nil) +// s.mockEngine.EXPECT().NotifyNewTransferTasks(gomock.Any(), gomock.Any()) +// s.mockEngine.EXPECT().NotifyNewTimerTasks(gomock.Any(), gomock.Any()) +// s.mockEngine.EXPECT().NotifyNewVisibilityTasks(gomock.Any()) +// s.mockEngine.EXPECT().NotifyNewReplicationTasks(gomock.Any()) +// +// s.mockExecutionManager.EXPECT().DeleteCurrentWorkflowExecution(gomock.Any()).Return(nil) +// s.mockExecutionManager.EXPECT().DeleteWorkflowExecution(gomock.Any()).Return(nil) +// s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any()).Return(nil) +// s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) +// +// err := s.timerQueueTaskExecutorBase.deleteWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState) +// s.NoError(err) +// } +// +// func (s *historyEngineDeleteWorkflowSuite) TestArchiveHistory_NoErr_InlineArchivalFailed() { +// task := &tasks.DeleteHistoryEventTask{ +// WorkflowKey: definition.NewWorkflowKey( +// tests.NamespaceID.String(), +// tests.WorkflowID, +// tests.RunID, +// ), +// Version: 123, +// TaskID: 12345, +// VisibilityTimestamp: time.Now().UTC(), +// } +// +// s.mockWorkflowExecutionContext.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ +// HistorySize: 1024, +// }, nil) +// s.mockWorkflowExecutionContext.EXPECT().Clear() +// s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) +// s.mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1234), nil) +// s.mockMutableState.EXPECT().GetNextEventID().Return(int64(101)) +// +// s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() +// s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() +// s.mockExecutionManager.EXPECT().AddTasks(gomock.Any()).Return(nil) +// s.mockEngine.EXPECT().NotifyNewTransferTasks(gomock.Any(), gomock.Any()) +// s.mockEngine.EXPECT().NotifyNewTimerTasks(gomock.Any(), gomock.Any()) +// s.mockEngine.EXPECT().NotifyNewVisibilityTasks(gomock.Any()) +// s.mockEngine.EXPECT().NotifyNewReplicationTasks(gomock.Any()) +// +// s.mockExecutionManager.EXPECT().DeleteCurrentWorkflowExecution(gomock.Any()).Return(nil) +// s.mockExecutionManager.EXPECT().DeleteWorkflowExecution(gomock.Any()).Return(nil) +// +// s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: true}). +// Return(&archiver.ClientResponse{ +// HistoryArchivedInline: false, +// }, nil) +// +// s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false) +// +// namespaceRegistryEntry := namespace.NewNamespaceForTest(&persistencespb.NamespaceInfo{}, &persistencespb.NamespaceConfig{}, false, nil, 0) +// err := s.timerQueueTaskExecutorBase.archiveWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState, namespaceRegistryEntry) +// s.NoError(err) +// } +// +// func (s *historyEngineDeleteWorkflowSuite) TestArchiveHistory_SendSignalErr() { +// task := &tasks.DeleteHistoryEventTask{ +// WorkflowKey: definition.NewWorkflowKey( +// tests.NamespaceID.String(), +// tests.WorkflowID, +// tests.RunID, +// ), +// Version: 123, +// TaskID: 12345, +// VisibilityTimestamp: time.Now().UTC(), +// } +// +// s.mockWorkflowExecutionContext.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ +// HistorySize: 1024 * 1024 * 1024, +// }, nil) +// +// s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) +// s.mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1234), nil) +// s.mockMutableState.EXPECT().GetNextEventID().Return(int64(101)) +// +// s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: false}). +// Return(nil, errors.New("failed to send signal")) +// s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false) +// +// namespaceRegistryEntry := namespace.NewNamespaceForTest(&persistencespb.NamespaceInfo{}, &persistencespb.NamespaceConfig{}, false, nil, 0) +// err := s.timerQueueTaskExecutorBase.archiveWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState, namespaceRegistryEntry) +// s.Error(err) +// } +// +// type ( +// archiverClientRequestMatcher struct { +// inline bool +// } +// ) +// +// func (m archiverClientRequestMatcher) Matches(x interface{}) bool { +// req := x.(*archiver.ClientRequest) +// return req.CallerService == common.HistoryServiceName && +// req.AttemptArchiveInline == m.inline && +// req.ArchiveRequest.Targets[0] == archiver.ArchiveTargetHistory +// } +// +// func (m archiverClientRequestMatcher) String() string { +// return "archiverClientRequestMatcher" +// } From 84e5743790e979c79e079669da040c29a16acfe4 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Thu, 16 Dec 2021 16:18:19 -0800 Subject: [PATCH 03/12] Refactor unit tests --- common/metrics/defs.go | 1 + service/history/workflow/delete_manager.go | 60 +- .../history/workflow/delete_manager_test.go | 512 ++++++++++-------- 3 files changed, 327 insertions(+), 246 deletions(-) diff --git a/common/metrics/defs.go b/common/metrics/defs.go index 8bd1d1fa0ca..33969c637f1 100644 --- a/common/metrics/defs.go +++ b/common/metrics/defs.go @@ -1588,6 +1588,7 @@ var ScopeDefs = map[ServiceIdx]map[int]scopeDefinition{ HistoryResetWorkflowExecutionScope: {operation: "ResetWorkflowExecution"}, HistoryQueryWorkflowScope: {operation: "QueryWorkflow"}, HistoryProcessDeleteHistoryEventScope: {operation: "ProcessDeleteHistoryEvent"}, + HistoryDeleteWorkflowExecutionScope: {operation: "DeleteWorkflowExecution"}, HistoryScheduleWorkflowTaskScope: {operation: "ScheduleWorkflowTask"}, HistoryRecordChildExecutionCompletedScope: {operation: "RecordChildExecutionCompleted"}, HistoryRequestCancelWorkflowExecutionScope: {operation: "RequestCancelWorkflowExecution"}, diff --git a/service/history/workflow/delete_manager.go b/service/history/workflow/delete_manager.go index 258862797be..7a1535fd0a1 100644 --- a/service/history/workflow/delete_manager.go +++ b/service/history/workflow/delete_manager.go @@ -1,3 +1,27 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + //go:generate mockgen -copyright_file ../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination delete_manager_mock.go package workflow @@ -54,14 +78,14 @@ func NewDeleteManager( return deleteManager } -func (e *DeleteManagerImpl) DeleteWorkflowExecution( +func (m *DeleteManagerImpl) DeleteWorkflowExecution( ctx context.Context, namespaceID namespace.ID, we commonpb.WorkflowExecution, archiveIfEnabled bool, ) (retError error) { - weCtx, release, err := e.historyCache.GetOrCreateWorkflowExecution( + weCtx, release, err := m.historyCache.GetOrCreateWorkflowExecution( ctx, namespaceID, we, @@ -77,18 +101,18 @@ func (e *DeleteManagerImpl) DeleteWorkflowExecution( return err } - return e.deleteWorkflowExecutionInternal( + return m.deleteWorkflowExecutionInternal( namespaceID, we, weCtx, mutableState, mutableState.GetCurrentVersion(), archiveIfEnabled, - e.metricsClient.Scope(metrics.HistoryDeleteWorkflowExecutionScope), + m.metricsClient.Scope(metrics.HistoryDeleteWorkflowExecutionScope), ) } -func (e *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( +func (m *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, @@ -96,14 +120,14 @@ func (e *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( timerTaskVersion int64, ) (retError error) { - err := e.deleteWorkflowExecutionInternal( + err := m.deleteWorkflowExecutionInternal( namespaceID, we, weCtx, ms, timerTaskVersion, true, - e.metricsClient.Scope(metrics.HistoryProcessDeleteHistoryEventScope), + m.metricsClient.Scope(metrics.HistoryProcessDeleteHistoryEventScope), ) if err != nil && errors.Is(err, consts.ErrWorkflowIsRunning) { @@ -116,7 +140,7 @@ func (e *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( return err } -func (e *DeleteManagerImpl) deleteWorkflowExecutionInternal( +func (m *DeleteManagerImpl) deleteWorkflowExecutionInternal( namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, @@ -137,7 +161,7 @@ func (e *DeleteManagerImpl) deleteWorkflowExecutionInternal( shouldDeleteHistory := true if archiveIfEnabled { - shouldDeleteHistory, err = e.archiveWorkflowIfEnabled(namespaceID, we, currentBranchToken, weCtx, ms, scope) + shouldDeleteHistory, err = m.archiveWorkflowIfEnabled(namespaceID, we, currentBranchToken, weCtx, ms, scope) if err != nil { return err } @@ -147,7 +171,7 @@ func (e *DeleteManagerImpl) deleteWorkflowExecutionInternal( // currentBranchToken == nil means don't delete history. currentBranchToken = nil } - if err := e.shard.DeleteWorkflowExecution( + if err := m.shard.DeleteWorkflowExecution( definition.WorkflowKey{ NamespaceID: namespaceID.String(), WorkflowID: we.GetWorkflowId(), @@ -166,7 +190,7 @@ func (e *DeleteManagerImpl) deleteWorkflowExecutionInternal( return nil } -func (e *DeleteManagerImpl) archiveWorkflowIfEnabled( +func (m *DeleteManagerImpl) archiveWorkflowIfEnabled( namespaceID namespace.ID, workflowExecution commonpb.WorkflowExecution, currentBranchToken []byte, @@ -175,11 +199,11 @@ func (e *DeleteManagerImpl) archiveWorkflowIfEnabled( scope metrics.Scope, ) (bool, error) { - namespaceRegistryEntry, err := e.shard.GetNamespaceRegistry().GetNamespaceByID(namespaceID) + namespaceRegistryEntry, err := m.shard.GetNamespaceRegistry().GetNamespaceByID(namespaceID) if err != nil { return false, err } - clusterConfiguredForHistoryArchival := e.shard.GetArchivalMetadata().GetHistoryConfig().ClusterConfiguredForArchival() + clusterConfiguredForHistoryArchival := m.shard.GetArchivalMetadata().GetHistoryConfig().ClusterConfiguredForArchival() namespaceConfiguredForHistoryArchival := namespaceRegistryEntry.HistoryArchivalState().State == enumspb.ARCHIVAL_STATE_ENABLED archiveHistory := clusterConfiguredForHistoryArchival && namespaceConfiguredForHistoryArchival @@ -199,7 +223,7 @@ func (e *DeleteManagerImpl) archiveWorkflowIfEnabled( WorkflowID: workflowExecution.GetWorkflowId(), RunID: workflowExecution.GetRunId(), Namespace: namespaceRegistryEntry.Name().String(), - ShardID: e.shard.GetShardID(), + ShardID: m.shard.GetShardID(), Targets: []archiver.ArchivalTarget{archiver.ArchiveTargetHistory}, HistoryURI: namespaceRegistryEntry.HistoryArchivalState().URI, NextEventID: mutableState.GetNextEventID(), @@ -210,11 +234,11 @@ func (e *DeleteManagerImpl) archiveWorkflowIfEnabled( AttemptArchiveInline: false, // archive in workflow by default } executionStats, err := weCtx.LoadExecutionStats() - if err == nil && executionStats.HistorySize < int64(e.config.TimerProcessorHistoryArchivalSizeLimit()) { + if err == nil && executionStats.HistorySize < int64(m.config.TimerProcessorHistoryArchivalSizeLimit()) { req.AttemptArchiveInline = true } - saTypeMap, err := e.shard.GetSearchAttributesProvider().GetSearchAttributes(e.config.DefaultVisibilityIndexName, false) + saTypeMap, err := m.shard.GetSearchAttributesProvider().GetSearchAttributes(m.config.DefaultVisibilityIndexName, false) if err != nil { return false, err } @@ -222,9 +246,9 @@ func (e *DeleteManagerImpl) archiveWorkflowIfEnabled( // and it might not have access to typeMap (i.e. type needs to be embedded). searchattribute.ApplyTypeMap(req.ArchiveRequest.SearchAttributes, saTypeMap) - ctx, cancel := context.WithTimeout(context.Background(), e.config.TimerProcessorArchivalTimeLimit()) + ctx, cancel := context.WithTimeout(context.Background(), m.config.TimerProcessorArchivalTimeLimit()) defer cancel() - resp, err := e.archivalClient.Archive(ctx, req) + resp, err := m.archivalClient.Archive(ctx, req) if err != nil { return false, err } diff --git a/service/history/workflow/delete_manager_test.go b/service/history/workflow/delete_manager_test.go index 726da49f44e..ec633e8340a 100644 --- a/service/history/workflow/delete_manager_test.go +++ b/service/history/workflow/delete_manager_test.go @@ -24,231 +24,287 @@ package workflow -// -// import ( -// "errors" -// "testing" -// "time" -// -// "github.com/golang/mock/gomock" -// "github.com/stretchr/testify/require" -// "github.com/stretchr/testify/suite" -// -// "go.temporal.io/server/common/definition" -// "go.temporal.io/server/common/searchattribute" -// "go.temporal.io/server/service/history/tasks" -// "go.temporal.io/server/service/history/workflow" -// -// persistencespb "go.temporal.io/server/api/persistence/v1" -// "go.temporal.io/server/common" -// "go.temporal.io/server/common/cluster" -// "go.temporal.io/server/common/namespace" -// "go.temporal.io/server/common/persistence" -// "go.temporal.io/server/service/history/shard" -// "go.temporal.io/server/service/history/tests" -// "go.temporal.io/server/service/worker/archiver" -// ) -// -// type ( -// historyEngineDeleteWorkflowSuite struct { -// suite.Suite -// *require.Assertions -// -// controller *gomock.Controller -// mockShard *shard.ContextTest -// mockEngine *shard.MockEngine -// mockWorkflowExecutionContext *workflow.MockContext -// mockMutableState *workflow.MockMutableState -// -// mockExecutionManager *persistence.MockExecutionManager -// mockArchivalClient *archiver.MockClient -// mockNamespaceCache *namespace.MockRegistry -// mockClusterMetadata *cluster.MockMetadata -// mockSearchAttributesProvider *searchattribute.MockProvider -// -// timerQueueTaskExecutorBase *timerQueueTaskExecutorBase -// } -// ) -// -// func TestHistoryEngineDeleteWorkflowSuite(t *testing.T) { -// s := new(timerQueueTaskExecutorBaseSuite) -// suite.Run(t, s) -// } -// -// func (s *historyEngineDeleteWorkflowSuite) SetupSuite() { -// -// } -// -// func (s *historyEngineDeleteWorkflowSuite) TearDownSuite() { -// -// } -// -// func (s *historyEngineDeleteWorkflowSuite) SetupTest() { -// s.Assertions = require.New(s.T()) -// -// s.controller = gomock.NewController(s.T()) -// s.mockWorkflowExecutionContext = workflow.NewMockContext(s.controller) -// s.mockMutableState = workflow.NewMockMutableState(s.controller) -// -// config := tests.NewDynamicConfig() -// s.mockShard = shard.NewTestContext( -// s.controller, -// &persistence.ShardInfoWithFailover{ -// ShardInfo: &persistencespb.ShardInfo{ -// ShardId: 0, -// RangeId: 1, -// TransferAckLevel: 0, -// }}, -// config, -// ) -// s.mockEngine = shard.NewMockEngine(s.controller) -// s.mockShard.SetEngineForTesting(s.mockEngine) -// -// s.mockExecutionManager = s.mockShard.Resource.ExecutionMgr -// s.mockArchivalClient = archiver.NewMockClient(s.controller) -// s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache -// s.mockClusterMetadata = s.mockShard.Resource.ClusterMetadata -// s.mockSearchAttributesProvider = s.mockShard.Resource.SearchAttributesProvider -// -// logger := s.mockShard.GetLogger() -// -// h := &historyEngineImpl{ -// shard: s.mockShard, -// logger: logger, -// metricsClient: s.mockShard.GetMetricsClient(), -// archivalClient: s.mockArchivalClient, -// clusterMetadata: s.mockClusterMetadata, -// } -// -// s.timerQueueTaskExecutorBase = newTimerQueueTaskExecutorBase( -// s.mockShard, -// h, -// logger, -// s.mockShard.GetMetricsClient(), -// config, -// ) -// } -// -// func (s *historyEngineDeleteWorkflowSuite) TearDownTest() { -// s.controller.Finish() -// } -// -// func (s *historyEngineDeleteWorkflowSuite) TestDeleteWorkflow_NoErr() { -// task := &tasks.DeleteHistoryEventTask{ -// WorkflowKey: definition.NewWorkflowKey( -// tests.NamespaceID.String(), -// tests.WorkflowID, -// tests.RunID, -// ), -// Version: 123, -// TaskID: 12345, -// VisibilityTimestamp: time.Now().UTC(), -// } -// -// s.mockWorkflowExecutionContext.EXPECT().Clear() -// -// s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() -// s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() -// s.mockExecutionManager.EXPECT().AddTasks(gomock.Any()).Return(nil) -// s.mockEngine.EXPECT().NotifyNewTransferTasks(gomock.Any(), gomock.Any()) -// s.mockEngine.EXPECT().NotifyNewTimerTasks(gomock.Any(), gomock.Any()) -// s.mockEngine.EXPECT().NotifyNewVisibilityTasks(gomock.Any()) -// s.mockEngine.EXPECT().NotifyNewReplicationTasks(gomock.Any()) -// -// s.mockExecutionManager.EXPECT().DeleteCurrentWorkflowExecution(gomock.Any()).Return(nil) -// s.mockExecutionManager.EXPECT().DeleteWorkflowExecution(gomock.Any()).Return(nil) -// s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any()).Return(nil) -// s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) -// -// err := s.timerQueueTaskExecutorBase.deleteWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState) -// s.NoError(err) -// } -// -// func (s *historyEngineDeleteWorkflowSuite) TestArchiveHistory_NoErr_InlineArchivalFailed() { -// task := &tasks.DeleteHistoryEventTask{ -// WorkflowKey: definition.NewWorkflowKey( -// tests.NamespaceID.String(), -// tests.WorkflowID, -// tests.RunID, -// ), -// Version: 123, -// TaskID: 12345, -// VisibilityTimestamp: time.Now().UTC(), -// } -// -// s.mockWorkflowExecutionContext.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ -// HistorySize: 1024, -// }, nil) -// s.mockWorkflowExecutionContext.EXPECT().Clear() -// s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) -// s.mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1234), nil) -// s.mockMutableState.EXPECT().GetNextEventID().Return(int64(101)) -// -// s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() -// s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() -// s.mockExecutionManager.EXPECT().AddTasks(gomock.Any()).Return(nil) -// s.mockEngine.EXPECT().NotifyNewTransferTasks(gomock.Any(), gomock.Any()) -// s.mockEngine.EXPECT().NotifyNewTimerTasks(gomock.Any(), gomock.Any()) -// s.mockEngine.EXPECT().NotifyNewVisibilityTasks(gomock.Any()) -// s.mockEngine.EXPECT().NotifyNewReplicationTasks(gomock.Any()) -// -// s.mockExecutionManager.EXPECT().DeleteCurrentWorkflowExecution(gomock.Any()).Return(nil) -// s.mockExecutionManager.EXPECT().DeleteWorkflowExecution(gomock.Any()).Return(nil) -// -// s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: true}). -// Return(&archiver.ClientResponse{ -// HistoryArchivedInline: false, -// }, nil) -// -// s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false) -// -// namespaceRegistryEntry := namespace.NewNamespaceForTest(&persistencespb.NamespaceInfo{}, &persistencespb.NamespaceConfig{}, false, nil, 0) -// err := s.timerQueueTaskExecutorBase.archiveWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState, namespaceRegistryEntry) -// s.NoError(err) -// } -// -// func (s *historyEngineDeleteWorkflowSuite) TestArchiveHistory_SendSignalErr() { -// task := &tasks.DeleteHistoryEventTask{ -// WorkflowKey: definition.NewWorkflowKey( -// tests.NamespaceID.String(), -// tests.WorkflowID, -// tests.RunID, -// ), -// Version: 123, -// TaskID: 12345, -// VisibilityTimestamp: time.Now().UTC(), -// } -// -// s.mockWorkflowExecutionContext.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ -// HistorySize: 1024 * 1024 * 1024, -// }, nil) -// -// s.mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{1, 2, 3}, nil) -// s.mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1234), nil) -// s.mockMutableState.EXPECT().GetNextEventID().Return(int64(101)) -// -// s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: false}). -// Return(nil, errors.New("failed to send signal")) -// s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false) -// -// namespaceRegistryEntry := namespace.NewNamespaceForTest(&persistencespb.NamespaceInfo{}, &persistencespb.NamespaceConfig{}, false, nil, 0) -// err := s.timerQueueTaskExecutorBase.archiveWorkflow(task, s.mockWorkflowExecutionContext, s.mockMutableState, namespaceRegistryEntry) -// s.Error(err) -// } -// -// type ( -// archiverClientRequestMatcher struct { -// inline bool -// } -// ) -// -// func (m archiverClientRequestMatcher) Matches(x interface{}) bool { -// req := x.(*archiver.ClientRequest) -// return req.CallerService == common.HistoryServiceName && -// req.AttemptArchiveInline == m.inline && -// req.ArchiveRequest.Targets[0] == archiver.ArchiveTargetHistory -// } -// -// func (m archiverClientRequestMatcher) String() string { -// return "archiverClientRequestMatcher" -// } +import ( + "context" + "errors" + "testing" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/enums/v1" + "go.temporal.io/api/serviceerror" + + persistencespb "go.temporal.io/server/api/persistence/v1" + "go.temporal.io/server/common" + carchiver "go.temporal.io/server/common/archiver" + "go.temporal.io/server/common/definition" + "go.temporal.io/server/common/metrics" + "go.temporal.io/server/common/namespace" + "go.temporal.io/server/common/searchattribute" + "go.temporal.io/server/service/history/shard" + "go.temporal.io/server/service/history/tests" + "go.temporal.io/server/service/worker/archiver" +) + +type ( + deleteManagerWorkflowSuite struct { + suite.Suite + *require.Assertions + + controller *gomock.Controller + mockCache *MockCache + mockArchivalClient *archiver.MockClient + mockShardContext *shard.MockContext + + deleteManager DeleteManager + } +) + +func TestDeleteManagerSuite(t *testing.T) { + s := &deleteManagerWorkflowSuite{} + suite.Run(t, s) +} + +func (s *deleteManagerWorkflowSuite) SetupSuite() { + +} + +func (s *deleteManagerWorkflowSuite) TearDownSuite() { + +} + +func (s *deleteManagerWorkflowSuite) SetupTest() { + s.Assertions = require.New(s.T()) + + s.controller = gomock.NewController(s.T()) + s.mockCache = NewMockCache(s.controller) + s.mockArchivalClient = archiver.NewMockClient(s.controller) + + config := tests.NewDynamicConfig() + s.mockShardContext = shard.NewMockContext(s.controller) + s.mockShardContext.EXPECT().GetMetricsClient().Return(metrics.NewNoopMetricsClient()).AnyTimes() + + s.deleteManager = NewDeleteManager( + s.mockShardContext, + s.mockCache, + config, + s.mockArchivalClient, + ) +} + +func (s *deleteManagerWorkflowSuite) TearDownTest() { + s.controller.Finish() +} + +func (s *deleteManagerWorkflowSuite) TestDeleteWorkflow_NoArchival() { + we := commonpb.WorkflowExecution{ + WorkflowId: tests.WorkflowID, + RunId: tests.RunID, + } + + mockWeCtx := NewMockContext(s.controller) + mockMutableState := NewMockMutableState(s.controller) + + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, CallerTypeAPI).Return(mockWeCtx, NoopReleaseFn, nil) + mockWeCtx.EXPECT().LoadWorkflowExecution().Return(mockMutableState, nil) + mockMutableState.EXPECT().GetCurrentVersion().Return(int64(1)) + + mockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(false) + mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) + + s.mockShardContext.EXPECT().DeleteWorkflowExecution( + definition.WorkflowKey{ + NamespaceID: tests.NamespaceID.String(), + WorkflowID: tests.WorkflowID, + RunID: tests.RunID, + }, + []byte{22, 8, 78}, + int64(1), + ).Return(nil) + mockWeCtx.EXPECT().Clear() + + err := s.deleteManager.DeleteWorkflowExecution( + context.Background(), + tests.NamespaceID, + we, + false, + ) + s.NoError(err) +} + +func (s *deleteManagerWorkflowSuite) TestDeleteWorkflow_NoArchival_Error() { + we := commonpb.WorkflowExecution{ + WorkflowId: tests.WorkflowID, + RunId: tests.RunID, + } + + mockWeCtx := NewMockContext(s.controller) + mockMutableState := NewMockMutableState(s.controller) + + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, CallerTypeAPI).Return(mockWeCtx, NoopReleaseFn, nil) + mockWeCtx.EXPECT().LoadWorkflowExecution().Return(mockMutableState, nil) + mockMutableState.EXPECT().GetCurrentVersion().Return(int64(1)) + + mockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(false) + mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) + + s.mockShardContext.EXPECT().DeleteWorkflowExecution( + definition.WorkflowKey{ + NamespaceID: tests.NamespaceID.String(), + WorkflowID: tests.WorkflowID, + RunID: tests.RunID, + }, + []byte{22, 8, 78}, + int64(1), + ).Return(serviceerror.NewInternal("test error")) + + err := s.deleteManager.DeleteWorkflowExecution( + context.Background(), + tests.NamespaceID, + we, + false, + ) + s.Error(err) +} + +func (s *deleteManagerWorkflowSuite) TestDeleteWorkflow_Archival_NotInline() { + we := commonpb.WorkflowExecution{ + WorkflowId: tests.WorkflowID, + RunId: tests.RunID, + } + + mockWeCtx := NewMockContext(s.controller) + mockMutableState := NewMockMutableState(s.controller) + + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, CallerTypeAPI).Return(mockWeCtx, NoopReleaseFn, nil) + mockWeCtx.EXPECT().LoadWorkflowExecution().Return(mockMutableState, nil) + mockMutableState.EXPECT().GetCurrentVersion().Return(int64(1)) + + mockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(false) + mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) + + // ====================== Archival mocks ======================================= + mockNamespaceRegistry := namespace.NewMockRegistry(s.controller) + s.mockShardContext.EXPECT().GetNamespaceRegistry().Return(mockNamespaceRegistry) + mockNamespaceRegistry.EXPECT().GetNamespaceByID(tests.NamespaceID).Return(namespace.NewLocalNamespaceForTest( + &persistencespb.NamespaceInfo{ + Name: tests.Namespace.String(), + }, + &persistencespb.NamespaceConfig{ + HistoryArchivalState: enums.ARCHIVAL_STATE_ENABLED, + }, + "target-cluster", + ), nil) + + mockClusterArchivalMetadata := carchiver.NewMockArchivalMetadata(s.controller) + mockClusterArchivalConfig := carchiver.NewMockArchivalConfig(s.controller) + s.mockShardContext.EXPECT().GetArchivalMetadata().Return(mockClusterArchivalMetadata) + mockClusterArchivalMetadata.EXPECT().GetHistoryConfig().Return(mockClusterArchivalConfig) + mockClusterArchivalConfig.EXPECT().ClusterConfiguredForArchival().Return(true) + mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) + s.mockShardContext.EXPECT().GetShardID().Return(int32(1)) + mockMutableState.EXPECT().GetNextEventID().Return(int64(1)) + mockWeCtx.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ + HistorySize: 22, + }, nil) + mockSearchAttributesProvider := searchattribute.NewMockProvider(s.controller) + mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), gomock.Any()).Return(searchattribute.TestNameTypeMap, nil) + s.mockShardContext.EXPECT().GetSearchAttributesProvider().Return(mockSearchAttributesProvider) + s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: true}).Return(&archiver.ClientResponse{ + HistoryArchivedInline: false, + }, nil) + // ============================================================= + + s.mockShardContext.EXPECT().DeleteWorkflowExecution( + definition.WorkflowKey{ + NamespaceID: tests.NamespaceID.String(), + WorkflowID: tests.WorkflowID, + RunID: tests.RunID, + }, + nil, + int64(1), + ).Return(nil) + mockWeCtx.EXPECT().Clear() + + err := s.deleteManager.DeleteWorkflowExecution( + context.Background(), + tests.NamespaceID, + we, + true, + ) + s.NoError(err) +} + +func (s *deleteManagerWorkflowSuite) TestDeleteWorkflow_Archival_SendSignalErr() { + we := commonpb.WorkflowExecution{ + WorkflowId: tests.WorkflowID, + RunId: tests.RunID, + } + + mockWeCtx := NewMockContext(s.controller) + mockMutableState := NewMockMutableState(s.controller) + + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, CallerTypeAPI).Return(mockWeCtx, NoopReleaseFn, nil) + mockWeCtx.EXPECT().LoadWorkflowExecution().Return(mockMutableState, nil) + mockMutableState.EXPECT().GetCurrentVersion().Return(int64(1)) + + mockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(false) + mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) + + // ====================== Archival mocks ======================================= + mockNamespaceRegistry := namespace.NewMockRegistry(s.controller) + s.mockShardContext.EXPECT().GetNamespaceRegistry().Return(mockNamespaceRegistry) + mockNamespaceRegistry.EXPECT().GetNamespaceByID(tests.NamespaceID).Return(namespace.NewLocalNamespaceForTest( + &persistencespb.NamespaceInfo{ + Name: tests.Namespace.String(), + }, + &persistencespb.NamespaceConfig{ + HistoryArchivalState: enums.ARCHIVAL_STATE_ENABLED, + }, + "target-cluster", + ), nil) + + mockClusterArchivalMetadata := carchiver.NewMockArchivalMetadata(s.controller) + mockClusterArchivalConfig := carchiver.NewMockArchivalConfig(s.controller) + s.mockShardContext.EXPECT().GetArchivalMetadata().Return(mockClusterArchivalMetadata) + mockClusterArchivalMetadata.EXPECT().GetHistoryConfig().Return(mockClusterArchivalConfig) + mockClusterArchivalConfig.EXPECT().ClusterConfiguredForArchival().Return(true) + mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) + s.mockShardContext.EXPECT().GetShardID().Return(int32(1)) + mockMutableState.EXPECT().GetNextEventID().Return(int64(1)) + mockWeCtx.EXPECT().LoadExecutionStats().Return(&persistencespb.ExecutionStats{ + HistorySize: 22 * 1024 * 1024 * 1024, + }, nil) + mockSearchAttributesProvider := searchattribute.NewMockProvider(s.controller) + mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), gomock.Any()).Return(searchattribute.TestNameTypeMap, nil) + s.mockShardContext.EXPECT().GetSearchAttributesProvider().Return(mockSearchAttributesProvider) + s.mockArchivalClient.EXPECT().Archive(gomock.Any(), archiverClientRequestMatcher{inline: false}).Return(nil, errors.New("failed to send signal")) + // ============================================================= + + err := s.deleteManager.DeleteWorkflowExecution( + context.Background(), + tests.NamespaceID, + we, + true, + ) + s.Error(err) +} + +type ( + archiverClientRequestMatcher struct { + inline bool + } +) + +func (m archiverClientRequestMatcher) Matches(x interface{}) bool { + req := x.(*archiver.ClientRequest) + return req.CallerService == common.HistoryServiceName && + req.AttemptArchiveInline == m.inline && + req.ArchiveRequest.Targets[0] == archiver.ArchiveTargetHistory +} + +func (m archiverClientRequestMatcher) String() string { + return "archiverClientRequestMatcher" +} From 32c38c25d856904f5ca88e545e230aad4791781d Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Fri, 17 Dec 2021 12:40:31 -0800 Subject: [PATCH 04/12] Add DeleteWorkflowExecution API to history service --- api/historyservice/v1/request_response.pb.go | 1138 ++++++++++++----- api/historyservice/v1/service.pb.go | 184 ++- api/historyservicemock/v1/service.pb.mock.go | 35 + client/history/client.go | 24 + client/history/metricClient.go | 17 + client/history/retryableClient.go | 16 + common/metrics/defs.go | 3 + proto/api-linter.yaml | 4 + .../historyservice/v1/request_response.proto | 8 + .../api/historyservice/v1/service.proto | 7 +- service/history/handler.go | 28 + service/history/historyEngine.go | 12 + service/history/shard/engine.go | 1 + service/history/shard/engine_mock.go | 14 + 14 files changed, 1073 insertions(+), 418 deletions(-) diff --git a/api/historyservice/v1/request_response.pb.go b/api/historyservice/v1/request_response.pb.go index 7c4858ba34f..38a7a67f1be 100644 --- a/api/historyservice/v1/request_response.pb.go +++ b/api/historyservice/v1/request_response.pb.go @@ -2043,6 +2043,92 @@ func (m *TerminateWorkflowExecutionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_TerminateWorkflowExecutionResponse proto.InternalMessageInfo +type DeleteWorkflowExecutionRequest struct { + NamespaceId string `protobuf:"bytes,1,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v14.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` +} + +func (m *DeleteWorkflowExecutionRequest) Reset() { *m = DeleteWorkflowExecutionRequest{} } +func (*DeleteWorkflowExecutionRequest) ProtoMessage() {} +func (*DeleteWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_b8c78c1d460a3711, []int{32} +} +func (m *DeleteWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteWorkflowExecutionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_DeleteWorkflowExecutionRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *DeleteWorkflowExecutionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteWorkflowExecutionRequest.Merge(m, src) +} +func (m *DeleteWorkflowExecutionRequest) XXX_Size() int { + return m.Size() +} +func (m *DeleteWorkflowExecutionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteWorkflowExecutionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DeleteWorkflowExecutionRequest proto.InternalMessageInfo + +func (m *DeleteWorkflowExecutionRequest) GetNamespaceId() string { + if m != nil { + return m.NamespaceId + } + return "" +} + +func (m *DeleteWorkflowExecutionRequest) GetWorkflowExecution() *v14.WorkflowExecution { + if m != nil { + return m.WorkflowExecution + } + return nil +} + +type DeleteWorkflowExecutionResponse struct { +} + +func (m *DeleteWorkflowExecutionResponse) Reset() { *m = DeleteWorkflowExecutionResponse{} } +func (*DeleteWorkflowExecutionResponse) ProtoMessage() {} +func (*DeleteWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_b8c78c1d460a3711, []int{33} +} +func (m *DeleteWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteWorkflowExecutionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_DeleteWorkflowExecutionResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *DeleteWorkflowExecutionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteWorkflowExecutionResponse.Merge(m, src) +} +func (m *DeleteWorkflowExecutionResponse) XXX_Size() int { + return m.Size() +} +func (m *DeleteWorkflowExecutionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteWorkflowExecutionResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_DeleteWorkflowExecutionResponse proto.InternalMessageInfo + type ResetWorkflowExecutionRequest struct { NamespaceId string `protobuf:"bytes,1,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` ResetRequest *v1.ResetWorkflowExecutionRequest `protobuf:"bytes,2,opt,name=reset_request,json=resetRequest,proto3" json:"reset_request,omitempty"` @@ -2051,7 +2137,7 @@ type ResetWorkflowExecutionRequest struct { func (m *ResetWorkflowExecutionRequest) Reset() { *m = ResetWorkflowExecutionRequest{} } func (*ResetWorkflowExecutionRequest) ProtoMessage() {} func (*ResetWorkflowExecutionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{32} + return fileDescriptor_b8c78c1d460a3711, []int{34} } func (m *ResetWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2101,7 +2187,7 @@ type ResetWorkflowExecutionResponse struct { func (m *ResetWorkflowExecutionResponse) Reset() { *m = ResetWorkflowExecutionResponse{} } func (*ResetWorkflowExecutionResponse) ProtoMessage() {} func (*ResetWorkflowExecutionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{33} + return fileDescriptor_b8c78c1d460a3711, []int{35} } func (m *ResetWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2148,7 +2234,7 @@ type RequestCancelWorkflowExecutionRequest struct { func (m *RequestCancelWorkflowExecutionRequest) Reset() { *m = RequestCancelWorkflowExecutionRequest{} } func (*RequestCancelWorkflowExecutionRequest) ProtoMessage() {} func (*RequestCancelWorkflowExecutionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{34} + return fileDescriptor_b8c78c1d460a3711, []int{36} } func (m *RequestCancelWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2220,7 +2306,7 @@ func (m *RequestCancelWorkflowExecutionResponse) Reset() { } func (*RequestCancelWorkflowExecutionResponse) ProtoMessage() {} func (*RequestCancelWorkflowExecutionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{35} + return fileDescriptor_b8c78c1d460a3711, []int{37} } func (m *RequestCancelWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2258,7 +2344,7 @@ type ScheduleWorkflowTaskRequest struct { func (m *ScheduleWorkflowTaskRequest) Reset() { *m = ScheduleWorkflowTaskRequest{} } func (*ScheduleWorkflowTaskRequest) ProtoMessage() {} func (*ScheduleWorkflowTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{36} + return fileDescriptor_b8c78c1d460a3711, []int{38} } func (m *ScheduleWorkflowTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2314,7 +2400,7 @@ type ScheduleWorkflowTaskResponse struct { func (m *ScheduleWorkflowTaskResponse) Reset() { *m = ScheduleWorkflowTaskResponse{} } func (*ScheduleWorkflowTaskResponse) ProtoMessage() {} func (*ScheduleWorkflowTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{37} + return fileDescriptor_b8c78c1d460a3711, []int{39} } func (m *ScheduleWorkflowTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2360,7 +2446,7 @@ type RecordChildExecutionCompletedRequest struct { func (m *RecordChildExecutionCompletedRequest) Reset() { *m = RecordChildExecutionCompletedRequest{} } func (*RecordChildExecutionCompletedRequest) ProtoMessage() {} func (*RecordChildExecutionCompletedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{38} + return fileDescriptor_b8c78c1d460a3711, []int{40} } func (m *RecordChildExecutionCompletedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2430,7 +2516,7 @@ type RecordChildExecutionCompletedResponse struct { func (m *RecordChildExecutionCompletedResponse) Reset() { *m = RecordChildExecutionCompletedResponse{} } func (*RecordChildExecutionCompletedResponse) ProtoMessage() {} func (*RecordChildExecutionCompletedResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{39} + return fileDescriptor_b8c78c1d460a3711, []int{41} } func (m *RecordChildExecutionCompletedResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2467,7 +2553,7 @@ type DescribeWorkflowExecutionRequest struct { func (m *DescribeWorkflowExecutionRequest) Reset() { *m = DescribeWorkflowExecutionRequest{} } func (*DescribeWorkflowExecutionRequest) ProtoMessage() {} func (*DescribeWorkflowExecutionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{40} + return fileDescriptor_b8c78c1d460a3711, []int{42} } func (m *DescribeWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2520,7 +2606,7 @@ type DescribeWorkflowExecutionResponse struct { func (m *DescribeWorkflowExecutionResponse) Reset() { *m = DescribeWorkflowExecutionResponse{} } func (*DescribeWorkflowExecutionResponse) ProtoMessage() {} func (*DescribeWorkflowExecutionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{41} + return fileDescriptor_b8c78c1d460a3711, []int{43} } func (m *DescribeWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2589,7 +2675,7 @@ type ReplicateEventsV2Request struct { func (m *ReplicateEventsV2Request) Reset() { *m = ReplicateEventsV2Request{} } func (*ReplicateEventsV2Request) ProtoMessage() {} func (*ReplicateEventsV2Request) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{42} + return fileDescriptor_b8c78c1d460a3711, []int{44} } func (m *ReplicateEventsV2Request) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2659,7 +2745,7 @@ type ReplicateEventsV2Response struct { func (m *ReplicateEventsV2Response) Reset() { *m = ReplicateEventsV2Response{} } func (*ReplicateEventsV2Response) ProtoMessage() {} func (*ReplicateEventsV2Response) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{43} + return fileDescriptor_b8c78c1d460a3711, []int{45} } func (m *ReplicateEventsV2Response) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2697,7 +2783,7 @@ type SyncShardStatusRequest struct { func (m *SyncShardStatusRequest) Reset() { *m = SyncShardStatusRequest{} } func (*SyncShardStatusRequest) ProtoMessage() {} func (*SyncShardStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{44} + return fileDescriptor_b8c78c1d460a3711, []int{46} } func (m *SyncShardStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2753,7 +2839,7 @@ type SyncShardStatusResponse struct { func (m *SyncShardStatusResponse) Reset() { *m = SyncShardStatusResponse{} } func (*SyncShardStatusResponse) ProtoMessage() {} func (*SyncShardStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{45} + return fileDescriptor_b8c78c1d460a3711, []int{47} } func (m *SyncShardStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2802,7 +2888,7 @@ type SyncActivityRequest struct { func (m *SyncActivityRequest) Reset() { *m = SyncActivityRequest{} } func (*SyncActivityRequest) ProtoMessage() {} func (*SyncActivityRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{46} + return fileDescriptor_b8c78c1d460a3711, []int{48} } func (m *SyncActivityRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2935,7 +3021,7 @@ type SyncActivityResponse struct { func (m *SyncActivityResponse) Reset() { *m = SyncActivityResponse{} } func (*SyncActivityResponse) ProtoMessage() {} func (*SyncActivityResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{47} + return fileDescriptor_b8c78c1d460a3711, []int{49} } func (m *SyncActivityResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2972,7 +3058,7 @@ type DescribeMutableStateRequest struct { func (m *DescribeMutableStateRequest) Reset() { *m = DescribeMutableStateRequest{} } func (*DescribeMutableStateRequest) ProtoMessage() {} func (*DescribeMutableStateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{48} + return fileDescriptor_b8c78c1d460a3711, []int{50} } func (m *DescribeMutableStateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3023,7 +3109,7 @@ type DescribeMutableStateResponse struct { func (m *DescribeMutableStateResponse) Reset() { *m = DescribeMutableStateResponse{} } func (*DescribeMutableStateResponse) ProtoMessage() {} func (*DescribeMutableStateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{49} + return fileDescriptor_b8c78c1d460a3711, []int{51} } func (m *DescribeMutableStateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3078,7 +3164,7 @@ type DescribeHistoryHostRequest struct { func (m *DescribeHistoryHostRequest) Reset() { *m = DescribeHistoryHostRequest{} } func (*DescribeHistoryHostRequest) ProtoMessage() {} func (*DescribeHistoryHostRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{50} + return fileDescriptor_b8c78c1d460a3711, []int{52} } func (m *DescribeHistoryHostRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3146,7 +3232,7 @@ type DescribeHistoryHostResponse struct { func (m *DescribeHistoryHostResponse) Reset() { *m = DescribeHistoryHostResponse{} } func (*DescribeHistoryHostResponse) ProtoMessage() {} func (*DescribeHistoryHostResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{51} + return fileDescriptor_b8c78c1d460a3711, []int{53} } func (m *DescribeHistoryHostResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3217,7 +3303,7 @@ type CloseShardRequest struct { func (m *CloseShardRequest) Reset() { *m = CloseShardRequest{} } func (*CloseShardRequest) ProtoMessage() {} func (*CloseShardRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{52} + return fileDescriptor_b8c78c1d460a3711, []int{54} } func (m *CloseShardRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3345,7 @@ type CloseShardResponse struct { func (m *CloseShardResponse) Reset() { *m = CloseShardResponse{} } func (*CloseShardResponse) ProtoMessage() {} func (*CloseShardResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{53} + return fileDescriptor_b8c78c1d460a3711, []int{55} } func (m *CloseShardResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3295,7 +3381,7 @@ type GetShardRequest struct { func (m *GetShardRequest) Reset() { *m = GetShardRequest{} } func (*GetShardRequest) ProtoMessage() {} func (*GetShardRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{54} + return fileDescriptor_b8c78c1d460a3711, []int{56} } func (m *GetShardRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3338,7 +3424,7 @@ type GetShardResponse struct { func (m *GetShardResponse) Reset() { *m = GetShardResponse{} } func (*GetShardResponse) ProtoMessage() {} func (*GetShardResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{55} + return fileDescriptor_b8c78c1d460a3711, []int{57} } func (m *GetShardResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3384,7 +3470,7 @@ type RemoveTaskRequest struct { func (m *RemoveTaskRequest) Reset() { *m = RemoveTaskRequest{} } func (*RemoveTaskRequest) ProtoMessage() {} func (*RemoveTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{56} + return fileDescriptor_b8c78c1d460a3711, []int{58} } func (m *RemoveTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3447,7 +3533,7 @@ type RemoveTaskResponse struct { func (m *RemoveTaskResponse) Reset() { *m = RemoveTaskResponse{} } func (*RemoveTaskResponse) ProtoMessage() {} func (*RemoveTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{57} + return fileDescriptor_b8c78c1d460a3711, []int{59} } func (m *RemoveTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3484,7 +3570,7 @@ type GetReplicationMessagesRequest struct { func (m *GetReplicationMessagesRequest) Reset() { *m = GetReplicationMessagesRequest{} } func (*GetReplicationMessagesRequest) ProtoMessage() {} func (*GetReplicationMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{58} + return fileDescriptor_b8c78c1d460a3711, []int{60} } func (m *GetReplicationMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3534,7 +3620,7 @@ type GetReplicationMessagesResponse struct { func (m *GetReplicationMessagesResponse) Reset() { *m = GetReplicationMessagesResponse{} } func (*GetReplicationMessagesResponse) ProtoMessage() {} func (*GetReplicationMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{59} + return fileDescriptor_b8c78c1d460a3711, []int{61} } func (m *GetReplicationMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3577,7 +3663,7 @@ type GetDLQReplicationMessagesRequest struct { func (m *GetDLQReplicationMessagesRequest) Reset() { *m = GetDLQReplicationMessagesRequest{} } func (*GetDLQReplicationMessagesRequest) ProtoMessage() {} func (*GetDLQReplicationMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{60} + return fileDescriptor_b8c78c1d460a3711, []int{62} } func (m *GetDLQReplicationMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3620,7 +3706,7 @@ type GetDLQReplicationMessagesResponse struct { func (m *GetDLQReplicationMessagesResponse) Reset() { *m = GetDLQReplicationMessagesResponse{} } func (*GetDLQReplicationMessagesResponse) ProtoMessage() {} func (*GetDLQReplicationMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{61} + return fileDescriptor_b8c78c1d460a3711, []int{63} } func (m *GetDLQReplicationMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3664,7 +3750,7 @@ type QueryWorkflowRequest struct { func (m *QueryWorkflowRequest) Reset() { *m = QueryWorkflowRequest{} } func (*QueryWorkflowRequest) ProtoMessage() {} func (*QueryWorkflowRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{62} + return fileDescriptor_b8c78c1d460a3711, []int{64} } func (m *QueryWorkflowRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3714,7 +3800,7 @@ type QueryWorkflowResponse struct { func (m *QueryWorkflowResponse) Reset() { *m = QueryWorkflowResponse{} } func (*QueryWorkflowResponse) ProtoMessage() {} func (*QueryWorkflowResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{63} + return fileDescriptor_b8c78c1d460a3711, []int{65} } func (m *QueryWorkflowResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3758,7 +3844,7 @@ type ReapplyEventsRequest struct { func (m *ReapplyEventsRequest) Reset() { *m = ReapplyEventsRequest{} } func (*ReapplyEventsRequest) ProtoMessage() {} func (*ReapplyEventsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{64} + return fileDescriptor_b8c78c1d460a3711, []int{66} } func (m *ReapplyEventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3807,7 +3893,7 @@ type ReapplyEventsResponse struct { func (m *ReapplyEventsResponse) Reset() { *m = ReapplyEventsResponse{} } func (*ReapplyEventsResponse) ProtoMessage() {} func (*ReapplyEventsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{65} + return fileDescriptor_b8c78c1d460a3711, []int{67} } func (m *ReapplyEventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3848,7 +3934,7 @@ type GetDLQMessagesRequest struct { func (m *GetDLQMessagesRequest) Reset() { *m = GetDLQMessagesRequest{} } func (*GetDLQMessagesRequest) ProtoMessage() {} func (*GetDLQMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{66} + return fileDescriptor_b8c78c1d460a3711, []int{68} } func (m *GetDLQMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3928,7 +4014,7 @@ type GetDLQMessagesResponse struct { func (m *GetDLQMessagesResponse) Reset() { *m = GetDLQMessagesResponse{} } func (*GetDLQMessagesResponse) ProtoMessage() {} func (*GetDLQMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{67} + return fileDescriptor_b8c78c1d460a3711, []int{69} } func (m *GetDLQMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3988,7 +4074,7 @@ type PurgeDLQMessagesRequest struct { func (m *PurgeDLQMessagesRequest) Reset() { *m = PurgeDLQMessagesRequest{} } func (*PurgeDLQMessagesRequest) ProtoMessage() {} func (*PurgeDLQMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{68} + return fileDescriptor_b8c78c1d460a3711, []int{70} } func (m *PurgeDLQMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4051,7 +4137,7 @@ type PurgeDLQMessagesResponse struct { func (m *PurgeDLQMessagesResponse) Reset() { *m = PurgeDLQMessagesResponse{} } func (*PurgeDLQMessagesResponse) ProtoMessage() {} func (*PurgeDLQMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{69} + return fileDescriptor_b8c78c1d460a3711, []int{71} } func (m *PurgeDLQMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4092,7 +4178,7 @@ type MergeDLQMessagesRequest struct { func (m *MergeDLQMessagesRequest) Reset() { *m = MergeDLQMessagesRequest{} } func (*MergeDLQMessagesRequest) ProtoMessage() {} func (*MergeDLQMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{70} + return fileDescriptor_b8c78c1d460a3711, []int{72} } func (m *MergeDLQMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4170,7 +4256,7 @@ type MergeDLQMessagesResponse struct { func (m *MergeDLQMessagesResponse) Reset() { *m = MergeDLQMessagesResponse{} } func (*MergeDLQMessagesResponse) ProtoMessage() {} func (*MergeDLQMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{71} + return fileDescriptor_b8c78c1d460a3711, []int{73} } func (m *MergeDLQMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4214,7 +4300,7 @@ type RefreshWorkflowTasksRequest struct { func (m *RefreshWorkflowTasksRequest) Reset() { *m = RefreshWorkflowTasksRequest{} } func (*RefreshWorkflowTasksRequest) ProtoMessage() {} func (*RefreshWorkflowTasksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{72} + return fileDescriptor_b8c78c1d460a3711, []int{74} } func (m *RefreshWorkflowTasksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4263,7 +4349,7 @@ type RefreshWorkflowTasksResponse struct { func (m *RefreshWorkflowTasksResponse) Reset() { *m = RefreshWorkflowTasksResponse{} } func (*RefreshWorkflowTasksResponse) ProtoMessage() {} func (*RefreshWorkflowTasksResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{73} + return fileDescriptor_b8c78c1d460a3711, []int{75} } func (m *RefreshWorkflowTasksResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4302,7 +4388,7 @@ func (m *GenerateLastHistoryReplicationTasksRequest) Reset() { } func (*GenerateLastHistoryReplicationTasksRequest) ProtoMessage() {} func (*GenerateLastHistoryReplicationTasksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{74} + return fileDescriptor_b8c78c1d460a3711, []int{76} } func (m *GenerateLastHistoryReplicationTasksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4353,7 +4439,7 @@ func (m *GenerateLastHistoryReplicationTasksResponse) Reset() { } func (*GenerateLastHistoryReplicationTasksResponse) ProtoMessage() {} func (*GenerateLastHistoryReplicationTasksResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{75} + return fileDescriptor_b8c78c1d460a3711, []int{77} } func (m *GenerateLastHistoryReplicationTasksResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4390,7 +4476,7 @@ type GetReplicationStatusRequest struct { func (m *GetReplicationStatusRequest) Reset() { *m = GetReplicationStatusRequest{} } func (*GetReplicationStatusRequest) ProtoMessage() {} func (*GetReplicationStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{76} + return fileDescriptor_b8c78c1d460a3711, []int{78} } func (m *GetReplicationStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4433,7 +4519,7 @@ type GetReplicationStatusResponse struct { func (m *GetReplicationStatusResponse) Reset() { *m = GetReplicationStatusResponse{} } func (*GetReplicationStatusResponse) ProtoMessage() {} func (*GetReplicationStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{77} + return fileDescriptor_b8c78c1d460a3711, []int{79} } func (m *GetReplicationStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4482,7 +4568,7 @@ type ShardReplicationStatus struct { func (m *ShardReplicationStatus) Reset() { *m = ShardReplicationStatus{} } func (*ShardReplicationStatus) ProtoMessage() {} func (*ShardReplicationStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{78} + return fileDescriptor_b8c78c1d460a3711, []int{80} } func (m *ShardReplicationStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4554,7 +4640,7 @@ type HandoverNamespaceInfo struct { func (m *HandoverNamespaceInfo) Reset() { *m = HandoverNamespaceInfo{} } func (*HandoverNamespaceInfo) ProtoMessage() {} func (*HandoverNamespaceInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{79} + return fileDescriptor_b8c78c1d460a3711, []int{81} } func (m *HandoverNamespaceInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4600,7 +4686,7 @@ type ShardReplicationStatusPerCluster struct { func (m *ShardReplicationStatusPerCluster) Reset() { *m = ShardReplicationStatusPerCluster{} } func (*ShardReplicationStatusPerCluster) ProtoMessage() {} func (*ShardReplicationStatusPerCluster) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{80} + return fileDescriptor_b8c78c1d460a3711, []int{82} } func (m *ShardReplicationStatusPerCluster) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4677,6 +4763,8 @@ func init() { proto.RegisterType((*RemoveSignalMutableStateResponse)(nil), "temporal.server.api.historyservice.v1.RemoveSignalMutableStateResponse") proto.RegisterType((*TerminateWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.TerminateWorkflowExecutionRequest") proto.RegisterType((*TerminateWorkflowExecutionResponse)(nil), "temporal.server.api.historyservice.v1.TerminateWorkflowExecutionResponse") + proto.RegisterType((*DeleteWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.DeleteWorkflowExecutionRequest") + proto.RegisterType((*DeleteWorkflowExecutionResponse)(nil), "temporal.server.api.historyservice.v1.DeleteWorkflowExecutionResponse") proto.RegisterType((*ResetWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.ResetWorkflowExecutionRequest") proto.RegisterType((*ResetWorkflowExecutionResponse)(nil), "temporal.server.api.historyservice.v1.ResetWorkflowExecutionResponse") proto.RegisterType((*RequestCancelWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.RequestCancelWorkflowExecutionRequest") @@ -4736,260 +4824,261 @@ func init() { } var fileDescriptor_b8c78c1d460a3711 = []byte{ - // 4039 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5c, 0x4d, 0x6c, 0x1c, 0x47, - 0x76, 0x56, 0x73, 0x38, 0xe4, 0xcc, 0x9b, 0xe1, 0xfc, 0x34, 0xff, 0x86, 0xa4, 0x34, 0x22, 0xdb, - 0x96, 0x45, 0xff, 0x68, 0x68, 0x49, 0xbb, 0xb6, 0x57, 0x59, 0xaf, 0x23, 0x52, 0x22, 0x35, 0x82, - 0xa4, 0xa5, 0x9b, 0x5c, 0xd9, 0xf0, 0xae, 0xb7, 0xdd, 0x9c, 0x2e, 0x72, 0x3a, 0x9c, 0xe9, 0x1e, - 0x77, 0xf5, 0x0c, 0x39, 0xce, 0x21, 0x3f, 0x8b, 0x04, 0xd8, 0x0d, 0x10, 0x18, 0xc8, 0x65, 0x81, - 0x6c, 0x2e, 0x01, 0x82, 0x2c, 0x02, 0x04, 0x39, 0xe4, 0x10, 0xec, 0x21, 0xd7, 0x20, 0xa7, 0xc4, - 0x08, 0x10, 0x64, 0xb1, 0x39, 0x24, 0x96, 0x11, 0x20, 0x41, 0x72, 0xd8, 0x43, 0x0e, 0x39, 0x06, - 0xf5, 0xd7, 0xd3, 0x7f, 0xf3, 0x47, 0xca, 0xd1, 0xc6, 0xeb, 0x1b, 0xbb, 0xea, 0xbd, 0x57, 0xf5, - 0xaa, 0xde, 0xfb, 0xea, 0xd5, 0xab, 0x37, 0x84, 0xaf, 0xbb, 0xa8, 0xd9, 0xb2, 0x1d, 0xbd, 0xb1, - 0x81, 0x91, 0xd3, 0x41, 0xce, 0x86, 0xde, 0x32, 0x37, 0xea, 0x26, 0x76, 0x6d, 0xa7, 0x4b, 0x5a, - 0xcc, 0x1a, 0xda, 0xe8, 0x5c, 0xdf, 0x70, 0xd0, 0x87, 0x6d, 0x84, 0x5d, 0xcd, 0x41, 0xb8, 0x65, - 0x5b, 0x18, 0x55, 0x5a, 0x8e, 0xed, 0xda, 0xf2, 0x15, 0xc1, 0x5d, 0x61, 0xdc, 0x15, 0xbd, 0x65, - 0x56, 0x82, 0xdc, 0x95, 0xce, 0xf5, 0xe5, 0xf2, 0x91, 0x6d, 0x1f, 0x35, 0xd0, 0x06, 0x65, 0x3a, - 0x68, 0x1f, 0x6e, 0x18, 0x6d, 0x47, 0x77, 0x4d, 0xdb, 0x62, 0x62, 0x96, 0x2f, 0x87, 0xfb, 0x5d, - 0xb3, 0x89, 0xb0, 0xab, 0x37, 0x5b, 0x9c, 0x60, 0xcd, 0x40, 0x2d, 0x64, 0x19, 0xc8, 0xaa, 0x99, - 0x08, 0x6f, 0x1c, 0xd9, 0x47, 0x36, 0x6d, 0xa7, 0x7f, 0x71, 0x92, 0xe7, 0x3d, 0x45, 0x88, 0x06, - 0x35, 0xbb, 0xd9, 0xb4, 0x2d, 0x32, 0xf3, 0x26, 0xc2, 0x58, 0x3f, 0xe2, 0x13, 0x5e, 0xbe, 0x12, - 0xa0, 0xe2, 0x33, 0x8d, 0x92, 0x5d, 0x0d, 0x90, 0xb9, 0x3a, 0x3e, 0xfe, 0xb0, 0x8d, 0xda, 0x28, - 0x4a, 0x18, 0x1c, 0x15, 0x59, 0xed, 0x26, 0x26, 0x44, 0x27, 0xb6, 0x73, 0x7c, 0xd8, 0xb0, 0x4f, - 0x38, 0xd5, 0x0b, 0x01, 0x2a, 0xd1, 0x19, 0x95, 0xf6, 0x5c, 0x80, 0xee, 0xc3, 0x36, 0x8a, 0x9b, - 0x5b, 0x50, 0x85, 0x43, 0xdd, 0x6c, 0xb4, 0x9d, 0x98, 0x99, 0xbd, 0x32, 0x60, 0x63, 0xa3, 0xd4, - 0x2f, 0xc6, 0x51, 0x7b, 0xea, 0xb0, 0xd5, 0xe4, 0xa4, 0x2f, 0x0f, 0x24, 0x0d, 0x69, 0x7e, 0x75, - 0x20, 0x31, 0x59, 0x58, 0x4e, 0x78, 0x2d, 0x8e, 0xb0, 0xff, 0x4a, 0x55, 0xe2, 0xc8, 0x2d, 0xbd, - 0x89, 0x70, 0x4b, 0xaf, 0xc5, 0xac, 0xc6, 0xab, 0x71, 0xf4, 0x0e, 0x6a, 0x35, 0xcc, 0x1a, 0x35, - 0xc4, 0x28, 0xc7, 0xcd, 0x38, 0x8e, 0x16, 0x72, 0xb0, 0x89, 0x5d, 0x64, 0xb1, 0x31, 0xd0, 0x29, - 0xaa, 0xb5, 0x09, 0x3b, 0xe6, 0x4c, 0x6f, 0x8d, 0xc0, 0x24, 0x94, 0xd2, 0x9a, 0x6d, 0x57, 0x3f, - 0x68, 0x20, 0x0d, 0xbb, 0xba, 0x2b, 0x46, 0x7d, 0x2d, 0xd6, 0x52, 0x86, 0x3a, 0xe2, 0xf2, 0xad, - 0xb8, 0x81, 0x75, 0xa3, 0x69, 0x5a, 0x43, 0x79, 0x95, 0xdf, 0x9b, 0x82, 0x4b, 0x7b, 0xae, 0xee, - 0xb8, 0xef, 0xf0, 0xe1, 0xee, 0x0a, 0xb5, 0x54, 0xc6, 0x20, 0xaf, 0x41, 0xd6, 0x5b, 0x5b, 0xcd, - 0x34, 0x4a, 0xd2, 0xaa, 0xb4, 0x9e, 0x56, 0x33, 0x5e, 0x5b, 0xd5, 0x90, 0x6b, 0x30, 0x83, 0x89, - 0x0c, 0x8d, 0x0f, 0x52, 0x9a, 0x58, 0x95, 0xd6, 0x33, 0x37, 0xbe, 0xe1, 0x6d, 0x14, 0x85, 0x86, - 0x90, 0x42, 0x95, 0xce, 0xf5, 0xca, 0xc0, 0x91, 0xd5, 0x2c, 0x15, 0x2a, 0xe6, 0x51, 0x87, 0xf9, - 0x96, 0xee, 0x20, 0xcb, 0xd5, 0xbc, 0x95, 0xd7, 0x4c, 0xeb, 0xd0, 0x2e, 0x25, 0xe8, 0x60, 0x5f, - 0xa9, 0xc4, 0xc1, 0x91, 0x67, 0x91, 0x9d, 0xeb, 0x95, 0x5d, 0xca, 0xed, 0x8d, 0x52, 0xb5, 0x0e, - 0x6d, 0x75, 0xb6, 0x15, 0x6d, 0x94, 0x4b, 0x30, 0xad, 0xbb, 0x44, 0x9a, 0x5b, 0x9a, 0x5c, 0x95, - 0xd6, 0x93, 0xaa, 0xf8, 0x94, 0x9b, 0xa0, 0x78, 0x3b, 0xd8, 0x9b, 0x05, 0x3a, 0x6d, 0x99, 0x0c, - 0xd2, 0x34, 0x82, 0x5d, 0xa5, 0x24, 0x9d, 0xd0, 0x72, 0x85, 0x01, 0x5b, 0x45, 0x00, 0x5b, 0x65, - 0x5f, 0x00, 0xdb, 0xe6, 0xe4, 0xc7, 0xff, 0x72, 0x59, 0x52, 0x2f, 0x9f, 0x84, 0x35, 0xbf, 0xeb, - 0x49, 0x22, 0xb4, 0x72, 0x1d, 0x96, 0x6a, 0xb6, 0xe5, 0x9a, 0x56, 0x1b, 0x69, 0x3a, 0xd6, 0x2c, - 0x74, 0xa2, 0x99, 0x96, 0xe9, 0x9a, 0xba, 0x6b, 0x3b, 0xa5, 0xa9, 0x55, 0x69, 0x3d, 0x77, 0xe3, - 0x5a, 0x70, 0x8d, 0xa9, 0x77, 0x11, 0x65, 0xb7, 0x38, 0xdf, 0x6d, 0xfc, 0x08, 0x9d, 0x54, 0x05, - 0x93, 0xba, 0x50, 0x8b, 0x6d, 0x97, 0x1f, 0x42, 0x51, 0xf4, 0x18, 0x1a, 0x87, 0x95, 0xd2, 0x34, - 0xd5, 0x63, 0x35, 0x38, 0x02, 0xef, 0x24, 0x63, 0x6c, 0xb3, 0x3f, 0xd5, 0x82, 0xc7, 0xca, 0x5b, - 0xe4, 0xc7, 0xb0, 0xd0, 0xd0, 0xb1, 0xab, 0xd5, 0xec, 0x66, 0xab, 0x81, 0xe8, 0xca, 0x38, 0x08, - 0xb7, 0x1b, 0x6e, 0x29, 0x15, 0x27, 0x93, 0x43, 0x0c, 0xdd, 0xa3, 0x6e, 0xc3, 0xd6, 0x0d, 0xac, - 0xce, 0x11, 0xfe, 0x2d, 0x8f, 0x5d, 0xa5, 0xdc, 0xf2, 0x77, 0x61, 0xe5, 0xd0, 0x74, 0xb0, 0xab, - 0x79, 0xbb, 0x40, 0x50, 0x44, 0x3b, 0xd0, 0x6b, 0xc7, 0xf6, 0xe1, 0x61, 0x29, 0x4d, 0x85, 0x2f, - 0x45, 0x16, 0xfe, 0x0e, 0x3f, 0x71, 0x36, 0x27, 0x7f, 0x48, 0xd6, 0xbd, 0x44, 0x65, 0x08, 0xb3, - 0xdb, 0xd7, 0xf1, 0xf1, 0x26, 0x13, 0xa0, 0xbc, 0x0e, 0xe5, 0x7e, 0x26, 0xc9, 0xbc, 0x46, 0x9e, - 0x87, 0x29, 0xa7, 0x6d, 0xf5, 0xfc, 0x20, 0xe9, 0xb4, 0xad, 0xaa, 0xa1, 0xfc, 0xa7, 0x04, 0x0b, - 0x3b, 0xc8, 0x7d, 0xc8, 0xbc, 0x7a, 0x8f, 0x38, 0xf5, 0x18, 0xfe, 0xb3, 0x03, 0x69, 0xcf, 0x9a, - 0xb8, 0xef, 0xbc, 0xd8, 0x6f, 0x85, 0xa2, 0x53, 0xeb, 0xf1, 0xca, 0x37, 0x61, 0x01, 0x9d, 0xb6, - 0x50, 0xcd, 0x45, 0x86, 0x66, 0xa1, 0x53, 0x57, 0x43, 0x1d, 0xe2, 0x30, 0xa6, 0x41, 0x9d, 0x24, - 0xa1, 0xce, 0x8a, 0xde, 0x47, 0xe8, 0xd4, 0xbd, 0x4b, 0xfa, 0xaa, 0x86, 0xfc, 0x2a, 0xcc, 0xd5, - 0xda, 0x0e, 0xf5, 0xac, 0x03, 0x47, 0xb7, 0x6a, 0x75, 0xcd, 0xb5, 0x8f, 0x91, 0x45, 0x6d, 0x3f, - 0xab, 0xca, 0xbc, 0x6f, 0x93, 0x76, 0xed, 0x93, 0x1e, 0xe5, 0xcf, 0x52, 0xb0, 0x18, 0xd1, 0x96, - 0x2f, 0x50, 0x40, 0x17, 0xe9, 0x1c, 0xba, 0x54, 0x61, 0xa6, 0xb7, 0xcb, 0xdd, 0x16, 0xe2, 0x0b, - 0xf3, 0xfc, 0x30, 0x61, 0xfb, 0xdd, 0x16, 0x52, 0xb3, 0x27, 0xbe, 0x2f, 0x59, 0x81, 0x99, 0xb8, - 0xd5, 0xc8, 0x58, 0xbe, 0x55, 0xf8, 0x1a, 0x2c, 0xb5, 0x1c, 0xd4, 0x31, 0xed, 0x36, 0xd6, 0x28, - 0xee, 0x20, 0xa3, 0x47, 0x3f, 0x49, 0xe9, 0x17, 0x04, 0xc1, 0x1e, 0xeb, 0x17, 0xac, 0xd7, 0x60, - 0x96, 0x5a, 0x3b, 0x33, 0x4d, 0x8f, 0x29, 0x49, 0x99, 0x0a, 0xa4, 0x6b, 0x9b, 0xf4, 0x08, 0xf2, - 0x2d, 0x00, 0x6a, 0xb5, 0x34, 0xaa, 0xa0, 0x6e, 0x1c, 0xd1, 0xca, 0x0b, 0x3a, 0x88, 0x62, 0xc4, - 0x40, 0xdf, 0x26, 0x1f, 0x6a, 0xda, 0x15, 0x7f, 0xca, 0xbb, 0x50, 0xc4, 0xae, 0x59, 0x3b, 0xee, - 0x6a, 0x3e, 0x59, 0xd3, 0x63, 0xc8, 0xca, 0x33, 0x76, 0xaf, 0x41, 0xfe, 0x75, 0x78, 0x39, 0x22, - 0x51, 0xc3, 0xb5, 0x3a, 0x32, 0xda, 0x0d, 0xa4, 0xb9, 0x36, 0x5b, 0x15, 0x8a, 0x70, 0x76, 0xdb, - 0x2d, 0x65, 0x46, 0xf3, 0xb5, 0x2b, 0xa1, 0x61, 0xf6, 0xb8, 0xc0, 0x7d, 0x9b, 0x2e, 0xe2, 0x3e, - 0x93, 0xd6, 0xd7, 0x06, 0x67, 0xfa, 0xd9, 0xa0, 0xfc, 0x6d, 0xc8, 0x79, 0xe6, 0x41, 0x0f, 0xd1, - 0x52, 0x9e, 0x02, 0x62, 0xfc, 0x39, 0xe0, 0xe1, 0x62, 0xc4, 0xe4, 0x98, 0xf5, 0x7a, 0xa6, 0x46, - 0x3f, 0xe5, 0x77, 0x20, 0x1f, 0x10, 0xde, 0xc6, 0xa5, 0x02, 0x95, 0x5e, 0xe9, 0x03, 0xb7, 0xb1, - 0x62, 0xdb, 0x58, 0xcd, 0xf9, 0xe5, 0xb6, 0xb1, 0xfc, 0x3e, 0x14, 0x3b, 0x24, 0x22, 0xb0, 0x2d, - 0x8d, 0x85, 0x63, 0x26, 0xc2, 0xa5, 0x22, 0x5d, 0xca, 0x57, 0x2b, 0x03, 0xe2, 0x69, 0x32, 0xc6, - 0x63, 0xc6, 0x78, 0x4f, 0xf0, 0xa9, 0x85, 0x4e, 0xa8, 0x45, 0xfe, 0x06, 0x5c, 0x34, 0x89, 0xf9, - 0x86, 0xb7, 0x11, 0x59, 0xc4, 0x51, 0x8d, 0x92, 0xbc, 0x2a, 0xad, 0xa7, 0xd4, 0x92, 0x89, 0xf7, - 0x82, 0xbb, 0x72, 0x97, 0xf5, 0xcb, 0x5f, 0x81, 0xc5, 0x88, 0x25, 0xbb, 0xa7, 0x14, 0xee, 0x66, - 0x19, 0x80, 0x04, 0xad, 0x79, 0xff, 0xd4, 0xaa, 0x1a, 0xf7, 0x27, 0x53, 0xa9, 0x42, 0xfa, 0xfe, - 0x64, 0x2a, 0x5d, 0x80, 0xfb, 0x93, 0x29, 0x28, 0x64, 0xee, 0x4f, 0xa6, 0xb2, 0x85, 0x99, 0xfb, - 0x93, 0xa9, 0x5c, 0x21, 0xaf, 0xfc, 0x97, 0x04, 0x8b, 0xbb, 0x76, 0xa3, 0xf1, 0x4b, 0x82, 0x8d, - 0xff, 0x36, 0x0d, 0xa5, 0xa8, 0xba, 0x5f, 0x82, 0xe3, 0x97, 0xe0, 0xf8, 0xd4, 0xc1, 0x31, 0xdb, - 0x17, 0x1c, 0x63, 0x61, 0x26, 0xf7, 0xd4, 0x60, 0xe6, 0xff, 0x27, 0xf6, 0x0e, 0x00, 0xb7, 0xe2, - 0x78, 0xe0, 0x36, 0x53, 0xc8, 0x29, 0xdf, 0x97, 0x60, 0x45, 0x45, 0x18, 0xb9, 0x21, 0x28, 0x7d, - 0x06, 0xd0, 0xa6, 0x94, 0xe1, 0x62, 0xfc, 0x54, 0x18, 0xec, 0x28, 0x3f, 0x9b, 0x80, 0x55, 0x15, - 0xd5, 0x6c, 0xc7, 0xf0, 0x07, 0xbd, 0xdc, 0x51, 0xc7, 0x98, 0xf0, 0xbb, 0x20, 0x47, 0xaf, 0x3f, - 0xe3, 0xcf, 0xbc, 0x18, 0xb9, 0xf7, 0xc8, 0x97, 0x21, 0xe3, 0x79, 0x93, 0x07, 0x41, 0x20, 0x9a, - 0xaa, 0x86, 0xbc, 0x08, 0xd3, 0xd4, 0xf3, 0x3c, 0xbc, 0x99, 0x22, 0x9f, 0x55, 0x43, 0xbe, 0x04, - 0x20, 0xae, 0xb6, 0x1c, 0x56, 0xd2, 0x6a, 0x9a, 0xb7, 0x54, 0x0d, 0xf9, 0x03, 0xc8, 0xb6, 0xec, - 0x46, 0xc3, 0xbb, 0x99, 0x32, 0x44, 0x79, 0x73, 0xe8, 0xcd, 0x94, 0x40, 0xb8, 0x7f, 0xb1, 0xfc, - 0x7b, 0xab, 0x66, 0x88, 0x48, 0xfe, 0xa1, 0xfc, 0xe3, 0x34, 0xac, 0x0d, 0x58, 0x5c, 0x8e, 0xfc, - 0x11, 0xc0, 0x96, 0xce, 0x0c, 0xd8, 0x03, 0xc1, 0x78, 0x62, 0x20, 0x18, 0xbf, 0x02, 0xb2, 0x58, - 0x53, 0x23, 0x0c, 0xf8, 0x05, 0xaf, 0x47, 0x50, 0xaf, 0x43, 0xa1, 0x0f, 0xd8, 0xe7, 0x70, 0x50, - 0x6e, 0xe4, 0x0c, 0x49, 0x46, 0xcf, 0x10, 0xdf, 0xad, 0x7a, 0x2a, 0x78, 0xab, 0x7e, 0x03, 0x4a, - 0x1c, 0x5c, 0x7d, 0x77, 0x6a, 0x1e, 0xb1, 0x4c, 0xd3, 0x88, 0x65, 0x81, 0xf5, 0xf7, 0xee, 0xc9, - 0x3c, 0x5e, 0x39, 0xf2, 0x19, 0x24, 0x33, 0x0f, 0xeb, 0xd0, 0xe6, 0x77, 0xcc, 0xaf, 0x0d, 0x03, - 0xba, 0x7d, 0x47, 0xb7, 0xb0, 0x89, 0xac, 0xc0, 0x4d, 0x90, 0x66, 0x05, 0x0a, 0x27, 0xa1, 0x16, - 0xf9, 0x08, 0x2e, 0xc5, 0x5c, 0xfc, 0x7d, 0xa7, 0x4b, 0x7a, 0x8c, 0xd3, 0x65, 0x39, 0x62, 0xff, - 0xbd, 0x83, 0x66, 0x0d, 0xb2, 0x01, 0x8c, 0xcf, 0x50, 0x8c, 0xcf, 0x1c, 0xf8, 0xc0, 0x7d, 0x07, - 0x72, 0xbd, 0x4d, 0xa4, 0x09, 0x87, 0xec, 0x88, 0x09, 0x87, 0x19, 0x8f, 0x8f, 0xa6, 0x17, 0xb6, - 0x20, 0x2b, 0xf6, 0x97, 0x8a, 0x99, 0x19, 0x51, 0x4c, 0x86, 0x73, 0x51, 0x21, 0x36, 0x4c, 0x7f, - 0xd8, 0x46, 0xfc, 0x80, 0x49, 0xac, 0x67, 0x6e, 0x7c, 0xab, 0x32, 0x52, 0x5e, 0xb8, 0x32, 0xd4, - 0x67, 0x2a, 0x6f, 0x33, 0xb9, 0x77, 0x2d, 0xd7, 0xe9, 0xaa, 0x62, 0x94, 0xe5, 0x0f, 0x20, 0xeb, - 0xef, 0x90, 0x0b, 0x90, 0x38, 0x46, 0x5d, 0x0e, 0x57, 0xe4, 0x4f, 0xf9, 0x16, 0x24, 0x3b, 0x7a, - 0xa3, 0xdd, 0x27, 0x28, 0xa2, 0x99, 0x55, 0xbf, 0x8b, 0x11, 0x69, 0x5d, 0x95, 0xb1, 0xdc, 0x9a, - 0x78, 0x43, 0x62, 0x30, 0xef, 0x03, 0xcd, 0xdb, 0x35, 0xd7, 0xec, 0x98, 0x6e, 0xf7, 0x4b, 0xd0, - 0x1c, 0x01, 0x34, 0xfd, 0x8b, 0xd5, 0x1f, 0x34, 0x7f, 0x7b, 0x52, 0x80, 0x66, 0xec, 0xe2, 0x72, - 0xd0, 0x7c, 0x04, 0xf9, 0x10, 0x5c, 0x71, 0xd8, 0xbc, 0x12, 0x9c, 0x8a, 0xcf, 0xa9, 0x59, 0x90, - 0xd2, 0xa5, 0xa0, 0xa3, 0xe6, 0x82, 0x90, 0x16, 0x31, 0xf8, 0x89, 0xb3, 0x18, 0xbc, 0x0f, 0xc7, - 0x12, 0x41, 0x1c, 0x43, 0x50, 0x16, 0x71, 0x1a, 0x6f, 0xd2, 0x42, 0x8e, 0x3a, 0x39, 0xe2, 0x80, - 0x2b, 0x5c, 0xce, 0x6d, 0x26, 0x66, 0x2f, 0xe0, 0xb6, 0x0f, 0xa1, 0x58, 0x47, 0xba, 0xe3, 0x1e, - 0x20, 0xdd, 0xd5, 0x0c, 0xe4, 0xea, 0x66, 0x03, 0xf3, 0x9c, 0xe3, 0xf0, 0xbc, 0x5a, 0xc1, 0x63, - 0xbd, 0xc3, 0x38, 0xa3, 0x27, 0xd3, 0xd4, 0x99, 0x4f, 0xa6, 0x6b, 0x3e, 0x53, 0xf7, 0x5c, 0x80, - 0x42, 0x78, 0xba, 0x67, 0xbf, 0x8f, 0x44, 0x87, 0xf2, 0x13, 0x09, 0x9e, 0x63, 0x7b, 0x1d, 0x80, - 0x01, 0x9e, 0xf5, 0x1b, 0xcb, 0xc9, 0x6c, 0x28, 0xf0, 0x5c, 0x23, 0x0a, 0x25, 0xa1, 0xef, 0x0c, - 0xb5, 0xda, 0x11, 0xa6, 0xa0, 0xe6, 0x85, 0x74, 0x61, 0xc0, 0x7f, 0x28, 0xc1, 0xf3, 0x83, 0x19, - 0xb9, 0x0d, 0xe3, 0xde, 0x21, 0x2a, 0x52, 0xef, 0xdc, 0x88, 0xef, 0x3d, 0x2d, 0xa0, 0x24, 0xd7, - 0x95, 0x40, 0x83, 0xf2, 0x17, 0x12, 0xc1, 0xae, 0xc8, 0xec, 0xb6, 0x75, 0xb3, 0x31, 0xd6, 0xb2, - 0xd6, 0x21, 0x77, 0x48, 0x79, 0x42, 0x8b, 0x7a, 0xfb, 0x2c, 0x8b, 0x1a, 0x18, 0x5d, 0x9d, 0x39, - 0xf4, 0x7f, 0x2a, 0xcf, 0x11, 0x3c, 0xe8, 0xcb, 0xc2, 0xd5, 0xfa, 0x89, 0x04, 0x4a, 0x14, 0x35, - 0xee, 0x09, 0x8b, 0x1e, 0x43, 0xb1, 0x96, 0xdf, 0x87, 0x82, 0xba, 0x6d, 0x8d, 0xa0, 0xdb, 0xb0, - 0x29, 0xf8, 0xdc, 0x4c, 0x28, 0xb8, 0x4b, 0x6c, 0x7d, 0x00, 0x1f, 0x37, 0x97, 0x17, 0xa1, 0x50, - 0xd3, 0xad, 0x1a, 0xf2, 0xc0, 0x17, 0xb1, 0xf9, 0xa7, 0xd4, 0x3c, 0x6b, 0x57, 0x45, 0xb3, 0xdf, - 0x7d, 0xfc, 0x32, 0x9f, 0x91, 0xfb, 0x0c, 0x9a, 0x42, 0xd4, 0x7d, 0x5e, 0xf0, 0xbc, 0xa7, 0x0f, - 0x5f, 0xd4, 0x90, 0xfd, 0x84, 0xff, 0xf7, 0x86, 0xdc, 0x77, 0xf4, 0xfe, 0x86, 0x1c, 0xc7, 0xc2, - 0xd5, 0xfa, 0x4b, 0x6a, 0xc8, 0x51, 0xfd, 0xe9, 0x0e, 0x8f, 0xa5, 0xd8, 0xaf, 0x41, 0x2e, 0x68, - 0x2f, 0x63, 0x58, 0xf1, 0xb0, 0xf1, 0xd5, 0x99, 0x80, 0xc9, 0x29, 0x57, 0xe2, 0xed, 0xcd, 0x63, - 0xe2, 0xca, 0xfd, 0xcd, 0x04, 0x94, 0xf7, 0xcc, 0x23, 0x4b, 0x6f, 0x9c, 0xe7, 0x4d, 0xf1, 0x10, - 0x72, 0x98, 0x0a, 0x09, 0x29, 0xf6, 0xd6, 0xf0, 0x47, 0xc5, 0x81, 0x63, 0xab, 0x33, 0x4c, 0xac, - 0x98, 0x8a, 0x09, 0x2b, 0xe8, 0xd4, 0x45, 0x0e, 0x19, 0x29, 0x26, 0x4e, 0x4b, 0x8c, 0x1b, 0xa7, - 0x2d, 0x09, 0x69, 0x91, 0x2e, 0xb9, 0x02, 0xb3, 0xb5, 0xba, 0xd9, 0x30, 0x7a, 0xe3, 0xd8, 0x56, - 0xa3, 0x4b, 0x83, 0x82, 0x94, 0x5a, 0xa4, 0x5d, 0x82, 0xe9, 0x9b, 0x56, 0xa3, 0xab, 0xac, 0xc1, - 0xe5, 0xbe, 0xba, 0xf0, 0xb5, 0xfe, 0x07, 0x09, 0xae, 0x72, 0x1a, 0xd3, 0xad, 0x9f, 0xfb, 0x21, - 0xf7, 0x7b, 0x12, 0x2c, 0xf1, 0x55, 0x3f, 0x31, 0xdd, 0xba, 0x16, 0xf7, 0xaa, 0x7b, 0x6f, 0xd4, - 0x0d, 0x18, 0x36, 0x21, 0x75, 0x01, 0x07, 0x09, 0x85, 0x9d, 0xdd, 0x86, 0xf5, 0xe1, 0x22, 0x06, - 0xbf, 0xc7, 0xfd, 0xb5, 0x04, 0x97, 0x55, 0xd4, 0xb4, 0x3b, 0x88, 0x49, 0x3a, 0x63, 0xf2, 0xf9, - 0xf3, 0x8b, 0xdd, 0x83, 0x11, 0x78, 0x22, 0x14, 0x81, 0x2b, 0x0a, 0x81, 0xbd, 0x7e, 0xd3, 0xe7, - 0x7b, 0xff, 0x57, 0x12, 0xac, 0xed, 0x23, 0xa7, 0x69, 0x5a, 0xba, 0x8b, 0xce, 0xb3, 0xeb, 0x36, - 0x14, 0x5d, 0x21, 0x27, 0xb4, 0xd9, 0x9b, 0x43, 0x37, 0x7b, 0xe8, 0x0c, 0xd4, 0x82, 0x27, 0x5c, - 0x6c, 0xf0, 0xf3, 0xa0, 0x0c, 0x62, 0xe3, 0xfa, 0xfd, 0xa9, 0x04, 0x97, 0x68, 0x5a, 0xeb, 0x9c, - 0xa5, 0x09, 0x0e, 0x91, 0x31, 0x76, 0x69, 0xc2, 0xc0, 0x91, 0xd5, 0x2c, 0x15, 0x2a, 0xf4, 0x79, - 0x1d, 0xca, 0xfd, 0xc8, 0x07, 0x9b, 0xe9, 0x1f, 0x24, 0xe0, 0x0a, 0x17, 0xc2, 0x60, 0xf4, 0x3c, - 0xaa, 0x36, 0xfb, 0x1c, 0x05, 0xdb, 0x23, 0xe8, 0x3a, 0xc2, 0x14, 0x42, 0xa7, 0x81, 0xfc, 0xa6, - 0x0f, 0x38, 0x79, 0x55, 0x42, 0x34, 0xa9, 0x54, 0x12, 0x24, 0x55, 0x41, 0x21, 0xd2, 0x41, 0x43, - 0x70, 0x77, 0xf2, 0xf3, 0xc7, 0xdd, 0x64, 0x3f, 0xdc, 0x5d, 0x87, 0x17, 0x86, 0xad, 0x08, 0x37, - 0xd1, 0xbf, 0x97, 0x60, 0x45, 0x5c, 0xce, 0xfc, 0x71, 0xeb, 0x2f, 0x04, 0xc4, 0xdc, 0x84, 0x05, - 0x13, 0x6b, 0x31, 0xf5, 0x12, 0x74, 0x6f, 0x52, 0xea, 0xac, 0x89, 0xb7, 0xc3, 0x85, 0x10, 0x4a, - 0x19, 0x2e, 0xc6, 0x2b, 0xc4, 0x35, 0xfe, 0xef, 0x09, 0x12, 0xb9, 0x91, 0x38, 0x76, 0x8b, 0xac, - 0x9b, 0x37, 0xda, 0x59, 0xa2, 0xce, 0xcf, 0x4f, 0xf5, 0x35, 0xc8, 0xf6, 0x4c, 0xb2, 0xf7, 0xa4, - 0xe5, 0xb5, 0x55, 0x0d, 0xf9, 0x3d, 0x98, 0x15, 0x41, 0xa9, 0x71, 0x1e, 0xbb, 0x93, 0x3d, 0x29, - 0xbd, 0xe1, 0x77, 0xbd, 0x70, 0x9a, 0xa6, 0x32, 0x69, 0xe2, 0x22, 0x39, 0x4e, 0xe2, 0x22, 0xdf, - 0x63, 0xa7, 0x0d, 0xca, 0x55, 0x82, 0x13, 0x03, 0x57, 0x9d, 0xef, 0xcf, 0x1f, 0x4b, 0xb0, 0x7a, - 0x07, 0xe1, 0x9a, 0x63, 0x1e, 0x9c, 0xeb, 0x4c, 0xf8, 0x36, 0x4c, 0x8f, 0x1b, 0x29, 0x0f, 0x1b, - 0x56, 0x15, 0x12, 0x95, 0x1f, 0x27, 0x60, 0x6d, 0x00, 0x35, 0xc7, 0xcc, 0xef, 0x40, 0xa1, 0x97, - 0x6a, 0xad, 0xd9, 0xd6, 0xa1, 0x79, 0xc4, 0x6f, 0xce, 0xd7, 0xe3, 0xe7, 0x12, 0xbb, 0x41, 0x5b, - 0x94, 0x51, 0xcd, 0xa3, 0x60, 0x83, 0x7c, 0x04, 0x8b, 0x31, 0x19, 0x5d, 0x9a, 0x3f, 0x66, 0x0a, - 0x6f, 0x8c, 0x31, 0x08, 0xcd, 0x1a, 0xcf, 0x9f, 0xc4, 0x35, 0xcb, 0xdf, 0x01, 0xb9, 0x85, 0x2c, - 0xc3, 0xb4, 0x8e, 0x34, 0x9d, 0x85, 0xcd, 0x26, 0xc2, 0xa5, 0x04, 0xcd, 0x95, 0x5e, 0xeb, 0x3f, - 0xc6, 0x2e, 0xe3, 0x11, 0x91, 0x36, 0x1d, 0xa1, 0xd8, 0x0a, 0x34, 0x9a, 0x08, 0xcb, 0xdf, 0x85, - 0x82, 0x90, 0x4e, 0x81, 0xcc, 0xa1, 0x8f, 0xd3, 0x44, 0xf6, 0xcd, 0xa1, 0xb2, 0x83, 0xb6, 0x44, - 0x47, 0xc8, 0xb7, 0x7c, 0x5d, 0x0e, 0xb2, 0x94, 0xdf, 0x4a, 0x40, 0x49, 0xe5, 0xa5, 0x92, 0x88, - 0xda, 0x22, 0x7e, 0x7c, 0xe3, 0x17, 0xc2, 0xc7, 0x0f, 0x61, 0x3e, 0xf8, 0xc6, 0xd9, 0xd5, 0x4c, - 0x17, 0x35, 0xc5, 0xd2, 0xde, 0x18, 0xeb, 0x9d, 0xb3, 0x5b, 0x75, 0x51, 0x53, 0x9d, 0xed, 0x44, - 0xda, 0xb0, 0xfc, 0x06, 0x4c, 0x51, 0x0f, 0xc6, 0x1c, 0x1b, 0xfa, 0xe6, 0xd8, 0xee, 0xe8, 0xae, - 0xbe, 0xd9, 0xb0, 0x0f, 0x54, 0x4e, 0x2f, 0x6f, 0x43, 0xce, 0x42, 0x27, 0x1a, 0x39, 0xf8, 0xb9, - 0x84, 0xe4, 0x88, 0x12, 0xb2, 0x16, 0x3a, 0x51, 0xdb, 0xcc, 0xf7, 0xb1, 0xb2, 0x02, 0x4b, 0x31, - 0x5b, 0xc0, 0x1d, 0xfe, 0x8f, 0x24, 0x58, 0xd8, 0xeb, 0x5a, 0xb5, 0xbd, 0xba, 0xee, 0x18, 0xfc, - 0xe5, 0x93, 0x6f, 0xcf, 0x15, 0xc8, 0x61, 0xbb, 0xed, 0xd4, 0x90, 0x56, 0x6b, 0xb4, 0xb1, 0x8b, - 0x1c, 0xbe, 0x41, 0x33, 0xac, 0x75, 0x8b, 0x35, 0xca, 0x4b, 0x90, 0xc2, 0x84, 0x59, 0x3c, 0x1f, - 0x25, 0xd5, 0x69, 0xfa, 0x5d, 0x35, 0xe4, 0xdb, 0x90, 0x61, 0x4f, 0xb0, 0x2c, 0x7d, 0x99, 0x18, - 0x31, 0x7d, 0x09, 0x8c, 0x89, 0x34, 0x2b, 0x4b, 0xb0, 0x18, 0x99, 0x9e, 0xb8, 0xbc, 0x24, 0x61, - 0x96, 0xf4, 0x09, 0x1b, 0x1f, 0xc3, 0xac, 0x2e, 0x43, 0xc6, 0x33, 0x2b, 0x3e, 0xed, 0xb4, 0x0a, - 0xa2, 0xa9, 0x6a, 0xf8, 0x02, 0xae, 0x84, 0x2f, 0xe0, 0x92, 0x4b, 0x30, 0xcd, 0xf7, 0x98, 0x67, - 0xc4, 0xc5, 0x27, 0x19, 0xb4, 0x97, 0xac, 0xed, 0xbd, 0x60, 0x79, 0x6d, 0xf4, 0xbd, 0x36, 0xfc, - 0xf0, 0x32, 0x75, 0xb6, 0x87, 0x97, 0x4b, 0x00, 0x22, 0x27, 0x68, 0xb2, 0x27, 0xae, 0x84, 0x9a, - 0xe6, 0x2d, 0xb4, 0x06, 0x22, 0x98, 0xa6, 0x4e, 0x9d, 0x25, 0x4d, 0xbd, 0xcb, 0xeb, 0x2e, 0x7a, - 0x69, 0x2e, 0x2a, 0x2b, 0x3d, 0xa2, 0xac, 0x22, 0x61, 0xf6, 0xd2, 0x53, 0x54, 0xe2, 0x2d, 0x98, - 0x16, 0xd9, 0x66, 0x18, 0x31, 0xdb, 0x2c, 0x18, 0xfc, 0x49, 0xf3, 0x4c, 0x30, 0x69, 0xbe, 0x05, - 0x59, 0xf6, 0x2a, 0xcf, 0x8b, 0x4e, 0xb3, 0x23, 0x16, 0x9d, 0x66, 0xe8, 0x63, 0x3d, 0xaf, 0x37, - 0x7d, 0x15, 0x68, 0xbd, 0x28, 0x0d, 0x73, 0x90, 0xa3, 0x99, 0x06, 0xb2, 0x5c, 0xd3, 0xed, 0xd2, - 0x17, 0xad, 0xb4, 0x2a, 0x93, 0xbe, 0x77, 0x68, 0x57, 0x95, 0xf7, 0xc8, 0xef, 0x40, 0x3e, 0x84, - 0x1e, 0xbc, 0x3e, 0xa2, 0x32, 0x1e, 0x6e, 0xa8, 0xb9, 0x20, 0x66, 0x28, 0x0b, 0x30, 0x17, 0xb4, - 0x69, 0x6e, 0xec, 0xdf, 0x97, 0x60, 0x45, 0x9c, 0x79, 0xcf, 0xb8, 0x14, 0x4a, 0xf9, 0x1f, 0x09, - 0x2e, 0xc6, 0xcf, 0x85, 0x1f, 0xbd, 0x75, 0x98, 0xad, 0xe9, 0xb5, 0x3a, 0x0a, 0x96, 0xa9, 0xf3, - 0xd3, 0xf7, 0x8d, 0xd8, 0x15, 0xf2, 0x15, 0xba, 0xfb, 0xc7, 0x0f, 0x88, 0x2f, 0x52, 0xa1, 0xfe, - 0x26, 0xd9, 0x82, 0x05, 0x43, 0x77, 0xf5, 0x03, 0x1d, 0x87, 0x07, 0x9b, 0x38, 0xe7, 0x60, 0x73, - 0x42, 0xae, 0xbf, 0x55, 0xf9, 0x27, 0x09, 0x96, 0x85, 0xea, 0x7c, 0xcb, 0xee, 0xd9, 0xd8, 0x9f, - 0x3a, 0xae, 0xdb, 0xd8, 0xd5, 0x74, 0xc3, 0x70, 0x10, 0xc6, 0x62, 0x17, 0x48, 0xdb, 0x6d, 0xd6, - 0x34, 0x08, 0x2e, 0xc3, 0x7b, 0x98, 0x18, 0xf5, 0x3c, 0x9c, 0x3c, 0xff, 0x79, 0xa8, 0x7c, 0x3c, - 0xd1, 0x33, 0xb0, 0x80, 0x66, 0x7c, 0x4f, 0x9f, 0x83, 0x19, 0x3a, 0x4f, 0xac, 0x59, 0xed, 0xe6, - 0x01, 0x3f, 0x0c, 0x92, 0x6a, 0x96, 0x35, 0x3e, 0xa2, 0x6d, 0xf2, 0x0a, 0xa4, 0x85, 0x72, 0xb8, - 0x34, 0xb1, 0x9a, 0x58, 0x4f, 0xaa, 0x29, 0xae, 0x1d, 0x96, 0xdf, 0x87, 0x7c, 0x4f, 0x3d, 0xba, - 0x95, 0x03, 0x6b, 0xef, 0x3d, 0x5a, 0xa2, 0x82, 0xf7, 0xea, 0xb3, 0x45, 0xf8, 0x68, 0xac, 0x91, - 0xb3, 0x02, 0x6d, 0xf2, 0x6b, 0xb0, 0xc8, 0xc6, 0xae, 0xd9, 0x96, 0xeb, 0xd8, 0x8d, 0x06, 0x72, - 0x44, 0x01, 0xd0, 0x24, 0x5d, 0xc8, 0x79, 0xda, 0xbd, 0xe5, 0xf5, 0xf2, 0xba, 0x1e, 0x82, 0x2d, - 0x7c, 0xbb, 0xd8, 0x4b, 0xa6, 0xf8, 0x54, 0x2a, 0x50, 0xdc, 0x6a, 0xd8, 0x18, 0xd1, 0xc3, 0x47, - 0x6c, 0xb1, 0x7f, 0xff, 0xa4, 0xc0, 0xfe, 0x29, 0x73, 0x20, 0xfb, 0xe9, 0xb9, 0xe7, 0xbe, 0x02, - 0xf9, 0x1d, 0xe4, 0x8e, 0x2a, 0xe3, 0x03, 0x28, 0xf4, 0xa8, 0xf9, 0xd2, 0x3f, 0x00, 0xe0, 0xe4, - 0x24, 0xbc, 0x64, 0x5e, 0x74, 0x6d, 0x14, 0xc3, 0xa6, 0x62, 0xe8, 0x62, 0xb1, 0x6d, 0x21, 0x7f, - 0x2a, 0x3f, 0x93, 0xa0, 0xc8, 0x92, 0x43, 0xfe, 0xab, 0x66, 0xff, 0x29, 0xc9, 0xdb, 0x90, 0x22, - 0xa1, 0xc3, 0x11, 0x01, 0xb9, 0x09, 0x5a, 0x4a, 0xf5, 0xd2, 0xe0, 0x42, 0x2d, 0x96, 0xd6, 0x65, - 0x1c, 0xaa, 0xc7, 0xeb, 0x7f, 0x4e, 0x4e, 0x04, 0x9e, 0x93, 0xab, 0x90, 0xef, 0x98, 0xd8, 0x3c, - 0x30, 0x1b, 0xa6, 0xdb, 0x1d, 0xef, 0xa5, 0x33, 0xd7, 0x63, 0xa4, 0xe1, 0xc2, 0x1c, 0xc8, 0x7e, - 0xdd, 0xf8, 0x16, 0x7c, 0x2c, 0xc1, 0xa5, 0x1d, 0xe4, 0xaa, 0xbd, 0xdf, 0xec, 0x3c, 0x64, 0xbf, - 0xd7, 0xf1, 0x62, 0x9d, 0x07, 0x30, 0x45, 0x0b, 0x26, 0x88, 0xcb, 0x26, 0xfa, 0x9a, 0xa4, 0xef, - 0x47, 0x3f, 0x2c, 0xef, 0xe1, 0x7d, 0xd2, 0xd2, 0x0a, 0x95, 0xcb, 0x20, 0x8e, 0xcc, 0x43, 0x26, - 0xfa, 0x8e, 0xc9, 0xe3, 0x8b, 0x0c, 0x6f, 0x23, 0xb6, 0xac, 0xfc, 0x68, 0x02, 0xca, 0xfd, 0xa6, - 0xc4, 0xb7, 0xfd, 0x37, 0x20, 0xc7, 0xb6, 0x84, 0xff, 0xb8, 0x48, 0xcc, 0xed, 0xdd, 0x11, 0x1f, - 0xfe, 0x06, 0x8b, 0x67, 0xc6, 0x21, 0x5a, 0x59, 0x91, 0x04, 0xf3, 0x70, 0xd1, 0xb6, 0xdc, 0x05, - 0x39, 0x4a, 0xe4, 0x2f, 0x98, 0x48, 0xb2, 0x82, 0x89, 0x87, 0xc1, 0x82, 0x89, 0xd7, 0xc7, 0x5c, - 0x3b, 0x6f, 0x66, 0xbd, 0x1a, 0x0a, 0xe5, 0x23, 0x58, 0xdd, 0x41, 0xee, 0x9d, 0x07, 0x6f, 0x0f, - 0xd8, 0xb3, 0xc7, 0xbc, 0xd6, 0x93, 0x78, 0x85, 0x58, 0x9b, 0x71, 0xc7, 0xf6, 0x6a, 0x76, 0x68, - 0xf9, 0x27, 0xf9, 0x0b, 0x2b, 0xbf, 0x23, 0xc1, 0xda, 0x80, 0xc1, 0xf9, 0xee, 0x7c, 0x00, 0x45, - 0x9f, 0x58, 0x9a, 0x18, 0x11, 0x93, 0xb8, 0x79, 0x86, 0x49, 0xa8, 0x05, 0x27, 0xd8, 0x80, 0x95, - 0x1f, 0x48, 0x30, 0x47, 0x8b, 0x4b, 0x04, 0x7e, 0x8f, 0x71, 0xd6, 0x7f, 0x33, 0x7c, 0xff, 0xfe, - 0xea, 0xd0, 0xfb, 0x77, 0xdc, 0x50, 0xbd, 0x3b, 0xf7, 0x31, 0xcc, 0x87, 0x08, 0xf8, 0x3a, 0xa8, - 0x90, 0x0a, 0x3d, 0x4c, 0xbf, 0x36, 0xee, 0x50, 0xfc, 0x19, 0xda, 0x93, 0xa3, 0xfc, 0xbe, 0x04, - 0x73, 0x2a, 0xd2, 0x5b, 0xad, 0x06, 0x4b, 0x68, 0xe0, 0x31, 0x34, 0xdf, 0x0b, 0x6b, 0x1e, 0x5f, - 0xc8, 0xe5, 0xff, 0x7d, 0x1b, 0xdb, 0x8e, 0xe8, 0x70, 0x3d, 0xed, 0x17, 0x61, 0x3e, 0x44, 0xc0, - 0x67, 0xfa, 0xe7, 0x13, 0x30, 0xcf, 0x6c, 0x25, 0x6c, 0x9d, 0x77, 0x61, 0xd2, 0x2b, 0xd4, 0xcb, - 0xf9, 0x53, 0x0e, 0x71, 0x88, 0x79, 0x07, 0xe9, 0xc6, 0x03, 0xe4, 0xba, 0xc8, 0xa1, 0x35, 0x2f, - 0xb4, 0x36, 0x82, 0xb2, 0x0f, 0x0a, 0x17, 0xa2, 0xf7, 0xb3, 0x44, 0xdc, 0xfd, 0xec, 0x75, 0x28, - 0x99, 0x16, 0xa1, 0x30, 0x3b, 0x48, 0x43, 0x96, 0x07, 0x27, 0xbd, 0xb2, 0x9e, 0x79, 0xaf, 0xff, - 0xae, 0x25, 0x9c, 0xbd, 0x6a, 0xc8, 0x2f, 0x41, 0xb1, 0xa9, 0x9f, 0x9a, 0xcd, 0x76, 0x53, 0x6b, - 0x11, 0x7a, 0x6c, 0x7e, 0xc4, 0x7e, 0x9c, 0x96, 0x54, 0xf3, 0xbc, 0x63, 0x57, 0x3f, 0x42, 0x7b, - 0xe6, 0x47, 0x48, 0x7e, 0x01, 0xf2, 0xb4, 0x82, 0x8f, 0x12, 0xb2, 0xd2, 0xb3, 0x29, 0x5a, 0x7a, - 0x46, 0x0b, 0xfb, 0x08, 0x19, 0x2b, 0x6f, 0xff, 0x0f, 0xf6, 0x43, 0xa7, 0xc0, 0x7a, 0x71, 0x43, - 0x7a, 0x4a, 0x0b, 0x16, 0xeb, 0x97, 0x13, 0x4f, 0xd1, 0x2f, 0xe3, 0x74, 0x4d, 0xc4, 0xe9, 0xfa, - 0xcf, 0x12, 0x2c, 0xee, 0xb6, 0x9d, 0x23, 0xf4, 0x45, 0xb4, 0x0e, 0x65, 0x19, 0x4a, 0x51, 0xe5, - 0xc4, 0xb3, 0xfb, 0x04, 0x2c, 0x3e, 0x44, 0x5f, 0x50, 0xcd, 0x3f, 0x17, 0xbf, 0xd8, 0x84, 0x52, - 0x74, 0xc1, 0xb8, 0x63, 0xc4, 0xc8, 0x90, 0xe2, 0x64, 0xfc, 0x88, 0x96, 0x94, 0x1f, 0x3a, 0x08, - 0xd7, 0xfd, 0xb9, 0xf7, 0x71, 0xc0, 0xf3, 0xbd, 0x30, 0x78, 0xfe, 0xea, 0x88, 0xe0, 0xd9, 0x77, - 0xd4, 0x1e, 0x86, 0xd2, 0x2a, 0xf3, 0x38, 0x3a, 0x6e, 0x34, 0x3f, 0x94, 0xe0, 0xa5, 0x1d, 0x64, - 0x21, 0x47, 0x77, 0xd1, 0x03, 0x1d, 0xbb, 0xe2, 0x86, 0x1c, 0x72, 0xbf, 0x67, 0x71, 0xe1, 0xbd, - 0x06, 0x2f, 0x8f, 0x34, 0x33, 0xae, 0xc9, 0x36, 0xac, 0x04, 0x63, 0xaf, 0x60, 0x5e, 0xed, 0x2a, - 0xe4, 0x1d, 0xd4, 0xb4, 0x5d, 0xcf, 0x3e, 0x59, 0xdc, 0x90, 0x56, 0x73, 0xac, 0x99, 0x1b, 0x28, - 0x56, 0xda, 0x70, 0x31, 0x5e, 0x0e, 0x37, 0x8c, 0x6f, 0xc1, 0x14, 0xbb, 0x7d, 0xf1, 0xb8, 0xe3, - 0xcd, 0x11, 0x03, 0x43, 0x7e, 0xbb, 0x08, 0x8b, 0xe5, 0xc2, 0x94, 0xbf, 0x4b, 0xc2, 0x42, 0x3c, - 0xc9, 0xa0, 0x5b, 0xc2, 0x57, 0x61, 0xb1, 0xa9, 0x9f, 0x6a, 0x61, 0xec, 0xed, 0x15, 0x95, 0xcf, - 0x35, 0xf5, 0xd3, 0x70, 0xe4, 0x65, 0xc8, 0xf7, 0xa1, 0xc0, 0x24, 0x36, 0xec, 0x9a, 0xde, 0x18, - 0x2f, 0x4f, 0xc8, 0xc2, 0xe3, 0x07, 0x84, 0x91, 0x66, 0x98, 0x3e, 0x8a, 0x2e, 0x2c, 0xcb, 0x65, - 0xbf, 0x7d, 0xae, 0x85, 0xa9, 0xa8, 0x81, 0x6d, 0x61, 0xa1, 0x72, 0x68, 0xaf, 0xe4, 0xdf, 0x95, - 0x60, 0xb6, 0xae, 0x5b, 0x86, 0xdd, 0xe1, 0x41, 0x3f, 0x35, 0x42, 0x72, 0xa5, 0x1c, 0xa7, 0xa8, - 0xb9, 0xcf, 0x04, 0xee, 0x71, 0xc1, 0xde, 0x2d, 0x98, 0x4f, 0x42, 0xae, 0x47, 0x3a, 0x96, 0x7f, - 0x20, 0xc1, 0x6c, 0xcc, 0x84, 0x63, 0xea, 0x9c, 0xdf, 0x0f, 0x86, 0xed, 0x3b, 0xe7, 0x9a, 0xe3, - 0x2e, 0x72, 0xf8, 0x78, 0xbe, 0x30, 0x7e, 0xf9, 0x7b, 0x12, 0x2c, 0xf6, 0x99, 0x7c, 0xcc, 0x84, - 0xd4, 0xe0, 0x84, 0xbe, 0x3e, 0xe2, 0x84, 0x22, 0x03, 0xd0, 0x80, 0xde, 0x77, 0x99, 0x78, 0x17, - 0xe6, 0x63, 0x69, 0xe4, 0xb7, 0xe0, 0xa2, 0xb7, 0x67, 0x71, 0x86, 0x2b, 0x51, 0xc3, 0x5d, 0x12, - 0x34, 0x11, 0xeb, 0x55, 0xfe, 0x44, 0x82, 0xd5, 0x61, 0xeb, 0x21, 0x2b, 0x30, 0xa3, 0xd7, 0x8e, - 0x91, 0x11, 0x12, 0x9b, 0xa1, 0x8d, 0xdc, 0x0d, 0xde, 0x87, 0x65, 0x1f, 0x4d, 0xf8, 0x36, 0x3c, - 0x6a, 0xa1, 0xf1, 0xa2, 0x27, 0xf2, 0x71, 0xe0, 0x5a, 0xbc, 0xd9, 0xfa, 0xe4, 0xd3, 0xf2, 0x85, - 0x9f, 0x7e, 0x5a, 0xbe, 0xf0, 0xf3, 0x4f, 0xcb, 0xd2, 0x6f, 0x3e, 0x29, 0x4b, 0x3f, 0x7e, 0x52, - 0x96, 0xfe, 0xf6, 0x49, 0x59, 0xfa, 0xe4, 0x49, 0x59, 0xfa, 0xd7, 0x27, 0x65, 0xe9, 0xdf, 0x9f, - 0x94, 0x2f, 0xfc, 0xfc, 0x49, 0x59, 0xfa, 0xf8, 0xb3, 0xf2, 0x85, 0x4f, 0x3e, 0x2b, 0x5f, 0xf8, - 0xe9, 0x67, 0xe5, 0x0b, 0xef, 0xdd, 0x3a, 0xb2, 0x7b, 0x5b, 0x60, 0xda, 0x03, 0xff, 0xcb, 0xcb, - 0xaf, 0x04, 0x5b, 0x0e, 0xa6, 0xe8, 0x24, 0x6f, 0xfe, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x33, - 0xd4, 0x5d, 0xc2, 0x24, 0x46, 0x00, 0x00, + // 4054 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x3c, 0x4b, 0x6c, 0x1c, 0x47, + 0x76, 0x6a, 0x0e, 0x3f, 0x33, 0x6f, 0x86, 0xf3, 0x69, 0xfe, 0x86, 0xa4, 0x34, 0x22, 0xdb, 0x96, + 0x45, 0x7f, 0x34, 0xb4, 0xa4, 0x5d, 0xdb, 0xab, 0xac, 0xd7, 0x91, 0x28, 0x89, 0x1a, 0x41, 0xd2, + 0xd2, 0x4d, 0xae, 0x6c, 0x78, 0xd7, 0xdb, 0x6e, 0x4e, 0x17, 0x39, 0x1d, 0xce, 0x74, 0x8f, 0xbb, + 0x7a, 0x86, 0x1c, 0xe7, 0x90, 0xcf, 0x22, 0x01, 0x76, 0x03, 0x04, 0x06, 0x72, 0x59, 0x20, 0x9b, + 0x4b, 0x80, 0x20, 0x8b, 0x00, 0x41, 0x0e, 0x39, 0x04, 0x7b, 0xc8, 0x35, 0xc8, 0x29, 0x31, 0x02, + 0x04, 0x59, 0x6c, 0x0e, 0x89, 0x65, 0x04, 0x48, 0x90, 0x1c, 0xf6, 0x90, 0x43, 0x8e, 0x41, 0xfd, + 0xfa, 0x3f, 0x3f, 0x52, 0x8a, 0x37, 0x5e, 0xdf, 0xd8, 0x55, 0xef, 0xbd, 0x7a, 0xaf, 0xde, 0xa7, + 0x5e, 0xbd, 0x7a, 0x43, 0xf8, 0xba, 0x8b, 0x5a, 0x6d, 0xdb, 0xd1, 0x9b, 0x9b, 0x18, 0x39, 0x5d, + 0xe4, 0x6c, 0xea, 0x6d, 0x73, 0xb3, 0x61, 0x62, 0xd7, 0x76, 0x7a, 0x64, 0xc4, 0xac, 0xa3, 0xcd, + 0xee, 0xd5, 0x4d, 0x07, 0x7d, 0xd8, 0x41, 0xd8, 0xd5, 0x1c, 0x84, 0xdb, 0xb6, 0x85, 0x51, 0xb5, + 0xed, 0xd8, 0xae, 0x2d, 0x5f, 0x12, 0xd8, 0x55, 0x86, 0x5d, 0xd5, 0xdb, 0x66, 0x35, 0x8c, 0x5d, + 0xed, 0x5e, 0x5d, 0xa9, 0x1c, 0xda, 0xf6, 0x61, 0x13, 0x6d, 0x52, 0xa4, 0xfd, 0xce, 0xc1, 0xa6, + 0xd1, 0x71, 0x74, 0xd7, 0xb4, 0x2d, 0x46, 0x66, 0xe5, 0x62, 0x74, 0xde, 0x35, 0x5b, 0x08, 0xbb, + 0x7a, 0xab, 0xcd, 0x01, 0xd6, 0x0d, 0xd4, 0x46, 0x96, 0x81, 0xac, 0xba, 0x89, 0xf0, 0xe6, 0xa1, + 0x7d, 0x68, 0xd3, 0x71, 0xfa, 0x17, 0x07, 0x79, 0xde, 0x13, 0x84, 0x48, 0x50, 0xb7, 0x5b, 0x2d, + 0xdb, 0x22, 0x9c, 0xb7, 0x10, 0xc6, 0xfa, 0x21, 0x67, 0x78, 0xe5, 0x52, 0x08, 0x8a, 0x73, 0x1a, + 0x07, 0xbb, 0x1c, 0x02, 0x73, 0x75, 0x7c, 0xf4, 0x61, 0x07, 0x75, 0x50, 0x1c, 0x30, 0xbc, 0x2a, + 0xb2, 0x3a, 0x2d, 0x4c, 0x80, 0x8e, 0x6d, 0xe7, 0xe8, 0xa0, 0x69, 0x1f, 0x73, 0xa8, 0x17, 0x42, + 0x50, 0x62, 0x32, 0x4e, 0xed, 0xb9, 0x10, 0xdc, 0x87, 0x1d, 0x94, 0xc4, 0x5b, 0x58, 0x84, 0x03, + 0xdd, 0x6c, 0x76, 0x9c, 0x04, 0xce, 0x5e, 0x19, 0xa0, 0xd8, 0x38, 0xf4, 0x8b, 0x49, 0xd0, 0x9e, + 0x38, 0x6c, 0x37, 0x39, 0xe8, 0xcb, 0x03, 0x41, 0x23, 0x92, 0x5f, 0x1e, 0x08, 0x4c, 0x36, 0x96, + 0x03, 0x5e, 0x49, 0x02, 0xec, 0xbf, 0x53, 0xd5, 0x24, 0x70, 0x4b, 0x6f, 0x21, 0xdc, 0xd6, 0xeb, + 0x09, 0xbb, 0xf1, 0x6a, 0x12, 0xbc, 0x83, 0xda, 0x4d, 0xb3, 0x4e, 0x0d, 0x31, 0x8e, 0x71, 0x3d, + 0x09, 0xa3, 0x8d, 0x1c, 0x6c, 0x62, 0x17, 0x59, 0x6c, 0x0d, 0x74, 0x82, 0xea, 0x1d, 0x82, 0x8e, + 0x39, 0xd2, 0x5b, 0x23, 0x20, 0x09, 0xa1, 0xb4, 0x56, 0xc7, 0xd5, 0xf7, 0x9b, 0x48, 0xc3, 0xae, + 0xee, 0x8a, 0x55, 0x5f, 0x4b, 0xb4, 0x94, 0xa1, 0x8e, 0xb8, 0x72, 0x23, 0x69, 0x61, 0xdd, 0x68, + 0x99, 0xd6, 0x50, 0x5c, 0xe5, 0xf7, 0xa6, 0xe1, 0xc2, 0xae, 0xab, 0x3b, 0xee, 0x3b, 0x7c, 0xb9, + 0x3b, 0x42, 0x2c, 0x95, 0x21, 0xc8, 0xeb, 0x90, 0xf3, 0xf6, 0x56, 0x33, 0x8d, 0xb2, 0xb4, 0x26, + 0x6d, 0x64, 0xd4, 0xac, 0x37, 0x56, 0x33, 0xe4, 0x3a, 0xcc, 0x62, 0x42, 0x43, 0xe3, 0x8b, 0x94, + 0x27, 0xd6, 0xa4, 0x8d, 0xec, 0xb5, 0x6f, 0x78, 0x8a, 0xa2, 0xa1, 0x21, 0x22, 0x50, 0xb5, 0x7b, + 0xb5, 0x3a, 0x70, 0x65, 0x35, 0x47, 0x89, 0x0a, 0x3e, 0x1a, 0xb0, 0xd0, 0xd6, 0x1d, 0x64, 0xb9, + 0x9a, 0xb7, 0xf3, 0x9a, 0x69, 0x1d, 0xd8, 0xe5, 0x14, 0x5d, 0xec, 0x2b, 0xd5, 0xa4, 0x70, 0xe4, + 0x59, 0x64, 0xf7, 0x6a, 0x75, 0x87, 0x62, 0x7b, 0xab, 0xd4, 0xac, 0x03, 0x5b, 0x9d, 0x6b, 0xc7, + 0x07, 0xe5, 0x32, 0xcc, 0xe8, 0x2e, 0xa1, 0xe6, 0x96, 0x27, 0xd7, 0xa4, 0x8d, 0x29, 0x55, 0x7c, + 0xca, 0x2d, 0x50, 0x3c, 0x0d, 0xfa, 0x5c, 0xa0, 0x93, 0xb6, 0xc9, 0x42, 0x9a, 0x46, 0x62, 0x57, + 0x79, 0x8a, 0x32, 0xb4, 0x52, 0x65, 0x81, 0xad, 0x2a, 0x02, 0x5b, 0x75, 0x4f, 0x04, 0xb6, 0x5b, + 0x93, 0x1f, 0xff, 0xcb, 0x45, 0x49, 0xbd, 0x78, 0x1c, 0x95, 0xfc, 0x8e, 0x47, 0x89, 0xc0, 0xca, + 0x0d, 0x58, 0xae, 0xdb, 0x96, 0x6b, 0x5a, 0x1d, 0xa4, 0xe9, 0x58, 0xb3, 0xd0, 0xb1, 0x66, 0x5a, + 0xa6, 0x6b, 0xea, 0xae, 0xed, 0x94, 0xa7, 0xd7, 0xa4, 0x8d, 0xfc, 0xb5, 0x2b, 0xe1, 0x3d, 0xa6, + 0xde, 0x45, 0x84, 0xdd, 0xe2, 0x78, 0x37, 0xf1, 0x23, 0x74, 0x5c, 0x13, 0x48, 0xea, 0x62, 0x3d, + 0x71, 0x5c, 0x7e, 0x08, 0x25, 0x31, 0x63, 0x68, 0x3c, 0xac, 0x94, 0x67, 0xa8, 0x1c, 0x6b, 0xe1, + 0x15, 0xf8, 0x24, 0x59, 0xe3, 0x2e, 0xfb, 0x53, 0x2d, 0x7a, 0xa8, 0x7c, 0x44, 0x7e, 0x0c, 0x8b, + 0x4d, 0x1d, 0xbb, 0x5a, 0xdd, 0x6e, 0xb5, 0x9b, 0x88, 0xee, 0x8c, 0x83, 0x70, 0xa7, 0xe9, 0x96, + 0xd3, 0x49, 0x34, 0x79, 0x88, 0xa1, 0x3a, 0xea, 0x35, 0x6d, 0xdd, 0xc0, 0xea, 0x3c, 0xc1, 0xdf, + 0xf2, 0xd0, 0x55, 0x8a, 0x2d, 0x7f, 0x17, 0x56, 0x0f, 0x4c, 0x07, 0xbb, 0x9a, 0xa7, 0x05, 0x12, + 0x45, 0xb4, 0x7d, 0xbd, 0x7e, 0x64, 0x1f, 0x1c, 0x94, 0x33, 0x94, 0xf8, 0x72, 0x6c, 0xe3, 0x6f, + 0xf3, 0x13, 0xe7, 0xd6, 0xe4, 0x0f, 0xc9, 0xbe, 0x97, 0x29, 0x0d, 0x61, 0x76, 0x7b, 0x3a, 0x3e, + 0xba, 0xc5, 0x08, 0x28, 0xaf, 0x43, 0xa5, 0x9f, 0x49, 0x32, 0xaf, 0x91, 0x17, 0x60, 0xda, 0xe9, + 0x58, 0xbe, 0x1f, 0x4c, 0x39, 0x1d, 0xab, 0x66, 0x28, 0xff, 0x29, 0xc1, 0xe2, 0x36, 0x72, 0x1f, + 0x32, 0xaf, 0xde, 0x25, 0x4e, 0x3d, 0x86, 0xff, 0x6c, 0x43, 0xc6, 0xb3, 0x26, 0xee, 0x3b, 0x2f, + 0xf6, 0xdb, 0xa1, 0x38, 0x6b, 0x3e, 0xae, 0x7c, 0x1d, 0x16, 0xd1, 0x49, 0x1b, 0xd5, 0x5d, 0x64, + 0x68, 0x16, 0x3a, 0x71, 0x35, 0xd4, 0x25, 0x0e, 0x63, 0x1a, 0xd4, 0x49, 0x52, 0xea, 0x9c, 0x98, + 0x7d, 0x84, 0x4e, 0xdc, 0x3b, 0x64, 0xae, 0x66, 0xc8, 0xaf, 0xc2, 0x7c, 0xbd, 0xe3, 0x50, 0xcf, + 0xda, 0x77, 0x74, 0xab, 0xde, 0xd0, 0x5c, 0xfb, 0x08, 0x59, 0xd4, 0xf6, 0x73, 0xaa, 0xcc, 0xe7, + 0x6e, 0xd1, 0xa9, 0x3d, 0x32, 0xa3, 0xfc, 0x59, 0x1a, 0x96, 0x62, 0xd2, 0xf2, 0x0d, 0x0a, 0xc9, + 0x22, 0x9d, 0x41, 0x96, 0x1a, 0xcc, 0xfa, 0x5a, 0xee, 0xb5, 0x11, 0xdf, 0x98, 0xe7, 0x87, 0x11, + 0xdb, 0xeb, 0xb5, 0x91, 0x9a, 0x3b, 0x0e, 0x7c, 0xc9, 0x0a, 0xcc, 0x26, 0xed, 0x46, 0xd6, 0x0a, + 0xec, 0xc2, 0xd7, 0x60, 0xb9, 0xed, 0xa0, 0xae, 0x69, 0x77, 0xb0, 0x46, 0xe3, 0x0e, 0x32, 0x7c, + 0xf8, 0x49, 0x0a, 0xbf, 0x28, 0x00, 0x76, 0xd9, 0xbc, 0x40, 0xbd, 0x02, 0x73, 0xd4, 0xda, 0x99, + 0x69, 0x7a, 0x48, 0x53, 0x14, 0xa9, 0x48, 0xa6, 0xee, 0x92, 0x19, 0x01, 0xbe, 0x05, 0x40, 0xad, + 0x96, 0x66, 0x15, 0xd4, 0x8d, 0x63, 0x52, 0x79, 0x49, 0x07, 0x11, 0x8c, 0x18, 0xe8, 0xdb, 0xe4, + 0x43, 0xcd, 0xb8, 0xe2, 0x4f, 0x79, 0x07, 0x4a, 0xd8, 0x35, 0xeb, 0x47, 0x3d, 0x2d, 0x40, 0x6b, + 0x66, 0x0c, 0x5a, 0x05, 0x86, 0xee, 0x0d, 0xc8, 0xbf, 0x0e, 0x2f, 0xc7, 0x28, 0x6a, 0xb8, 0xde, + 0x40, 0x46, 0xa7, 0x89, 0x34, 0xd7, 0x66, 0xbb, 0x42, 0x23, 0x9c, 0xdd, 0x71, 0xcb, 0xd9, 0xd1, + 0x7c, 0xed, 0x52, 0x64, 0x99, 0x5d, 0x4e, 0x70, 0xcf, 0xa6, 0x9b, 0xb8, 0xc7, 0xa8, 0xf5, 0xb5, + 0xc1, 0xd9, 0x7e, 0x36, 0x28, 0x7f, 0x1b, 0xf2, 0x9e, 0x79, 0xd0, 0x43, 0xb4, 0x5c, 0xa0, 0x01, + 0x31, 0xf9, 0x1c, 0xf0, 0xe2, 0x62, 0xcc, 0xe4, 0x98, 0xf5, 0x7a, 0xa6, 0x46, 0x3f, 0xe5, 0x77, + 0xa0, 0x10, 0x22, 0xde, 0xc1, 0xe5, 0x22, 0xa5, 0x5e, 0xed, 0x13, 0x6e, 0x13, 0xc9, 0x76, 0xb0, + 0x9a, 0x0f, 0xd2, 0xed, 0x60, 0xf9, 0x7d, 0x28, 0x75, 0x49, 0x46, 0x60, 0x5b, 0x1a, 0x4b, 0xc7, + 0x4c, 0x84, 0xcb, 0x25, 0xba, 0x95, 0xaf, 0x56, 0x07, 0xe4, 0xd3, 0x64, 0x8d, 0xc7, 0x0c, 0xf1, + 0x9e, 0xc0, 0x53, 0x8b, 0xdd, 0xc8, 0x88, 0xfc, 0x0d, 0x38, 0x6f, 0x12, 0xf3, 0x8d, 0xaa, 0x11, + 0x59, 0xc4, 0x51, 0x8d, 0xb2, 0xbc, 0x26, 0x6d, 0xa4, 0xd5, 0xb2, 0x89, 0x77, 0xc3, 0x5a, 0xb9, + 0xc3, 0xe6, 0xe5, 0xaf, 0xc0, 0x52, 0xcc, 0x92, 0xdd, 0x13, 0x1a, 0xee, 0xe6, 0x58, 0x00, 0x09, + 0x5b, 0xf3, 0xde, 0x89, 0x55, 0x33, 0xee, 0x4f, 0xa6, 0xd3, 0xc5, 0xcc, 0xfd, 0xc9, 0x74, 0xa6, + 0x08, 0xf7, 0x27, 0xd3, 0x50, 0xcc, 0xde, 0x9f, 0x4c, 0xe7, 0x8a, 0xb3, 0xf7, 0x27, 0xd3, 0xf9, + 0x62, 0x41, 0xf9, 0x2f, 0x09, 0x96, 0x76, 0xec, 0x66, 0xf3, 0x97, 0x24, 0x36, 0xfe, 0xdb, 0x0c, + 0x94, 0xe3, 0xe2, 0x7e, 0x19, 0x1c, 0xbf, 0x0c, 0x8e, 0x4f, 0x3d, 0x38, 0xe6, 0xfa, 0x06, 0xc7, + 0xc4, 0x30, 0x93, 0x7f, 0x6a, 0x61, 0xe6, 0xff, 0x67, 0xec, 0x1d, 0x10, 0xdc, 0x4a, 0xe3, 0x05, + 0xb7, 0xd9, 0x62, 0x5e, 0xf9, 0xbe, 0x04, 0xab, 0x2a, 0xc2, 0xc8, 0x8d, 0x84, 0xd2, 0xcf, 0x21, + 0xb4, 0x29, 0x15, 0x38, 0x9f, 0xcc, 0x0a, 0x0b, 0x3b, 0xca, 0xcf, 0x26, 0x60, 0x4d, 0x45, 0x75, + 0xdb, 0x31, 0x82, 0x49, 0x2f, 0x77, 0xd4, 0x31, 0x18, 0x7e, 0x17, 0xe4, 0xf8, 0xf5, 0x67, 0x7c, + 0xce, 0x4b, 0xb1, 0x7b, 0x8f, 0x7c, 0x11, 0xb2, 0x9e, 0x37, 0x79, 0x21, 0x08, 0xc4, 0x50, 0xcd, + 0x90, 0x97, 0x60, 0x86, 0x7a, 0x9e, 0x17, 0x6f, 0xa6, 0xc9, 0x67, 0xcd, 0x90, 0x2f, 0x00, 0x88, + 0xab, 0x2d, 0x0f, 0x2b, 0x19, 0x35, 0xc3, 0x47, 0x6a, 0x86, 0xfc, 0x01, 0xe4, 0xda, 0x76, 0xb3, + 0xe9, 0xdd, 0x4c, 0x59, 0x44, 0x79, 0x73, 0xe8, 0xcd, 0x94, 0x84, 0xf0, 0xe0, 0x66, 0x05, 0x75, + 0xab, 0x66, 0x09, 0x49, 0xfe, 0xa1, 0xfc, 0xe3, 0x0c, 0xac, 0x0f, 0xd8, 0x5c, 0x1e, 0xf9, 0x63, + 0x01, 0x5b, 0x3a, 0x75, 0xc0, 0x1e, 0x18, 0x8c, 0x27, 0x06, 0x06, 0xe3, 0x57, 0x40, 0x16, 0x7b, + 0x6a, 0x44, 0x03, 0x7e, 0xd1, 0x9b, 0x11, 0xd0, 0x1b, 0x50, 0xec, 0x13, 0xec, 0xf3, 0x38, 0x4c, + 0x37, 0x76, 0x86, 0x4c, 0xc5, 0xcf, 0x90, 0xc0, 0xad, 0x7a, 0x3a, 0x7c, 0xab, 0x7e, 0x03, 0xca, + 0x3c, 0xb8, 0x06, 0xee, 0xd4, 0x3c, 0x63, 0x99, 0xa1, 0x19, 0xcb, 0x22, 0x9b, 0xf7, 0xef, 0xc9, + 0x3c, 0x5f, 0x39, 0x0c, 0x18, 0x24, 0x33, 0x0f, 0xeb, 0xc0, 0xe6, 0x77, 0xcc, 0xaf, 0x0d, 0x0b, + 0x74, 0x7b, 0x8e, 0x6e, 0x61, 0x13, 0x59, 0xa1, 0x9b, 0x20, 0xad, 0x0a, 0x14, 0x8f, 0x23, 0x23, + 0xf2, 0x21, 0x5c, 0x48, 0xb8, 0xf8, 0x07, 0x4e, 0x97, 0xcc, 0x18, 0xa7, 0xcb, 0x4a, 0xcc, 0xfe, + 0xfd, 0x83, 0x66, 0x1d, 0x72, 0xa1, 0x18, 0x9f, 0xa5, 0x31, 0x3e, 0xbb, 0x1f, 0x08, 0xee, 0xdb, + 0x90, 0xf7, 0x95, 0x48, 0x0b, 0x0e, 0xb9, 0x11, 0x0b, 0x0e, 0xb3, 0x1e, 0x1e, 0x2d, 0x2f, 0x6c, + 0x41, 0x4e, 0xe8, 0x97, 0x92, 0x99, 0x1d, 0x91, 0x4c, 0x96, 0x63, 0x51, 0x22, 0x36, 0xcc, 0x7c, + 0xd8, 0x41, 0xfc, 0x80, 0x49, 0x6d, 0x64, 0xaf, 0x7d, 0xab, 0x3a, 0x52, 0x5d, 0xb8, 0x3a, 0xd4, + 0x67, 0xaa, 0x6f, 0x33, 0xba, 0x77, 0x2c, 0xd7, 0xe9, 0xa9, 0x62, 0x95, 0x95, 0x0f, 0x20, 0x17, + 0x9c, 0x90, 0x8b, 0x90, 0x3a, 0x42, 0x3d, 0x1e, 0xae, 0xc8, 0x9f, 0xf2, 0x0d, 0x98, 0xea, 0xea, + 0xcd, 0x4e, 0x9f, 0xa4, 0x88, 0x56, 0x56, 0x83, 0x2e, 0x46, 0xa8, 0xf5, 0x54, 0x86, 0x72, 0x63, + 0xe2, 0x0d, 0x89, 0x85, 0xf9, 0x40, 0xd0, 0xbc, 0x59, 0x77, 0xcd, 0xae, 0xe9, 0xf6, 0xbe, 0x0c, + 0x9a, 0x23, 0x04, 0xcd, 0xe0, 0x66, 0xf5, 0x0f, 0x9a, 0xbf, 0x3d, 0x29, 0x82, 0x66, 0xe2, 0xe6, + 0xf2, 0xa0, 0xf9, 0x08, 0x0a, 0x91, 0x70, 0xc5, 0xc3, 0xe6, 0xa5, 0x30, 0x2b, 0x01, 0xa7, 0x66, + 0x49, 0x4a, 0x8f, 0x06, 0x1d, 0x35, 0x1f, 0x0e, 0x69, 0x31, 0x83, 0x9f, 0x38, 0x8d, 0xc1, 0x07, + 0xe2, 0x58, 0x2a, 0x1c, 0xc7, 0x10, 0x54, 0x44, 0x9e, 0xc6, 0x87, 0xb4, 0x88, 0xa3, 0x4e, 0x8e, + 0xb8, 0xe0, 0x2a, 0xa7, 0x73, 0x93, 0x91, 0xd9, 0x0d, 0xb9, 0xed, 0x43, 0x28, 0x35, 0x90, 0xee, + 0xb8, 0xfb, 0x48, 0x77, 0x35, 0x03, 0xb9, 0xba, 0xd9, 0xc4, 0xbc, 0xe6, 0x38, 0xbc, 0xae, 0x56, + 0xf4, 0x50, 0x6f, 0x33, 0xcc, 0xf8, 0xc9, 0x34, 0x7d, 0xea, 0x93, 0xe9, 0x4a, 0xc0, 0xd4, 0x3d, + 0x17, 0xa0, 0x21, 0x3c, 0xe3, 0xdb, 0xef, 0x23, 0x31, 0xa1, 0xfc, 0x44, 0x82, 0xe7, 0x98, 0xae, + 0x43, 0x61, 0x80, 0x57, 0xfd, 0xc6, 0x72, 0x32, 0x1b, 0x8a, 0xbc, 0xd6, 0x88, 0x22, 0x45, 0xe8, + 0xdb, 0x43, 0xad, 0x76, 0x04, 0x16, 0xd4, 0x82, 0xa0, 0x2e, 0x0c, 0xf8, 0x0f, 0x25, 0x78, 0x7e, + 0x30, 0x22, 0xb7, 0x61, 0xec, 0x1f, 0xa2, 0xa2, 0xf4, 0xce, 0x8d, 0xf8, 0xde, 0xd3, 0x0a, 0x94, + 0xe4, 0xba, 0x12, 0x1a, 0x50, 0xfe, 0x42, 0x22, 0xb1, 0x2b, 0xc6, 0xdd, 0x5d, 0xdd, 0x6c, 0x8e, + 0xb5, 0xad, 0x0d, 0xc8, 0x1f, 0x50, 0x9c, 0xc8, 0xa6, 0xde, 0x3c, 0xcd, 0xa6, 0x86, 0x56, 0x57, + 0x67, 0x0f, 0x82, 0x9f, 0xca, 0x73, 0x24, 0x1e, 0xf4, 0x45, 0xe1, 0x62, 0xfd, 0x44, 0x02, 0x25, + 0x1e, 0x35, 0xee, 0x09, 0x8b, 0x1e, 0x43, 0xb0, 0x76, 0xd0, 0x87, 0xc2, 0xb2, 0x6d, 0x8d, 0x20, + 0xdb, 0x30, 0x16, 0x02, 0x6e, 0x26, 0x04, 0xdc, 0x21, 0xb6, 0x3e, 0x00, 0x8f, 0x9b, 0xcb, 0x8b, + 0x50, 0xac, 0xeb, 0x56, 0x1d, 0x79, 0xc1, 0x17, 0x31, 0xfe, 0xd3, 0x6a, 0x81, 0x8d, 0xab, 0x62, + 0x38, 0xe8, 0x3e, 0x41, 0x9a, 0x9f, 0x93, 0xfb, 0x0c, 0x62, 0x21, 0xee, 0x3e, 0x2f, 0x78, 0xde, + 0xd3, 0x07, 0x2f, 0x6e, 0xc8, 0x41, 0xc0, 0xff, 0x7b, 0x43, 0xee, 0xbb, 0x7a, 0x7f, 0x43, 0x4e, + 0x42, 0xe1, 0x62, 0xfd, 0x25, 0x35, 0xe4, 0xb8, 0xfc, 0x54, 0xc3, 0x63, 0x09, 0xf6, 0x6b, 0x90, + 0x0f, 0xdb, 0xcb, 0x18, 0x56, 0x3c, 0x6c, 0x7d, 0x75, 0x36, 0x64, 0x72, 0xca, 0xa5, 0x64, 0x7b, + 0xf3, 0x90, 0xb8, 0x70, 0x7f, 0x33, 0x01, 0x95, 0x5d, 0xf3, 0xd0, 0xd2, 0x9b, 0x67, 0x79, 0x53, + 0x3c, 0x80, 0x3c, 0xa6, 0x44, 0x22, 0x82, 0xbd, 0x35, 0xfc, 0x51, 0x71, 0xe0, 0xda, 0xea, 0x2c, + 0x23, 0x2b, 0x58, 0x31, 0x61, 0x15, 0x9d, 0xb8, 0xc8, 0x21, 0x2b, 0x25, 0xe4, 0x69, 0xa9, 0x71, + 0xf3, 0xb4, 0x65, 0x41, 0x2d, 0x36, 0x25, 0x57, 0x61, 0xae, 0xde, 0x30, 0x9b, 0x86, 0xbf, 0x8e, + 0x6d, 0x35, 0x7b, 0x34, 0x29, 0x48, 0xab, 0x25, 0x3a, 0x25, 0x90, 0xbe, 0x69, 0x35, 0x7b, 0xca, + 0x3a, 0x5c, 0xec, 0x2b, 0x0b, 0xdf, 0xeb, 0x7f, 0x90, 0xe0, 0x32, 0x87, 0x31, 0xdd, 0xc6, 0x99, + 0x1f, 0x72, 0xbf, 0x27, 0xc1, 0x32, 0xdf, 0xf5, 0x63, 0xd3, 0x6d, 0x68, 0x49, 0xaf, 0xba, 0xf7, + 0x46, 0x55, 0xc0, 0x30, 0x86, 0xd4, 0x45, 0x1c, 0x06, 0x14, 0x76, 0x76, 0x13, 0x36, 0x86, 0x93, + 0x18, 0xfc, 0x1e, 0xf7, 0xd7, 0x12, 0x5c, 0x54, 0x51, 0xcb, 0xee, 0x22, 0x46, 0xe9, 0x94, 0xc5, + 0xe7, 0x67, 0x97, 0xbb, 0x87, 0x33, 0xf0, 0x54, 0x24, 0x03, 0x57, 0x14, 0x12, 0xf6, 0xfa, 0xb1, + 0xcf, 0x75, 0xff, 0x57, 0x12, 0xac, 0xef, 0x21, 0xa7, 0x65, 0x5a, 0xba, 0x8b, 0xce, 0xa2, 0x75, + 0x1b, 0x4a, 0xae, 0xa0, 0x13, 0x51, 0xf6, 0xad, 0xa1, 0xca, 0x1e, 0xca, 0x81, 0x5a, 0xf4, 0x88, + 0x0b, 0x05, 0x3f, 0x0f, 0xca, 0x20, 0x34, 0x2e, 0xdf, 0x1f, 0x49, 0x50, 0xb9, 0x8d, 0xc8, 0x79, + 0x70, 0x16, 0xe1, 0x9e, 0x99, 0x0a, 0x89, 0x7b, 0xf6, 0x65, 0x8f, 0x8b, 0xf0, 0xa7, 0x12, 0x5c, + 0xa0, 0x95, 0xb9, 0x33, 0x76, 0x57, 0x38, 0x84, 0xc6, 0xd8, 0xdd, 0x15, 0x03, 0x57, 0x56, 0x73, + 0x94, 0xa8, 0x50, 0xc9, 0xeb, 0x50, 0xe9, 0x07, 0x3e, 0xd8, 0xd3, 0xfe, 0x20, 0x05, 0x97, 0x38, + 0x11, 0x76, 0x12, 0x9c, 0x45, 0xd4, 0x56, 0x9f, 0xd3, 0xec, 0xee, 0x08, 0xb2, 0x8e, 0xc0, 0x42, + 0xe4, 0x40, 0x93, 0xdf, 0x0c, 0xc4, 0x7e, 0xde, 0x58, 0x11, 0xaf, 0x8b, 0x95, 0x05, 0x48, 0x4d, + 0x40, 0x88, 0x8a, 0xd6, 0x90, 0xa3, 0x63, 0xf2, 0xd9, 0x1f, 0x1d, 0x53, 0xfd, 0x8e, 0x8e, 0x0d, + 0x78, 0x61, 0xd8, 0x8e, 0x70, 0x13, 0xfd, 0x7b, 0x09, 0x56, 0xc5, 0xfd, 0x32, 0x98, 0x7a, 0xff, + 0x42, 0x44, 0xc9, 0xeb, 0xb0, 0x68, 0x62, 0x2d, 0xa1, 0xe5, 0x83, 0xea, 0x26, 0xad, 0xce, 0x99, + 0xf8, 0x6e, 0xb4, 0x97, 0x43, 0xa9, 0xc0, 0xf9, 0x64, 0x81, 0xb8, 0xc4, 0xff, 0x3d, 0x41, 0x92, + 0x4f, 0x92, 0x8a, 0x6f, 0x91, 0x7d, 0xf3, 0x56, 0x3b, 0x4d, 0xe2, 0xfc, 0xec, 0x44, 0x5f, 0x87, + 0x9c, 0x6f, 0x92, 0xfe, 0xab, 0x9c, 0x37, 0x56, 0x33, 0xe4, 0xf7, 0x60, 0x4e, 0xe4, 0xd5, 0xc6, + 0x59, 0xec, 0x4e, 0xf6, 0xa8, 0xf8, 0xcb, 0xef, 0x78, 0x37, 0x02, 0x5a, 0x8d, 0xa5, 0xb5, 0x97, + 0xa9, 0x71, 0x6a, 0x2f, 0x05, 0x1f, 0x9d, 0x0e, 0x28, 0x97, 0x49, 0x9c, 0x18, 0xb8, 0xeb, 0x5c, + 0x3f, 0x7f, 0x2c, 0xc1, 0xda, 0x6d, 0x84, 0xeb, 0x8e, 0xb9, 0x7f, 0xa6, 0xc8, 0xff, 0x6d, 0x98, + 0x19, 0x37, 0xd9, 0x1f, 0xb6, 0xac, 0x2a, 0x28, 0x2a, 0x3f, 0x4e, 0xc1, 0xfa, 0x00, 0x68, 0x1e, + 0x33, 0xbf, 0x03, 0x45, 0xbf, 0x5a, 0x5c, 0xb7, 0xad, 0x03, 0xf3, 0x90, 0x5f, 0xfe, 0xaf, 0x26, + 0xf3, 0x92, 0xa8, 0xa0, 0x2d, 0x8a, 0xa8, 0x16, 0x50, 0x78, 0x40, 0x3e, 0x84, 0xa5, 0x84, 0xa2, + 0x34, 0x2d, 0x81, 0x33, 0x81, 0x37, 0xc7, 0x58, 0x84, 0x16, 0xbe, 0x17, 0x8e, 0x93, 0x86, 0xe5, + 0xef, 0x80, 0xdc, 0x46, 0x96, 0x61, 0x5a, 0x87, 0x9a, 0xce, 0x32, 0x7f, 0x13, 0xe1, 0x72, 0x8a, + 0x96, 0x7b, 0xaf, 0xf4, 0x5f, 0x63, 0x87, 0xe1, 0x88, 0xcb, 0x02, 0x5d, 0xa1, 0xd4, 0x0e, 0x0d, + 0x9a, 0x08, 0xcb, 0xdf, 0x85, 0xa2, 0xa0, 0x4e, 0x03, 0x99, 0x43, 0xdf, 0xd7, 0x09, 0xed, 0xeb, + 0x43, 0x69, 0x87, 0x6d, 0x89, 0xae, 0x50, 0x68, 0x07, 0xa6, 0x1c, 0x64, 0x29, 0xbf, 0x95, 0x82, + 0xb2, 0xca, 0xbb, 0x3d, 0x11, 0xb5, 0x45, 0xfc, 0xf8, 0xda, 0x2f, 0x84, 0x8f, 0x1f, 0xc0, 0x42, + 0xf8, 0x99, 0xb6, 0xa7, 0x99, 0x2e, 0x6a, 0x89, 0xad, 0xbd, 0x36, 0xd6, 0x53, 0x6d, 0xaf, 0xe6, + 0xa2, 0x96, 0x3a, 0xd7, 0x8d, 0x8d, 0x61, 0xf9, 0x0d, 0x98, 0xa6, 0x1e, 0x8c, 0x79, 0x6c, 0xe8, + 0x5b, 0x26, 0xbc, 0xad, 0xbb, 0xfa, 0xad, 0xa6, 0xbd, 0xaf, 0x72, 0x78, 0xf9, 0x2e, 0xe4, 0x2d, + 0x74, 0xac, 0x91, 0x83, 0x9f, 0x53, 0x98, 0x1a, 0x91, 0x42, 0xce, 0x42, 0xc7, 0x6a, 0x87, 0xf9, + 0x3e, 0x56, 0x56, 0x61, 0x39, 0x41, 0x05, 0x7e, 0xa2, 0xb7, 0xb8, 0xdb, 0xb3, 0xea, 0xbb, 0x0d, + 0xdd, 0x31, 0xf8, 0xe3, 0x2d, 0x57, 0xcf, 0x25, 0xc8, 0x63, 0xbb, 0xe3, 0xd4, 0x91, 0x56, 0x6f, + 0x76, 0xb0, 0x8b, 0x1c, 0xae, 0xa0, 0x59, 0x36, 0xba, 0xc5, 0x06, 0xe5, 0x65, 0x48, 0x63, 0x82, + 0x2c, 0x5e, 0xc0, 0xa6, 0xd4, 0x19, 0xfa, 0x5d, 0x33, 0xe4, 0x9b, 0x90, 0x65, 0xaf, 0xc8, 0xac, + 0x02, 0x9b, 0x1a, 0xb1, 0x02, 0x0b, 0x0c, 0x89, 0x0c, 0x2b, 0xcb, 0xb0, 0x14, 0x63, 0x4f, 0xdc, + 0xbf, 0xa6, 0x60, 0x8e, 0xcc, 0x09, 0x1b, 0x1f, 0xc3, 0xac, 0x2e, 0x42, 0xd6, 0x33, 0x2b, 0xce, + 0x76, 0x46, 0x05, 0x31, 0x54, 0x33, 0x02, 0x09, 0x57, 0x2a, 0x90, 0x70, 0xc9, 0x65, 0x98, 0xe1, + 0x3a, 0xe6, 0x45, 0x7d, 0xf1, 0x49, 0x16, 0xf5, 0xeb, 0xcd, 0xfe, 0x23, 0x9c, 0x37, 0x46, 0x9f, + 0x9c, 0xa3, 0x6f, 0x47, 0xd3, 0xa7, 0x7b, 0x3b, 0xba, 0x00, 0x20, 0xca, 0x9a, 0x26, 0x7b, 0xa5, + 0x4b, 0xa9, 0x19, 0x3e, 0x42, 0xdb, 0x38, 0xc2, 0x95, 0xf6, 0xf4, 0x69, 0x2a, 0xed, 0x3b, 0xbc, + 0x75, 0xc4, 0xaf, 0xd4, 0x51, 0x5a, 0x99, 0x11, 0x69, 0x95, 0x08, 0xb2, 0x57, 0x61, 0xa3, 0x14, + 0x6f, 0xc0, 0x8c, 0x28, 0x98, 0xc3, 0x88, 0x05, 0x73, 0x81, 0x10, 0xac, 0xfb, 0x67, 0xc3, 0x75, + 0xff, 0x2d, 0xc8, 0xb1, 0xc6, 0x02, 0xde, 0x37, 0x9b, 0x1b, 0xb1, 0x6f, 0x36, 0x4b, 0xfb, 0x0d, + 0x78, 0xcb, 0xec, 0xab, 0x40, 0x5b, 0x5e, 0x69, 0x9a, 0x83, 0x1c, 0xcd, 0x34, 0x90, 0xe5, 0x9a, + 0x6e, 0x8f, 0x3e, 0xca, 0x65, 0x54, 0x99, 0xcc, 0xbd, 0x43, 0xa7, 0x6a, 0x7c, 0x46, 0x7e, 0x07, + 0x0a, 0x91, 0xe8, 0xc1, 0x5b, 0x3c, 0xaa, 0xe3, 0xc5, 0x0d, 0x35, 0x1f, 0x8e, 0x19, 0xca, 0x22, + 0xcc, 0x87, 0x6d, 0x9a, 0x1b, 0xfb, 0xf7, 0x25, 0x58, 0x15, 0x67, 0xde, 0xe7, 0xdc, 0xcd, 0xa5, + 0xfc, 0x8f, 0x04, 0xe7, 0x93, 0x79, 0xe1, 0x47, 0x6f, 0x03, 0xe6, 0xea, 0x7a, 0xbd, 0x81, 0xc2, + 0x9d, 0xf6, 0xfc, 0xf4, 0x7d, 0x23, 0x71, 0x87, 0x02, 0xbd, 0xfa, 0xc1, 0xf5, 0x43, 0xe4, 0x4b, + 0x94, 0x68, 0x70, 0x48, 0xb6, 0x60, 0xd1, 0xd0, 0x5d, 0x7d, 0x5f, 0xc7, 0xd1, 0xc5, 0x26, 0xce, + 0xb8, 0xd8, 0xbc, 0xa0, 0x1b, 0x1c, 0x55, 0xfe, 0x49, 0x82, 0x15, 0x21, 0x3a, 0x57, 0xd9, 0x3d, + 0x1b, 0x07, 0xab, 0xdf, 0x0d, 0x1b, 0xbb, 0x9a, 0x6e, 0x18, 0x0e, 0xc2, 0x58, 0x68, 0x81, 0x8c, + 0xdd, 0x64, 0x43, 0x83, 0xc2, 0x65, 0x54, 0x87, 0xa9, 0x51, 0xcf, 0xc3, 0xc9, 0xa7, 0x70, 0xa3, + 0xfe, 0x78, 0xc2, 0x37, 0xb0, 0x90, 0x64, 0x5c, 0xa7, 0xcf, 0xc1, 0x2c, 0xe5, 0x13, 0x6b, 0x56, + 0xa7, 0xb5, 0xcf, 0x0f, 0x83, 0x29, 0x35, 0xc7, 0x06, 0x1f, 0xd1, 0x31, 0x79, 0x15, 0x32, 0x42, + 0x38, 0x5c, 0x9e, 0x58, 0x4b, 0x6d, 0x4c, 0xa9, 0x69, 0x2e, 0x1d, 0x96, 0xdf, 0x87, 0x82, 0x2f, + 0x1e, 0x55, 0xe5, 0xc0, 0x9f, 0x0f, 0x78, 0xb0, 0x44, 0x04, 0xef, 0xe1, 0x6a, 0x8b, 0xe0, 0xd1, + 0x5c, 0x23, 0x6f, 0x85, 0xc6, 0xe4, 0xd7, 0x60, 0x89, 0xad, 0x5d, 0xb7, 0x2d, 0xd7, 0xb1, 0x9b, + 0x4d, 0xe4, 0x88, 0x1e, 0xa6, 0x49, 0xba, 0x91, 0x0b, 0x74, 0x7a, 0xcb, 0x9b, 0xe5, 0xad, 0x49, + 0x24, 0xb6, 0x70, 0x75, 0xb1, 0xc7, 0x58, 0xf1, 0xa9, 0x54, 0xa1, 0xb4, 0xd5, 0xb4, 0x31, 0xa2, + 0x87, 0x8f, 0x50, 0x71, 0x50, 0x7f, 0x52, 0x48, 0x7f, 0xca, 0x3c, 0xc8, 0x41, 0x78, 0xee, 0xb9, + 0xaf, 0x40, 0x61, 0x1b, 0xb9, 0xa3, 0xd2, 0xf8, 0x00, 0x8a, 0x3e, 0x34, 0xdf, 0xfa, 0x07, 0x00, + 0x1c, 0x9c, 0xa4, 0x97, 0xcc, 0x8b, 0xae, 0x8c, 0x62, 0xd8, 0x94, 0x0c, 0xdd, 0x2c, 0xa6, 0x16, + 0xf2, 0xa7, 0xf2, 0x33, 0x09, 0x4a, 0xac, 0xbe, 0x15, 0xbc, 0x6a, 0xf6, 0x67, 0x49, 0xbe, 0x0b, + 0x69, 0x92, 0x3a, 0x1c, 0x92, 0x20, 0x37, 0x41, 0xbb, 0xc1, 0x5e, 0x1a, 0xdc, 0x6b, 0xc6, 0x2a, + 0xd3, 0x0c, 0x43, 0xf5, 0x70, 0x83, 0x2f, 0xe2, 0xa9, 0xd0, 0x8b, 0x78, 0x0d, 0x0a, 0x5d, 0x13, + 0x9b, 0xfb, 0x66, 0xd3, 0x74, 0x7b, 0xe3, 0x3d, 0xd6, 0xe6, 0x7d, 0x44, 0x9a, 0x2e, 0xcc, 0x83, + 0x1c, 0x94, 0x8d, 0xab, 0xe0, 0x63, 0x09, 0x2e, 0x6c, 0x23, 0x57, 0xf5, 0x7f, 0x76, 0xf4, 0x90, + 0xfd, 0xe4, 0xc8, 0xcb, 0x75, 0x1e, 0xc0, 0x34, 0xed, 0xf9, 0x20, 0x2e, 0x9b, 0xea, 0x6b, 0x92, + 0x81, 0xdf, 0x2d, 0xb1, 0xba, 0x87, 0xf7, 0x49, 0xbb, 0x43, 0x54, 0x4e, 0x83, 0x38, 0x32, 0x4f, + 0x99, 0xe8, 0x53, 0x2c, 0xcf, 0x2f, 0xb2, 0x7c, 0x8c, 0xd8, 0xb2, 0xf2, 0xa3, 0x09, 0xa8, 0xf4, + 0x63, 0x89, 0xab, 0xfd, 0x37, 0x20, 0xcf, 0x54, 0xc2, 0x7f, 0x1f, 0x25, 0x78, 0x7b, 0x77, 0xc4, + 0xb7, 0xcb, 0xc1, 0xe4, 0x99, 0x71, 0x88, 0x51, 0xd6, 0xe7, 0xc1, 0x3c, 0x5c, 0x8c, 0xad, 0xf4, + 0x40, 0x8e, 0x03, 0x05, 0x7b, 0x3e, 0xa6, 0x58, 0xcf, 0xc7, 0xc3, 0x70, 0xcf, 0xc7, 0xeb, 0x63, + 0xee, 0x9d, 0xc7, 0x99, 0xdf, 0x06, 0xa2, 0x7c, 0x04, 0x6b, 0xdb, 0xc8, 0xbd, 0xfd, 0xe0, 0xed, + 0x01, 0x3a, 0x7b, 0xcc, 0xdb, 0x55, 0x89, 0x57, 0x88, 0xbd, 0x19, 0x77, 0x6d, 0xaf, 0xed, 0x88, + 0x76, 0xb0, 0x92, 0xbf, 0xb0, 0xf2, 0x3b, 0x12, 0xac, 0x0f, 0x58, 0x9c, 0x6b, 0xe7, 0x03, 0x28, + 0x05, 0xc8, 0xd2, 0xc2, 0x88, 0x60, 0xe2, 0xfa, 0x29, 0x98, 0x50, 0x8b, 0x4e, 0x78, 0x00, 0x2b, + 0x3f, 0x90, 0x60, 0x9e, 0xf6, 0xc7, 0x88, 0xf8, 0x3d, 0xc6, 0x59, 0xff, 0xcd, 0xe8, 0xfd, 0xfb, + 0xab, 0x43, 0xef, 0xdf, 0x49, 0x4b, 0xf9, 0x77, 0xee, 0x23, 0x58, 0x88, 0x00, 0xf0, 0x7d, 0x50, + 0x21, 0x1d, 0x79, 0x5b, 0x7f, 0x6d, 0xdc, 0xa5, 0xf8, 0x4b, 0xba, 0x47, 0x47, 0xf9, 0x7d, 0x09, + 0xe6, 0x55, 0xa4, 0xb7, 0xdb, 0x4d, 0x56, 0xd0, 0xc0, 0x63, 0x48, 0xbe, 0x1b, 0x95, 0x3c, 0xb9, + 0x17, 0x2d, 0xf8, 0x13, 0x3d, 0xa6, 0x8e, 0xf8, 0x72, 0xbe, 0xf4, 0x4b, 0xb0, 0x10, 0x01, 0xe0, + 0x9c, 0xfe, 0xf9, 0x04, 0x2c, 0x30, 0x5b, 0x89, 0x5a, 0xe7, 0x1d, 0x98, 0xf4, 0x7a, 0x0d, 0xf3, + 0xc1, 0x92, 0x43, 0x52, 0xc4, 0xbc, 0x8d, 0x74, 0xe3, 0x01, 0x72, 0x5d, 0xe4, 0xd0, 0xb6, 0x1d, + 0xda, 0xde, 0x41, 0xd1, 0x07, 0xa5, 0x0b, 0xf1, 0xfb, 0x59, 0x2a, 0xe9, 0x7e, 0xf6, 0x3a, 0x94, + 0x4d, 0x8b, 0x40, 0x98, 0x5d, 0xa4, 0x21, 0xcb, 0x0b, 0x27, 0x7e, 0x67, 0xd2, 0x82, 0x37, 0x7f, + 0xc7, 0x12, 0xce, 0x5e, 0x33, 0xe4, 0x97, 0xa0, 0xd4, 0xd2, 0x4f, 0xcc, 0x56, 0xa7, 0xa5, 0xb5, + 0x09, 0x3c, 0x36, 0x3f, 0x62, 0xbf, 0xaf, 0x9b, 0x52, 0x0b, 0x7c, 0x62, 0x47, 0x3f, 0x44, 0xbb, + 0xe6, 0x47, 0x48, 0x7e, 0x01, 0x0a, 0xb4, 0x09, 0x91, 0x02, 0xb2, 0xee, 0xb9, 0x69, 0xda, 0x3d, + 0x47, 0x7b, 0x13, 0x09, 0x18, 0xeb, 0xd0, 0xff, 0x0f, 0xf6, 0x5b, 0xad, 0xd0, 0x7e, 0x71, 0x43, + 0x7a, 0x4a, 0x1b, 0x96, 0xe8, 0x97, 0x13, 0x4f, 0xd1, 0x2f, 0x93, 0x64, 0x4d, 0x25, 0xc9, 0xfa, + 0xcf, 0x12, 0x2c, 0xed, 0x74, 0x9c, 0x43, 0xf4, 0x45, 0xb4, 0x0e, 0x65, 0x05, 0xca, 0x71, 0xe1, + 0x44, 0xe7, 0xc0, 0x04, 0x2c, 0x3d, 0x44, 0x5f, 0x50, 0xc9, 0x9f, 0x89, 0x5f, 0xdc, 0x82, 0x72, + 0x7c, 0xc3, 0xb8, 0x63, 0x24, 0xd0, 0x90, 0x92, 0x68, 0xfc, 0x88, 0x76, 0xc5, 0x1f, 0x38, 0x08, + 0x37, 0x82, 0xb5, 0xf7, 0x71, 0x82, 0xe7, 0x7b, 0xd1, 0xe0, 0xf9, 0xab, 0x23, 0x06, 0xcf, 0xbe, + 0xab, 0xfa, 0x31, 0x94, 0x36, 0xca, 0x27, 0xc1, 0x71, 0xa3, 0xf9, 0xa1, 0x04, 0x2f, 0x6d, 0x23, + 0x0b, 0x39, 0xba, 0x8b, 0x1e, 0xe8, 0xd8, 0x15, 0x37, 0xe4, 0x88, 0xfb, 0x7d, 0x1e, 0x17, 0xde, + 0x2b, 0xf0, 0xf2, 0x48, 0x9c, 0x71, 0x49, 0xee, 0xc2, 0x6a, 0x38, 0xf7, 0x0a, 0xd7, 0xd5, 0x2e, + 0x43, 0xc1, 0x41, 0x2d, 0xdb, 0xf5, 0xec, 0x93, 0xe5, 0x0d, 0x19, 0x35, 0xcf, 0x86, 0xb9, 0x81, + 0x62, 0xa5, 0x03, 0xe7, 0x93, 0xe9, 0x70, 0xc3, 0xf8, 0x16, 0x4c, 0xb3, 0xdb, 0x17, 0xcf, 0x3b, + 0xde, 0x1c, 0x31, 0x31, 0xe4, 0xb7, 0x8b, 0x28, 0x59, 0x4e, 0x4c, 0xf9, 0xbb, 0x29, 0x58, 0x4c, + 0x06, 0x19, 0x74, 0x4b, 0xf8, 0x2a, 0x2c, 0xb5, 0xf4, 0x13, 0x2d, 0x1a, 0x7b, 0xfd, 0xbe, 0xf8, + 0xf9, 0x96, 0x7e, 0x12, 0xcd, 0xbc, 0x0c, 0xf9, 0x3e, 0x14, 0x19, 0xc5, 0xa6, 0x5d, 0xd7, 0x9b, + 0xe3, 0xd5, 0x09, 0x59, 0x7a, 0xfc, 0x80, 0x20, 0xd2, 0x0a, 0xd3, 0x47, 0xf1, 0x8d, 0x65, 0xb5, + 0xec, 0xb7, 0xcf, 0xb4, 0x31, 0x55, 0x35, 0xa4, 0x16, 0x96, 0x2a, 0x47, 0x74, 0x25, 0xff, 0xae, + 0x04, 0x73, 0x0d, 0xdd, 0x32, 0xec, 0x2e, 0x4f, 0xfa, 0xa9, 0x11, 0x92, 0x2b, 0xe5, 0x38, 0x7d, + 0xd9, 0x7d, 0x18, 0xb8, 0xc7, 0x09, 0x7b, 0xb7, 0x60, 0xce, 0x84, 0xdc, 0x88, 0x4d, 0xac, 0xfc, + 0x40, 0x82, 0xb9, 0x04, 0x86, 0x13, 0x5a, 0xb5, 0xdf, 0x0f, 0xa7, 0xed, 0xdb, 0x67, 0xe2, 0x71, + 0x07, 0x39, 0x7c, 0xbd, 0x40, 0x1a, 0xbf, 0xf2, 0x3d, 0x09, 0x96, 0xfa, 0x30, 0x9f, 0xc0, 0x90, + 0x1a, 0x66, 0xe8, 0xeb, 0x23, 0x32, 0x14, 0x5b, 0x80, 0x26, 0xf4, 0x81, 0xcb, 0xc4, 0xbb, 0xb0, + 0x90, 0x08, 0x23, 0xbf, 0x05, 0xe7, 0x3d, 0x9d, 0x25, 0x19, 0xae, 0x44, 0x0d, 0x77, 0x59, 0xc0, + 0xc4, 0xac, 0x57, 0xf9, 0x13, 0x09, 0xd6, 0x86, 0xed, 0x87, 0xac, 0xc0, 0xac, 0x5e, 0x3f, 0x42, + 0x46, 0x84, 0x6c, 0x96, 0x0e, 0x72, 0x37, 0x78, 0x1f, 0x56, 0x02, 0x30, 0xd1, 0xdb, 0xf0, 0xa8, + 0xbd, 0xd2, 0x4b, 0x1e, 0xc9, 0xc7, 0xa1, 0x6b, 0xf1, 0xad, 0xf6, 0x27, 0x9f, 0x56, 0xce, 0xfd, + 0xf4, 0xd3, 0xca, 0xb9, 0x9f, 0x7f, 0x5a, 0x91, 0x7e, 0xf3, 0x49, 0x45, 0xfa, 0xf1, 0x93, 0x8a, + 0xf4, 0xb7, 0x4f, 0x2a, 0xd2, 0x27, 0x4f, 0x2a, 0xd2, 0xbf, 0x3e, 0xa9, 0x48, 0xff, 0xfe, 0xa4, + 0x72, 0xee, 0xe7, 0x4f, 0x2a, 0xd2, 0xc7, 0x9f, 0x55, 0xce, 0x7d, 0xf2, 0x59, 0xe5, 0xdc, 0x4f, + 0x3f, 0xab, 0x9c, 0x7b, 0xef, 0xc6, 0xa1, 0xed, 0xab, 0xc0, 0xb4, 0x07, 0xfe, 0xa3, 0x9a, 0x5f, + 0x09, 0x8f, 0xec, 0x4f, 0x53, 0x26, 0xaf, 0xff, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x23, 0x9f, + 0x57, 0x69, 0xe7, 0x46, 0x00, 0x00, } func (this *StartWorkflowExecutionRequest) Equal(that interface{}) bool { @@ -6022,6 +6111,54 @@ func (this *TerminateWorkflowExecutionResponse) Equal(that interface{}) bool { } return true } +func (this *DeleteWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DeleteWorkflowExecutionRequest) + if !ok { + that2, ok := that.(DeleteWorkflowExecutionRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.NamespaceId != that1.NamespaceId { + return false + } + if !this.WorkflowExecution.Equal(that1.WorkflowExecution) { + return false + } + return true +} +func (this *DeleteWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DeleteWorkflowExecutionResponse) + if !ok { + that2, ok := that.(DeleteWorkflowExecutionResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + return true +} func (this *ResetWorkflowExecutionRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -7951,6 +8088,28 @@ func (this *TerminateWorkflowExecutionResponse) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *DeleteWorkflowExecutionRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&historyservice.DeleteWorkflowExecutionRequest{") + s = append(s, "NamespaceId: "+fmt.Sprintf("%#v", this.NamespaceId)+",\n") + if this.WorkflowExecution != nil { + s = append(s, "WorkflowExecution: "+fmt.Sprintf("%#v", this.WorkflowExecution)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *DeleteWorkflowExecutionResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 4) + s = append(s, "&historyservice.DeleteWorkflowExecutionResponse{") + s = append(s, "}") + return strings.Join(s, "") +} func (this *ResetWorkflowExecutionRequest) GoString() string { if this == nil { return "nil" @@ -10295,6 +10454,71 @@ func (m *TerminateWorkflowExecutionResponse) MarshalToSizedBuffer(dAtA []byte) ( return len(dAtA) - i, nil } +func (m *DeleteWorkflowExecutionRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DeleteWorkflowExecutionRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeleteWorkflowExecutionRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.WorkflowExecution != nil { + { + size, err := m.WorkflowExecution.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRequestResponse(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.NamespaceId) > 0 { + i -= len(m.NamespaceId) + copy(dAtA[i:], m.NamespaceId) + i = encodeVarintRequestResponse(dAtA, i, uint64(len(m.NamespaceId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DeleteWorkflowExecutionResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DeleteWorkflowExecutionResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeleteWorkflowExecutionResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + func (m *ResetWorkflowExecutionRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -10869,12 +11093,12 @@ func (m *SyncShardStatusRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) var l int _ = l if m.StatusTime != nil { - n62, err62 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StatusTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StatusTime):]) - if err62 != nil { - return 0, err62 + n63, err63 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StatusTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StatusTime):]) + if err63 != nil { + return 0, err63 } - i -= n62 - i = encodeVarintRequestResponse(dAtA, i, uint64(n62)) + i -= n63 + i = encodeVarintRequestResponse(dAtA, i, uint64(n63)) i-- dAtA[i] = 0x1a } @@ -10985,22 +11209,22 @@ func (m *SyncActivityRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x52 } if m.LastHeartbeatTime != nil { - n66, err66 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.LastHeartbeatTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.LastHeartbeatTime):]) - if err66 != nil { - return 0, err66 + n67, err67 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.LastHeartbeatTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.LastHeartbeatTime):]) + if err67 != nil { + return 0, err67 } - i -= n66 - i = encodeVarintRequestResponse(dAtA, i, uint64(n66)) + i -= n67 + i = encodeVarintRequestResponse(dAtA, i, uint64(n67)) i-- dAtA[i] = 0x4a } if m.StartedTime != nil { - n67, err67 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StartedTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StartedTime):]) - if err67 != nil { - return 0, err67 + n68, err68 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StartedTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StartedTime):]) + if err68 != nil { + return 0, err68 } - i -= n67 - i = encodeVarintRequestResponse(dAtA, i, uint64(n67)) + i -= n68 + i = encodeVarintRequestResponse(dAtA, i, uint64(n68)) i-- dAtA[i] = 0x42 } @@ -11010,12 +11234,12 @@ func (m *SyncActivityRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x38 } if m.ScheduledTime != nil { - n68, err68 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ScheduledTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ScheduledTime):]) - if err68 != nil { - return 0, err68 + n69, err69 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ScheduledTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ScheduledTime):]) + if err69 != nil { + return 0, err69 } - i -= n68 - i = encodeVarintRequestResponse(dAtA, i, uint64(n68)) + i -= n69 + i = encodeVarintRequestResponse(dAtA, i, uint64(n69)) i-- dAtA[i] = 0x32 } @@ -11266,21 +11490,21 @@ func (m *DescribeHistoryHostResponse) MarshalToSizedBuffer(dAtA []byte) (int, er dAtA[i] = 0x1a } if len(m.ShardIds) > 0 { - dAtA75 := make([]byte, len(m.ShardIds)*10) - var j74 int + dAtA76 := make([]byte, len(m.ShardIds)*10) + var j75 int for _, num1 := range m.ShardIds { num := uint64(num1) for num >= 1<<7 { - dAtA75[j74] = uint8(uint64(num)&0x7f | 0x80) + dAtA76[j75] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j74++ + j75++ } - dAtA75[j74] = uint8(num) - j74++ + dAtA76[j75] = uint8(num) + j75++ } - i -= j74 - copy(dAtA[i:], dAtA75[:j74]) - i = encodeVarintRequestResponse(dAtA, i, uint64(j74)) + i -= j75 + copy(dAtA[i:], dAtA76[:j75]) + i = encodeVarintRequestResponse(dAtA, i, uint64(j75)) i-- dAtA[i] = 0x12 } @@ -11427,12 +11651,12 @@ func (m *RemoveTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.VisibilityTime != nil { - n77, err77 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.VisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.VisibilityTime):]) - if err77 != nil { - return 0, err77 + n78, err78 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.VisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.VisibilityTime):]) + if err78 != nil { + return 0, err78 } - i -= n77 - i = encodeVarintRequestResponse(dAtA, i, uint64(n77)) + i -= n78 + i = encodeVarintRequestResponse(dAtA, i, uint64(n78)) i-- dAtA[i] = 0x22 } @@ -12317,12 +12541,12 @@ func (m *ShardReplicationStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) } } if m.ShardLocalTime != nil { - n86, err86 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ShardLocalTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ShardLocalTime):]) - if err86 != nil { - return 0, err86 + n87, err87 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ShardLocalTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ShardLocalTime):]) + if err87 != nil { + return 0, err87 } - i -= n86 - i = encodeVarintRequestResponse(dAtA, i, uint64(n86)) + i -= n87 + i = encodeVarintRequestResponse(dAtA, i, uint64(n87)) i-- dAtA[i] = 0x1a } @@ -12388,12 +12612,12 @@ func (m *ShardReplicationStatusPerCluster) MarshalToSizedBuffer(dAtA []byte) (in var l int _ = l if m.AckedTaskVisibilityTime != nil { - n87, err87 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.AckedTaskVisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.AckedTaskVisibilityTime):]) - if err87 != nil { - return 0, err87 + n88, err88 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.AckedTaskVisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.AckedTaskVisibilityTime):]) + if err88 != nil { + return 0, err88 } - i -= n87 - i = encodeVarintRequestResponse(dAtA, i, uint64(n87)) + i -= n88 + i = encodeVarintRequestResponse(dAtA, i, uint64(n88)) i-- dAtA[i] = 0x12 } @@ -13103,6 +13327,32 @@ func (m *TerminateWorkflowExecutionResponse) Size() (n int) { return n } +func (m *DeleteWorkflowExecutionRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NamespaceId) + if l > 0 { + n += 1 + l + sovRequestResponse(uint64(l)) + } + if m.WorkflowExecution != nil { + l = m.WorkflowExecution.Size() + n += 1 + l + sovRequestResponse(uint64(l)) + } + return n +} + +func (m *DeleteWorkflowExecutionResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func (m *ResetWorkflowExecutionRequest) Size() (n int) { if m == nil { return 0 @@ -14401,6 +14651,26 @@ func (this *TerminateWorkflowExecutionResponse) String() string { }, "") return s } +func (this *DeleteWorkflowExecutionRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&DeleteWorkflowExecutionRequest{`, + `NamespaceId:` + fmt.Sprintf("%v", this.NamespaceId) + `,`, + `WorkflowExecution:` + strings.Replace(fmt.Sprintf("%v", this.WorkflowExecution), "WorkflowExecution", "v14.WorkflowExecution", 1) + `,`, + `}`, + }, "") + return s +} +func (this *DeleteWorkflowExecutionResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&DeleteWorkflowExecutionResponse{`, + `}`, + }, "") + return s +} func (this *ResetWorkflowExecutionRequest) String() string { if this == nil { return "nil" @@ -19993,6 +20263,180 @@ func (m *TerminateWorkflowExecutionResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *DeleteWorkflowExecutionRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteWorkflowExecutionRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteWorkflowExecutionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NamespaceId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRequestResponse + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRequestResponse + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NamespaceId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowExecution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRequestResponse + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRequestResponse + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WorkflowExecution == nil { + m.WorkflowExecution = &v14.WorkflowExecution{} + } + if err := m.WorkflowExecution.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRequestResponse(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DeleteWorkflowExecutionResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteWorkflowExecutionResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteWorkflowExecutionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipRequestResponse(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ResetWorkflowExecutionRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/api/historyservice/v1/service.pb.go b/api/historyservice/v1/service.pb.go index 296a1b5cc52..bb71939ba16 100644 --- a/api/historyservice/v1/service.pb.go +++ b/api/historyservice/v1/service.pb.go @@ -54,76 +54,78 @@ func init() { } var fileDescriptor_655983da427ae822 = []byte{ - // 1104 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x99, 0xcd, 0x6b, 0x24, 0x45, - 0x18, 0x87, 0xa7, 0x2e, 0x22, 0x85, 0xae, 0xda, 0x8a, 0x1f, 0x51, 0x1b, 0x51, 0xbc, 0x4e, 0xd8, - 0x0d, 0xe8, 0x7e, 0x64, 0x5d, 0x77, 0x26, 0xc9, 0x24, 0xbb, 0x89, 0x9a, 0x99, 0x45, 0xc1, 0x8b, - 0x54, 0x7a, 0xde, 0xcd, 0x34, 0xe9, 0x74, 0xb7, 0x55, 0xd5, 0xa3, 0x73, 0x10, 0x16, 0x3c, 0x09, - 0x82, 0x22, 0x08, 0x9e, 0x04, 0x4f, 0x8a, 0x20, 0x08, 0x82, 0x20, 0x08, 0x9e, 0x04, 0x4f, 0x92, - 0xe3, 0x1e, 0xcd, 0xe4, 0xe2, 0x71, 0xff, 0x04, 0x99, 0xe9, 0xa9, 0xca, 0x54, 0x77, 0xf5, 0x58, - 0x55, 0x3d, 0xb7, 0xdd, 0x49, 0xff, 0x9e, 0x7e, 0xba, 0xaa, 0xba, 0xde, 0x77, 0x6a, 0xf0, 0x1a, - 0x87, 0xe3, 0x34, 0xa1, 0x24, 0x5a, 0x65, 0x40, 0x87, 0x40, 0x57, 0x49, 0x1a, 0xae, 0x0e, 0x42, - 0xc6, 0x13, 0x3a, 0x9a, 0x7c, 0x12, 0x06, 0xb0, 0x3a, 0xbc, 0xb8, 0x3a, 0xfb, 0x67, 0x33, 0xa5, - 0x09, 0x4f, 0xbc, 0x57, 0x45, 0xa8, 0x99, 0x87, 0x9a, 0x24, 0x0d, 0x9b, 0x6a, 0xa8, 0x39, 0xbc, - 0xb8, 0xb2, 0x6e, 0xc6, 0xa6, 0xf0, 0x61, 0x06, 0x8c, 0x7f, 0x40, 0x81, 0xa5, 0x49, 0xcc, 0x66, - 0x37, 0xb9, 0x74, 0x6f, 0x0d, 0x5f, 0xd8, 0xce, 0x2f, 0xee, 0xe5, 0x17, 0x7b, 0xdf, 0x23, 0xfc, - 0x74, 0x8f, 0x13, 0xca, 0xdf, 0x4b, 0xe8, 0xd1, 0xdd, 0x28, 0xf9, 0x68, 0xf3, 0x63, 0x08, 0x32, - 0x1e, 0x26, 0xb1, 0xb7, 0xd1, 0x34, 0x72, 0x6a, 0xea, 0xe3, 0xdd, 0x5c, 0x61, 0x65, 0xb3, 0x26, - 0x25, 0x7f, 0x80, 0x97, 0x1b, 0xde, 0x57, 0x08, 0x3f, 0xd6, 0x01, 0xbe, 0x97, 0x71, 0x72, 0x10, - 0x41, 0x8f, 0x13, 0x0e, 0xde, 0x75, 0x43, 0x78, 0x21, 0x27, 0xdc, 0xde, 0x70, 0x8d, 0x4b, 0xa9, - 0xaf, 0x11, 0x7e, 0xfc, 0x9d, 0x24, 0x8a, 0x14, 0x2b, 0x53, 0x6c, 0x31, 0x28, 0xb4, 0x6e, 0x38, - 0xe7, 0xa5, 0xd7, 0x77, 0x08, 0x3f, 0xd5, 0x05, 0x06, 0xbc, 0xc7, 0xc3, 0xe0, 0x68, 0x74, 0x87, - 0xb0, 0xa3, 0xfd, 0x0c, 0x32, 0xf0, 0x5a, 0x86, 0x6c, 0x5d, 0x58, 0xf8, 0xb5, 0x6b, 0x31, 0xa4, - 0xe3, 0xcf, 0x08, 0x3f, 0xd7, 0x85, 0x20, 0xa1, 0x7d, 0x31, 0xed, 0x93, 0xab, 0xa6, 0xeb, 0x00, - 0xfa, 0x5e, 0xc7, 0xf8, 0x26, 0x15, 0x04, 0x61, 0xbb, 0x5d, 0x1f, 0xa4, 0x51, 0xbe, 0x19, 0xf0, - 0x70, 0x18, 0xf2, 0x91, 0xbb, 0xb2, 0x86, 0xe0, 0xa6, 0xac, 0x05, 0x49, 0xe5, 0xdf, 0x10, 0x7e, - 0x21, 0xff, 0xaf, 0xf2, 0x6c, 0xed, 0xe4, 0x38, 0x8d, 0x60, 0x62, 0x7d, 0xcb, 0x7c, 0x36, 0x2b, - 0x21, 0x42, 0xfc, 0xf6, 0x52, 0x58, 0x85, 0xe1, 0x2e, 0x5d, 0xba, 0x45, 0xc2, 0xc8, 0x6a, 0xb8, - 0x2b, 0x08, 0xf6, 0xc3, 0x5d, 0x09, 0x92, 0xca, 0xbf, 0x22, 0xfc, 0x7c, 0x79, 0x5a, 0xb6, 0x81, - 0x50, 0x7e, 0x00, 0x84, 0x7b, 0x3b, 0xce, 0x53, 0x2b, 0x19, 0x42, 0xfb, 0xd6, 0x32, 0x50, 0xba, - 0x75, 0x32, 0x7f, 0xa9, 0xf3, 0x3a, 0xd1, 0x42, 0x1c, 0xd7, 0x49, 0x05, 0x4b, 0xb7, 0x4e, 0xe6, - 0x2f, 0x75, 0x5b, 0x27, 0x65, 0x82, 0xe3, 0x3a, 0xd1, 0x81, 0x0a, 0xeb, 0xa4, 0xfc, 0x74, 0x24, - 0x0e, 0x60, 0x22, 0xbd, 0x53, 0x63, 0x84, 0x66, 0x0c, 0xfb, 0x75, 0xb2, 0x00, 0x25, 0xc5, 0x7f, - 0x44, 0xf8, 0x99, 0x5e, 0x78, 0x18, 0x93, 0xa8, 0xdc, 0x31, 0x18, 0xd7, 0x7a, 0x7d, 0x5e, 0x08, - 0x6f, 0xd5, 0xc5, 0x48, 0xd9, 0x3f, 0x11, 0x7e, 0x69, 0x76, 0x55, 0xc8, 0x07, 0x15, 0x7d, 0xce, - 0x5b, 0x76, 0xb7, 0xab, 0x04, 0x09, 0xfd, 0xb7, 0x97, 0xc6, 0x93, 0xcf, 0xf1, 0x13, 0xc2, 0xcf, - 0x76, 0xe1, 0x38, 0x19, 0x42, 0x1e, 0x52, 0xda, 0x8d, 0x2d, 0xe3, 0xf9, 0xd5, 0x03, 0x84, 0x77, - 0xa7, 0x36, 0x47, 0xfa, 0xfe, 0x82, 0xf0, 0xca, 0x1d, 0xa0, 0xc7, 0x61, 0x4c, 0x38, 0x94, 0x47, - 0xdc, 0xf4, 0x45, 0xaa, 0x46, 0x08, 0xe7, 0x9d, 0x25, 0x90, 0xa4, 0xf5, 0xa4, 0x17, 0x9e, 0xf6, - 0x2c, 0xee, 0xbd, 0xb0, 0x3e, 0x6e, 0xdb, 0x0b, 0x57, 0x51, 0xa4, 0xe9, 0x1f, 0x08, 0xfb, 0x33, - 0x68, 0xfe, 0x8a, 0x96, 0x8d, 0x77, 0x8d, 0xef, 0xb5, 0x08, 0x23, 0xcc, 0xf7, 0x96, 0x44, 0x53, - 0x1a, 0xd4, 0x5e, 0x30, 0x80, 0x7e, 0x16, 0xc1, 0x7c, 0x41, 0x35, 0x6e, 0x50, 0x75, 0x61, 0xdb, - 0x06, 0x55, 0xcf, 0x90, 0x8e, 0xbf, 0x23, 0xfc, 0x62, 0x5e, 0x3c, 0xdb, 0x83, 0x30, 0xea, 0xcb, - 0xc7, 0x38, 0xaf, 0x89, 0xb7, 0xad, 0x4a, 0x70, 0x05, 0x45, 0x58, 0xef, 0x2e, 0x07, 0xa6, 0x54, - 0xc5, 0x0d, 0x60, 0x01, 0x0d, 0x0f, 0x34, 0xef, 0xa0, 0xe9, 0xdb, 0x5e, 0x49, 0xb0, 0xad, 0x8a, - 0x0b, 0x40, 0x52, 0xf9, 0x1b, 0x84, 0x9f, 0xe8, 0x42, 0x1a, 0x85, 0x01, 0xe1, 0xb0, 0x39, 0x84, - 0x98, 0xb3, 0x77, 0x2f, 0x79, 0x37, 0x8c, 0x07, 0xa6, 0x90, 0x14, 0x8a, 0x6f, 0xba, 0x03, 0x94, - 0xaf, 0x9f, 0xbd, 0x51, 0x1c, 0xf4, 0x06, 0x84, 0xf6, 0x27, 0xfb, 0x5d, 0xc6, 0x8c, 0xbf, 0x7e, - 0x16, 0x72, 0xb6, 0x5f, 0x3f, 0x4b, 0x71, 0x29, 0xf5, 0x19, 0xc2, 0x8f, 0x4c, 0xfe, 0x2a, 0x6a, - 0xb6, 0x77, 0xd5, 0x02, 0x29, 0x42, 0x42, 0xe7, 0x9a, 0x53, 0x56, 0x79, 0xa3, 0xc5, 0x1c, 0x2b, - 0xf5, 0xa9, 0x65, 0xb9, 0x40, 0x74, 0xb5, 0xa9, 0x5d, 0x8b, 0x21, 0x1d, 0xbf, 0x45, 0xf8, 0x49, - 0x71, 0xc9, 0xec, 0x20, 0x64, 0x3b, 0x61, 0xdc, 0xbb, 0x69, 0x89, 0x9f, 0xcb, 0x0a, 0xc3, 0x56, - 0x1d, 0x84, 0x14, 0xfc, 0x14, 0x61, 0xdc, 0x8e, 0x12, 0x06, 0xd3, 0xf9, 0xf6, 0x2e, 0x1b, 0x42, - 0xcf, 0x23, 0x42, 0xe7, 0x8a, 0x43, 0x52, 0x5a, 0x7c, 0x82, 0x1f, 0xee, 0x00, 0xcf, 0x15, 0x5e, - 0x33, 0x3f, 0x23, 0x51, 0x04, 0x5e, 0xb7, 0xce, 0x29, 0x83, 0x90, 0x37, 0x19, 0xd3, 0x8a, 0x70, - 0xd9, 0xaa, 0x2f, 0x99, 0xaf, 0x03, 0x57, 0x1c, 0x92, 0x4a, 0x37, 0xd0, 0x01, 0x2e, 0xf6, 0x84, - 0x30, 0x89, 0xf7, 0x80, 0x31, 0x72, 0x08, 0xcc, 0xb8, 0x1b, 0xd0, 0xc7, 0x6d, 0xbb, 0x81, 0x2a, - 0x8a, 0xb2, 0xd1, 0x77, 0x80, 0x6f, 0xec, 0xee, 0xeb, 0x64, 0x3b, 0xe6, 0xb7, 0xd1, 0x13, 0x6c, - 0x37, 0xfa, 0x05, 0x20, 0xa9, 0xfc, 0x39, 0xc2, 0x8f, 0xee, 0x67, 0x40, 0x47, 0xa2, 0x1a, 0x78, - 0xa6, 0xbb, 0x8f, 0x92, 0x12, 0x6a, 0xeb, 0x6e, 0x61, 0x45, 0xa7, 0x0b, 0x24, 0x4d, 0xa3, 0x51, - 0xbe, 0xf5, 0x1b, 0xeb, 0x28, 0x29, 0x5b, 0x9d, 0x42, 0x58, 0xea, 0x7c, 0x81, 0xf0, 0x85, 0x7c, - 0x14, 0xe5, 0x2c, 0xae, 0x5b, 0x0d, 0x7e, 0x71, 0xea, 0xae, 0x3b, 0xa6, 0xd5, 0x73, 0xce, 0x8c, - 0x1e, 0xc2, 0xbc, 0x93, 0xf1, 0x39, 0x67, 0x21, 0x68, 0x7d, 0xce, 0x59, 0xca, 0x2b, 0x5e, 0x7b, - 0xe0, 0xe8, 0x55, 0x0c, 0xda, 0x7a, 0x95, 0xf3, 0x85, 0xf3, 0xd7, 0xbb, 0x14, 0xd8, 0x60, 0xbe, - 0xb9, 0x64, 0x16, 0xe7, 0xaf, 0xe5, 0xb0, 0xfd, 0xf9, 0xab, 0x8e, 0x21, 0x1d, 0xff, 0x46, 0xf8, - 0x95, 0x0e, 0xc4, 0x40, 0x09, 0x87, 0x5d, 0xc2, 0xf8, 0xac, 0x22, 0xcd, 0xbd, 0xb8, 0xb9, 0xf2, - 0xbe, 0xf1, 0xe2, 0xf9, 0x5f, 0x96, 0x78, 0x82, 0xee, 0x32, 0x91, 0xca, 0xa0, 0xab, 0x9b, 0xe5, - 0xac, 0x4f, 0x6b, 0x39, 0xed, 0xb4, 0x6a, 0xb3, 0xd6, 0xae, 0xc5, 0x10, 0x8e, 0xad, 0xf4, 0xe4, - 0xd4, 0x6f, 0xdc, 0x3f, 0xf5, 0x1b, 0x0f, 0x4e, 0x7d, 0x74, 0x6f, 0xec, 0xa3, 0x1f, 0xc6, 0x3e, - 0xfa, 0x6b, 0xec, 0xa3, 0x93, 0xb1, 0x8f, 0xfe, 0x19, 0xfb, 0xe8, 0xdf, 0xb1, 0xdf, 0x78, 0x30, - 0xf6, 0xd1, 0x97, 0x67, 0x7e, 0xe3, 0xe4, 0xcc, 0x6f, 0xdc, 0x3f, 0xf3, 0x1b, 0xef, 0x5f, 0x3d, - 0x4c, 0xce, 0x6f, 0x1f, 0x26, 0x0b, 0x7f, 0xfc, 0xb9, 0xa6, 0x7e, 0x72, 0xf0, 0xd0, 0xf4, 0xb7, - 0x9f, 0xb5, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0xbb, 0x79, 0x47, 0xd1, 0x97, 0x1a, 0x00, 0x00, + // 1121 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x99, 0xcd, 0x8b, 0x23, 0x45, + 0x18, 0x87, 0x53, 0x17, 0x91, 0x42, 0x57, 0x6d, 0xc5, 0x8f, 0x51, 0x1b, 0x51, 0xbc, 0x66, 0xd8, + 0x5d, 0xd1, 0xfd, 0x98, 0x75, 0xdd, 0x64, 0x66, 0x32, 0xb3, 0x3b, 0xa3, 0x4e, 0xb2, 0x28, 0x78, + 0x91, 0x9a, 0xce, 0xbb, 0x93, 0x66, 0x7a, 0xba, 0xdb, 0xaa, 0xea, 0x68, 0x0e, 0x82, 0xe0, 0x49, + 0x10, 0x14, 0x41, 0xf0, 0x24, 0x78, 0x52, 0x84, 0x05, 0x41, 0x10, 0x04, 0xc1, 0x93, 0xe0, 0x49, + 0xe6, 0xb8, 0x47, 0x27, 0x73, 0xf1, 0xb8, 0x7f, 0x82, 0x24, 0x9d, 0xaa, 0x49, 0x75, 0x57, 0x67, + 0xab, 0xaa, 0x73, 0xdb, 0xcd, 0xf4, 0xef, 0xe9, 0xa7, 0xab, 0xde, 0xae, 0x7a, 0x53, 0xc1, 0x17, + 0x39, 0x1c, 0xa5, 0x09, 0x25, 0xd1, 0x2a, 0x03, 0x3a, 0x04, 0xba, 0x4a, 0xd2, 0x70, 0x75, 0x10, + 0x32, 0x9e, 0xd0, 0xd1, 0xe4, 0x93, 0x30, 0x80, 0xd5, 0xe1, 0xf9, 0xd5, 0xd9, 0x3f, 0x9b, 0x29, + 0x4d, 0x78, 0xe2, 0xbd, 0x2a, 0x42, 0xcd, 0x3c, 0xd4, 0x24, 0x69, 0xd8, 0x54, 0x43, 0xcd, 0xe1, + 0xf9, 0x95, 0x35, 0x33, 0x36, 0x85, 0x8f, 0x32, 0x60, 0xfc, 0x43, 0x0a, 0x2c, 0x4d, 0x62, 0x36, + 0xbb, 0xc9, 0x85, 0xbb, 0xaf, 0xe1, 0x73, 0x5b, 0xf9, 0xc5, 0xbd, 0xfc, 0x62, 0xef, 0x47, 0x84, + 0x9f, 0xee, 0x71, 0x42, 0xf9, 0xfb, 0x09, 0x3d, 0xbc, 0x13, 0x25, 0x1f, 0x6f, 0x7c, 0x02, 0x41, + 0xc6, 0xc3, 0x24, 0xf6, 0xd6, 0x9b, 0x46, 0x4e, 0x4d, 0x7d, 0xbc, 0x9b, 0x2b, 0xac, 0x6c, 0xd4, + 0xa4, 0xe4, 0x0f, 0xf0, 0x72, 0xc3, 0xfb, 0x06, 0xe1, 0xc7, 0x3a, 0xc0, 0x77, 0x33, 0x4e, 0xf6, + 0x23, 0xe8, 0x71, 0xc2, 0xc1, 0xbb, 0x66, 0x08, 0x2f, 0xe4, 0x84, 0xdb, 0x9b, 0xae, 0x71, 0x29, + 0xf5, 0x2d, 0xc2, 0x8f, 0xbf, 0x9b, 0x44, 0x91, 0x62, 0x65, 0x8a, 0x2d, 0x06, 0x85, 0xd6, 0x75, + 0xe7, 0xbc, 0xf4, 0xfa, 0x01, 0xe1, 0xa7, 0xba, 0xc0, 0x80, 0xf7, 0x78, 0x18, 0x1c, 0x8e, 0x6e, + 0x13, 0x76, 0xb8, 0x97, 0x41, 0x06, 0x5e, 0xcb, 0x90, 0xad, 0x0b, 0x0b, 0xbf, 0x76, 0x2d, 0x86, + 0x74, 0xfc, 0x05, 0xe1, 0xe7, 0xba, 0x10, 0x24, 0xb4, 0x2f, 0xa6, 0x7d, 0x72, 0xd5, 0xb4, 0x0e, + 0xa0, 0xef, 0x75, 0x8c, 0x6f, 0x52, 0x41, 0x10, 0xb6, 0x5b, 0xf5, 0x41, 0x1a, 0xe5, 0x1b, 0x01, + 0x0f, 0x87, 0x21, 0x1f, 0xb9, 0x2b, 0x6b, 0x08, 0x6e, 0xca, 0x5a, 0x90, 0x54, 0xfe, 0x1d, 0xe1, + 0x17, 0xf2, 0xff, 0x2a, 0xcf, 0xd6, 0x4e, 0x8e, 0xd2, 0x08, 0x26, 0xd6, 0x37, 0xcd, 0x67, 0xb3, + 0x12, 0x22, 0xc4, 0x6f, 0x2d, 0x85, 0x55, 0x18, 0xee, 0xd2, 0xa5, 0x9b, 0x24, 0x8c, 0xac, 0x86, + 0xbb, 0x82, 0x60, 0x3f, 0xdc, 0x95, 0x20, 0xa9, 0xfc, 0x1b, 0xc2, 0xcf, 0x97, 0xa7, 0x65, 0x0b, + 0x08, 0xe5, 0xfb, 0x40, 0xb8, 0xb7, 0xed, 0x3c, 0xb5, 0x92, 0x21, 0xb4, 0x6f, 0x2e, 0x03, 0xa5, + 0xab, 0x93, 0xf9, 0x4b, 0x9d, 0xeb, 0x44, 0x0b, 0x71, 0xac, 0x93, 0x0a, 0x96, 0xae, 0x4e, 0xe6, + 0x2f, 0x75, 0xab, 0x93, 0x32, 0xc1, 0xb1, 0x4e, 0x74, 0xa0, 0x42, 0x9d, 0x94, 0x9f, 0x8e, 0xc4, + 0x01, 0x4c, 0xa4, 0xb7, 0x6b, 0x8c, 0xd0, 0x8c, 0x61, 0x5f, 0x27, 0x0b, 0x50, 0x52, 0xfc, 0x67, + 0x84, 0x9f, 0xe9, 0x85, 0x07, 0x31, 0x89, 0xca, 0x1d, 0x83, 0xf1, 0x5e, 0xaf, 0xcf, 0x0b, 0xe1, + 0xcd, 0xba, 0x18, 0x29, 0xfb, 0x17, 0xc2, 0x2f, 0xcd, 0xae, 0x0a, 0xf9, 0xa0, 0xa2, 0xcf, 0x79, + 0xdb, 0xee, 0x76, 0x95, 0x20, 0xa1, 0xff, 0xce, 0xd2, 0x78, 0xf2, 0x39, 0xee, 0x22, 0xfc, 0x6c, + 0x17, 0x8e, 0x92, 0x21, 0xe4, 0x21, 0xa5, 0xdd, 0xd8, 0x34, 0x9e, 0x5f, 0x3d, 0x40, 0x78, 0x77, + 0x6a, 0x73, 0xa4, 0xef, 0xaf, 0x08, 0xaf, 0xdc, 0x06, 0x7a, 0x14, 0xc6, 0x84, 0x43, 0x79, 0xc4, + 0x4d, 0x5f, 0xa4, 0x6a, 0x84, 0x70, 0xde, 0x5e, 0x02, 0x49, 0x29, 0xed, 0x75, 0x98, 0x2c, 0x2e, + 0xee, 0xa5, 0x5d, 0x91, 0xb7, 0x2d, 0xed, 0x4a, 0x8c, 0x94, 0x9d, 0x34, 0xee, 0xd3, 0x06, 0xcb, + 0xbd, 0x71, 0xd7, 0xc7, 0x6d, 0x1b, 0xf7, 0x2a, 0x8a, 0x34, 0xfd, 0x13, 0x61, 0x7f, 0x06, 0xcd, + 0xd7, 0x93, 0xb2, 0xf1, 0x8e, 0xf1, 0xbd, 0x16, 0x61, 0x84, 0xf9, 0xee, 0x92, 0x68, 0x4a, 0x37, + 0xdd, 0x0b, 0x06, 0xd0, 0xcf, 0x22, 0x98, 0xdf, 0xfd, 0x8d, 0xbb, 0x69, 0x5d, 0xd8, 0xb6, 0x9b, + 0xd6, 0x33, 0xa4, 0xe3, 0x1f, 0x08, 0xbf, 0x98, 0xef, 0xf4, 0xed, 0x41, 0x18, 0xf5, 0xe5, 0x63, + 0x9c, 0x6d, 0xe0, 0xb7, 0xac, 0xfa, 0x85, 0x0a, 0x8a, 0xb0, 0xde, 0x59, 0x0e, 0x4c, 0xd9, 0xc2, + 0xd7, 0x81, 0x05, 0x34, 0xdc, 0xd7, 0xbc, 0x7d, 0x1d, 0xe3, 0xd7, 0xa6, 0x82, 0x60, 0xbb, 0x85, + 0x2f, 0x00, 0x49, 0xe5, 0xef, 0x10, 0x7e, 0xa2, 0x0b, 0x69, 0x14, 0x06, 0x84, 0xc3, 0xc6, 0x10, + 0x62, 0xce, 0xde, 0xbb, 0xe0, 0x5d, 0x37, 0x1e, 0x98, 0x42, 0x52, 0x28, 0xbe, 0xe5, 0x0e, 0x50, + 0xbe, 0x2b, 0xf7, 0x46, 0x71, 0xd0, 0x1b, 0x10, 0xda, 0x9f, 0x2c, 0xce, 0x19, 0x33, 0xfe, 0xae, + 0x5c, 0xc8, 0xd9, 0x7e, 0x57, 0x2e, 0xc5, 0xa5, 0xd4, 0x17, 0x08, 0x3f, 0x32, 0xf9, 0xab, 0x68, + 0x30, 0xbc, 0x2b, 0x16, 0x48, 0x11, 0x12, 0x3a, 0x57, 0x9d, 0xb2, 0xca, 0x1b, 0x2d, 0xe6, 0x58, + 0xd9, 0x4c, 0x5b, 0x96, 0x05, 0xa2, 0xdb, 0x48, 0xdb, 0xb5, 0x18, 0xd2, 0xf1, 0x7b, 0x84, 0x9f, + 0x14, 0x97, 0xcc, 0x4e, 0x6d, 0xb6, 0x12, 0xc6, 0xbd, 0x1b, 0x96, 0xf8, 0xb9, 0xac, 0x30, 0x6c, + 0xd5, 0x41, 0x48, 0xc1, 0xcf, 0x11, 0xc6, 0xed, 0x28, 0x61, 0x30, 0x9d, 0x6f, 0xef, 0x92, 0x21, + 0xf4, 0x2c, 0x22, 0x74, 0x2e, 0x3b, 0x24, 0xa5, 0xc5, 0xa7, 0xf8, 0xe1, 0x0e, 0xf0, 0x5c, 0xe1, + 0x75, 0xf3, 0x03, 0x1d, 0x45, 0xe0, 0x0d, 0xeb, 0x9c, 0x32, 0x08, 0x79, 0x47, 0x34, 0xdd, 0x11, + 0x2e, 0x59, 0x35, 0x51, 0xf3, 0xfb, 0xc0, 0x65, 0x87, 0xa4, 0xd2, 0x0d, 0x74, 0x80, 0x8b, 0x35, + 0x21, 0x4c, 0xe2, 0x5d, 0x60, 0x8c, 0x1c, 0x00, 0x33, 0xee, 0x06, 0xf4, 0x71, 0xdb, 0x6e, 0xa0, + 0x8a, 0xa2, 0x2c, 0xf4, 0x1d, 0xe0, 0xeb, 0x3b, 0x7b, 0x3a, 0xd9, 0x8e, 0xf9, 0x6d, 0xf4, 0x04, + 0xdb, 0x85, 0x7e, 0x01, 0x48, 0x2a, 0x7f, 0x89, 0xf0, 0xa3, 0x7b, 0x19, 0xd0, 0x91, 0xd8, 0x0d, + 0x3c, 0xd3, 0xd5, 0x47, 0x49, 0x09, 0xb5, 0x35, 0xb7, 0xb0, 0xa2, 0xd3, 0x05, 0x92, 0xa6, 0xd1, + 0x28, 0x5f, 0xfa, 0x8d, 0x75, 0x94, 0x94, 0xad, 0x4e, 0x21, 0x2c, 0x75, 0xbe, 0x42, 0xf8, 0x5c, + 0x3e, 0x8a, 0x72, 0x16, 0xd7, 0xac, 0x06, 0xbf, 0x38, 0x75, 0xd7, 0x1c, 0xd3, 0xea, 0xa1, 0x6c, + 0x46, 0x0f, 0x60, 0xde, 0xc9, 0xf8, 0x50, 0xb6, 0x10, 0xb4, 0x3e, 0x94, 0x2d, 0xe5, 0x15, 0xaf, + 0x5d, 0x70, 0xf4, 0x2a, 0x06, 0x6d, 0xbd, 0xca, 0xf9, 0xc2, 0x61, 0xf1, 0x1d, 0x0a, 0x6c, 0x30, + 0xdf, 0x5c, 0x32, 0x8b, 0xc3, 0xe2, 0x72, 0xd8, 0xfe, 0xb0, 0x58, 0xc7, 0x90, 0x8e, 0xff, 0x20, + 0xfc, 0x4a, 0x07, 0x62, 0xa0, 0x84, 0xc3, 0x0e, 0x61, 0x7c, 0xb6, 0x23, 0xcd, 0xbd, 0xb8, 0xb9, + 0xf2, 0x9e, 0x71, 0xf1, 0x3c, 0x90, 0x25, 0x9e, 0xa0, 0xbb, 0x4c, 0xa4, 0x32, 0xe8, 0xea, 0x62, + 0x39, 0xeb, 0xd3, 0x5a, 0x4e, 0x2b, 0xad, 0xda, 0xac, 0xb5, 0x6b, 0x31, 0x84, 0x63, 0x2b, 0x3d, + 0x3e, 0xf1, 0x1b, 0xf7, 0x4e, 0xfc, 0xc6, 0xfd, 0x13, 0x1f, 0x7d, 0x36, 0xf6, 0xd1, 0x4f, 0x63, + 0x1f, 0xfd, 0x3d, 0xf6, 0xd1, 0xf1, 0xd8, 0x47, 0xff, 0x8e, 0x7d, 0xf4, 0xdf, 0xd8, 0x6f, 0xdc, + 0x1f, 0xfb, 0xe8, 0xeb, 0x53, 0xbf, 0x71, 0x7c, 0xea, 0x37, 0xee, 0x9d, 0xfa, 0x8d, 0x0f, 0xae, + 0x1c, 0x24, 0x67, 0xb7, 0x0f, 0x93, 0x85, 0xbf, 0x54, 0x5d, 0x55, 0x3f, 0xd9, 0x7f, 0x68, 0xfa, + 0x43, 0xd5, 0xc5, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0x32, 0x6f, 0x20, 0x89, 0x44, 0x1b, 0x00, + 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -216,6 +218,10 @@ type HistoryServiceClient interface { // TerminateWorkflowExecution terminates an existing workflow execution by recording WorkflowExecutionTerminated event // in the history and immediately terminating the execution instance. TerminateWorkflowExecution(ctx context.Context, in *TerminateWorkflowExecutionRequest, opts ...grpc.CallOption) (*TerminateWorkflowExecutionResponse, error) + // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history. + // It also asynchronously deletes visibility record. + // If call to DeleteWorkflowExecution fails, it must be retried to prevent unreferenced data. + DeleteWorkflowExecution(ctx context.Context, in *DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*DeleteWorkflowExecutionResponse, error) // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. // After reset, the history will grow from nextFirstEventId. @@ -425,6 +431,15 @@ func (c *historyServiceClient) TerminateWorkflowExecution(ctx context.Context, i return out, nil } +func (c *historyServiceClient) DeleteWorkflowExecution(ctx context.Context, in *DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*DeleteWorkflowExecutionResponse, error) { + out := new(DeleteWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, "/temporal.server.api.historyservice.v1.HistoryService/DeleteWorkflowExecution", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *historyServiceClient) ResetWorkflowExecution(ctx context.Context, in *ResetWorkflowExecutionRequest, opts ...grpc.CallOption) (*ResetWorkflowExecutionResponse, error) { out := new(ResetWorkflowExecutionResponse) err := c.cc.Invoke(ctx, "/temporal.server.api.historyservice.v1.HistoryService/ResetWorkflowExecution", in, out, opts...) @@ -712,6 +727,10 @@ type HistoryServiceServer interface { // TerminateWorkflowExecution terminates an existing workflow execution by recording WorkflowExecutionTerminated event // in the history and immediately terminating the execution instance. TerminateWorkflowExecution(context.Context, *TerminateWorkflowExecutionRequest) (*TerminateWorkflowExecutionResponse, error) + // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history. + // It also asynchronously deletes visibility record. + // If call to DeleteWorkflowExecution fails, it must be retried to prevent unreferenced data. + DeleteWorkflowExecution(context.Context, *DeleteWorkflowExecutionRequest) (*DeleteWorkflowExecutionResponse, error) // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. // After reset, the history will grow from nextFirstEventId. @@ -821,6 +840,9 @@ func (*UnimplementedHistoryServiceServer) RemoveSignalMutableState(ctx context.C func (*UnimplementedHistoryServiceServer) TerminateWorkflowExecution(ctx context.Context, req *TerminateWorkflowExecutionRequest) (*TerminateWorkflowExecutionResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method TerminateWorkflowExecution not implemented") } +func (*UnimplementedHistoryServiceServer) DeleteWorkflowExecution(ctx context.Context, req *DeleteWorkflowExecutionRequest) (*DeleteWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteWorkflowExecution not implemented") +} func (*UnimplementedHistoryServiceServer) ResetWorkflowExecution(ctx context.Context, req *ResetWorkflowExecutionRequest) (*ResetWorkflowExecutionResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResetWorkflowExecution not implemented") } @@ -1183,6 +1205,24 @@ func _HistoryService_TerminateWorkflowExecution_Handler(srv interface{}, ctx con return interceptor(ctx, in, info, handler) } +func _HistoryService_DeleteWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(HistoryServiceServer).DeleteWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/temporal.server.api.historyservice.v1.HistoryService/DeleteWorkflowExecution", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(HistoryServiceServer).DeleteWorkflowExecution(ctx, req.(*DeleteWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _HistoryService_ResetWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ResetWorkflowExecutionRequest) if err := dec(in); err != nil { @@ -1665,6 +1705,10 @@ var _HistoryService_serviceDesc = grpc.ServiceDesc{ MethodName: "TerminateWorkflowExecution", Handler: _HistoryService_TerminateWorkflowExecution_Handler, }, + { + MethodName: "DeleteWorkflowExecution", + Handler: _HistoryService_DeleteWorkflowExecution_Handler, + }, { MethodName: "ResetWorkflowExecution", Handler: _HistoryService_ResetWorkflowExecution_Handler, diff --git a/api/historyservicemock/v1/service.pb.mock.go b/api/historyservicemock/v1/service.pb.mock.go index d092fbd5582..7fc2aaef872 100644 --- a/api/historyservicemock/v1/service.pb.mock.go +++ b/api/historyservicemock/v1/service.pb.mock.go @@ -80,6 +80,26 @@ func (mr *MockHistoryServiceClientMockRecorder) CloseShard(ctx, in interface{}, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloseShard", reflect.TypeOf((*MockHistoryServiceClient)(nil).CloseShard), varargs...) } +// DeleteWorkflowExecution mocks base method. +func (m *MockHistoryServiceClient) DeleteWorkflowExecution(ctx context.Context, in *historyservice.DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*historyservice.DeleteWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DeleteWorkflowExecution", varargs...) + ret0, _ := ret[0].(*historyservice.DeleteWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. +func (mr *MockHistoryServiceClientMockRecorder) DeleteWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockHistoryServiceClient)(nil).DeleteWorkflowExecution), varargs...) +} + // DescribeHistoryHost mocks base method. func (m *MockHistoryServiceClient) DescribeHistoryHost(ctx context.Context, in *historyservice.DescribeHistoryHostRequest, opts ...grpc.CallOption) (*historyservice.DescribeHistoryHostResponse, error) { m.ctrl.T.Helper() @@ -878,6 +898,21 @@ func (mr *MockHistoryServiceServerMockRecorder) CloseShard(arg0, arg1 interface{ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloseShard", reflect.TypeOf((*MockHistoryServiceServer)(nil).CloseShard), arg0, arg1) } +// DeleteWorkflowExecution mocks base method. +func (m *MockHistoryServiceServer) DeleteWorkflowExecution(arg0 context.Context, arg1 *historyservice.DeleteWorkflowExecutionRequest) (*historyservice.DeleteWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*historyservice.DeleteWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. +func (mr *MockHistoryServiceServerMockRecorder) DeleteWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockHistoryServiceServer)(nil).DeleteWorkflowExecution), arg0, arg1) +} + // DescribeHistoryHost mocks base method. func (m *MockHistoryServiceServer) DescribeHistoryHost(arg0 context.Context, arg1 *historyservice.DescribeHistoryHostRequest) (*historyservice.DescribeHistoryHostResponse, error) { m.ctrl.T.Helper() diff --git a/client/history/client.go b/client/history/client.go index 09f8ad83857..83b91baa368 100644 --- a/client/history/client.go +++ b/client/history/client.go @@ -678,6 +678,30 @@ func (c *clientImpl) TerminateWorkflowExecution( } +func (c *clientImpl) DeleteWorkflowExecution( + ctx context.Context, + request *historyservice.DeleteWorkflowExecutionRequest, + opts ...grpc.CallOption) (*historyservice.DeleteWorkflowExecutionResponse, error) { + client, err := c.getClientForWorkflowID(request.NamespaceId, request.WorkflowExecution.WorkflowId) + if err != nil { + return nil, err + } + + var response *historyservice.DeleteWorkflowExecutionResponse + op := func(ctx context.Context, client historyservice.HistoryServiceClient) error { + var err error + ctx, cancel := c.createContext(ctx) + defer cancel() + response, err = client.DeleteWorkflowExecution(ctx, request, opts...) + return err + } + err = c.executeWithRedirect(ctx, client, op) + if err != nil { + return nil, err + } + return response, nil +} + func (c *clientImpl) ResetWorkflowExecution( ctx context.Context, request *historyservice.ResetWorkflowExecutionRequest, diff --git a/client/history/metricClient.go b/client/history/metricClient.go index 30fc768073a..eb4804a7915 100644 --- a/client/history/metricClient.go +++ b/client/history/metricClient.go @@ -322,6 +322,23 @@ func (c *metricClient) TerminateWorkflowExecution( return c.client.TerminateWorkflowExecution(context, request, opts...) } +func (c *metricClient) DeleteWorkflowExecution( + context context.Context, + request *historyservice.DeleteWorkflowExecutionRequest, + opts ...grpc.CallOption) (*historyservice.DeleteWorkflowExecutionResponse, error) { + c.metricsClient.IncCounter(metrics.HistoryClientDeleteWorkflowExecutionScope, metrics.ClientRequests) + + sw := c.metricsClient.StartTimer(metrics.HistoryClientDeleteWorkflowExecutionScope, metrics.ClientLatency) + resp, err := c.client.DeleteWorkflowExecution(context, request, opts...) + sw.Stop() + + if err != nil { + c.metricsClient.IncCounter(metrics.HistoryClientDeleteWorkflowExecutionScope, metrics.ClientFailures) + } + + return resp, err +} + func (c *metricClient) ResetWorkflowExecution( context context.Context, request *historyservice.ResetWorkflowExecutionRequest, diff --git a/client/history/retryableClient.go b/client/history/retryableClient.go index f816061f122..f32d6c6f13f 100644 --- a/client/history/retryableClient.go +++ b/client/history/retryableClient.go @@ -418,6 +418,22 @@ func (c *retryableClient) TerminateWorkflowExecution( return resp, err } +func (c *retryableClient) DeleteWorkflowExecution( + ctx context.Context, + request *historyservice.DeleteWorkflowExecutionRequest, + opts ...grpc.CallOption) (*historyservice.DeleteWorkflowExecutionResponse, error) { + + var resp *historyservice.DeleteWorkflowExecutionResponse + op := func() error { + var err error + resp, err = c.client.DeleteWorkflowExecution(ctx, request, opts...) + return err + } + + err := backoff.Retry(op, c.policy, c.isRetryable) + return resp, err +} + func (c *retryableClient) ResetWorkflowExecution( ctx context.Context, request *historyservice.ResetWorkflowExecutionRequest, diff --git a/common/metrics/defs.go b/common/metrics/defs.go index 33969c637f1..d039bae0150 100644 --- a/common/metrics/defs.go +++ b/common/metrics/defs.go @@ -377,6 +377,8 @@ const ( HistoryClientRemoveSignalMutableStateScope // HistoryClientTerminateWorkflowExecutionScope tracks RPC calls to history service HistoryClientTerminateWorkflowExecutionScope + // HistoryClientDeleteWorkflowExecutionScope tracks RPC calls to history service + HistoryClientDeleteWorkflowExecutionScope // HistoryClientResetWorkflowExecutionScope tracks RPC calls to history service HistoryClientResetWorkflowExecutionScope // HistoryClientScheduleWorkflowTaskScope tracks RPC calls to history service @@ -1324,6 +1326,7 @@ var ScopeDefs = map[ServiceIdx]map[int]scopeDefinition{ HistoryClientSignalWithStartWorkflowExecutionScope: {operation: "HistoryClientSignalWithStartWorkflowExecution", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, HistoryClientRemoveSignalMutableStateScope: {operation: "HistoryClientRemoveSignalMutableStateScope", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, HistoryClientTerminateWorkflowExecutionScope: {operation: "HistoryClientTerminateWorkflowExecution", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, + HistoryClientDeleteWorkflowExecutionScope: {operation: "HistoryClientDeleteWorkflowExecution", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, HistoryClientResetWorkflowExecutionScope: {operation: "HistoryClientResetWorkflowExecution", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, HistoryClientScheduleWorkflowTaskScope: {operation: "HistoryClientScheduleWorkflowTask", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, HistoryClientRecordChildExecutionCompletedScope: {operation: "HistoryClientRecordChildExecutionCompleted", tags: map[string]string{ServiceRoleTagName: HistoryRoleTagValue}}, diff --git a/proto/api-linter.yaml b/proto/api-linter.yaml index 4767fd854c2..82e05104781 100644 --- a/proto/api-linter.yaml +++ b/proto/api-linter.yaml @@ -23,6 +23,8 @@ - 'core::0132::request-unknown-fields' - 'core::0132::response-unknown-fields' - 'core::0134::request-unknown-fields' + - 'core::0135::request-name-required' + - 'core::0135::request-unknown-fields' - 'core::0158::response-next-page-token-field' - 'core::0158::response-plural-first-field' - 'core::0158::response-repeated-first-field' @@ -40,3 +42,5 @@ - 'core::0127::http-annotation' - 'core::0131::method-signature' - 'core::0131::response-message-name' + - 'core::0135::method-signature' + - 'core::0135::response-message-name' diff --git a/proto/internal/temporal/server/api/historyservice/v1/request_response.proto b/proto/internal/temporal/server/api/historyservice/v1/request_response.proto index 95b250d8e17..b40458fc092 100644 --- a/proto/internal/temporal/server/api/historyservice/v1/request_response.proto +++ b/proto/internal/temporal/server/api/historyservice/v1/request_response.proto @@ -269,6 +269,14 @@ message TerminateWorkflowExecutionRequest { message TerminateWorkflowExecutionResponse { } +message DeleteWorkflowExecutionRequest { + string namespace_id = 1; + temporal.api.common.v1.WorkflowExecution workflow_execution = 2; +} + +message DeleteWorkflowExecutionResponse { +} + message ResetWorkflowExecutionRequest { string namespace_id = 1; temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest reset_request = 2; diff --git a/proto/internal/temporal/server/api/historyservice/v1/service.proto b/proto/internal/temporal/server/api/historyservice/v1/service.proto index 2adb21ddcab..fddb0aeac14 100644 --- a/proto/internal/temporal/server/api/historyservice/v1/service.proto +++ b/proto/internal/temporal/server/api/historyservice/v1/service.proto @@ -139,6 +139,12 @@ service HistoryService { rpc TerminateWorkflowExecution (TerminateWorkflowExecutionRequest) returns (TerminateWorkflowExecutionResponse) { } + // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history. + // It also asynchronously deletes visibility record. + // If call to DeleteWorkflowExecution fails, it must be retried to prevent unreferenced data. + rpc DeleteWorkflowExecution (DeleteWorkflowExecutionRequest) returns (DeleteWorkflowExecutionResponse) { + } + // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. // After reset, the history will grow from nextFirstEventId. @@ -238,6 +244,5 @@ service HistoryService { } rpc GetReplicationStatus(GetReplicationStatusRequest) returns (GetReplicationStatusResponse) { - } } diff --git a/service/history/handler.go b/service/history/handler.go index 65ae1ec042b..e7630ef8be4 100644 --- a/service/history/handler.go +++ b/service/history/handler.go @@ -947,6 +947,34 @@ func (h *Handler) TerminateWorkflowExecution(ctx context.Context, request *histo return &historyservice.TerminateWorkflowExecutionResponse{}, nil } +func (h *Handler) DeleteWorkflowExecution(ctx context.Context, request *historyservice.DeleteWorkflowExecutionRequest) (_ *historyservice.DeleteWorkflowExecutionResponse, retError error) { + defer log.CapturePanic(h.logger, &retError) + h.startWG.Wait() + + if h.isStopped() { + return nil, errShuttingDown + } + + namespaceID := namespace.ID(request.GetNamespaceId()) + if namespaceID == "" { + return nil, h.convertError(errNamespaceNotSet) + } + + workflowExecution := request.WorkflowExecution + workflowID := workflowExecution.GetWorkflowId() + engine, err1 := h.controller.GetEngine(ctx, namespaceID, workflowID) + if err1 != nil { + return nil, h.convertError(err1) + } + + err2 := engine.DeleteWorkflowExecution(ctx, request) + if err2 != nil { + return nil, h.convertError(err2) + } + + return &historyservice.DeleteWorkflowExecutionResponse{}, nil +} + // ResetWorkflowExecution reset an existing workflow execution // in the history and immediately terminating the execution instance. func (h *Handler) ResetWorkflowExecution(ctx context.Context, request *historyservice.ResetWorkflowExecutionRequest) (_ *historyservice.ResetWorkflowExecutionResponse, retError error) { diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 04889a4bb73..841b102bcf0 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -2249,6 +2249,18 @@ func (e *historyEngineImpl) TerminateWorkflowExecution( }) } +func (e *historyEngineImpl) DeleteWorkflowExecution( + ctx context.Context, + deleteRequest *historyservice.DeleteWorkflowExecutionRequest, +) error { + return e.workflowDeleteManager.DeleteWorkflowExecution( + ctx, + namespace.ID(deleteRequest.NamespaceId), + *deleteRequest.WorkflowExecution, + false, + ) +} + // RecordChildExecutionCompleted records the completion of child execution into parent execution history func (e *historyEngineImpl) RecordChildExecutionCompleted( ctx context.Context, diff --git a/service/history/shard/engine.go b/service/history/shard/engine.go index 4b01c0455f1..c6e85723409 100644 --- a/service/history/shard/engine.go +++ b/service/history/shard/engine.go @@ -65,6 +65,7 @@ type ( SignalWithStartWorkflowExecution(ctx context.Context, request *historyservice.SignalWithStartWorkflowExecutionRequest) (*historyservice.SignalWithStartWorkflowExecutionResponse, error) RemoveSignalMutableState(ctx context.Context, request *historyservice.RemoveSignalMutableStateRequest) error TerminateWorkflowExecution(ctx context.Context, request *historyservice.TerminateWorkflowExecutionRequest) error + DeleteWorkflowExecution(ctx context.Context, deleteRequest *historyservice.DeleteWorkflowExecutionRequest) error ResetWorkflowExecution(ctx context.Context, request *historyservice.ResetWorkflowExecutionRequest) (*historyservice.ResetWorkflowExecutionResponse, error) ScheduleWorkflowTask(ctx context.Context, request *historyservice.ScheduleWorkflowTaskRequest) error RecordChildExecutionCompleted(ctx context.Context, request *historyservice.RecordChildExecutionCompletedRequest) error diff --git a/service/history/shard/engine_mock.go b/service/history/shard/engine_mock.go index 4d7002eb84e..51991fa3698 100644 --- a/service/history/shard/engine_mock.go +++ b/service/history/shard/engine_mock.go @@ -66,6 +66,20 @@ func (m *MockEngine) EXPECT() *MockEngineMockRecorder { return m.recorder } +// DeleteWorkflowExecution mocks base method. +func (m *MockEngine) DeleteWorkflowExecution(ctx context.Context, deleteRequest *historyservice.DeleteWorkflowExecutionRequest) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteWorkflowExecution", ctx, deleteRequest) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. +func (mr *MockEngineMockRecorder) DeleteWorkflowExecution(ctx, deleteRequest interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockEngine)(nil).DeleteWorkflowExecution), ctx, deleteRequest) +} + // DescribeMutableState mocks base method. func (m *MockEngine) DescribeMutableState(ctx context.Context, request *historyservice.DescribeMutableStateRequest) (*historyservice.DescribeMutableStateResponse, error) { m.ctrl.T.Helper() From 9aded93dc18385fe83c76bc5c9001a7fcb97a707 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Fri, 17 Dec 2021 14:05:01 -0800 Subject: [PATCH 05/12] Fix unit test --- service/history/configs/quotas.go | 1 + 1 file changed, 1 insertion(+) diff --git a/service/history/configs/quotas.go b/service/history/configs/quotas.go index 03919704415..29c70f7e206 100644 --- a/service/history/configs/quotas.go +++ b/service/history/configs/quotas.go @@ -32,6 +32,7 @@ var ( APIToPriority = map[string]int{ "CloseShard": 0, "GetShard": 0, + "DeleteWorkflowExecution": 0, "DescribeHistoryHost": 0, "DescribeMutableState": 0, "DescribeWorkflowExecution": 0, From de9418b05bfd55999b4fa9573d0bcf3a607336b9 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Sat, 18 Dec 2021 12:40:30 -0800 Subject: [PATCH 06/12] Fix proto comment --- .../temporal/server/api/historyservice/v1/service.proto | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/proto/internal/temporal/server/api/historyservice/v1/service.proto b/proto/internal/temporal/server/api/historyservice/v1/service.proto index fddb0aeac14..de7286a823f 100644 --- a/proto/internal/temporal/server/api/historyservice/v1/service.proto +++ b/proto/internal/temporal/server/api/historyservice/v1/service.proto @@ -139,9 +139,9 @@ service HistoryService { rpc TerminateWorkflowExecution (TerminateWorkflowExecutionRequest) returns (TerminateWorkflowExecutionResponse) { } - // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history. - // It also asynchronously deletes visibility record. - // If call to DeleteWorkflowExecution fails, it must be retried to prevent unreferenced data. + // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history synchronously. + // Visibility record is deleted asynchronously. + // If call to DeleteWorkflowExecution fails, it must be retried to prevent orphan DB records. rpc DeleteWorkflowExecution (DeleteWorkflowExecutionRequest) returns (DeleteWorkflowExecutionResponse) { } From 04a8452d176f99f89e8610dc4eb9447aebcd602e Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Thu, 13 Jan 2022 10:34:07 -0800 Subject: [PATCH 07/12] Add DeleteExecution transfer task --- api/enums/v1/task.pb.go | 83 ++++++++++--------- api/enums/v1/workflow.pb.go | 28 ++++--- api/historyservice/v1/service.pb.go | 12 +-- common/persistence/operationModeValidator.go | 3 +- .../serialization/task_serializer.go | 39 +++++++++ .../temporal/server/api/enums/v1/task.proto | 1 + .../server/api/enums/v1/workflow.proto | 1 + service/history/historyEngine.go | 32 ++++++- .../history/tasks/delete_execution_task.go | 71 ++++++++++++++++ service/history/timerQueueTaskExecutorBase.go | 2 +- .../timerQueueTaskExecutorBase_test.go | 4 +- .../transferQueueActiveTaskExecutor.go | 3 + .../transferQueueStandbyTaskExecutor.go | 3 + .../history/transferQueueTaskExecutorBase.go | 50 +++++++++++ service/history/workflow/delete_manager.go | 63 ++++++-------- .../history/workflow/delete_manager_mock.go | 25 +++--- .../history/workflow/mutable_state_impl.go | 2 + .../workflow/mutable_state_state_status.go | 24 ++++++ service/history/workflow/task_generator.go | 19 +++++ .../history/workflow/task_generator_mock.go | 14 ++++ 20 files changed, 363 insertions(+), 116 deletions(-) create mode 100644 service/history/tasks/delete_execution_task.go diff --git a/api/enums/v1/task.pb.go b/api/enums/v1/task.pb.go index d2e88258614..a9619b6e122 100644 --- a/api/enums/v1/task.pb.go +++ b/api/enums/v1/task.pb.go @@ -132,6 +132,7 @@ const ( TASK_TYPE_VISIBILITY_CLOSE_EXECUTION TaskType = 21 TASK_TYPE_VISIBILITY_DELETE_EXECUTION TaskType = 22 TASK_TYPE_TIERED_STORAGE TaskType = 23 + TASK_TYPE_TRANSFER_DELETE_EXECUTION TaskType = 24 ) var TaskType_name = map[int32]string{ @@ -157,6 +158,7 @@ var TaskType_name = map[int32]string{ 21: "VisibilityCloseExecution", 22: "VisibilityDeleteExecution", 23: "TieredStorage", + 24: "TransferDeleteExecution", } var TaskType_value = map[string]int32{ @@ -182,6 +184,7 @@ var TaskType_value = map[string]int32{ "VisibilityCloseExecution": 21, "VisibilityDeleteExecution": 22, "TieredStorage": 23, + "TransferDeleteExecution": 24, } func (TaskType) EnumDescriptor() ([]byte, []int) { @@ -199,46 +202,46 @@ func init() { } var fileDescriptor_36a3d3674ca3cfa6 = []byte{ - // 610 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x94, 0xcd, 0x4e, 0xdb, 0x4e, - 0x14, 0xc5, 0x6d, 0xc8, 0x9f, 0x7f, 0xb8, 0xd0, 0x76, 0x3a, 0x7c, 0x53, 0x98, 0x96, 0x00, 0x85, - 0xa2, 0x2a, 0x11, 0xea, 0xb2, 0x2b, 0x67, 0x32, 0x09, 0x23, 0x5c, 0x3b, 0x9a, 0x99, 0x40, 0xd3, - 0x05, 0x56, 0x5a, 0x59, 0x08, 0x51, 0xea, 0xc8, 0x09, 0x48, 0xec, 0xfa, 0x08, 0x7d, 0x83, 0x6e, - 0xfb, 0x28, 0x5d, 0xd2, 0x1d, 0xcb, 0x62, 0x36, 0x5d, 0xf2, 0x08, 0x95, 0x4d, 0xe2, 0x0f, 0xea, - 0xec, 0x22, 0x9d, 0x5f, 0xce, 0xbd, 0xf7, 0xdc, 0xeb, 0x81, 0xad, 0xbe, 0x7b, 0xd6, 0xf5, 0xfc, - 0xce, 0xe7, 0x4a, 0xcf, 0xf5, 0x2f, 0x5c, 0xbf, 0xd2, 0xe9, 0x9e, 0x54, 0xdc, 0x2f, 0xe7, 0x67, - 0xbd, 0xca, 0xc5, 0x6e, 0xa5, 0xdf, 0xe9, 0x9d, 0x96, 0xbb, 0xbe, 0xd7, 0xf7, 0xf0, 0xca, 0x10, - 0x2c, 0xdf, 0x83, 0xe5, 0x4e, 0xf7, 0xa4, 0x1c, 0x81, 0xe5, 0x8b, 0xdd, 0x9d, 0x23, 0x00, 0xd5, - 0xe9, 0x9d, 0x4a, 0xef, 0xdc, 0xff, 0xe4, 0xe2, 0x67, 0xb0, 0xa0, 0x0c, 0xb9, 0xef, 0x48, 0xbb, - 0x25, 0x28, 0x73, 0x5a, 0x96, 0x6c, 0x32, 0xca, 0xeb, 0x9c, 0xd5, 0x90, 0x86, 0x17, 0x60, 0x26, - 0x2d, 0xee, 0x71, 0xa9, 0x6c, 0xd1, 0x46, 0x3a, 0x5e, 0x86, 0xf9, 0xb4, 0x50, 0xab, 0x3a, 0x55, - 0x83, 0xee, 0x9b, 0x76, 0x03, 0x8d, 0xed, 0x7c, 0xd7, 0x61, 0x3a, 0x2c, 0x40, 0x3b, 0x7d, 0xf7, - 0xd8, 0xf3, 0x2f, 0xf1, 0x2a, 0x2c, 0x45, 0x30, 0x35, 0x14, 0x6b, 0xd8, 0xa2, 0xfd, 0xa0, 0xc8, - 0xd0, 0x2b, 0x96, 0x95, 0x30, 0x2c, 0x59, 0x67, 0x02, 0xe9, 0x71, 0x03, 0x89, 0xc6, 0xdf, 0x31, - 0x81, 0xc6, 0xfe, 0xf5, 0x14, 0xac, 0x69, 0x72, 0x6a, 0x28, 0x6e, 0x5b, 0x68, 0x1c, 0xaf, 0xc0, - 0x62, 0x56, 0x3e, 0xe0, 0x92, 0x57, 0xb9, 0xc9, 0x55, 0x1b, 0x15, 0x76, 0x7e, 0x4d, 0x40, 0x31, - 0xec, 0x50, 0x5d, 0x76, 0x5d, 0xbc, 0x04, 0x73, 0x11, 0xaa, 0xda, 0xcd, 0x87, 0xe3, 0xaf, 0xc1, - 0x6a, 0x22, 0xa5, 0x0a, 0xa4, 0x82, 0xd8, 0x82, 0xf5, 0x7c, 0x44, 0xb6, 0x2d, 0xea, 0x18, 0x54, - 0xf1, 0x83, 0xb0, 0xe6, 0x18, 0xde, 0x80, 0x17, 0x09, 0x38, 0x9c, 0xd0, 0x39, 0xb4, 0xc5, 0x7e, - 0xdd, 0xb4, 0x0f, 0x9d, 0x50, 0x43, 0xe3, 0x23, 0xa8, 0xa1, 0xcd, 0x3d, 0x55, 0xc0, 0x2f, 0xa1, - 0x94, 0x43, 0x51, 0xd3, 0x96, 0xcc, 0x61, 0xef, 0x19, 0x6d, 0x45, 0x29, 0xfc, 0x97, 0x6d, 0x2e, - 0xe1, 0x0c, 0x8b, 0x32, 0x33, 0x05, 0x4e, 0xe0, 0xd7, 0xb0, 0x9d, 0x03, 0x4a, 0x65, 0x08, 0xe5, - 0xd0, 0x3d, 0x6e, 0xd6, 0x52, 0xf4, 0xff, 0x23, 0x6c, 0x25, 0x6f, 0x58, 0x46, 0xda, 0xb6, 0x88, - 0x37, 0x61, 0x2d, 0x07, 0x14, 0x4c, 0x32, 0x15, 0x4f, 0x8e, 0x00, 0xaf, 0xc3, 0xf3, 0x04, 0xcb, - 0x24, 0x12, 0xad, 0xdb, 0x6e, 0x29, 0x34, 0x8d, 0x09, 0x2c, 0x27, 0x50, 0x12, 0xc8, 0x40, 0x7f, - 0x84, 0x17, 0x61, 0x36, 0xb5, 0x46, 0xc9, 0xc4, 0xe0, 0x54, 0x1e, 0xe3, 0x12, 0x90, 0x1c, 0x7b, - 0xd1, 0xb2, 0xe2, 0x7f, 0x3f, 0xc9, 0x32, 0x35, 0x66, 0x32, 0x15, 0x5f, 0xbb, 0xc3, 0x0e, 0x98, - 0xa5, 0x10, 0xca, 0x32, 0x71, 0x07, 0x82, 0xa9, 0xf8, 0x2c, 0x9f, 0x66, 0xf7, 0x17, 0xd7, 0x0a, - 0xbf, 0x0d, 0xbb, 0x5e, 0x1f, 0x50, 0x18, 0x6f, 0xc3, 0x46, 0x42, 0x25, 0x97, 0x39, 0x08, 0x3c, - 0x49, 0x70, 0x06, 0xbf, 0x82, 0xcd, 0x5c, 0xb2, 0xd5, 0x94, 0x2c, 0x83, 0xce, 0x8e, 0x34, 0x7d, - 0x78, 0x16, 0x73, 0x23, 0x4d, 0x07, 0x73, 0x27, 0xe8, 0x7c, 0xfc, 0x1d, 0xdd, 0x6f, 0x90, 0x33, - 0xc1, 0x6a, 0x4e, 0x98, 0x8a, 0xd1, 0x60, 0x68, 0xa1, 0x54, 0x28, 0x4e, 0xa2, 0xc9, 0x52, 0xa1, - 0x38, 0x85, 0xa6, 0xaa, 0x47, 0x57, 0x37, 0x44, 0xbb, 0xbe, 0x21, 0xda, 0xdd, 0x0d, 0xd1, 0xbf, - 0x06, 0x44, 0xff, 0x11, 0x10, 0xfd, 0x67, 0x40, 0xf4, 0xab, 0x80, 0xe8, 0xbf, 0x03, 0xa2, 0xff, - 0x09, 0x88, 0x76, 0x17, 0x10, 0xfd, 0xdb, 0x2d, 0xd1, 0xae, 0x6e, 0x89, 0x76, 0x7d, 0x4b, 0xb4, - 0x0f, 0xdb, 0xc7, 0x5e, 0x39, 0x7e, 0xac, 0x4e, 0xbc, 0xbc, 0x87, 0xed, 0x6d, 0xf4, 0xe3, 0xe3, - 0x44, 0xf4, 0xb4, 0xbd, 0xf9, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x82, 0xb9, 0x79, 0xa8, 0x05, 0x05, - 0x00, 0x00, + // 615 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x94, 0xcf, 0x4f, 0x13, 0x41, + 0x14, 0xc7, 0x77, 0xa1, 0x62, 0x79, 0xa0, 0x8e, 0xc3, 0x6f, 0x84, 0x51, 0x0a, 0x08, 0x12, 0xd3, + 0x86, 0x78, 0xf4, 0xb4, 0x9d, 0x4e, 0xcb, 0x84, 0x75, 0xb7, 0x99, 0x99, 0x82, 0xf5, 0xc0, 0xa6, + 0x9a, 0x0d, 0x21, 0x88, 0xdb, 0xb4, 0x85, 0x84, 0x9b, 0x7f, 0x82, 0xff, 0x81, 0x57, 0xff, 0x14, + 0x8f, 0x1c, 0x39, 0xca, 0x72, 0xf1, 0xc8, 0xd9, 0x93, 0xd9, 0xa5, 0xdd, 0x1f, 0x65, 0x7b, 0x6b, + 0xf2, 0xfd, 0xf4, 0xfb, 0xde, 0xfb, 0xbe, 0xb7, 0x03, 0x5b, 0x3d, 0xf7, 0xac, 0xed, 0x75, 0x5a, + 0x5f, 0x4b, 0x5d, 0xb7, 0x73, 0xe1, 0x76, 0x4a, 0xad, 0xf6, 0x49, 0xc9, 0xfd, 0x76, 0x7e, 0xd6, + 0x2d, 0x5d, 0xec, 0x96, 0x7a, 0xad, 0xee, 0x69, 0xb1, 0xdd, 0xf1, 0x7a, 0x1e, 0x5e, 0x19, 0x80, + 0xc5, 0x7b, 0xb0, 0xd8, 0x6a, 0x9f, 0x14, 0x43, 0xb0, 0x78, 0xb1, 0xbb, 0x73, 0x04, 0xa0, 0x5a, + 0xdd, 0x53, 0xe9, 0x9d, 0x77, 0xbe, 0xb8, 0xf8, 0x05, 0x2c, 0x28, 0x43, 0xee, 0x3b, 0xd2, 0x6e, + 0x08, 0xca, 0x9c, 0x86, 0x25, 0xeb, 0x8c, 0xf2, 0x2a, 0x67, 0x15, 0xa4, 0xe1, 0x05, 0x98, 0x49, + 0x8a, 0x7b, 0x5c, 0x2a, 0x5b, 0x34, 0x91, 0x8e, 0x97, 0x61, 0x3e, 0x29, 0x54, 0xca, 0x4e, 0xd9, + 0xa0, 0xfb, 0xa6, 0x5d, 0x43, 0x63, 0x3b, 0x3f, 0x75, 0x98, 0x0e, 0x0a, 0xd0, 0x56, 0xcf, 0x3d, + 0xf6, 0x3a, 0x97, 0x78, 0x15, 0x96, 0x42, 0x98, 0x1a, 0x8a, 0xd5, 0x6c, 0xd1, 0x1c, 0x2a, 0x32, + 0xf0, 0x8a, 0x64, 0x25, 0x0c, 0x4b, 0x56, 0x99, 0x40, 0x7a, 0xd4, 0x40, 0xac, 0xf1, 0x0f, 0x4c, + 0xa0, 0xb1, 0x87, 0x9e, 0x82, 0xd5, 0x4d, 0x4e, 0x0d, 0xc5, 0x6d, 0x0b, 0x8d, 0xe3, 0x15, 0x58, + 0x4c, 0xcb, 0x07, 0x5c, 0xf2, 0x32, 0x37, 0xb9, 0x6a, 0xa2, 0xdc, 0xce, 0xbf, 0x09, 0xc8, 0x07, + 0x1d, 0xaa, 0xcb, 0xb6, 0x8b, 0x97, 0x60, 0x2e, 0x44, 0x55, 0xb3, 0x3e, 0x3c, 0xfe, 0x1a, 0xac, + 0xc6, 0x52, 0xa2, 0x40, 0x22, 0x88, 0x2d, 0x58, 0xcf, 0x46, 0x64, 0xd3, 0xa2, 0x8e, 0x41, 0x15, + 0x3f, 0x08, 0x6a, 0x8e, 0xe1, 0x0d, 0x78, 0x15, 0x83, 0x83, 0x09, 0x9d, 0x43, 0x5b, 0xec, 0x57, + 0x4d, 0xfb, 0xd0, 0x09, 0x34, 0x34, 0x3e, 0x82, 0x1a, 0xd8, 0xdc, 0x53, 0x39, 0xfc, 0x1a, 0x0a, + 0x19, 0x14, 0x35, 0x6d, 0xc9, 0x1c, 0xf6, 0x91, 0xd1, 0x46, 0x98, 0xc2, 0xa3, 0x74, 0x73, 0x31, + 0x67, 0x58, 0x94, 0x99, 0x09, 0x70, 0x02, 0xbf, 0x85, 0xed, 0x0c, 0x50, 0x2a, 0x43, 0x28, 0x87, + 0xee, 0x71, 0xb3, 0x92, 0xa0, 0x1f, 0x8f, 0xb0, 0x95, 0xbc, 0x66, 0x19, 0x49, 0xdb, 0x3c, 0xde, + 0x84, 0xb5, 0x0c, 0x50, 0x30, 0xc9, 0x54, 0x34, 0x39, 0x02, 0xbc, 0x0e, 0x2f, 0x63, 0x2c, 0x95, + 0x48, 0xb8, 0x6e, 0xbb, 0xa1, 0xd0, 0x34, 0x26, 0xb0, 0x1c, 0x43, 0x71, 0x20, 0x7d, 0xfd, 0x09, + 0x5e, 0x84, 0xd9, 0xc4, 0x1a, 0x25, 0x13, 0xfd, 0x53, 0x79, 0x8a, 0x0b, 0x40, 0x32, 0xec, 0x45, + 0xc3, 0x8a, 0xfe, 0xfd, 0x2c, 0xcd, 0x54, 0x98, 0xc9, 0x54, 0x74, 0xed, 0x0e, 0x3b, 0x60, 0x96, + 0x42, 0x28, 0xcd, 0x44, 0x1d, 0x08, 0xa6, 0xa2, 0xb3, 0x7c, 0x9e, 0xde, 0x5f, 0x54, 0x2b, 0xf8, + 0x36, 0xec, 0x6a, 0xb5, 0x4f, 0x61, 0xbc, 0x0d, 0x1b, 0x31, 0x15, 0x5f, 0x66, 0x3f, 0xf0, 0x38, + 0xc1, 0x19, 0xfc, 0x06, 0x36, 0x33, 0xc9, 0x46, 0x5d, 0xb2, 0x14, 0x3a, 0x3b, 0xd2, 0x74, 0xf8, + 0x2c, 0xe6, 0x46, 0x9a, 0xf6, 0xe7, 0x8e, 0xd1, 0xf9, 0xe8, 0x3b, 0xba, 0xdf, 0x20, 0x67, 0x82, + 0x55, 0x9c, 0x20, 0x15, 0xa3, 0xc6, 0xd0, 0xc2, 0x88, 0x43, 0x78, 0x60, 0xb3, 0x58, 0xc8, 0xe5, + 0x27, 0xd1, 0x64, 0x21, 0x97, 0x9f, 0x42, 0x53, 0xe5, 0xa3, 0xab, 0x1b, 0xa2, 0x5d, 0xdf, 0x10, + 0xed, 0xee, 0x86, 0xe8, 0xdf, 0x7d, 0xa2, 0xff, 0xf2, 0x89, 0xfe, 0xdb, 0x27, 0xfa, 0x95, 0x4f, + 0xf4, 0x3f, 0x3e, 0xd1, 0xff, 0xfa, 0x44, 0xbb, 0xf3, 0x89, 0xfe, 0xe3, 0x96, 0x68, 0x57, 0xb7, + 0x44, 0xbb, 0xbe, 0x25, 0xda, 0xa7, 0xed, 0x63, 0xaf, 0x18, 0xbd, 0x6a, 0x27, 0x5e, 0xd6, 0x0b, + 0xf8, 0x3e, 0xfc, 0xf1, 0x79, 0x22, 0x7c, 0x03, 0xdf, 0xfd, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x0f, + 0x02, 0x01, 0x38, 0x2e, 0x05, 0x00, 0x00, } func (x TaskSource) String() string { diff --git a/api/enums/v1/workflow.pb.go b/api/enums/v1/workflow.pb.go index de8803f5938..e502c1932ef 100644 --- a/api/enums/v1/workflow.pb.go +++ b/api/enums/v1/workflow.pb.go @@ -56,6 +56,7 @@ const ( WORKFLOW_EXECUTION_STATE_ZOMBIE WorkflowExecutionState = 4 WORKFLOW_EXECUTION_STATE_VOID WorkflowExecutionState = 5 WORKFLOW_EXECUTION_STATE_CORRUPTED WorkflowExecutionState = 6 + WORKFLOW_EXECUTION_STATE_DELETED WorkflowExecutionState = 7 ) var WorkflowExecutionState_name = map[int32]string{ @@ -66,6 +67,7 @@ var WorkflowExecutionState_name = map[int32]string{ 4: "Zombie", 5: "Void", 6: "Corrupted", + 7: "Deleted", } var WorkflowExecutionState_value = map[string]int32{ @@ -76,6 +78,7 @@ var WorkflowExecutionState_value = map[string]int32{ "Zombie": 4, "Void": 5, "Corrupted": 6, + "Deleted": 7, } func (WorkflowExecutionState) EnumDescriptor() ([]byte, []int) { @@ -116,12 +119,12 @@ func init() { } var fileDescriptor_004b7fefe981a755 = []byte{ - // 366 bytes of a gzipped FileDescriptorProto + // 374 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xd2, 0x2e, 0x49, 0xcd, 0x2d, 0xc8, 0x2f, 0x4a, 0xcc, 0xd1, 0x2f, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x4f, 0x2c, 0xc8, 0xd4, 0x4f, 0xcd, 0x2b, 0xcd, 0x2d, 0xd6, 0x2f, 0x33, 0xd4, 0x2f, 0xcf, 0x2f, 0xca, 0x4e, 0xcb, 0xc9, 0x2f, 0xd7, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x92, 0x81, 0x29, 0xd6, 0x83, 0x28, 0xd6, 0x4b, - 0x2c, 0xc8, 0xd4, 0x03, 0x2b, 0xd6, 0x2b, 0x33, 0xd4, 0x5a, 0xc6, 0xc4, 0x25, 0x16, 0x0e, 0xd5, + 0x2c, 0xc8, 0xd4, 0x03, 0x2b, 0xd6, 0x2b, 0x33, 0xd4, 0x3a, 0xc3, 0xc4, 0x25, 0x16, 0x0e, 0xd5, 0xe0, 0x5a, 0x91, 0x9a, 0x5c, 0x5a, 0x92, 0x99, 0x9f, 0x17, 0x5c, 0x92, 0x58, 0x92, 0x2a, 0xa4, 0xc1, 0xa5, 0x12, 0xee, 0x1f, 0xe4, 0xed, 0xe6, 0xe3, 0x1f, 0x1e, 0xef, 0x1a, 0xe1, 0xea, 0x1c, 0x1a, 0xe2, 0xe9, 0xef, 0x17, 0x1f, 0x1c, 0xe2, 0x18, 0xe2, 0x1a, 0x1f, 0xea, 0x17, 0x1c, 0xe0, @@ -130,16 +133,17 @@ var fileDescriptor_004b7fefe981a755 = []byte{ 0xee, 0x02, 0x4c, 0x42, 0x6a, 0x5c, 0x4a, 0xb8, 0xcd, 0xf2, 0xf7, 0x0d, 0xf0, 0x71, 0x05, 0x99, 0xc6, 0x2c, 0xa4, 0xcc, 0x25, 0x8f, 0x53, 0x5d, 0x94, 0xbf, 0xaf, 0x93, 0xa7, 0xab, 0x00, 0x8b, 0x90, 0x22, 0x97, 0x2c, 0x4e, 0x45, 0x61, 0xfe, 0x9e, 0x2e, 0x02, 0xac, 0x04, 0xec, 0x0b, 0x0a, - 0x0a, 0x0d, 0x00, 0xd9, 0xc7, 0xa6, 0x55, 0xcb, 0x25, 0x0c, 0x0b, 0x27, 0xa7, 0xc4, 0xe4, 0xec, - 0xfc, 0xb4, 0xb4, 0x90, 0xca, 0x82, 0x54, 0x21, 0x55, 0x2e, 0x45, 0xb8, 0x76, 0x27, 0x47, 0x67, - 0x6f, 0x7f, 0x37, 0xb7, 0xf8, 0x90, 0xc8, 0x00, 0xf4, 0x10, 0x92, 0xe7, 0x92, 0xc6, 0xae, 0x2c, - 0xc8, 0x35, 0x24, 0x28, 0x52, 0x80, 0x51, 0x48, 0x8e, 0x4b, 0x0a, 0xbb, 0x02, 0xe7, 0x20, 0x7f, - 0x3f, 0x01, 0x26, 0xa7, 0xb8, 0x0b, 0x0f, 0xe5, 0x18, 0x6e, 0x3c, 0x94, 0x63, 0xf8, 0xf0, 0x50, - 0x8e, 0xb1, 0xe1, 0x91, 0x1c, 0xe3, 0x8a, 0x47, 0x72, 0x8c, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x78, - 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x8b, 0x47, 0x72, 0x0c, 0x1f, 0x1e, 0xc9, 0x31, 0x4e, - 0x78, 0x2c, 0xc7, 0x70, 0xe1, 0xb1, 0x1c, 0xc3, 0x8d, 0xc7, 0x72, 0x0c, 0x51, 0x1a, 0xe9, 0xf9, - 0x7a, 0xf0, 0xe8, 0xcf, 0xcc, 0xc7, 0x96, 0x5c, 0xac, 0xc1, 0x8c, 0x24, 0x36, 0x70, 0x62, 0x31, - 0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0xfd, 0x0f, 0x05, 0x23, 0x5b, 0x02, 0x00, 0x00, + 0x0a, 0x0d, 0x00, 0xd9, 0xc7, 0x86, 0xd7, 0xf5, 0x2e, 0xae, 0x10, 0x57, 0xb1, 0x6b, 0xd5, 0x72, + 0x09, 0xc3, 0x42, 0xd3, 0x29, 0x31, 0x39, 0x3b, 0x3f, 0x2d, 0x2d, 0xa4, 0xb2, 0x20, 0x55, 0x48, + 0x95, 0x4b, 0x11, 0xae, 0xd9, 0xc9, 0xd1, 0xd9, 0xdb, 0xdf, 0xcd, 0x2d, 0x3e, 0x24, 0x32, 0x00, + 0x3d, 0x1c, 0xe5, 0xb9, 0xa4, 0xb1, 0x2b, 0x0b, 0x72, 0x0d, 0x09, 0x8a, 0x14, 0x60, 0x14, 0x92, + 0xe3, 0x92, 0xc2, 0xae, 0xc0, 0x39, 0xc8, 0xdf, 0x4f, 0x80, 0xc9, 0x29, 0xee, 0xc2, 0x43, 0x39, + 0x86, 0x1b, 0x0f, 0xe5, 0x18, 0x3e, 0x3c, 0x94, 0x63, 0x6c, 0x78, 0x24, 0xc7, 0xb8, 0xe2, 0x91, + 0x1c, 0xe3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0x3e, 0x78, 0x24, 0xc7, 0xf8, 0xe2, + 0x91, 0x1c, 0xc3, 0x87, 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, 0x5c, 0x78, 0x2c, 0xc7, 0x70, + 0xe3, 0xb1, 0x1c, 0x43, 0x94, 0x46, 0x7a, 0xbe, 0x1e, 0x3c, 0x91, 0x64, 0xe6, 0x63, 0x4b, 0x54, + 0xd6, 0x60, 0x46, 0x12, 0x1b, 0x38, 0x49, 0x19, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0x74, 0x1c, + 0x53, 0x64, 0x81, 0x02, 0x00, 0x00, } func (x WorkflowExecutionState) String() string { diff --git a/api/historyservice/v1/service.pb.go b/api/historyservice/v1/service.pb.go index bb71939ba16..53f2f1c8c74 100644 --- a/api/historyservice/v1/service.pb.go +++ b/api/historyservice/v1/service.pb.go @@ -218,9 +218,9 @@ type HistoryServiceClient interface { // TerminateWorkflowExecution terminates an existing workflow execution by recording WorkflowExecutionTerminated event // in the history and immediately terminating the execution instance. TerminateWorkflowExecution(ctx context.Context, in *TerminateWorkflowExecutionRequest, opts ...grpc.CallOption) (*TerminateWorkflowExecutionResponse, error) - // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history. - // It also asynchronously deletes visibility record. - // If call to DeleteWorkflowExecution fails, it must be retried to prevent unreferenced data. + // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history synchronously. + // Visibility record is deleted asynchronously. + // If call to DeleteWorkflowExecution fails, it must be retried to prevent orphan DB records. DeleteWorkflowExecution(ctx context.Context, in *DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*DeleteWorkflowExecutionResponse, error) // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. @@ -727,9 +727,9 @@ type HistoryServiceServer interface { // TerminateWorkflowExecution terminates an existing workflow execution by recording WorkflowExecutionTerminated event // in the history and immediately terminating the execution instance. TerminateWorkflowExecution(context.Context, *TerminateWorkflowExecutionRequest) (*TerminateWorkflowExecutionResponse, error) - // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history. - // It also asynchronously deletes visibility record. - // If call to DeleteWorkflowExecution fails, it must be retried to prevent unreferenced data. + // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history synchronously. + // Visibility record is deleted asynchronously. + // If call to DeleteWorkflowExecution fails, it must be retried to prevent orphan DB records. DeleteWorkflowExecution(context.Context, *DeleteWorkflowExecutionRequest) (*DeleteWorkflowExecutionResponse, error) // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. diff --git a/common/persistence/operationModeValidator.go b/common/persistence/operationModeValidator.go index c4838dfae41..1902ba03a1c 100644 --- a/common/persistence/operationModeValidator.go +++ b/common/persistence/operationModeValidator.go @@ -316,7 +316,8 @@ func checkWorkflowState(state enumsspb.WorkflowExecutionState) error { enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING, enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE, enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, - enumsspb.WORKFLOW_EXECUTION_STATE_CORRUPTED: + enumsspb.WORKFLOW_EXECUTION_STATE_CORRUPTED, + enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: return nil default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) diff --git a/common/persistence/serialization/task_serializer.go b/common/persistence/serialization/task_serializer.go index 76716aab0f4..a51a60eabb4 100644 --- a/common/persistence/serialization/task_serializer.go +++ b/common/persistence/serialization/task_serializer.go @@ -69,6 +69,8 @@ func (s *TaskSerializer) SerializeTransferTasks( transferTask = s.TransferCloseTaskToProto(task) case *tasks.ResetWorkflowTask: transferTask = s.TransferResetTaskToProto(task) + case *tasks.DeleteExecutionTask: + transferTask = s.TransferDeleteExecutionTaskToProto(task) default: return nil, serviceerror.NewInternal(fmt.Sprintf("Unknown transfer task type: %v", task)) } @@ -107,6 +109,8 @@ func (s *TaskSerializer) DeserializeTransferTasks( task = s.transferCloseTaskFromProto(transferTask) case enumsspb.TASK_TYPE_TRANSFER_RESET_WORKFLOW: task = s.transferResetTaskFromProto(transferTask) + case enumsspb.TASK_TYPE_TRANSFER_DELETE_EXECUTION: + task = s.transferDeleteExecutionTaskFromProto(transferTask) default: return nil, serviceerror.NewInternal(fmt.Sprintf("Unknown transfer task type: %v", transferTask.TaskType)) } @@ -595,6 +599,41 @@ func (s *TaskSerializer) transferResetTaskFromProto( } } +func (s *TaskSerializer) TransferDeleteExecutionTaskToProto( + deleteExecutionTask *tasks.DeleteExecutionTask, +) *persistencespb.TransferTaskInfo { + return &persistencespb.TransferTaskInfo{ + NamespaceId: deleteExecutionTask.WorkflowKey.NamespaceID, + WorkflowId: deleteExecutionTask.WorkflowKey.WorkflowID, + RunId: deleteExecutionTask.WorkflowKey.RunID, + TaskType: enumsspb.TASK_TYPE_TRANSFER_DELETE_EXECUTION, + TargetNamespaceId: "", + TargetWorkflowId: "", + TargetRunId: "", + TargetChildWorkflowOnly: false, + TaskQueue: "", + ScheduleId: 0, + Version: deleteExecutionTask.Version, + TaskId: deleteExecutionTask.TaskID, + VisibilityTime: timestamp.TimePtr(deleteExecutionTask.VisibilityTimestamp), + } +} + +func (s *TaskSerializer) transferDeleteExecutionTaskFromProto( + deleteExecutionTask *persistencespb.TransferTaskInfo, +) *tasks.DeleteExecutionTask { + return &tasks.DeleteExecutionTask{ + WorkflowKey: definition.NewWorkflowKey( + deleteExecutionTask.NamespaceId, + deleteExecutionTask.WorkflowId, + deleteExecutionTask.RunId, + ), + VisibilityTimestamp: *deleteExecutionTask.VisibilityTime, + TaskID: deleteExecutionTask.TaskId, + Version: deleteExecutionTask.Version, + } +} + func (s *TaskSerializer) TimerWorkflowTaskToProto( workflowTimer *tasks.WorkflowTaskTimeoutTask, ) *persistencespb.TimerTaskInfo { diff --git a/proto/internal/temporal/server/api/enums/v1/task.proto b/proto/internal/temporal/server/api/enums/v1/task.proto index ba37043681f..d8e6608f7c6 100644 --- a/proto/internal/temporal/server/api/enums/v1/task.proto +++ b/proto/internal/temporal/server/api/enums/v1/task.proto @@ -70,4 +70,5 @@ enum TaskType { TASK_TYPE_VISIBILITY_CLOSE_EXECUTION = 21; TASK_TYPE_VISIBILITY_DELETE_EXECUTION = 22; TASK_TYPE_TIERED_STORAGE = 23; + TASK_TYPE_TRANSFER_DELETE_EXECUTION = 24; } diff --git a/proto/internal/temporal/server/api/enums/v1/workflow.proto b/proto/internal/temporal/server/api/enums/v1/workflow.proto index d18793b6a70..40dc83e6940 100644 --- a/proto/internal/temporal/server/api/enums/v1/workflow.proto +++ b/proto/internal/temporal/server/api/enums/v1/workflow.proto @@ -32,6 +32,7 @@ enum WorkflowExecutionState { WORKFLOW_EXECUTION_STATE_ZOMBIE = 4; WORKFLOW_EXECUTION_STATE_VOID = 5; WORKFLOW_EXECUTION_STATE_CORRUPTED = 6; + WORKFLOW_EXECUTION_STATE_DELETED = 7; } enum WorkflowBackoffType { diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 841b102bcf0..b35ed426a9a 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -2253,12 +2253,36 @@ func (e *historyEngineImpl) DeleteWorkflowExecution( ctx context.Context, deleteRequest *historyservice.DeleteWorkflowExecutionRequest, ) error { - return e.workflowDeleteManager.DeleteWorkflowExecution( + + return e.updateWorkflow( ctx, namespace.ID(deleteRequest.NamespaceId), - *deleteRequest.WorkflowExecution, - false, - ) + *deleteRequest.GetWorkflowExecution(), + func(weCtx workflow.Context, mutableState workflow.MutableState) (*updateWorkflowAction, error) { + if mutableState.IsWorkflowExecutionRunning() { + return nil, consts.ErrWorkflowIsRunning // workflow is running, cannot be deleted + } + + taskGenerator := workflow.NewTaskGenerator( + e.shard.GetNamespaceRegistry(), + e.logger, + mutableState, + ) + + err := taskGenerator.GenerateDeleteExecutionTask(e.timeSource.Now()) + if err != nil { + return nil, err + } + state, status := mutableState.GetWorkflowStateStatus() + if state != enumsspb.WORKFLOW_EXECUTION_STATE_DELETED { + err = mutableState.UpdateWorkflowStateStatus(enumsspb.WORKFLOW_EXECUTION_STATE_DELETED, status) + if err != nil { + return nil, err + } + } + + return updateWorkflowWithoutWorkflowTask, nil + }) } // RecordChildExecutionCompleted records the completion of child execution into parent execution history diff --git a/service/history/tasks/delete_execution_task.go b/service/history/tasks/delete_execution_task.go new file mode 100644 index 00000000000..ef8ede91989 --- /dev/null +++ b/service/history/tasks/delete_execution_task.go @@ -0,0 +1,71 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package tasks + +import ( + "time" + + "go.temporal.io/server/common/definition" +) + +type ( + DeleteExecutionTask struct { + definition.WorkflowKey + VisibilityTimestamp time.Time + TaskID int64 + Version int64 + } +) + +func (a *DeleteExecutionTask) GetKey() Key { + return Key{ + FireTime: time.Unix(0, 0), + TaskID: a.TaskID, + } +} + +func (a *DeleteExecutionTask) GetVersion() int64 { + return a.Version +} + +func (a *DeleteExecutionTask) SetVersion(version int64) { + a.Version = version +} + +func (a *DeleteExecutionTask) GetTaskID() int64 { + return a.TaskID +} + +func (a *DeleteExecutionTask) SetTaskID(id int64) { + a.TaskID = id +} + +func (a *DeleteExecutionTask) GetVisibilityTime() time.Time { + return a.VisibilityTimestamp +} + +func (a *DeleteExecutionTask) SetVisibilityTime(timestamp time.Time) { + a.VisibilityTimestamp = timestamp +} diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index 4c82e78acaa..216ae544491 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -105,7 +105,7 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( return err } - return t.deleteManager.DeleteWorkflowExecutionFromTimerTask( + return t.deleteManager.DeleteWorkflowExecutionRetention( namespace.ID(task.GetNamespaceID()), workflowExecution, weContext, diff --git a/service/history/timerQueueTaskExecutorBase_test.go b/service/history/timerQueueTaskExecutorBase_test.go index 95a16b9c954..8160b3a676e 100644 --- a/service/history/timerQueueTaskExecutorBase_test.go +++ b/service/history/timerQueueTaskExecutorBase_test.go @@ -132,7 +132,7 @@ func (s *timerQueueTaskExecutorBaseSuite) Test_executeDeleteHistoryEventTask_NoE mockMutableState.EXPECT().GetNextEventID().Return(int64(2)) s.testShardContext.Resource.ClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(false) - s.mockDeleteManager.EXPECT().DeleteWorkflowExecutionFromTimerTask( + s.mockDeleteManager.EXPECT().DeleteWorkflowExecutionRetention( tests.NamespaceID, we, mockWeCtx, @@ -173,7 +173,7 @@ func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_DeleteFailed() { mockMutableState.EXPECT().GetNextEventID().Return(int64(2)) s.testShardContext.Resource.ClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(false) - s.mockDeleteManager.EXPECT().DeleteWorkflowExecutionFromTimerTask( + s.mockDeleteManager.EXPECT().DeleteWorkflowExecutionRetention( tests.NamespaceID, we, mockWeCtx, diff --git a/service/history/transferQueueActiveTaskExecutor.go b/service/history/transferQueueActiveTaskExecutor.go index 7c1833148b0..34f1704f254 100644 --- a/service/history/transferQueueActiveTaskExecutor.go +++ b/service/history/transferQueueActiveTaskExecutor.go @@ -84,6 +84,7 @@ func newTransferQueueActiveTaskExecutor( transferQueueTaskExecutorBase: newTransferQueueTaskExecutorBase( shard, historyEngine, + historyEngine.workflowDeleteManager, logger, metricsClient, config, @@ -125,6 +126,8 @@ func (t *transferQueueActiveTaskExecutor) execute( return t.processStartChildExecution(ctx, task) case *tasks.ResetWorkflowTask: return t.processResetWorkflow(ctx, task) + case *tasks.DeleteExecutionTask: + return t.processDeleteExecutionTask(ctx, task) default: return errUnknownTransferTask } diff --git a/service/history/transferQueueStandbyTaskExecutor.go b/service/history/transferQueueStandbyTaskExecutor.go index 6628c24b3bc..6e148937614 100644 --- a/service/history/transferQueueStandbyTaskExecutor.go +++ b/service/history/transferQueueStandbyTaskExecutor.go @@ -74,6 +74,7 @@ func newTransferQueueStandbyTaskExecutor( transferQueueTaskExecutorBase: newTransferQueueTaskExecutorBase( shard, historyEngine, + historyEngine.workflowDeleteManager, logger, metricsClient, config, @@ -122,6 +123,8 @@ func (t *transferQueueStandbyTaskExecutor) execute( return nil case *tasks.CloseExecutionTask: return t.processCloseExecution(ctx, task) + case *tasks.DeleteExecutionTask: + return t.processDeleteExecutionTask(ctx, task) default: return errUnknownTransferTask } diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index 5c7e4bf52fe..caf22149e93 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -60,12 +60,14 @@ type ( matchingClient matchingservice.MatchingServiceClient config *configs.Config searchAttributesProvider searchattribute.Provider + workflowDeleteManager workflow.DeleteManager } ) func newTransferQueueTaskExecutorBase( shard shard.Context, historyEngine *historyEngineImpl, + workflowDeleteManager workflow.DeleteManager, logger log.Logger, metricsClient metrics.Client, config *configs.Config, @@ -80,6 +82,7 @@ func newTransferQueueTaskExecutorBase( matchingClient: matchingClient, config: config, searchAttributesProvider: shard.GetSearchAttributesProvider(), + workflowDeleteManager: workflowDeleteManager, } } @@ -204,3 +207,50 @@ func (t *transferQueueTaskExecutorBase) recordWorkflowClosed( return err } + +func (t *transferQueueTaskExecutorBase) processDeleteExecutionTask( + ctx context.Context, + task *tasks.DeleteExecutionTask, +) (retError error) { + + ctx, cancel := context.WithTimeout(ctx, taskTimeout) + defer cancel() + + workflowExecution := commonpb.WorkflowExecution{ + WorkflowId: task.GetWorkflowID(), + RunId: task.GetRunID(), + } + + weCtx, release, err := t.cache.GetOrCreateWorkflowExecution( + ctx, + namespace.ID(task.GetNamespaceID()), + workflowExecution, + workflow.CallerTypeTask, + ) + if err != nil { + return err + } + defer func() { release(retError) }() + + mutableState, err := loadMutableStateForTransferTask(weCtx, task, t.metricsClient, t.logger) + if err != nil { + return err + } + + lastWriteVersion, err := mutableState.GetLastWriteVersion() + if err != nil { + return err + } + ok, err := verifyTaskVersion(t.shard, t.logger, namespace.ID(task.NamespaceID), lastWriteVersion, task.Version, task) + if err != nil || !ok { + return err + } + + return t.workflowDeleteManager.DeleteDeletedWorkflowExecution( + namespace.ID(task.GetNamespaceID()), + workflowExecution, + weCtx, + mutableState, + task.GetVersion(), + ) +} diff --git a/service/history/workflow/delete_manager.go b/service/history/workflow/delete_manager.go index 7a1535fd0a1..0fa73a3e3d4 100644 --- a/service/history/workflow/delete_manager.go +++ b/service/history/workflow/delete_manager.go @@ -33,6 +33,7 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" + enumsspb "go.temporal.io/server/api/enums/v1" "go.temporal.io/server/common" "go.temporal.io/server/common/definition" "go.temporal.io/server/common/metrics" @@ -46,8 +47,8 @@ import ( type ( DeleteManager interface { - DeleteWorkflowExecution(ctx context.Context, namespaceID namespace.ID, we commonpb.WorkflowExecution, archiveIfEnabled bool) (retError error) - DeleteWorkflowExecutionFromTimerTask(namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64) (retError error) + DeleteDeletedWorkflowExecution(namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, ms MutableState, transferTaskVersion int64) error + DeleteWorkflowExecutionRetention(namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64) error } DeleteManagerImpl struct { @@ -78,47 +79,46 @@ func NewDeleteManager( return deleteManager } -func (m *DeleteManagerImpl) DeleteWorkflowExecution( - ctx context.Context, +func (m *DeleteManagerImpl) DeleteDeletedWorkflowExecution( namespaceID namespace.ID, we commonpb.WorkflowExecution, - archiveIfEnabled bool, -) (retError error) { - - weCtx, release, err := m.historyCache.GetOrCreateWorkflowExecution( - ctx, - namespaceID, - we, - CallerTypeAPI, - ) - if err != nil { - return err - } - defer func() { release(retError) }() + weCtx Context, + ms MutableState, + transferTaskVersion int64, +) error { - mutableState, err := weCtx.LoadWorkflowExecution() - if err != nil { - return err + // Workflow execution must be marked as deleted prior to actual deletion. + if ms.GetExecutionState().GetState() != enumsspb.WORKFLOW_EXECUTION_STATE_DELETED { + return errors.New("unable to delete a workflow that is not in DELETED state") } - return m.deleteWorkflowExecutionInternal( + err := m.deleteWorkflowExecutionInternal( namespaceID, we, weCtx, - mutableState, - mutableState.GetCurrentVersion(), - archiveIfEnabled, + ms, + transferTaskVersion, + false, m.metricsClient.Scope(metrics.HistoryDeleteWorkflowExecutionScope), ) + + return err } -func (m *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( +func (m *DeleteManagerImpl) DeleteWorkflowExecutionRetention( namespaceID namespace.ID, we commonpb.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64, -) (retError error) { +) error { + + if ms.IsWorkflowExecutionRunning() { + // If workflow is running then just ignore DeleteHistoryEventTask timer task. + // This should almost never happen because DeleteHistoryEventTask is created only for closed workflows. + // But cross DC replication can resurrect workflow and therefore DeleteHistoryEventTask should be ignored. + return consts.ErrWorkflowIsRunning + } err := m.deleteWorkflowExecutionInternal( namespaceID, @@ -130,13 +130,6 @@ func (m *DeleteManagerImpl) DeleteWorkflowExecutionFromTimerTask( m.metricsClient.Scope(metrics.HistoryProcessDeleteHistoryEventScope), ) - if err != nil && errors.Is(err, consts.ErrWorkflowIsRunning) { - // If workflow is running then just ignore DeleteHistoryEventTask timer task. - // This should almost never happen because DeleteHistoryEventTask is created only for closed workflows. - // But cross DC replication can resurrect workflow and therefore DeleteHistoryEventTask should be ignored. - return nil - } - return err } @@ -150,10 +143,6 @@ func (m *DeleteManagerImpl) deleteWorkflowExecutionInternal( scope metrics.Scope, ) error { - if ms.IsWorkflowExecutionRunning() { - return consts.ErrWorkflowIsRunning - } - currentBranchToken, err := ms.GetCurrentBranchToken() if err != nil { return err diff --git a/service/history/workflow/delete_manager_mock.go b/service/history/workflow/delete_manager_mock.go index a1754390d8d..9663401e6e4 100644 --- a/service/history/workflow/delete_manager_mock.go +++ b/service/history/workflow/delete_manager_mock.go @@ -29,7 +29,6 @@ package workflow import ( - context "context" reflect "reflect" gomock "github.com/golang/mock/gomock" @@ -60,30 +59,30 @@ func (m *MockDeleteManager) EXPECT() *MockDeleteManagerMockRecorder { return m.recorder } -// DeleteWorkflowExecution mocks base method. -func (m *MockDeleteManager) DeleteWorkflowExecution(ctx context.Context, namespaceID namespace.ID, we v1.WorkflowExecution, archiveIfEnabled bool) error { +// DeleteDeletedWorkflowExecution mocks base method. +func (m *MockDeleteManager) DeleteDeletedWorkflowExecution(namespaceID namespace.ID, we v1.WorkflowExecution, weCtx Context, ms MutableState, transferTaskVersion int64) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteWorkflowExecution", ctx, namespaceID, we, archiveIfEnabled) + ret := m.ctrl.Call(m, "DeleteDeletedWorkflowExecution", namespaceID, we, weCtx, ms, transferTaskVersion) ret0, _ := ret[0].(error) return ret0 } -// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. -func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecution(ctx, namespaceID, we, archiveIfEnabled interface{}) *gomock.Call { +// DeleteDeletedWorkflowExecution indicates an expected call of DeleteDeletedWorkflowExecution. +func (mr *MockDeleteManagerMockRecorder) DeleteDeletedWorkflowExecution(namespaceID, we, weCtx, ms, transferTaskVersion interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockDeleteManager)(nil).DeleteWorkflowExecution), ctx, namespaceID, we, archiveIfEnabled) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteDeletedWorkflowExecution", reflect.TypeOf((*MockDeleteManager)(nil).DeleteDeletedWorkflowExecution), namespaceID, we, weCtx, ms, transferTaskVersion) } -// DeleteWorkflowExecutionFromTimerTask mocks base method. -func (m *MockDeleteManager) DeleteWorkflowExecutionFromTimerTask(namespaceID namespace.ID, we v1.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64) error { +// DeleteWorkflowExecutionRetention mocks base method. +func (m *MockDeleteManager) DeleteWorkflowExecutionRetention(namespaceID namespace.ID, we v1.WorkflowExecution, weCtx Context, ms MutableState, timerTaskVersion int64) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteWorkflowExecutionFromTimerTask", namespaceID, we, weCtx, ms, timerTaskVersion) + ret := m.ctrl.Call(m, "DeleteWorkflowExecutionRetention", namespaceID, we, weCtx, ms, timerTaskVersion) ret0, _ := ret[0].(error) return ret0 } -// DeleteWorkflowExecutionFromTimerTask indicates an expected call of DeleteWorkflowExecutionFromTimerTask. -func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecutionFromTimerTask(namespaceID, we, weCtx, ms, timerTaskVersion interface{}) *gomock.Call { +// DeleteWorkflowExecutionRetention indicates an expected call of DeleteWorkflowExecutionRetention. +func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecutionRetention(namespaceID, we, weCtx, ms, timerTaskVersion interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecutionFromTimerTask", reflect.TypeOf((*MockDeleteManager)(nil).DeleteWorkflowExecutionFromTimerTask), namespaceID, we, weCtx, ms, timerTaskVersion) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecutionRetention", reflect.TypeOf((*MockDeleteManager)(nil).DeleteWorkflowExecutionRetention), namespaceID, we, weCtx, ms, timerTaskVersion) } diff --git a/service/history/workflow/mutable_state_impl.go b/service/history/workflow/mutable_state_impl.go index 81cea4c8ca9..795aefa859b 100644 --- a/service/history/workflow/mutable_state_impl.go +++ b/service/history/workflow/mutable_state_impl.go @@ -1250,6 +1250,8 @@ func (e *MutableStateImpl) IsWorkflowExecutionRunning() bool { return true case enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING: return true + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + return false case enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED: return false case enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: diff --git a/service/history/workflow/mutable_state_state_status.go b/service/history/workflow/mutable_state_state_status.go index 80c1c79d4cf..fe738cf1700 100644 --- a/service/history/workflow/mutable_state_state_status.go +++ b/service/history/workflow/mutable_state_state_status.go @@ -72,6 +72,9 @@ func setStateStatus( return invalidStateTransitionErr(e.GetState(), state, status) } + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + return invalidStateTransitionErr(e.GetState(), state, status) + default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } @@ -95,6 +98,9 @@ func setStateStatus( return invalidStateTransitionErr(e.GetState(), state, status) } + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + return invalidStateTransitionErr(e.GetState(), state, status) + default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } @@ -114,6 +120,8 @@ func setStateStatus( case enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: return invalidStateTransitionErr(e.GetState(), state, status) + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } @@ -139,9 +147,25 @@ func setStateStatus( return invalidStateTransitionErr(e.GetState(), state, status) } + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + + default: + return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) + } + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + switch state { + case enumsspb.WORKFLOW_EXECUTION_STATE_CREATED, + enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING, + enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, + enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: + return invalidStateTransitionErr(e.GetState(), state, status) + + case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } + default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } diff --git a/service/history/workflow/task_generator.go b/service/history/workflow/task_generator.go index 693e77f7df6..ef1af631095 100644 --- a/service/history/workflow/task_generator.go +++ b/service/history/workflow/task_generator.go @@ -52,6 +52,9 @@ type ( GenerateWorkflowCloseTasks( now time.Time, ) error + GenerateDeleteExecutionTask( + now time.Time, + ) error GenerateRecordWorkflowStartedTasks( now time.Time, startEvent *historypb.HistoryEvent, @@ -207,6 +210,22 @@ func (r *TaskGeneratorImpl) GenerateWorkflowCloseTasks( return nil } +func (r *TaskGeneratorImpl) GenerateDeleteExecutionTask( + now time.Time, +) error { + + currentVersion := r.mutableState.GetCurrentVersion() + + r.mutableState.AddTransferTasks(&tasks.DeleteExecutionTask{ + // TaskID is set by shard + WorkflowKey: r.mutableState.GetWorkflowKey(), + VisibilityTimestamp: now, + Version: currentVersion, + }) + + return nil +} + func (r *TaskGeneratorImpl) GenerateDelayedWorkflowTasks( now time.Time, startEvent *historypb.HistoryEvent, diff --git a/service/history/workflow/task_generator_mock.go b/service/history/workflow/task_generator_mock.go index 39b47c36f23..81cb516e6bf 100644 --- a/service/history/workflow/task_generator_mock.go +++ b/service/history/workflow/task_generator_mock.go @@ -130,6 +130,20 @@ func (mr *MockTaskGeneratorMockRecorder) GenerateDelayedWorkflowTasks(now, start return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GenerateDelayedWorkflowTasks", reflect.TypeOf((*MockTaskGenerator)(nil).GenerateDelayedWorkflowTasks), now, startEvent) } +// GenerateDeleteExecutionTask mocks base method. +func (m *MockTaskGenerator) GenerateDeleteExecutionTask(now time.Time) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GenerateDeleteExecutionTask", now) + ret0, _ := ret[0].(error) + return ret0 +} + +// GenerateDeleteExecutionTask indicates an expected call of GenerateDeleteExecutionTask. +func (mr *MockTaskGeneratorMockRecorder) GenerateDeleteExecutionTask(now interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GenerateDeleteExecutionTask", reflect.TypeOf((*MockTaskGenerator)(nil).GenerateDeleteExecutionTask), now) +} + // GenerateHistoryReplicationTasks mocks base method. func (m *MockTaskGenerator) GenerateHistoryReplicationTasks(now time.Time, branchToken []byte, events []*history.HistoryEvent) error { m.ctrl.T.Helper() From 8c7c9a26f200317be1b208b89c11c738402e5b9c Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Thu, 13 Jan 2022 17:53:38 -0800 Subject: [PATCH 08/12] Fixes to cherry-pick --- common/persistence/workflowStateStatusValidator.go | 11 ++++++++++- service/history/nDCTaskUtil.go | 2 ++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/common/persistence/workflowStateStatusValidator.go b/common/persistence/workflowStateStatusValidator.go index 760ac76cd3e..942558a5072 100644 --- a/common/persistence/workflowStateStatusValidator.go +++ b/common/persistence/workflowStateStatusValidator.go @@ -40,6 +40,7 @@ var ( enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED: {}, enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: {}, enumsspb.WORKFLOW_EXECUTION_STATE_CORRUPTED: {}, + enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: {}, } validWorkflowStatuses = map[enumspb.WorkflowExecutionStatus]struct{}{ @@ -66,6 +67,10 @@ func ValidateCreateWorkflowStateStatus( return err } + if state == enumsspb.WORKFLOW_EXECUTION_STATE_DELETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING { + return nil + } + // validate workflow state & status if (state == enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status == enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) || (state != enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) { @@ -87,10 +92,14 @@ func ValidateUpdateWorkflowStateStatus( return err } + if state == enumsspb.WORKFLOW_EXECUTION_STATE_DELETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING { + return nil + } + // validate workflow state & status if (state == enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status == enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) || (state != enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) { - return serviceerror.NewInternal(fmt.Sprintf("Create workflow with invalid state: %v or status: %v", state, status)) + return serviceerror.NewInternal(fmt.Sprintf("Update workflow with invalid state: %v or status: %v", state, status)) } return nil } diff --git a/service/history/nDCTaskUtil.go b/service/history/nDCTaskUtil.go index bb1183e8c9e..8f05ee26dae 100644 --- a/service/history/nDCTaskUtil.go +++ b/service/history/nDCTaskUtil.go @@ -176,6 +176,8 @@ func getTransferTaskEventIDAndRetryable( retryable = !(executionInfo.WorkflowTaskScheduleId == task.ScheduleID && executionInfo.WorkflowTaskAttempt > 1) case *tasks.CloseExecutionTask: eventID = common.FirstEventID + case *tasks.DeleteExecutionTask: + eventID = common.FirstEventID case *tasks.CancelExecutionTask: eventID = task.InitiatedID case *tasks.SignalExecutionTask: From b6aa5ff0dbcddb35f68d12a2f3e05c2cfe76ff51 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Mon, 24 Jan 2022 17:46:35 -0800 Subject: [PATCH 09/12] Remove Deleted state --- api/enums/v1/workflow.pb.go | 28 +- api/historyservice/v1/request_response.pb.go | 1138 ++++++++++++----- common/persistence/operationModeValidator.go | 3 +- .../workflowStateStatusValidator.go | 9 - .../server/api/enums/v1/workflow.proto | 1 - service/history/historyEngine.go | 9 +- .../history/transferQueueTaskExecutorBase.go | 2 +- .../history/workflow/mutable_state_impl.go | 2 - .../workflow/mutable_state_state_status.go | 24 - 9 files changed, 806 insertions(+), 410 deletions(-) diff --git a/api/enums/v1/workflow.pb.go b/api/enums/v1/workflow.pb.go index e502c1932ef..de8803f5938 100644 --- a/api/enums/v1/workflow.pb.go +++ b/api/enums/v1/workflow.pb.go @@ -56,7 +56,6 @@ const ( WORKFLOW_EXECUTION_STATE_ZOMBIE WorkflowExecutionState = 4 WORKFLOW_EXECUTION_STATE_VOID WorkflowExecutionState = 5 WORKFLOW_EXECUTION_STATE_CORRUPTED WorkflowExecutionState = 6 - WORKFLOW_EXECUTION_STATE_DELETED WorkflowExecutionState = 7 ) var WorkflowExecutionState_name = map[int32]string{ @@ -67,7 +66,6 @@ var WorkflowExecutionState_name = map[int32]string{ 4: "Zombie", 5: "Void", 6: "Corrupted", - 7: "Deleted", } var WorkflowExecutionState_value = map[string]int32{ @@ -78,7 +76,6 @@ var WorkflowExecutionState_value = map[string]int32{ "Zombie": 4, "Void": 5, "Corrupted": 6, - "Deleted": 7, } func (WorkflowExecutionState) EnumDescriptor() ([]byte, []int) { @@ -119,12 +116,12 @@ func init() { } var fileDescriptor_004b7fefe981a755 = []byte{ - // 374 bytes of a gzipped FileDescriptorProto + // 366 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xd2, 0x2e, 0x49, 0xcd, 0x2d, 0xc8, 0x2f, 0x4a, 0xcc, 0xd1, 0x2f, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x4f, 0x2c, 0xc8, 0xd4, 0x4f, 0xcd, 0x2b, 0xcd, 0x2d, 0xd6, 0x2f, 0x33, 0xd4, 0x2f, 0xcf, 0x2f, 0xca, 0x4e, 0xcb, 0xc9, 0x2f, 0xd7, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x92, 0x81, 0x29, 0xd6, 0x83, 0x28, 0xd6, 0x4b, - 0x2c, 0xc8, 0xd4, 0x03, 0x2b, 0xd6, 0x2b, 0x33, 0xd4, 0x3a, 0xc3, 0xc4, 0x25, 0x16, 0x0e, 0xd5, + 0x2c, 0xc8, 0xd4, 0x03, 0x2b, 0xd6, 0x2b, 0x33, 0xd4, 0x5a, 0xc6, 0xc4, 0x25, 0x16, 0x0e, 0xd5, 0xe0, 0x5a, 0x91, 0x9a, 0x5c, 0x5a, 0x92, 0x99, 0x9f, 0x17, 0x5c, 0x92, 0x58, 0x92, 0x2a, 0xa4, 0xc1, 0xa5, 0x12, 0xee, 0x1f, 0xe4, 0xed, 0xe6, 0xe3, 0x1f, 0x1e, 0xef, 0x1a, 0xe1, 0xea, 0x1c, 0x1a, 0xe2, 0xe9, 0xef, 0x17, 0x1f, 0x1c, 0xe2, 0x18, 0xe2, 0x1a, 0x1f, 0xea, 0x17, 0x1c, 0xe0, @@ -133,17 +130,16 @@ var fileDescriptor_004b7fefe981a755 = []byte{ 0xee, 0x02, 0x4c, 0x42, 0x6a, 0x5c, 0x4a, 0xb8, 0xcd, 0xf2, 0xf7, 0x0d, 0xf0, 0x71, 0x05, 0x99, 0xc6, 0x2c, 0xa4, 0xcc, 0x25, 0x8f, 0x53, 0x5d, 0x94, 0xbf, 0xaf, 0x93, 0xa7, 0xab, 0x00, 0x8b, 0x90, 0x22, 0x97, 0x2c, 0x4e, 0x45, 0x61, 0xfe, 0x9e, 0x2e, 0x02, 0xac, 0x04, 0xec, 0x0b, 0x0a, - 0x0a, 0x0d, 0x00, 0xd9, 0xc7, 0x86, 0xd7, 0xf5, 0x2e, 0xae, 0x10, 0x57, 0xb1, 0x6b, 0xd5, 0x72, - 0x09, 0xc3, 0x42, 0xd3, 0x29, 0x31, 0x39, 0x3b, 0x3f, 0x2d, 0x2d, 0xa4, 0xb2, 0x20, 0x55, 0x48, - 0x95, 0x4b, 0x11, 0xae, 0xd9, 0xc9, 0xd1, 0xd9, 0xdb, 0xdf, 0xcd, 0x2d, 0x3e, 0x24, 0x32, 0x00, - 0x3d, 0x1c, 0xe5, 0xb9, 0xa4, 0xb1, 0x2b, 0x0b, 0x72, 0x0d, 0x09, 0x8a, 0x14, 0x60, 0x14, 0x92, - 0xe3, 0x92, 0xc2, 0xae, 0xc0, 0x39, 0xc8, 0xdf, 0x4f, 0x80, 0xc9, 0x29, 0xee, 0xc2, 0x43, 0x39, - 0x86, 0x1b, 0x0f, 0xe5, 0x18, 0x3e, 0x3c, 0x94, 0x63, 0x6c, 0x78, 0x24, 0xc7, 0xb8, 0xe2, 0x91, - 0x1c, 0xe3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0x3e, 0x78, 0x24, 0xc7, 0xf8, 0xe2, - 0x91, 0x1c, 0xc3, 0x87, 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, 0x5c, 0x78, 0x2c, 0xc7, 0x70, - 0xe3, 0xb1, 0x1c, 0x43, 0x94, 0x46, 0x7a, 0xbe, 0x1e, 0x3c, 0x91, 0x64, 0xe6, 0x63, 0x4b, 0x54, - 0xd6, 0x60, 0x46, 0x12, 0x1b, 0x38, 0x49, 0x19, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0x74, 0x1c, - 0x53, 0x64, 0x81, 0x02, 0x00, 0x00, + 0x0a, 0x0d, 0x00, 0xd9, 0xc7, 0xa6, 0x55, 0xcb, 0x25, 0x0c, 0x0b, 0x27, 0xa7, 0xc4, 0xe4, 0xec, + 0xfc, 0xb4, 0xb4, 0x90, 0xca, 0x82, 0x54, 0x21, 0x55, 0x2e, 0x45, 0xb8, 0x76, 0x27, 0x47, 0x67, + 0x6f, 0x7f, 0x37, 0xb7, 0xf8, 0x90, 0xc8, 0x00, 0xf4, 0x10, 0x92, 0xe7, 0x92, 0xc6, 0xae, 0x2c, + 0xc8, 0x35, 0x24, 0x28, 0x52, 0x80, 0x51, 0x48, 0x8e, 0x4b, 0x0a, 0xbb, 0x02, 0xe7, 0x20, 0x7f, + 0x3f, 0x01, 0x26, 0xa7, 0xb8, 0x0b, 0x0f, 0xe5, 0x18, 0x6e, 0x3c, 0x94, 0x63, 0xf8, 0xf0, 0x50, + 0x8e, 0xb1, 0xe1, 0x91, 0x1c, 0xe3, 0x8a, 0x47, 0x72, 0x8c, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x78, + 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x8b, 0x47, 0x72, 0x0c, 0x1f, 0x1e, 0xc9, 0x31, 0x4e, + 0x78, 0x2c, 0xc7, 0x70, 0xe1, 0xb1, 0x1c, 0xc3, 0x8d, 0xc7, 0x72, 0x0c, 0x51, 0x1a, 0xe9, 0xf9, + 0x7a, 0xf0, 0xe8, 0xcf, 0xcc, 0xc7, 0x96, 0x5c, 0xac, 0xc1, 0x8c, 0x24, 0x36, 0x70, 0x62, 0x31, + 0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0xfd, 0x0f, 0x05, 0x23, 0x5b, 0x02, 0x00, 0x00, } func (x WorkflowExecutionState) String() string { diff --git a/api/historyservice/v1/request_response.pb.go b/api/historyservice/v1/request_response.pb.go index c19985a84e3..06727f4b22f 100644 --- a/api/historyservice/v1/request_response.pb.go +++ b/api/historyservice/v1/request_response.pb.go @@ -2059,6 +2059,92 @@ func (m *TerminateWorkflowExecutionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_TerminateWorkflowExecutionResponse proto.InternalMessageInfo +type DeleteWorkflowExecutionRequest struct { + NamespaceId string `protobuf:"bytes,1,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v14.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` +} + +func (m *DeleteWorkflowExecutionRequest) Reset() { *m = DeleteWorkflowExecutionRequest{} } +func (*DeleteWorkflowExecutionRequest) ProtoMessage() {} +func (*DeleteWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_b8c78c1d460a3711, []int{32} +} +func (m *DeleteWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteWorkflowExecutionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_DeleteWorkflowExecutionRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *DeleteWorkflowExecutionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteWorkflowExecutionRequest.Merge(m, src) +} +func (m *DeleteWorkflowExecutionRequest) XXX_Size() int { + return m.Size() +} +func (m *DeleteWorkflowExecutionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteWorkflowExecutionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DeleteWorkflowExecutionRequest proto.InternalMessageInfo + +func (m *DeleteWorkflowExecutionRequest) GetNamespaceId() string { + if m != nil { + return m.NamespaceId + } + return "" +} + +func (m *DeleteWorkflowExecutionRequest) GetWorkflowExecution() *v14.WorkflowExecution { + if m != nil { + return m.WorkflowExecution + } + return nil +} + +type DeleteWorkflowExecutionResponse struct { +} + +func (m *DeleteWorkflowExecutionResponse) Reset() { *m = DeleteWorkflowExecutionResponse{} } +func (*DeleteWorkflowExecutionResponse) ProtoMessage() {} +func (*DeleteWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_b8c78c1d460a3711, []int{33} +} +func (m *DeleteWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteWorkflowExecutionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_DeleteWorkflowExecutionResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *DeleteWorkflowExecutionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteWorkflowExecutionResponse.Merge(m, src) +} +func (m *DeleteWorkflowExecutionResponse) XXX_Size() int { + return m.Size() +} +func (m *DeleteWorkflowExecutionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteWorkflowExecutionResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_DeleteWorkflowExecutionResponse proto.InternalMessageInfo + type ResetWorkflowExecutionRequest struct { NamespaceId string `protobuf:"bytes,1,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` ResetRequest *v1.ResetWorkflowExecutionRequest `protobuf:"bytes,2,opt,name=reset_request,json=resetRequest,proto3" json:"reset_request,omitempty"` @@ -2067,7 +2153,7 @@ type ResetWorkflowExecutionRequest struct { func (m *ResetWorkflowExecutionRequest) Reset() { *m = ResetWorkflowExecutionRequest{} } func (*ResetWorkflowExecutionRequest) ProtoMessage() {} func (*ResetWorkflowExecutionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{32} + return fileDescriptor_b8c78c1d460a3711, []int{34} } func (m *ResetWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2117,7 +2203,7 @@ type ResetWorkflowExecutionResponse struct { func (m *ResetWorkflowExecutionResponse) Reset() { *m = ResetWorkflowExecutionResponse{} } func (*ResetWorkflowExecutionResponse) ProtoMessage() {} func (*ResetWorkflowExecutionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{33} + return fileDescriptor_b8c78c1d460a3711, []int{35} } func (m *ResetWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2164,7 +2250,7 @@ type RequestCancelWorkflowExecutionRequest struct { func (m *RequestCancelWorkflowExecutionRequest) Reset() { *m = RequestCancelWorkflowExecutionRequest{} } func (*RequestCancelWorkflowExecutionRequest) ProtoMessage() {} func (*RequestCancelWorkflowExecutionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{34} + return fileDescriptor_b8c78c1d460a3711, []int{36} } func (m *RequestCancelWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2236,7 +2322,7 @@ func (m *RequestCancelWorkflowExecutionResponse) Reset() { } func (*RequestCancelWorkflowExecutionResponse) ProtoMessage() {} func (*RequestCancelWorkflowExecutionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{35} + return fileDescriptor_b8c78c1d460a3711, []int{37} } func (m *RequestCancelWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2274,7 +2360,7 @@ type ScheduleWorkflowTaskRequest struct { func (m *ScheduleWorkflowTaskRequest) Reset() { *m = ScheduleWorkflowTaskRequest{} } func (*ScheduleWorkflowTaskRequest) ProtoMessage() {} func (*ScheduleWorkflowTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{36} + return fileDescriptor_b8c78c1d460a3711, []int{38} } func (m *ScheduleWorkflowTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2330,7 +2416,7 @@ type ScheduleWorkflowTaskResponse struct { func (m *ScheduleWorkflowTaskResponse) Reset() { *m = ScheduleWorkflowTaskResponse{} } func (*ScheduleWorkflowTaskResponse) ProtoMessage() {} func (*ScheduleWorkflowTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{37} + return fileDescriptor_b8c78c1d460a3711, []int{39} } func (m *ScheduleWorkflowTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2376,7 +2462,7 @@ type RecordChildExecutionCompletedRequest struct { func (m *RecordChildExecutionCompletedRequest) Reset() { *m = RecordChildExecutionCompletedRequest{} } func (*RecordChildExecutionCompletedRequest) ProtoMessage() {} func (*RecordChildExecutionCompletedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{38} + return fileDescriptor_b8c78c1d460a3711, []int{40} } func (m *RecordChildExecutionCompletedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2446,7 +2532,7 @@ type RecordChildExecutionCompletedResponse struct { func (m *RecordChildExecutionCompletedResponse) Reset() { *m = RecordChildExecutionCompletedResponse{} } func (*RecordChildExecutionCompletedResponse) ProtoMessage() {} func (*RecordChildExecutionCompletedResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{39} + return fileDescriptor_b8c78c1d460a3711, []int{41} } func (m *RecordChildExecutionCompletedResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2483,7 +2569,7 @@ type DescribeWorkflowExecutionRequest struct { func (m *DescribeWorkflowExecutionRequest) Reset() { *m = DescribeWorkflowExecutionRequest{} } func (*DescribeWorkflowExecutionRequest) ProtoMessage() {} func (*DescribeWorkflowExecutionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{40} + return fileDescriptor_b8c78c1d460a3711, []int{42} } func (m *DescribeWorkflowExecutionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2536,7 +2622,7 @@ type DescribeWorkflowExecutionResponse struct { func (m *DescribeWorkflowExecutionResponse) Reset() { *m = DescribeWorkflowExecutionResponse{} } func (*DescribeWorkflowExecutionResponse) ProtoMessage() {} func (*DescribeWorkflowExecutionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{41} + return fileDescriptor_b8c78c1d460a3711, []int{43} } func (m *DescribeWorkflowExecutionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2605,7 +2691,7 @@ type ReplicateEventsV2Request struct { func (m *ReplicateEventsV2Request) Reset() { *m = ReplicateEventsV2Request{} } func (*ReplicateEventsV2Request) ProtoMessage() {} func (*ReplicateEventsV2Request) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{42} + return fileDescriptor_b8c78c1d460a3711, []int{44} } func (m *ReplicateEventsV2Request) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2675,7 +2761,7 @@ type ReplicateEventsV2Response struct { func (m *ReplicateEventsV2Response) Reset() { *m = ReplicateEventsV2Response{} } func (*ReplicateEventsV2Response) ProtoMessage() {} func (*ReplicateEventsV2Response) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{43} + return fileDescriptor_b8c78c1d460a3711, []int{45} } func (m *ReplicateEventsV2Response) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2713,7 +2799,7 @@ type SyncShardStatusRequest struct { func (m *SyncShardStatusRequest) Reset() { *m = SyncShardStatusRequest{} } func (*SyncShardStatusRequest) ProtoMessage() {} func (*SyncShardStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{44} + return fileDescriptor_b8c78c1d460a3711, []int{46} } func (m *SyncShardStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2769,7 +2855,7 @@ type SyncShardStatusResponse struct { func (m *SyncShardStatusResponse) Reset() { *m = SyncShardStatusResponse{} } func (*SyncShardStatusResponse) ProtoMessage() {} func (*SyncShardStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{45} + return fileDescriptor_b8c78c1d460a3711, []int{47} } func (m *SyncShardStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2818,7 +2904,7 @@ type SyncActivityRequest struct { func (m *SyncActivityRequest) Reset() { *m = SyncActivityRequest{} } func (*SyncActivityRequest) ProtoMessage() {} func (*SyncActivityRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{46} + return fileDescriptor_b8c78c1d460a3711, []int{48} } func (m *SyncActivityRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2951,7 +3037,7 @@ type SyncActivityResponse struct { func (m *SyncActivityResponse) Reset() { *m = SyncActivityResponse{} } func (*SyncActivityResponse) ProtoMessage() {} func (*SyncActivityResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{47} + return fileDescriptor_b8c78c1d460a3711, []int{49} } func (m *SyncActivityResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2988,7 +3074,7 @@ type DescribeMutableStateRequest struct { func (m *DescribeMutableStateRequest) Reset() { *m = DescribeMutableStateRequest{} } func (*DescribeMutableStateRequest) ProtoMessage() {} func (*DescribeMutableStateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{48} + return fileDescriptor_b8c78c1d460a3711, []int{50} } func (m *DescribeMutableStateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3039,7 +3125,7 @@ type DescribeMutableStateResponse struct { func (m *DescribeMutableStateResponse) Reset() { *m = DescribeMutableStateResponse{} } func (*DescribeMutableStateResponse) ProtoMessage() {} func (*DescribeMutableStateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{49} + return fileDescriptor_b8c78c1d460a3711, []int{51} } func (m *DescribeMutableStateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3094,7 +3180,7 @@ type DescribeHistoryHostRequest struct { func (m *DescribeHistoryHostRequest) Reset() { *m = DescribeHistoryHostRequest{} } func (*DescribeHistoryHostRequest) ProtoMessage() {} func (*DescribeHistoryHostRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{50} + return fileDescriptor_b8c78c1d460a3711, []int{52} } func (m *DescribeHistoryHostRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3162,7 +3248,7 @@ type DescribeHistoryHostResponse struct { func (m *DescribeHistoryHostResponse) Reset() { *m = DescribeHistoryHostResponse{} } func (*DescribeHistoryHostResponse) ProtoMessage() {} func (*DescribeHistoryHostResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{51} + return fileDescriptor_b8c78c1d460a3711, []int{53} } func (m *DescribeHistoryHostResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3233,7 +3319,7 @@ type CloseShardRequest struct { func (m *CloseShardRequest) Reset() { *m = CloseShardRequest{} } func (*CloseShardRequest) ProtoMessage() {} func (*CloseShardRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{52} + return fileDescriptor_b8c78c1d460a3711, []int{54} } func (m *CloseShardRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3275,7 +3361,7 @@ type CloseShardResponse struct { func (m *CloseShardResponse) Reset() { *m = CloseShardResponse{} } func (*CloseShardResponse) ProtoMessage() {} func (*CloseShardResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{53} + return fileDescriptor_b8c78c1d460a3711, []int{55} } func (m *CloseShardResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3311,7 +3397,7 @@ type GetShardRequest struct { func (m *GetShardRequest) Reset() { *m = GetShardRequest{} } func (*GetShardRequest) ProtoMessage() {} func (*GetShardRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{54} + return fileDescriptor_b8c78c1d460a3711, []int{56} } func (m *GetShardRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3354,7 +3440,7 @@ type GetShardResponse struct { func (m *GetShardResponse) Reset() { *m = GetShardResponse{} } func (*GetShardResponse) ProtoMessage() {} func (*GetShardResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{55} + return fileDescriptor_b8c78c1d460a3711, []int{57} } func (m *GetShardResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3400,7 +3486,7 @@ type RemoveTaskRequest struct { func (m *RemoveTaskRequest) Reset() { *m = RemoveTaskRequest{} } func (*RemoveTaskRequest) ProtoMessage() {} func (*RemoveTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{56} + return fileDescriptor_b8c78c1d460a3711, []int{58} } func (m *RemoveTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3463,7 +3549,7 @@ type RemoveTaskResponse struct { func (m *RemoveTaskResponse) Reset() { *m = RemoveTaskResponse{} } func (*RemoveTaskResponse) ProtoMessage() {} func (*RemoveTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{57} + return fileDescriptor_b8c78c1d460a3711, []int{59} } func (m *RemoveTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3500,7 +3586,7 @@ type GetReplicationMessagesRequest struct { func (m *GetReplicationMessagesRequest) Reset() { *m = GetReplicationMessagesRequest{} } func (*GetReplicationMessagesRequest) ProtoMessage() {} func (*GetReplicationMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{58} + return fileDescriptor_b8c78c1d460a3711, []int{60} } func (m *GetReplicationMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3550,7 +3636,7 @@ type GetReplicationMessagesResponse struct { func (m *GetReplicationMessagesResponse) Reset() { *m = GetReplicationMessagesResponse{} } func (*GetReplicationMessagesResponse) ProtoMessage() {} func (*GetReplicationMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{59} + return fileDescriptor_b8c78c1d460a3711, []int{61} } func (m *GetReplicationMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3593,7 +3679,7 @@ type GetDLQReplicationMessagesRequest struct { func (m *GetDLQReplicationMessagesRequest) Reset() { *m = GetDLQReplicationMessagesRequest{} } func (*GetDLQReplicationMessagesRequest) ProtoMessage() {} func (*GetDLQReplicationMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{60} + return fileDescriptor_b8c78c1d460a3711, []int{62} } func (m *GetDLQReplicationMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3636,7 +3722,7 @@ type GetDLQReplicationMessagesResponse struct { func (m *GetDLQReplicationMessagesResponse) Reset() { *m = GetDLQReplicationMessagesResponse{} } func (*GetDLQReplicationMessagesResponse) ProtoMessage() {} func (*GetDLQReplicationMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{61} + return fileDescriptor_b8c78c1d460a3711, []int{63} } func (m *GetDLQReplicationMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3680,7 +3766,7 @@ type QueryWorkflowRequest struct { func (m *QueryWorkflowRequest) Reset() { *m = QueryWorkflowRequest{} } func (*QueryWorkflowRequest) ProtoMessage() {} func (*QueryWorkflowRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{62} + return fileDescriptor_b8c78c1d460a3711, []int{64} } func (m *QueryWorkflowRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3730,7 +3816,7 @@ type QueryWorkflowResponse struct { func (m *QueryWorkflowResponse) Reset() { *m = QueryWorkflowResponse{} } func (*QueryWorkflowResponse) ProtoMessage() {} func (*QueryWorkflowResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{63} + return fileDescriptor_b8c78c1d460a3711, []int{65} } func (m *QueryWorkflowResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3774,7 +3860,7 @@ type ReapplyEventsRequest struct { func (m *ReapplyEventsRequest) Reset() { *m = ReapplyEventsRequest{} } func (*ReapplyEventsRequest) ProtoMessage() {} func (*ReapplyEventsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{64} + return fileDescriptor_b8c78c1d460a3711, []int{66} } func (m *ReapplyEventsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3823,7 +3909,7 @@ type ReapplyEventsResponse struct { func (m *ReapplyEventsResponse) Reset() { *m = ReapplyEventsResponse{} } func (*ReapplyEventsResponse) ProtoMessage() {} func (*ReapplyEventsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{65} + return fileDescriptor_b8c78c1d460a3711, []int{67} } func (m *ReapplyEventsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3864,7 +3950,7 @@ type GetDLQMessagesRequest struct { func (m *GetDLQMessagesRequest) Reset() { *m = GetDLQMessagesRequest{} } func (*GetDLQMessagesRequest) ProtoMessage() {} func (*GetDLQMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{66} + return fileDescriptor_b8c78c1d460a3711, []int{68} } func (m *GetDLQMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3944,7 +4030,7 @@ type GetDLQMessagesResponse struct { func (m *GetDLQMessagesResponse) Reset() { *m = GetDLQMessagesResponse{} } func (*GetDLQMessagesResponse) ProtoMessage() {} func (*GetDLQMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{67} + return fileDescriptor_b8c78c1d460a3711, []int{69} } func (m *GetDLQMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4004,7 +4090,7 @@ type PurgeDLQMessagesRequest struct { func (m *PurgeDLQMessagesRequest) Reset() { *m = PurgeDLQMessagesRequest{} } func (*PurgeDLQMessagesRequest) ProtoMessage() {} func (*PurgeDLQMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{68} + return fileDescriptor_b8c78c1d460a3711, []int{70} } func (m *PurgeDLQMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4067,7 +4153,7 @@ type PurgeDLQMessagesResponse struct { func (m *PurgeDLQMessagesResponse) Reset() { *m = PurgeDLQMessagesResponse{} } func (*PurgeDLQMessagesResponse) ProtoMessage() {} func (*PurgeDLQMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{69} + return fileDescriptor_b8c78c1d460a3711, []int{71} } func (m *PurgeDLQMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4108,7 +4194,7 @@ type MergeDLQMessagesRequest struct { func (m *MergeDLQMessagesRequest) Reset() { *m = MergeDLQMessagesRequest{} } func (*MergeDLQMessagesRequest) ProtoMessage() {} func (*MergeDLQMessagesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{70} + return fileDescriptor_b8c78c1d460a3711, []int{72} } func (m *MergeDLQMessagesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4186,7 +4272,7 @@ type MergeDLQMessagesResponse struct { func (m *MergeDLQMessagesResponse) Reset() { *m = MergeDLQMessagesResponse{} } func (*MergeDLQMessagesResponse) ProtoMessage() {} func (*MergeDLQMessagesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{71} + return fileDescriptor_b8c78c1d460a3711, []int{73} } func (m *MergeDLQMessagesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4230,7 +4316,7 @@ type RefreshWorkflowTasksRequest struct { func (m *RefreshWorkflowTasksRequest) Reset() { *m = RefreshWorkflowTasksRequest{} } func (*RefreshWorkflowTasksRequest) ProtoMessage() {} func (*RefreshWorkflowTasksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{72} + return fileDescriptor_b8c78c1d460a3711, []int{74} } func (m *RefreshWorkflowTasksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4279,7 +4365,7 @@ type RefreshWorkflowTasksResponse struct { func (m *RefreshWorkflowTasksResponse) Reset() { *m = RefreshWorkflowTasksResponse{} } func (*RefreshWorkflowTasksResponse) ProtoMessage() {} func (*RefreshWorkflowTasksResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{73} + return fileDescriptor_b8c78c1d460a3711, []int{75} } func (m *RefreshWorkflowTasksResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4318,7 +4404,7 @@ func (m *GenerateLastHistoryReplicationTasksRequest) Reset() { } func (*GenerateLastHistoryReplicationTasksRequest) ProtoMessage() {} func (*GenerateLastHistoryReplicationTasksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{74} + return fileDescriptor_b8c78c1d460a3711, []int{76} } func (m *GenerateLastHistoryReplicationTasksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4369,7 +4455,7 @@ func (m *GenerateLastHistoryReplicationTasksResponse) Reset() { } func (*GenerateLastHistoryReplicationTasksResponse) ProtoMessage() {} func (*GenerateLastHistoryReplicationTasksResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{75} + return fileDescriptor_b8c78c1d460a3711, []int{77} } func (m *GenerateLastHistoryReplicationTasksResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4406,7 +4492,7 @@ type GetReplicationStatusRequest struct { func (m *GetReplicationStatusRequest) Reset() { *m = GetReplicationStatusRequest{} } func (*GetReplicationStatusRequest) ProtoMessage() {} func (*GetReplicationStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{76} + return fileDescriptor_b8c78c1d460a3711, []int{78} } func (m *GetReplicationStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4449,7 +4535,7 @@ type GetReplicationStatusResponse struct { func (m *GetReplicationStatusResponse) Reset() { *m = GetReplicationStatusResponse{} } func (*GetReplicationStatusResponse) ProtoMessage() {} func (*GetReplicationStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{77} + return fileDescriptor_b8c78c1d460a3711, []int{79} } func (m *GetReplicationStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4498,7 +4584,7 @@ type ShardReplicationStatus struct { func (m *ShardReplicationStatus) Reset() { *m = ShardReplicationStatus{} } func (*ShardReplicationStatus) ProtoMessage() {} func (*ShardReplicationStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{78} + return fileDescriptor_b8c78c1d460a3711, []int{80} } func (m *ShardReplicationStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4570,7 +4656,7 @@ type HandoverNamespaceInfo struct { func (m *HandoverNamespaceInfo) Reset() { *m = HandoverNamespaceInfo{} } func (*HandoverNamespaceInfo) ProtoMessage() {} func (*HandoverNamespaceInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{79} + return fileDescriptor_b8c78c1d460a3711, []int{81} } func (m *HandoverNamespaceInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4616,7 +4702,7 @@ type ShardReplicationStatusPerCluster struct { func (m *ShardReplicationStatusPerCluster) Reset() { *m = ShardReplicationStatusPerCluster{} } func (*ShardReplicationStatusPerCluster) ProtoMessage() {} func (*ShardReplicationStatusPerCluster) Descriptor() ([]byte, []int) { - return fileDescriptor_b8c78c1d460a3711, []int{80} + return fileDescriptor_b8c78c1d460a3711, []int{82} } func (m *ShardReplicationStatusPerCluster) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4693,6 +4779,8 @@ func init() { proto.RegisterType((*RemoveSignalMutableStateResponse)(nil), "temporal.server.api.historyservice.v1.RemoveSignalMutableStateResponse") proto.RegisterType((*TerminateWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.TerminateWorkflowExecutionRequest") proto.RegisterType((*TerminateWorkflowExecutionResponse)(nil), "temporal.server.api.historyservice.v1.TerminateWorkflowExecutionResponse") + proto.RegisterType((*DeleteWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.DeleteWorkflowExecutionRequest") + proto.RegisterType((*DeleteWorkflowExecutionResponse)(nil), "temporal.server.api.historyservice.v1.DeleteWorkflowExecutionResponse") proto.RegisterType((*ResetWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.ResetWorkflowExecutionRequest") proto.RegisterType((*ResetWorkflowExecutionResponse)(nil), "temporal.server.api.historyservice.v1.ResetWorkflowExecutionResponse") proto.RegisterType((*RequestCancelWorkflowExecutionRequest)(nil), "temporal.server.api.historyservice.v1.RequestCancelWorkflowExecutionRequest") @@ -4752,260 +4840,261 @@ func init() { } var fileDescriptor_b8c78c1d460a3711 = []byte{ - // 4036 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x3c, 0x4b, 0x6c, 0x1c, 0x47, - 0x76, 0xea, 0x19, 0x0e, 0x39, 0xf3, 0x66, 0x38, 0x9f, 0xe6, 0x6f, 0x48, 0x4a, 0x23, 0xb2, 0x6d, - 0x59, 0xf4, 0x47, 0x43, 0x4b, 0xda, 0xb5, 0xbd, 0xca, 0x7a, 0x1d, 0x89, 0x92, 0xa8, 0x11, 0x24, - 0x2d, 0xdd, 0xe4, 0xca, 0x86, 0x77, 0xbd, 0xed, 0xe6, 0x74, 0x91, 0xd3, 0xe1, 0x4c, 0xf7, 0xb8, - 0xab, 0x67, 0xc8, 0x71, 0x0e, 0xf9, 0x2c, 0x12, 0x60, 0x37, 0x40, 0x60, 0x20, 0x97, 0x05, 0xb2, - 0xb9, 0x04, 0x08, 0xb2, 0x08, 0x10, 0xe4, 0x90, 0xd3, 0x1e, 0x72, 0x0d, 0x72, 0x4a, 0x8c, 0x00, - 0x41, 0x16, 0x9b, 0x43, 0x62, 0x19, 0x01, 0x12, 0x24, 0x87, 0x3d, 0xe4, 0x90, 0x63, 0x50, 0xbf, - 0x9e, 0xfe, 0xcd, 0x8f, 0x94, 0xa3, 0x5d, 0xaf, 0x6f, 0xec, 0xaa, 0xf7, 0x5e, 0xd5, 0xfb, 0xd6, - 0xab, 0x57, 0x6f, 0x08, 0x5f, 0x77, 0x51, 0xab, 0x6d, 0x3b, 0x7a, 0x73, 0x13, 0x23, 0xa7, 0x8b, - 0x9c, 0x4d, 0xbd, 0x6d, 0x6e, 0x36, 0x4c, 0xec, 0xda, 0x4e, 0x8f, 0x8c, 0x98, 0x75, 0xb4, 0xd9, - 0xbd, 0xba, 0xe9, 0xa0, 0x0f, 0x3b, 0x08, 0xbb, 0x9a, 0x83, 0x70, 0xdb, 0xb6, 0x30, 0xaa, 0xb6, - 0x1d, 0xdb, 0xb5, 0xe5, 0x4b, 0x02, 0xbb, 0xca, 0xb0, 0xab, 0x7a, 0xdb, 0xac, 0x06, 0xb1, 0xab, - 0xdd, 0xab, 0x2b, 0x95, 0x43, 0xdb, 0x3e, 0x6c, 0xa2, 0x4d, 0x8a, 0xb4, 0xdf, 0x39, 0xd8, 0x34, - 0x3a, 0x8e, 0xee, 0x9a, 0xb6, 0xc5, 0xc8, 0xac, 0x5c, 0x0c, 0xcf, 0xbb, 0x66, 0x0b, 0x61, 0x57, - 0x6f, 0xb5, 0x39, 0xc0, 0xba, 0x81, 0xda, 0xc8, 0x32, 0x90, 0x55, 0x37, 0x11, 0xde, 0x3c, 0xb4, - 0x0f, 0x6d, 0x3a, 0x4e, 0xff, 0xe2, 0x20, 0xcf, 0x7b, 0x8c, 0x10, 0x0e, 0xea, 0x76, 0xab, 0x65, - 0x5b, 0x64, 0xe7, 0x2d, 0x84, 0xb1, 0x7e, 0xc8, 0x37, 0xbc, 0x72, 0x29, 0x00, 0xc5, 0x77, 0x1a, - 0x05, 0xbb, 0x1c, 0x00, 0x73, 0x75, 0x7c, 0xf4, 0x61, 0x07, 0x75, 0x50, 0x14, 0x30, 0xb8, 0x2a, - 0xb2, 0x3a, 0x2d, 0x4c, 0x80, 0x8e, 0x6d, 0xe7, 0xe8, 0xa0, 0x69, 0x1f, 0x73, 0xa8, 0x17, 0x02, - 0x50, 0x62, 0x32, 0x4a, 0xed, 0xb9, 0x00, 0xdc, 0x87, 0x1d, 0x14, 0xb7, 0xb7, 0x20, 0x0b, 0x07, - 0xba, 0xd9, 0xec, 0x38, 0x31, 0x3b, 0x7b, 0x65, 0x88, 0x62, 0xa3, 0xd0, 0x2f, 0xc6, 0x41, 0x7b, - 0xec, 0x30, 0x69, 0x72, 0xd0, 0x97, 0x87, 0x82, 0x86, 0x38, 0xbf, 0x3c, 0x14, 0x98, 0x08, 0x96, - 0x03, 0x5e, 0x89, 0x03, 0x1c, 0x2c, 0xa9, 0x6a, 0x1c, 0xb8, 0xa5, 0xb7, 0x10, 0x6e, 0xeb, 0xf5, - 0x18, 0x69, 0xbc, 0x1a, 0x07, 0xef, 0xa0, 0x76, 0xd3, 0xac, 0x53, 0x43, 0x8c, 0x62, 0x5c, 0x8f, - 0xc3, 0x68, 0x23, 0x07, 0x9b, 0xd8, 0x45, 0x16, 0x5b, 0x03, 0x9d, 0xa0, 0x7a, 0x87, 0xa0, 0x63, - 0x8e, 0xf4, 0xd6, 0x18, 0x48, 0x82, 0x29, 0xad, 0xd5, 0x71, 0xf5, 0xfd, 0x26, 0xd2, 0xb0, 0xab, - 0xbb, 0x62, 0xd5, 0xd7, 0x62, 0x2d, 0x65, 0xa4, 0x23, 0xae, 0xdc, 0x88, 0x5b, 0x58, 0x37, 0x5a, - 0xa6, 0x35, 0x12, 0x57, 0xf9, 0x83, 0x69, 0xb8, 0xb0, 0xeb, 0xea, 0x8e, 0xfb, 0x0e, 0x5f, 0xee, - 0x8e, 0x60, 0x4b, 0x65, 0x08, 0xf2, 0x3a, 0xe4, 0x3c, 0xd9, 0x6a, 0xa6, 0x51, 0x96, 0xd6, 0xa4, - 0x8d, 0x8c, 0x9a, 0xf5, 0xc6, 0x6a, 0x86, 0x5c, 0x87, 0x59, 0x4c, 0x68, 0x68, 0x7c, 0x91, 0x72, - 0x62, 0x4d, 0xda, 0xc8, 0x5e, 0xfb, 0x86, 0xa7, 0x28, 0x1a, 0x1a, 0x42, 0x0c, 0x55, 0xbb, 0x57, - 0xab, 0x43, 0x57, 0x56, 0x73, 0x94, 0xa8, 0xd8, 0x47, 0x03, 0x16, 0xda, 0xba, 0x83, 0x2c, 0x57, - 0xf3, 0x24, 0xaf, 0x99, 0xd6, 0x81, 0x5d, 0x4e, 0xd2, 0xc5, 0xbe, 0x52, 0x8d, 0x0b, 0x47, 0x9e, - 0x45, 0x76, 0xaf, 0x56, 0x77, 0x28, 0xb6, 0xb7, 0x4a, 0xcd, 0x3a, 0xb0, 0xd5, 0xb9, 0x76, 0x74, - 0x50, 0x2e, 0xc3, 0x8c, 0xee, 0x12, 0x6a, 0x6e, 0x79, 0x6a, 0x4d, 0xda, 0x48, 0xa9, 0xe2, 0x53, - 0x6e, 0x81, 0xe2, 0x69, 0xb0, 0xbf, 0x0b, 0x74, 0xd2, 0x36, 0x59, 0x48, 0xd3, 0x48, 0xec, 0x2a, - 0xa7, 0xe8, 0x86, 0x56, 0xaa, 0x2c, 0xb0, 0x55, 0x45, 0x60, 0xab, 0xee, 0x89, 0xc0, 0x76, 0x6b, - 0xea, 0xe3, 0x7f, 0xbd, 0x28, 0xa9, 0x17, 0x8f, 0xc3, 0x9c, 0xdf, 0xf1, 0x28, 0x11, 0x58, 0xb9, - 0x01, 0xcb, 0x75, 0xdb, 0x72, 0x4d, 0xab, 0x83, 0x34, 0x1d, 0x6b, 0x16, 0x3a, 0xd6, 0x4c, 0xcb, - 0x74, 0x4d, 0xdd, 0xb5, 0x9d, 0xf2, 0xf4, 0x9a, 0xb4, 0x91, 0xbf, 0x76, 0x25, 0x28, 0x63, 0xea, - 0x5d, 0x84, 0xd9, 0x2d, 0x8e, 0x77, 0x13, 0x3f, 0x42, 0xc7, 0x35, 0x81, 0xa4, 0x2e, 0xd6, 0x63, - 0xc7, 0xe5, 0x87, 0x50, 0x12, 0x33, 0x86, 0xc6, 0xc3, 0x4a, 0x79, 0x86, 0xf2, 0xb1, 0x16, 0x5c, - 0x81, 0x4f, 0x92, 0x35, 0xee, 0xb2, 0x3f, 0xd5, 0xa2, 0x87, 0xca, 0x47, 0xe4, 0xc7, 0xb0, 0xd8, - 0xd4, 0xb1, 0xab, 0xd5, 0xed, 0x56, 0xbb, 0x89, 0xa8, 0x64, 0x1c, 0x84, 0x3b, 0x4d, 0xb7, 0x9c, - 0x8e, 0xa3, 0xc9, 0x43, 0x0c, 0xd5, 0x51, 0xaf, 0x69, 0xeb, 0x06, 0x56, 0xe7, 0x09, 0xfe, 0x96, - 0x87, 0xae, 0x52, 0x6c, 0xf9, 0xbb, 0xb0, 0x7a, 0x60, 0x3a, 0xd8, 0xd5, 0x3c, 0x2d, 0x90, 0x28, - 0xa2, 0xed, 0xeb, 0xf5, 0x23, 0xfb, 0xe0, 0xa0, 0x9c, 0xa1, 0xc4, 0x97, 0x23, 0x82, 0xbf, 0xcd, - 0x4f, 0x9c, 0x5b, 0x53, 0x3f, 0x24, 0x72, 0x2f, 0x53, 0x1a, 0xc2, 0xec, 0xf6, 0x74, 0x7c, 0x74, - 0x8b, 0x11, 0x50, 0x5e, 0x87, 0xca, 0x20, 0x93, 0x64, 0x5e, 0x23, 0x2f, 0xc0, 0xb4, 0xd3, 0xb1, - 0xfa, 0x7e, 0x90, 0x72, 0x3a, 0x56, 0xcd, 0x50, 0xfe, 0x4b, 0x82, 0xc5, 0x6d, 0xe4, 0x3e, 0x64, - 0x5e, 0xbd, 0x4b, 0x9c, 0x7a, 0x02, 0xff, 0xd9, 0x86, 0x8c, 0x67, 0x4d, 0xdc, 0x77, 0x5e, 0x1c, - 0x24, 0xa1, 0xe8, 0xd6, 0xfa, 0xb8, 0xf2, 0x75, 0x58, 0x44, 0x27, 0x6d, 0x54, 0x77, 0x91, 0xa1, - 0x59, 0xe8, 0xc4, 0xd5, 0x50, 0x97, 0x38, 0x8c, 0x69, 0x50, 0x27, 0x49, 0xaa, 0x73, 0x62, 0xf6, - 0x11, 0x3a, 0x71, 0xef, 0x90, 0xb9, 0x9a, 0x21, 0xbf, 0x0a, 0xf3, 0xf5, 0x8e, 0x43, 0x3d, 0x6b, - 0xdf, 0xd1, 0xad, 0x7a, 0x43, 0x73, 0xed, 0x23, 0x64, 0x51, 0xdb, 0xcf, 0xa9, 0x32, 0x9f, 0xbb, - 0x45, 0xa7, 0xf6, 0xc8, 0x8c, 0xf2, 0x17, 0x69, 0x58, 0x8a, 0x70, 0xcb, 0x05, 0x14, 0xe0, 0x45, - 0x3a, 0x03, 0x2f, 0x35, 0x98, 0xed, 0x6b, 0xb9, 0xd7, 0x46, 0x5c, 0x30, 0xcf, 0x8f, 0x22, 0xb6, - 0xd7, 0x6b, 0x23, 0x35, 0x77, 0xec, 0xfb, 0x92, 0x15, 0x98, 0x8d, 0x93, 0x46, 0xd6, 0xf2, 0x49, - 0xe1, 0x6b, 0xb0, 0xdc, 0x76, 0x50, 0xd7, 0xb4, 0x3b, 0x58, 0xa3, 0x71, 0x07, 0x19, 0x7d, 0xf8, - 0x29, 0x0a, 0xbf, 0x28, 0x00, 0x76, 0xd9, 0xbc, 0x40, 0xbd, 0x02, 0x73, 0xd4, 0xda, 0x99, 0x69, - 0x7a, 0x48, 0x29, 0x8a, 0x54, 0x24, 0x53, 0x77, 0xc9, 0x8c, 0x00, 0xdf, 0x02, 0xa0, 0x56, 0x4b, - 0xb3, 0x0a, 0xea, 0xc6, 0x11, 0xae, 0xbc, 0xa4, 0x83, 0x30, 0x46, 0x0c, 0xf4, 0x6d, 0xf2, 0xa1, - 0x66, 0x5c, 0xf1, 0xa7, 0xbc, 0x03, 0x25, 0xec, 0x9a, 0xf5, 0xa3, 0x9e, 0xe6, 0xa3, 0x35, 0x33, - 0x01, 0xad, 0x02, 0x43, 0xf7, 0x06, 0xe4, 0xdf, 0x84, 0x97, 0x23, 0x14, 0x35, 0x5c, 0x6f, 0x20, - 0xa3, 0xd3, 0x44, 0x9a, 0x6b, 0x33, 0xa9, 0xd0, 0x08, 0x67, 0x77, 0xdc, 0x72, 0x76, 0x3c, 0x5f, - 0xbb, 0x14, 0x5a, 0x66, 0x97, 0x13, 0xdc, 0xb3, 0xa9, 0x10, 0xf7, 0x18, 0xb5, 0x81, 0x36, 0x38, - 0x3b, 0xc8, 0x06, 0xe5, 0x6f, 0x43, 0xde, 0x33, 0x0f, 0x7a, 0x88, 0x96, 0x0b, 0x34, 0x20, 0xc6, - 0x9f, 0x03, 0x5e, 0x5c, 0x8c, 0x98, 0x1c, 0xb3, 0x5e, 0xcf, 0xd4, 0xe8, 0xa7, 0xfc, 0x0e, 0x14, - 0x02, 0xc4, 0x3b, 0xb8, 0x5c, 0xa4, 0xd4, 0xab, 0x03, 0xc2, 0x6d, 0x2c, 0xd9, 0x0e, 0x56, 0xf3, - 0x7e, 0xba, 0x1d, 0x2c, 0xbf, 0x0f, 0xa5, 0x2e, 0xc9, 0x08, 0x6c, 0x4b, 0x63, 0xe9, 0x98, 0x89, - 0x70, 0xb9, 0x44, 0x45, 0xf9, 0x6a, 0x75, 0x48, 0x3e, 0x4d, 0xd6, 0x78, 0xcc, 0x10, 0xef, 0x09, - 0x3c, 0xb5, 0xd8, 0x0d, 0x8d, 0xc8, 0xdf, 0x80, 0xf3, 0x26, 0x31, 0xdf, 0xb0, 0x1a, 0x91, 0x45, - 0x1c, 0xd5, 0x28, 0xcb, 0x6b, 0xd2, 0x46, 0x5a, 0x2d, 0x9b, 0x78, 0x37, 0xa8, 0x95, 0x3b, 0x6c, - 0x5e, 0xfe, 0x0a, 0x2c, 0x45, 0x2c, 0xd9, 0x3d, 0xa1, 0xe1, 0x6e, 0x8e, 0x05, 0x90, 0xa0, 0x35, - 0xef, 0x9d, 0x58, 0x35, 0xe3, 0xfe, 0x54, 0x3a, 0x5d, 0xcc, 0xdc, 0x9f, 0x4a, 0x67, 0x8a, 0x70, - 0x7f, 0x2a, 0x0d, 0xc5, 0xec, 0xfd, 0xa9, 0x74, 0xae, 0x38, 0x7b, 0x7f, 0x2a, 0x9d, 0x2f, 0x16, - 0x94, 0xff, 0x96, 0x60, 0x69, 0xc7, 0x6e, 0x36, 0x7f, 0x45, 0x62, 0xe3, 0xbf, 0xcf, 0x40, 0x39, - 0xca, 0xee, 0x97, 0xc1, 0xf1, 0xcb, 0xe0, 0xf8, 0xd4, 0x83, 0x63, 0x6e, 0x60, 0x70, 0x8c, 0x0d, - 0x33, 0xf9, 0xa7, 0x16, 0x66, 0x7e, 0x39, 0x63, 0xef, 0x90, 0xe0, 0x56, 0x9a, 0x2c, 0xb8, 0xcd, - 0x16, 0xf3, 0xca, 0xf7, 0x25, 0x58, 0x55, 0x11, 0x46, 0x6e, 0x28, 0x94, 0x3e, 0x83, 0xd0, 0xa6, - 0x54, 0xe0, 0x7c, 0xfc, 0x56, 0x58, 0xd8, 0x51, 0x7e, 0x96, 0x80, 0x35, 0x15, 0xd5, 0x6d, 0xc7, - 0xf0, 0x27, 0xbd, 0xdc, 0x51, 0x27, 0xd8, 0xf0, 0xbb, 0x20, 0x47, 0xaf, 0x3f, 0x93, 0xef, 0xbc, - 0x14, 0xb9, 0xf7, 0xc8, 0x17, 0x21, 0xeb, 0x79, 0x93, 0x17, 0x82, 0x40, 0x0c, 0xd5, 0x0c, 0x79, - 0x09, 0x66, 0xa8, 0xe7, 0x79, 0xf1, 0x66, 0x9a, 0x7c, 0xd6, 0x0c, 0xf9, 0x02, 0x80, 0xb8, 0xda, - 0xf2, 0xb0, 0x92, 0x51, 0x33, 0x7c, 0xa4, 0x66, 0xc8, 0x1f, 0x40, 0xae, 0x6d, 0x37, 0x9b, 0xde, - 0xcd, 0x94, 0x45, 0x94, 0x37, 0x47, 0xde, 0x4c, 0x49, 0x08, 0xf7, 0x0b, 0xcb, 0xaf, 0x5b, 0x35, - 0x4b, 0x48, 0xf2, 0x0f, 0xe5, 0x9f, 0x66, 0x60, 0x7d, 0x88, 0x70, 0x79, 0xe4, 0x8f, 0x04, 0x6c, - 0xe9, 0xd4, 0x01, 0x7b, 0x68, 0x30, 0x4e, 0x0c, 0x0d, 0xc6, 0xaf, 0x80, 0x2c, 0x64, 0x6a, 0x84, - 0x03, 0x7e, 0xd1, 0x9b, 0x11, 0xd0, 0x1b, 0x50, 0x1c, 0x10, 0xec, 0xf3, 0x38, 0x48, 0x37, 0x72, - 0x86, 0xa4, 0xa2, 0x67, 0x88, 0xef, 0x56, 0x3d, 0x1d, 0xbc, 0x55, 0xbf, 0x01, 0x65, 0x1e, 0x5c, - 0x7d, 0x77, 0x6a, 0x9e, 0xb1, 0xcc, 0xd0, 0x8c, 0x65, 0x91, 0xcd, 0xf7, 0xef, 0xc9, 0x3c, 0x5f, - 0x39, 0xf4, 0x19, 0x24, 0x33, 0x0f, 0xeb, 0xc0, 0xe6, 0x77, 0xcc, 0xaf, 0x8d, 0x0a, 0x74, 0x7b, - 0x8e, 0x6e, 0x61, 0x13, 0x59, 0x81, 0x9b, 0x20, 0xad, 0x0a, 0x14, 0x8f, 0x43, 0x23, 0xf2, 0x21, - 0x5c, 0x88, 0xb9, 0xf8, 0xfb, 0x4e, 0x97, 0xcc, 0x04, 0xa7, 0xcb, 0x4a, 0xc4, 0xfe, 0xfb, 0x07, - 0xcd, 0x3a, 0xe4, 0x02, 0x31, 0x3e, 0x4b, 0x63, 0x7c, 0x76, 0xdf, 0x17, 0xdc, 0xb7, 0x21, 0xdf, - 0x57, 0x22, 0x2d, 0x38, 0xe4, 0xc6, 0x2c, 0x38, 0xcc, 0x7a, 0x78, 0xb4, 0xbc, 0xb0, 0x05, 0x39, - 0xa1, 0x5f, 0x4a, 0x66, 0x76, 0x4c, 0x32, 0x59, 0x8e, 0x45, 0x89, 0xd8, 0x30, 0xf3, 0x61, 0x07, - 0xf1, 0x03, 0x26, 0xb9, 0x91, 0xbd, 0xf6, 0xad, 0xea, 0x58, 0x75, 0xe1, 0xea, 0x48, 0x9f, 0xa9, - 0xbe, 0xcd, 0xe8, 0xde, 0xb1, 0x5c, 0xa7, 0xa7, 0x8a, 0x55, 0x56, 0x3e, 0x80, 0x9c, 0x7f, 0x42, - 0x2e, 0x42, 0xf2, 0x08, 0xf5, 0x78, 0xb8, 0x22, 0x7f, 0xca, 0x37, 0x20, 0xd5, 0xd5, 0x9b, 0x9d, - 0x01, 0x49, 0x11, 0xad, 0xac, 0xfa, 0x5d, 0x8c, 0x50, 0xeb, 0xa9, 0x0c, 0xe5, 0x46, 0xe2, 0x0d, - 0x89, 0x85, 0x79, 0x5f, 0xd0, 0xbc, 0x59, 0x77, 0xcd, 0xae, 0xe9, 0xf6, 0xbe, 0x0c, 0x9a, 0x63, - 0x04, 0x4d, 0xbf, 0xb0, 0x06, 0x07, 0xcd, 0xdf, 0x9d, 0x12, 0x41, 0x33, 0x56, 0xb8, 0x3c, 0x68, - 0x3e, 0x82, 0x42, 0x28, 0x5c, 0xf1, 0xb0, 0x79, 0x29, 0xb8, 0x15, 0x9f, 0x53, 0xb3, 0x24, 0xa5, - 0x47, 0x83, 0x8e, 0x9a, 0x0f, 0x86, 0xb4, 0x88, 0xc1, 0x27, 0x4e, 0x63, 0xf0, 0xbe, 0x38, 0x96, - 0x0c, 0xc6, 0x31, 0x04, 0x15, 0x91, 0xa7, 0xf1, 0x21, 0x2d, 0xe4, 0xa8, 0x53, 0x63, 0x2e, 0xb8, - 0xca, 0xe9, 0xdc, 0x64, 0x64, 0x76, 0x03, 0x6e, 0xfb, 0x10, 0x4a, 0x0d, 0xa4, 0x3b, 0xee, 0x3e, - 0xd2, 0x5d, 0xcd, 0x40, 0xae, 0x6e, 0x36, 0x31, 0xaf, 0x39, 0x8e, 0xae, 0xab, 0x15, 0x3d, 0xd4, - 0xdb, 0x0c, 0x33, 0x7a, 0x32, 0x4d, 0x9f, 0xfa, 0x64, 0xba, 0xe2, 0x33, 0x75, 0xcf, 0x05, 0x68, - 0x08, 0xcf, 0xf4, 0xed, 0xf7, 0x91, 0x98, 0x50, 0x7e, 0x22, 0xc1, 0x73, 0x4c, 0xd7, 0x81, 0x30, - 0xc0, 0xab, 0x7e, 0x13, 0x39, 0x99, 0x0d, 0x45, 0x5e, 0x6b, 0x44, 0xa1, 0x22, 0xf4, 0xed, 0x91, - 0x56, 0x3b, 0xc6, 0x16, 0xd4, 0x82, 0xa0, 0x2e, 0x0c, 0xf8, 0x8f, 0x25, 0x78, 0x7e, 0x38, 0x22, - 0xb7, 0x61, 0xdc, 0x3f, 0x44, 0x45, 0xe9, 0x9d, 0x1b, 0xf1, 0xbd, 0xa7, 0x15, 0x28, 0xc9, 0x75, - 0x25, 0x30, 0xa0, 0xfc, 0x95, 0x44, 0x62, 0x57, 0x64, 0x77, 0x77, 0x75, 0xb3, 0x39, 0x91, 0x58, - 0x1b, 0x90, 0x3f, 0xa0, 0x38, 0x21, 0xa1, 0xde, 0x3c, 0x8d, 0x50, 0x03, 0xab, 0xab, 0xb3, 0x07, - 0xfe, 0x4f, 0xe5, 0x39, 0x12, 0x0f, 0x06, 0xa2, 0x70, 0xb6, 0x7e, 0x22, 0x81, 0x12, 0x8d, 0x1a, - 0xf7, 0x84, 0x45, 0x4f, 0xc0, 0x58, 0xdb, 0xef, 0x43, 0x41, 0xde, 0xb6, 0xc6, 0xe0, 0x6d, 0xd4, - 0x16, 0x7c, 0x6e, 0x26, 0x18, 0xdc, 0x21, 0xb6, 0x3e, 0x04, 0x8f, 0x9b, 0xcb, 0x8b, 0x50, 0xac, - 0xeb, 0x56, 0x1d, 0x79, 0xc1, 0x17, 0xb1, 0xfd, 0xa7, 0xd5, 0x02, 0x1b, 0x57, 0xc5, 0xb0, 0xdf, - 0x7d, 0xfc, 0x34, 0x9f, 0x91, 0xfb, 0x0c, 0xdb, 0x42, 0xd4, 0x7d, 0x5e, 0xf0, 0xbc, 0x67, 0x00, - 0x5e, 0xd4, 0x90, 0xfd, 0x80, 0xff, 0xff, 0x86, 0x3c, 0x70, 0xf5, 0xc1, 0x86, 0x1c, 0x87, 0xc2, - 0xd9, 0xfa, 0x6b, 0x6a, 0xc8, 0x51, 0xfe, 0xa9, 0x86, 0x27, 0x62, 0xec, 0x37, 0x20, 0x1f, 0xb4, - 0x97, 0x09, 0xac, 0x78, 0xd4, 0xfa, 0xea, 0x6c, 0xc0, 0xe4, 0x94, 0x4b, 0xf1, 0xf6, 0xe6, 0x21, - 0x71, 0xe6, 0xfe, 0x36, 0x01, 0x95, 0x5d, 0xf3, 0xd0, 0xd2, 0x9b, 0x67, 0x79, 0x53, 0x3c, 0x80, - 0x3c, 0xa6, 0x44, 0x42, 0x8c, 0xbd, 0x35, 0xfa, 0x51, 0x71, 0xe8, 0xda, 0xea, 0x2c, 0x23, 0x2b, - 0xb6, 0x62, 0xc2, 0x2a, 0x3a, 0x71, 0x91, 0x43, 0x56, 0x8a, 0xc9, 0xd3, 0x92, 0x93, 0xe6, 0x69, - 0xcb, 0x82, 0x5a, 0x64, 0x4a, 0xae, 0xc2, 0x5c, 0xbd, 0x61, 0x36, 0x8d, 0xfe, 0x3a, 0xb6, 0xd5, - 0xec, 0xd1, 0xa4, 0x20, 0xad, 0x96, 0xe8, 0x94, 0x40, 0xfa, 0xa6, 0xd5, 0xec, 0x29, 0xeb, 0x70, - 0x71, 0x20, 0x2f, 0x5c, 0xd6, 0xff, 0x28, 0xc1, 0x65, 0x0e, 0x63, 0xba, 0x8d, 0x33, 0x3f, 0xe4, - 0x7e, 0x4f, 0x82, 0x65, 0x2e, 0xf5, 0x63, 0xd3, 0x6d, 0x68, 0x71, 0xaf, 0xba, 0xf7, 0xc6, 0x55, - 0xc0, 0xa8, 0x0d, 0xa9, 0x8b, 0x38, 0x08, 0x28, 0xec, 0xec, 0x26, 0x6c, 0x8c, 0x26, 0x31, 0xfc, - 0x3d, 0xee, 0x6f, 0x24, 0xb8, 0xa8, 0xa2, 0x96, 0xdd, 0x45, 0x8c, 0xd2, 0x29, 0x8b, 0xcf, 0x9f, - 0x5f, 0xee, 0x1e, 0xcc, 0xc0, 0x93, 0xa1, 0x0c, 0x5c, 0x51, 0x48, 0xd8, 0x1b, 0xb4, 0x7d, 0xa1, - 0xfb, 0x04, 0xac, 0xef, 0x21, 0xa7, 0x65, 0x5a, 0xba, 0x8b, 0xce, 0xa2, 0x75, 0x1b, 0x4a, 0xae, - 0xa0, 0x13, 0x52, 0xf6, 0xad, 0x91, 0xca, 0x1e, 0xb9, 0x03, 0xb5, 0xe8, 0x11, 0xff, 0x25, 0xf0, - 0xb9, 0xe7, 0x41, 0x19, 0xc6, 0x11, 0x17, 0xfd, 0x9f, 0x4b, 0x70, 0x81, 0x56, 0xdc, 0xce, 0xd8, - 0x35, 0xe1, 0x10, 0x1a, 0x13, 0x77, 0x4d, 0x0c, 0x5d, 0x59, 0xcd, 0x51, 0xa2, 0xc2, 0x97, 0x5e, - 0x87, 0xca, 0x20, 0xf0, 0xe1, 0x1e, 0xf4, 0x47, 0x49, 0xb8, 0xc4, 0x89, 0xb0, 0x08, 0x7f, 0x16, - 0x56, 0x5b, 0x03, 0x4e, 0xa9, 0xbb, 0x63, 0xf0, 0x3a, 0xc6, 0x16, 0x42, 0x07, 0x95, 0xfc, 0xa6, - 0xcf, 0xbe, 0x78, 0xc3, 0x44, 0xb4, 0xde, 0x55, 0x16, 0x20, 0x35, 0x01, 0x21, 0x2a, 0x55, 0x23, - 0xcc, 0x73, 0xea, 0xf3, 0x37, 0xcf, 0xd4, 0x20, 0xf3, 0xdc, 0x80, 0x17, 0x46, 0x49, 0x84, 0x9b, - 0xe8, 0x3f, 0x48, 0xb0, 0x2a, 0xee, 0x8d, 0xfe, 0x94, 0xfa, 0x17, 0x22, 0xfa, 0x5d, 0x87, 0x45, - 0x13, 0x6b, 0x31, 0xad, 0x1c, 0x54, 0x37, 0x69, 0x75, 0xce, 0xc4, 0x77, 0xc3, 0x3d, 0x1a, 0x4a, - 0x05, 0xce, 0xc7, 0x33, 0xc4, 0x39, 0xfe, 0x9f, 0x04, 0x49, 0x2a, 0x49, 0x8a, 0xbd, 0x45, 0xe4, - 0xe6, 0xad, 0x76, 0x9a, 0x84, 0xf8, 0xf3, 0x63, 0x7d, 0x1d, 0x72, 0x7d, 0x93, 0xec, 0xbf, 0xb6, - 0x79, 0x63, 0x35, 0x43, 0x7e, 0x0f, 0xe6, 0x44, 0xbe, 0x6c, 0x9c, 0xc5, 0xee, 0x64, 0x8f, 0x4a, - 0x7f, 0xf9, 0x1d, 0x2f, 0xd3, 0xa7, 0x55, 0x56, 0x5a, 0x53, 0x49, 0x4d, 0x52, 0x53, 0x29, 0xf4, - 0xd1, 0xe9, 0x80, 0x72, 0x99, 0xc4, 0x89, 0xa1, 0x52, 0xe7, 0xfa, 0xf9, 0x53, 0x09, 0xd6, 0x6e, - 0x23, 0x5c, 0x77, 0xcc, 0xfd, 0x33, 0x1d, 0x57, 0xdf, 0x86, 0x99, 0x49, 0x93, 0xf8, 0x51, 0xcb, - 0xaa, 0x82, 0xa2, 0xf2, 0xe3, 0x24, 0xac, 0x0f, 0x81, 0xe6, 0x31, 0xf3, 0x3b, 0x50, 0xec, 0x57, - 0x81, 0xeb, 0xb6, 0x75, 0x60, 0x1e, 0xf2, 0x4b, 0xfd, 0xd5, 0xf8, 0xbd, 0xc4, 0x2a, 0x68, 0x8b, - 0x22, 0xaa, 0x05, 0x14, 0x1c, 0x90, 0x0f, 0x61, 0x29, 0xa6, 0xd8, 0x4c, 0x4b, 0xdb, 0x8c, 0xe1, - 0xcd, 0x09, 0x16, 0xa1, 0x05, 0xed, 0x85, 0xe3, 0xb8, 0x61, 0xf9, 0x3b, 0x20, 0xb7, 0x91, 0x65, - 0x98, 0xd6, 0xa1, 0xa6, 0xb3, 0x8c, 0xde, 0x44, 0xb8, 0x9c, 0xa4, 0x65, 0xdc, 0x2b, 0x83, 0xd7, - 0xd8, 0x61, 0x38, 0xe2, 0x12, 0x40, 0x57, 0x28, 0xb5, 0x03, 0x83, 0x26, 0xc2, 0xf2, 0x77, 0xa1, - 0x28, 0xa8, 0xd3, 0x40, 0xe6, 0xd0, 0x77, 0x73, 0x42, 0xfb, 0xfa, 0x48, 0xda, 0x41, 0x5b, 0xa2, - 0x2b, 0x14, 0xda, 0xbe, 0x29, 0x07, 0x59, 0xca, 0xef, 0x24, 0xa1, 0xac, 0xf2, 0x2e, 0x4e, 0x44, - 0x6d, 0x11, 0x3f, 0xbe, 0xf6, 0x0b, 0xe1, 0xe3, 0x07, 0xb0, 0x10, 0x7c, 0x7e, 0xed, 0x69, 0xa6, - 0x8b, 0x5a, 0x42, 0xb4, 0xd7, 0x26, 0x7a, 0x82, 0xed, 0xd5, 0x5c, 0xd4, 0x52, 0xe7, 0xba, 0x91, - 0x31, 0x2c, 0xbf, 0x01, 0xd3, 0xd4, 0x83, 0x31, 0x8f, 0x0d, 0x03, 0xcb, 0x7f, 0xb7, 0x75, 0x57, - 0xbf, 0xd5, 0xb4, 0xf7, 0x55, 0x0e, 0x2f, 0xdf, 0x85, 0xbc, 0x85, 0x8e, 0x35, 0x72, 0xf0, 0x73, - 0x0a, 0xa9, 0x31, 0x29, 0xe4, 0x2c, 0x74, 0xac, 0x76, 0x98, 0xef, 0x63, 0x65, 0x15, 0x96, 0x63, - 0x54, 0xc0, 0x1d, 0xfe, 0x4f, 0x24, 0x58, 0xdc, 0xed, 0x59, 0xf5, 0xdd, 0x86, 0xee, 0x18, 0xfc, - 0x51, 0x96, 0xab, 0xe7, 0x12, 0xe4, 0xb1, 0xdd, 0x71, 0xea, 0x48, 0xab, 0x37, 0x3b, 0xd8, 0x45, - 0x0e, 0x57, 0xd0, 0x2c, 0x1b, 0xdd, 0x62, 0x83, 0xf2, 0x32, 0xa4, 0x31, 0x41, 0x16, 0x2f, 0x5b, - 0x29, 0x75, 0x86, 0x7e, 0xd7, 0x0c, 0xf9, 0x26, 0x64, 0xd9, 0xeb, 0x30, 0xab, 0xac, 0x26, 0xc7, - 0xac, 0xac, 0x02, 0x43, 0x22, 0xc3, 0xca, 0x32, 0x2c, 0x45, 0xb6, 0x27, 0x72, 0xeb, 0x14, 0xcc, - 0x91, 0x39, 0x61, 0xe3, 0x13, 0x98, 0xd5, 0x45, 0xc8, 0x7a, 0x66, 0xc5, 0xb7, 0x9d, 0x51, 0x41, - 0x0c, 0xd5, 0x0c, 0x5f, 0xc2, 0x95, 0xf4, 0x25, 0x5c, 0x72, 0x19, 0x66, 0xb8, 0x8e, 0x79, 0xb1, - 0x5e, 0x7c, 0x92, 0x45, 0xfb, 0x75, 0xe4, 0xfe, 0xe3, 0x9a, 0x37, 0x46, 0x9f, 0x92, 0xc3, 0x6f, - 0x42, 0xd3, 0xa7, 0x7b, 0x13, 0xba, 0x00, 0x20, 0xca, 0x95, 0x26, 0x7b, 0x7d, 0x4b, 0xaa, 0x19, - 0x3e, 0x42, 0xdb, 0x33, 0x82, 0x15, 0xf4, 0xf4, 0x69, 0x2a, 0xe8, 0x3b, 0xbc, 0x25, 0xa4, 0x5f, - 0x81, 0xa3, 0xb4, 0x32, 0x63, 0xd2, 0x2a, 0x11, 0x64, 0xaf, 0x72, 0x46, 0x29, 0xde, 0x80, 0x19, - 0x51, 0x08, 0x87, 0x31, 0x0b, 0xe1, 0x02, 0xc1, 0x5f, 0xcf, 0xcf, 0x06, 0xeb, 0xf9, 0x5b, 0x90, - 0x63, 0x0d, 0x03, 0xbc, 0x1f, 0x36, 0x37, 0x66, 0x3f, 0x6c, 0x96, 0xf6, 0x11, 0xf0, 0x56, 0xd8, - 0x57, 0x81, 0xb6, 0xb2, 0xd2, 0x34, 0x07, 0x39, 0x9a, 0x69, 0x20, 0xcb, 0x35, 0xdd, 0x1e, 0x7d, - 0x6c, 0xcb, 0xa8, 0x32, 0x99, 0x7b, 0x87, 0x4e, 0xd5, 0xf8, 0x8c, 0xfc, 0x0e, 0x14, 0x42, 0xd1, - 0x83, 0xb7, 0x6e, 0x54, 0x27, 0x8b, 0x1b, 0x6a, 0x3e, 0x18, 0x33, 0x94, 0x45, 0x98, 0x0f, 0xda, - 0x34, 0x37, 0xf6, 0xef, 0x4b, 0xb0, 0x2a, 0xce, 0xbc, 0x67, 0xdc, 0xa5, 0xa5, 0xfc, 0xaf, 0x04, - 0xe7, 0xe3, 0xf7, 0xc2, 0x8f, 0xde, 0x06, 0xcc, 0xd5, 0xf5, 0x7a, 0x03, 0x05, 0x3b, 0xe8, 0xf9, - 0xe9, 0xfb, 0x46, 0xac, 0x84, 0x7c, 0x3d, 0xf8, 0xfe, 0xf5, 0x03, 0xe4, 0x4b, 0x94, 0xa8, 0x7f, - 0x48, 0xb6, 0x60, 0xd1, 0xd0, 0x5d, 0x7d, 0x5f, 0xc7, 0xe1, 0xc5, 0x12, 0x67, 0x5c, 0x6c, 0x5e, - 0xd0, 0xf5, 0x8f, 0x2a, 0xff, 0x2c, 0xc1, 0x8a, 0x60, 0x9d, 0xab, 0xec, 0x9e, 0x8d, 0xfd, 0x55, - 0xed, 0x86, 0x8d, 0x5d, 0x4d, 0x37, 0x0c, 0x07, 0x61, 0x2c, 0xb4, 0x40, 0xc6, 0x6e, 0xb2, 0xa1, - 0x61, 0xe1, 0x32, 0xac, 0xc3, 0xe4, 0xb8, 0xe7, 0xe1, 0xd4, 0xd9, 0xcf, 0x43, 0xe5, 0xe3, 0x44, - 0xdf, 0xc0, 0x02, 0x9c, 0x71, 0x9d, 0x3e, 0x07, 0xb3, 0x74, 0x9f, 0x58, 0xb3, 0x3a, 0xad, 0x7d, - 0x7e, 0x18, 0xa4, 0xd4, 0x1c, 0x1b, 0x7c, 0x44, 0xc7, 0xe4, 0x55, 0xc8, 0x08, 0xe6, 0x70, 0x39, - 0xb1, 0x96, 0xdc, 0x48, 0xa9, 0x69, 0xce, 0x1d, 0x96, 0xdf, 0x87, 0x42, 0x9f, 0x3d, 0xaa, 0xca, - 0xa1, 0x3f, 0x0b, 0xf0, 0x60, 0x09, 0x0b, 0xde, 0x83, 0xd4, 0x16, 0xc1, 0xa3, 0xb9, 0x46, 0xde, - 0x0a, 0x8c, 0xc9, 0xaf, 0xc1, 0x12, 0x5b, 0xbb, 0x6e, 0x5b, 0xae, 0x63, 0x37, 0x9b, 0xc8, 0x11, - 0xbd, 0x49, 0x53, 0x54, 0x90, 0x0b, 0x74, 0x7a, 0xcb, 0x9b, 0xe5, 0x2d, 0x47, 0x24, 0xb6, 0x70, - 0x75, 0xb1, 0x47, 0x56, 0xf1, 0xa9, 0x54, 0xa1, 0xb4, 0xd5, 0xb4, 0x31, 0xa2, 0x87, 0x8f, 0x50, - 0xb1, 0x5f, 0x7f, 0x52, 0x40, 0x7f, 0xca, 0x3c, 0xc8, 0x7e, 0x78, 0xee, 0xb9, 0xaf, 0x40, 0x61, - 0x1b, 0xb9, 0xe3, 0xd2, 0xf8, 0x00, 0x8a, 0x7d, 0x68, 0x2e, 0xfa, 0x07, 0x00, 0x1c, 0x9c, 0xa4, - 0x97, 0xcc, 0x8b, 0xae, 0x8c, 0x63, 0xd8, 0x94, 0x0c, 0x15, 0x16, 0x53, 0x0b, 0xf9, 0x53, 0xf9, - 0x99, 0x04, 0x25, 0x56, 0xb7, 0xf2, 0x5f, 0x35, 0x07, 0x6f, 0x49, 0xbe, 0x0b, 0x69, 0x92, 0x3a, - 0x1c, 0x92, 0x20, 0x97, 0xa0, 0x5d, 0x5e, 0x2f, 0x0d, 0xef, 0x21, 0x63, 0x15, 0x67, 0x86, 0xa1, - 0x7a, 0xb8, 0xfe, 0x97, 0xee, 0x64, 0xe0, 0xa5, 0xbb, 0x06, 0x85, 0xae, 0x89, 0xcd, 0x7d, 0xb3, - 0x69, 0xba, 0xbd, 0xc9, 0x1e, 0x61, 0xf3, 0x7d, 0x44, 0x9a, 0x2e, 0xcc, 0x83, 0xec, 0xe7, 0x8d, - 0xab, 0xe0, 0x63, 0x09, 0x2e, 0x6c, 0x23, 0x57, 0xed, 0xff, 0x9c, 0xe8, 0x21, 0xfb, 0x29, 0x91, - 0x97, 0xeb, 0x3c, 0x80, 0x69, 0xda, 0xcb, 0x41, 0x5c, 0x36, 0x39, 0xd0, 0x24, 0x7d, 0xbf, 0x47, - 0x62, 0x75, 0x0f, 0xef, 0x93, 0x76, 0x7d, 0xa8, 0x9c, 0x06, 0x71, 0x64, 0x9e, 0x32, 0xd1, 0x27, - 0x56, 0x9e, 0x5f, 0x64, 0xf9, 0x18, 0xb1, 0x65, 0xe5, 0x47, 0x09, 0xa8, 0x0c, 0xda, 0x12, 0x57, - 0xfb, 0x6f, 0x41, 0x9e, 0xa9, 0x84, 0xff, 0xee, 0x49, 0xec, 0xed, 0xdd, 0x31, 0xdf, 0x24, 0x87, - 0x93, 0x67, 0xc6, 0x21, 0x46, 0x59, 0xff, 0x06, 0xf3, 0x70, 0x31, 0xb6, 0xd2, 0x03, 0x39, 0x0a, - 0xe4, 0xef, 0xe5, 0x48, 0xb1, 0x5e, 0x8e, 0x87, 0xc1, 0x5e, 0x8e, 0xd7, 0x27, 0x94, 0x9d, 0xb7, - 0xb3, 0x7e, 0x7b, 0x87, 0xf2, 0x11, 0xac, 0x6d, 0x23, 0xf7, 0xf6, 0x83, 0xb7, 0x87, 0xe8, 0xec, - 0x31, 0x6f, 0x43, 0x25, 0x5e, 0x21, 0x64, 0x33, 0xe9, 0xda, 0x5e, 0x3b, 0x11, 0xed, 0x4c, 0x25, - 0x7f, 0x61, 0xe5, 0xf7, 0x24, 0x58, 0x1f, 0xb2, 0x38, 0xd7, 0xce, 0x07, 0x50, 0xf2, 0x91, 0xa5, - 0x85, 0x11, 0xb1, 0x89, 0xeb, 0xa7, 0xd8, 0x84, 0x5a, 0x74, 0x82, 0x03, 0x58, 0xf9, 0x81, 0x04, - 0xf3, 0xb4, 0xef, 0x45, 0xc4, 0xef, 0x09, 0xce, 0xfa, 0x6f, 0x86, 0xef, 0xdf, 0x5f, 0x1d, 0x79, - 0xff, 0x8e, 0x5b, 0xaa, 0x7f, 0xe7, 0x3e, 0x82, 0x85, 0x10, 0x00, 0x97, 0x83, 0x0a, 0xe9, 0xd0, - 0x9b, 0xf9, 0x6b, 0x93, 0x2e, 0xc5, 0x5f, 0xc8, 0x3d, 0x3a, 0xca, 0x1f, 0x4a, 0x30, 0xaf, 0x22, - 0xbd, 0xdd, 0x6e, 0xb2, 0x82, 0x06, 0x9e, 0x80, 0xf3, 0xdd, 0x30, 0xe7, 0xf1, 0x3d, 0x66, 0xfe, - 0x9f, 0xde, 0x31, 0x75, 0x44, 0x97, 0xeb, 0x73, 0xbf, 0x04, 0x0b, 0x21, 0x00, 0xbe, 0xd3, 0xbf, - 0x4c, 0xc0, 0x02, 0xb3, 0x95, 0xb0, 0x75, 0xde, 0x81, 0x29, 0xaf, 0x87, 0x30, 0xef, 0x2f, 0x39, - 0xc4, 0x45, 0xcc, 0xdb, 0x48, 0x37, 0x1e, 0x20, 0xd7, 0x45, 0x0e, 0x6d, 0xc7, 0xa1, 0x6d, 0x1b, - 0x14, 0x7d, 0x58, 0xba, 0x10, 0xbd, 0x9f, 0x25, 0xe3, 0xee, 0x67, 0xaf, 0x43, 0xd9, 0xb4, 0x08, - 0x84, 0xd9, 0x45, 0x1a, 0xb2, 0xbc, 0x70, 0xd2, 0xef, 0x38, 0x5a, 0xf0, 0xe6, 0xef, 0x58, 0xc2, - 0xd9, 0x6b, 0x86, 0xfc, 0x12, 0x94, 0x5a, 0xfa, 0x89, 0xd9, 0xea, 0xb4, 0xb4, 0x36, 0x81, 0xc7, - 0xe6, 0x47, 0xec, 0x77, 0x73, 0x29, 0xb5, 0xc0, 0x27, 0x76, 0xf4, 0x43, 0xb4, 0x6b, 0x7e, 0x84, - 0xe4, 0x17, 0xa0, 0x40, 0x9b, 0x0b, 0x29, 0x20, 0xeb, 0x8a, 0x9b, 0xa6, 0x5d, 0x71, 0xb4, 0xe7, - 0x90, 0x80, 0xb1, 0xce, 0xfb, 0xff, 0x64, 0xbf, 0xc1, 0x0a, 0xc8, 0x8b, 0x1b, 0xd2, 0x53, 0x12, - 0x58, 0xac, 0x5f, 0x26, 0x9e, 0xa2, 0x5f, 0xc6, 0xf1, 0x9a, 0x8c, 0xe3, 0xf5, 0x5f, 0x24, 0x58, - 0xda, 0xe9, 0x38, 0x87, 0xe8, 0x8b, 0x68, 0x1d, 0xca, 0x0a, 0x94, 0xa3, 0xcc, 0x89, 0x8e, 0x80, - 0x04, 0x2c, 0x3d, 0x44, 0x5f, 0x50, 0xce, 0x3f, 0x17, 0xbf, 0xb8, 0x05, 0xe5, 0xa8, 0xc0, 0xb8, - 0x63, 0xc4, 0xd0, 0x90, 0xe2, 0x68, 0xfc, 0x88, 0x76, 0xbb, 0x1f, 0x38, 0x08, 0x37, 0xfc, 0xb5, - 0xf7, 0x49, 0x82, 0xe7, 0x7b, 0xe1, 0xe0, 0xf9, 0xeb, 0x63, 0x06, 0xcf, 0x81, 0xab, 0xf6, 0x63, - 0x28, 0x6d, 0x80, 0x8f, 0x83, 0xe3, 0x46, 0xf3, 0x43, 0x09, 0x5e, 0xda, 0x46, 0x16, 0x72, 0x74, - 0x17, 0x3d, 0xd0, 0xb1, 0x2b, 0x6e, 0xc8, 0x21, 0xf7, 0x7b, 0x16, 0x17, 0xde, 0x2b, 0xf0, 0xf2, - 0x58, 0x3b, 0xe3, 0x9c, 0xdc, 0x85, 0xd5, 0x60, 0xee, 0x15, 0xac, 0xab, 0x5d, 0x86, 0x82, 0x83, - 0x5a, 0xb6, 0xeb, 0xd9, 0x27, 0xcb, 0x1b, 0x32, 0x6a, 0x9e, 0x0d, 0x73, 0x03, 0xc5, 0x4a, 0x07, - 0xce, 0xc7, 0xd3, 0xe1, 0x86, 0xf1, 0x2d, 0x98, 0x66, 0xb7, 0x2f, 0x9e, 0x77, 0xbc, 0x39, 0x66, - 0x62, 0xc8, 0x6f, 0x17, 0x61, 0xb2, 0x9c, 0x98, 0xf2, 0xf7, 0x29, 0x58, 0x8c, 0x07, 0x19, 0x76, - 0x4b, 0xf8, 0x2a, 0x2c, 0xb5, 0xf4, 0x13, 0x2d, 0x1c, 0x7b, 0xfb, 0xfd, 0xee, 0xf3, 0x2d, 0xfd, - 0x24, 0x9c, 0x79, 0x19, 0xf2, 0x7d, 0x28, 0x32, 0x8a, 0x4d, 0xbb, 0xae, 0x37, 0x27, 0xab, 0x13, - 0xb2, 0xf4, 0xf8, 0x01, 0x41, 0xa4, 0x15, 0xa6, 0x8f, 0xa2, 0x82, 0x65, 0xb5, 0xec, 0xb7, 0xcf, - 0x24, 0x98, 0xaa, 0x1a, 0x50, 0x0b, 0x4b, 0x95, 0x43, 0xba, 0x92, 0x7f, 0x5f, 0x82, 0xb9, 0x86, - 0x6e, 0x19, 0x76, 0x97, 0x27, 0xfd, 0xd4, 0x08, 0xc9, 0x95, 0x72, 0x92, 0x7e, 0xeb, 0x01, 0x1b, - 0xb8, 0xc7, 0x09, 0x7b, 0xb7, 0x60, 0xbe, 0x09, 0xb9, 0x11, 0x99, 0x58, 0xf9, 0x81, 0x04, 0x73, - 0x31, 0x1b, 0x8e, 0x69, 0xc1, 0x7e, 0x3f, 0x98, 0xb6, 0x6f, 0x9f, 0x69, 0x8f, 0x3b, 0xc8, 0xe1, - 0xeb, 0xf9, 0xd2, 0xf8, 0x95, 0xef, 0x49, 0xb0, 0x34, 0x60, 0xf3, 0x31, 0x1b, 0x52, 0x83, 0x1b, - 0xfa, 0xfa, 0x98, 0x1b, 0x8a, 0x2c, 0x40, 0x13, 0x7a, 0xdf, 0x65, 0xe2, 0x5d, 0x58, 0x88, 0x85, - 0x91, 0xdf, 0x82, 0xf3, 0x9e, 0xce, 0xe2, 0x0c, 0x57, 0xa2, 0x86, 0xbb, 0x2c, 0x60, 0x22, 0xd6, - 0xab, 0xfc, 0x99, 0x04, 0x6b, 0xa3, 0xe4, 0x21, 0x2b, 0x30, 0xab, 0xd7, 0x8f, 0x90, 0x11, 0x22, - 0x9b, 0xa5, 0x83, 0xdc, 0x0d, 0xde, 0x87, 0x15, 0x1f, 0x4c, 0xf8, 0x36, 0x3c, 0x6e, 0x0f, 0xf4, - 0x92, 0x47, 0xf2, 0x71, 0xe0, 0x5a, 0x7c, 0xab, 0xfd, 0xc9, 0xa7, 0x95, 0x73, 0x3f, 0xfd, 0xb4, - 0x72, 0xee, 0xe7, 0x9f, 0x56, 0xa4, 0xdf, 0x7e, 0x52, 0x91, 0x7e, 0xfc, 0xa4, 0x22, 0xfd, 0xdd, - 0x93, 0x8a, 0xf4, 0xc9, 0x93, 0x8a, 0xf4, 0x6f, 0x4f, 0x2a, 0xd2, 0x7f, 0x3c, 0xa9, 0x9c, 0xfb, - 0xf9, 0x93, 0x8a, 0xf4, 0xf1, 0x67, 0x95, 0x73, 0x9f, 0x7c, 0x56, 0x39, 0xf7, 0xd3, 0xcf, 0x2a, - 0xe7, 0xde, 0xbb, 0x71, 0x68, 0xf7, 0x55, 0x60, 0xda, 0x43, 0xff, 0x01, 0xcd, 0xaf, 0x05, 0x47, - 0xf6, 0xa7, 0xe9, 0x26, 0xaf, 0xff, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xee, 0x03, 0xc5, 0xb8, - 0xbf, 0x46, 0x00, 0x00, + // 4058 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5c, 0x4d, 0x6c, 0x1c, 0x47, + 0x76, 0x56, 0x73, 0xf8, 0x33, 0xf3, 0x66, 0x38, 0x3f, 0xcd, 0xbf, 0x21, 0x29, 0x8d, 0xc8, 0xb6, + 0x65, 0xd1, 0x3f, 0x1a, 0x5a, 0xd2, 0xae, 0xed, 0x55, 0xd6, 0xeb, 0x48, 0x94, 0x44, 0x8d, 0x20, + 0x69, 0xe9, 0x26, 0x57, 0x36, 0xbc, 0xeb, 0x6d, 0x37, 0xa7, 0x8b, 0x9c, 0x0e, 0x67, 0xba, 0xc7, + 0x5d, 0x3d, 0x43, 0x8e, 0x73, 0xc8, 0xcf, 0x22, 0x01, 0x76, 0x03, 0x04, 0x06, 0x72, 0x59, 0x20, + 0x9b, 0x4b, 0x80, 0x20, 0x8b, 0x00, 0x41, 0x0e, 0x39, 0xed, 0x21, 0xd7, 0x20, 0xa7, 0xc4, 0x08, + 0x10, 0x64, 0xb1, 0x39, 0x24, 0x96, 0x11, 0x20, 0x41, 0x72, 0xd8, 0x43, 0x0e, 0x39, 0x06, 0xf5, + 0xd7, 0xff, 0xf3, 0x47, 0x4a, 0xf1, 0xae, 0xd7, 0x37, 0x4e, 0xd5, 0x7b, 0xaf, 0xea, 0xbd, 0x7a, + 0xef, 0xab, 0xaa, 0x57, 0xaf, 0x09, 0x5f, 0x77, 0x51, 0xab, 0x6d, 0x3b, 0x7a, 0x73, 0x13, 0x23, + 0xa7, 0x8b, 0x9c, 0x4d, 0xbd, 0x6d, 0x6e, 0x36, 0x4c, 0xec, 0xda, 0x4e, 0x8f, 0xb4, 0x98, 0x75, + 0xb4, 0xd9, 0xbd, 0xba, 0xe9, 0xa0, 0x0f, 0x3b, 0x08, 0xbb, 0x9a, 0x83, 0x70, 0xdb, 0xb6, 0x30, + 0xaa, 0xb6, 0x1d, 0xdb, 0xb5, 0xe5, 0x4b, 0x82, 0xbb, 0xca, 0xb8, 0xab, 0x7a, 0xdb, 0xac, 0x86, + 0xb9, 0xab, 0xdd, 0xab, 0x2b, 0x95, 0x43, 0xdb, 0x3e, 0x6c, 0xa2, 0x4d, 0xca, 0xb4, 0xdf, 0x39, + 0xd8, 0x34, 0x3a, 0x8e, 0xee, 0x9a, 0xb6, 0xc5, 0xc4, 0xac, 0x5c, 0x8c, 0xf6, 0xbb, 0x66, 0x0b, + 0x61, 0x57, 0x6f, 0xb5, 0x39, 0xc1, 0xba, 0x81, 0xda, 0xc8, 0x32, 0x90, 0x55, 0x37, 0x11, 0xde, + 0x3c, 0xb4, 0x0f, 0x6d, 0xda, 0x4e, 0xff, 0xe2, 0x24, 0xcf, 0x7b, 0x8a, 0x10, 0x0d, 0xea, 0x76, + 0xab, 0x65, 0x5b, 0x64, 0xe6, 0x2d, 0x84, 0xb1, 0x7e, 0xc8, 0x27, 0xbc, 0x72, 0x29, 0x44, 0xc5, + 0x67, 0x1a, 0x27, 0xbb, 0x1c, 0x22, 0x73, 0x75, 0x7c, 0xf4, 0x61, 0x07, 0x75, 0x50, 0x9c, 0x30, + 0x3c, 0x2a, 0xb2, 0x3a, 0x2d, 0x4c, 0x88, 0x8e, 0x6d, 0xe7, 0xe8, 0xa0, 0x69, 0x1f, 0x73, 0xaa, + 0x17, 0x42, 0x54, 0xa2, 0x33, 0x2e, 0xed, 0xb9, 0x10, 0xdd, 0x87, 0x1d, 0x94, 0x34, 0xb7, 0xb0, + 0x0a, 0x07, 0xba, 0xd9, 0xec, 0x38, 0x09, 0x33, 0x7b, 0x65, 0xc0, 0xc2, 0xc6, 0xa9, 0x5f, 0x4c, + 0xa2, 0xf6, 0xd4, 0x61, 0xd6, 0xe4, 0xa4, 0x2f, 0x0f, 0x24, 0x8d, 0x68, 0x7e, 0x79, 0x20, 0x31, + 0x31, 0x2c, 0x27, 0xbc, 0x92, 0x44, 0xd8, 0xdf, 0x52, 0xd5, 0x24, 0x72, 0x4b, 0x6f, 0x21, 0xdc, + 0xd6, 0xeb, 0x09, 0xd6, 0x78, 0x35, 0x89, 0xde, 0x41, 0xed, 0xa6, 0x59, 0xa7, 0x8e, 0x18, 0xe7, + 0xb8, 0x9e, 0xc4, 0xd1, 0x46, 0x0e, 0x36, 0xb1, 0x8b, 0x2c, 0x36, 0x06, 0x3a, 0x41, 0xf5, 0x0e, + 0x61, 0xc7, 0x9c, 0xe9, 0xad, 0x11, 0x98, 0x84, 0x52, 0x5a, 0xab, 0xe3, 0xea, 0xfb, 0x4d, 0xa4, + 0x61, 0x57, 0x77, 0xc5, 0xa8, 0xaf, 0x25, 0x7a, 0xca, 0xd0, 0x40, 0x5c, 0xb9, 0x91, 0x34, 0xb0, + 0x6e, 0xb4, 0x4c, 0x6b, 0x28, 0xaf, 0xf2, 0x07, 0xd3, 0x70, 0x61, 0xd7, 0xd5, 0x1d, 0xf7, 0x1d, + 0x3e, 0xdc, 0x1d, 0xa1, 0x96, 0xca, 0x18, 0xe4, 0x75, 0xc8, 0x79, 0xb6, 0xd5, 0x4c, 0xa3, 0x2c, + 0xad, 0x49, 0x1b, 0x19, 0x35, 0xeb, 0xb5, 0xd5, 0x0c, 0xb9, 0x0e, 0xb3, 0x98, 0xc8, 0xd0, 0xf8, + 0x20, 0xe5, 0x89, 0x35, 0x69, 0x23, 0x7b, 0xed, 0x1b, 0xde, 0x42, 0x51, 0x68, 0x88, 0x28, 0x54, + 0xed, 0x5e, 0xad, 0x0e, 0x1c, 0x59, 0xcd, 0x51, 0xa1, 0x62, 0x1e, 0x0d, 0x58, 0x68, 0xeb, 0x0e, + 0xb2, 0x5c, 0xcd, 0xb3, 0xbc, 0x66, 0x5a, 0x07, 0x76, 0x39, 0x45, 0x07, 0xfb, 0x4a, 0x35, 0x09, + 0x8e, 0x3c, 0x8f, 0xec, 0x5e, 0xad, 0xee, 0x50, 0x6e, 0x6f, 0x94, 0x9a, 0x75, 0x60, 0xab, 0x73, + 0xed, 0x78, 0xa3, 0x5c, 0x86, 0x19, 0xdd, 0x25, 0xd2, 0xdc, 0xf2, 0xe4, 0x9a, 0xb4, 0x31, 0xa5, + 0x8a, 0x9f, 0x72, 0x0b, 0x14, 0x6f, 0x05, 0xfd, 0x59, 0xa0, 0x93, 0xb6, 0xc9, 0x20, 0x4d, 0x23, + 0xd8, 0x55, 0x9e, 0xa2, 0x13, 0x5a, 0xa9, 0x32, 0x60, 0xab, 0x0a, 0x60, 0xab, 0xee, 0x09, 0x60, + 0xbb, 0x35, 0xf9, 0xf1, 0xbf, 0x5e, 0x94, 0xd4, 0x8b, 0xc7, 0x51, 0xcd, 0xef, 0x78, 0x92, 0x08, + 0xad, 0xdc, 0x80, 0xe5, 0xba, 0x6d, 0xb9, 0xa6, 0xd5, 0x41, 0x9a, 0x8e, 0x35, 0x0b, 0x1d, 0x6b, + 0xa6, 0x65, 0xba, 0xa6, 0xee, 0xda, 0x4e, 0x79, 0x7a, 0x4d, 0xda, 0xc8, 0x5f, 0xbb, 0x12, 0xb6, + 0x31, 0x8d, 0x2e, 0xa2, 0xec, 0x16, 0xe7, 0xbb, 0x89, 0x1f, 0xa1, 0xe3, 0x9a, 0x60, 0x52, 0x17, + 0xeb, 0x89, 0xed, 0xf2, 0x43, 0x28, 0x89, 0x1e, 0x43, 0xe3, 0xb0, 0x52, 0x9e, 0xa1, 0x7a, 0xac, + 0x85, 0x47, 0xe0, 0x9d, 0x64, 0x8c, 0xbb, 0xec, 0x4f, 0xb5, 0xe8, 0xb1, 0xf2, 0x16, 0xf9, 0x31, + 0x2c, 0x36, 0x75, 0xec, 0x6a, 0x75, 0xbb, 0xd5, 0x6e, 0x22, 0x6a, 0x19, 0x07, 0xe1, 0x4e, 0xd3, + 0x2d, 0xa7, 0x93, 0x64, 0x72, 0x88, 0xa1, 0x6b, 0xd4, 0x6b, 0xda, 0xba, 0x81, 0xd5, 0x79, 0xc2, + 0xbf, 0xe5, 0xb1, 0xab, 0x94, 0x5b, 0xfe, 0x2e, 0xac, 0x1e, 0x98, 0x0e, 0x76, 0x35, 0x6f, 0x15, + 0x08, 0x8a, 0x68, 0xfb, 0x7a, 0xfd, 0xc8, 0x3e, 0x38, 0x28, 0x67, 0xa8, 0xf0, 0xe5, 0x98, 0xe1, + 0x6f, 0xf3, 0x1d, 0xe7, 0xd6, 0xe4, 0x0f, 0x89, 0xdd, 0xcb, 0x54, 0x86, 0x70, 0xbb, 0x3d, 0x1d, + 0x1f, 0xdd, 0x62, 0x02, 0x94, 0xd7, 0xa1, 0xd2, 0xcf, 0x25, 0x59, 0xd4, 0xc8, 0x0b, 0x30, 0xed, + 0x74, 0x2c, 0x3f, 0x0e, 0xa6, 0x9c, 0x8e, 0x55, 0x33, 0x94, 0xff, 0x92, 0x60, 0x71, 0x1b, 0xb9, + 0x0f, 0x59, 0x54, 0xef, 0x92, 0xa0, 0x1e, 0x23, 0x7e, 0xb6, 0x21, 0xe3, 0x79, 0x13, 0x8f, 0x9d, + 0x17, 0xfb, 0x59, 0x28, 0x3e, 0x35, 0x9f, 0x57, 0xbe, 0x0e, 0x8b, 0xe8, 0xa4, 0x8d, 0xea, 0x2e, + 0x32, 0x34, 0x0b, 0x9d, 0xb8, 0x1a, 0xea, 0x92, 0x80, 0x31, 0x0d, 0x1a, 0x24, 0x29, 0x75, 0x4e, + 0xf4, 0x3e, 0x42, 0x27, 0xee, 0x1d, 0xd2, 0x57, 0x33, 0xe4, 0x57, 0x61, 0xbe, 0xde, 0x71, 0x68, + 0x64, 0xed, 0x3b, 0xba, 0x55, 0x6f, 0x68, 0xae, 0x7d, 0x84, 0x2c, 0xea, 0xfb, 0x39, 0x55, 0xe6, + 0x7d, 0xb7, 0x68, 0xd7, 0x1e, 0xe9, 0x51, 0xfe, 0x22, 0x0d, 0x4b, 0x31, 0x6d, 0xb9, 0x81, 0x42, + 0xba, 0x48, 0x67, 0xd0, 0xa5, 0x06, 0xb3, 0xfe, 0x2a, 0xf7, 0xda, 0x88, 0x1b, 0xe6, 0xf9, 0x61, + 0xc2, 0xf6, 0x7a, 0x6d, 0xa4, 0xe6, 0x8e, 0x03, 0xbf, 0x64, 0x05, 0x66, 0x93, 0xac, 0x91, 0xb5, + 0x02, 0x56, 0xf8, 0x1a, 0x2c, 0xb7, 0x1d, 0xd4, 0x35, 0xed, 0x0e, 0xd6, 0x28, 0xee, 0x20, 0xc3, + 0xa7, 0x9f, 0xa4, 0xf4, 0x8b, 0x82, 0x60, 0x97, 0xf5, 0x0b, 0xd6, 0x2b, 0x30, 0x47, 0xbd, 0x9d, + 0xb9, 0xa6, 0xc7, 0x34, 0x45, 0x99, 0x8a, 0xa4, 0xeb, 0x2e, 0xe9, 0x11, 0xe4, 0x5b, 0x00, 0xd4, + 0x6b, 0xe9, 0xa9, 0x82, 0x86, 0x71, 0x4c, 0x2b, 0xef, 0xd0, 0x41, 0x14, 0x23, 0x0e, 0xfa, 0x36, + 0xf9, 0xa1, 0x66, 0x5c, 0xf1, 0xa7, 0xbc, 0x03, 0x25, 0xec, 0x9a, 0xf5, 0xa3, 0x9e, 0x16, 0x90, + 0x35, 0x33, 0x86, 0xac, 0x02, 0x63, 0xf7, 0x1a, 0xe4, 0xdf, 0x84, 0x97, 0x63, 0x12, 0x35, 0x5c, + 0x6f, 0x20, 0xa3, 0xd3, 0x44, 0x9a, 0x6b, 0x33, 0xab, 0x50, 0x84, 0xb3, 0x3b, 0x6e, 0x39, 0x3b, + 0x5a, 0xac, 0x5d, 0x8a, 0x0c, 0xb3, 0xcb, 0x05, 0xee, 0xd9, 0xd4, 0x88, 0x7b, 0x4c, 0x5a, 0x5f, + 0x1f, 0x9c, 0xed, 0xe7, 0x83, 0xf2, 0xb7, 0x21, 0xef, 0xb9, 0x07, 0xdd, 0x44, 0xcb, 0x05, 0x0a, + 0x88, 0xc9, 0xfb, 0x80, 0x87, 0x8b, 0x31, 0x97, 0x63, 0xde, 0xeb, 0xb9, 0x1a, 0xfd, 0x29, 0xbf, + 0x03, 0x85, 0x90, 0xf0, 0x0e, 0x2e, 0x17, 0xa9, 0xf4, 0x6a, 0x1f, 0xb8, 0x4d, 0x14, 0xdb, 0xc1, + 0x6a, 0x3e, 0x28, 0xb7, 0x83, 0xe5, 0xf7, 0xa1, 0xd4, 0x25, 0x27, 0x02, 0xdb, 0xd2, 0xd8, 0x71, + 0xcc, 0x44, 0xb8, 0x5c, 0xa2, 0xa6, 0x7c, 0xb5, 0x3a, 0xe0, 0x3c, 0x4d, 0xc6, 0x78, 0xcc, 0x18, + 0xef, 0x09, 0x3e, 0xb5, 0xd8, 0x8d, 0xb4, 0xc8, 0xdf, 0x80, 0xf3, 0x26, 0x71, 0xdf, 0xe8, 0x32, + 0x22, 0x8b, 0x04, 0xaa, 0x51, 0x96, 0xd7, 0xa4, 0x8d, 0xb4, 0x5a, 0x36, 0xf1, 0x6e, 0x78, 0x55, + 0xee, 0xb0, 0x7e, 0xf9, 0x2b, 0xb0, 0x14, 0xf3, 0x64, 0xf7, 0x84, 0xc2, 0xdd, 0x1c, 0x03, 0x90, + 0xb0, 0x37, 0xef, 0x9d, 0x58, 0x35, 0xe3, 0xfe, 0x64, 0x3a, 0x5d, 0xcc, 0xdc, 0x9f, 0x4c, 0x67, + 0x8a, 0x70, 0x7f, 0x32, 0x0d, 0xc5, 0xec, 0xfd, 0xc9, 0x74, 0xae, 0x38, 0x7b, 0x7f, 0x32, 0x9d, + 0x2f, 0x16, 0x94, 0xff, 0x96, 0x60, 0x69, 0xc7, 0x6e, 0x36, 0x7f, 0x45, 0xb0, 0xf1, 0xdf, 0x67, + 0xa0, 0x1c, 0x57, 0xf7, 0x4b, 0x70, 0xfc, 0x12, 0x1c, 0x9f, 0x3a, 0x38, 0xe6, 0xfa, 0x82, 0x63, + 0x22, 0xcc, 0xe4, 0x9f, 0x1a, 0xcc, 0xfc, 0x72, 0x62, 0xef, 0x00, 0x70, 0x2b, 0x8d, 0x07, 0x6e, + 0xb3, 0xc5, 0xbc, 0xf2, 0x7d, 0x09, 0x56, 0x55, 0x84, 0x91, 0x1b, 0x81, 0xd2, 0xcf, 0x01, 0xda, + 0x94, 0x0a, 0x9c, 0x4f, 0x9e, 0x0a, 0x83, 0x1d, 0xe5, 0x67, 0x13, 0xb0, 0xa6, 0xa2, 0xba, 0xed, + 0x18, 0xc1, 0x43, 0x2f, 0x0f, 0xd4, 0x31, 0x26, 0xfc, 0x2e, 0xc8, 0xf1, 0xeb, 0xcf, 0xf8, 0x33, + 0x2f, 0xc5, 0xee, 0x3d, 0xf2, 0x45, 0xc8, 0x7a, 0xd1, 0xe4, 0x41, 0x10, 0x88, 0xa6, 0x9a, 0x21, + 0x2f, 0xc1, 0x0c, 0x8d, 0x3c, 0x0f, 0x6f, 0xa6, 0xc9, 0xcf, 0x9a, 0x21, 0x5f, 0x00, 0x10, 0x57, + 0x5b, 0x0e, 0x2b, 0x19, 0x35, 0xc3, 0x5b, 0x6a, 0x86, 0xfc, 0x01, 0xe4, 0xda, 0x76, 0xb3, 0xe9, + 0xdd, 0x4c, 0x19, 0xa2, 0xbc, 0x39, 0xf4, 0x66, 0x4a, 0x20, 0x3c, 0x68, 0xac, 0xe0, 0xda, 0xaa, + 0x59, 0x22, 0x92, 0xff, 0x50, 0xfe, 0x69, 0x06, 0xd6, 0x07, 0x18, 0x97, 0x23, 0x7f, 0x0c, 0xb0, + 0xa5, 0x53, 0x03, 0xf6, 0x40, 0x30, 0x9e, 0x18, 0x08, 0xc6, 0xaf, 0x80, 0x2c, 0x6c, 0x6a, 0x44, + 0x01, 0xbf, 0xe8, 0xf5, 0x08, 0xea, 0x0d, 0x28, 0xf6, 0x01, 0xfb, 0x3c, 0x0e, 0xcb, 0x8d, 0xed, + 0x21, 0x53, 0xf1, 0x3d, 0x24, 0x70, 0xab, 0x9e, 0x0e, 0xdf, 0xaa, 0xdf, 0x80, 0x32, 0x07, 0xd7, + 0xc0, 0x9d, 0x9a, 0x9f, 0x58, 0x66, 0xe8, 0x89, 0x65, 0x91, 0xf5, 0xfb, 0xf7, 0x64, 0x7e, 0x5e, + 0x39, 0x0c, 0x38, 0x24, 0x73, 0x0f, 0xeb, 0xc0, 0xe6, 0x77, 0xcc, 0xaf, 0x0d, 0x03, 0xba, 0x3d, + 0x47, 0xb7, 0xb0, 0x89, 0xac, 0xd0, 0x4d, 0x90, 0x66, 0x05, 0x8a, 0xc7, 0x91, 0x16, 0xf9, 0x10, + 0x2e, 0x24, 0x5c, 0xfc, 0x03, 0xbb, 0x4b, 0x66, 0x8c, 0xdd, 0x65, 0x25, 0xe6, 0xff, 0xfe, 0x46, + 0xb3, 0x0e, 0xb9, 0x10, 0xc6, 0x67, 0x29, 0xc6, 0x67, 0xf7, 0x03, 0xe0, 0xbe, 0x0d, 0x79, 0x7f, + 0x11, 0x69, 0xc2, 0x21, 0x37, 0x62, 0xc2, 0x61, 0xd6, 0xe3, 0xa3, 0xe9, 0x85, 0x2d, 0xc8, 0x89, + 0xf5, 0xa5, 0x62, 0x66, 0x47, 0x14, 0x93, 0xe5, 0x5c, 0x54, 0x88, 0x0d, 0x33, 0x1f, 0x76, 0x10, + 0xdf, 0x60, 0x52, 0x1b, 0xd9, 0x6b, 0xdf, 0xaa, 0x8e, 0x94, 0x17, 0xae, 0x0e, 0x8d, 0x99, 0xea, + 0xdb, 0x4c, 0xee, 0x1d, 0xcb, 0x75, 0x7a, 0xaa, 0x18, 0x65, 0xe5, 0x03, 0xc8, 0x05, 0x3b, 0xe4, + 0x22, 0xa4, 0x8e, 0x50, 0x8f, 0xc3, 0x15, 0xf9, 0x53, 0xbe, 0x01, 0x53, 0x5d, 0xbd, 0xd9, 0xe9, + 0x73, 0x28, 0xa2, 0x99, 0xd5, 0x60, 0x88, 0x11, 0x69, 0x3d, 0x95, 0xb1, 0xdc, 0x98, 0x78, 0x43, + 0x62, 0x30, 0x1f, 0x00, 0xcd, 0x9b, 0x75, 0xd7, 0xec, 0x9a, 0x6e, 0xef, 0x4b, 0xd0, 0x1c, 0x01, + 0x34, 0x83, 0xc6, 0xea, 0x0f, 0x9a, 0xbf, 0x3b, 0x29, 0x40, 0x33, 0xd1, 0xb8, 0x1c, 0x34, 0x1f, + 0x41, 0x21, 0x02, 0x57, 0x1c, 0x36, 0x2f, 0x85, 0xa7, 0x12, 0x08, 0x6a, 0x76, 0x48, 0xe9, 0x51, + 0xd0, 0x51, 0xf3, 0x61, 0x48, 0x8b, 0x39, 0xfc, 0xc4, 0x69, 0x1c, 0x3e, 0x80, 0x63, 0xa9, 0x30, + 0x8e, 0x21, 0xa8, 0x88, 0x73, 0x1a, 0x6f, 0xd2, 0x22, 0x81, 0x3a, 0x39, 0xe2, 0x80, 0xab, 0x5c, + 0xce, 0x4d, 0x26, 0x66, 0x37, 0x14, 0xb6, 0x0f, 0xa1, 0xd4, 0x40, 0xba, 0xe3, 0xee, 0x23, 0xdd, + 0xd5, 0x0c, 0xe4, 0xea, 0x66, 0x13, 0xf3, 0x9c, 0xe3, 0xf0, 0xbc, 0x5a, 0xd1, 0x63, 0xbd, 0xcd, + 0x38, 0xe3, 0x3b, 0xd3, 0xf4, 0xa9, 0x77, 0xa6, 0x2b, 0x01, 0x57, 0xf7, 0x42, 0x80, 0x42, 0x78, + 0xc6, 0xf7, 0xdf, 0x47, 0xa2, 0x43, 0xf9, 0x89, 0x04, 0xcf, 0xb1, 0xb5, 0x0e, 0xc1, 0x00, 0xcf, + 0xfa, 0x8d, 0x15, 0x64, 0x36, 0x14, 0x79, 0xae, 0x11, 0x45, 0x92, 0xd0, 0xb7, 0x87, 0x7a, 0xed, + 0x08, 0x53, 0x50, 0x0b, 0x42, 0xba, 0x70, 0xe0, 0x3f, 0x96, 0xe0, 0xf9, 0xc1, 0x8c, 0xdc, 0x87, + 0xb1, 0xbf, 0x89, 0x8a, 0xd4, 0x3b, 0x77, 0xe2, 0x7b, 0x4f, 0x0b, 0x28, 0xc9, 0x75, 0x25, 0xd4, + 0xa0, 0xfc, 0x95, 0x44, 0xb0, 0x2b, 0x36, 0xbb, 0xbb, 0xba, 0xd9, 0x1c, 0xcb, 0xac, 0x0d, 0xc8, + 0x1f, 0x50, 0x9e, 0x88, 0x51, 0x6f, 0x9e, 0xc6, 0xa8, 0xa1, 0xd1, 0xd5, 0xd9, 0x83, 0xe0, 0x4f, + 0xe5, 0x39, 0x82, 0x07, 0x7d, 0x59, 0xb8, 0x5a, 0x3f, 0x91, 0x40, 0x89, 0xa3, 0xc6, 0x3d, 0xe1, + 0xd1, 0x63, 0x28, 0xd6, 0x0e, 0xc6, 0x50, 0x58, 0xb7, 0xad, 0x11, 0x74, 0x1b, 0x36, 0x85, 0x40, + 0x98, 0x09, 0x05, 0x77, 0x88, 0xaf, 0x0f, 0xe0, 0xe3, 0xee, 0xf2, 0x22, 0x14, 0xeb, 0xba, 0x55, + 0x47, 0x1e, 0xf8, 0x22, 0x36, 0xff, 0xb4, 0x5a, 0x60, 0xed, 0xaa, 0x68, 0x0e, 0x86, 0x4f, 0x50, + 0xe6, 0xe7, 0x14, 0x3e, 0x83, 0xa6, 0x10, 0x0f, 0x9f, 0x17, 0xbc, 0xe8, 0xe9, 0xc3, 0x17, 0x77, + 0xe4, 0x20, 0xe1, 0xff, 0xbf, 0x23, 0xf7, 0x1d, 0xbd, 0xbf, 0x23, 0x27, 0xb1, 0x70, 0xb5, 0xfe, + 0x9a, 0x3a, 0x72, 0x5c, 0x7f, 0xba, 0xc2, 0x63, 0x29, 0xf6, 0x1b, 0x90, 0x0f, 0xfb, 0xcb, 0x18, + 0x5e, 0x3c, 0x6c, 0x7c, 0x75, 0x36, 0xe4, 0x72, 0xca, 0xa5, 0x64, 0x7f, 0xf3, 0x98, 0xb8, 0x72, + 0x7f, 0x3b, 0x01, 0x95, 0x5d, 0xf3, 0xd0, 0xd2, 0x9b, 0x67, 0x79, 0x53, 0x3c, 0x80, 0x3c, 0xa6, + 0x42, 0x22, 0x8a, 0xbd, 0x35, 0xfc, 0x51, 0x71, 0xe0, 0xd8, 0xea, 0x2c, 0x13, 0x2b, 0xa6, 0x62, + 0xc2, 0x2a, 0x3a, 0x71, 0x91, 0x43, 0x46, 0x4a, 0x38, 0xa7, 0xa5, 0xc6, 0x3d, 0xa7, 0x2d, 0x0b, + 0x69, 0xb1, 0x2e, 0xb9, 0x0a, 0x73, 0xf5, 0x86, 0xd9, 0x34, 0xfc, 0x71, 0x6c, 0xab, 0xd9, 0xa3, + 0x87, 0x82, 0xb4, 0x5a, 0xa2, 0x5d, 0x82, 0xe9, 0x9b, 0x56, 0xb3, 0xa7, 0xac, 0xc3, 0xc5, 0xbe, + 0xba, 0x70, 0x5b, 0xff, 0xa3, 0x04, 0x97, 0x39, 0x8d, 0xe9, 0x36, 0xce, 0xfc, 0x90, 0xfb, 0x3d, + 0x09, 0x96, 0xb9, 0xd5, 0x8f, 0x4d, 0xb7, 0xa1, 0x25, 0xbd, 0xea, 0xde, 0x1b, 0x75, 0x01, 0x86, + 0x4d, 0x48, 0x5d, 0xc4, 0x61, 0x42, 0xe1, 0x67, 0x37, 0x61, 0x63, 0xb8, 0x88, 0xc1, 0xef, 0x71, + 0x7f, 0x23, 0xc1, 0x45, 0x15, 0xb5, 0xec, 0x2e, 0x62, 0x92, 0x4e, 0x99, 0x7c, 0x7e, 0x76, 0x67, + 0xf7, 0xf0, 0x09, 0x3c, 0x15, 0x39, 0x81, 0x2b, 0x0a, 0x81, 0xbd, 0x7e, 0xd3, 0x17, 0x6b, 0x3f, + 0x01, 0xeb, 0x7b, 0xc8, 0x69, 0x99, 0x96, 0xee, 0xa2, 0xb3, 0xac, 0xba, 0x0d, 0x25, 0x57, 0xc8, + 0x89, 0x2c, 0xf6, 0xad, 0xa1, 0x8b, 0x3d, 0x74, 0x06, 0x6a, 0xd1, 0x13, 0xfe, 0x4b, 0x10, 0x73, + 0xcf, 0x83, 0x32, 0x48, 0x23, 0x6e, 0xfa, 0x3f, 0x91, 0xa0, 0x72, 0x1b, 0x91, 0xad, 0xea, 0x2c, + 0x76, 0x7f, 0x66, 0xde, 0x45, 0x90, 0xa3, 0xef, 0xf4, 0xb8, 0x0a, 0x7f, 0x2e, 0xc1, 0x05, 0x9a, + 0x34, 0x3c, 0x63, 0xe1, 0x87, 0x43, 0x64, 0x8c, 0x5d, 0xf8, 0x31, 0x70, 0x64, 0x35, 0x47, 0x85, + 0x0a, 0x38, 0x78, 0x1d, 0x2a, 0xfd, 0xc8, 0x07, 0x83, 0xc0, 0x1f, 0xa5, 0xe0, 0x12, 0x17, 0xc2, + 0x36, 0xa9, 0xb3, 0xa8, 0xda, 0xea, 0xb3, 0xd1, 0xde, 0x1d, 0x41, 0xd7, 0x11, 0xa6, 0x10, 0xd9, + 0x6b, 0xe5, 0x37, 0x03, 0x21, 0xc2, 0x6b, 0x3e, 0xe2, 0x29, 0xbb, 0xb2, 0x20, 0xa9, 0x09, 0x0a, + 0x91, 0x6c, 0x1b, 0x12, 0x61, 0x93, 0xcf, 0x3e, 0xc2, 0xa6, 0xfa, 0x45, 0xd8, 0x06, 0xbc, 0x30, + 0xcc, 0x22, 0xdc, 0x45, 0xff, 0x41, 0x82, 0x55, 0x71, 0xf5, 0x0d, 0xde, 0x0a, 0x7e, 0x21, 0x00, + 0xfc, 0x3a, 0x2c, 0x9a, 0x58, 0x4b, 0xa8, 0x46, 0xa1, 0x6b, 0x93, 0x56, 0xe7, 0x4c, 0x7c, 0x37, + 0x5a, 0x66, 0xa2, 0x54, 0xe0, 0x7c, 0xb2, 0x42, 0x5c, 0xe3, 0xff, 0x99, 0x20, 0xe7, 0x62, 0x72, + 0x4b, 0xd8, 0x22, 0x76, 0xf3, 0x46, 0x3b, 0xcd, 0x99, 0xfe, 0xd9, 0xa9, 0xbe, 0x0e, 0x39, 0xdf, + 0x25, 0xfd, 0x07, 0x43, 0xaf, 0xad, 0x66, 0xc8, 0xef, 0xc1, 0x9c, 0x38, 0xf2, 0x1b, 0x67, 0xf1, + 0x3b, 0xd9, 0x93, 0xe2, 0x0f, 0xbf, 0xe3, 0x5d, 0x56, 0x68, 0xa2, 0x98, 0xa6, 0x85, 0xa6, 0xc6, + 0x49, 0x0b, 0x15, 0x7c, 0x76, 0xda, 0xa0, 0x5c, 0x26, 0x38, 0x31, 0xd0, 0xea, 0x7c, 0x7d, 0xfe, + 0x54, 0x82, 0xb5, 0xdb, 0x08, 0xd7, 0x1d, 0x73, 0xff, 0x4c, 0xc8, 0xff, 0x6d, 0x98, 0x19, 0xf7, + 0x1e, 0x32, 0x6c, 0x58, 0x55, 0x48, 0x54, 0x7e, 0x9c, 0x82, 0xf5, 0x01, 0xd4, 0x1c, 0x33, 0xbf, + 0x03, 0x45, 0x3f, 0x91, 0x5d, 0xb7, 0xad, 0x03, 0xf3, 0x90, 0xe7, 0x25, 0xae, 0x26, 0xcf, 0x25, + 0x71, 0x81, 0xb6, 0x28, 0xa3, 0x5a, 0x40, 0xe1, 0x06, 0xf9, 0x10, 0x96, 0x12, 0xf2, 0xe5, 0x34, + 0x3b, 0xcf, 0x14, 0xde, 0x1c, 0x63, 0x10, 0x9a, 0x93, 0x5f, 0x38, 0x4e, 0x6a, 0x96, 0xbf, 0x03, + 0x72, 0x1b, 0x59, 0x86, 0x69, 0x1d, 0x6a, 0x3a, 0xbb, 0x94, 0x98, 0x08, 0x97, 0x53, 0x34, 0x13, + 0x7d, 0xa5, 0xff, 0x18, 0x3b, 0x8c, 0x47, 0xdc, 0x63, 0xe8, 0x08, 0xa5, 0x76, 0xa8, 0xd1, 0x44, + 0x58, 0xfe, 0x2e, 0x14, 0x85, 0x74, 0x0a, 0x64, 0x0e, 0x7d, 0xfa, 0x27, 0xb2, 0xaf, 0x0f, 0x95, + 0x1d, 0xf6, 0x25, 0x3a, 0x42, 0xa1, 0x1d, 0xe8, 0x72, 0x90, 0xa5, 0xfc, 0x4e, 0x0a, 0xca, 0x2a, + 0x2f, 0x44, 0x45, 0xd4, 0x17, 0xf1, 0xe3, 0x6b, 0xbf, 0x10, 0x31, 0x7e, 0x00, 0x0b, 0xe1, 0x17, + 0xe4, 0x9e, 0x66, 0xba, 0xa8, 0x25, 0x4c, 0x7b, 0x6d, 0xac, 0x57, 0xe4, 0x5e, 0xcd, 0x45, 0x2d, + 0x75, 0xae, 0x1b, 0x6b, 0xc3, 0xf2, 0x1b, 0x30, 0x4d, 0x23, 0x18, 0x73, 0x6c, 0xe8, 0x9b, 0xc1, + 0xbc, 0xad, 0xbb, 0xfa, 0xad, 0xa6, 0xbd, 0xaf, 0x72, 0x7a, 0xf9, 0x2e, 0xe4, 0x2d, 0x74, 0xac, + 0x91, 0x8d, 0x9f, 0x4b, 0x98, 0x1a, 0x51, 0x42, 0xce, 0x42, 0xc7, 0x6a, 0x87, 0xc5, 0x3e, 0x56, + 0x56, 0x61, 0x39, 0x61, 0x09, 0xfc, 0x83, 0xde, 0xe2, 0x6e, 0xcf, 0xaa, 0xef, 0x36, 0x74, 0xc7, + 0xe0, 0xef, 0xca, 0x7c, 0x79, 0x2e, 0x41, 0x1e, 0xdb, 0x1d, 0xa7, 0x8e, 0xb4, 0x7a, 0xb3, 0x83, + 0x5d, 0xe4, 0xf0, 0x05, 0x9a, 0x65, 0xad, 0x5b, 0xac, 0x51, 0x5e, 0x86, 0x34, 0x26, 0xcc, 0xe2, + 0x71, 0x6e, 0x4a, 0x9d, 0xa1, 0xbf, 0x6b, 0x86, 0x7c, 0x13, 0xb2, 0xec, 0x81, 0x9b, 0x25, 0x87, + 0x53, 0x23, 0x26, 0x87, 0x81, 0x31, 0x91, 0x66, 0x65, 0x19, 0x96, 0x62, 0xd3, 0x13, 0xd7, 0x83, + 0x29, 0x98, 0x23, 0x7d, 0xc2, 0xc7, 0xc7, 0x70, 0xab, 0x8b, 0x90, 0xf5, 0xdc, 0x8a, 0x4f, 0x3b, + 0xa3, 0x82, 0x68, 0xaa, 0x19, 0x81, 0x03, 0x57, 0x2a, 0x70, 0xe0, 0x92, 0xcb, 0x30, 0xc3, 0xd7, + 0x98, 0xbf, 0x37, 0x88, 0x9f, 0x64, 0x50, 0x3f, 0x15, 0xee, 0xbf, 0x0f, 0x7a, 0x6d, 0xf4, 0x35, + 0x3c, 0xfa, 0xac, 0x35, 0x7d, 0xba, 0x67, 0xad, 0x0b, 0x00, 0x22, 0xe3, 0x6a, 0xb2, 0x07, 0xc4, + 0x94, 0x9a, 0xe1, 0x2d, 0xb4, 0xc2, 0x24, 0xfc, 0x08, 0x90, 0x3e, 0xcd, 0x23, 0xc0, 0x0e, 0xaf, + 0x6a, 0xf1, 0x93, 0x88, 0x54, 0x56, 0x66, 0x44, 0x59, 0x25, 0xc2, 0xec, 0x25, 0xff, 0xa8, 0xc4, + 0x1b, 0x30, 0x23, 0x72, 0xf9, 0x30, 0x62, 0x2e, 0x5f, 0x30, 0x04, 0x9f, 0x24, 0xb2, 0xe1, 0x27, + 0x89, 0x2d, 0xc8, 0xb1, 0x9a, 0x07, 0x5e, 0xd2, 0x9b, 0x1b, 0xb1, 0xa4, 0x37, 0x4b, 0x4b, 0x21, + 0x78, 0x35, 0xef, 0xab, 0x40, 0xab, 0x71, 0xe9, 0x31, 0x07, 0x39, 0x9a, 0x69, 0x20, 0xcb, 0x35, + 0xdd, 0x1e, 0x7d, 0x2f, 0xcc, 0xa8, 0x32, 0xe9, 0x7b, 0x87, 0x76, 0xd5, 0x78, 0x8f, 0xfc, 0x0e, + 0x14, 0x22, 0xe8, 0xc1, 0xab, 0x4f, 0xaa, 0xe3, 0xe1, 0x86, 0x9a, 0x0f, 0x63, 0x86, 0xb2, 0x08, + 0xf3, 0x61, 0x9f, 0xe6, 0xce, 0xfe, 0x7d, 0x09, 0x56, 0xc5, 0x9e, 0xf7, 0x39, 0x17, 0x9a, 0x29, + 0xff, 0x2b, 0xc1, 0xf9, 0xe4, 0xb9, 0xf0, 0xad, 0xb7, 0x01, 0x73, 0x75, 0xbd, 0xde, 0x40, 0xe1, + 0x8f, 0x00, 0xf8, 0xee, 0xfb, 0x46, 0xa2, 0x85, 0x02, 0x9f, 0x11, 0x04, 0xc7, 0x0f, 0x89, 0x2f, + 0x51, 0xa1, 0xc1, 0x26, 0xd9, 0x82, 0x45, 0x43, 0x77, 0xf5, 0x7d, 0x1d, 0x47, 0x07, 0x9b, 0x38, + 0xe3, 0x60, 0xf3, 0x42, 0x6e, 0xb0, 0x55, 0xf9, 0x67, 0x09, 0x56, 0x84, 0xea, 0x7c, 0xc9, 0xee, + 0xd9, 0x38, 0x98, 0x98, 0x6f, 0xd8, 0xd8, 0xd5, 0x74, 0xc3, 0x70, 0x10, 0xc6, 0x62, 0x15, 0x48, + 0xdb, 0x4d, 0xd6, 0x34, 0x08, 0x2e, 0xa3, 0x6b, 0x98, 0x1a, 0x75, 0x3f, 0x9c, 0x7c, 0x0a, 0x37, + 0xea, 0x8f, 0x27, 0x7c, 0x07, 0x0b, 0x69, 0xc6, 0xd7, 0xf4, 0x39, 0x98, 0xa5, 0xf3, 0xc4, 0x9a, + 0xd5, 0x69, 0xed, 0xf3, 0xcd, 0x60, 0x4a, 0xcd, 0xb1, 0xc6, 0x47, 0xb4, 0x4d, 0x5e, 0x85, 0x8c, + 0x50, 0x0e, 0x97, 0x27, 0xd6, 0x52, 0x1b, 0x53, 0x6a, 0x9a, 0x6b, 0x87, 0xe5, 0xf7, 0xa1, 0xe0, + 0xab, 0x47, 0x97, 0x72, 0xe0, 0x97, 0x0d, 0x1e, 0x2d, 0x51, 0xc1, 0x7b, 0x53, 0xdb, 0x22, 0x7c, + 0xf4, 0xac, 0x91, 0xb7, 0x42, 0x6d, 0xf2, 0x6b, 0xb0, 0xc4, 0xc6, 0xae, 0xdb, 0x96, 0xeb, 0xd8, + 0xcd, 0x26, 0x72, 0x44, 0x79, 0xd5, 0x24, 0x35, 0xe4, 0x02, 0xed, 0xde, 0xf2, 0x7a, 0x79, 0xd5, + 0x14, 0xc1, 0x16, 0xbe, 0x5c, 0xec, 0x9d, 0x58, 0xfc, 0x54, 0xaa, 0x50, 0xda, 0x6a, 0xda, 0x18, + 0xd1, 0xcd, 0x47, 0x2c, 0x71, 0x70, 0xfd, 0xa4, 0xd0, 0xfa, 0x29, 0xf3, 0x20, 0x07, 0xe9, 0x79, + 0xe4, 0xbe, 0x02, 0x85, 0x6d, 0xe4, 0x8e, 0x2a, 0xe3, 0x03, 0x28, 0xfa, 0xd4, 0xdc, 0xf4, 0x0f, + 0x00, 0x38, 0x39, 0x39, 0x5e, 0xb2, 0x28, 0xba, 0x32, 0x8a, 0x63, 0x53, 0x31, 0xd4, 0x58, 0x6c, + 0x59, 0xc8, 0x9f, 0xca, 0xcf, 0x24, 0x28, 0xb1, 0xd4, 0x5b, 0xf0, 0xaa, 0xd9, 0x7f, 0x4a, 0xf2, + 0x5d, 0x48, 0x93, 0xa3, 0xc3, 0x21, 0x01, 0xb9, 0x09, 0x5a, 0xa8, 0xf6, 0xd2, 0xe0, 0x32, 0x38, + 0x96, 0x34, 0x67, 0x1c, 0xaa, 0xc7, 0x1b, 0x7c, 0xac, 0x4f, 0x85, 0x1e, 0xeb, 0x6b, 0x50, 0xe8, + 0x9a, 0xd8, 0xdc, 0x37, 0x9b, 0xa6, 0xdb, 0x1b, 0xef, 0x1d, 0x39, 0xef, 0x33, 0xd2, 0xe3, 0xc2, + 0x3c, 0xc8, 0x41, 0xdd, 0xf8, 0x12, 0x7c, 0x2c, 0xc1, 0x85, 0x6d, 0xe4, 0xaa, 0xfe, 0x17, 0x51, + 0x0f, 0xd9, 0xd7, 0x50, 0xde, 0x59, 0xe7, 0x01, 0x4c, 0xd3, 0x72, 0x14, 0x12, 0xb2, 0xa9, 0xbe, + 0x2e, 0x19, 0xf8, 0xa4, 0x8a, 0xe5, 0x3d, 0xbc, 0x9f, 0xb4, 0x70, 0x45, 0xe5, 0x32, 0x48, 0x20, + 0xf3, 0x23, 0x13, 0x7d, 0x25, 0xe6, 0xe7, 0x8b, 0x2c, 0x6f, 0x23, 0xbe, 0xac, 0xfc, 0x68, 0x02, + 0x2a, 0xfd, 0xa6, 0xc4, 0x97, 0xfd, 0xb7, 0x20, 0xcf, 0x96, 0x84, 0x7f, 0xba, 0x25, 0xe6, 0xf6, + 0xee, 0x88, 0xcf, 0xaa, 0x83, 0xc5, 0x33, 0xe7, 0x10, 0xad, 0xac, 0x04, 0x85, 0x45, 0xb8, 0x68, + 0x5b, 0xe9, 0x81, 0x1c, 0x27, 0x0a, 0x96, 0xa3, 0x4c, 0xb1, 0x72, 0x94, 0x87, 0xe1, 0x72, 0x94, + 0xd7, 0xc7, 0xb4, 0x9d, 0x37, 0x33, 0xbf, 0x42, 0x45, 0xf9, 0x08, 0xd6, 0xb6, 0x91, 0x7b, 0xfb, + 0xc1, 0xdb, 0x03, 0xd6, 0xec, 0x31, 0xaf, 0xa4, 0x25, 0x51, 0x21, 0x6c, 0x33, 0xee, 0xd8, 0x5e, + 0x45, 0x14, 0x2d, 0xae, 0x25, 0x7f, 0x61, 0xe5, 0xf7, 0x24, 0x58, 0x1f, 0x30, 0x38, 0x5f, 0x9d, + 0x0f, 0xa0, 0x14, 0x10, 0x4b, 0x13, 0x23, 0x62, 0x12, 0xd7, 0x4f, 0x31, 0x09, 0xb5, 0xe8, 0x84, + 0x1b, 0xb0, 0xf2, 0x03, 0x09, 0xe6, 0x69, 0xe9, 0x8e, 0xc0, 0xef, 0x31, 0xf6, 0xfa, 0x6f, 0x46, + 0xef, 0xdf, 0x5f, 0x1d, 0x7a, 0xff, 0x4e, 0x1a, 0xca, 0xbf, 0x73, 0x1f, 0xc1, 0x42, 0x84, 0x80, + 0xdb, 0x41, 0x85, 0x74, 0xe4, 0xd9, 0xff, 0xb5, 0x71, 0x87, 0xe2, 0x8f, 0xfc, 0x9e, 0x1c, 0xe5, + 0x0f, 0x25, 0x98, 0x57, 0x91, 0xde, 0x6e, 0x37, 0x59, 0x42, 0x03, 0x8f, 0xa1, 0xf9, 0x6e, 0x54, + 0xf3, 0xe4, 0x32, 0xb9, 0xe0, 0xd7, 0x83, 0x6c, 0x39, 0xe2, 0xc3, 0xf9, 0xda, 0x2f, 0xc1, 0x42, + 0x84, 0x80, 0xcf, 0xf4, 0x2f, 0x27, 0x60, 0x81, 0xf9, 0x4a, 0xd4, 0x3b, 0xef, 0xc0, 0xa4, 0x57, + 0x06, 0x99, 0x0f, 0xa6, 0x1c, 0x92, 0x10, 0xf3, 0x36, 0xd2, 0x8d, 0x07, 0xc8, 0x75, 0x91, 0x43, + 0x2b, 0x8a, 0x68, 0xe5, 0x09, 0x65, 0x1f, 0x74, 0x5c, 0x88, 0xdf, 0xcf, 0x52, 0x49, 0xf7, 0xb3, + 0xd7, 0xa1, 0x6c, 0x5a, 0x84, 0xc2, 0xec, 0x22, 0x0d, 0x59, 0x1e, 0x9c, 0xf8, 0x45, 0x53, 0x0b, + 0x5e, 0xff, 0x1d, 0x4b, 0x04, 0x7b, 0xcd, 0x90, 0x5f, 0x82, 0x52, 0x4b, 0x3f, 0x31, 0x5b, 0x9d, + 0x96, 0xd6, 0x26, 0xf4, 0xd8, 0xfc, 0x88, 0x7d, 0xfa, 0x37, 0xa5, 0x16, 0x78, 0xc7, 0x8e, 0x7e, + 0x88, 0x76, 0xcd, 0x8f, 0x90, 0xfc, 0x02, 0x14, 0x68, 0x7d, 0x24, 0x25, 0x64, 0x85, 0x7d, 0xd3, + 0xb4, 0xb0, 0x8f, 0x96, 0x4d, 0x12, 0x32, 0xf6, 0xf1, 0xc0, 0x7f, 0xb2, 0xcf, 0xc8, 0x42, 0xf6, + 0xe2, 0x8e, 0xf4, 0x94, 0x0c, 0x96, 0x18, 0x97, 0x13, 0x4f, 0x31, 0x2e, 0x93, 0x74, 0x4d, 0x25, + 0xe9, 0xfa, 0x2f, 0x12, 0x2c, 0xed, 0x74, 0x9c, 0x43, 0xf4, 0x45, 0xf4, 0x0e, 0x65, 0x05, 0xca, + 0x71, 0xe5, 0x44, 0x51, 0xc3, 0x04, 0x2c, 0x3d, 0x44, 0x5f, 0x50, 0xcd, 0x9f, 0x49, 0x5c, 0xdc, + 0x82, 0x72, 0xdc, 0x60, 0x3c, 0x30, 0x12, 0x64, 0x48, 0x49, 0x32, 0x7e, 0x44, 0x0b, 0xf6, 0x0f, + 0x1c, 0x84, 0x1b, 0xc1, 0xdc, 0xfb, 0x38, 0xe0, 0xf9, 0x5e, 0x14, 0x3c, 0x7f, 0x7d, 0x44, 0xf0, + 0xec, 0x3b, 0xaa, 0x8f, 0xa1, 0xb4, 0x86, 0x3f, 0x89, 0x8e, 0x3b, 0xcd, 0x0f, 0x25, 0x78, 0x69, + 0x1b, 0x59, 0xc8, 0xd1, 0x5d, 0xf4, 0x40, 0xc7, 0xae, 0xb8, 0x21, 0x47, 0xc2, 0xef, 0xf3, 0xb8, + 0xf0, 0x5e, 0x81, 0x97, 0x47, 0x9a, 0x19, 0xd7, 0xe4, 0x2e, 0xac, 0x86, 0xcf, 0x5e, 0xe1, 0xbc, + 0xda, 0x65, 0x28, 0x38, 0xa8, 0x65, 0xbb, 0x9e, 0x7f, 0xb2, 0x73, 0x43, 0x46, 0xcd, 0xb3, 0x66, + 0xee, 0xa0, 0x58, 0xe9, 0xc0, 0xf9, 0x64, 0x39, 0xdc, 0x31, 0xbe, 0x05, 0xd3, 0xec, 0xf6, 0xc5, + 0xcf, 0x1d, 0x6f, 0x8e, 0x78, 0x30, 0xe4, 0xb7, 0x8b, 0xa8, 0x58, 0x2e, 0x4c, 0xf9, 0xfb, 0x29, + 0x58, 0x4c, 0x26, 0x19, 0x74, 0x4b, 0xf8, 0x2a, 0x2c, 0xb5, 0xf4, 0x13, 0x2d, 0x8a, 0xbd, 0x7e, + 0xc9, 0xfe, 0x7c, 0x4b, 0x3f, 0x89, 0x9e, 0xbc, 0x0c, 0xf9, 0x3e, 0x14, 0x99, 0xc4, 0xa6, 0x5d, + 0xd7, 0x9b, 0xe3, 0xe5, 0x09, 0xd9, 0xf1, 0xf8, 0x01, 0x61, 0xa4, 0x19, 0xa6, 0x8f, 0xe2, 0x86, + 0x65, 0xb9, 0xec, 0xb7, 0xcf, 0x64, 0x98, 0xaa, 0x1a, 0x5a, 0x16, 0x76, 0x54, 0x8e, 0xac, 0x95, + 0xfc, 0xfb, 0x12, 0xcc, 0x35, 0x74, 0xcb, 0xb0, 0xbb, 0xfc, 0xd0, 0x4f, 0x9d, 0x90, 0x5c, 0x29, + 0xc7, 0x29, 0x19, 0xef, 0x33, 0x81, 0x7b, 0x5c, 0xb0, 0x77, 0x0b, 0xe6, 0x93, 0x90, 0x1b, 0xb1, + 0x8e, 0x95, 0x1f, 0x48, 0x30, 0x97, 0x30, 0xe1, 0x84, 0x2a, 0xf2, 0xf7, 0xc3, 0xc7, 0xf6, 0xed, + 0x33, 0xcd, 0x71, 0x07, 0x39, 0x7c, 0xbc, 0xc0, 0x31, 0x7e, 0xe5, 0x7b, 0x12, 0x2c, 0xf5, 0x99, + 0x7c, 0xc2, 0x84, 0xd4, 0xf0, 0x84, 0xbe, 0x3e, 0xe2, 0x84, 0x62, 0x03, 0xd0, 0x03, 0x7d, 0xe0, + 0x32, 0xf1, 0x2e, 0x2c, 0x24, 0xd2, 0xc8, 0x6f, 0xc1, 0x79, 0x6f, 0xcd, 0x92, 0x1c, 0x57, 0xa2, + 0x8e, 0xbb, 0x2c, 0x68, 0x62, 0xde, 0xab, 0xfc, 0x99, 0x04, 0x6b, 0xc3, 0xec, 0x21, 0x2b, 0x30, + 0xab, 0xd7, 0x8f, 0x90, 0x11, 0x11, 0x9b, 0xa5, 0x8d, 0x3c, 0x0c, 0xde, 0x87, 0x95, 0x00, 0x4d, + 0xf4, 0x36, 0x3c, 0x6a, 0x19, 0xf7, 0x92, 0x27, 0xf2, 0x71, 0xe8, 0x5a, 0x7c, 0xab, 0xfd, 0xc9, + 0xa7, 0x95, 0x73, 0x3f, 0xfd, 0xb4, 0x72, 0xee, 0xe7, 0x9f, 0x56, 0xa4, 0xdf, 0x7e, 0x52, 0x91, + 0x7e, 0xfc, 0xa4, 0x22, 0xfd, 0xdd, 0x93, 0x8a, 0xf4, 0xc9, 0x93, 0x8a, 0xf4, 0x6f, 0x4f, 0x2a, + 0xd2, 0x7f, 0x3c, 0xa9, 0x9c, 0xfb, 0xf9, 0x93, 0x8a, 0xf4, 0xf1, 0x67, 0x95, 0x73, 0x9f, 0x7c, + 0x56, 0x39, 0xf7, 0xd3, 0xcf, 0x2a, 0xe7, 0xde, 0xbb, 0x71, 0x68, 0xfb, 0x4b, 0x60, 0xda, 0x03, + 0xff, 0x87, 0xce, 0xaf, 0x85, 0x5b, 0xf6, 0xa7, 0xe9, 0x24, 0xaf, 0xff, 0x5f, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x1e, 0xac, 0x9f, 0x4c, 0x82, 0x47, 0x00, 0x00, } func (this *StartWorkflowExecutionRequest) Equal(that interface{}) bool { @@ -6044,6 +6133,54 @@ func (this *TerminateWorkflowExecutionResponse) Equal(that interface{}) bool { } return true } +func (this *DeleteWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DeleteWorkflowExecutionRequest) + if !ok { + that2, ok := that.(DeleteWorkflowExecutionRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.NamespaceId != that1.NamespaceId { + return false + } + if !this.WorkflowExecution.Equal(that1.WorkflowExecution) { + return false + } + return true +} +func (this *DeleteWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DeleteWorkflowExecutionResponse) + if !ok { + that2, ok := that.(DeleteWorkflowExecutionResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + return true +} func (this *ResetWorkflowExecutionRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -7977,6 +8114,28 @@ func (this *TerminateWorkflowExecutionResponse) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *DeleteWorkflowExecutionRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&historyservice.DeleteWorkflowExecutionRequest{") + s = append(s, "NamespaceId: "+fmt.Sprintf("%#v", this.NamespaceId)+",\n") + if this.WorkflowExecution != nil { + s = append(s, "WorkflowExecution: "+fmt.Sprintf("%#v", this.WorkflowExecution)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *DeleteWorkflowExecutionResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 4) + s = append(s, "&historyservice.DeleteWorkflowExecutionResponse{") + s = append(s, "}") + return strings.Join(s, "") +} func (this *ResetWorkflowExecutionRequest) GoString() string { if this == nil { return "nil" @@ -10343,6 +10502,71 @@ func (m *TerminateWorkflowExecutionResponse) MarshalToSizedBuffer(dAtA []byte) ( return len(dAtA) - i, nil } +func (m *DeleteWorkflowExecutionRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DeleteWorkflowExecutionRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeleteWorkflowExecutionRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.WorkflowExecution != nil { + { + size, err := m.WorkflowExecution.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRequestResponse(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.NamespaceId) > 0 { + i -= len(m.NamespaceId) + copy(dAtA[i:], m.NamespaceId) + i = encodeVarintRequestResponse(dAtA, i, uint64(len(m.NamespaceId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *DeleteWorkflowExecutionResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DeleteWorkflowExecutionResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeleteWorkflowExecutionResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + return len(dAtA) - i, nil +} + func (m *ResetWorkflowExecutionRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -10917,12 +11141,12 @@ func (m *SyncShardStatusRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) var l int _ = l if m.StatusTime != nil { - n63, err63 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StatusTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StatusTime):]) - if err63 != nil { - return 0, err63 + n64, err64 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StatusTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StatusTime):]) + if err64 != nil { + return 0, err64 } - i -= n63 - i = encodeVarintRequestResponse(dAtA, i, uint64(n63)) + i -= n64 + i = encodeVarintRequestResponse(dAtA, i, uint64(n64)) i-- dAtA[i] = 0x1a } @@ -11033,22 +11257,22 @@ func (m *SyncActivityRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x52 } if m.LastHeartbeatTime != nil { - n67, err67 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.LastHeartbeatTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.LastHeartbeatTime):]) - if err67 != nil { - return 0, err67 + n68, err68 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.LastHeartbeatTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.LastHeartbeatTime):]) + if err68 != nil { + return 0, err68 } - i -= n67 - i = encodeVarintRequestResponse(dAtA, i, uint64(n67)) + i -= n68 + i = encodeVarintRequestResponse(dAtA, i, uint64(n68)) i-- dAtA[i] = 0x4a } if m.StartedTime != nil { - n68, err68 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StartedTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StartedTime):]) - if err68 != nil { - return 0, err68 + n69, err69 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.StartedTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.StartedTime):]) + if err69 != nil { + return 0, err69 } - i -= n68 - i = encodeVarintRequestResponse(dAtA, i, uint64(n68)) + i -= n69 + i = encodeVarintRequestResponse(dAtA, i, uint64(n69)) i-- dAtA[i] = 0x42 } @@ -11058,12 +11282,12 @@ func (m *SyncActivityRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x38 } if m.ScheduledTime != nil { - n69, err69 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ScheduledTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ScheduledTime):]) - if err69 != nil { - return 0, err69 + n70, err70 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ScheduledTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ScheduledTime):]) + if err70 != nil { + return 0, err70 } - i -= n69 - i = encodeVarintRequestResponse(dAtA, i, uint64(n69)) + i -= n70 + i = encodeVarintRequestResponse(dAtA, i, uint64(n70)) i-- dAtA[i] = 0x32 } @@ -11314,21 +11538,21 @@ func (m *DescribeHistoryHostResponse) MarshalToSizedBuffer(dAtA []byte) (int, er dAtA[i] = 0x1a } if len(m.ShardIds) > 0 { - dAtA76 := make([]byte, len(m.ShardIds)*10) - var j75 int + dAtA77 := make([]byte, len(m.ShardIds)*10) + var j76 int for _, num1 := range m.ShardIds { num := uint64(num1) for num >= 1<<7 { - dAtA76[j75] = uint8(uint64(num)&0x7f | 0x80) + dAtA77[j76] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j75++ + j76++ } - dAtA76[j75] = uint8(num) - j75++ + dAtA77[j76] = uint8(num) + j76++ } - i -= j75 - copy(dAtA[i:], dAtA76[:j75]) - i = encodeVarintRequestResponse(dAtA, i, uint64(j75)) + i -= j76 + copy(dAtA[i:], dAtA77[:j76]) + i = encodeVarintRequestResponse(dAtA, i, uint64(j76)) i-- dAtA[i] = 0x12 } @@ -11475,12 +11699,12 @@ func (m *RemoveTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.VisibilityTime != nil { - n78, err78 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.VisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.VisibilityTime):]) - if err78 != nil { - return 0, err78 + n79, err79 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.VisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.VisibilityTime):]) + if err79 != nil { + return 0, err79 } - i -= n78 - i = encodeVarintRequestResponse(dAtA, i, uint64(n78)) + i -= n79 + i = encodeVarintRequestResponse(dAtA, i, uint64(n79)) i-- dAtA[i] = 0x22 } @@ -12365,12 +12589,12 @@ func (m *ShardReplicationStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) } } if m.ShardLocalTime != nil { - n87, err87 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ShardLocalTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ShardLocalTime):]) - if err87 != nil { - return 0, err87 + n88, err88 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ShardLocalTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.ShardLocalTime):]) + if err88 != nil { + return 0, err88 } - i -= n87 - i = encodeVarintRequestResponse(dAtA, i, uint64(n87)) + i -= n88 + i = encodeVarintRequestResponse(dAtA, i, uint64(n88)) i-- dAtA[i] = 0x1a } @@ -12436,12 +12660,12 @@ func (m *ShardReplicationStatusPerCluster) MarshalToSizedBuffer(dAtA []byte) (in var l int _ = l if m.AckedTaskVisibilityTime != nil { - n88, err88 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.AckedTaskVisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.AckedTaskVisibilityTime):]) - if err88 != nil { - return 0, err88 + n89, err89 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.AckedTaskVisibilityTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.AckedTaskVisibilityTime):]) + if err89 != nil { + return 0, err89 } - i -= n88 - i = encodeVarintRequestResponse(dAtA, i, uint64(n88)) + i -= n89 + i = encodeVarintRequestResponse(dAtA, i, uint64(n89)) i-- dAtA[i] = 0x12 } @@ -13158,6 +13382,32 @@ func (m *TerminateWorkflowExecutionResponse) Size() (n int) { return n } +func (m *DeleteWorkflowExecutionRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.NamespaceId) + if l > 0 { + n += 1 + l + sovRequestResponse(uint64(l)) + } + if m.WorkflowExecution != nil { + l = m.WorkflowExecution.Size() + n += 1 + l + sovRequestResponse(uint64(l)) + } + return n +} + +func (m *DeleteWorkflowExecutionResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func (m *ResetWorkflowExecutionRequest) Size() (n int) { if m == nil { return 0 @@ -14458,6 +14708,26 @@ func (this *TerminateWorkflowExecutionResponse) String() string { }, "") return s } +func (this *DeleteWorkflowExecutionRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&DeleteWorkflowExecutionRequest{`, + `NamespaceId:` + fmt.Sprintf("%v", this.NamespaceId) + `,`, + `WorkflowExecution:` + strings.Replace(fmt.Sprintf("%v", this.WorkflowExecution), "WorkflowExecution", "v14.WorkflowExecution", 1) + `,`, + `}`, + }, "") + return s +} +func (this *DeleteWorkflowExecutionResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&DeleteWorkflowExecutionResponse{`, + `}`, + }, "") + return s +} func (this *ResetWorkflowExecutionRequest) String() string { if this == nil { return "nil" @@ -20106,6 +20376,180 @@ func (m *TerminateWorkflowExecutionResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *DeleteWorkflowExecutionRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteWorkflowExecutionRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteWorkflowExecutionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NamespaceId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRequestResponse + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRequestResponse + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NamespaceId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowExecution", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRequestResponse + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRequestResponse + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WorkflowExecution == nil { + m.WorkflowExecution = &v14.WorkflowExecution{} + } + if err := m.WorkflowExecution.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRequestResponse(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DeleteWorkflowExecutionResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRequestResponse + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteWorkflowExecutionResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteWorkflowExecutionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipRequestResponse(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthRequestResponse + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ResetWorkflowExecutionRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/common/persistence/operationModeValidator.go b/common/persistence/operationModeValidator.go index 1902ba03a1c..c4838dfae41 100644 --- a/common/persistence/operationModeValidator.go +++ b/common/persistence/operationModeValidator.go @@ -316,8 +316,7 @@ func checkWorkflowState(state enumsspb.WorkflowExecutionState) error { enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING, enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE, enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, - enumsspb.WORKFLOW_EXECUTION_STATE_CORRUPTED, - enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: + enumsspb.WORKFLOW_EXECUTION_STATE_CORRUPTED: return nil default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) diff --git a/common/persistence/workflowStateStatusValidator.go b/common/persistence/workflowStateStatusValidator.go index 942558a5072..850774b927a 100644 --- a/common/persistence/workflowStateStatusValidator.go +++ b/common/persistence/workflowStateStatusValidator.go @@ -40,7 +40,6 @@ var ( enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED: {}, enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: {}, enumsspb.WORKFLOW_EXECUTION_STATE_CORRUPTED: {}, - enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: {}, } validWorkflowStatuses = map[enumspb.WorkflowExecutionStatus]struct{}{ @@ -67,10 +66,6 @@ func ValidateCreateWorkflowStateStatus( return err } - if state == enumsspb.WORKFLOW_EXECUTION_STATE_DELETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING { - return nil - } - // validate workflow state & status if (state == enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status == enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) || (state != enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) { @@ -92,10 +87,6 @@ func ValidateUpdateWorkflowStateStatus( return err } - if state == enumsspb.WORKFLOW_EXECUTION_STATE_DELETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING { - return nil - } - // validate workflow state & status if (state == enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status == enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) || (state != enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED && status != enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING) { diff --git a/proto/internal/temporal/server/api/enums/v1/workflow.proto b/proto/internal/temporal/server/api/enums/v1/workflow.proto index 40dc83e6940..d18793b6a70 100644 --- a/proto/internal/temporal/server/api/enums/v1/workflow.proto +++ b/proto/internal/temporal/server/api/enums/v1/workflow.proto @@ -32,7 +32,6 @@ enum WorkflowExecutionState { WORKFLOW_EXECUTION_STATE_ZOMBIE = 4; WORKFLOW_EXECUTION_STATE_VOID = 5; WORKFLOW_EXECUTION_STATE_CORRUPTED = 6; - WORKFLOW_EXECUTION_STATE_DELETED = 7; } enum WorkflowBackoffType { diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 595e7c0af94..6782958ba70 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -2269,7 +2269,7 @@ func (e *historyEngineImpl) DeleteWorkflowExecution( *deleteRequest.GetWorkflowExecution(), func(weCtx workflow.Context, mutableState workflow.MutableState) (*updateWorkflowAction, error) { if mutableState.IsWorkflowExecutionRunning() { - return nil, consts.ErrWorkflowIsRunning // workflow is running, cannot be deleted + return nil, consts.ErrWorkflowNotCompleted // workflow is running, cannot be deleted } taskGenerator := workflow.NewTaskGenerator( @@ -2282,13 +2282,6 @@ func (e *historyEngineImpl) DeleteWorkflowExecution( if err != nil { return nil, err } - state, status := mutableState.GetWorkflowStateStatus() - if state != enumsspb.WORKFLOW_EXECUTION_STATE_DELETED { - err = mutableState.UpdateWorkflowStateStatus(enumsspb.WORKFLOW_EXECUTION_STATE_DELETED, status) - if err != nil { - return nil, err - } - } return updateWorkflowWithoutWorkflowTask, nil }) diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index caf22149e93..1620f0ca2c5 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -246,7 +246,7 @@ func (t *transferQueueTaskExecutorBase) processDeleteExecutionTask( return err } - return t.workflowDeleteManager.DeleteDeletedWorkflowExecution( + return t.workflowDeleteManager.DeleteWorkflowExecution( namespace.ID(task.GetNamespaceID()), workflowExecution, weCtx, diff --git a/service/history/workflow/mutable_state_impl.go b/service/history/workflow/mutable_state_impl.go index 795aefa859b..81cea4c8ca9 100644 --- a/service/history/workflow/mutable_state_impl.go +++ b/service/history/workflow/mutable_state_impl.go @@ -1250,8 +1250,6 @@ func (e *MutableStateImpl) IsWorkflowExecutionRunning() bool { return true case enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING: return true - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - return false case enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED: return false case enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: diff --git a/service/history/workflow/mutable_state_state_status.go b/service/history/workflow/mutable_state_state_status.go index fe738cf1700..80c1c79d4cf 100644 --- a/service/history/workflow/mutable_state_state_status.go +++ b/service/history/workflow/mutable_state_state_status.go @@ -72,9 +72,6 @@ func setStateStatus( return invalidStateTransitionErr(e.GetState(), state, status) } - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - return invalidStateTransitionErr(e.GetState(), state, status) - default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } @@ -98,9 +95,6 @@ func setStateStatus( return invalidStateTransitionErr(e.GetState(), state, status) } - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - return invalidStateTransitionErr(e.GetState(), state, status) - default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } @@ -120,8 +114,6 @@ func setStateStatus( case enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: return invalidStateTransitionErr(e.GetState(), state, status) - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } @@ -147,25 +139,9 @@ func setStateStatus( return invalidStateTransitionErr(e.GetState(), state, status) } - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - - default: - return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) - } - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - switch state { - case enumsspb.WORKFLOW_EXECUTION_STATE_CREATED, - enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING, - enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, - enumsspb.WORKFLOW_EXECUTION_STATE_ZOMBIE: - return invalidStateTransitionErr(e.GetState(), state, status) - - case enumsspb.WORKFLOW_EXECUTION_STATE_DELETED: - default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } - default: return serviceerror.NewInternal(fmt.Sprintf("unknown workflow state: %v", state)) } From b16bd6099c81910ec65f4958054954c2a360fb69 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Tue, 25 Jan 2022 11:26:40 -0800 Subject: [PATCH 10/12] Fix comments --- .../temporal/server/api/historyservice/v1/service.proto | 5 ++--- service/history/workflow/delete_manager.go | 4 ++++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/proto/internal/temporal/server/api/historyservice/v1/service.proto b/proto/internal/temporal/server/api/historyservice/v1/service.proto index de7286a823f..e216e95d3c8 100644 --- a/proto/internal/temporal/server/api/historyservice/v1/service.proto +++ b/proto/internal/temporal/server/api/historyservice/v1/service.proto @@ -139,9 +139,8 @@ service HistoryService { rpc TerminateWorkflowExecution (TerminateWorkflowExecutionRequest) returns (TerminateWorkflowExecutionResponse) { } - // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history synchronously. - // Visibility record is deleted asynchronously. - // If call to DeleteWorkflowExecution fails, it must be retried to prevent orphan DB records. + // DeleteWorkflowExecution deletes a closed workflow execution asynchronously (workflow must be completed or terminated before). + // This method is EXPERIMENTAL and may be changed or removed in a later release. rpc DeleteWorkflowExecution (DeleteWorkflowExecutionRequest) returns (DeleteWorkflowExecutionResponse) { } diff --git a/service/history/workflow/delete_manager.go b/service/history/workflow/delete_manager.go index f51b891ccbb..462d0aa47b1 100644 --- a/service/history/workflow/delete_manager.go +++ b/service/history/workflow/delete_manager.go @@ -86,6 +86,10 @@ func (m *DeleteManagerImpl) DeleteWorkflowExecution( ) error { if ms.IsWorkflowExecutionRunning() { + // DeleteWorkflowExecution is called from transfer task queue processor + // and corresponding transfer task is created only if workflow is not running. + // Therefore, this should almost never happen but if it does (cross DC resurrection, for example), + // workflow should not be deleted. NotFound errors are ignored by task processor. return consts.ErrWorkflowNotCompleted } From 94f2b431e16f2a335f74b2bd01bb2b2bd6c3478c Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Tue, 25 Jan 2022 11:35:27 -0800 Subject: [PATCH 11/12] Rebuild proot --- api/historyservice/v1/service.pb.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/api/historyservice/v1/service.pb.go b/api/historyservice/v1/service.pb.go index 53f2f1c8c74..f9502d817a5 100644 --- a/api/historyservice/v1/service.pb.go +++ b/api/historyservice/v1/service.pb.go @@ -218,9 +218,8 @@ type HistoryServiceClient interface { // TerminateWorkflowExecution terminates an existing workflow execution by recording WorkflowExecutionTerminated event // in the history and immediately terminating the execution instance. TerminateWorkflowExecution(ctx context.Context, in *TerminateWorkflowExecutionRequest, opts ...grpc.CallOption) (*TerminateWorkflowExecutionResponse, error) - // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history synchronously. - // Visibility record is deleted asynchronously. - // If call to DeleteWorkflowExecution fails, it must be retried to prevent orphan DB records. + // DeleteWorkflowExecution deletes a closed workflow execution asynchronously (workflow must be completed or terminated before). + // This method is EXPERIMENTAL and may be changed or removed in a later release. DeleteWorkflowExecution(ctx context.Context, in *DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*DeleteWorkflowExecutionResponse, error) // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. @@ -727,9 +726,8 @@ type HistoryServiceServer interface { // TerminateWorkflowExecution terminates an existing workflow execution by recording WorkflowExecutionTerminated event // in the history and immediately terminating the execution instance. TerminateWorkflowExecution(context.Context, *TerminateWorkflowExecutionRequest) (*TerminateWorkflowExecutionResponse, error) - // DeleteWorkflowExecution delete closed workflow execution (workflow must be completed or terminated before) and its history synchronously. - // Visibility record is deleted asynchronously. - // If call to DeleteWorkflowExecution fails, it must be retried to prevent orphan DB records. + // DeleteWorkflowExecution deletes a closed workflow execution asynchronously (workflow must be completed or terminated before). + // This method is EXPERIMENTAL and may be changed or removed in a later release. DeleteWorkflowExecution(context.Context, *DeleteWorkflowExecutionRequest) (*DeleteWorkflowExecutionResponse, error) // ResetWorkflowExecution reset an existing workflow execution by a firstEventId of a existing event batch // in the history and immediately terminating the current execution instance. From 9f15394168a48e2d5bf2b0328163a43fc9267521 Mon Sep 17 00:00:00 2001 From: Alex Shtin Date: Thu, 27 Jan 2022 17:47:34 -0800 Subject: [PATCH 12/12] Remove default values set --- .../serialization/task_serializer.go | 20 +++++++------------ 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/common/persistence/serialization/task_serializer.go b/common/persistence/serialization/task_serializer.go index 3027e89115f..1d42a33ea4c 100644 --- a/common/persistence/serialization/task_serializer.go +++ b/common/persistence/serialization/task_serializer.go @@ -603,19 +603,13 @@ func (s *TaskSerializer) TransferDeleteExecutionTaskToProto( deleteExecutionTask *tasks.DeleteExecutionTask, ) *persistencespb.TransferTaskInfo { return &persistencespb.TransferTaskInfo{ - NamespaceId: deleteExecutionTask.WorkflowKey.NamespaceID, - WorkflowId: deleteExecutionTask.WorkflowKey.WorkflowID, - RunId: deleteExecutionTask.WorkflowKey.RunID, - TaskType: enumsspb.TASK_TYPE_TRANSFER_DELETE_EXECUTION, - TargetNamespaceId: "", - TargetWorkflowId: "", - TargetRunId: "", - TargetChildWorkflowOnly: false, - TaskQueue: "", - ScheduleId: 0, - Version: deleteExecutionTask.Version, - TaskId: deleteExecutionTask.TaskID, - VisibilityTime: timestamp.TimePtr(deleteExecutionTask.VisibilityTimestamp), + NamespaceId: deleteExecutionTask.WorkflowKey.NamespaceID, + WorkflowId: deleteExecutionTask.WorkflowKey.WorkflowID, + RunId: deleteExecutionTask.WorkflowKey.RunID, + TaskType: enumsspb.TASK_TYPE_TRANSFER_DELETE_EXECUTION, + Version: deleteExecutionTask.Version, + TaskId: deleteExecutionTask.TaskID, + VisibilityTime: timestamp.TimePtr(deleteExecutionTask.VisibilityTimestamp), } }