From d9dc526bbc55905c87105e8b527e2a28504341d0 Mon Sep 17 00:00:00 2001 From: Yichao Yang Date: Fri, 24 Jul 2020 14:45:43 -0700 Subject: [PATCH 1/6] Task Redispatcher (#3406) * Create a new component for all task redispatching logic * Assign task priority only once * Stop redispatching tasks for a certain priority if task ch for that priority is full * Redispatch only part of the tasks when redispatch queue is full * Throttle/backoff task loading when redispatch queue still has too many tasks after redispatching --- common/service/dynamicconfig/constants.go | 292 +++++++++--------- common/task/interface.go | 5 - common/task/priority.go | 5 + service/history/config/config.go | 100 +++--- service/history/queue/processor_base.go | 94 +----- .../queue/timer_queue_processor_base.go | 26 +- .../queue/transfer_queue_processor_base.go | 24 +- service/history/queueProcessor.go | 117 ++----- service/history/replicatorQueueProcessor.go | 4 +- service/history/task/interface.go | 9 + service/history/task/interface_mock.go | 85 +++++ service/history/task/priority_assigner.go | 4 + .../history/task/priority_assigner_test.go | 20 +- service/history/task/redispatcher.go | 285 +++++++++++++++++ service/history/task/redispatcher_test.go | 163 ++++++++++ service/history/task/task.go | 1 + service/history/timerQueueProcessorBase.go | 134 ++------ .../history/transferQueueActiveProcessor.go | 8 +- .../history/transferQueueStandbyProcessor.go | 4 +- 19 files changed, 876 insertions(+), 504 deletions(-) create mode 100644 service/history/task/redispatcher.go create mode 100644 service/history/task/redispatcher_test.go diff --git a/common/service/dynamicconfig/constants.go b/common/service/dynamicconfig/constants.go index 4328b903618..87d37d35ffe 100644 --- a/common/service/dynamicconfig/constants.go +++ b/common/service/dynamicconfig/constants.go @@ -136,146 +136,142 @@ var keys = map[Key]string{ MatchingShutdownDrainDuration: "matching.shutdownDrainDuration", // history settings - HistoryRPS: "history.rps", - HistoryPersistenceMaxQPS: "history.persistenceMaxQPS", - HistoryPersistenceGlobalMaxQPS: "history.persistenceGlobalMaxQPS", - HistoryVisibilityOpenMaxQPS: "history.historyVisibilityOpenMaxQPS", - HistoryVisibilityClosedMaxQPS: "history.historyVisibilityClosedMaxQPS", - HistoryLongPollExpirationInterval: "history.longPollExpirationInterval", - HistoryCacheInitialSize: "history.cacheInitialSize", - HistoryMaxAutoResetPoints: "history.historyMaxAutoResetPoints", - HistoryCacheMaxSize: "history.cacheMaxSize", - HistoryCacheTTL: "history.cacheTTL", - HistoryShutdownDrainDuration: "history.shutdownDrainDuration", - EventsCacheInitialCount: "history.eventsCacheInitialSize", - EventsCacheMaxCount: "history.eventsCacheMaxSize", - EventsCacheMaxSize: "history.eventsCacheMaxSizeInBytes", - EventsCacheTTL: "history.eventsCacheTTL", - EventsCacheGlobalEnable: "history.eventsCacheGlobalEnable", - EventsCacheGlobalInitialCount: "history.eventsCacheGlobalInitialSize", - EventsCacheGlobalMaxCount: "history.eventsCacheGlobalMaxSize", - AcquireShardInterval: "history.acquireShardInterval", - AcquireShardConcurrency: "history.acquireShardConcurrency", - StandbyClusterDelay: "history.standbyClusterDelay", - StandbyTaskMissingEventsResendDelay: "history.standbyTaskMissingEventsResendDelay", - StandbyTaskMissingEventsDiscardDelay: "history.standbyTaskMissingEventsDiscardDelay", - TaskProcessRPS: "history.taskProcessRPS", - TaskSchedulerType: "history.taskSchedulerType", - TaskSchedulerWorkerCount: "history.taskSchedulerWorkerCount", - TaskSchedulerShardWorkerCount: "history.taskSchedulerShardWorkerCount", - TaskSchedulerQueueSize: "history.taskSchedulerQueueSize", - TaskSchedulerShardQueueSize: "history.taskSchedulerShardQueueSize", - TaskSchedulerDispatcherCount: "history.taskSchedulerDispatcherCount", - TaskSchedulerRoundRobinWeights: "history.taskSchedulerRoundRobinWeight", - QueueProcessorEnableDomainTaggedMetrics: "history.queueProcessorEnableDomainTaggedMetrics", - QueueProcessorEnableSplit: "history.queueProcessorEnableSplit", - QueueProcessorSplitMaxLevel: "history.queueProcessorSplitMaxLevel", - QueueProcessorEnableRandomSplitByDomainID: "history.queueProcessorEnableRandomSplitByDomain", - QueueProcessorRandomSplitProbability: "history.queueProcessorRandomSplitProbability", - QueueProcessorEnablePendingTaskSplit: "history.queueProcessorEnablePendingTaskSplit", - QueueProcessorPendingTaskSplitThreshold: "history.queueProcessorPendingTaskSplitThreshold", - QueueProcessorEnableStuckTaskSplit: "history.queueProcessorEnableStuckTaskSplit", - QueueProcessorStuckTaskSplitThreshold: "history.queueProcessorStuckTaskSplitThreshold", - QueueProcessorSplitLookAheadDurationByDomainID: "history.queueProcessorSplitLookAheadDuration", - TimerTaskBatchSize: "history.timerTaskBatchSize", - TimerTaskWorkerCount: "history.timerTaskWorkerCount", - TimerTaskMaxRetryCount: "history.timerTaskMaxRetryCount", - TimerProcessorGetFailureRetryCount: "history.timerProcessorGetFailureRetryCount", - TimerProcessorCompleteTimerFailureRetryCount: "history.timerProcessorCompleteTimerFailureRetryCount", - TimerProcessorUpdateShardTaskCount: "history.timerProcessorUpdateShardTaskCount", - TimerProcessorUpdateAckInterval: "history.timerProcessorUpdateAckInterval", - TimerProcessorUpdateAckIntervalJitterCoefficient: "history.timerProcessorUpdateAckIntervalJitterCoefficient", - TimerProcessorCompleteTimerInterval: "history.timerProcessorCompleteTimerInterval", - TimerProcessorFailoverMaxPollRPS: "history.timerProcessorFailoverMaxPollRPS", - TimerProcessorMaxPollRPS: "history.timerProcessorMaxPollRPS", - TimerProcessorMaxPollInterval: "history.timerProcessorMaxPollInterval", - TimerProcessorMaxPollIntervalJitterCoefficient: "history.timerProcessorMaxPollIntervalJitterCoefficient", - TimerProcessorSplitQueueInterval: "history.timerProcessorSplitQueueInterval", - TimerProcessorSplitQueueIntervalJitterCoefficient: "history.timerProcessorSplitQueueIntervalJitterCoefficient", - TimerProcessorRedispatchInterval: "history.timerProcessorRedispatchInterval", - TimerProcessorRedispatchIntervalJitterCoefficient: "history.timerProcessorRedispatchIntervalJitterCoefficient", - TimerProcessorMaxRedispatchQueueSize: "history.timerProcessorMaxRedispatchQueueSize", - TimerProcessorEnablePriorityTaskProcessor: "history.timerProcessorEnablePriorityTaskProcessor", - TimerProcessorEnableMultiCurosrProcessor: "history.timerProcessorEnableMultiCursorProcessor", - TimerProcessorMaxTimeShift: "history.timerProcessorMaxTimeShift", - TimerProcessorHistoryArchivalSizeLimit: "history.timerProcessorHistoryArchivalSizeLimit", - TimerProcessorArchivalTimeLimit: "history.timerProcessorArchivalTimeLimit", - TransferTaskBatchSize: "history.transferTaskBatchSize", - TransferProcessorFailoverMaxPollRPS: "history.transferProcessorFailoverMaxPollRPS", - TransferProcessorMaxPollRPS: "history.transferProcessorMaxPollRPS", - TransferTaskWorkerCount: "history.transferTaskWorkerCount", - TransferTaskMaxRetryCount: "history.transferTaskMaxRetryCount", - TransferProcessorCompleteTransferFailureRetryCount: "history.transferProcessorCompleteTransferFailureRetryCount", - TransferProcessorUpdateShardTaskCount: "history.transferProcessorUpdateShardTaskCount", - TransferProcessorMaxPollInterval: "history.transferProcessorMaxPollInterval", - TransferProcessorMaxPollIntervalJitterCoefficient: "history.transferProcessorMaxPollIntervalJitterCoefficient", - TransferProcessorSplitQueueInterval: "history.transferProcessorSplitQueueInterval", - TransferProcessorSplitQueueIntervalJitterCoefficient: "history.transferProcessorSplitQueueIntervalJitterCoefficient", - TransferProcessorUpdateAckInterval: "history.transferProcessorUpdateAckInterval", - TransferProcessorUpdateAckIntervalJitterCoefficient: "history.transferProcessorUpdateAckIntervalJitterCoefficient", - TransferProcessorCompleteTransferInterval: "history.transferProcessorCompleteTransferInterval", - TransferProcessorRedispatchInterval: "history.transferProcessorRedispatchInterval", - TransferProcessorRedispatchIntervalJitterCoefficient: "history.transferProcessorRedispatchIntervalJitterCoefficient", - TransferProcessorMaxRedispatchQueueSize: "history.transferProcessorMaxRedispatchQueueSize", - TransferProcessorEnablePriorityTaskProcessor: "history.transferProcessorEnablePriorityTaskProcessor", - TransferProcessorEnableMultiCurosrProcessor: "history.transferProcessorEnableMultiCursorProcessor", - TransferProcessorVisibilityArchivalTimeLimit: "history.transferProcessorVisibilityArchivalTimeLimit", - ReplicatorTaskBatchSize: "history.replicatorTaskBatchSize", - ReplicatorTaskWorkerCount: "history.replicatorTaskWorkerCount", - ReplicatorReadTaskMaxRetryCount: "history.replicatorReadTaskMaxRetryCount", - ReplicatorTaskMaxRetryCount: "history.replicatorTaskMaxRetryCount", - ReplicatorProcessorMaxPollRPS: "history.replicatorProcessorMaxPollRPS", - ReplicatorProcessorUpdateShardTaskCount: "history.replicatorProcessorUpdateShardTaskCount", - ReplicatorProcessorMaxPollInterval: "history.replicatorProcessorMaxPollInterval", - ReplicatorProcessorMaxPollIntervalJitterCoefficient: "history.replicatorProcessorMaxPollIntervalJitterCoefficient", - ReplicatorProcessorUpdateAckInterval: "history.replicatorProcessorUpdateAckInterval", - ReplicatorProcessorUpdateAckIntervalJitterCoefficient: "history.replicatorProcessorUpdateAckIntervalJitterCoefficient", - ReplicatorProcessorRedispatchInterval: "history.replicatorProcessorRedispatchInterval", - ReplicatorProcessorRedispatchIntervalJitterCoefficient: "history.replicatorProcessorRedispatchIntervalJitterCoefficient", - ReplicatorProcessorMaxRedispatchQueueSize: "history.replicatorProcessorMaxRedispatchQueueSize", - ReplicatorProcessorEnablePriorityTaskProcessor: "history.replicatorProcessorEnablePriorityTaskProcessor", - ExecutionMgrNumConns: "history.executionMgrNumConns", - HistoryMgrNumConns: "history.historyMgrNumConns", - MaximumBufferedEventsBatch: "history.maximumBufferedEventsBatch", - MaximumSignalsPerExecution: "history.maximumSignalsPerExecution", - ShardUpdateMinInterval: "history.shardUpdateMinInterval", - ShardSyncMinInterval: "history.shardSyncMinInterval", - ShardSyncTimerJitterCoefficient: "history.shardSyncMinInterval", - DefaultEventEncoding: "history.defaultEventEncoding", - EnableAdminProtection: "history.enableAdminProtection", - AdminOperationToken: "history.adminOperationToken", - EnableParentClosePolicy: "history.enableParentClosePolicy", - NumArchiveSystemWorkflows: "history.numArchiveSystemWorkflows", - ArchiveRequestRPS: "history.archiveRequestRPS", - EmitShardDiffLog: "history.emitShardDiffLog", - HistoryThrottledLogRPS: "history.throttledLogRPS", - StickyTTL: "history.stickyTTL", - DecisionHeartbeatTimeout: "history.decisionHeartbeatTimeout", - ParentClosePolicyThreshold: "history.parentClosePolicyThreshold", - NumParentClosePolicySystemWorkflows: "history.numParentClosePolicySystemWorkflows", - ReplicationTaskFetcherParallelism: "history.ReplicationTaskFetcherParallelism", - ReplicationTaskFetcherAggregationInterval: "history.ReplicationTaskFetcherAggregationInterval", - ReplicationTaskFetcherTimerJitterCoefficient: "history.ReplicationTaskFetcherTimerJitterCoefficient", - ReplicationTaskFetcherErrorRetryWait: "history.ReplicationTaskFetcherErrorRetryWait", - ReplicationTaskFetcherServiceBusyWait: "history.ReplicationTaskFetcherServiceBusyWait", - ReplicationTaskProcessorErrorRetryWait: "history.ReplicationTaskProcessorErrorRetryWait", - ReplicationTaskProcessorErrorRetryMaxAttempts: "history.ReplicationTaskProcessorErrorRetryMaxAttempts", - ReplicationTaskProcessorNoTaskInitialWait: "history.ReplicationTaskProcessorNoTaskInitialWait", - ReplicationTaskProcessorCleanupInterval: "history.ReplicationTaskProcessorCleanupInterval", - ReplicationTaskProcessorCleanupJitterCoefficient: "history.ReplicationTaskProcessorCleanupJitterCoefficient", - ReplicationTaskProcessorReadHistoryBatchSize: "history.ReplicationTaskProcessorReadHistoryBatchSize", - HistoryEnableRPCReplication: "history.EnableRPCReplication", - HistoryEnableKafkaReplication: "history.EnableKafkaReplication", - HistoryEnableCleanupReplicationTask: "history.EnableCleanupReplicationTask", - EnableConsistentQuery: "history.EnableConsistentQuery", - EnableConsistentQueryByDomain: "history.EnableConsistentQueryByDomain", - MaxBufferedQueryCount: "history.MaxBufferedQueryCount", - MutableStateChecksumGenProbability: "history.mutableStateChecksumGenProbability", - MutableStateChecksumVerifyProbability: "history.mutableStateChecksumVerifyProbability", - MutableStateChecksumInvalidateBefore: "history.mutableStateChecksumInvalidateBefore", - ReplicationEventsFromCurrentCluster: "history.ReplicationEventsFromCurrentCluster", - NotifyFailoverMarkerInterval: "history.NotifyFailoverMarkerInterval", - NotifyFailoverMarkerTimerJitterCoefficient: "history.NotifyFailoverMarkerTimerJitterCoefficient", + HistoryRPS: "history.rps", + HistoryPersistenceMaxQPS: "history.persistenceMaxQPS", + HistoryPersistenceGlobalMaxQPS: "history.persistenceGlobalMaxQPS", + HistoryVisibilityOpenMaxQPS: "history.historyVisibilityOpenMaxQPS", + HistoryVisibilityClosedMaxQPS: "history.historyVisibilityClosedMaxQPS", + HistoryLongPollExpirationInterval: "history.longPollExpirationInterval", + HistoryCacheInitialSize: "history.cacheInitialSize", + HistoryMaxAutoResetPoints: "history.historyMaxAutoResetPoints", + HistoryCacheMaxSize: "history.cacheMaxSize", + HistoryCacheTTL: "history.cacheTTL", + HistoryShutdownDrainDuration: "history.shutdownDrainDuration", + EventsCacheInitialCount: "history.eventsCacheInitialSize", + EventsCacheMaxCount: "history.eventsCacheMaxSize", + EventsCacheMaxSize: "history.eventsCacheMaxSizeInBytes", + EventsCacheTTL: "history.eventsCacheTTL", + EventsCacheGlobalEnable: "history.eventsCacheGlobalEnable", + EventsCacheGlobalInitialCount: "history.eventsCacheGlobalInitialSize", + EventsCacheGlobalMaxCount: "history.eventsCacheGlobalMaxSize", + AcquireShardInterval: "history.acquireShardInterval", + AcquireShardConcurrency: "history.acquireShardConcurrency", + StandbyClusterDelay: "history.standbyClusterDelay", + StandbyTaskMissingEventsResendDelay: "history.standbyTaskMissingEventsResendDelay", + StandbyTaskMissingEventsDiscardDelay: "history.standbyTaskMissingEventsDiscardDelay", + TaskProcessRPS: "history.taskProcessRPS", + TaskSchedulerType: "history.taskSchedulerType", + TaskSchedulerWorkerCount: "history.taskSchedulerWorkerCount", + TaskSchedulerShardWorkerCount: "history.taskSchedulerShardWorkerCount", + TaskSchedulerQueueSize: "history.taskSchedulerQueueSize", + TaskSchedulerShardQueueSize: "history.taskSchedulerShardQueueSize", + TaskSchedulerDispatcherCount: "history.taskSchedulerDispatcherCount", + TaskSchedulerRoundRobinWeights: "history.taskSchedulerRoundRobinWeight", + TaskRedispatchInterval: "history.taskRedispatchInterval", + TaskRedispatchIntervalJitterCoefficient: "history.taskRedispatchIntervalJitterCoefficient", + QueueProcessorEnableDomainTaggedMetrics: "history.queueProcessorEnableDomainTaggedMetrics", + QueueProcessorEnableSplit: "history.queueProcessorEnableSplit", + QueueProcessorSplitMaxLevel: "history.queueProcessorSplitMaxLevel", + QueueProcessorEnableRandomSplitByDomainID: "history.queueProcessorEnableRandomSplitByDomain", + QueueProcessorRandomSplitProbability: "history.queueProcessorRandomSplitProbability", + QueueProcessorEnablePendingTaskSplit: "history.queueProcessorEnablePendingTaskSplit", + QueueProcessorPendingTaskSplitThreshold: "history.queueProcessorPendingTaskSplitThreshold", + QueueProcessorEnableStuckTaskSplit: "history.queueProcessorEnableStuckTaskSplit", + QueueProcessorStuckTaskSplitThreshold: "history.queueProcessorStuckTaskSplitThreshold", + QueueProcessorSplitLookAheadDurationByDomainID: "history.queueProcessorSplitLookAheadDuration", + TimerTaskBatchSize: "history.timerTaskBatchSize", + TimerTaskWorkerCount: "history.timerTaskWorkerCount", + TimerTaskMaxRetryCount: "history.timerTaskMaxRetryCount", + TimerProcessorGetFailureRetryCount: "history.timerProcessorGetFailureRetryCount", + TimerProcessorCompleteTimerFailureRetryCount: "history.timerProcessorCompleteTimerFailureRetryCount", + TimerProcessorUpdateShardTaskCount: "history.timerProcessorUpdateShardTaskCount", + TimerProcessorUpdateAckInterval: "history.timerProcessorUpdateAckInterval", + TimerProcessorUpdateAckIntervalJitterCoefficient: "history.timerProcessorUpdateAckIntervalJitterCoefficient", + TimerProcessorCompleteTimerInterval: "history.timerProcessorCompleteTimerInterval", + TimerProcessorFailoverMaxPollRPS: "history.timerProcessorFailoverMaxPollRPS", + TimerProcessorMaxPollRPS: "history.timerProcessorMaxPollRPS", + TimerProcessorMaxPollInterval: "history.timerProcessorMaxPollInterval", + TimerProcessorMaxPollIntervalJitterCoefficient: "history.timerProcessorMaxPollIntervalJitterCoefficient", + TimerProcessorSplitQueueInterval: "history.timerProcessorSplitQueueInterval", + TimerProcessorSplitQueueIntervalJitterCoefficient: "history.timerProcessorSplitQueueIntervalJitterCoefficient", + TimerProcessorMaxRedispatchQueueSize: "history.timerProcessorMaxRedispatchQueueSize", + TimerProcessorEnablePriorityTaskProcessor: "history.timerProcessorEnablePriorityTaskProcessor", + TimerProcessorEnableMultiCurosrProcessor: "history.timerProcessorEnableMultiCursorProcessor", + TimerProcessorMaxTimeShift: "history.timerProcessorMaxTimeShift", + TimerProcessorHistoryArchivalSizeLimit: "history.timerProcessorHistoryArchivalSizeLimit", + TimerProcessorArchivalTimeLimit: "history.timerProcessorArchivalTimeLimit", + TransferTaskBatchSize: "history.transferTaskBatchSize", + TransferProcessorFailoverMaxPollRPS: "history.transferProcessorFailoverMaxPollRPS", + TransferProcessorMaxPollRPS: "history.transferProcessorMaxPollRPS", + TransferTaskWorkerCount: "history.transferTaskWorkerCount", + TransferTaskMaxRetryCount: "history.transferTaskMaxRetryCount", + TransferProcessorCompleteTransferFailureRetryCount: "history.transferProcessorCompleteTransferFailureRetryCount", + TransferProcessorUpdateShardTaskCount: "history.transferProcessorUpdateShardTaskCount", + TransferProcessorMaxPollInterval: "history.transferProcessorMaxPollInterval", + TransferProcessorMaxPollIntervalJitterCoefficient: "history.transferProcessorMaxPollIntervalJitterCoefficient", + TransferProcessorSplitQueueInterval: "history.transferProcessorSplitQueueInterval", + TransferProcessorSplitQueueIntervalJitterCoefficient: "history.transferProcessorSplitQueueIntervalJitterCoefficient", + TransferProcessorUpdateAckInterval: "history.transferProcessorUpdateAckInterval", + TransferProcessorUpdateAckIntervalJitterCoefficient: "history.transferProcessorUpdateAckIntervalJitterCoefficient", + TransferProcessorCompleteTransferInterval: "history.transferProcessorCompleteTransferInterval", + TransferProcessorMaxRedispatchQueueSize: "history.transferProcessorMaxRedispatchQueueSize", + TransferProcessorEnablePriorityTaskProcessor: "history.transferProcessorEnablePriorityTaskProcessor", + TransferProcessorEnableMultiCurosrProcessor: "history.transferProcessorEnableMultiCursorProcessor", + TransferProcessorVisibilityArchivalTimeLimit: "history.transferProcessorVisibilityArchivalTimeLimit", + ReplicatorTaskBatchSize: "history.replicatorTaskBatchSize", + ReplicatorTaskWorkerCount: "history.replicatorTaskWorkerCount", + ReplicatorReadTaskMaxRetryCount: "history.replicatorReadTaskMaxRetryCount", + ReplicatorTaskMaxRetryCount: "history.replicatorTaskMaxRetryCount", + ReplicatorProcessorMaxPollRPS: "history.replicatorProcessorMaxPollRPS", + ReplicatorProcessorUpdateShardTaskCount: "history.replicatorProcessorUpdateShardTaskCount", + ReplicatorProcessorMaxPollInterval: "history.replicatorProcessorMaxPollInterval", + ReplicatorProcessorMaxPollIntervalJitterCoefficient: "history.replicatorProcessorMaxPollIntervalJitterCoefficient", + ReplicatorProcessorUpdateAckInterval: "history.replicatorProcessorUpdateAckInterval", + ReplicatorProcessorUpdateAckIntervalJitterCoefficient: "history.replicatorProcessorUpdateAckIntervalJitterCoefficient", + ReplicatorProcessorMaxRedispatchQueueSize: "history.replicatorProcessorMaxRedispatchQueueSize", + ReplicatorProcessorEnablePriorityTaskProcessor: "history.replicatorProcessorEnablePriorityTaskProcessor", + ExecutionMgrNumConns: "history.executionMgrNumConns", + HistoryMgrNumConns: "history.historyMgrNumConns", + MaximumBufferedEventsBatch: "history.maximumBufferedEventsBatch", + MaximumSignalsPerExecution: "history.maximumSignalsPerExecution", + ShardUpdateMinInterval: "history.shardUpdateMinInterval", + ShardSyncMinInterval: "history.shardSyncMinInterval", + ShardSyncTimerJitterCoefficient: "history.shardSyncMinInterval", + DefaultEventEncoding: "history.defaultEventEncoding", + EnableAdminProtection: "history.enableAdminProtection", + AdminOperationToken: "history.adminOperationToken", + EnableParentClosePolicy: "history.enableParentClosePolicy", + NumArchiveSystemWorkflows: "history.numArchiveSystemWorkflows", + ArchiveRequestRPS: "history.archiveRequestRPS", + EmitShardDiffLog: "history.emitShardDiffLog", + HistoryThrottledLogRPS: "history.throttledLogRPS", + StickyTTL: "history.stickyTTL", + DecisionHeartbeatTimeout: "history.decisionHeartbeatTimeout", + ParentClosePolicyThreshold: "history.parentClosePolicyThreshold", + NumParentClosePolicySystemWorkflows: "history.numParentClosePolicySystemWorkflows", + ReplicationTaskFetcherParallelism: "history.ReplicationTaskFetcherParallelism", + ReplicationTaskFetcherAggregationInterval: "history.ReplicationTaskFetcherAggregationInterval", + ReplicationTaskFetcherTimerJitterCoefficient: "history.ReplicationTaskFetcherTimerJitterCoefficient", + ReplicationTaskFetcherErrorRetryWait: "history.ReplicationTaskFetcherErrorRetryWait", + ReplicationTaskFetcherServiceBusyWait: "history.ReplicationTaskFetcherServiceBusyWait", + ReplicationTaskProcessorErrorRetryWait: "history.ReplicationTaskProcessorErrorRetryWait", + ReplicationTaskProcessorErrorRetryMaxAttempts: "history.ReplicationTaskProcessorErrorRetryMaxAttempts", + ReplicationTaskProcessorNoTaskInitialWait: "history.ReplicationTaskProcessorNoTaskInitialWait", + ReplicationTaskProcessorCleanupInterval: "history.ReplicationTaskProcessorCleanupInterval", + ReplicationTaskProcessorCleanupJitterCoefficient: "history.ReplicationTaskProcessorCleanupJitterCoefficient", + ReplicationTaskProcessorReadHistoryBatchSize: "history.ReplicationTaskProcessorReadHistoryBatchSize", + HistoryEnableRPCReplication: "history.EnableRPCReplication", + HistoryEnableKafkaReplication: "history.EnableKafkaReplication", + HistoryEnableCleanupReplicationTask: "history.EnableCleanupReplicationTask", + EnableConsistentQuery: "history.EnableConsistentQuery", + EnableConsistentQueryByDomain: "history.EnableConsistentQueryByDomain", + MaxBufferedQueryCount: "history.MaxBufferedQueryCount", + MutableStateChecksumGenProbability: "history.mutableStateChecksumGenProbability", + MutableStateChecksumVerifyProbability: "history.mutableStateChecksumVerifyProbability", + MutableStateChecksumInvalidateBefore: "history.mutableStateChecksumInvalidateBefore", + ReplicationEventsFromCurrentCluster: "history.ReplicationEventsFromCurrentCluster", + NotifyFailoverMarkerInterval: "history.NotifyFailoverMarkerInterval", + NotifyFailoverMarkerTimerJitterCoefficient: "history.NotifyFailoverMarkerTimerJitterCoefficient", WorkerPersistenceMaxQPS: "worker.persistenceMaxQPS", WorkerPersistenceGlobalMaxQPS: "worker.persistenceGlobalMaxQPS", @@ -562,6 +558,10 @@ const ( TaskSchedulerDispatcherCount // TaskSchedulerRoundRobinWeights is the priority weight for weighted round robin task scheduler TaskSchedulerRoundRobinWeights + // TaskRedispatchInterval is the task redispatch interval + TaskRedispatchInterval + // TaskRedispatchIntervalJitterCoefficient is the task redispatch interval jitter coefficient + TaskRedispatchIntervalJitterCoefficient // QueueProcessorEnableDomainTaggedMetrics indicates whether task processing metrics should include domain tag QueueProcessorEnableDomainTaggedMetrics // QueueProcessorEnableSplit indicates whether processing queue split policy should be enabled @@ -612,10 +612,6 @@ const ( TimerProcessorSplitQueueInterval // TimerProcessorSplitQueueIntervalJitterCoefficient is the split processing queue interval jitter coefficient TimerProcessorSplitQueueIntervalJitterCoefficient - // TimerProcessorRedispatchInterval is the redispatch interval for timer processor - TimerProcessorRedispatchInterval - // TimerProcessorRedispatchIntervalJitterCoefficient is the redispatch interval jitter coefficient - TimerProcessorRedispatchIntervalJitterCoefficient // TimerProcessorMaxRedispatchQueueSize is the threshold of the number of tasks in the redispatch queue for timer processor TimerProcessorMaxRedispatchQueueSize // TimerProcessorEnablePriorityTaskProcessor indicates whether priority task processor should be used for timer processor @@ -656,10 +652,6 @@ const ( TransferProcessorUpdateAckIntervalJitterCoefficient // TransferProcessorCompleteTransferInterval is complete timer interval for transferQueueProcessor TransferProcessorCompleteTransferInterval - // TransferProcessorRedispatchInterval is the redispatch interval for transferQueueProcessor - TransferProcessorRedispatchInterval - // TransferProcessorRedispatchIntervalJitterCoefficient is the redispatch interval jitter coefficient - TransferProcessorRedispatchIntervalJitterCoefficient // TransferProcessorMaxRedispatchQueueSize is the threshold of the number of tasks in the redispatch queue for transferQueueProcessor TransferProcessorMaxRedispatchQueueSize // TransferProcessorEnablePriorityTaskProcessor indicates whether priority task processor should be used for transferQueueProcessor @@ -688,10 +680,6 @@ const ( ReplicatorProcessorUpdateAckInterval // ReplicatorProcessorUpdateAckIntervalJitterCoefficient is the update interval jitter coefficient ReplicatorProcessorUpdateAckIntervalJitterCoefficient - // ReplicatorProcessorRedispatchInterval is the redispatch interval for ReplicatorProcessor - ReplicatorProcessorRedispatchInterval - // ReplicatorProcessorRedispatchIntervalJitterCoefficient is the redispatch interval jitter coefficient - ReplicatorProcessorRedispatchIntervalJitterCoefficient // ReplicatorProcessorMaxRedispatchQueueSize is the threshold of the number of tasks in the redispatch queue for ReplicatorProcessor ReplicatorProcessorMaxRedispatchQueueSize // ReplicatorProcessorEnablePriorityTaskProcessor indicates whether priority task processor should be used for ReplicatorProcessor diff --git a/common/task/interface.go b/common/task/interface.go index 56c03c8d262..dddb0a5440b 100644 --- a/common/task/interface.go +++ b/common/task/interface.go @@ -108,8 +108,3 @@ const ( // TaskStateNacked is the state for a task if it can not be processed TaskStateNacked ) - -const ( - // NoPriority is the value returned if no priority is ever assigned to the task - NoPriority = -1 -) diff --git a/common/task/priority.go b/common/task/priority.go index 34a3ee5d709..4519ed0897a 100644 --- a/common/task/priority.go +++ b/common/task/priority.go @@ -24,6 +24,11 @@ const ( numBitsPerLevel = 3 ) +const ( + // NoPriority is the value returned if no priority is ever assigned to the task + NoPriority = -1 +) + const ( // HighPriorityClass is the priority class for high priority tasks HighPriorityClass = iota << numBitsPerLevel diff --git a/service/history/config/config.go b/service/history/config/config.go index f0bfccfab52..2761a9ca70a 100644 --- a/service/history/config/config.go +++ b/service/history/config/config.go @@ -77,15 +77,17 @@ type Config struct { StandbyTaskMissingEventsDiscardDelay dynamicconfig.DurationPropertyFn // Task process settings - TaskProcessRPS dynamicconfig.IntPropertyFnWithDomainFilter - EnablePriorityTaskProcessor dynamicconfig.BoolPropertyFn - TaskSchedulerType dynamicconfig.IntPropertyFn - TaskSchedulerWorkerCount dynamicconfig.IntPropertyFn - TaskSchedulerShardWorkerCount dynamicconfig.IntPropertyFn - TaskSchedulerQueueSize dynamicconfig.IntPropertyFn - TaskSchedulerShardQueueSize dynamicconfig.IntPropertyFn - TaskSchedulerDispatcherCount dynamicconfig.IntPropertyFn - TaskSchedulerRoundRobinWeights dynamicconfig.MapPropertyFn + TaskProcessRPS dynamicconfig.IntPropertyFnWithDomainFilter + EnablePriorityTaskProcessor dynamicconfig.BoolPropertyFn + TaskSchedulerType dynamicconfig.IntPropertyFn + TaskSchedulerWorkerCount dynamicconfig.IntPropertyFn + TaskSchedulerShardWorkerCount dynamicconfig.IntPropertyFn + TaskSchedulerQueueSize dynamicconfig.IntPropertyFn + TaskSchedulerShardQueueSize dynamicconfig.IntPropertyFn + TaskSchedulerDispatcherCount dynamicconfig.IntPropertyFn + TaskSchedulerRoundRobinWeights dynamicconfig.MapPropertyFn + TaskRedispatchInterval dynamicconfig.DurationPropertyFn + TaskRedispatchIntervalJitterCoefficient dynamicconfig.FloatPropertyFn // QueueProcessor settings QueueProcessorEnableDomainTaggedMetrics dynamicconfig.BoolPropertyFn @@ -114,8 +116,6 @@ type Config struct { TimerProcessorMaxPollIntervalJitterCoefficient dynamicconfig.FloatPropertyFn TimerProcessorSplitQueueInterval dynamicconfig.DurationPropertyFn TimerProcessorSplitQueueIntervalJitterCoefficient dynamicconfig.FloatPropertyFn - TimerProcessorRedispatchInterval dynamicconfig.DurationPropertyFn - TimerProcessorRedispatchIntervalJitterCoefficient dynamicconfig.FloatPropertyFn TimerProcessorMaxRedispatchQueueSize dynamicconfig.IntPropertyFn TimerProcessorEnablePriorityTaskProcessor dynamicconfig.BoolPropertyFn TimerProcessorEnableMultiCurosrProcessor dynamicconfig.BoolPropertyFn @@ -137,28 +137,24 @@ type Config struct { TransferProcessorUpdateAckInterval dynamicconfig.DurationPropertyFn TransferProcessorUpdateAckIntervalJitterCoefficient dynamicconfig.FloatPropertyFn TransferProcessorCompleteTransferInterval dynamicconfig.DurationPropertyFn - TransferProcessorRedispatchInterval dynamicconfig.DurationPropertyFn - TransferProcessorRedispatchIntervalJitterCoefficient dynamicconfig.FloatPropertyFn TransferProcessorMaxRedispatchQueueSize dynamicconfig.IntPropertyFn TransferProcessorEnablePriorityTaskProcessor dynamicconfig.BoolPropertyFn TransferProcessorEnableMultiCurosrProcessor dynamicconfig.BoolPropertyFn TransferProcessorVisibilityArchivalTimeLimit dynamicconfig.DurationPropertyFn // ReplicatorQueueProcessor settings - ReplicatorTaskBatchSize dynamicconfig.IntPropertyFn - ReplicatorTaskWorkerCount dynamicconfig.IntPropertyFn - ReplicatorTaskMaxRetryCount dynamicconfig.IntPropertyFn - ReplicatorReadTaskMaxRetryCount dynamicconfig.IntPropertyFn - ReplicatorProcessorMaxPollRPS dynamicconfig.IntPropertyFn - ReplicatorProcessorMaxPollInterval dynamicconfig.DurationPropertyFn - ReplicatorProcessorMaxPollIntervalJitterCoefficient dynamicconfig.FloatPropertyFn - ReplicatorProcessorUpdateAckInterval dynamicconfig.DurationPropertyFn - ReplicatorProcessorUpdateAckIntervalJitterCoefficient dynamicconfig.FloatPropertyFn - ReplicatorProcessorRedispatchInterval dynamicconfig.DurationPropertyFn - ReplicatorProcessorRedispatchIntervalJitterCoefficient dynamicconfig.FloatPropertyFn - ReplicatorProcessorMaxRedispatchQueueSize dynamicconfig.IntPropertyFn - ReplicatorProcessorEnablePriorityTaskProcessor dynamicconfig.BoolPropertyFn - ReplicatorProcessorFetchTasksBatchSize dynamicconfig.IntPropertyFn + ReplicatorTaskBatchSize dynamicconfig.IntPropertyFn + ReplicatorTaskWorkerCount dynamicconfig.IntPropertyFn + ReplicatorTaskMaxRetryCount dynamicconfig.IntPropertyFn + ReplicatorReadTaskMaxRetryCount dynamicconfig.IntPropertyFn + ReplicatorProcessorMaxPollRPS dynamicconfig.IntPropertyFn + ReplicatorProcessorMaxPollInterval dynamicconfig.DurationPropertyFn + ReplicatorProcessorMaxPollIntervalJitterCoefficient dynamicconfig.FloatPropertyFn + ReplicatorProcessorUpdateAckInterval dynamicconfig.DurationPropertyFn + ReplicatorProcessorUpdateAckIntervalJitterCoefficient dynamicconfig.FloatPropertyFn + ReplicatorProcessorMaxRedispatchQueueSize dynamicconfig.IntPropertyFn + ReplicatorProcessorEnablePriorityTaskProcessor dynamicconfig.BoolPropertyFn + ReplicatorProcessorFetchTasksBatchSize dynamicconfig.IntPropertyFn // Persistence settings ExecutionMgrNumConns dynamicconfig.IntPropertyFn @@ -315,15 +311,17 @@ func New(dc *dynamicconfig.Collection, numberOfShards int, storeType string, isA StandbyTaskMissingEventsResendDelay: dc.GetDurationProperty(dynamicconfig.StandbyTaskMissingEventsResendDelay, 15*time.Minute), StandbyTaskMissingEventsDiscardDelay: dc.GetDurationProperty(dynamicconfig.StandbyTaskMissingEventsDiscardDelay, 25*time.Minute), - TaskProcessRPS: dc.GetIntPropertyFilteredByDomain(dynamicconfig.TaskProcessRPS, 1000), - EnablePriorityTaskProcessor: dc.GetBoolProperty(dynamicconfig.EnablePriorityTaskProcessor, false), - TaskSchedulerType: dc.GetIntProperty(dynamicconfig.TaskSchedulerType, int(task.SchedulerTypeWRR)), - TaskSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerWorkerCount, 400), - TaskSchedulerShardWorkerCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerShardWorkerCount, 2), - TaskSchedulerQueueSize: dc.GetIntProperty(dynamicconfig.TaskSchedulerQueueSize, 10000), - TaskSchedulerShardQueueSize: dc.GetIntProperty(dynamicconfig.TaskSchedulerShardQueueSize, 200), - TaskSchedulerDispatcherCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerDispatcherCount, 10), - TaskSchedulerRoundRobinWeights: dc.GetMapProperty(dynamicconfig.TaskSchedulerRoundRobinWeights, common.ConvertIntMapToDynamicConfigMapProperty(DefaultTaskPriorityWeight)), + TaskProcessRPS: dc.GetIntPropertyFilteredByDomain(dynamicconfig.TaskProcessRPS, 1000), + EnablePriorityTaskProcessor: dc.GetBoolProperty(dynamicconfig.EnablePriorityTaskProcessor, false), + TaskSchedulerType: dc.GetIntProperty(dynamicconfig.TaskSchedulerType, int(task.SchedulerTypeWRR)), + TaskSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerWorkerCount, 400), + TaskSchedulerShardWorkerCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerShardWorkerCount, 2), + TaskSchedulerQueueSize: dc.GetIntProperty(dynamicconfig.TaskSchedulerQueueSize, 10000), + TaskSchedulerShardQueueSize: dc.GetIntProperty(dynamicconfig.TaskSchedulerShardQueueSize, 200), + TaskSchedulerDispatcherCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerDispatcherCount, 10), + TaskSchedulerRoundRobinWeights: dc.GetMapProperty(dynamicconfig.TaskSchedulerRoundRobinWeights, common.ConvertIntMapToDynamicConfigMapProperty(DefaultTaskPriorityWeight)), + TaskRedispatchInterval: dc.GetDurationProperty(dynamicconfig.TaskRedispatchInterval, 5*time.Second), + TaskRedispatchIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TimerProcessorSplitQueueIntervalJitterCoefficient, 0.15), QueueProcessorEnableDomainTaggedMetrics: dc.GetBoolProperty(dynamicconfig.QueueProcessorEnableDomainTaggedMetrics, false), QueueProcessorEnableSplit: dc.GetBoolProperty(dynamicconfig.QueueProcessorEnableSplit, false), @@ -350,8 +348,6 @@ func New(dc *dynamicconfig.Collection, numberOfShards int, storeType string, isA TimerProcessorMaxPollIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TimerProcessorMaxPollIntervalJitterCoefficient, 0.15), TimerProcessorSplitQueueInterval: dc.GetDurationProperty(dynamicconfig.TimerProcessorSplitQueueInterval, 1*time.Minute), TimerProcessorSplitQueueIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TimerProcessorSplitQueueIntervalJitterCoefficient, 0.15), - TimerProcessorRedispatchInterval: dc.GetDurationProperty(dynamicconfig.TimerProcessorRedispatchInterval, 5*time.Second), - TimerProcessorRedispatchIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TimerProcessorRedispatchIntervalJitterCoefficient, 0.15), TimerProcessorMaxRedispatchQueueSize: dc.GetIntProperty(dynamicconfig.TimerProcessorMaxRedispatchQueueSize, 10000), TimerProcessorEnablePriorityTaskProcessor: dc.GetBoolProperty(dynamicconfig.TimerProcessorEnablePriorityTaskProcessor, false), TimerProcessorEnableMultiCurosrProcessor: dc.GetBoolProperty(dynamicconfig.TimerProcessorEnableMultiCurosrProcessor, false), @@ -372,27 +368,23 @@ func New(dc *dynamicconfig.Collection, numberOfShards int, storeType string, isA TransferProcessorUpdateAckInterval: dc.GetDurationProperty(dynamicconfig.TransferProcessorUpdateAckInterval, 30*time.Second), TransferProcessorUpdateAckIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TransferProcessorUpdateAckIntervalJitterCoefficient, 0.15), TransferProcessorCompleteTransferInterval: dc.GetDurationProperty(dynamicconfig.TransferProcessorCompleteTransferInterval, 60*time.Second), - TransferProcessorRedispatchInterval: dc.GetDurationProperty(dynamicconfig.TransferProcessorRedispatchInterval, 5*time.Second), - TransferProcessorRedispatchIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TransferProcessorRedispatchIntervalJitterCoefficient, 0.15), TransferProcessorMaxRedispatchQueueSize: dc.GetIntProperty(dynamicconfig.TransferProcessorMaxRedispatchQueueSize, 10000), TransferProcessorEnablePriorityTaskProcessor: dc.GetBoolProperty(dynamicconfig.TransferProcessorEnablePriorityTaskProcessor, false), TransferProcessorEnableMultiCurosrProcessor: dc.GetBoolProperty(dynamicconfig.TransferProcessorEnableMultiCurosrProcessor, false), TransferProcessorVisibilityArchivalTimeLimit: dc.GetDurationProperty(dynamicconfig.TransferProcessorVisibilityArchivalTimeLimit, 200*time.Millisecond), - ReplicatorTaskBatchSize: dc.GetIntProperty(dynamicconfig.ReplicatorTaskBatchSize, 100), - ReplicatorTaskWorkerCount: dc.GetIntProperty(dynamicconfig.ReplicatorTaskWorkerCount, 10), - ReplicatorTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.ReplicatorTaskMaxRetryCount, 100), - ReplicatorReadTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.ReplicatorReadTaskMaxRetryCount, 3), - ReplicatorProcessorMaxPollRPS: dc.GetIntProperty(dynamicconfig.ReplicatorProcessorMaxPollRPS, 20), - ReplicatorProcessorMaxPollInterval: dc.GetDurationProperty(dynamicconfig.ReplicatorProcessorMaxPollInterval, 1*time.Minute), - ReplicatorProcessorMaxPollIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.ReplicatorProcessorMaxPollIntervalJitterCoefficient, 0.15), - ReplicatorProcessorUpdateAckInterval: dc.GetDurationProperty(dynamicconfig.ReplicatorProcessorUpdateAckInterval, 5*time.Second), - ReplicatorProcessorUpdateAckIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.ReplicatorProcessorUpdateAckIntervalJitterCoefficient, 0.15), - ReplicatorProcessorRedispatchInterval: dc.GetDurationProperty(dynamicconfig.ReplicatorProcessorRedispatchInterval, 5*time.Second), - ReplicatorProcessorRedispatchIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.ReplicatorProcessorRedispatchIntervalJitterCoefficient, 0.15), - ReplicatorProcessorMaxRedispatchQueueSize: dc.GetIntProperty(dynamicconfig.ReplicatorProcessorMaxRedispatchQueueSize, 10000), - ReplicatorProcessorEnablePriorityTaskProcessor: dc.GetBoolProperty(dynamicconfig.ReplicatorProcessorEnablePriorityTaskProcessor, false), - ReplicatorProcessorFetchTasksBatchSize: dc.GetIntProperty(dynamicconfig.ReplicatorTaskBatchSize, 25), + ReplicatorTaskBatchSize: dc.GetIntProperty(dynamicconfig.ReplicatorTaskBatchSize, 100), + ReplicatorTaskWorkerCount: dc.GetIntProperty(dynamicconfig.ReplicatorTaskWorkerCount, 10), + ReplicatorTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.ReplicatorTaskMaxRetryCount, 100), + ReplicatorReadTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.ReplicatorReadTaskMaxRetryCount, 3), + ReplicatorProcessorMaxPollRPS: dc.GetIntProperty(dynamicconfig.ReplicatorProcessorMaxPollRPS, 20), + ReplicatorProcessorMaxPollInterval: dc.GetDurationProperty(dynamicconfig.ReplicatorProcessorMaxPollInterval, 1*time.Minute), + ReplicatorProcessorMaxPollIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.ReplicatorProcessorMaxPollIntervalJitterCoefficient, 0.15), + ReplicatorProcessorUpdateAckInterval: dc.GetDurationProperty(dynamicconfig.ReplicatorProcessorUpdateAckInterval, 5*time.Second), + ReplicatorProcessorUpdateAckIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.ReplicatorProcessorUpdateAckIntervalJitterCoefficient, 0.15), + ReplicatorProcessorMaxRedispatchQueueSize: dc.GetIntProperty(dynamicconfig.ReplicatorProcessorMaxRedispatchQueueSize, 10000), + ReplicatorProcessorEnablePriorityTaskProcessor: dc.GetBoolProperty(dynamicconfig.ReplicatorProcessorEnablePriorityTaskProcessor, false), + ReplicatorProcessorFetchTasksBatchSize: dc.GetIntProperty(dynamicconfig.ReplicatorTaskBatchSize, 25), ExecutionMgrNumConns: dc.GetIntProperty(dynamicconfig.ExecutionMgrNumConns, 50), HistoryMgrNumConns: dc.GetIntProperty(dynamicconfig.HistoryMgrNumConns, 50), diff --git a/service/history/queue/processor_base.go b/service/history/queue/processor_base.go index efc6a59756f..a50545cb5b0 100644 --- a/service/history/queue/processor_base.go +++ b/service/history/queue/processor_base.go @@ -26,7 +26,6 @@ import ( "time" "github.com/uber/cadence/common" - "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/collection" "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/tag" @@ -71,9 +70,9 @@ type ( } processorBase struct { - shard shard.Context - taskProcessor task.Processor - redispatchQueue collection.Queue + shard shard.Context + taskProcessor task.Processor + redispatcher task.Redispatcher options *queueProcessorOptions updateMaxReadLevel updateMaxReadLevelFn @@ -90,8 +89,6 @@ type ( shutdownWG sync.WaitGroup shutdownCh chan struct{} - redispatchNotifyCh chan struct{} - queueCollectionsLock sync.RWMutex processingQueueCollections []ProcessingQueueCollection } @@ -108,10 +105,19 @@ func newProcessorBase( logger log.Logger, metricsClient metrics.Client, ) *processorBase { + metricsScope := metricsClient.Scope(options.MetricScope) return &processorBase{ - shard: shard, - taskProcessor: taskProcessor, - redispatchQueue: collection.NewConcurrentQueue(), + shard: shard, + taskProcessor: taskProcessor, + redispatcher: task.NewRedispatcher( + taskProcessor, + &task.RedispatcherOptions{ + TaskRedispatchInterval: options.RedispatchInterval, + TaskRedispatchIntervalJitterCoefficient: options.RedispatchIntervalJitterCoefficient, + }, + logger, + metricsScope, + ), options: options, updateMaxReadLevel: updateMaxReadLevel, @@ -120,7 +126,7 @@ func newProcessorBase( logger: logger, metricsClient: metricsClient, - metricsScope: metricsClient.Scope(options.MetricScope), + metricsScope: metricsScope, rateLimiter: quotas.NewDynamicRateLimiter( func() float64 { @@ -131,8 +137,6 @@ func newProcessorBase( status: common.DaemonStatusInitialized, shutdownCh: make(chan struct{}), - redispatchNotifyCh: make(chan struct{}, 1), - processingQueueCollections: newProcessingQueueCollections( processingQueueStates, logger, @@ -141,70 +145,6 @@ func newProcessorBase( } } -func (p *processorBase) redispatchLoop() { - defer p.shutdownWG.Done() - -redispatchTaskLoop: - for { - select { - case <-p.shutdownCh: - break redispatchTaskLoop - case <-p.redispatchNotifyCh: - // TODO: revisit the cpu usage and gc activity caused by - // creating timers and reading dynamicconfig if it becomes a problem. - backoffTimer := time.NewTimer(backoff.JitDuration( - p.options.RedispatchInterval(), - p.options.RedispatchIntervalJitterCoefficient(), - )) - select { - case <-p.shutdownCh: - backoffTimer.Stop() - break redispatchTaskLoop - case <-backoffTimer.C: - } - backoffTimer.Stop() - - // drain redispatchNotifyCh again - select { - case <-p.redispatchNotifyCh: - default: - } - - p.redispatchTasks() - } - } - - p.logger.Info("Queue processor task redispatch loop shut down.") -} - -func (p *processorBase) redispatchSingleTask( - task task.Task, -) { - p.redispatchQueue.Add(task) - p.notifyRedispatch() -} - -func (p *processorBase) notifyRedispatch() { - select { - case p.redispatchNotifyCh <- struct{}{}: - default: - } -} - -func (p *processorBase) redispatchTasks() { - RedispatchTasks( - p.redispatchQueue, - p.taskProcessor, - p.logger, - p.metricsScope, - p.shutdownCh, - ) - - if !p.redispatchQueue.IsEmpty() { - p.notifyRedispatch() - } -} - func (p *processorBase) updateAckLevel() (bool, error) { // TODO: only for now, find the min ack level across all processing queues // and update DB with that value. @@ -372,7 +312,7 @@ func (p *processorBase) submitTask( } } if err != nil || !submitted { - p.redispatchSingleTask(task) + p.redispatcher.AddTask(task) return false, nil } diff --git a/service/history/queue/timer_queue_processor_base.go b/service/history/queue/timer_queue_processor_base.go index 5ceed55476e..8431f1cd5fd 100644 --- a/service/history/queue/timer_queue_processor_base.go +++ b/service/history/queue/timer_queue_processor_base.go @@ -126,7 +126,7 @@ func newTimerQueueProcessorBase( task.InitializeLoggerForTask(shard.GetShardID(), taskInfo, logger), taskFilter, taskExecutor, - processorBase.redispatchSingleTask, + processorBase.redispatcher.AddTask, shard.GetTimeSource(), shard.GetConfig().TimerTaskMaxRetryCount, emitDomainTag, @@ -154,11 +154,12 @@ func (t *timerQueueProcessorBase) Start() { t.logger.Info("", tag.LifeCycleStarting) defer t.logger.Info("", tag.LifeCycleStarted) + t.redispatcher.Start() + t.notifyNewTimer(time.Time{}) - t.shutdownWG.Add(2) + t.shutdownWG.Add(1) go t.processorPump() - go t.redispatchLoop() } func (t *timerQueueProcessorBase) Stop() { @@ -180,6 +181,8 @@ func (t *timerQueueProcessorBase) Stop() { if success := common.AwaitWaitGroup(&t.shutdownWG, time.Minute); !success { t.logger.Warn("", tag.LifeCycleStopTimedout) } + + t.redispatcher.Stop() } func (t *timerQueueProcessorBase) processorPump() { @@ -203,8 +206,17 @@ processorPumpLoop: case <-t.shutdownCh: break processorPumpLoop case <-t.timerGate.FireChan(): - if t.redispatchQueue.Len() > t.options.MaxRedispatchQueueSize() { - t.redispatchTasks() + maxRedispatchQueueSize := t.options.MaxRedispatchQueueSize() + if t.redispatcher.Size() > maxRedispatchQueueSize { + t.redispatcher.Redispatch(maxRedispatchQueueSize) + if t.redispatcher.Size() > maxRedispatchQueueSize { + // if redispatcher still has a large number of tasks + // this only happens when system is under very high load + // we should backoff here instead of keeping submitting tasks to task processor + // don't call t.timerGate.Update(time.Now() + loadQueueTaskThrottleRetryDelay) as the time in + // standby timer processor is not real time and is managed separately + time.Sleep(loadQueueTaskThrottleRetryDelay) + } t.timerGate.Update(time.Time{}) continue processorPumpLoop } @@ -612,8 +624,8 @@ func newTimerQueueProcessorOptions( MaxPollIntervalJitterCoefficient: config.TimerProcessorMaxPollIntervalJitterCoefficient, UpdateAckInterval: config.TimerProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TimerProcessorUpdateAckIntervalJitterCoefficient, - RedispatchInterval: config.TimerProcessorRedispatchInterval, - RedispatchIntervalJitterCoefficient: config.TimerProcessorRedispatchIntervalJitterCoefficient, + RedispatchInterval: config.TaskRedispatchInterval, + RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TimerProcessorMaxRedispatchQueueSize, SplitQueueInterval: config.TimerProcessorSplitQueueInterval, SplitQueueIntervalJitterCoefficient: config.TimerProcessorSplitQueueIntervalJitterCoefficient, diff --git a/service/history/queue/transfer_queue_processor_base.go b/service/history/queue/transfer_queue_processor_base.go index c9feed7e5a6..019760a26a2 100644 --- a/service/history/queue/transfer_queue_processor_base.go +++ b/service/history/queue/transfer_queue_processor_base.go @@ -108,7 +108,7 @@ func newTransferQueueProcessorBase( task.InitializeLoggerForTask(shard.GetShardID(), taskInfo, logger), taskFilter, taskExecutor, - processorBase.redispatchSingleTask, + processorBase.redispatcher.AddTask, shard.GetTimeSource(), shard.GetConfig().TransferTaskMaxRetryCount, emitDomainTag, @@ -133,13 +133,14 @@ func (t *transferQueueProcessorBase) Start() { t.logger.Info("", tag.LifeCycleStarting) defer t.logger.Info("", tag.LifeCycleStarted) + t.redispatcher.Start() + for _, queueCollections := range t.processingQueueCollections { t.upsertPollTime(queueCollections.Level(), time.Time{}) } - t.shutdownWG.Add(2) + t.shutdownWG.Add(1) go t.processorPump() - go t.redispatchLoop() } func (t *transferQueueProcessorBase) Stop() { @@ -156,6 +157,8 @@ func (t *transferQueueProcessorBase) Stop() { if success := common.AwaitWaitGroup(&t.shutdownWG, time.Minute); !success { t.logger.Warn("", tag.LifeCycleStopTimedout) } + + t.redispatcher.Stop() } func (t *transferQueueProcessorBase) notifyNewTask() { @@ -195,9 +198,16 @@ processorPumpLoop: case <-t.notifyCh: t.upsertPollTime(defaultProcessingQueueLevel, time.Time{}) case <-t.nextPollTimer.FireChan(): - if t.redispatchQueue.Len() > t.options.MaxRedispatchQueueSize() { + maxRedispatchQueueSize := t.options.MaxRedispatchQueueSize() + if t.redispatcher.Size() > maxRedispatchQueueSize { // has too many pending tasks in re-dispatch queue, block loading tasks from persistence - t.redispatchTasks() + t.redispatcher.Redispatch(maxRedispatchQueueSize) + if t.redispatcher.Size() > maxRedispatchQueueSize { + // if redispatcher still has a large number of tasks + // this only happens when system is under very high load + // we should backoff here instead of keeping submitting tasks to task processor + time.Sleep(loadQueueTaskThrottleRetryDelay) + } // re-enqueue the event to see if we need keep re-dispatching or load new tasks from persistence t.nextPollTimer.Update(time.Time{}) continue processorPumpLoop @@ -416,8 +426,8 @@ func newTransferQueueProcessorOptions( MaxPollIntervalJitterCoefficient: config.TransferProcessorMaxPollIntervalJitterCoefficient, UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, - RedispatchInterval: config.TransferProcessorRedispatchInterval, - RedispatchIntervalJitterCoefficient: config.TransferProcessorRedispatchIntervalJitterCoefficient, + RedispatchInterval: config.TaskRedispatchInterval, + RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, SplitQueueInterval: config.TransferProcessorSplitQueueInterval, SplitQueueIntervalJitterCoefficient: config.TransferProcessorSplitQueueIntervalJitterCoefficient, diff --git a/service/history/queueProcessor.go b/service/history/queueProcessor.go index 255c351e8f1..aa51fe870d4 100644 --- a/service/history/queueProcessor.go +++ b/service/history/queueProcessor.go @@ -30,14 +30,12 @@ import ( "github.com/uber/cadence/common" "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/clock" - "github.com/uber/cadence/common/collection" "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/tag" "github.com/uber/cadence/common/metrics" "github.com/uber/cadence/common/quotas" "github.com/uber/cadence/common/service/dynamicconfig" "github.com/uber/cadence/service/history/execution" - "github.com/uber/cadence/service/history/queue" "github.com/uber/cadence/service/history/shard" "github.com/uber/cadence/service/history/task" ) @@ -72,13 +70,12 @@ type ( ackMgr *queueAckMgrImpl taskProcessor *taskProcessor // TODO: deprecate task processor, in favor of queueTaskProcessor queueTaskProcessor task.Processor - redispatchQueue collection.Queue + redispatcher task.Redispatcher queueTaskInitializer task.Initializer lastPollTime time.Time - notifyCh chan struct{} - redispatchNotifyCh chan struct{} + notifyCh chan struct{} status int32 shutdownWG sync.WaitGroup @@ -126,7 +123,6 @@ func newQueueProcessorBase( }, ), status: common.DaemonStatusInitialized, - redispatchNotifyCh: make(chan struct{}, 1), notifyCh: make(chan struct{}, 1), shutdownCh: make(chan struct{}), logger: logger, @@ -135,7 +131,15 @@ func newQueueProcessorBase( lastPollTime: time.Time{}, taskProcessor: taskProcessor, queueTaskProcessor: queueTaskProcessor, - redispatchQueue: collection.NewConcurrentQueue(), + redispatcher: task.NewRedispatcher( + queueTaskProcessor, + &task.RedispatcherOptions{ + TaskRedispatchInterval: options.RedispatchInterval, + TaskRedispatchIntervalJitterCoefficient: options.RedispatchIntervalJitterCoefficient, + }, + logger, + metricsScope, + ), } if options.QueueType != task.QueueTypeReplication { @@ -149,7 +153,7 @@ func newQueueProcessorBase( task.InitializeLoggerForTask(shard.GetShardID(), taskInfo, logger), taskFilter, taskExecutor, - p.redispatchSingleTask, + p.redispatcher.AddTask, p.timeSource, options.MaxRetryCount, emitDomainTag, @@ -166,21 +170,18 @@ func (p *queueProcessorBase) Start() { return } - p.logger.Info("", tag.LifeCycleStarting, tag.ComponentTransferQueue) - defer p.logger.Info("", tag.LifeCycleStarted, tag.ComponentTransferQueue) + p.logger.Info("", tag.LifeCycleStarting) + defer p.logger.Info("", tag.LifeCycleStarted) if p.taskProcessor != nil { p.taskProcessor.start() } + p.redispatcher.Start() + p.notifyNewTask() p.shutdownWG.Add(1) go p.processorPump() - - if p.isPriorityTaskProcessorEnabled() { - p.shutdownWG.Add(1) - go p.redispatchLoop() - } } func (p *queueProcessorBase) Stop() { @@ -188,19 +189,20 @@ func (p *queueProcessorBase) Stop() { return } - p.logger.Info("", tag.LifeCycleStopping, tag.ComponentTransferQueue) - defer p.logger.Info("", tag.LifeCycleStopped, tag.ComponentTransferQueue) + p.logger.Info("", tag.LifeCycleStopping) + defer p.logger.Info("", tag.LifeCycleStopped) close(p.shutdownCh) p.retryTasks() if success := common.AwaitWaitGroup(&p.shutdownWG, time.Minute); !success { - p.logger.Warn("", tag.LifeCycleStopTimedout, tag.ComponentTransferQueue) + p.logger.Warn("", tag.LifeCycleStopTimedout) } if p.taskProcessor != nil { p.taskProcessor.stop() } + p.redispatcher.Stop() } func (p *queueProcessorBase) notifyNewTask() { @@ -211,18 +213,6 @@ func (p *queueProcessorBase) notifyNewTask() { } } -func (p *queueProcessorBase) redispatchSingleTask(task task.Task) { - p.redispatchQueue.Add(task) - p.notifyRedispatch() -} - -func (p *queueProcessorBase) notifyRedispatch() { - select { - case p.redispatchNotifyCh <- struct{}{}: - default: - } -} - func (p *queueProcessorBase) processorPump() { defer p.shutdownWG.Done() @@ -247,13 +237,20 @@ processorPumpLoop: // use a separate goroutine since the caller hold the shutdownWG go p.Stop() case <-p.notifyCh: - if !p.isPriorityTaskProcessorEnabled() || p.redispatchQueue.Len() <= p.options.MaxRedispatchQueueSize() { + maxRedispatchQueueSize := p.options.MaxRedispatchQueueSize() + if !p.isPriorityTaskProcessorEnabled() || p.redispatcher.Size() <= maxRedispatchQueueSize { p.processBatch() continue } // has too many pending tasks in re-dispatch queue, block loading tasks from persistence - p.redispatchTasks() + p.redispatcher.Redispatch(maxRedispatchQueueSize) + if p.redispatcher.Size() > maxRedispatchQueueSize { + // if redispatcher still has a large number of tasks + // this only happens when system is under very high load + // we should backoff here instead of keeping submitting tasks to task processor + time.Sleep(loadQueueTaskThrottleRetryDelay) + } // re-enqueue the event to see if we need keep re-dispatching or load new tasks from persistence p.notifyNewTask() case <-pollTimer.C: @@ -280,42 +277,6 @@ processorPumpLoop: p.logger.Info("Queue processor pump shut down.") } -func (p *queueProcessorBase) redispatchLoop() { - defer p.shutdownWG.Done() - -redispatchTaskLoop: - for { - select { - case <-p.shutdownCh: - break redispatchTaskLoop - case <-p.redispatchNotifyCh: - // TODO: revisit the cpu usage and gc activity caused by - // creating timers and reading dynamicconfig if it becomes a problem. - backoffTimer := time.NewTimer(backoff.JitDuration( - p.options.RedispatchInterval(), - p.options.RedispatchIntervalJitterCoefficient(), - )) - select { - case <-p.shutdownCh: - backoffTimer.Stop() - break redispatchTaskLoop - case <-backoffTimer.C: - } - backoffTimer.Stop() - - // drain redispatchNotifyCh again - select { - case <-p.redispatchNotifyCh: - default: - } - - p.redispatchTasks() - } - } - - p.logger.Info("Queue processor task redispatch loop shut down.") -} - func (p *queueProcessorBase) processBatch() { ctx, cancel := context.WithTimeout(context.Background(), loadQueueTaskThrottleRetryDelay) @@ -385,30 +346,12 @@ func (p *queueProcessorBase) submitTask( } } if err != nil || !submitted { - p.redispatchSingleTask(queueTask) + p.redispatcher.AddTask(queueTask) } return true } -func (p *queueProcessorBase) redispatchTasks() { - if !p.isPriorityTaskProcessorEnabled() { - return - } - - queue.RedispatchTasks( - p.redispatchQueue, - p.queueTaskProcessor, - p.logger, - p.metricsScope, - p.shutdownCh, - ) - - if !p.redispatchQueue.IsEmpty() { - p.notifyRedispatch() - } -} - func (p *queueProcessorBase) retryTasks() { if p.taskProcessor != nil { p.taskProcessor.retryTasks() diff --git a/service/history/replicatorQueueProcessor.go b/service/history/replicatorQueueProcessor.go index 7617747ee1d..22b7befacbd 100644 --- a/service/history/replicatorQueueProcessor.go +++ b/service/history/replicatorQueueProcessor.go @@ -92,8 +92,8 @@ func newReplicatorQueueProcessor( UpdateAckInterval: config.ReplicatorProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.ReplicatorProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.ReplicatorTaskMaxRetryCount, - RedispatchInterval: config.ReplicatorProcessorRedispatchInterval, - RedispatchIntervalJitterCoefficient: config.ReplicatorProcessorRedispatchIntervalJitterCoefficient, + RedispatchInterval: config.TaskRedispatchInterval, + RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.ReplicatorProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.ReplicatorProcessorEnablePriorityTaskProcessor, MetricScope: metrics.ReplicatorQueueProcessorScope, diff --git a/service/history/task/interface.go b/service/history/task/interface.go index d5249cb26ae..e3bc75b29f0 100644 --- a/service/history/task/interface.go +++ b/service/history/task/interface.go @@ -80,6 +80,15 @@ type ( TrySubmit(Task) (bool, error) } + // Redispatcher buffers tasks and periodically redispatch them to Processor + // redispatch can also be triggered immediately by calling the Redispatch method + Redispatcher interface { + common.Daemon + AddTask(Task) + Redispatch(targetSize int) + Size() int + } + // QueueType is the type of task queue QueueType int ) diff --git a/service/history/task/interface_mock.go b/service/history/task/interface_mock.go index 917051b8f8e..f0073b4a2ca 100644 --- a/service/history/task/interface_mock.go +++ b/service/history/task/interface_mock.go @@ -625,3 +625,88 @@ func (mr *MockProcessorMockRecorder) TrySubmit(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TrySubmit", reflect.TypeOf((*MockProcessor)(nil).TrySubmit), arg0) } + +// MockRedispatcher is a mock of Redispatcher interface +type MockRedispatcher struct { + ctrl *gomock.Controller + recorder *MockRedispatcherMockRecorder +} + +// MockRedispatcherMockRecorder is the mock recorder for MockRedispatcher +type MockRedispatcherMockRecorder struct { + mock *MockRedispatcher +} + +// NewMockRedispatcher creates a new mock instance +func NewMockRedispatcher(ctrl *gomock.Controller) *MockRedispatcher { + mock := &MockRedispatcher{ctrl: ctrl} + mock.recorder = &MockRedispatcherMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockRedispatcher) EXPECT() *MockRedispatcherMockRecorder { + return m.recorder +} + +// Start mocks base method +func (m *MockRedispatcher) Start() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Start") +} + +// Start indicates an expected call of Start +func (mr *MockRedispatcherMockRecorder) Start() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Start", reflect.TypeOf((*MockRedispatcher)(nil).Start)) +} + +// Stop mocks base method +func (m *MockRedispatcher) Stop() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Stop") +} + +// Stop indicates an expected call of Stop +func (mr *MockRedispatcherMockRecorder) Stop() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockRedispatcher)(nil).Stop)) +} + +// AddTask mocks base method +func (m *MockRedispatcher) AddTask(arg0 Task) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "AddTask", arg0) +} + +// AddTask indicates an expected call of AddTask +func (mr *MockRedispatcherMockRecorder) AddTask(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddTask", reflect.TypeOf((*MockRedispatcher)(nil).AddTask), arg0) +} + +// Redispatch mocks base method +func (m *MockRedispatcher) Redispatch(targetSize int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Redispatch", targetSize) +} + +// Redispatch indicates an expected call of Redispatch +func (mr *MockRedispatcherMockRecorder) Redispatch(targetSize interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Redispatch", reflect.TypeOf((*MockRedispatcher)(nil).Redispatch), targetSize) +} + +// Size mocks base method +func (m *MockRedispatcher) Size() int { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Size") + ret0, _ := ret[0].(int) + return ret0 +} + +// Size indicates an expected call of Size +func (mr *MockRedispatcherMockRecorder) Size() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Size", reflect.TypeOf((*MockRedispatcher)(nil).Size)) +} diff --git a/service/history/task/priority_assigner.go b/service/history/task/priority_assigner.go index 1ce9a46f679..3fe94e60436 100644 --- a/service/history/task/priority_assigner.go +++ b/service/history/task/priority_assigner.go @@ -69,6 +69,10 @@ func NewPriorityAssigner( func (a *priorityAssignerImpl) Assign( queueTask Task, ) error { + if queueTask.Priority() != task.NoPriority { + return nil + } + queueType := queueTask.GetQueueType() if queueType == QueueTypeReplication { diff --git a/service/history/task/priority_assigner_test.go b/service/history/task/priority_assigner_test.go index 45339e0b2c4..62c004beb18 100644 --- a/service/history/task/priority_assigner_test.go +++ b/service/history/task/priority_assigner_test.go @@ -24,7 +24,7 @@ import ( "errors" "testing" - gomock "github.com/golang/mock/gomock" + "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/uber-go/tally" @@ -140,6 +140,7 @@ func (s *taskPriorityAssignerSuite) TestGetDomainInfo_Fail_UnknownError() { func (s *taskPriorityAssignerSuite) TestAssign_ReplicationTask() { mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeReplication).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.LowPriorityClass, task.DefaultPrioritySubclass)).Times(1) err := s.priorityAssigner.Assign(mockTask) @@ -156,6 +157,7 @@ func (s *taskPriorityAssignerSuite) TestAssign_StandbyTask_StandbyDomain() { mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeStandbyTransfer).AnyTimes() mockTask.EXPECT().GetDomainID().Return(constants.TestDomainID).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.LowPriorityClass, task.DefaultPrioritySubclass)).Times(1) err := s.priorityAssigner.Assign(mockTask) @@ -168,6 +170,7 @@ func (s *taskPriorityAssignerSuite) TestAssign_StandbyTask_ActiveDomain() { mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeStandbyTransfer).AnyTimes() mockTask.EXPECT().GetDomainID().Return(constants.TestDomainID).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.HighPriorityClass, task.DefaultPrioritySubclass)).Times(1) err := s.priorityAssigner.Assign(mockTask) @@ -184,6 +187,7 @@ func (s *taskPriorityAssignerSuite) TestAssign_ActiveTask_StandbyDomain() { mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeActiveTimer).AnyTimes() mockTask.EXPECT().GetDomainID().Return(constants.TestDomainID).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.HighPriorityClass, task.DefaultPrioritySubclass)).Times(1) err := s.priorityAssigner.Assign(mockTask) @@ -196,6 +200,7 @@ func (s *taskPriorityAssignerSuite) TestAssign_ActiveTransferTask_ActiveDomain() mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeActiveTransfer).AnyTimes() mockTask.EXPECT().GetDomainID().Return(constants.TestDomainID).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.HighPriorityClass, task.DefaultPrioritySubclass)).Times(1) err := s.priorityAssigner.Assign(mockTask) @@ -208,6 +213,7 @@ func (s *taskPriorityAssignerSuite) TestAssign_ActiveTimerTask_ActiveDomain() { mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeActiveTimer).AnyTimes() mockTask.EXPECT().GetDomainID().Return(constants.TestDomainID).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.HighPriorityClass, task.DefaultPrioritySubclass)).Times(1) err := s.priorityAssigner.Assign(mockTask) @@ -221,6 +227,7 @@ func (s *taskPriorityAssignerSuite) TestAssign_ThrottledTask() { mockTask := NewMockTask(s.controller) mockTask.EXPECT().GetQueueType().Return(QueueTypeActiveTimer).AnyTimes() mockTask.EXPECT().GetDomainID().Return(constants.TestDomainID).Times(1) + mockTask.EXPECT().Priority().Return(task.NoPriority).Times(1) if i < s.testTaskProcessRPS { mockTask.EXPECT().SetPriority(task.GetTaskPriority(task.HighPriorityClass, task.DefaultPrioritySubclass)).Times(1) } else { @@ -232,6 +239,17 @@ func (s *taskPriorityAssignerSuite) TestAssign_ThrottledTask() { } } +func (s *taskPriorityAssignerSuite) TestAssign_AlreadyAssigned() { + priority := 5 + + mockTask := NewMockTask(s.controller) + mockTask.EXPECT().Priority().Return(priority).AnyTimes() + + err := s.priorityAssigner.Assign(mockTask) + s.NoError(err) + s.Equal(priority, mockTask.Priority()) +} + func (s *taskPriorityAssignerSuite) TestGetTaskPriority() { testCases := []struct { class int diff --git a/service/history/task/redispatcher.go b/service/history/task/redispatcher.go new file mode 100644 index 00000000000..29a53656250 --- /dev/null +++ b/service/history/task/redispatcher.go @@ -0,0 +1,285 @@ +// 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 task + +import ( + "sync" + "sync/atomic" + "time" + + "github.com/uber/cadence/common" + "github.com/uber/cadence/common/backoff" + "github.com/uber/cadence/common/log" + "github.com/uber/cadence/common/log/tag" + "github.com/uber/cadence/common/metrics" + "github.com/uber/cadence/common/service/dynamicconfig" +) + +const ( + defaultBufferSize = 200 +) + +type ( + redispatchNotification struct { + targetSize int + doneCh chan struct{} + } + + // RedispatcherOptions configs redispatch interval + RedispatcherOptions struct { + TaskRedispatchInterval dynamicconfig.DurationPropertyFn + TaskRedispatchIntervalJitterCoefficient dynamicconfig.FloatPropertyFn + } + + redispatcherImpl struct { + sync.Mutex + + taskProcessor Processor + options *RedispatcherOptions + logger log.Logger + metricsScope metrics.Scope + + status int32 + shutdownCh chan struct{} + shutdownWG sync.WaitGroup + redispatchCh chan redispatchNotification + redispatchTimer *time.Timer + taskQueues map[int][]Task // priority -> redispatch queue + } +) + +// NewRedispatcher creates a new task Redispatcher +func NewRedispatcher( + taskProcessor Processor, + options *RedispatcherOptions, + logger log.Logger, + metricsScope metrics.Scope, +) Redispatcher { + return &redispatcherImpl{ + taskProcessor: taskProcessor, + options: options, + logger: logger, + metricsScope: metricsScope, + status: common.DaemonStatusInitialized, + shutdownCh: make(chan struct{}), + redispatchCh: make(chan redispatchNotification, 1), + redispatchTimer: nil, + taskQueues: make(map[int][]Task), + } +} + +func (r *redispatcherImpl) Start() { + if !atomic.CompareAndSwapInt32(&r.status, common.DaemonStatusInitialized, common.DaemonStatusStarted) { + return + } + + r.shutdownWG.Add(1) + go r.redispatchLoop() + + r.logger.Info("Task redispatcher started.", tag.LifeCycleStarted) +} + +func (r *redispatcherImpl) Stop() { + if !atomic.CompareAndSwapInt32(&r.status, common.DaemonStatusStarted, common.DaemonStatusStopped) { + return + } + + close(r.shutdownCh) + + r.Lock() + if r.redispatchTimer != nil { + r.redispatchTimer.Stop() + } + r.redispatchTimer = nil + r.Unlock() + + if success := common.AwaitWaitGroup(&r.shutdownWG, time.Minute); !success { + r.logger.Warn("Task redispatcher timedout on shutdown.", tag.LifeCycleStopTimedout) + } + + r.logger.Info("Task redispatcher stopped.", tag.LifeCycleStopped) +} + +func (r *redispatcherImpl) AddTask( + task Task, +) { + r.Lock() + defer r.Unlock() + + priority := task.Priority() + queue, ok := r.taskQueues[priority] + if !ok { + queue = make([]Task, 0) + } + r.taskQueues[priority] = append(queue, task) + + r.setupTimerLocked() +} + +func (r *redispatcherImpl) Redispatch( + targetSize int, +) { + doneCh := make(chan struct{}) + + select { + case r.redispatchCh <- redispatchNotification{ + targetSize: targetSize, + doneCh: doneCh, + }: + case <-r.shutdownCh: + close(doneCh) + } + + // block until the redispatch is done + <-doneCh +} + +func (r *redispatcherImpl) Size() int { + r.Lock() + defer r.Unlock() + + return r.sizeLocked() +} + +func (r *redispatcherImpl) redispatchLoop() { + defer r.shutdownWG.Done() + + for { + select { + case <-r.shutdownCh: + return + case notification := <-r.redispatchCh: + r.redispatchTasks(notification) + } + } +} + +func (r *redispatcherImpl) redispatchTasks( + notification redispatchNotification, +) { + r.Lock() + defer r.Unlock() + + defer func() { + if notification.doneCh != nil { + close(notification.doneCh) + } + if r.sizeLocked() > 0 && !r.isStopped() { + // there are still tasks left in the queue, setup a redispatch timer for those tasks + r.setupTimerLocked() + } + }() + + if r.isStopped() { + return + } + + queueSize := r.sizeLocked() + r.metricsScope.RecordTimer(metrics.TaskRedispatchQueuePendingTasksTimer, time.Duration(queueSize)) + + // add some buffer here as new tasks may be added + targetRedispatched := queueSize + defaultBufferSize - notification.targetSize + if targetRedispatched <= 0 { + // target size has already been met, no need to redispatch + return + } + + totalRedispatched := 0 + for priority, queue := range r.taskQueues { + queueLen := len(queue) + for i := 0; i != queueLen; i++ { + if totalRedispatched >= targetRedispatched { + break + } + + task := queue[0] + queue[0] = nil + queue = queue[1:] + + submitted, err := r.taskProcessor.TrySubmit(task) + if err != nil { + if r.isStopped() { + // if error is due to shard shutdown + break + } else { + // otherwise it might be error from domain cache etc, add + // the task to redispatch queue so that it can be retried + r.logger.Error("Failed to redispatch task", tag.Error(err)) + } + } + + if err != nil || !submitted { + // failed to submit, enqueue again + queue = append(queue, task) + } + + if err == nil && !submitted { + // task chan is full for this priority, continue to next priority + break + } + + totalRedispatched++ + } + + r.taskQueues[priority] = queue + + if r.isStopped() { + return + } + } +} + +func (r *redispatcherImpl) setupTimerLocked() { + if r.redispatchTimer == nil { + r.redispatchTimer = time.AfterFunc( + backoff.JitDuration( + r.options.TaskRedispatchInterval(), + r.options.TaskRedispatchIntervalJitterCoefficient(), + ), + func() { + r.Lock() + defer r.Unlock() + r.redispatchTimer = nil + + select { + case r.redispatchCh <- redispatchNotification{ + targetSize: 0, + doneCh: nil, + }: + default: + } + }, + ) + } +} + +func (r *redispatcherImpl) sizeLocked() int { + size := 0 + for _, queue := range r.taskQueues { + size += len(queue) + } + + return size +} + +func (r *redispatcherImpl) isStopped() bool { + return atomic.LoadInt32(&r.status) == common.DaemonStatusStopped +} diff --git a/service/history/task/redispatcher_test.go b/service/history/task/redispatcher_test.go new file mode 100644 index 00000000000..0a6b7ed7ce7 --- /dev/null +++ b/service/history/task/redispatcher_test.go @@ -0,0 +1,163 @@ +// 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 task + +import ( + "errors" + "math/rand" + "testing" + "time" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/uber-go/tally" + + "github.com/uber/cadence/common/log" + "github.com/uber/cadence/common/log/loggerimpl" + "github.com/uber/cadence/common/metrics" + "github.com/uber/cadence/common/service/dynamicconfig" +) + +type ( + redispatcherSuite struct { + suite.Suite + *require.Assertions + + controller *gomock.Controller + mockProcessor *MockProcessor + + metricsScope metrics.Scope + logger log.Logger + + redispatcher *redispatcherImpl + } +) + +func TestRedispatcherSuite(t *testing.T) { + s := new(redispatcherSuite) + suite.Run(t, s) +} + +func (s *redispatcherSuite) SetupTest() { + s.Assertions = require.New(s.T()) + + s.controller = gomock.NewController(s.T()) + s.mockProcessor = NewMockProcessor(s.controller) + + s.metricsScope = metrics.NewClient(tally.NoopScope, metrics.History).Scope(0) + s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) + + s.redispatcher = s.newTestRedispatcher() + s.redispatcher.Start() +} + +func (s *redispatcherSuite) TearDownTest() { + s.redispatcher.Stop() + s.Nil(s.redispatcher.redispatchTimer) + s.controller.Finish() +} + +func (s *redispatcherSuite) TestRedispatch_ProcessorShutDown() { + numTasks := 5 + + successfullyRedispatched := 3 + stoppedCh := make(chan struct{}) + for i := 0; i != successfullyRedispatched-1; i++ { + s.mockProcessor.EXPECT().TrySubmit(gomock.Any()).Return(true, nil).Times(1) + } + s.mockProcessor.EXPECT().TrySubmit(gomock.Any()).DoAndReturn(func(_ interface{}) (bool, error) { + go func() { + s.redispatcher.Stop() + close(stoppedCh) + }() + return true, nil + }).Times(1) + s.mockProcessor.EXPECT().TrySubmit(gomock.Any()).DoAndReturn(func(_ interface{}) (bool, error) { + <-s.redispatcher.shutdownCh + return false, errors.New("processor shutdown") + }).Times(1) + + for i := 0; i != numTasks; i++ { + mockTask := NewMockTask(s.controller) + mockTask.EXPECT().Priority().Return(rand.Intn(5)).AnyTimes() + s.redispatcher.AddTask(mockTask) + } + + s.Equal(numTasks, s.redispatcher.Size()) + + // redispatch will be triggered by the background redispatch loop + <-stoppedCh + + s.Equal(numTasks-successfullyRedispatched-1, s.redispatcher.Size()) +} + +func (s *redispatcherSuite) TestRedispatch_WithTargetSize() { + numTasks := defaultBufferSize + 20 + targetSize := defaultBufferSize + 10 + + for i := 0; i != numTasks; i++ { + mockTask := NewMockTask(s.controller) + mockTask.EXPECT().Priority().Return(rand.Intn(5)).AnyTimes() + s.redispatcher.AddTask(mockTask) + s.mockProcessor.EXPECT().TrySubmit(gomock.Any()).Return(true, nil).MaxTimes(1) + } + + s.redispatcher.Redispatch(targetSize) + + // implementation can choose to redispatch more tasks than needed + s.True(s.redispatcher.Size() <= targetSize) + s.True(s.redispatcher.Size() > 0) +} + +func (s *redispatcherSuite) TestRedispatch_Random() { + numTasks := 10 + dispatched := 0 + + for i := 0; i != numTasks; i++ { + mockTask := NewMockTask(s.controller) + mockTask.EXPECT().Priority().Return(rand.Intn(5)).AnyTimes() + s.redispatcher.AddTask(mockTask) + submitted := false + if rand.Intn(2) == 0 { + submitted = true + dispatched++ + } + s.mockProcessor.EXPECT().TrySubmit(NewMockTaskMatcher(mockTask)).Return(submitted, nil).MaxTimes(1) + } + + s.redispatcher.Redispatch(0) + + // implementation can choose to stop redispatch for a certain priority when previous submit has failed + s.True(s.redispatcher.Size() >= numTasks-dispatched) +} + +func (s *redispatcherSuite) newTestRedispatcher() *redispatcherImpl { + return NewRedispatcher( + s.mockProcessor, + &RedispatcherOptions{ + TaskRedispatchInterval: dynamicconfig.GetDurationPropertyFn(time.Millisecond * 50), + TaskRedispatchIntervalJitterCoefficient: dynamicconfig.GetFloatPropertyFn(0.15), + }, + s.logger, + s.metricsScope, + ).(*redispatcherImpl) +} diff --git a/service/history/task/task.go b/service/history/task/task.go index b1f033162dc..04053e94f0d 100644 --- a/service/history/task/task.go +++ b/service/history/task/task.go @@ -182,6 +182,7 @@ func newQueueTaskBase( Info: taskInfo, shard: shard, state: ctask.TaskStatePending, + priority: ctask.NoPriority, queueType: queueType, scopeIdx: scopeIdx, emitDomainTag: emitDomainTag, diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index 01e48eecc37..4e7e6fba94a 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -29,7 +29,6 @@ import ( "github.com/uber/cadence/common" "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/clock" - "github.com/uber/cadence/common/collection" "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/tag" "github.com/uber/cadence/common/metrics" @@ -46,7 +45,6 @@ var ( emptyTime = time.Time{} loadDomainEntryForTimerTaskRetryDelay = 100 * time.Millisecond - loadTimerTaskThrottleRetryDelay = 5 * time.Second ) type ( @@ -69,8 +67,7 @@ type ( lastPollTime time.Time taskProcessor *taskProcessor // TODO: deprecate task processor, in favor of queueTaskProcessor queueTaskProcessor task.Processor - redispatchNotifyCh chan struct{} - redispatchQueue collection.Queue + redispatcher task.Redispatcher queueTaskInitializer task.Initializer // timer notification @@ -124,8 +121,15 @@ func newTimerQueueProcessorBase( lastPollTime: time.Time{}, taskProcessor: taskProcessor, queueTaskProcessor: queueTaskProcessor, - redispatchQueue: collection.NewConcurrentQueue(), - redispatchNotifyCh: make(chan struct{}, 1), + redispatcher: task.NewRedispatcher( + queueTaskProcessor, + &task.RedispatcherOptions{ + TaskRedispatchInterval: config.TaskRedispatchInterval, + TaskRedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, + }, + logger, + metricsScope, + ), rateLimiter: quotas.NewDynamicRateLimiter( func() float64 { return float64(maxPollRPS()) @@ -148,7 +152,7 @@ func newTimerQueueProcessorBase( task.InitializeLoggerForTask(shard.GetShardID(), taskInfo, logger), taskFilter, taskExecutor, - base.redispatchSingleTask, + base.redispatcher.AddTask, shard.GetTimeSource(), config.TimerTaskMaxRetryCount, emitDomainTag, @@ -167,17 +171,14 @@ func (t *timerQueueProcessorBase) Start() { if t.taskProcessor != nil { t.taskProcessor.start() } + t.redispatcher.Start() + // notify a initial scan t.notifyNewTimer(time.Time{}) t.shutdownWG.Add(1) go t.processorPump() - if t.isPriorityTaskProcessorEnabled() { - t.shutdownWG.Add(1) - go t.redispatchLoop() - } - t.logger.Info("Timer queue processor started.") } @@ -197,6 +198,8 @@ func (t *timerQueueProcessorBase) Stop() { if t.taskProcessor != nil { t.taskProcessor.stop() } + t.redispatcher.Stop() + t.logger.Info("Timer queue processor stopped.") } @@ -219,42 +222,6 @@ RetryProcessor: t.logger.Info("Timer queue processor pump shutting down.") } -func (t *timerQueueProcessorBase) redispatchLoop() { - defer t.shutdownWG.Done() - -redispatchTaskLoop: - for { - select { - case <-t.shutdownCh: - break redispatchTaskLoop - case <-t.redispatchNotifyCh: - // TODO: revisit the cpu usage and gc activity caused by - // creating timers and reading dynamicconfig if it becomes a problem. - backoffTimer := time.NewTimer(backoff.JitDuration( - t.config.TimerProcessorRedispatchInterval(), - t.config.TimerProcessorRedispatchIntervalJitterCoefficient(), - )) - select { - case <-t.shutdownCh: - backoffTimer.Stop() - break redispatchTaskLoop - case <-backoffTimer.C: - } - backoffTimer.Stop() - - // drain redispatchNotifyCh again - select { - case <-t.redispatchNotifyCh: - default: - } - - t.redispatchTasks() - } - } - - t.logger.Info("Timer queue processor redispatch loop shut down.") -} - // NotifyNewTimers - Notify the processor about the new timer events arrival. // This should be called each time new timer events arrives, otherwise timers maybe fired unexpected. func (t *timerQueueProcessorBase) notifyNewTimers( @@ -298,18 +265,6 @@ func (t *timerQueueProcessorBase) notifyNewTimer( } } -func (t *timerQueueProcessorBase) redispatchSingleTask(task task.Task) { - t.redispatchQueue.Add(task) - t.notifyRedispatch() -} - -func (t *timerQueueProcessorBase) notifyRedispatch() { - select { - case t.redispatchNotifyCh <- struct{}{}: - default: - } -} - func (t *timerQueueProcessorBase) internalProcessor() error { pollTimer := time.NewTimer(backoff.JitDuration( t.config.TimerProcessorMaxPollInterval(), @@ -341,7 +296,8 @@ func (t *timerQueueProcessorBase) internalProcessor() error { go t.Stop() return nil case <-t.timerGate.FireChan(): - if !t.isPriorityTaskProcessorEnabled() || t.redispatchQueue.Len() <= t.config.TimerProcessorMaxRedispatchQueueSize() { + maxRedispatchQueueSize := t.config.TimerProcessorMaxRedispatchQueueSize() + if !t.isPriorityTaskProcessorEnabled() || t.redispatcher.Size() <= maxRedispatchQueueSize { lookAheadTimer, err := t.readAndFanoutTimerTasks() if err != nil { return err @@ -353,7 +309,15 @@ func (t *timerQueueProcessorBase) internalProcessor() error { } // has too many pending tasks in re-dispatch queue, block loading tasks from persistence - t.redispatchTasks() + t.redispatcher.Redispatch(maxRedispatchQueueSize) + if t.redispatcher.Size() > maxRedispatchQueueSize { + // if redispatcher still has a large number of tasks + // this only happens when system is under very high load + // we should backoff here instead of keeping submitting tasks to task processor + // don't call t.notifyNewTime(time.Now() + loadQueueTaskThrottleRetryDelay) as the time in + // standby timer processor is not real time and is managed separately + time.Sleep(loadQueueTaskThrottleRetryDelay) + } // re-enqueue the event to see if we need keep re-dispatching or load new tasks from persistence t.notifyNewTimer(time.Time{}) case <-pollTimer.C: @@ -393,7 +357,7 @@ func (t *timerQueueProcessorBase) internalProcessor() error { } func (t *timerQueueProcessorBase) readAndFanoutTimerTasks() (*persistence.TimerTaskInfo, error) { - ctx, cancel := ctx.WithTimeout(ctx.Background(), loadTimerTaskThrottleRetryDelay) + ctx, cancel := ctx.WithTimeout(ctx.Background(), loadQueueTaskThrottleRetryDelay) if err := t.rateLimiter.Wait(ctx); err != nil { cancel() t.notifyNewTimer(time.Time{}) // re-enqueue the event @@ -453,30 +417,12 @@ func (t *timerQueueProcessorBase) submitTask( } } if err != nil || !submitted { - t.redispatchSingleTask(timerQueueTask) + t.redispatcher.AddTask(timerQueueTask) } return true } -func (t *timerQueueProcessorBase) redispatchTasks() { - if !t.isPriorityTaskProcessorEnabled() { - return - } - - queue.RedispatchTasks( - t.redispatchQueue, - t.queueTaskProcessor, - t.logger, - t.metricsScope, - t.shutdownCh, - ) - - if !t.redispatchQueue.IsEmpty() { - t.notifyRedispatch() - } -} - func (t *timerQueueProcessorBase) retryTasks() { if t.taskProcessor != nil { t.taskProcessor.retryTasks() @@ -497,27 +443,3 @@ func (t *timerQueueProcessorBase) isPriorityTaskProcessorEnabled() bool { func (t *timerQueueProcessorBase) getTimerFiredCount() uint64 { return atomic.LoadUint64(&t.timerFiredCount) } - -//nolint:unused -func (t *timerQueueProcessorBase) getTimerTaskType( - taskType int, -) string { - - switch taskType { - case persistence.TaskTypeUserTimer: - return "UserTimer" - case persistence.TaskTypeActivityTimeout: - return "ActivityTimeout" - case persistence.TaskTypeDecisionTimeout: - return "DecisionTimeout" - case persistence.TaskTypeWorkflowTimeout: - return "WorkflowTimeout" - case persistence.TaskTypeDeleteHistoryEvent: - return "DeleteHistoryEvent" - case persistence.TaskTypeActivityRetryTimer: - return "ActivityRetryTimerTask" - case persistence.TaskTypeWorkflowBackoffTimer: - return "WorkflowBackoffTimerTask" - } - return "UnKnown" -} diff --git a/service/history/transferQueueActiveProcessor.go b/service/history/transferQueueActiveProcessor.go index f2d0692b993..50cea3df813 100644 --- a/service/history/transferQueueActiveProcessor.go +++ b/service/history/transferQueueActiveProcessor.go @@ -74,8 +74,8 @@ func newTransferQueueActiveProcessor( UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.TransferTaskMaxRetryCount, - RedispatchInterval: config.TransferProcessorRedispatchInterval, - RedispatchIntervalJitterCoefficient: config.TransferProcessorRedispatchIntervalJitterCoefficient, + RedispatchInterval: config.TaskRedispatchInterval, + RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.TransferProcessorEnablePriorityTaskProcessor, MetricScope: metrics.TransferActiveQueueProcessorScope, @@ -180,8 +180,8 @@ func newTransferQueueFailoverProcessor( UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.TransferTaskMaxRetryCount, - RedispatchInterval: config.TransferProcessorRedispatchInterval, - RedispatchIntervalJitterCoefficient: config.TransferProcessorRedispatchIntervalJitterCoefficient, + RedispatchInterval: config.TaskRedispatchInterval, + RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.TransferProcessorEnablePriorityTaskProcessor, MetricScope: metrics.TransferActiveQueueProcessorScope, diff --git a/service/history/transferQueueStandbyProcessor.go b/service/history/transferQueueStandbyProcessor.go index b6d3d78ce7e..9f6256190d1 100644 --- a/service/history/transferQueueStandbyProcessor.go +++ b/service/history/transferQueueStandbyProcessor.go @@ -72,8 +72,8 @@ func newTransferQueueStandbyProcessor( UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.TransferTaskMaxRetryCount, - RedispatchInterval: config.TransferProcessorRedispatchInterval, - RedispatchIntervalJitterCoefficient: config.TransferProcessorRedispatchIntervalJitterCoefficient, + RedispatchInterval: config.TaskRedispatchInterval, + RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.TransferProcessorEnablePriorityTaskProcessor, MetricScope: metrics.TransferStandbyQueueProcessorScope, From 624862f63ee0809f852884d8a278f411a5ac02ad Mon Sep 17 00:00:00 2001 From: Yichao Yang Date: Fri, 24 Jul 2020 15:13:38 -0700 Subject: [PATCH 2/6] Reduce standby task attempts (#3410) * Use different redispatch interval for active and standby tasks * Rename ErrTaskRetry to ErrTaskRedispatch and make it non-retryable. --- common/reconciliation/common/util.go | 2 +- common/service/dynamicconfig/constants.go | 9 ++++++--- service/history/config/config.go | 6 ++++-- service/history/queue/task_allocator.go | 2 +- .../queue/timer_queue_processor_base.go | 3 ++- .../queue/transfer_queue_processor_base.go | 3 ++- service/history/replicatorQueueProcessor.go | 2 +- service/history/task/processor_test.go | 2 ++ service/history/task/standby_task_util.go | 2 +- service/history/task/task.go | 12 +++++------ service/history/task/task_test.go | 4 ++-- .../task/timer_standby_task_executor.go | 4 ++-- .../task/timer_standby_task_executor_test.go | 20 +++++++++---------- .../task/transfer_standby_task_executor.go | 4 ++-- .../transfer_standby_task_executor_test.go | 16 +++++++-------- service/history/taskProcessor.go | 2 +- service/history/taskProcessor_test.go | 4 ++-- service/history/timerQueueProcessorBase.go | 20 ++++++++++--------- .../history/transferQueueActiveProcessor.go | 4 ++-- .../history/transferQueueStandbyProcessor.go | 2 +- .../worker/scanner/executions/workflows.go | 2 ++ 21 files changed, 69 insertions(+), 56 deletions(-) diff --git a/common/reconciliation/common/util.go b/common/reconciliation/common/util.go index 4b6a26b4d59..9edcce5c0b6 100644 --- a/common/reconciliation/common/util.go +++ b/common/reconciliation/common/util.go @@ -144,7 +144,7 @@ func Open(state int) bool { return state == persistence.WorkflowStateCreated || state == persistence.WorkflowStateRunning } -// Open returns true if execution state is open false if workflow is closed +// ExecutionOpen returns true if execution state is open false if workflow is closed func ExecutionOpen(execution interface{}) bool { return Open(getExecution(execution).State) } diff --git a/common/service/dynamicconfig/constants.go b/common/service/dynamicconfig/constants.go index 87d37d35ffe..4e176d89497 100644 --- a/common/service/dynamicconfig/constants.go +++ b/common/service/dynamicconfig/constants.go @@ -167,7 +167,8 @@ var keys = map[Key]string{ TaskSchedulerShardQueueSize: "history.taskSchedulerShardQueueSize", TaskSchedulerDispatcherCount: "history.taskSchedulerDispatcherCount", TaskSchedulerRoundRobinWeights: "history.taskSchedulerRoundRobinWeight", - TaskRedispatchInterval: "history.taskRedispatchInterval", + ActiveTaskRedispatchInterval: "history.activeTaskRedispatchInterval", + StandbyTaskRedispatchInterval: "history.standbyTaskRedispatchInterval", TaskRedispatchIntervalJitterCoefficient: "history.taskRedispatchIntervalJitterCoefficient", QueueProcessorEnableDomainTaggedMetrics: "history.queueProcessorEnableDomainTaggedMetrics", QueueProcessorEnableSplit: "history.queueProcessorEnableSplit", @@ -558,8 +559,10 @@ const ( TaskSchedulerDispatcherCount // TaskSchedulerRoundRobinWeights is the priority weight for weighted round robin task scheduler TaskSchedulerRoundRobinWeights - // TaskRedispatchInterval is the task redispatch interval - TaskRedispatchInterval + // ActiveTaskRedispatchInterval is the active task redispatch interval + ActiveTaskRedispatchInterval + // StandbyTaskRedispatchInterval is the active task redispatch interval + StandbyTaskRedispatchInterval // TaskRedispatchIntervalJitterCoefficient is the task redispatch interval jitter coefficient TaskRedispatchIntervalJitterCoefficient // QueueProcessorEnableDomainTaggedMetrics indicates whether task processing metrics should include domain tag diff --git a/service/history/config/config.go b/service/history/config/config.go index 2761a9ca70a..09121ccd66d 100644 --- a/service/history/config/config.go +++ b/service/history/config/config.go @@ -86,7 +86,8 @@ type Config struct { TaskSchedulerShardQueueSize dynamicconfig.IntPropertyFn TaskSchedulerDispatcherCount dynamicconfig.IntPropertyFn TaskSchedulerRoundRobinWeights dynamicconfig.MapPropertyFn - TaskRedispatchInterval dynamicconfig.DurationPropertyFn + ActiveTaskRedispatchInterval dynamicconfig.DurationPropertyFn + StandbyTaskRedispatchInterval dynamicconfig.DurationPropertyFn TaskRedispatchIntervalJitterCoefficient dynamicconfig.FloatPropertyFn // QueueProcessor settings @@ -320,7 +321,8 @@ func New(dc *dynamicconfig.Collection, numberOfShards int, storeType string, isA TaskSchedulerShardQueueSize: dc.GetIntProperty(dynamicconfig.TaskSchedulerShardQueueSize, 200), TaskSchedulerDispatcherCount: dc.GetIntProperty(dynamicconfig.TaskSchedulerDispatcherCount, 10), TaskSchedulerRoundRobinWeights: dc.GetMapProperty(dynamicconfig.TaskSchedulerRoundRobinWeights, common.ConvertIntMapToDynamicConfigMapProperty(DefaultTaskPriorityWeight)), - TaskRedispatchInterval: dc.GetDurationProperty(dynamicconfig.TaskRedispatchInterval, 5*time.Second), + ActiveTaskRedispatchInterval: dc.GetDurationProperty(dynamicconfig.ActiveTaskRedispatchInterval, 5*time.Second), + StandbyTaskRedispatchInterval: dc.GetDurationProperty(dynamicconfig.StandbyTaskRedispatchInterval, 30*time.Second), TaskRedispatchIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.TimerProcessorSplitQueueIntervalJitterCoefficient, 0.15), QueueProcessorEnableDomainTaggedMetrics: dc.GetBoolProperty(dynamicconfig.QueueProcessorEnableDomainTaggedMetrics, false), diff --git a/service/history/queue/task_allocator.go b/service/history/queue/task_allocator.go index 6ad83b715bb..d295d67d742 100644 --- a/service/history/queue/task_allocator.go +++ b/service/history/queue/task_allocator.go @@ -175,7 +175,7 @@ func (t *taskAllocatorImpl) checkDomainPendingActive( if domainEntry.IsGlobalDomain() && domainEntry.GetFailoverEndTime() != nil { // the domain is pending active, pause on processing this task t.logger.Debug("Domain is not in pending active, skip task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return htask.ErrTaskRedispatch + return htask.ErrTaskPendingActive } return nil } diff --git a/service/history/queue/timer_queue_processor_base.go b/service/history/queue/timer_queue_processor_base.go index 8431f1cd5fd..43e93a6b915 100644 --- a/service/history/queue/timer_queue_processor_base.go +++ b/service/history/queue/timer_queue_processor_base.go @@ -624,7 +624,6 @@ func newTimerQueueProcessorOptions( MaxPollIntervalJitterCoefficient: config.TimerProcessorMaxPollIntervalJitterCoefficient, UpdateAckInterval: config.TimerProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TimerProcessorUpdateAckIntervalJitterCoefficient, - RedispatchInterval: config.TaskRedispatchInterval, RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TimerProcessorMaxRedispatchQueueSize, SplitQueueInterval: config.TimerProcessorSplitQueueInterval, @@ -648,8 +647,10 @@ func newTimerQueueProcessorOptions( if isActive { options.MetricScope = metrics.TimerActiveQueueProcessorScope + options.RedispatchInterval = config.ActiveTaskRedispatchInterval } else { options.MetricScope = metrics.TimerStandbyQueueProcessorScope + options.RedispatchInterval = config.StandbyTaskRedispatchInterval } return options diff --git a/service/history/queue/transfer_queue_processor_base.go b/service/history/queue/transfer_queue_processor_base.go index 019760a26a2..9d0d7e7823e 100644 --- a/service/history/queue/transfer_queue_processor_base.go +++ b/service/history/queue/transfer_queue_processor_base.go @@ -426,7 +426,6 @@ func newTransferQueueProcessorOptions( MaxPollIntervalJitterCoefficient: config.TransferProcessorMaxPollIntervalJitterCoefficient, UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, - RedispatchInterval: config.TaskRedispatchInterval, RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, SplitQueueInterval: config.TransferProcessorSplitQueueInterval, @@ -450,8 +449,10 @@ func newTransferQueueProcessorOptions( if isActive { options.MetricScope = metrics.TransferActiveQueueProcessorScope + options.RedispatchInterval = config.ActiveTaskRedispatchInterval } else { options.MetricScope = metrics.TransferStandbyQueueProcessorScope + options.RedispatchInterval = config.StandbyTaskRedispatchInterval } return options diff --git a/service/history/replicatorQueueProcessor.go b/service/history/replicatorQueueProcessor.go index 22b7befacbd..8c99c045676 100644 --- a/service/history/replicatorQueueProcessor.go +++ b/service/history/replicatorQueueProcessor.go @@ -92,7 +92,7 @@ func newReplicatorQueueProcessor( UpdateAckInterval: config.ReplicatorProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.ReplicatorProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.ReplicatorTaskMaxRetryCount, - RedispatchInterval: config.TaskRedispatchInterval, + RedispatchInterval: config.StandbyTaskRedispatchInterval, RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.ReplicatorProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.ReplicatorProcessorEnablePriorityTaskProcessor, diff --git a/service/history/task/processor_test.go b/service/history/task/processor_test.go index 8a85c95648d..2331c71e77e 100644 --- a/service/history/task/processor_test.go +++ b/service/history/task/processor_test.go @@ -33,6 +33,7 @@ import ( "github.com/uber/cadence/common/log/loggerimpl" "github.com/uber/cadence/common/metrics" "github.com/uber/cadence/common/persistence" + "github.com/uber/cadence/common/service/dynamicconfig" "github.com/uber/cadence/common/task" "github.com/uber/cadence/service/history/config" "github.com/uber/cadence/service/history/shard" @@ -211,6 +212,7 @@ func (s *queueTaskProcessorSuite) TestNewSchedulerOptions_UnknownSchedulerType() func (s *queueTaskProcessorSuite) newTestQueueTaskProcessor() *processorImpl { config := config.NewForTest() + config.TaskSchedulerShardWorkerCount = dynamicconfig.GetIntPropertyFn(1) processor, err := NewProcessor( s.mockPriorityAssigner, config, diff --git a/service/history/task/standby_task_util.go b/service/history/task/standby_task_util.go index fbc7cf65afe..907c0eca2ab 100644 --- a/service/history/task/standby_task_util.go +++ b/service/history/task/standby_task_util.go @@ -49,7 +49,7 @@ func standbyTaskPostActionNoOp( } // return error so task processing logic will retry - return ErrTaskRetry + return ErrTaskRedispatch } func standbyTransferTaskPostActionTaskDiscarded( diff --git a/service/history/task/task.go b/service/history/task/task.go index 04053e94f0d..71f7aab9566 100644 --- a/service/history/task/task.go +++ b/service/history/task/task.go @@ -44,10 +44,10 @@ const ( var ( // ErrTaskDiscarded is the error indicating that the timer / transfer task is pending for too long and discarded. ErrTaskDiscarded = errors.New("passive task pending for too long") - // ErrTaskRetry is the error indicating that the timer / transfer task should be retried. - ErrTaskRetry = errors.New("passive task should retry due to condition in mutable state is not met") - // ErrTaskRedispatch is the error indicating that the task should be re-dispatch - ErrTaskRedispatch = errors.New("redispatch the task while the domain is pending-acitve") + // ErrTaskRedispatch is the error indicating that the timer / transfer task should be re0dispatched and retried. + ErrTaskRedispatch = errors.New("passive task should be redispatched due to condition in mutable state is not met") + // ErrTaskPendingActive is the error indicating that the task should be re-dispatched + ErrTaskPendingActive = errors.New("redispatch the task while the domain is pending-active") ) type ( @@ -291,7 +291,7 @@ func (t *taskBase) HandleErr( } // this is a transient error - if err == ErrTaskRetry { + if err == ErrTaskRedispatch { t.scope.IncCounter(metrics.TaskStandbyRetryCounter) return err } @@ -327,7 +327,7 @@ func (t *taskBase) HandleErr( func (t *taskBase) RetryErr( err error, ) bool { - if err == ErrTaskRedispatch { + if err == ErrTaskRedispatch || err == ErrTaskPendingActive { return false } diff --git a/service/history/task/task_test.go b/service/history/task/task_test.go index 520fd8f681e..c2abd01a84e 100644 --- a/service/history/task/task_test.go +++ b/service/history/task/task_test.go @@ -136,8 +136,8 @@ func (s *taskSuite) TestHandleErr_ErrTaskRetry() { return true, nil }) - err := ErrTaskRetry - s.Equal(ErrTaskRetry, taskBase.HandleErr(err)) + err := ErrTaskRedispatch + s.Equal(ErrTaskRedispatch, taskBase.HandleErr(err)) } func (s *taskSuite) TestHandleErr_ErrTaskDiscarded() { diff --git a/service/history/task/timer_standby_task_executor.go b/service/history/task/timer_standby_task_executor.go index 28fc1fa843e..e2b6dc9e483 100644 --- a/service/history/task/timer_standby_task_executor.go +++ b/service/history/task/timer_standby_task_executor.go @@ -410,7 +410,7 @@ func (t *timerStandbyTaskExecutor) processTimer( return err } defer func() { - if retError == ErrTaskRetry { + if retError == ErrTaskRedispatch { release(nil) } else { release(retError) @@ -492,7 +492,7 @@ func (t *timerStandbyTaskExecutor) fetchHistoryFromRemote( } // return error so task processing logic will retry - return ErrTaskRetry + return ErrTaskRedispatch } func (t *timerStandbyTaskExecutor) getCurrentTime() time.Time { diff --git a/service/history/task/timer_standby_task_executor_test.go b/service/history/task/timer_standby_task_executor_test.go index a6cad6a5206..d9f34a37cb6 100644 --- a/service/history/task/timer_standby_task_executor_test.go +++ b/service/history/task/timer_standby_task_executor_test.go @@ -222,7 +222,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Pending() { s.mockShard.SetCurrentTime(s.clusterName, s.now) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -231,7 +231,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Pending() { timerTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.discardDuration)) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) @@ -442,7 +442,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessActivityTimeout_Pending() { s.mockShard.SetCurrentTime(s.clusterName, s.now) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -451,7 +451,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessActivityTimeout_Pending() { timerTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.discardDuration)) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) @@ -770,7 +770,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessDecisionTimeout_Pending() { s.mockShard.SetCurrentTime(s.clusterName, s.now) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -779,7 +779,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessDecisionTimeout_Pending() { timerTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.discardDuration)) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) @@ -913,7 +913,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessWorkflowBackoffTimer_Pending( s.mockShard.SetCurrentTime(s.clusterName, s.now) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, time.Now().Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -922,7 +922,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessWorkflowBackoffTimer_Pending( timerTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, time.Now().Add(s.discardDuration)) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) @@ -1031,7 +1031,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessWorkflowTimeout_Pending() { s.mockShard.SetCurrentTime(s.clusterName, s.now) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -1040,7 +1040,7 @@ func (s *timerStandbyTaskExecutorSuite) TestProcessWorkflowTimeout_Pending() { timerTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.timerStandbyTaskExecutor.Execute(timerTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, s.now.Add(s.discardDuration)) err = s.timerStandbyTaskExecutor.Execute(timerTask, true) diff --git a/service/history/task/transfer_standby_task_executor.go b/service/history/task/transfer_standby_task_executor.go index ceb6832c95e..71b56403b39 100644 --- a/service/history/task/transfer_standby_task_executor.go +++ b/service/history/task/transfer_standby_task_executor.go @@ -486,7 +486,7 @@ func (t *transferStandbyTaskExecutor) processTransfer( return err } defer func() { - if retError == ErrTaskRetry { + if retError == ErrTaskRedispatch { release(nil) } else { release(retError) @@ -602,7 +602,7 @@ func (t *transferStandbyTaskExecutor) fetchHistoryFromRemote( } // return error so task processing logic will retry - return ErrTaskRetry + return ErrTaskRedispatch } func (t *transferStandbyTaskExecutor) getCurrentTime() time.Time { diff --git a/service/history/task/transfer_standby_task_executor_test.go b/service/history/task/transfer_standby_task_executor_test.go index 1f4f71e81c3..c389e2c19a4 100644 --- a/service/history/task/transfer_standby_task_executor_test.go +++ b/service/history/task/transfer_standby_task_executor_test.go @@ -229,7 +229,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessActivityTask_Pending() { s.mockShard.SetCurrentTime(s.clusterName, now) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) } func (s *transferStandbyTaskExecutorSuite) TestProcessActivityTask_Pending_PushToMatching() { @@ -409,7 +409,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessDecisionTask_Pending() { s.mockShard.SetCurrentTime(s.clusterName, now) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) } func (s *transferStandbyTaskExecutorSuite) TestProcessDecisionTask_Pending_PushToMatching() { @@ -713,7 +713,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessCancelExecution_Pending() s.mockShard.SetCurrentTime(s.clusterName, now) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -722,7 +722,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessCancelExecution_Pending() transferTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.discardDuration)) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) @@ -865,7 +865,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessSignalExecution_Pending() s.mockShard.SetCurrentTime(s.clusterName, now) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -874,7 +874,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessSignalExecution_Pending() transferTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.discardDuration)) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) @@ -1017,7 +1017,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessStartChildExecution_Pendin s.mockShard.SetCurrentTime(s.clusterName, now) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.fetchHistoryDuration)) s.mockHistoryRereplicator.On("SendMultiWorkflowHistory", @@ -1026,7 +1026,7 @@ func (s *transferStandbyTaskExecutorSuite) TestProcessStartChildExecution_Pendin transferTask.RunID, common.EndEventID, ).Return(nil).Once() err = s.transferStandbyTaskExecutor.Execute(transferTask, true) - s.Equal(ErrTaskRetry, err) + s.Equal(ErrTaskRedispatch, err) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.discardDuration)) err = s.transferStandbyTaskExecutor.Execute(transferTask, true) diff --git a/service/history/taskProcessor.go b/service/history/taskProcessor.go index 6fd684799ce..f5e2c9c8fe3 100644 --- a/service/history/taskProcessor.go +++ b/service/history/taskProcessor.go @@ -280,7 +280,7 @@ func (t *taskProcessor) handleTaskError( } // this is a transient error - if err == task.ErrTaskRetry { + if err == task.ErrTaskRedispatch { scope.IncCounter(metrics.TaskStandbyRetryCounter) select { case <-notificationChan: diff --git a/service/history/taskProcessor_test.go b/service/history/taskProcessor_test.go index b74e62cff13..0e0237a487d 100644 --- a/service/history/taskProcessor_test.go +++ b/service/history/taskProcessor_test.go @@ -198,7 +198,7 @@ func (s *taskProcessorSuite) TestHandleTaskError_EntityNotExists() { } func (s *taskProcessorSuite) TestHandleTaskError_ErrTaskRetry() { - err := task.ErrTaskRetry + err := task.ErrTaskRedispatch delay := time.Second taskInfo := newTaskInfo(s.mockProcessor, nil, s.logger, s.mockShard.GetTimeSource().Now()) @@ -210,7 +210,7 @@ func (s *taskProcessorSuite) TestHandleTaskError_ErrTaskRetry() { err = s.taskProcessor.handleTaskError(s.scope, taskInfo, s.notificationChan, err) duration := time.Since(taskInfo.startTime) s.True(duration >= delay) - s.Equal(task.ErrTaskRetry, err) + s.Equal(task.ErrTaskRedispatch, err) } func (s *taskProcessorSuite) TestHandleTaskError_ErrTaskDiscarded() { diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index 4e7e6fba94a..f76d85b89d7 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -104,6 +104,16 @@ func newTimerQueueProcessorBase( taskProcessor = newTaskProcessor(options, shard, historyService.executionCache, logger) } + queueType := task.QueueTypeActiveTimer + redispatcherOptions := &task.RedispatcherOptions{ + TaskRedispatchInterval: config.ActiveTaskRedispatchInterval, + TaskRedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, + } + if scope == metrics.TimerStandbyQueueProcessorScope { + queueType = task.QueueTypeStandbyTimer + redispatcherOptions.TaskRedispatchInterval = config.StandbyTaskRedispatchInterval + } + base := &timerQueueProcessorBase{ scope: scope, shard: shard, @@ -123,10 +133,7 @@ func newTimerQueueProcessorBase( queueTaskProcessor: queueTaskProcessor, redispatcher: task.NewRedispatcher( queueTaskProcessor, - &task.RedispatcherOptions{ - TaskRedispatchInterval: config.TaskRedispatchInterval, - TaskRedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, - }, + redispatcherOptions, logger, metricsScope, ), @@ -137,11 +144,6 @@ func newTimerQueueProcessorBase( ), } - queueType := task.QueueTypeActiveTimer - if scope == metrics.TimerStandbyQueueProcessorScope { - queueType = task.QueueTypeStandbyTimer - } - // read dynamic config only once on startup to avoid gc pressure caused by keeping reading dynamic config emitDomainTag := config.QueueProcessorEnableDomainTaggedMetrics() base.queueTaskInitializer = func(taskInfo task.Info) task.Task { diff --git a/service/history/transferQueueActiveProcessor.go b/service/history/transferQueueActiveProcessor.go index 50cea3df813..419181cfeec 100644 --- a/service/history/transferQueueActiveProcessor.go +++ b/service/history/transferQueueActiveProcessor.go @@ -74,7 +74,7 @@ func newTransferQueueActiveProcessor( UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.TransferTaskMaxRetryCount, - RedispatchInterval: config.TaskRedispatchInterval, + RedispatchInterval: config.ActiveTaskRedispatchInterval, RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.TransferProcessorEnablePriorityTaskProcessor, @@ -180,7 +180,7 @@ func newTransferQueueFailoverProcessor( UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.TransferTaskMaxRetryCount, - RedispatchInterval: config.TaskRedispatchInterval, + RedispatchInterval: config.ActiveTaskRedispatchInterval, RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.TransferProcessorEnablePriorityTaskProcessor, diff --git a/service/history/transferQueueStandbyProcessor.go b/service/history/transferQueueStandbyProcessor.go index 9f6256190d1..65442ffd5bc 100644 --- a/service/history/transferQueueStandbyProcessor.go +++ b/service/history/transferQueueStandbyProcessor.go @@ -72,7 +72,7 @@ func newTransferQueueStandbyProcessor( UpdateAckInterval: config.TransferProcessorUpdateAckInterval, UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, MaxRetryCount: config.TransferTaskMaxRetryCount, - RedispatchInterval: config.TaskRedispatchInterval, + RedispatchInterval: config.StandbyTaskRedispatchInterval, RedispatchIntervalJitterCoefficient: config.TaskRedispatchIntervalJitterCoefficient, MaxRedispatchQueueSize: config.TransferProcessorMaxRedispatchQueueSize, EnablePriorityTaskProcessor: config.TransferProcessorEnablePriorityTaskProcessor, diff --git a/service/worker/scanner/executions/workflows.go b/service/worker/scanner/executions/workflows.go index dbd586e71e0..6894982e932 100644 --- a/service/worker/scanner/executions/workflows.go +++ b/service/worker/scanner/executions/workflows.go @@ -197,6 +197,7 @@ var ( errQueryNotReady = errors.New("query is not yet ready to be handled, please try again shortly") ) +// Validate validates shard list or range func (s Shards) Validate() error { if s.List == nil && s.Range == nil { return errors.New("must provide either List or Range") @@ -213,6 +214,7 @@ func (s Shards) Validate() error { return nil } +// Flatten flattens Shards to a list of shard IDs and finds the min/max shardID func (s Shards) Flatten() ([]int, int, int) { shardList := s.List if len(shardList) == 0 { From 8904368e39c894dbd9dce73f7dad5746025aa90c Mon Sep 17 00:00:00 2001 From: Max K <25259015+mkolodezny@users.noreply.github.com> Date: Mon, 27 Jul 2020 10:40:43 -0700 Subject: [PATCH 3/6] [Scanner] Add concrete execution check for current execution (#3409) * Add current execution check --- common/metrics/defs.go | 3 + common/mocks/ExecutionManager.go | 23 ++++ .../cassandra/cassandraPersistence.go | 38 ++++++ common/persistence/dataInterfaces.go | 13 ++ common/persistence/executionStore.go | 6 + common/persistence/persistenceInterface.go | 1 + .../persistence/persistenceMetricClients.go | 14 +++ .../persistenceRateLimitedClients.go | 9 ++ common/persistence/sql/sqlExecutionManager.go | 6 + common/reconciliation/common/interfaces.go | 1 + .../common/persistenceRetryer.go | 17 +++ common/reconciliation/common/types.go | 8 ++ .../invariants/concreteExecutionExists.go | 117 ++++++++++++++++++ .../concreteExecutionExists_test.go | 104 ++++++++++++++++ .../invariants/historyExists.go | 2 +- .../invariants/historyExists_test.go | 2 +- .../invariants/openCurrentExecution.go | 2 +- .../invariants/openCurrentExecution_test.go | 16 +-- common/reconciliation/invariants/test_util.go | 51 ++++++-- 19 files changed, 410 insertions(+), 23 deletions(-) create mode 100644 common/reconciliation/invariants/concreteExecutionExists.go create mode 100644 common/reconciliation/invariants/concreteExecutionExists_test.go diff --git a/common/metrics/defs.go b/common/metrics/defs.go index 017503b9557..18f8ce78a8f 100644 --- a/common/metrics/defs.go +++ b/common/metrics/defs.go @@ -148,6 +148,8 @@ const ( PersistenceDeleteCurrentWorkflowExecutionScope // PersistenceGetCurrentExecutionScope tracks GetCurrentExecution calls made by service to persistence layer PersistenceGetCurrentExecutionScope + // PersistenceIsWorkflowExecutionExistsScope tracks IsWorkflowExecutionExists calls made by service to persistence layer + PersistenceIsWorkflowExecutionExistsScope // PersistenceListConcreteExecutionsScope tracks ListConcreteExecutions calls made by service to persistence layer PersistenceListConcreteExecutionsScope // PersistenceGetTransferTasksScope tracks GetTransferTasks calls made by service to persistence layer @@ -1069,6 +1071,7 @@ var ScopeDefs = map[ServiceIdx]map[int]scopeDefinition{ PersistenceDeleteWorkflowExecutionScope: {operation: "DeleteWorkflowExecution"}, PersistenceDeleteCurrentWorkflowExecutionScope: {operation: "DeleteCurrentWorkflowExecution"}, PersistenceGetCurrentExecutionScope: {operation: "GetCurrentExecution"}, + PersistenceIsWorkflowExecutionExistsScope: {operation: "IsWorkflowExecutionExists"}, PersistenceListConcreteExecutionsScope: {operation: "ListConcreteExecutions"}, PersistenceGetTransferTasksScope: {operation: "GetTransferTasks"}, PersistenceCompleteTransferTaskScope: {operation: "CompleteTransferTask"}, diff --git a/common/mocks/ExecutionManager.go b/common/mocks/ExecutionManager.go index 4056a243dbc..5fed40cd136 100644 --- a/common/mocks/ExecutionManager.go +++ b/common/mocks/ExecutionManager.go @@ -206,6 +206,29 @@ func (_m *ExecutionManager) GetCurrentExecution(request *persistence.GetCurrentE return r0, r1 } +// IsWorkflowExecutionExists provides a mock function with given fields: request +func (_m *ExecutionManager) IsWorkflowExecutionExists(request *persistence.IsWorkflowExecutionExistsRequest) (*persistence.IsWorkflowExecutionExistsResponse, error) { + ret := _m.Called(request) + + var r0 *persistence.IsWorkflowExecutionExistsResponse + if rf, ok := ret.Get(0).(func(*persistence.IsWorkflowExecutionExistsRequest) *persistence.IsWorkflowExecutionExistsResponse); ok { + r0 = rf(request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*persistence.IsWorkflowExecutionExistsResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(*persistence.IsWorkflowExecutionExistsRequest) error); ok { + r1 = rf(request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // ListConcreteExecutions provides a mock function with given fields: request func (_m *ExecutionManager) ListConcreteExecutions(request *persistence.ListConcreteExecutionsRequest) (*persistence.ListConcreteExecutionsResponse, error) { ret := _m.Called(request) diff --git a/common/persistence/cassandra/cassandraPersistence.go b/common/persistence/cassandra/cassandraPersistence.go index a2132294c44..8c9302a38b8 100644 --- a/common/persistence/cassandra/cassandraPersistence.go +++ b/common/persistence/cassandra/cassandraPersistence.go @@ -440,6 +440,16 @@ workflow_state = ? ` + `and visibility_ts = ? ` + `and task_id = ?` + templateIsWorkflowExecutionExistsQuery = `SELECT shard_id, type, domain_id, workflow_id, run_id, visibility_ts, task_id ` + + `FROM executions ` + + `WHERE shard_id = ? ` + + `and type = ? ` + + `and domain_id = ? ` + + `and workflow_id = ? ` + + `and run_id = ? ` + + `and visibility_ts = ? ` + + `and task_id = ?` + templateListWorkflowExecutionQuery = `SELECT run_id, execution, version_histories, version_histories_encoding ` + `FROM executions ` + `WHERE shard_id = ? ` + @@ -2053,6 +2063,34 @@ func (d *cassandraPersistence) GetCurrentExecution(request *p.GetCurrentExecutio }, nil } +func (d *cassandraPersistence) IsWorkflowExecutionExists(request *p.IsWorkflowExecutionExistsRequest) (*p.IsWorkflowExecutionExistsResponse, + error) { + query := d.session.Query(templateIsWorkflowExecutionExistsQuery, + d.shardID, + rowTypeExecution, + request.DomainID, + request.WorkflowID, + request.RunID, + defaultVisibilityTimestamp, + rowTypeExecutionTaskID) + + result := make(map[string]interface{}) + if err := query.MapScan(result); err != nil { + if err == gocql.ErrNotFound { + return &p.IsWorkflowExecutionExistsResponse{Exists: false}, nil + } else if isThrottlingError(err) { + return nil, &workflow.ServiceBusyError{ + Message: fmt.Sprintf("IsWorkflowExecutionExists operation failed. Error: %v", err), + } + } + + return nil, &workflow.InternalServiceError{ + Message: fmt.Sprintf("IsWorkflowExecutionExists operation failed. Error: %v", err), + } + } + return &p.IsWorkflowExecutionExistsResponse{Exists: true}, nil +} + func (d *cassandraPersistence) ListConcreteExecutions( request *p.ListConcreteExecutionsRequest, ) (*p.InternalListConcreteExecutionsResponse, error) { diff --git a/common/persistence/dataInterfaces.go b/common/persistence/dataInterfaces.go index d20c4808883..3b2772b3408 100644 --- a/common/persistence/dataInterfaces.go +++ b/common/persistence/dataInterfaces.go @@ -795,6 +795,13 @@ type ( WorkflowID string } + // IsWorkflowExecutionExistsRequest is used to check if the concrete execution exists + IsWorkflowExecutionExistsRequest struct { + DomainID string + WorkflowID string + RunID string + } + // ListConcreteExecutionsRequest is request to ListConcreteExecutions ListConcreteExecutionsRequest struct { PageSize int @@ -822,6 +829,11 @@ type ( LastWriteVersion int64 } + // IsWorkflowExecutionExistsResponse is the response to IsWorkflowExecutionExists + IsWorkflowExecutionExistsResponse struct { + Exists bool + } + // UpdateWorkflowExecutionRequest is used to update a workflow execution UpdateWorkflowExecutionRequest struct { RangeID int64 @@ -1521,6 +1533,7 @@ type ( DeleteWorkflowExecution(request *DeleteWorkflowExecutionRequest) error DeleteCurrentWorkflowExecution(request *DeleteCurrentWorkflowExecutionRequest) error GetCurrentExecution(request *GetCurrentExecutionRequest) (*GetCurrentExecutionResponse, error) + IsWorkflowExecutionExists(request *IsWorkflowExecutionExistsRequest) (*IsWorkflowExecutionExistsResponse, error) // Transfer task related methods GetTransferTasks(request *GetTransferTasksRequest) (*GetTransferTasksResponse, error) diff --git a/common/persistence/executionStore.go b/common/persistence/executionStore.go index 38044b14a09..30d3bd57a7a 100644 --- a/common/persistence/executionStore.go +++ b/common/persistence/executionStore.go @@ -778,6 +778,12 @@ func (m *executionManagerImpl) GetCurrentExecution( return m.persistence.GetCurrentExecution(request) } +func (m *executionManagerImpl) IsWorkflowExecutionExists( + request *IsWorkflowExecutionExistsRequest, +) (*IsWorkflowExecutionExistsResponse, error) { + return m.persistence.IsWorkflowExecutionExists(request) +} + func (m *executionManagerImpl) ListConcreteExecutions( request *ListConcreteExecutionsRequest, ) (*ListConcreteExecutionsResponse, error) { diff --git a/common/persistence/persistenceInterface.go b/common/persistence/persistenceInterface.go index 773dd71fd0a..2eabc4ec524 100644 --- a/common/persistence/persistenceInterface.go +++ b/common/persistence/persistenceInterface.go @@ -69,6 +69,7 @@ type ( DeleteWorkflowExecution(request *DeleteWorkflowExecutionRequest) error DeleteCurrentWorkflowExecution(request *DeleteCurrentWorkflowExecutionRequest) error GetCurrentExecution(request *GetCurrentExecutionRequest) (*GetCurrentExecutionResponse, error) + IsWorkflowExecutionExists(request *IsWorkflowExecutionExistsRequest) (*IsWorkflowExecutionExistsResponse, error) // Transfer task related methods GetTransferTasks(request *GetTransferTasksRequest) (*GetTransferTasksResponse, error) diff --git a/common/persistence/persistenceMetricClients.go b/common/persistence/persistenceMetricClients.go index a2018a33731..932e5984b14 100644 --- a/common/persistence/persistenceMetricClients.go +++ b/common/persistence/persistenceMetricClients.go @@ -330,6 +330,20 @@ func (p *workflowExecutionPersistenceClient) GetCurrentExecution(request *GetCur return response, err } +func (p *workflowExecutionPersistenceClient) IsWorkflowExecutionExists(request *IsWorkflowExecutionExistsRequest) (*IsWorkflowExecutionExistsResponse, error) { + p.metricClient.IncCounter(metrics.PersistenceIsWorkflowExecutionExistsScope, metrics.PersistenceRequests) + + sw := p.metricClient.StartTimer(metrics.PersistenceIsWorkflowExecutionExistsScope, metrics.PersistenceLatency) + response, err := p.persistence.IsWorkflowExecutionExists(request) + sw.Stop() + + if err != nil { + p.updateErrorMetric(metrics.PersistenceIsWorkflowExecutionExistsScope, err) + } + + return response, err +} + func (p *workflowExecutionPersistenceClient) ListConcreteExecutions(request *ListConcreteExecutionsRequest) (*ListConcreteExecutionsResponse, error) { p.metricClient.IncCounter(metrics.PersistenceListConcreteExecutionsScope, metrics.PersistenceRequests) diff --git a/common/persistence/persistenceRateLimitedClients.go b/common/persistence/persistenceRateLimitedClients.go index 73aba605212..85eafb3e13b 100644 --- a/common/persistence/persistenceRateLimitedClients.go +++ b/common/persistence/persistenceRateLimitedClients.go @@ -263,6 +263,15 @@ func (p *workflowExecutionRateLimitedPersistenceClient) GetCurrentExecution(requ return response, err } +func (p *workflowExecutionRateLimitedPersistenceClient) IsWorkflowExecutionExists(request *IsWorkflowExecutionExistsRequest) (*IsWorkflowExecutionExistsResponse, error) { + if ok := p.rateLimiter.Allow(); !ok { + return nil, ErrPersistenceLimitExceeded + } + + response, err := p.persistence.IsWorkflowExecutionExists(request) + return response, err +} + func (p *workflowExecutionRateLimitedPersistenceClient) ListConcreteExecutions(request *ListConcreteExecutionsRequest) (*ListConcreteExecutionsResponse, error) { if ok := p.rateLimiter.Allow(); !ok { return nil, ErrPersistenceLimitExceeded diff --git a/common/persistence/sql/sqlExecutionManager.go b/common/persistence/sql/sqlExecutionManager.go index 82d45795e80..0e8e332fa51 100644 --- a/common/persistence/sql/sqlExecutionManager.go +++ b/common/persistence/sql/sqlExecutionManager.go @@ -836,6 +836,12 @@ func (m *sqlExecutionManager) GetCurrentExecution( }, nil } +func (m *sqlExecutionManager) IsWorkflowExecutionExists( + request *p.IsWorkflowExecutionExistsRequest, +) (*p.IsWorkflowExecutionExistsResponse, error) { + panic("not implemented yet") +} + func (m *sqlExecutionManager) ListConcreteExecutions( _ *p.ListConcreteExecutionsRequest, ) (*p.InternalListConcreteExecutionsResponse, error) { diff --git a/common/reconciliation/common/interfaces.go b/common/reconciliation/common/interfaces.go index f8948cba199..d9fa4589409 100644 --- a/common/reconciliation/common/interfaces.go +++ b/common/reconciliation/common/interfaces.go @@ -30,6 +30,7 @@ type ( ListConcreteExecutions(*persistence.ListConcreteExecutionsRequest) (*persistence.ListConcreteExecutionsResponse, error) GetWorkflowExecution(*persistence.GetWorkflowExecutionRequest) (*persistence.GetWorkflowExecutionResponse, error) GetCurrentExecution(*persistence.GetCurrentExecutionRequest) (*persistence.GetCurrentExecutionResponse, error) + IsWorkflowExecutionExists(request *persistence.IsWorkflowExecutionExistsRequest) (*persistence.IsWorkflowExecutionExistsResponse, error) ReadHistoryBranch(*persistence.ReadHistoryBranchRequest) (*persistence.ReadHistoryBranchResponse, error) DeleteWorkflowExecution(*persistence.DeleteWorkflowExecutionRequest) error DeleteCurrentWorkflowExecution(request *persistence.DeleteCurrentWorkflowExecutionRequest) error diff --git a/common/reconciliation/common/persistenceRetryer.go b/common/reconciliation/common/persistenceRetryer.go index 9a351fc6ca3..6d844313708 100644 --- a/common/reconciliation/common/persistenceRetryer.go +++ b/common/reconciliation/common/persistenceRetryer.go @@ -102,6 +102,23 @@ func (pr *persistenceRetryer) GetCurrentExecution( return resp, nil } +// IsWorkflowExecutionExists retries IsWorkflowExecutionExists +func (pr *persistenceRetryer) IsWorkflowExecutionExists( + req *persistence.IsWorkflowExecutionExistsRequest, +) (*persistence.IsWorkflowExecutionExistsResponse, error) { + var resp *persistence.IsWorkflowExecutionExistsResponse + op := func() error { + var err error + resp, err = pr.execManager.IsWorkflowExecutionExists(req) + return err + } + err := backoff.Retry(op, retryPolicy, common.IsPersistenceTransientError) + if err != nil { + return nil, err + } + return resp, nil +} + // ReadHistoryBranch retries ReadHistoryBranch func (pr *persistenceRetryer) ReadHistoryBranch( req *persistence.ReadHistoryBranchRequest, diff --git a/common/reconciliation/common/types.go b/common/reconciliation/common/types.go index b638dea209a..2d13d491516 100644 --- a/common/reconciliation/common/types.go +++ b/common/reconciliation/common/types.go @@ -65,6 +65,8 @@ const ( HistoryExistsInvariantType InvariantType = "history_exists" // OpenCurrentExecutionInvariantType asserts that an open concrete execution must have a valid current execution OpenCurrentExecutionInvariantType InvariantType = "open_current_execution" + // ConcreteExecutionExistsInvariantType asserts that an open current execution must have a valid concrete execution + ConcreteExecutionExistsInvariantType InvariantType = "concrete_execution_exists" // InvariantCollectionMutableState is the collection of invariants relating to mutable state InvariantCollectionMutableState InvariantCollection = 0 @@ -100,6 +102,12 @@ type ( Execution } + // CurrentExecution is a current execution. + CurrentExecution struct { + CurrentRunID string + Execution + } + // CheckResult is the result of running Check. CheckResult struct { CheckResultType CheckResultType diff --git a/common/reconciliation/invariants/concreteExecutionExists.go b/common/reconciliation/invariants/concreteExecutionExists.go new file mode 100644 index 00000000000..f9b7cccbe10 --- /dev/null +++ b/common/reconciliation/invariants/concreteExecutionExists.go @@ -0,0 +1,117 @@ +// The MIT License (MIT) +// +// Copyright (c) 2017-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 invariants + +import ( + "fmt" + + "github.com/uber/cadence/common/persistence" + "github.com/uber/cadence/common/reconciliation/common" +) + +type ( + concreteExecutionExists struct { + pr common.PersistenceRetryer + } +) + +// NewConcreteExecutionExists returns a new invariant for checking concrete execution +func NewConcreteExecutionExists( + pr common.PersistenceRetryer, +) common.Invariant { + return &concreteExecutionExists{ + pr: pr, + } +} + +func (c *concreteExecutionExists) Check(execution interface{}) common.CheckResult { + currentExecution, ok := execution.(*common.CurrentExecution) + if !ok { + return common.CheckResult{ + CheckResultType: common.CheckResultTypeFailed, + InvariantType: c.InvariantType(), + Info: "failed to check: expected current execution", + } + } + if !common.Open(currentExecution.State) { + return common.CheckResult{ + CheckResultType: common.CheckResultTypeHealthy, + InvariantType: c.InvariantType(), + } + } + // by this point the corresponding concrete execution must exist and can be already closed + currentExecResp, currentExecErr := c.pr.IsWorkflowExecutionExists(&persistence.IsWorkflowExecutionExistsRequest{ + DomainID: currentExecution.DomainID, + WorkflowID: currentExecution.WorkflowID, + RunID: currentExecution.CurrentRunID, + }) + if currentExecErr != nil { + return common.CheckResult{ + CheckResultType: common.CheckResultTypeFailed, + InvariantType: c.InvariantType(), + Info: "failed to check if concrete execution exists", + InfoDetails: currentExecErr.Error(), + } + } + if !currentExecResp.Exists { + return common.CheckResult{ + CheckResultType: common.CheckResultTypeCorrupted, + InvariantType: c.InvariantType(), + Info: "execution is open without having concrete execution", + InfoDetails: fmt.Sprintf("concrete execution not found. WorkflowId: %v, RunId: %v", + currentExecution.WorkflowID, currentExecution.CurrentRunID), + } + } + return common.CheckResult{ + CheckResultType: common.CheckResultTypeHealthy, + InvariantType: c.InvariantType(), + } +} + +func (c *concreteExecutionExists) Fix(execution interface{}) common.FixResult { + fixResult, checkResult := checkBeforeFix(c, execution) + if fixResult != nil { + return *fixResult + } + currentExecution, _ := execution.(*common.CurrentExecution) + if err := c.pr.DeleteCurrentWorkflowExecution(&persistence.DeleteCurrentWorkflowExecutionRequest{ + DomainID: currentExecution.DomainID, + WorkflowID: currentExecution.WorkflowID, + RunID: currentExecution.CurrentRunID, + }); err != nil { + return common.FixResult{ + FixResultType: common.FixResultTypeFailed, + Info: "failed to delete current workflow execution", + InfoDetails: err.Error(), + } + } + return common.FixResult{ + FixResultType: common.FixResultTypeFixed, + CheckResult: *checkResult, + InvariantType: c.InvariantType(), + } +} + +func (c *concreteExecutionExists) InvariantType() common.InvariantType { + return common.ConcreteExecutionExistsInvariantType +} diff --git a/common/reconciliation/invariants/concreteExecutionExists_test.go b/common/reconciliation/invariants/concreteExecutionExists_test.go new file mode 100644 index 00000000000..d344a691a08 --- /dev/null +++ b/common/reconciliation/invariants/concreteExecutionExists_test.go @@ -0,0 +1,104 @@ +// The MIT License (MIT) +// +// Copyright (c) 2017-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 invariants + +import ( + "errors" + "fmt" + "testing" + + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/uber/cadence/common/mocks" + "github.com/uber/cadence/common/persistence" + "github.com/uber/cadence/common/reconciliation/common" +) + +type ConcreteExecutionExistsSuite struct { + *require.Assertions + suite.Suite +} + +func TestConcreteExecutionExistsSuite(t *testing.T) { + suite.Run(t, new(ConcreteExecutionExistsSuite)) +} + +func (s *ConcreteExecutionExistsSuite) SetupTest() { + s.Assertions = require.New(s.T()) +} + +func (s *ConcreteExecutionExistsSuite) TestCheck() { + testCases := []struct { + execution *common.CurrentExecution + getConcreteResp *persistence.IsWorkflowExecutionExistsResponse + getConcreteErr error + expectedResult common.CheckResult + }{ + { + execution: getClosedCurrentExecution(), + expectedResult: common.CheckResult{ + CheckResultType: common.CheckResultTypeHealthy, + InvariantType: common.ConcreteExecutionExistsInvariantType, + }, + }, + { + execution: getOpenCurrentExecution(), + getConcreteErr: errors.New("error getting concrete execution"), + expectedResult: common.CheckResult{ + CheckResultType: common.CheckResultTypeFailed, + InvariantType: common.ConcreteExecutionExistsInvariantType, + Info: "failed to check if concrete execution exists", + InfoDetails: "error getting concrete execution", + }, + }, + { + execution: getOpenCurrentExecution(), + getConcreteResp: &persistence.IsWorkflowExecutionExistsResponse{Exists: false}, + expectedResult: common.CheckResult{ + CheckResultType: common.CheckResultTypeCorrupted, + InvariantType: common.ConcreteExecutionExistsInvariantType, + Info: "execution is open without having concrete execution", + InfoDetails: fmt.Sprintf("concrete execution not found. WorkflowId: %v, RunId: %v", + workflowID, currentRunID), + }, + }, + { + execution: getOpenCurrentExecution(), + getConcreteErr: nil, + getConcreteResp: &persistence.IsWorkflowExecutionExistsResponse{Exists: true}, + expectedResult: common.CheckResult{ + CheckResultType: common.CheckResultTypeHealthy, + InvariantType: common.ConcreteExecutionExistsInvariantType, + }, + }, + } + + for _, tc := range testCases { + execManager := &mocks.ExecutionManager{} + execManager.On("IsWorkflowExecutionExists", mock.Anything).Return(tc.getConcreteResp, tc.getConcreteErr) + o := NewConcreteExecutionExists(common.NewPersistenceRetryer(execManager, nil)) + s.Equal(tc.expectedResult, o.Check(tc.execution)) + } +} diff --git a/common/reconciliation/invariants/historyExists.go b/common/reconciliation/invariants/historyExists.go index fe782ac43d4..0be85de4957 100644 --- a/common/reconciliation/invariants/historyExists.go +++ b/common/reconciliation/invariants/historyExists.go @@ -50,7 +50,7 @@ func NewHistoryExists( } func (h *historyExists) Check(execution interface{}) common.CheckResult { - concreteExecution, ok := execution.(common.ConcreteExecution) + concreteExecution, ok := execution.(*common.ConcreteExecution) if !ok { return common.CheckResult{ CheckResultType: common.CheckResultTypeFailed, diff --git a/common/reconciliation/invariants/historyExists_test.go b/common/reconciliation/invariants/historyExists_test.go index bbd827d5436..8fbcfea1f6e 100644 --- a/common/reconciliation/invariants/historyExists_test.go +++ b/common/reconciliation/invariants/historyExists_test.go @@ -134,7 +134,7 @@ func (s *HistoryExistsSuite) TestCheck() { execManager.On("GetWorkflowExecution", mock.Anything).Return(tc.getExecResp, tc.getExecErr) historyManager.On("ReadHistoryBranch", mock.Anything).Return(tc.getHistoryResp, tc.getHistoryErr) i := NewHistoryExists(common.NewPersistenceRetryer(execManager, historyManager)) - result := i.Check(getOpenExecution()) + result := i.Check(getOpenConcreteExecution()) s.Equal(tc.expectedResult, result) } } diff --git a/common/reconciliation/invariants/openCurrentExecution.go b/common/reconciliation/invariants/openCurrentExecution.go index 5cabb15f535..4a057a70933 100644 --- a/common/reconciliation/invariants/openCurrentExecution.go +++ b/common/reconciliation/invariants/openCurrentExecution.go @@ -46,7 +46,7 @@ func NewOpenCurrentExecution( } func (o *openCurrentExecution) Check(execution interface{}) common.CheckResult { - concreteExecution, ok := execution.(common.ConcreteExecution) + concreteExecution, ok := execution.(*common.ConcreteExecution) if !ok { return common.CheckResult{ CheckResultType: common.CheckResultTypeFailed, diff --git a/common/reconciliation/invariants/openCurrentExecution_test.go b/common/reconciliation/invariants/openCurrentExecution_test.go index 3d85f390865..8e45cd2d67d 100644 --- a/common/reconciliation/invariants/openCurrentExecution_test.go +++ b/common/reconciliation/invariants/openCurrentExecution_test.go @@ -51,7 +51,7 @@ func (s *OpenCurrentExecutionSuite) SetupTest() { func (s *OpenCurrentExecutionSuite) TestCheck() { testCases := []struct { - execution common.ConcreteExecution + execution *common.ConcreteExecution getCurrentResp *persistence.GetCurrentExecutionResponse getCurrentErr error getConcreteResp *persistence.GetWorkflowExecutionResponse @@ -59,14 +59,14 @@ func (s *OpenCurrentExecutionSuite) TestCheck() { expectedResult common.CheckResult }{ { - execution: getClosedExecution(), + execution: getClosedConcreteExecution(), expectedResult: common.CheckResult{ CheckResultType: common.CheckResultTypeHealthy, InvariantType: common.OpenCurrentExecutionInvariantType, }, }, { - execution: getOpenExecution(), + execution: getOpenConcreteExecution(), getConcreteErr: errors.New("got error checking if concrete is open"), expectedResult: common.CheckResult{ CheckResultType: common.CheckResultTypeFailed, @@ -76,7 +76,7 @@ func (s *OpenCurrentExecutionSuite) TestCheck() { }, }, { - execution: getOpenExecution(), + execution: getOpenConcreteExecution(), getConcreteResp: &persistence.GetWorkflowExecutionResponse{ State: &persistence.WorkflowMutableState{ ExecutionInfo: &persistence.WorkflowExecutionInfo{ @@ -91,7 +91,7 @@ func (s *OpenCurrentExecutionSuite) TestCheck() { }, }, { - execution: getOpenExecution(), + execution: getOpenConcreteExecution(), getConcreteResp: &persistence.GetWorkflowExecutionResponse{ State: &persistence.WorkflowMutableState{ ExecutionInfo: &persistence.WorkflowExecutionInfo{ @@ -109,7 +109,7 @@ func (s *OpenCurrentExecutionSuite) TestCheck() { }, }, { - execution: getOpenExecution(), + execution: getOpenConcreteExecution(), getConcreteResp: &persistence.GetWorkflowExecutionResponse{ State: &persistence.WorkflowMutableState{ ExecutionInfo: &persistence.WorkflowExecutionInfo{ @@ -127,7 +127,7 @@ func (s *OpenCurrentExecutionSuite) TestCheck() { }, }, { - execution: getOpenExecution(), + execution: getOpenConcreteExecution(), getConcreteResp: &persistence.GetWorkflowExecutionResponse{ State: &persistence.WorkflowMutableState{ ExecutionInfo: &persistence.WorkflowExecutionInfo{ @@ -148,7 +148,7 @@ func (s *OpenCurrentExecutionSuite) TestCheck() { }, }, { - execution: getOpenExecution(), + execution: getOpenConcreteExecution(), getConcreteResp: &persistence.GetWorkflowExecutionResponse{ State: &persistence.WorkflowMutableState{ ExecutionInfo: &persistence.WorkflowExecutionInfo{ diff --git a/common/reconciliation/invariants/test_util.go b/common/reconciliation/invariants/test_util.go index c9013bab956..edded71092c 100644 --- a/common/reconciliation/invariants/test_util.go +++ b/common/reconciliation/invariants/test_util.go @@ -28,22 +28,23 @@ import ( ) const ( - domainID = "test-domain-id" - workflowID = "test-workflow-id" - runID = "test-run-id" - shardID = 0 - treeID = "test-tree-id" - branchID = "test-branch-id" - openState = persistence.WorkflowStateCreated - closedState = persistence.WorkflowStateCompleted + domainID = "test-domain-id" + workflowID = "test-workflow-id" + runID = "test-run-id" + shardID = 0 + treeID = "test-tree-id" + branchID = "test-branch-id" + openState = persistence.WorkflowStateCreated + closedState = persistence.WorkflowStateCompleted + currentRunID = "test-current-run-id" ) var ( branchToken = []byte{1, 2, 3} ) -func getOpenExecution() common.ConcreteExecution { - return common.ConcreteExecution{ +func getOpenConcreteExecution() *common.ConcreteExecution { + return &common.ConcreteExecution{ Execution: common.Execution{ ShardID: shardID, DomainID: domainID, @@ -57,8 +58,8 @@ func getOpenExecution() common.ConcreteExecution { } } -func getClosedExecution() common.ConcreteExecution { - return common.ConcreteExecution{ +func getClosedConcreteExecution() *common.ConcreteExecution { + return &common.ConcreteExecution{ Execution: common.Execution{ ShardID: shardID, DomainID: domainID, @@ -71,3 +72,29 @@ func getClosedExecution() common.ConcreteExecution { BranchID: branchID, } } + +func getOpenCurrentExecution() *common.CurrentExecution { + return &common.CurrentExecution{ + Execution: common.Execution{ + ShardID: shardID, + DomainID: domainID, + WorkflowID: workflowID, + RunID: runID, + State: openState, + }, + CurrentRunID: currentRunID, + } +} + +func getClosedCurrentExecution() *common.CurrentExecution { + return &common.CurrentExecution{ + Execution: common.Execution{ + ShardID: shardID, + DomainID: domainID, + WorkflowID: workflowID, + RunID: runID, + State: closedState, + }, + CurrentRunID: currentRunID, + } +} From 2db63eaac7db74b0d08c091fd0c42709bab551fb Mon Sep 17 00:00:00 2001 From: Yichao Yang Date: Mon, 27 Jul 2020 11:19:07 -0700 Subject: [PATCH 4/6] Enforce time resolution for timerMaxReadLevel (#3411) * Truncate timerMaxReadLevel to millisecond * Add Stringer method for timerTaskKey --- service/history/queue/timer_queue_processor_base.go | 5 +++++ service/history/shard/context.go | 4 +++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/service/history/queue/timer_queue_processor_base.go b/service/history/queue/timer_queue_processor_base.go index 43e93a6b915..c79fee70408 100644 --- a/service/history/queue/timer_queue_processor_base.go +++ b/service/history/queue/timer_queue_processor_base.go @@ -22,6 +22,7 @@ package queue import ( "context" + "fmt" "math" "sync" "sync/atomic" @@ -612,6 +613,10 @@ func (k timerTaskKey) Less( return k.visibilityTimestamp.Before(timerKey.visibilityTimestamp) } +func (k timerTaskKey) String() string { + return fmt.Sprintf("{visibilityTimestamp: %v, taskID: %v}", k.visibilityTimestamp, k.taskID) +} + func newTimerQueueProcessorOptions( config *config.Config, isActive bool, diff --git a/service/history/shard/context.go b/service/history/shard/context.go index 5ed62799d76..4b3f4517466 100644 --- a/service/history/shard/context.go +++ b/service/history/shard/context.go @@ -453,7 +453,7 @@ func (s *contextImpl) UpdateTimerMaxReadLevel(cluster string) time.Time { currentTime = s.remoteClusterCurrentTime[cluster] } - s.timerMaxReadLevelMap[cluster] = currentTime.Add(s.config.TimerProcessorMaxTimeShift()) + s.timerMaxReadLevelMap[cluster] = currentTime.Add(s.config.TimerProcessorMaxTimeShift()).Truncate(time.Millisecond) return s.timerMaxReadLevelMap[cluster] } @@ -1441,6 +1441,8 @@ func acquireShard( } else { // active cluster timerMaxReadLevelMap[clusterName] = shardInfo.TimerAckLevel } + + timerMaxReadLevelMap[clusterName] = timerMaxReadLevelMap[clusterName].Truncate(time.Millisecond) } executionMgr, err := shardItem.GetExecutionManager(shardItem.shardID) From 713a7a9accc63a523e314051bf744ac2d183a658 Mon Sep 17 00:00:00 2001 From: Yu Xia Date: Mon, 27 Jul 2020 12:39:23 -0700 Subject: [PATCH 5/6] Fix failover marker update lock (#3412) --- service/history/failover/marker_notifier.go | 6 ++++-- service/history/shard/context.go | 1 + 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/service/history/failover/marker_notifier.go b/service/history/failover/marker_notifier.go index 5ae4717ff20..38f0c3bba27 100644 --- a/service/history/failover/marker_notifier.go +++ b/service/history/failover/marker_notifier.go @@ -97,11 +97,14 @@ func (m *markerNotifierImpl) Stop() { func (m *markerNotifierImpl) notifyPendingFailoverMarker() { + ticker := time.NewTicker(m.config.NotifyFailoverMarkerInterval()) + defer ticker.Stop() + for { select { case <-m.shutdownCh: return - default: + case <-ticker.C: markers, err := m.shard.ValidateAndUpdateFailoverMarkers() if err != nil { m.logger.Error("Failed to update pending failover markers in shard info.", tag.Error(err)) @@ -110,7 +113,6 @@ func (m *markerNotifierImpl) notifyPendingFailoverMarker() { if len(markers) > 0 { m.failoverCoordinator.NotifyFailoverMarkers(int32(m.shard.GetShardID()), markers) } - time.Sleep(m.config.NotifyFailoverMarkerInterval()) } } } diff --git a/service/history/shard/context.go b/service/history/shard/context.go index 4b3f4517466..aed5f80ddb1 100644 --- a/service/history/shard/context.go +++ b/service/history/shard/context.go @@ -1335,6 +1335,7 @@ func (s *contextImpl) ValidateAndUpdateFailoverMarkers() ([]*replicator.Failover } if len(completedFailoverMarkers) == 0 { + s.RUnlock() return s.pendingFailoverMarkers, nil } s.RUnlock() From 78031ab88745839aad85a9ce8cf6ced0d74c2805 Mon Sep 17 00:00:00 2001 From: Max K <25259015+mkolodezny@users.noreply.github.com> Date: Wed, 29 Jul 2020 11:53:07 -0700 Subject: [PATCH 6/6] [Scanner] Add persistence APIs for current execution (#3416) * [Scanner] Add persistence APIs for current executions --- common/metrics/defs.go | 3 ++ common/mocks/ExecutionManager.go | 23 +++++++++ .../cassandra/cassandraPersistence.go | 49 +++++++++++++++++++ common/persistence/dataInterfaces.go | 22 +++++++++ common/persistence/executionStore.go | 6 +++ common/persistence/persistenceInterface.go | 1 + .../persistence/persistenceMetricClients.go | 14 ++++++ .../persistenceRateLimitedClients.go | 9 ++++ common/persistence/sql/sqlExecutionManager.go | 6 +++ common/reconciliation/common/interfaces.go | 1 + .../common/persistenceIterator.go | 40 ++++++++++++++- .../common/persistenceRetryer.go | 18 +++++++ common/reconciliation/common/util.go | 12 +++++ 13 files changed, 203 insertions(+), 1 deletion(-) diff --git a/common/metrics/defs.go b/common/metrics/defs.go index 18f8ce78a8f..8fd7da07db5 100644 --- a/common/metrics/defs.go +++ b/common/metrics/defs.go @@ -150,6 +150,8 @@ const ( PersistenceGetCurrentExecutionScope // PersistenceIsWorkflowExecutionExistsScope tracks IsWorkflowExecutionExists calls made by service to persistence layer PersistenceIsWorkflowExecutionExistsScope + // PersistenceListCurrentExecutionsScope tracks ListCurrentExecutions calls made by service to persistence layer + PersistenceListCurrentExecutionsScope // PersistenceListConcreteExecutionsScope tracks ListConcreteExecutions calls made by service to persistence layer PersistenceListConcreteExecutionsScope // PersistenceGetTransferTasksScope tracks GetTransferTasks calls made by service to persistence layer @@ -1072,6 +1074,7 @@ var ScopeDefs = map[ServiceIdx]map[int]scopeDefinition{ PersistenceDeleteCurrentWorkflowExecutionScope: {operation: "DeleteCurrentWorkflowExecution"}, PersistenceGetCurrentExecutionScope: {operation: "GetCurrentExecution"}, PersistenceIsWorkflowExecutionExistsScope: {operation: "IsWorkflowExecutionExists"}, + PersistenceListCurrentExecutionsScope: {operation: "ListCurrentExecutions"}, PersistenceListConcreteExecutionsScope: {operation: "ListConcreteExecutions"}, PersistenceGetTransferTasksScope: {operation: "GetTransferTasks"}, PersistenceCompleteTransferTaskScope: {operation: "CompleteTransferTask"}, diff --git a/common/mocks/ExecutionManager.go b/common/mocks/ExecutionManager.go index 5fed40cd136..6692aef36ca 100644 --- a/common/mocks/ExecutionManager.go +++ b/common/mocks/ExecutionManager.go @@ -206,6 +206,29 @@ func (_m *ExecutionManager) GetCurrentExecution(request *persistence.GetCurrentE return r0, r1 } +// ListCurrentExecutions provides a mock function with given fields: request +func (_m *ExecutionManager) ListCurrentExecutions(request *persistence.ListCurrentExecutionsRequest) (*persistence.ListCurrentExecutionsResponse, error) { + ret := _m.Called(request) + + var r0 *persistence.ListCurrentExecutionsResponse + if rf, ok := ret.Get(0).(func(*persistence.ListCurrentExecutionsRequest) *persistence.ListCurrentExecutionsResponse); ok { + r0 = rf(request) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*persistence.ListCurrentExecutionsResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(*persistence.ListCurrentExecutionsRequest) error); ok { + r1 = rf(request) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // IsWorkflowExecutionExists provides a mock function with given fields: request func (_m *ExecutionManager) IsWorkflowExecutionExists(request *persistence.IsWorkflowExecutionExistsRequest) (*persistence.IsWorkflowExecutionExistsResponse, error) { ret := _m.Called(request) diff --git a/common/persistence/cassandra/cassandraPersistence.go b/common/persistence/cassandra/cassandraPersistence.go index 8c9302a38b8..1a6025299aa 100644 --- a/common/persistence/cassandra/cassandraPersistence.go +++ b/common/persistence/cassandra/cassandraPersistence.go @@ -440,6 +440,11 @@ workflow_state = ? ` + `and visibility_ts = ? ` + `and task_id = ?` + templateListCurrentExecutionsQuery = `SELECT domain_id, workflow_id, run_id, current_run_id, workflow_state ` + + `FROM executions ` + + `WHERE shard_id = ? ` + + `and type = ?` + templateIsWorkflowExecutionExistsQuery = `SELECT shard_id, type, domain_id, workflow_id, run_id, visibility_ts, task_id ` + `FROM executions ` + `WHERE shard_id = ? ` + @@ -2063,6 +2068,50 @@ func (d *cassandraPersistence) GetCurrentExecution(request *p.GetCurrentExecutio }, nil } +func (d *cassandraPersistence) ListCurrentExecutions( + request *p.ListCurrentExecutionsRequest, +) (*p.ListCurrentExecutionsResponse, error) { + query := d.session.Query( + templateListCurrentExecutionsQuery, + d.shardID, + rowTypeExecution, + ).PageSize(request.PageSize).PageState(request.PageToken) + + iter := query.Iter() + if iter == nil { + return nil, &workflow.InternalServiceError{ + Message: "ListCurrentExecutions operation failed. Not able to create query iterator.", + } + } + response := &p.ListCurrentExecutionsResponse{} + result := make(map[string]interface{}) + for iter.MapScan(result) { + runID := result["run_id"].(gocql.UUID).String() + if runID != permanentRunID { + result = make(map[string]interface{}) + continue + } + response.Executions = append(response.Executions, &p.CurrentWorkflowExecution{ + DomainID: result["domain_id"].(gocql.UUID).String(), + WorkflowID: result["workflow_id"].(string), + RunID: result["run_id"].(gocql.UUID).String(), + State: result["workflow_state"].(int), + CurrentRunID: result["current_run_id"].(gocql.UUID).String(), + }) + result = make(map[string]interface{}) + } + nextPageToken := iter.PageState() + response.PageToken = make([]byte, len(nextPageToken)) + copy(response.PageToken, nextPageToken) + + if err := iter.Close(); err != nil { + return nil, &workflow.InternalServiceError{ + Message: fmt.Sprintf("ListCurrentExecutions operation failed. Error: %v", err), + } + } + return response, nil +} + func (d *cassandraPersistence) IsWorkflowExecutionExists(request *p.IsWorkflowExecutionExistsRequest) (*p.IsWorkflowExecutionExistsResponse, error) { query := d.session.Query(templateIsWorkflowExecutionExistsQuery, diff --git a/common/persistence/dataInterfaces.go b/common/persistence/dataInterfaces.go index 3b2772b3408..b84348832c3 100644 --- a/common/persistence/dataInterfaces.go +++ b/common/persistence/dataInterfaces.go @@ -353,6 +353,15 @@ type ( LastReplicationInfo map[string]*ReplicationInfo } + // CurrentWorkflowExecution describes a current execution record + CurrentWorkflowExecution struct { + DomainID string + WorkflowID string + RunID string + State int + CurrentRunID string + } + // TransferTaskInfo describes a transfer task TransferTaskInfo struct { DomainID string @@ -795,6 +804,18 @@ type ( WorkflowID string } + // ListCurrentExecutionsRequest is request to ListCurrentExecutions + ListCurrentExecutionsRequest struct { + PageSize int + PageToken []byte + } + + // ListCurrentExecutionsResponse is the response to ListCurrentExecutionsRequest + ListCurrentExecutionsResponse struct { + Executions []*CurrentWorkflowExecution + PageToken []byte + } + // IsWorkflowExecutionExistsRequest is used to check if the concrete execution exists IsWorkflowExecutionExistsRequest struct { DomainID string @@ -1558,6 +1579,7 @@ type ( // Scan operations ListConcreteExecutions(request *ListConcreteExecutionsRequest) (*ListConcreteExecutionsResponse, error) + ListCurrentExecutions(request *ListCurrentExecutionsRequest) (*ListCurrentExecutionsResponse, error) } // ExecutionManagerFactory creates an instance of ExecutionManager for a given shard diff --git a/common/persistence/executionStore.go b/common/persistence/executionStore.go index 30d3bd57a7a..142cbd9b9d5 100644 --- a/common/persistence/executionStore.go +++ b/common/persistence/executionStore.go @@ -778,6 +778,12 @@ func (m *executionManagerImpl) GetCurrentExecution( return m.persistence.GetCurrentExecution(request) } +func (m *executionManagerImpl) ListCurrentExecutions( + request *ListCurrentExecutionsRequest, +) (*ListCurrentExecutionsResponse, error) { + return m.persistence.ListCurrentExecutions(request) +} + func (m *executionManagerImpl) IsWorkflowExecutionExists( request *IsWorkflowExecutionExistsRequest, ) (*IsWorkflowExecutionExistsResponse, error) { diff --git a/common/persistence/persistenceInterface.go b/common/persistence/persistenceInterface.go index 2eabc4ec524..2978dd72d4f 100644 --- a/common/persistence/persistenceInterface.go +++ b/common/persistence/persistenceInterface.go @@ -94,6 +94,7 @@ type ( // Scan related methods ListConcreteExecutions(request *ListConcreteExecutionsRequest) (*InternalListConcreteExecutionsResponse, error) + ListCurrentExecutions(request *ListCurrentExecutionsRequest) (*ListCurrentExecutionsResponse, error) } // HistoryStore is to manager workflow history events diff --git a/common/persistence/persistenceMetricClients.go b/common/persistence/persistenceMetricClients.go index 932e5984b14..ff379ee8aae 100644 --- a/common/persistence/persistenceMetricClients.go +++ b/common/persistence/persistenceMetricClients.go @@ -330,6 +330,20 @@ func (p *workflowExecutionPersistenceClient) GetCurrentExecution(request *GetCur return response, err } +func (p *workflowExecutionPersistenceClient) ListCurrentExecutions(request *ListCurrentExecutionsRequest) (*ListCurrentExecutionsResponse, error) { + p.metricClient.IncCounter(metrics.PersistenceListCurrentExecutionsScope, metrics.PersistenceRequests) + + sw := p.metricClient.StartTimer(metrics.PersistenceListCurrentExecutionsScope, metrics.PersistenceLatency) + response, err := p.persistence.ListCurrentExecutions(request) + sw.Stop() + + if err != nil { + p.updateErrorMetric(metrics.PersistenceListCurrentExecutionsScope, err) + } + + return response, err +} + func (p *workflowExecutionPersistenceClient) IsWorkflowExecutionExists(request *IsWorkflowExecutionExistsRequest) (*IsWorkflowExecutionExistsResponse, error) { p.metricClient.IncCounter(metrics.PersistenceIsWorkflowExecutionExistsScope, metrics.PersistenceRequests) diff --git a/common/persistence/persistenceRateLimitedClients.go b/common/persistence/persistenceRateLimitedClients.go index 85eafb3e13b..ca55755fef3 100644 --- a/common/persistence/persistenceRateLimitedClients.go +++ b/common/persistence/persistenceRateLimitedClients.go @@ -263,6 +263,15 @@ func (p *workflowExecutionRateLimitedPersistenceClient) GetCurrentExecution(requ return response, err } +func (p *workflowExecutionRateLimitedPersistenceClient) ListCurrentExecutions(request *ListCurrentExecutionsRequest) (*ListCurrentExecutionsResponse, error) { + if ok := p.rateLimiter.Allow(); !ok { + return nil, ErrPersistenceLimitExceeded + } + + response, err := p.persistence.ListCurrentExecutions(request) + return response, err +} + func (p *workflowExecutionRateLimitedPersistenceClient) IsWorkflowExecutionExists(request *IsWorkflowExecutionExistsRequest) (*IsWorkflowExecutionExistsResponse, error) { if ok := p.rateLimiter.Allow(); !ok { return nil, ErrPersistenceLimitExceeded diff --git a/common/persistence/sql/sqlExecutionManager.go b/common/persistence/sql/sqlExecutionManager.go index 0e8e332fa51..9ec008618b7 100644 --- a/common/persistence/sql/sqlExecutionManager.go +++ b/common/persistence/sql/sqlExecutionManager.go @@ -836,6 +836,12 @@ func (m *sqlExecutionManager) GetCurrentExecution( }, nil } +func (m *sqlExecutionManager) ListCurrentExecutions( + _ *p.ListCurrentExecutionsRequest, +) (*p.ListCurrentExecutionsResponse, error) { + return nil, &workflow.InternalServiceError{Message: "Not yet implemented"} +} + func (m *sqlExecutionManager) IsWorkflowExecutionExists( request *p.IsWorkflowExecutionExistsRequest, ) (*p.IsWorkflowExecutionExistsResponse, error) { diff --git a/common/reconciliation/common/interfaces.go b/common/reconciliation/common/interfaces.go index d9fa4589409..82fd27e8b66 100644 --- a/common/reconciliation/common/interfaces.go +++ b/common/reconciliation/common/interfaces.go @@ -28,6 +28,7 @@ type ( // PersistenceRetryer is used to retry requests to persistence PersistenceRetryer interface { ListConcreteExecutions(*persistence.ListConcreteExecutionsRequest) (*persistence.ListConcreteExecutionsResponse, error) + ListCurrentExecutions(request *persistence.ListCurrentExecutionsRequest) (*persistence.ListCurrentExecutionsResponse, error) GetWorkflowExecution(*persistence.GetWorkflowExecutionRequest) (*persistence.GetWorkflowExecutionResponse, error) GetCurrentExecution(*persistence.GetCurrentExecutionRequest) (*persistence.GetCurrentExecutionResponse, error) IsWorkflowExecutionExists(request *persistence.IsWorkflowExecutionExistsRequest) (*persistence.IsWorkflowExecutionExistsResponse, error) diff --git a/common/reconciliation/common/persistenceIterator.go b/common/reconciliation/common/persistenceIterator.go index 98b66cec2af..2d1c8fd7ae1 100644 --- a/common/reconciliation/common/persistenceIterator.go +++ b/common/reconciliation/common/persistenceIterator.go @@ -125,5 +125,43 @@ func getCurrentExecutionsPersistenceFetchPageFn( pageSize int, shardID int, ) pagination.FetchFn { - panic("not implemented yet") + return func(token pagination.PageToken) (pagination.Page, error) { + req := &persistence.ListCurrentExecutionsRequest{ + PageSize: pageSize, + } + if token != nil { + req.PageToken = token.([]byte) + } + resp, err := pr.ListCurrentExecutions(req) + if err != nil { + return pagination.Page{}, err + } + executions := make([]pagination.Entity, len(resp.Executions), len(resp.Executions)) + for i, e := range resp.Executions { + currentExec := &CurrentExecution{ + CurrentRunID: e.CurrentRunID, + Execution: Execution{ + ShardID: shardID, + DomainID: e.DomainID, + WorkflowID: e.WorkflowID, + RunID: e.RunID, + State: e.State, + }, + } + if err := ValidateCurrentExecution(currentExec); err != nil { + return pagination.Page{}, err + } + executions[i] = currentExec + } + var nextToken interface{} = resp.PageToken + if len(resp.PageToken) == 0 { + nextToken = nil + } + page := pagination.Page{ + CurrentToken: token, + NextToken: nextToken, + Entities: executions, + } + return page, nil + } } diff --git a/common/reconciliation/common/persistenceRetryer.go b/common/reconciliation/common/persistenceRetryer.go index 6d844313708..c8db8b25a1d 100644 --- a/common/reconciliation/common/persistenceRetryer.go +++ b/common/reconciliation/common/persistenceRetryer.go @@ -102,6 +102,24 @@ func (pr *persistenceRetryer) GetCurrentExecution( return resp, nil } +// ListCurrentExecutions retries ListCurrentExecutions +func (pr *persistenceRetryer) ListCurrentExecutions( + req *persistence.ListCurrentExecutionsRequest, +) (*persistence.ListCurrentExecutionsResponse, error) { + var resp *persistence.ListCurrentExecutionsResponse + op := func() error { + var err error + resp, err = pr.execManager.ListCurrentExecutions(req) + return err + } + var err error + err = backoff.Retry(op, retryPolicy, common.IsPersistenceTransientError) + if err == nil { + return resp, nil + } + return nil, err +} + // IsWorkflowExecutionExists retries IsWorkflowExecutionExists func (pr *persistenceRetryer) IsWorkflowExecutionExists( req *persistence.IsWorkflowExecutionExistsRequest, diff --git a/common/reconciliation/common/util.go b/common/reconciliation/common/util.go index 9edcce5c0b6..24ddf18fe05 100644 --- a/common/reconciliation/common/util.go +++ b/common/reconciliation/common/util.go @@ -69,6 +69,18 @@ func ValidateConcreteExecution(concreteExecution *ConcreteExecution) error { return nil } +// ValidateCurrentExecution returns an error if CurrentExecution is not valid, nil otherwise. +func ValidateCurrentExecution(currentExecution *CurrentExecution) error { + err := validateExecution(¤tExecution.Execution) + if err != nil { + return err + } + if len(currentExecution.CurrentRunID) == 0 { + return errors.New("empty CurrentRunID") + } + return nil +} + // GetBranchToken returns the branchToken, treeID and branchID or error on failure. func GetBranchToken( entity *persistence.ListConcreteExecutionsEntity,