From 5d7228d6c196499fa396bc8fe956a527a4c5487b Mon Sep 17 00:00:00 2001 From: Yichao Yang Date: Thu, 15 Dec 2022 12:09:19 -0800 Subject: [PATCH 1/2] Remove old queue processing logic --- common/dynamicconfig/constants.go | 27 - service/history/configs/config.go | 22 - service/history/queueAckMgr.go | 255 ----- service/history/queueAckMgr_test.go | 536 ---------- service/history/queueProcessorBase.go | 299 ------ service/history/timerQueueAckMgr.go | 401 -------- service/history/timerQueueAckMgr_test.go | 913 ------------------ service/history/timerQueueActiveProcessor.go | 355 ------- .../history/timerQueueActiveTaskExecutor.go | 15 +- .../timerQueueActiveTaskExecutor_test.go | 22 - service/history/timerQueueFactory.go | 210 ++-- service/history/timerQueueProcessor.go | 447 --------- service/history/timerQueueProcessorBase.go | 395 -------- service/history/timerQueueStandbyProcessor.go | 217 ----- service/history/timerQueueTaskExecutorBase.go | 2 + .../history/transferQueueActiveProcessor.go | 388 -------- service/history/transferQueueFactory.go | 181 ++-- service/history/transferQueueProcessor.go | 424 -------- service/history/transferQueueProcessorBase.go | 130 --- .../history/transferQueueStandbyProcessor.go | 220 ----- .../history/transferQueueTaskExecutorBase.go | 2 + service/history/visibilityQueueFactory.go | 117 +-- service/history/visibilityQueueProcessor.go | 375 ------- 23 files changed, 230 insertions(+), 5723 deletions(-) delete mode 100644 service/history/queueAckMgr.go delete mode 100644 service/history/queueAckMgr_test.go delete mode 100644 service/history/queueProcessorBase.go delete mode 100644 service/history/timerQueueAckMgr.go delete mode 100644 service/history/timerQueueAckMgr_test.go delete mode 100644 service/history/timerQueueActiveProcessor.go delete mode 100644 service/history/timerQueueProcessor.go delete mode 100644 service/history/timerQueueProcessorBase.go delete mode 100644 service/history/timerQueueStandbyProcessor.go delete mode 100644 service/history/transferQueueActiveProcessor.go delete mode 100644 service/history/transferQueueProcessor.go delete mode 100644 service/history/transferQueueProcessorBase.go delete mode 100644 service/history/transferQueueStandbyProcessor.go delete mode 100644 service/history/visibilityQueueProcessor.go diff --git a/common/dynamicconfig/constants.go b/common/dynamicconfig/constants.go index 3447d95881c..a3ad2eebda4 100644 --- a/common/dynamicconfig/constants.go +++ b/common/dynamicconfig/constants.go @@ -387,18 +387,8 @@ const ( // TimerTaskBatchSize is batch size for timer processor to process tasks TimerTaskBatchSize = "history.timerTaskBatchSize" - // TimerTaskWorkerCount is number of task workers for timer processor - TimerTaskWorkerCount = "history.timerTaskWorkerCount" // TimerTaskMaxRetryCount is max retry count for timer processor TimerTaskMaxRetryCount = "history.timerTaskMaxRetryCount" - // TimerProcessorEnableSingleProcessor indicates if only one timer processor should be started for both active and standby tasks - // NOTE: this is an experimental flag for guarantee task execution and will be deprecated after multicursor solution is ready - TimerProcessorEnableSingleProcessor = "history.timerProcessorEnableSingleProcessor" - // TimerProcessorEnableMultiCursor indicates if the new single processor multi-cursor timer processor implementation should be used - // NOTE: this flag only takes effect when history.timerProcessorEnablePriorityTaskScheduler is enabled. - TimerProcessorEnableMultiCursor = "history.timerProcessorEnableMultiCursor" - // TimerProcessorEnablePriorityTaskScheduler indicates whether host level priority task scheduler should be used for timer processor - TimerProcessorEnablePriorityTaskScheduler = "history.timerProcessorEnablePriorityTaskScheduler" // TimerProcessorSchedulerWorkerCount is the number of workers in the host level task scheduler for timer processor TimerProcessorSchedulerWorkerCount = "history.timerProcessorSchedulerWorkerCount" // TimerProcessorSchedulerActiveRoundRobinWeights is the priority round robin weights used by timer task scheduler for active namespaces @@ -442,18 +432,8 @@ const ( TransferProcessorMaxPollRPS = "history.transferProcessorMaxPollRPS" // TransferProcessorMaxPollHostRPS is max poll rate per second for all transferQueueProcessor on a host TransferProcessorMaxPollHostRPS = "history.transferProcessorMaxPollHostRPS" - // TransferTaskWorkerCount is number of worker for transferQueueProcessor - TransferTaskWorkerCount = "history.transferTaskWorkerCount" // TransferTaskMaxRetryCount is max times of retry for transferQueueProcessor TransferTaskMaxRetryCount = "history.transferTaskMaxRetryCount" - // TransferProcessorEnableSingleProcessor indicates if only one transfer processor should be started for both active and standby tasks - // NOTE: this is an experimental flag for guarantee task execution and will be deprecated after multicursor solution is ready - TransferProcessorEnableSingleProcessor = "history.transferProcessorEnableSingleProcessor" - // TransferProcessorEnableMultiCursor indicates if the new single processor multi-cursor transfer processor implementation should be used - // NOTE: this flag only takes effect when history.transferProcessorEnablePriorityTaskScheduler is enabled. - TransferProcessorEnableMultiCursor = "history.transferProcessorEnableMultiCursor" - // TransferProcessorEnablePriorityTaskScheduler indicates whether host level priority task scheduler should be used for transferQueueProcessor - TransferProcessorEnablePriorityTaskScheduler = "history.transferProcessorEnablePriorityTaskScheduler" // TransferProcessorSchedulerWorkerCount is the number of workers in the host level task scheduler for transferQueueProcessor TransferProcessorSchedulerWorkerCount = "history.transferProcessorSchedulerWorkerCount" // TransferProcessorSchedulerActiveRoundRobinWeights is the priority round robin weights used by transfer task scheduler for active namespaces @@ -487,15 +467,8 @@ const ( VisibilityProcessorMaxPollRPS = "history.visibilityProcessorMaxPollRPS" // VisibilityProcessorMaxPollHostRPS is max poll rate per second for all visibilityQueueProcessor on a host VisibilityProcessorMaxPollHostRPS = "history.visibilityProcessorMaxPollHostRPS" - // VisibilityTaskWorkerCount is number of worker for visibilityQueueProcessor - VisibilityTaskWorkerCount = "history.visibilityTaskWorkerCount" // VisibilityTaskMaxRetryCount is max times of retry for visibilityQueueProcessor VisibilityTaskMaxRetryCount = "history.visibilityTaskMaxRetryCount" - // VisibilityProcessorEnableMultiCursor indicates if the new single processor multi-cursor visibility processor implementation should be used - // NOTE: this flag only takes effect when history.visibilityProcessorEnablePriorityTaskScheduler is enabled. - VisibilityProcessorEnableMultiCursor = "history.visibilityProcessorEnableMultiCursor" - // VisibilityProcessorEnablePriorityTaskScheduler indicates whether host level priority task scheduler should be used for visibilityQueueProcessor - VisibilityProcessorEnablePriorityTaskScheduler = "history.visibilityProcessorEnablePriorityTaskScheduler" // VisibilityProcessorSchedulerWorkerCount is the number of workers in the host level task scheduler for visibilityQueueProcessor VisibilityProcessorSchedulerWorkerCount = "history.visibilityProcessorSchedulerWorkerCount" // VisibilityProcessorSchedulerActiveRoundRobinWeights is the priority round robin weights by visibility task scheduler for active namespaces diff --git a/service/history/configs/config.go b/service/history/configs/config.go index 4bb1224caef..f6cdcd78a1e 100644 --- a/service/history/configs/config.go +++ b/service/history/configs/config.go @@ -98,11 +98,7 @@ type Config struct { // TimerQueueProcessor settings TimerTaskHighPriorityRPS dynamicconfig.IntPropertyFnWithNamespaceFilter TimerTaskBatchSize dynamicconfig.IntPropertyFn - TimerTaskWorkerCount dynamicconfig.IntPropertyFn TimerTaskMaxRetryCount dynamicconfig.IntPropertyFn - TimerProcessorEnableSingleProcessor dynamicconfig.BoolPropertyFn - TimerProcessorEnableMultiCursor dynamicconfig.BoolPropertyFn - TimerProcessorEnablePriorityTaskScheduler dynamicconfig.BoolPropertyFn TimerProcessorSchedulerWorkerCount dynamicconfig.IntPropertyFn TimerProcessorSchedulerActiveRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter TimerProcessorSchedulerStandbyRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter @@ -124,11 +120,7 @@ type Config struct { // TransferQueueProcessor settings TransferTaskHighPriorityRPS dynamicconfig.IntPropertyFnWithNamespaceFilter TransferTaskBatchSize dynamicconfig.IntPropertyFn - TransferTaskWorkerCount dynamicconfig.IntPropertyFn TransferTaskMaxRetryCount dynamicconfig.IntPropertyFn - TransferProcessorEnableSingleProcessor dynamicconfig.BoolPropertyFn - TransferProcessorEnableMultiCursor dynamicconfig.BoolPropertyFn - TransferProcessorEnablePriorityTaskScheduler dynamicconfig.BoolPropertyFn TransferProcessorSchedulerWorkerCount dynamicconfig.IntPropertyFn TransferProcessorSchedulerActiveRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter TransferProcessorSchedulerStandbyRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter @@ -260,10 +252,7 @@ type Config struct { // VisibilityQueueProcessor settings VisibilityTaskHighPriorityRPS dynamicconfig.IntPropertyFnWithNamespaceFilter VisibilityTaskBatchSize dynamicconfig.IntPropertyFn - VisibilityTaskWorkerCount dynamicconfig.IntPropertyFn VisibilityTaskMaxRetryCount dynamicconfig.IntPropertyFn - VisibilityProcessorEnableMultiCursor dynamicconfig.BoolPropertyFn - VisibilityProcessorEnablePriorityTaskScheduler dynamicconfig.BoolPropertyFn VisibilityProcessorSchedulerWorkerCount dynamicconfig.IntPropertyFn VisibilityProcessorSchedulerActiveRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter VisibilityProcessorSchedulerStandbyRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter @@ -364,11 +353,7 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis TaskSchedulerNamespaceMaxQPS: dc.GetIntPropertyFilteredByNamespace(dynamicconfig.TaskSchedulerNamespaceMaxQPS, 0), TimerTaskBatchSize: dc.GetIntProperty(dynamicconfig.TimerTaskBatchSize, 100), - TimerTaskWorkerCount: dc.GetIntProperty(dynamicconfig.TimerTaskWorkerCount, 10), TimerTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.TimerTaskMaxRetryCount, 20), - TimerProcessorEnableSingleProcessor: dc.GetBoolProperty(dynamicconfig.TimerProcessorEnableSingleProcessor, false), - TimerProcessorEnableMultiCursor: dc.GetBoolProperty(dynamicconfig.TimerProcessorEnableMultiCursor, true), - TimerProcessorEnablePriorityTaskScheduler: dc.GetBoolProperty(dynamicconfig.TimerProcessorEnablePriorityTaskScheduler, true), TimerProcessorSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.TimerProcessorSchedulerWorkerCount, 512), TimerProcessorSchedulerActiveRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TimerProcessorSchedulerActiveRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultActiveTaskPriorityWeight)), TimerProcessorSchedulerStandbyRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TimerProcessorSchedulerStandbyRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultStandbyTaskPriorityWeight)), @@ -388,11 +373,7 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis RetentionTimerJitterDuration: dc.GetDurationProperty(dynamicconfig.RetentionTimerJitterDuration, 30*time.Minute), TransferTaskBatchSize: dc.GetIntProperty(dynamicconfig.TransferTaskBatchSize, 100), - TransferTaskWorkerCount: dc.GetIntProperty(dynamicconfig.TransferTaskWorkerCount, 10), TransferTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.TransferTaskMaxRetryCount, 20), - TransferProcessorEnableSingleProcessor: dc.GetBoolProperty(dynamicconfig.TransferProcessorEnableSingleProcessor, false), - TransferProcessorEnableMultiCursor: dc.GetBoolProperty(dynamicconfig.TransferProcessorEnableMultiCursor, true), - TransferProcessorEnablePriorityTaskScheduler: dc.GetBoolProperty(dynamicconfig.TransferProcessorEnablePriorityTaskScheduler, true), TransferProcessorSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.TransferProcessorSchedulerWorkerCount, 512), TransferProcessorSchedulerActiveRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TransferProcessorSchedulerActiveRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultActiveTaskPriorityWeight)), TransferProcessorSchedulerStandbyRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TransferProcessorSchedulerStandbyRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultStandbyTaskPriorityWeight)), @@ -497,10 +478,7 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis VisibilityTaskBatchSize: dc.GetIntProperty(dynamicconfig.VisibilityTaskBatchSize, 100), VisibilityProcessorMaxPollRPS: dc.GetIntProperty(dynamicconfig.VisibilityProcessorMaxPollRPS, 20), VisibilityProcessorMaxPollHostRPS: dc.GetIntProperty(dynamicconfig.VisibilityProcessorMaxPollHostRPS, 0), - VisibilityTaskWorkerCount: dc.GetIntProperty(dynamicconfig.VisibilityTaskWorkerCount, 10), VisibilityTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.VisibilityTaskMaxRetryCount, 20), - VisibilityProcessorEnableMultiCursor: dc.GetBoolProperty(dynamicconfig.VisibilityProcessorEnableMultiCursor, true), - VisibilityProcessorEnablePriorityTaskScheduler: dc.GetBoolProperty(dynamicconfig.VisibilityProcessorEnablePriorityTaskScheduler, true), VisibilityProcessorSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.VisibilityProcessorSchedulerWorkerCount, 512), VisibilityProcessorSchedulerActiveRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.VisibilityProcessorSchedulerActiveRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultActiveTaskPriorityWeight)), VisibilityProcessorSchedulerStandbyRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.VisibilityProcessorSchedulerStandbyRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultStandbyTaskPriorityWeight)), diff --git a/service/history/queueAckMgr.go b/service/history/queueAckMgr.go deleted file mode 100644 index e6424e97547..00000000000 --- a/service/history/queueAckMgr.go +++ /dev/null @@ -1,255 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "sync" - - "golang.org/x/exp/maps" - - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - ctasks "go.temporal.io/server/common/tasks" - "go.temporal.io/server/common/util" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" -) - -type ( - // queueAckMgr is created by QueueProcessor to keep track of the queue ackLevel for the shard. - // It keeps track of read level when dispatching tasks to processor and maintains a map of outstanding tasks. - // Outstanding tasks map uses the task id sequencer as the key, which is used by updateAckLevel to move the ack level - // for the shard when all preceding tasks are acknowledged. - queueAckMgrImpl struct { - isFailover bool - shard shard.Context - options *QueueProcessorOptions - processor processor - logger log.Logger - metricsHandler metrics.MetricsHandler - finishedChan chan struct{} - - sync.RWMutex - outstandingExecutables map[int64]queues.Executable - readLevel int64 - ackLevel int64 - isReadFinished bool - - executableInitializer taskExecutableInitializer - } -) - -const ( - warnPendingTasks = 2000 -) - -var _ queueAckMgr = (*queueAckMgrImpl)(nil) - -func newQueueAckMgr( - shard shard.Context, - options *QueueProcessorOptions, - processor processor, - ackLevel int64, - logger log.Logger, - executableInitializer taskExecutableInitializer, -) *queueAckMgrImpl { - - return &queueAckMgrImpl{ - isFailover: false, - shard: shard, - options: options, - processor: processor, - outstandingExecutables: make(map[int64]queues.Executable), - readLevel: ackLevel, - ackLevel: ackLevel, - logger: logger, - metricsHandler: shard.GetMetricsHandler(), - finishedChan: nil, - executableInitializer: executableInitializer, - } -} - -func newQueueFailoverAckMgr( - shard shard.Context, - options *QueueProcessorOptions, - processor processor, - ackLevel int64, - logger log.Logger, - executableInitializer taskExecutableInitializer, -) *queueAckMgrImpl { - - return &queueAckMgrImpl{ - isFailover: true, - shard: shard, - options: options, - processor: processor, - outstandingExecutables: make(map[int64]queues.Executable), - readLevel: ackLevel, - ackLevel: ackLevel, - logger: logger, - metricsHandler: shard.GetMetricsHandler(), - finishedChan: make(chan struct{}, 1), - executableInitializer: executableInitializer, - } -} - -func (a *queueAckMgrImpl) readQueueTasks() ([]queues.Executable, bool, error) { - a.RLock() - readLevel := a.readLevel - a.RUnlock() - - tasks, morePage, err := a.processor.readTasks(readLevel) - if err != nil { - return nil, false, err - } - - a.Lock() - defer a.Unlock() - if a.isFailover && !morePage { - a.isReadFinished = true - } - - filteredExecutables := make([]queues.Executable, 0, len(tasks)) - -TaskFilterLoop: - for _, task := range tasks { - _, isLoaded := a.outstandingExecutables[task.GetTaskID()] - if isLoaded { - // task already loaded - a.logger.Debug("Skipping transfer task", tag.Task(task)) - continue TaskFilterLoop - } - - if a.readLevel >= task.GetTaskID() { - a.logger.Fatal("Next task ID is less than current read level.", - tag.TaskID(task.GetTaskID()), - tag.ReadLevel(a.readLevel)) - } - a.logger.Debug("Moving read level", tag.TaskID(task.GetTaskID())) - a.readLevel = task.GetTaskID() - - taskExecutable := a.executableInitializer(task) - a.outstandingExecutables[task.GetTaskID()] = taskExecutable - filteredExecutables = append(filteredExecutables, taskExecutable) - } - - return filteredExecutables, morePage, nil -} - -func (a *queueAckMgrImpl) getQueueAckLevel() int64 { - a.Lock() - defer a.Unlock() - return a.ackLevel -} - -func (a *queueAckMgrImpl) getQueueReadLevel() int64 { - a.Lock() - defer a.Unlock() - return a.readLevel -} - -func (a *queueAckMgrImpl) getFinishedChan() <-chan struct{} { - return a.finishedChan -} - -func (a *queueAckMgrImpl) updateQueueAckLevel() error { - a.metricsHandler.Counter(metrics.AckLevelUpdateCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(a.options.Operation)) - - a.Lock() - ackLevel := a.ackLevel - - // task ID is not sequential, meaning there are a ton of missing chunks, - // so to optimize the performance, a sort is required - taskIDs := maps.Keys(a.outstandingExecutables) - util.SortSlice(taskIDs) - - pendingTasks := len(taskIDs) - if pendingTasks > warnPendingTasks { - a.logger.Warn("Too many pending tasks") - } - - metricsScope := a.metricsHandler.WithTags(metrics.OperationTag(metrics.ShardInfoScope)) - switch a.options.Operation { - case metrics.ReplicatorQueueProcessorScope: - metricsScope.Histogram( - metrics.ShardInfoReplicationPendingTasksTimer.GetMetricName(), - metrics.ShardInfoReplicationPendingTasksTimer.GetMetricUnit(), - ).Record(int64(pendingTasks)) - case metrics.TransferActiveQueueProcessorScope: - metricsScope.Histogram( - metrics.ShardInfoTransferActivePendingTasksTimer.GetMetricName(), - metrics.ShardInfoTransferActivePendingTasksTimer.GetMetricUnit(), - ).Record(int64(pendingTasks)) - case metrics.TransferStandbyQueueProcessorScope: - metricsScope.Histogram( - metrics.ShardInfoTransferStandbyPendingTasksTimer.GetMetricName(), - metrics.ShardInfoTransferStandbyPendingTasksTimer.GetMetricUnit(), - ).Record(int64(pendingTasks)) - case metrics.VisibilityQueueProcessorScope: - metricsScope.Histogram( - metrics.ShardInfoVisibilityPendingTasksTimer.GetMetricName(), - metrics.ShardInfoVisibilityPendingTasksTimer.GetMetricUnit(), - ).Record(int64(pendingTasks)) - } - -MoveAckLevelLoop: - for _, current := range taskIDs { - acked := a.outstandingExecutables[current].State() == ctasks.TaskStateAcked - if acked { - ackLevel = current - delete(a.outstandingExecutables, current) - a.logger.Debug("Moving timer ack level to", tag.AckLevel(ackLevel)) - } else { - break MoveAckLevelLoop - } - } - a.ackLevel = ackLevel - - if a.isFailover && a.isReadFinished && len(a.outstandingExecutables) == 0 { - a.Unlock() - // this means in failover mode, all possible failover transfer tasks - // are processed and we are free to shundown - a.logger.Debug("Queue ack manager shutdown.") - a.finishedChan <- struct{}{} - err := a.processor.queueShutdown() - if err != nil { - a.logger.Error("Error shutdown queue", tag.Error(err)) - } - return nil - } - - a.Unlock() - if err := a.processor.updateAckLevel(ackLevel); err != nil { - a.metricsHandler.Counter(metrics.AckLevelUpdateFailedCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(a.options.Operation)) - a.logger.Error("Error updating ack level for shard", tag.Error(err), tag.OperationFailed) - return err - } - return nil -} diff --git a/service/history/queueAckMgr_test.go b/service/history/queueAckMgr_test.go deleted file mode 100644 index a2314c4fc0c..00000000000 --- a/service/history/queueAckMgr_test.go +++ /dev/null @@ -1,536 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "testing" - "time" - - "github.com/golang/mock/gomock" - "github.com/pborman/uuid" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - - persistencespb "go.temporal.io/server/api/persistence/v1" - "go.temporal.io/server/common/cluster" - "go.temporal.io/server/common/definition" - "go.temporal.io/server/common/dynamicconfig" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/metrics" - p "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/primitives/timestamp" - ctasks "go.temporal.io/server/common/tasks" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/tests" -) - -type ( - queueAckMgrSuite struct { - suite.Suite - *require.Assertions - - controller *gomock.Controller - mockShard *shard.ContextTest - - mockProcessor *Mockprocessor - - logger log.Logger - queueAckMgr *queueAckMgrImpl - } - - queueFailoverAckMgrSuite struct { - suite.Suite - *require.Assertions - - controller *gomock.Controller - mockShard *shard.ContextTest - - mockProcessor *Mockprocessor - - logger log.Logger - queueFailoverAckMgr *queueAckMgrImpl - } -) - -func TestQueueAckMgrSuite(t *testing.T) { - s := new(queueAckMgrSuite) - suite.Run(t, s) -} - -func TestQueueFailoverAckMgrSuite(t *testing.T) { - s := new(queueFailoverAckMgrSuite) - suite.Run(t, s) -} - -func (s *queueAckMgrSuite) SetupSuite() { - -} - -func (s *queueAckMgrSuite) TearDownSuite() { - -} - -func (s *queueAckMgrSuite) SetupTest() { - s.Assertions = require.New(s.T()) - - config := tests.NewDynamicConfig() - config.ShardUpdateMinInterval = dynamicconfig.GetDurationPropertyFn(0 * time.Second) - - s.controller = gomock.NewController(s.T()) - s.mockShard = shard.NewTestContext( - s.controller, - &p.ShardInfoWithFailover{ - ShardInfo: &persistencespb.ShardInfo{ - ShardId: 1, - RangeId: 1, - QueueAckLevels: map[int32]*persistencespb.QueueAckLevel{ - tasks.CategoryTimer.ID(): { - ClusterAckLevel: map[string]int64{ - cluster.TestCurrentClusterName: timestamp.TimeNowPtrUtcAddSeconds(-8).UnixNano(), - cluster.TestAlternativeClusterName: timestamp.TimeNowPtrUtcAddSeconds(-10).UnixNano(), - }, - }, - }, - }, - }, - config, - ) - - s.mockProcessor = NewMockprocessor(s.controller) - - s.logger = s.mockShard.GetLogger() - - s.queueAckMgr = newQueueAckMgr( - s.mockShard, - &QueueProcessorOptions{ - Operation: metrics.TransferActiveQueueProcessorScope, - }, - s.mockProcessor, - 0, - s.logger, - func(task tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - task, - nil, - nil, - nil, - nil, - queues.NewNoopPriorityAssigner(), - s.mockShard.GetTimeSource(), - nil, - nil, - metrics.NoopMetricsHandler, - nil, - nil, - ) - }, - ) -} - -func (s *queueAckMgrSuite) TearDownTest() { - s.controller.Finish() - s.mockShard.StopForTest() -} - -func (s *queueAckMgrSuite) TestReadTimerTasks() { - readLevel := s.queueAckMgr.readLevel - // when the ack manager is first initialized, read == ack level - s.Equal(s.queueAckMgr.getQueueAckLevel(), readLevel) - - moreInput := false - taskID1 := int64(59) - tasksInput := []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID1, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - } - - s.mockProcessor.EXPECT().readTasks(readLevel).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err := s.queueAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput := make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueAckMgr.outstandingExecutables, 1) - - moreInput = true - taskID2 := int64(60) - tasksInput = []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID2, - TaskQueue: "some random task queue", - ScheduledEventID: 29, - }, - } - - s.mockProcessor.EXPECT().readTasks(taskID1).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err = s.queueAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput = make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueAckMgr.outstandingExecutables, 2) -} - -func (s *queueAckMgrSuite) TestReadCompleteTimerTasks() { - readLevel := s.queueAckMgr.readLevel - // when the ack manager is first initialized, read == ack level - s.Equal(s.queueAckMgr.getQueueAckLevel(), readLevel) - - moreInput := false - taskID := int64(59) - tasksInput := []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - } - - s.mockProcessor.EXPECT().readTasks(readLevel).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err := s.queueAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput := make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueAckMgr.outstandingExecutables, 1) - - taskExecutables[0].Ack() - s.Equal(ctasks.TaskStateAcked, s.queueAckMgr.outstandingExecutables[taskID].State()) -} - -func (s *queueAckMgrSuite) TestReadCompleteUpdateTimerTasks() { - readLevel := s.queueAckMgr.readLevel - // when the ack manager is first initialized, read == ack level - s.Equal(s.queueAckMgr.getQueueAckLevel(), readLevel) - - moreInput := true - taskID1 := int64(59) - taskID2 := int64(60) - taskID3 := int64(61) - tasksInput := []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID1, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID2, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID3, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - } - - s.mockProcessor.EXPECT().readTasks(readLevel).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err := s.queueAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput := make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueAckMgr.outstandingExecutables, 3) - - s.mockProcessor.EXPECT().updateAckLevel(taskID1).Return(nil) - taskExecutables[0].Ack() - s.queueAckMgr.updateQueueAckLevel() - s.Equal(taskID1, s.queueAckMgr.getQueueAckLevel()) - - s.mockProcessor.EXPECT().updateAckLevel(taskID1).Return(nil) - taskExecutables[2].Ack() - s.queueAckMgr.updateQueueAckLevel() - s.Equal(taskID1, s.queueAckMgr.getQueueAckLevel()) - - s.mockProcessor.EXPECT().updateAckLevel(taskID3).Return(nil) - taskExecutables[1].Ack() - s.queueAckMgr.updateQueueAckLevel() - s.Equal(taskID3, s.queueAckMgr.getQueueAckLevel()) -} - -// Tests for failover ack manager -func (s *queueFailoverAckMgrSuite) SetupSuite() { - -} - -func (s *queueFailoverAckMgrSuite) TearDownSuite() { - -} - -func (s *queueFailoverAckMgrSuite) SetupTest() { - s.Assertions = require.New(s.T()) - - config := tests.NewDynamicConfig() - config.ShardUpdateMinInterval = dynamicconfig.GetDurationPropertyFn(0 * time.Second) - - s.controller = gomock.NewController(s.T()) - s.mockShard = shard.NewTestContext( - s.controller, - &p.ShardInfoWithFailover{ - ShardInfo: &persistencespb.ShardInfo{ - ShardId: 1, - RangeId: 1, - QueueAckLevels: map[int32]*persistencespb.QueueAckLevel{ - tasks.CategoryTimer.ID(): { - ClusterAckLevel: map[string]int64{ - cluster.TestCurrentClusterName: timestamp.TimeNowPtrUtc().UnixNano(), - cluster.TestAlternativeClusterName: timestamp.TimeNowPtrUtcAddSeconds(-10).UnixNano(), - }, - }, - }, - }, - }, - config, - ) - - s.mockProcessor = NewMockprocessor(s.controller) - - s.logger = s.mockShard.GetLogger() - - s.queueFailoverAckMgr = newQueueFailoverAckMgr( - s.mockShard, - &QueueProcessorOptions{ - Operation: metrics.TransferQueueProcessorScope, - }, - s.mockProcessor, - 0, - s.logger, - func(task tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - task, - nil, - nil, - nil, - nil, - queues.NewNoopPriorityAssigner(), - s.mockShard.GetTimeSource(), - nil, - nil, - metrics.NoopMetricsHandler, - nil, - nil, - ) - }, - ) -} - -func (s *queueFailoverAckMgrSuite) TearDownTest() { - s.controller.Finish() - s.mockShard.StopForTest() -} - -func (s *queueFailoverAckMgrSuite) TestReadQueueTasks() { - readLevel := s.queueFailoverAckMgr.readLevel - // when the ack manager is first initialized, read == ack level - s.Equal(s.queueFailoverAckMgr.getQueueAckLevel(), readLevel) - - moreInput := true - taskID1 := int64(59) - tasksInput := []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID1, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - } - - s.mockProcessor.EXPECT().readTasks(readLevel).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err := s.queueFailoverAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput := make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueFailoverAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueFailoverAckMgr.outstandingExecutables, 1) - s.False(s.queueFailoverAckMgr.isReadFinished) - - moreInput = false - taskID2 := int64(60) - tasksInput = []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID2, - TaskQueue: "some random task queue", - ScheduledEventID: 29, - }, - } - - s.mockProcessor.EXPECT().readTasks(taskID1).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err = s.queueFailoverAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput = make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueFailoverAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueFailoverAckMgr.outstandingExecutables, 2) - s.True(s.queueFailoverAckMgr.isReadFinished) -} - -func (s *queueFailoverAckMgrSuite) TestReadCompleteQueueTasks() { - readLevel := s.queueFailoverAckMgr.readLevel - // when the ack manager is first initialized, read == ack level - s.Equal(s.queueFailoverAckMgr.getQueueAckLevel(), readLevel) - - moreInput := false - taskID1 := int64(59) - taskID2 := int64(60) - tasksInput := []tasks.Task{ - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID1, - TaskQueue: "some random task queue", - ScheduledEventID: 28, - }, - &tasks.WorkflowTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - TaskID: taskID2, - TaskQueue: "some random task queue", - ScheduledEventID: 29, - }, - } - - s.mockProcessor.EXPECT().readTasks(readLevel).Return(tasksInput, moreInput, nil) - - taskExecutables, moreOutput, err := s.queueFailoverAckMgr.readQueueTasks() - s.Nil(err) - - tasksOutput := make([]tasks.Task, 0, len(taskExecutables)) - for _, executable := range taskExecutables { - tasksOutput = append(tasksOutput, executable.GetTask()) - s.Equal(ctasks.TaskStatePending, s.queueFailoverAckMgr.outstandingExecutables[executable.GetTaskID()].State()) - } - s.Equal(tasksOutput, tasksInput) - s.Equal(moreOutput, moreInput) - s.Len(s.queueFailoverAckMgr.outstandingExecutables, 2) - - taskExecutables[1].Ack() - s.Equal(ctasks.TaskStateAcked, s.queueFailoverAckMgr.outstandingExecutables[taskID2].State()) - s.mockProcessor.EXPECT().updateAckLevel(s.queueFailoverAckMgr.getQueueAckLevel()).Return(nil) - s.queueFailoverAckMgr.updateQueueAckLevel() - select { - case <-s.queueFailoverAckMgr.getFinishedChan(): - s.Fail("finished channel should not fire") - default: - } - - taskExecutables[0].Ack() - s.Equal(ctasks.TaskStateAcked, s.queueFailoverAckMgr.outstandingExecutables[taskID1].State()) - s.mockProcessor.EXPECT().queueShutdown().Return(nil) - s.queueFailoverAckMgr.updateQueueAckLevel() - select { - case <-s.queueFailoverAckMgr.getFinishedChan(): - default: - s.Fail("finished channel should fire") - } -} diff --git a/service/history/queueProcessorBase.go b/service/history/queueProcessorBase.go deleted file mode 100644 index 88b2605d8d3..00000000000 --- a/service/history/queueProcessorBase.go +++ /dev/null @@ -1,299 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - "sync" - "sync/atomic" - "time" - - "go.temporal.io/server/common" - "go.temporal.io/server/common/backoff" - "go.temporal.io/server/common/clock" - "go.temporal.io/server/common/dynamicconfig" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - wcache "go.temporal.io/server/service/history/workflow/cache" -) - -type ( - // QueueProcessorOptions is options passed to queue processor implementation - QueueProcessorOptions struct { - BatchSize dynamicconfig.IntPropertyFn - MaxPollInterval dynamicconfig.DurationPropertyFn - MaxPollIntervalJitterCoefficient dynamicconfig.FloatPropertyFn - UpdateAckInterval dynamicconfig.DurationPropertyFn - UpdateAckIntervalJitterCoefficient dynamicconfig.FloatPropertyFn - MaxReschdulerSize dynamicconfig.IntPropertyFn - PollBackoffInterval dynamicconfig.DurationPropertyFn - Operation string - } - - queueProcessorBase struct { - clusterName string - shard shard.Context - timeSource clock.TimeSource - options *QueueProcessorOptions - queueProcessor common.Daemon - logger log.Logger - rateLimiter quotas.RateLimiter // Read rate limiter - ackMgr queueAckMgr - scheduler queues.Scheduler - rescheduler queues.Rescheduler - - lastPollTime time.Time - backoffTimerLock sync.Mutex - backoffTimer *time.Timer - readTaskRetrier backoff.Retrier - - notifyCh chan struct{} - status int32 - shutdownWG sync.WaitGroup - shutdownCh chan struct{} - } -) - -var ( - loadQueueTaskThrottleRetryDelay = 3 * time.Second -) - -func newQueueProcessorBase( - clusterName string, - shard shard.Context, - options *QueueProcessorOptions, - queueProcessor common.Daemon, - queueAckMgr queueAckMgr, - wcache wcache.Cache, - scheduler queues.Scheduler, - rescheduler queues.Rescheduler, - rateLimiter quotas.RateLimiter, - logger log.Logger, -) *queueProcessorBase { - - p := &queueProcessorBase{ - clusterName: clusterName, - shard: shard, - timeSource: shard.GetTimeSource(), - options: options, - queueProcessor: queueProcessor, - rateLimiter: rateLimiter, - status: common.DaemonStatusInitialized, - notifyCh: make(chan struct{}, 1), - shutdownCh: make(chan struct{}), - logger: logger, - ackMgr: queueAckMgr, - lastPollTime: time.Time{}, - readTaskRetrier: backoff.NewRetrier( - common.CreateReadTaskRetryPolicy(), - backoff.SystemClock, - ), - scheduler: scheduler, - rescheduler: rescheduler, - } - - return p -} - -func (p *queueProcessorBase) Start() { - if !atomic.CompareAndSwapInt32(&p.status, common.DaemonStatusInitialized, common.DaemonStatusStarted) { - return - } - - p.logger.Info("", tag.LifeCycleStarting, tag.ComponentTransferQueue) - defer p.logger.Info("", tag.LifeCycleStarted, tag.ComponentTransferQueue) - - p.rescheduler.Start() - - p.shutdownWG.Add(1) - p.notifyNewTask() - go p.processorPump() -} - -func (p *queueProcessorBase) Stop() { - if !atomic.CompareAndSwapInt32(&p.status, common.DaemonStatusStarted, common.DaemonStatusStopped) { - return - } - - p.logger.Info("", tag.LifeCycleStopping, tag.ComponentTransferQueue) - defer p.logger.Info("", tag.LifeCycleStopped, tag.ComponentTransferQueue) - - p.rescheduler.Stop() - - close(p.shutdownCh) - - if success := common.AwaitWaitGroup(&p.shutdownWG, time.Minute); !success { - p.logger.Warn("", tag.LifeCycleStopTimedout, tag.ComponentTransferQueue) - } -} - -func (p *queueProcessorBase) notifyNewTask() { - var event struct{} - select { - case p.notifyCh <- event: - default: // channel already has an event, don't block - } -} - -func (p *queueProcessorBase) processorPump() { - defer p.shutdownWG.Done() - - pollTimer := time.NewTimer(backoff.JitDuration( - p.options.MaxPollInterval(), - p.options.MaxPollIntervalJitterCoefficient(), - )) - defer pollTimer.Stop() - - updateAckTimer := time.NewTimer(backoff.JitDuration( - p.options.UpdateAckInterval(), - p.options.UpdateAckIntervalJitterCoefficient(), - )) - defer updateAckTimer.Stop() - -eventLoop: - for { - // prioritize shutdown - select { - case <-p.shutdownCh: - break eventLoop - default: - // noop - } - - select { - case <-p.shutdownCh: - break eventLoop - case <-p.ackMgr.getFinishedChan(): - // use a separate gorouting since the caller hold the shutdownWG - // stop the entire queue processor, not just processor base. - go p.queueProcessor.Stop() - case <-p.notifyCh: - p.processBatch() - case <-pollTimer.C: - pollTimer.Reset(backoff.JitDuration( - p.options.MaxPollInterval(), - p.options.MaxPollIntervalJitterCoefficient(), - )) - if p.lastPollTime.Add(p.options.MaxPollInterval()).Before(p.timeSource.Now()) { - p.processBatch() - } - case <-updateAckTimer.C: - updateAckTimer.Reset(backoff.JitDuration( - p.options.UpdateAckInterval(), - p.options.UpdateAckIntervalJitterCoefficient(), - )) - if err := p.ackMgr.updateQueueAckLevel(); shard.IsShardOwnershipLostError(err) { - // shard is no longer owned by this instance, bail out - // stop the entire queue processor, not just processor base. - go p.queueProcessor.Stop() - break eventLoop - } - } - } - - p.logger.Info("Queue processor pump shut down.") -} - -func (p *queueProcessorBase) processBatch() { - ctx, cancel := context.WithTimeout(context.Background(), loadQueueTaskThrottleRetryDelay) - if err := p.rateLimiter.Wait(ctx); err != nil { - deadline, _ := ctx.Deadline() - p.throttle(deadline.Sub(p.timeSource.Now())) - cancel() - return - } - cancel() - - if !p.verifyReschedulerSize() { - return - } - - p.lastPollTime = p.timeSource.Now() - tasks, more, err := p.ackMgr.readQueueTasks() - if err != nil { - p.logger.Error("Processor unable to retrieve tasks", tag.Error(err)) - if common.IsResourceExhausted(err) { - p.throttle(loadQueueTaskThrottleRetryDelay) - } else { - p.throttle(p.readTaskRetrier.NextBackOff()) - } - return - } - p.readTaskRetrier.Reset() - - if len(tasks) == 0 { - return - } - - for _, task := range tasks { - p.submitTask(task) - select { - case <-p.shutdownCh: - return - default: - } - } - - if more { - // There might be more task - // We return now to yield, but enqueue an event to poll later - p.notifyNewTask() - } -} - -func (p *queueProcessorBase) verifyReschedulerSize() bool { - passed := p.rescheduler.Len() < p.options.MaxReschdulerSize() - if !passed { - p.throttle(p.options.PollBackoffInterval()) - } - return passed -} - -func (p *queueProcessorBase) throttle(duration time.Duration) { - p.backoffTimerLock.Lock() - defer p.backoffTimerLock.Unlock() - - if p.backoffTimer == nil { - p.backoffTimer = time.AfterFunc(duration, func() { - p.backoffTimerLock.Lock() - defer p.backoffTimerLock.Unlock() - - p.notifyNewTask() // re-enqueue the event - p.backoffTimer = nil - }) - } -} - -func (p *queueProcessorBase) submitTask( - executable queues.Executable, -) { - executable.SetScheduledTime(p.timeSource.Now()) - if !p.scheduler.TrySubmit(executable) { - executable.Reschedule() - } -} diff --git a/service/history/timerQueueAckMgr.go b/service/history/timerQueueAckMgr.go deleted file mode 100644 index 713e31d964a..00000000000 --- a/service/history/timerQueueAckMgr.go +++ /dev/null @@ -1,401 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "math" - "sort" - "sync" - "time" - - "golang.org/x/exp/maps" - - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/primitives/timestamp" - ctasks "go.temporal.io/server/common/tasks" - "go.temporal.io/server/common/util" - "go.temporal.io/server/service/history/configs" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" -) - -var ( - maximumTime = time.Unix(0, math.MaxInt64).UTC() -) - -type ( - timerQueueAckMgrImpl struct { - operation string - isFailover bool - shard shard.Context - executionMgr persistence.ExecutionManager - logger log.Logger - metricHandler metrics.MetricsHandler - config *configs.Config - timeNow timeNow - updateTimerAckLevel updateTimerAckLevel - timerQueueShutdown timerQueueShutdown - singleProcessorMode bool - // isReadFinished indicate timer queue ack manager - // have no more task to send out - isReadFinished bool - // finishedChan will send out signal when timer - // queue ack manager have no more task to send out and all - // tasks sent are finished - finishedChan chan struct{} - - sync.Mutex - // outstanding timer task key -> time task executable - outstandingExecutables map[tasks.Key]queues.Executable - // timer task ack level - ackLevel tasks.Key - // timer task read level, used by failover - readLevel tasks.Key - // mutable timer level - minQueryLevel time.Time - maxQueryLevel time.Time - pageToken []byte - - clusterName string - - executableInitializer taskExecutableInitializer - } - // for each cluster, the ack level is the point in time when - // all timers before the ack level are processed. - // for each cluster, the read level is the point in time when - // all timers from ack level to read level are loaded in memory. - - // TODO this processing logic potentially has bug, refer to #605, #608 -) - -var _ timerQueueAckMgr = (*timerQueueAckMgrImpl)(nil) - -func newTimerQueueAckMgr( - operation string, - shard shard.Context, - minLevel time.Time, - timeNow timeNow, - updateTimerAckLevel updateTimerAckLevel, - logger log.Logger, - clusterName string, - executableInitializer taskExecutableInitializer, - singleProcessorMode bool, -) *timerQueueAckMgrImpl { - ackLevel := tasks.NewKey(minLevel, 0) - - timerQueueAckMgrImpl := &timerQueueAckMgrImpl{ - operation: operation, - isFailover: false, - shard: shard, - executionMgr: shard.GetExecutionManager(), - metricHandler: shard.GetMetricsHandler(), - logger: logger, - config: shard.GetConfig(), - timeNow: timeNow, - updateTimerAckLevel: updateTimerAckLevel, - timerQueueShutdown: func() error { return nil }, - singleProcessorMode: singleProcessorMode, - outstandingExecutables: make(map[tasks.Key]queues.Executable), - ackLevel: ackLevel, - readLevel: ackLevel, - minQueryLevel: ackLevel.FireTime, - pageToken: nil, - maxQueryLevel: ackLevel.FireTime, - isReadFinished: false, - finishedChan: nil, - clusterName: clusterName, - executableInitializer: executableInitializer, - } - - return timerQueueAckMgrImpl -} - -func newTimerQueueFailoverAckMgr( - shard shard.Context, - minLevel time.Time, - maxLevel time.Time, - timeNow timeNow, - updateTimerAckLevel updateTimerAckLevel, - timerQueueShutdown timerQueueShutdown, - logger log.Logger, - executableInitializer taskExecutableInitializer, -) *timerQueueAckMgrImpl { - // failover ack manager will start from the standby cluster's ack level to active cluster's ack level - ackLevel := tasks.NewKey(minLevel, 0) - - timerQueueAckMgrImpl := &timerQueueAckMgrImpl{ - operation: metrics.TimerActiveQueueProcessorScope, - isFailover: true, - shard: shard, - executionMgr: shard.GetExecutionManager(), - metricHandler: shard.GetMetricsHandler().WithTags(metrics.OperationTag(metrics.TimerActiveQueueProcessorScope)), - logger: logger, - config: shard.GetConfig(), - timeNow: timeNow, - updateTimerAckLevel: updateTimerAckLevel, - timerQueueShutdown: timerQueueShutdown, - singleProcessorMode: false, - outstandingExecutables: make(map[tasks.Key]queues.Executable), - ackLevel: ackLevel, - readLevel: ackLevel, - minQueryLevel: ackLevel.FireTime, - pageToken: nil, - maxQueryLevel: maxLevel, - isReadFinished: false, - finishedChan: make(chan struct{}, 1), - executableInitializer: executableInitializer, - } - - return timerQueueAckMgrImpl -} - -func (t *timerQueueAckMgrImpl) getFinishedChan() <-chan struct{} { - return t.finishedChan -} - -func (t *timerQueueAckMgrImpl) readTimerTasks() ([]queues.Executable, *time.Time, bool, error) { - if t.maxQueryLevel == t.minQueryLevel { - highReadWatermark, err := t.shard.UpdateScheduledQueueExclusiveHighReadWatermark(t.clusterName, t.singleProcessorMode) - if err != nil { - return nil, nil, false, err - } - t.maxQueryLevel = highReadWatermark.FireTime - t.maxQueryLevel = util.MaxTime(t.minQueryLevel, t.maxQueryLevel) - } - minQueryLevel := t.minQueryLevel - maxQueryLevel := t.maxQueryLevel - pageToken := t.pageToken - - var timerTasks []tasks.Task - morePage := false - var err error - if minQueryLevel.Before(maxQueryLevel) { - timerTasks, pageToken, err = t.getTimerTasks(minQueryLevel, maxQueryLevel, t.config.TimerTaskBatchSize(), pageToken) - if err != nil { - return nil, nil, false, err - } - morePage = len(pageToken) != 0 - t.logger.Debug("readTimerTasks", - tag.MinQueryLevel(minQueryLevel), tag.MaxQueryLevel(maxQueryLevel), tag.Counter(len(timerTasks)), tag.Bool(morePage)) - } - - t.Lock() - t.pageToken = pageToken - if t.isFailover && !morePage { - t.isReadFinished = true - } - - // We filter tasks so read only moves to desired timer tasks. - // We also get a look ahead task but it doesn't move the read level, this is for timer - // to wait on it instead of doing queries. - - var nextFireTime *time.Time - filteredExecutables := make([]queues.Executable, 0, len(timerTasks)) - -TaskFilterLoop: - for _, task := range timerTasks { - timerKey := tasks.NewKey(task.GetVisibilityTime(), task.GetTaskID()) - _, isLoaded := t.outstandingExecutables[timerKey] - if isLoaded { - // timer already loaded - continue TaskFilterLoop - } - - if !t.isProcessNow(timerKey.FireTime) { - nextFireTime = timestamp.TimePtr(task.GetVisibilityTime()) // this means there is task in the time range (now, now + offset) - t.maxQueryLevel = timerKey.FireTime // adjust maxQueryLevel so that this task will be read next time - break TaskFilterLoop - } - - t.logger.Debug("Moving timer read level", tag.Task(timerKey)) - t.readLevel = timerKey - - taskExecutable := t.executableInitializer(task) - t.outstandingExecutables[timerKey] = taskExecutable - filteredExecutables = append(filteredExecutables, taskExecutable) - } - - if nextFireTime != nil || !morePage { - if t.isReadFinished { - t.minQueryLevel = maximumTime // set it to the maximum time to avoid any mistakenly read - } else { - t.minQueryLevel = t.maxQueryLevel - } - t.logger.Debug("Moved timer minQueryLevel", tag.MinQueryLevel(t.minQueryLevel)) - t.pageToken = nil - } - t.Unlock() - - // only do lookahead when not in failover mode - if len(t.pageToken) == 0 && nextFireTime == nil && !t.isFailover { - nextFireTime, err = t.readLookAheadTask() - if err != nil { - // NOTE do not return nil filtered task - // or otherwise the tasks are loaded and will never be dispatched - // return true so timer quque process base will do another call - return filteredExecutables, nil, true, nil - } - } - - // We may have large number of timers which need to be fired immediately. Return true in such case so the pump - // can call back immediately to retrieve more tasks - moreTasks := nextFireTime == nil && morePage - - return filteredExecutables, nextFireTime, moreTasks, nil -} - -// read lookAheadTask from s.GetTimerMaxReadLevel to poll interval from there. -func (t *timerQueueAckMgrImpl) readLookAheadTask() (*time.Time, error) { - minQueryLevel := t.maxQueryLevel - maxQueryLevel := minQueryLevel.Add(t.config.TimerProcessorMaxPollInterval()) - - var tasks []tasks.Task - var err error - tasks, _, err = t.getTimerTasks(minQueryLevel, maxQueryLevel, 1, nil) - if err != nil { - return nil, err - } - if len(tasks) == 1 { - return timestamp.TimePtr(tasks[0].GetVisibilityTime()), nil - } - return timestamp.TimePtr(maxQueryLevel), nil -} - -func (t *timerQueueAckMgrImpl) getReadLevel() tasks.Key { - t.Lock() - defer t.Unlock() - return t.readLevel -} - -func (t *timerQueueAckMgrImpl) getAckLevel() tasks.Key { - t.Lock() - defer t.Unlock() - return t.ackLevel -} - -func (t *timerQueueAckMgrImpl) updateAckLevel() error { - t.metricHandler.Counter(metrics.AckLevelUpdateCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(t.operation)) - - t.Lock() - ackLevel := t.ackLevel - - // Timer Sequence IDs can have holes in the middle. So we sort the map to get the order to - // check. TODO: we can maintain a sorted slice as well. - sequenceIDs := tasks.Keys(maps.Keys(t.outstandingExecutables)) - sort.Sort(sequenceIDs) - - pendingTasks := len(sequenceIDs) - if pendingTasks > warnPendingTasks { - t.logger.Warn("Too many pending tasks.") - } - switch t.operation { - case metrics.TimerActiveQueueProcessorScope: - t.metricHandler.Histogram( - metrics.ShardInfoTimerActivePendingTasksTimer.GetMetricName(), - metrics.ShardInfoTimerActivePendingTasksTimer.GetMetricUnit(), - ).Record( - int64(pendingTasks), - metrics.OperationTag(metrics.ShardInfoScope), - ) - case metrics.TimerStandbyQueueProcessorScope: - t.metricHandler.Histogram( - metrics.ShardInfoTimerStandbyPendingTasksTimer.GetMetricName(), - metrics.ShardInfoTimerStandbyPendingTasksTimer.GetMetricUnit(), - ).Record( - int64(pendingTasks), - metrics.OperationTag(metrics.ShardInfoScope), - ) - } - -MoveAckLevelLoop: - for _, current := range sequenceIDs { - acked := t.outstandingExecutables[current].State() == ctasks.TaskStateAcked - if acked { - ackLevel = current - delete(t.outstandingExecutables, current) - t.logger.Debug("Moving timer ack level", tag.AckLevel(ackLevel)) - } else { - break MoveAckLevelLoop - } - } - t.ackLevel = ackLevel - - if t.isFailover && t.isReadFinished && len(t.outstandingExecutables) == 0 { - t.Unlock() - // this means in failover mode, all possible failover timer tasks - // are processed and we are free to shutdown - t.logger.Debug("Timer ack manager shutdown") - t.finishedChan <- struct{}{} - err := t.timerQueueShutdown() - if err != nil { - t.logger.Error("Error shutting down timer queue", tag.Error(err)) - } - return err - } - - t.Unlock() - if err := t.updateTimerAckLevel(ackLevel); err != nil { - t.metricHandler.Counter(metrics.AckLevelUpdateFailedCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(t.operation)) - t.logger.Error("Error updating timer ack level for shard", tag.Error(err)) - return err - } - return nil -} - -// this function does not take cluster name as parameter, due to we only have one timer queue on Cassandra -// all timer tasks are in this queue and filter will be applied. -func (t *timerQueueAckMgrImpl) getTimerTasks(minTimestamp time.Time, maxTimestamp time.Time, batchSize int, pageToken []byte) ([]tasks.Task, []byte, error) { - ctx, cancel := newQueueIOContext() - defer cancel() - - request := &persistence.GetHistoryTasksRequest{ - ShardID: t.shard.GetShardID(), - TaskCategory: tasks.CategoryTimer, - InclusiveMinTaskKey: tasks.NewKey(minTimestamp, 0), - ExclusiveMaxTaskKey: tasks.NewKey(maxTimestamp, 0), - BatchSize: batchSize, - NextPageToken: pageToken, - } - response, err := t.executionMgr.GetHistoryTasks(ctx, request) - if err != nil { - return nil, nil, err - } - return response.Tasks, response.NextPageToken, nil -} - -func (t *timerQueueAckMgrImpl) isProcessNow(expiryTime time.Time) bool { - if expiryTime.IsZero() { // return true, but somewhere probably have bug creating empty timerTask. - t.logger.Warn("Timer task has timestamp zero") - } - return expiryTime.UnixNano() <= t.timeNow().UnixNano() -} diff --git a/service/history/timerQueueAckMgr_test.go b/service/history/timerQueueAckMgr_test.go deleted file mode 100644 index f3eb4d4b484..00000000000 --- a/service/history/timerQueueAckMgr_test.go +++ /dev/null @@ -1,913 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "testing" - "time" - - "github.com/golang/mock/gomock" - "github.com/pborman/uuid" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - enumspb "go.temporal.io/api/enums/v1" - - persistencespb "go.temporal.io/server/api/persistence/v1" - "go.temporal.io/server/common/cluster" - "go.temporal.io/server/common/definition" - "go.temporal.io/server/common/dynamicconfig" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/primitives" - "go.temporal.io/server/common/primitives/timestamp" - ctasks "go.temporal.io/server/common/tasks" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/tests" -) - -type ( - timerQueueAckMgrSuite struct { - suite.Suite - *require.Assertions - - controller *gomock.Controller - mockShard *shard.ContextTest - mockClusterMetadata *cluster.MockMetadata - - mockExecutionMgr *persistence.MockExecutionManager - mockShardMgr *persistence.MockShardManager - - logger log.Logger - clusterName string - timerQueueAckMgr *timerQueueAckMgrImpl - } - - timerQueueFailoverAckMgrSuite struct { - suite.Suite - *require.Assertions - - controller *gomock.Controller - mockShard *shard.ContextTest - mockClusterMetadata *cluster.MockMetadata - - mockExecutionMgr *persistence.MockExecutionManager - mockShardMgr *persistence.MockShardManager - - logger log.Logger - namespaceID string - timerQueueFailoverAckMgr *timerQueueAckMgrImpl - minLevel time.Time - maxLevel time.Time - } -) - -var ( - TestNamespaceId = primitives.MustValidateUUID("deadbeef-c001-face-0000-000000000000") -) - -func TestTimerQueueAckMgrSuite(t *testing.T) { - s := new(timerQueueAckMgrSuite) - suite.Run(t, s) -} - -func TestTimerQueueFailoverAckMgrSuite(t *testing.T) { - s := new(timerQueueFailoverAckMgrSuite) - suite.Run(t, s) -} - -func (s *timerQueueAckMgrSuite) SetupSuite() { - -} - -func (s *timerQueueAckMgrSuite) TearDownSuite() { - -} - -func (s *timerQueueAckMgrSuite) SetupTest() { - s.Assertions = require.New(s.T()) - - config := tests.NewDynamicConfig() - config.ShardUpdateMinInterval = dynamicconfig.GetDurationPropertyFn(0 * time.Second) - - s.controller = gomock.NewController(s.T()) - s.mockShard = shard.NewTestContext( - s.controller, - &persistence.ShardInfoWithFailover{ - ShardInfo: &persistencespb.ShardInfo{ - ShardId: 1, - RangeId: 1, - QueueAckLevels: map[int32]*persistencespb.QueueAckLevel{ - tasks.CategoryTimer.ID(): { - ClusterAckLevel: map[string]int64{ - cluster.TestCurrentClusterName: timestamp.TimeNowPtrUtcAddSeconds(-8).UnixNano(), - cluster.TestAlternativeClusterName: timestamp.TimeNowPtrUtcAddSeconds(-10).UnixNano(), - }, - }, - }, - }, - }, - config, - ) - - s.mockShardMgr = s.mockShard.Resource.ShardMgr - s.mockExecutionMgr = s.mockShard.Resource.ExecutionMgr - s.mockClusterMetadata = s.mockShard.Resource.ClusterMetadata - - s.logger = s.mockShard.GetLogger() - - // this is used by shard context, not relevant to this test, so we do not care how many times "GetCurrentClusterName" is called - s.clusterName = cluster.TestCurrentClusterName - s.timerQueueAckMgr = newTimerQueueAckMgr( - metrics.TimerQueueProcessorScope, - s.mockShard, - s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime, - func() time.Time { - return s.mockShard.GetCurrentTime(s.clusterName) - }, - func(ackLevel tasks.Key) error { - return s.mockShard.UpdateQueueClusterAckLevel( - tasks.CategoryTimer, - s.clusterName, - ackLevel, - ) - }, - s.logger, - s.clusterName, - func(task tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - task, - nil, - nil, - nil, - nil, - queues.NewNoopPriorityAssigner(), - s.mockShard.GetTimeSource(), - nil, - nil, - metrics.NoopMetricsHandler, - nil, - nil, - ) - }, - false, - ) -} - -func (s *timerQueueAckMgrSuite) TearDownTest() { - s.controller.Finish() - s.mockShard.StopForTest() -} - -// Test for normal ack manager - -func (s *timerQueueAckMgrSuite) TestIsProcessNow() { - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.True(s.timerQueueAckMgr.isProcessNow(time.Time{})) - - now := s.mockShard.GetCurrentTime(s.clusterName) - s.True(s.timerQueueAckMgr.isProcessNow(now)) - - timeBefore := now.Add(-10 * time.Second) - s.True(s.timerQueueAckMgr.isProcessNow(timeBefore)) - - timeAfter := now.Add(10 * time.Second) - s.False(s.timerQueueAckMgr.isProcessNow(timeAfter)) -} - -func (s *timerQueueAckMgrSuite) TestGetTimerTasks_More() { - minTimestamp := time.Now().UTC().Add(-10 * time.Second) - maxTimestamp := time.Now().UTC().Add(10 * time.Second) - batchSize := 10 - - request := &persistence.GetHistoryTasksRequest{ - ShardID: s.mockShard.GetShardID(), - TaskCategory: tasks.CategoryTimer, - InclusiveMinTaskKey: tasks.NewKey(minTimestamp, 0), - ExclusiveMaxTaskKey: tasks.NewKey(maxTimestamp, 0), - BatchSize: batchSize, - NextPageToken: []byte("some random input next page token"), - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{ - &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - }, - }, - NextPageToken: []byte("some random output next page token"), - } - - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), request).Return(response, nil) - - timers, token, err := s.timerQueueAckMgr.getTimerTasks(minTimestamp, maxTimestamp, batchSize, request.NextPageToken) - s.Nil(err) - s.Equal(response.Tasks, timers) - s.Equal(response.NextPageToken, token) -} - -func (s *timerQueueAckMgrSuite) TestGetTimerTasks_NoMore() { - minTimestamp := time.Now().UTC().Add(-10 * time.Second) - maxTimestamp := time.Now().UTC().Add(10 * time.Second) - batchSize := 10 - - request := &persistence.GetHistoryTasksRequest{ - ShardID: s.mockShard.GetShardID(), - TaskCategory: tasks.CategoryTimer, - InclusiveMinTaskKey: tasks.NewKey(minTimestamp, 0), - ExclusiveMaxTaskKey: tasks.NewKey(maxTimestamp, 0), - BatchSize: batchSize, - NextPageToken: nil, - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{ - &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - }, - }, - NextPageToken: nil, - } - - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), request).Return(response, nil) - - timers, token, err := s.timerQueueAckMgr.getTimerTasks(minTimestamp, maxTimestamp, batchSize, request.NextPageToken) - s.Nil(err) - s.Equal(response.Tasks, timers) - s.Empty(token) -} - -func (s *timerQueueAckMgrSuite) TestReadTimerTasks_NoLookAhead_NoNextPage() { - ackLevel := s.timerQueueAckMgr.ackLevel - minQueryLevel := s.timerQueueAckMgr.minQueryLevel - token := s.timerQueueAckMgr.pageToken - maxQueryLevel := s.timerQueueAckMgr.maxQueryLevel - - // test ack && read level is initialized correctly - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName), ackLevel) - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime, minQueryLevel) - s.Empty(token) - s.Equal(minQueryLevel, maxQueryLevel) - - timer := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer}, - NextPageToken: nil, - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(&persistence.GetHistoryTasksResponse{}, nil) - filteredExecutables, nextFireTime, moreTasks, err := s.timerQueueAckMgr.readTimerTasks() - s.Nil(err) - - filteredTasks := make([]tasks.Task, 0, len(filteredExecutables)) - for _, executable := range filteredExecutables { - filteredTasks = append(filteredTasks, executable.GetTask()) - } - s.Equal([]tasks.Task{timer}, filteredTasks) - s.Equal(s.timerQueueAckMgr.maxQueryLevel.Add(s.timerQueueAckMgr.config.TimerProcessorMaxPollInterval()), *nextFireTime) - s.False(moreTasks) - - timerSequenceID := tasks.NewKey(timer.VisibilityTimestamp, timer.TaskID) - s.Len(s.timerQueueAckMgr.outstandingExecutables, 1) - s.Equal(ctasks.TaskStatePending, s.timerQueueAckMgr.outstandingExecutables[timerSequenceID].State()) - s.Equal(ackLevel, s.timerQueueAckMgr.ackLevel) - s.Empty(s.timerQueueAckMgr.pageToken) - s.Equal(s.timerQueueAckMgr.minQueryLevel, s.timerQueueAckMgr.maxQueryLevel) -} - -func (s *timerQueueAckMgrSuite) TestReadTimerTasks_NoLookAhead_HasNextPage() { - ackLevel := s.timerQueueAckMgr.ackLevel - minQueryLevel := s.timerQueueAckMgr.minQueryLevel - token := s.timerQueueAckMgr.pageToken - maxQueryLevel := s.timerQueueAckMgr.maxQueryLevel - - // test ack && read level is initialized correctly - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName), ackLevel) - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime, minQueryLevel) - s.Empty(token) - s.Equal(minQueryLevel, maxQueryLevel) - - timer := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Version: int64(79), - Attempt: 1, - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer}, - NextPageToken: []byte("some random next page token"), - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - readTimestamp := time.Now().UTC() // the approximate time of calling readTimerTasks - filteredExecutables, nextFireTime, moreTasks, err := s.timerQueueAckMgr.readTimerTasks() - s.Nil(err) - - filteredTasks := make([]tasks.Task, 0, len(filteredExecutables)) - for _, executable := range filteredExecutables { - filteredTasks = append(filteredTasks, executable.GetTask()) - } - s.Equal([]tasks.Task{timer}, filteredTasks) - s.Nil(nextFireTime) - s.True(moreTasks) - timerSequenceID := tasks.NewKey(timer.VisibilityTimestamp, timer.TaskID) - s.Len(s.timerQueueAckMgr.outstandingExecutables, 1) - s.Equal(ctasks.TaskStatePending, s.timerQueueAckMgr.outstandingExecutables[timerSequenceID].State()) - s.Equal(ackLevel, s.timerQueueAckMgr.ackLevel) - s.Equal(minQueryLevel, s.timerQueueAckMgr.minQueryLevel) - s.Equal(response.NextPageToken, s.timerQueueAckMgr.pageToken) - s.True(s.timerQueueAckMgr.maxQueryLevel.After(readTimestamp)) - s.True(s.timerQueueAckMgr.maxQueryLevel.Before(readTimestamp.Add(s.mockShard.GetConfig().TimerProcessorMaxTimeShift()).Add(time.Second))) -} - -func (s *timerQueueAckMgrSuite) TestReadTimerTasks_HasLookAhead_NoNextPage() { - ackLevel := s.timerQueueAckMgr.ackLevel - minQueryLevel := s.timerQueueAckMgr.minQueryLevel - token := s.timerQueueAckMgr.pageToken - maxQueryLevel := s.timerQueueAckMgr.maxQueryLevel - - // test ack && read level is initialized correctly - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName), ackLevel) - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime, minQueryLevel) - s.Empty(token) - s.Equal(minQueryLevel, maxQueryLevel) - - timer := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(s.mockShard.GetConfig().TimerProcessorMaxTimeShift()), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer}, - NextPageToken: nil, - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - filteredExecutables, nextFireTime, moreTasks, err := s.timerQueueAckMgr.readTimerTasks() - s.Nil(err) - - filteredTasks := make([]tasks.Task, 0, len(filteredExecutables)) - for _, executable := range filteredExecutables { - filteredTasks = append(filteredTasks, executable.GetTask()) - } - s.Equal([]tasks.Task{}, filteredTasks) - s.Equal(timer.GetVisibilityTime(), *nextFireTime) - s.False(moreTasks) - - s.Len(s.timerQueueAckMgr.outstandingExecutables, 0) - s.Equal(ackLevel, s.timerQueueAckMgr.ackLevel) - s.Equal(s.timerQueueAckMgr.maxQueryLevel, s.timerQueueAckMgr.minQueryLevel) - s.Empty(s.timerQueueAckMgr.pageToken) - s.Equal(timer.VisibilityTimestamp.UnixNano(), s.timerQueueAckMgr.maxQueryLevel.UnixNano()) -} - -func (s *timerQueueAckMgrSuite) TestReadTimerTasks_HasLookAhead_HasNextPage() { - ackLevel := s.timerQueueAckMgr.ackLevel - minQueryLevel := s.timerQueueAckMgr.minQueryLevel - token := s.timerQueueAckMgr.pageToken - maxQueryLevel := s.timerQueueAckMgr.maxQueryLevel - - // test ack && read level is initialized correctly - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName), ackLevel) - s.Equal(s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime, minQueryLevel) - s.Empty(token) - s.Equal(minQueryLevel, maxQueryLevel) - - timer := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(s.mockShard.GetConfig().TimerProcessorMaxTimeShift()), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Version: int64(79), - Attempt: 1, - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer}, - NextPageToken: []byte("some random next page token"), - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - filteredExecutables, nextFireTime, moreTasks, err := s.timerQueueAckMgr.readTimerTasks() - s.Nil(err) - - s.Equal([]queues.Executable{}, filteredExecutables) - s.Equal(timer.GetVisibilityTime(), *nextFireTime) - s.False(moreTasks) - - s.Len(s.timerQueueAckMgr.outstandingExecutables, 0) - s.Equal(ackLevel, s.timerQueueAckMgr.ackLevel) - s.Equal(s.timerQueueAckMgr.maxQueryLevel, s.timerQueueAckMgr.minQueryLevel) - s.Empty(s.timerQueueAckMgr.pageToken) - s.Equal(timer.VisibilityTimestamp.UnixNano(), s.timerQueueAckMgr.maxQueryLevel.UnixNano()) -} - -func (s *timerQueueAckMgrSuite) TestReadCompleteUpdateTimerTasks() { - // create 3 timers, timer1 < timer2 < timer3 < now - timer1 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - } - - timer2 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: timer1.VisibilityTimestamp, - TaskID: timer1.TaskID + 1, - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(29), - Attempt: 1, - } - timer3 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: timer2.VisibilityTimestamp.Add(time.Second), - TaskID: timer2.TaskID + 1, - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(30), - Attempt: 1, - } - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer1, timer2, timer3}, - NextPageToken: nil, - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockClusterMetadata.EXPECT().GetAllClusterInfo().Return(cluster.TestAllClusterInfo).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(&persistence.GetHistoryTasksResponse{}, nil) - filteredExecutables, nextFireTime, moreTasks, err := s.timerQueueAckMgr.readTimerTasks() - s.Nil(err) - - filteredTasks := make([]tasks.Task, 0, len(filteredExecutables)) - for _, executable := range filteredExecutables { - filteredTasks = append(filteredTasks, executable.GetTask()) - } - s.Equal([]tasks.Task{timer1, timer2, timer3}, filteredTasks) - s.Equal(s.timerQueueAckMgr.maxQueryLevel.Add(s.timerQueueAckMgr.config.TimerProcessorMaxPollInterval()), *nextFireTime) - s.False(moreTasks) - - // we are not testing shard context - s.mockShardMgr.EXPECT().UpdateShard(gomock.Any(), gomock.Any()).Return(nil) - timerSequenceID1 := tasks.NewKey(timer1.VisibilityTimestamp, timer1.TaskID) - filteredExecutables[0].Ack() - s.Equal(ctasks.TaskStateAcked, s.timerQueueAckMgr.outstandingExecutables[timerSequenceID1].State()) - _ = s.timerQueueAckMgr.updateAckLevel() - s.Equal(timer1.VisibilityTimestamp.UnixNano(), s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime.UnixNano()) - - s.mockShardMgr.EXPECT().UpdateShard(gomock.Any(), gomock.Any()).Return(nil) - timerSequenceID3 := tasks.NewKey(timer3.VisibilityTimestamp, timer3.TaskID) - filteredExecutables[2].Ack() - s.Equal(ctasks.TaskStateAcked, s.timerQueueAckMgr.outstandingExecutables[timerSequenceID3].State()) - _ = s.timerQueueAckMgr.updateAckLevel() - // ack level remains unchanged - s.Equal(timer1.VisibilityTimestamp.UnixNano(), s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime.UnixNano()) - - // we are not testing shard context - s.mockShardMgr.EXPECT().UpdateShard(gomock.Any(), gomock.Any()).Return(nil) - timerSequenceID2 := tasks.NewKey(timer2.VisibilityTimestamp, timer2.TaskID) - filteredExecutables[1].Ack() - s.Equal(ctasks.TaskStateAcked, s.timerQueueAckMgr.outstandingExecutables[timerSequenceID2].State()) - _ = s.timerQueueAckMgr.updateAckLevel() - s.Equal(timer3.VisibilityTimestamp.UnixNano(), s.mockShard.GetQueueClusterAckLevel(tasks.CategoryTimer, s.clusterName).FireTime.UnixNano()) -} - -func (s *timerQueueAckMgrSuite) TestReadLookAheadTask() { - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(s.clusterName).AnyTimes() - highReadWatermark, err := s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(s.clusterName, false) - s.NoError(err) - level := highReadWatermark.FireTime - - s.timerQueueAckMgr.minQueryLevel = level - s.timerQueueAckMgr.maxQueryLevel = s.timerQueueAckMgr.minQueryLevel - - timer := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - Version: int64(79), - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer}, - NextPageToken: []byte("some random next page token"), - } - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - nextFireTime, err := s.timerQueueAckMgr.readLookAheadTask() - s.Nil(err) - s.Equal(timer.GetVisibilityTime(), *nextFireTime) -} - -// Tests for failover ack manager -func (s *timerQueueFailoverAckMgrSuite) SetupSuite() { - -} - -func (s *timerQueueFailoverAckMgrSuite) TearDownSuite() { - -} - -func (s *timerQueueFailoverAckMgrSuite) SetupTest() { - s.Assertions = require.New(s.T()) - - config := tests.NewDynamicConfig() - config.ShardUpdateMinInterval = dynamicconfig.GetDurationPropertyFn(0 * time.Second) - - s.controller = gomock.NewController(s.T()) - s.mockShard = shard.NewTestContext( - s.controller, - &persistence.ShardInfoWithFailover{ - ShardInfo: &persistencespb.ShardInfo{ - ShardId: 1, - RangeId: 1, - QueueAckLevels: map[int32]*persistencespb.QueueAckLevel{ - tasks.CategoryTimer.ID(): { - ClusterAckLevel: map[string]int64{ - cluster.TestCurrentClusterName: timestamp.TimeNowPtrUtc().UnixNano(), - cluster.TestAlternativeClusterName: timestamp.TimeNowPtrUtcAddSeconds(-10).UnixNano(), - }, - }, - }, - }, - FailoverLevels: make(map[tasks.Category]map[string]persistence.FailoverLevel), - }, - config, - ) - - s.mockShardMgr = s.mockShard.Resource.ShardMgr - s.mockExecutionMgr = s.mockShard.Resource.ExecutionMgr - s.mockClusterMetadata = s.mockShard.Resource.ClusterMetadata - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - - s.logger = s.mockShard.GetLogger() - - s.namespaceID = "deadd0d0-c001-face-d00d-020000000000" - s.minLevel = time.Now().UTC().Add(-10 * time.Minute) - s.maxLevel = time.Now().UTC().Add(10 * time.Minute) - s.timerQueueFailoverAckMgr = newTimerQueueFailoverAckMgr( - s.mockShard, - s.minLevel, - s.maxLevel, - func() time.Time { - return s.mockShard.GetCurrentTime(s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName()) - }, - func(ackLevel tasks.Key) error { - return s.mockShard.UpdateFailoverLevel( - tasks.CategoryTimer, - s.namespaceID, - persistence.FailoverLevel{ - MinLevel: ackLevel, - MaxLevel: ackLevel, - NamespaceIDs: map[string]struct{}{s.namespaceID: {}}, - }, - ) - }, - func() error { - return s.mockShard.DeleteFailoverLevel(tasks.CategoryTimer, s.namespaceID) - }, - s.logger, - func(task tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - task, - nil, - nil, - nil, - nil, - queues.NewNoopPriorityAssigner(), - s.mockShard.GetTimeSource(), - nil, - nil, - metrics.NoopMetricsHandler, - nil, - nil, - ) - }, - ) -} - -func (s *timerQueueFailoverAckMgrSuite) TearDownTest() { - s.controller.Finish() - s.mockShard.StopForTest() -} - -func (s *timerQueueFailoverAckMgrSuite) TestIsProcessNow() { - // failover test to process whether to process a timer is use the current cluster's time - now := s.mockShard.GetCurrentTime(s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName()) - s.True(s.timerQueueFailoverAckMgr.isProcessNow(time.Time{})) - s.True(s.timerQueueFailoverAckMgr.isProcessNow(now)) - - timeBefore := now.Add(-5 * time.Second) - s.True(s.timerQueueFailoverAckMgr.isProcessNow(timeBefore)) - - timeAfter := now.Add(5 * time.Second) - s.False(s.timerQueueFailoverAckMgr.isProcessNow(timeAfter)) -} - -func (s *timerQueueFailoverAckMgrSuite) TestReadTimerTasks_HasNextPage() { - ackLevel := s.timerQueueFailoverAckMgr.ackLevel - minQueryLevel := s.timerQueueFailoverAckMgr.minQueryLevel - token := s.timerQueueFailoverAckMgr.pageToken - maxQueryLevel := s.timerQueueFailoverAckMgr.maxQueryLevel - - // test ack && read level is initialized correctly - s.Equal(s.minLevel, ackLevel.FireTime) - s.Equal(s.minLevel, minQueryLevel) - s.Empty(token) - s.Equal(s.maxLevel, maxQueryLevel) - - timer1 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - } - - timer2 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(60), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - } - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer1, timer2}, - NextPageToken: []byte("some random next page token"), - } - - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - readTimestamp := time.Now().UTC() // the approximate time of calling readTimerTasks - filteredExecutables, lookAheadTimer, more, err := s.timerQueueFailoverAckMgr.readTimerTasks() - s.Nil(err) - - filteredTasks := make([]tasks.Task, 0, len(filteredExecutables)) - for _, executable := range filteredExecutables { - filteredTasks = append(filteredTasks, executable.GetTask()) - } - s.Equal([]tasks.Task{timer1, timer2}, filteredTasks) - s.Nil(lookAheadTimer) - s.True(more) - s.Equal(ackLevel, s.timerQueueFailoverAckMgr.ackLevel) - s.Equal(minQueryLevel, s.timerQueueFailoverAckMgr.minQueryLevel) - s.Equal(response.NextPageToken, s.timerQueueFailoverAckMgr.pageToken) - s.True(s.timerQueueFailoverAckMgr.maxQueryLevel.After(readTimestamp)) - s.Equal(maxQueryLevel, s.timerQueueFailoverAckMgr.maxQueryLevel) -} - -func (s *timerQueueFailoverAckMgrSuite) TestReadTimerTasks_NoNextPage() { - ackLevel := s.timerQueueFailoverAckMgr.ackLevel - minQueryLevel := s.timerQueueFailoverAckMgr.minQueryLevel - token := s.timerQueueFailoverAckMgr.pageToken - maxQueryLevel := s.timerQueueFailoverAckMgr.maxQueryLevel - - // test ack && read level is initialized correctly - s.Equal(s.minLevel, ackLevel.FireTime) - s.Equal(s.minLevel, minQueryLevel) - s.Empty(token) - s.Equal(s.maxLevel, maxQueryLevel) - - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{}, - NextPageToken: nil, - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - - readTimestamp := time.Now().UTC() // the approximate time of calling readTimerTasks - timers, lookAheadTimer, more, err := s.timerQueueFailoverAckMgr.readTimerTasks() - s.Nil(err) - s.Equal([]queues.Executable{}, timers) - s.Nil(lookAheadTimer) - s.False(more) - - s.Equal(ackLevel, s.timerQueueFailoverAckMgr.ackLevel) - s.Equal(maximumTime, s.timerQueueFailoverAckMgr.minQueryLevel) - s.Empty(s.timerQueueFailoverAckMgr.pageToken) - s.True(s.timerQueueFailoverAckMgr.maxQueryLevel.After(readTimestamp)) - s.Equal(maxQueryLevel, s.timerQueueFailoverAckMgr.maxQueryLevel) -} - -func (s *timerQueueFailoverAckMgrSuite) TestReadTimerTasks_InTheFuture() { - ackLevel := s.timerQueueFailoverAckMgr.ackLevel - - // when namespace failover happen, it is possible that remote cluster's time is after - // current cluster's time - maxQueryLevel := time.Now().UTC() - s.timerQueueFailoverAckMgr.minQueryLevel = maxQueryLevel.Add(1 * time.Second) - s.timerQueueFailoverAckMgr.maxQueryLevel = maxQueryLevel - - timers, lookAheadTimer, more, err := s.timerQueueFailoverAckMgr.readTimerTasks() - s.Nil(err) - s.Equal(0, len(timers)) - s.Nil(lookAheadTimer) - s.False(more) - - s.Equal(ackLevel, s.timerQueueFailoverAckMgr.ackLevel) - s.Equal(maximumTime, s.timerQueueFailoverAckMgr.minQueryLevel) - s.Empty(s.timerQueueFailoverAckMgr.pageToken) - s.Equal(maxQueryLevel, s.timerQueueFailoverAckMgr.maxQueryLevel) -} - -func (s *timerQueueFailoverAckMgrSuite) TestReadCompleteUpdateTimerTasks() { - from := time.Now().UTC().Add(-10 * time.Second) - s.timerQueueFailoverAckMgr.minQueryLevel = from - s.timerQueueFailoverAckMgr.maxQueryLevel = from - s.timerQueueFailoverAckMgr.ackLevel = tasks.NewKey(from, 0) - - // create 3 timers, timer1 < timer2 < timer3 < now - timer1 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: time.Now().UTC().Add(-5 * time.Second), - TaskID: int64(59), - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(28), - Attempt: 1, - } - - timer2 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: timer1.VisibilityTimestamp, - TaskID: timer1.TaskID + 1, - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(29), - Attempt: 1, - } - timer3 := &tasks.ActivityTimeoutTask{ - WorkflowKey: definition.NewWorkflowKey( - TestNamespaceId, - "some random workflow ID", - uuid.New(), - ), - VisibilityTimestamp: timer2.VisibilityTimestamp.Add(time.Second), - TaskID: timer2.TaskID + 1, - TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START, - EventID: int64(30), - Attempt: 1, - } - response := &persistence.GetHistoryTasksResponse{ - Tasks: []tasks.Task{timer1, timer2, timer3}, - NextPageToken: nil, - } - s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.mockClusterMetadata.EXPECT().GetAllClusterInfo().Return(cluster.TestAllClusterInfo).AnyTimes() - s.mockExecutionMgr.EXPECT().GetHistoryTasks(gomock.Any(), gomock.Any()).Return(response, nil) - filteredExecutables, nextFireTime, moreTasks, err := s.timerQueueFailoverAckMgr.readTimerTasks() - s.Nil(err) - filteredTasks := make([]tasks.Task, 0, len(filteredExecutables)) - for _, executable := range filteredExecutables { - filteredTasks = append(filteredTasks, executable.GetTask()) - } - s.Equal([]tasks.Task{timer1, timer2, timer3}, filteredTasks) - s.Nil(nextFireTime) - s.False(moreTasks) - - timerSequenceID2 := tasks.NewKey(timer2.VisibilityTimestamp, timer2.TaskID) - filteredExecutables[1].Ack() - s.Equal(ctasks.TaskStateAcked, s.timerQueueFailoverAckMgr.outstandingExecutables[timerSequenceID2].State()) - s.mockShardMgr.EXPECT().UpdateShard(gomock.Any(), gomock.Any()).Return(nil) - _ = s.timerQueueFailoverAckMgr.updateAckLevel() - select { - case <-s.timerQueueFailoverAckMgr.getFinishedChan(): - s.Fail("timer queue ack mgr finished chan should not be fired") - default: - } - - timerSequenceID3 := tasks.NewKey(timer3.VisibilityTimestamp, timer3.TaskID) - filteredExecutables[2].Ack() - s.Equal(ctasks.TaskStateAcked, s.timerQueueFailoverAckMgr.outstandingExecutables[timerSequenceID3].State()) - s.mockShardMgr.EXPECT().UpdateShard(gomock.Any(), gomock.Any()).Return(nil) - _ = s.timerQueueFailoverAckMgr.updateAckLevel() - select { - case <-s.timerQueueFailoverAckMgr.getFinishedChan(): - s.Fail("timer queue ack mgr finished chan should not be fired") - default: - } - - timerSequenceID1 := tasks.NewKey(timer1.VisibilityTimestamp, timer1.TaskID) - filteredExecutables[0].Ack() - s.Equal(ctasks.TaskStateAcked, s.timerQueueFailoverAckMgr.outstandingExecutables[timerSequenceID1].State()) - s.mockShardMgr.EXPECT().UpdateShard(gomock.Any(), gomock.Any()).Return(nil) - _ = s.timerQueueFailoverAckMgr.updateAckLevel() - select { - case <-s.timerQueueFailoverAckMgr.getFinishedChan(): - default: - s.Fail("timer queue ack mgr finished chan should be fired") - } -} diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go deleted file mode 100644 index 4965522f747..00000000000 --- a/service/history/timerQueueActiveProcessor.go +++ /dev/null @@ -1,355 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - "time" - - "github.com/pborman/uuid" - - "go.temporal.io/server/api/historyservice/v1" - "go.temporal.io/server/api/matchingservice/v1" - "go.temporal.io/server/client" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/namespace" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/common/timer" - "go.temporal.io/server/common/xdc" - deletemanager "go.temporal.io/server/service/history/deletemanager" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" -) - -type ( - timerQueueActiveProcessorImpl struct { - timerQueueProcessorBase *timerQueueProcessorBase - - // this is the scheduler owned by this active queue processor - ownedScheduler queues.Scheduler - } -) - -func newTimerQueueActiveProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - workflowDeleteManager deletemanager.DeleteManager, - matchingClient matchingservice.MatchingServiceClient, - taskAllocator taskAllocator, - clientBean client.Bean, - rateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, - logger log.Logger, - metricProvider metrics.MetricsHandler, - singleProcessor bool, -) *timerQueueActiveProcessorImpl { - currentClusterName := shard.GetClusterMetadata().GetCurrentClusterName() - timeNow := func() time.Time { - return shard.GetCurrentTime(currentClusterName) - } - updateShardAckLevel := func(ackLevel tasks.Key) error { - // in single cursor mode, continue to update cluster ack level - // complete task loop will update overall ack level and - // shard.UpdateQueueAcklevel will then forward it to standby cluster ack level entries - // so that we can later disable single cursor mode without encountering tombstone issues - return shard.UpdateQueueClusterAckLevel( - tasks.CategoryTimer, - currentClusterName, - ackLevel, - ) - } - logger = log.With(logger, tag.ClusterName(currentClusterName)) - config := shard.GetConfig() - - processor := &timerQueueActiveProcessorImpl{} - - if scheduler == nil { - scheduler = newTimerTaskShardScheduler(shard, schedulerRateLimiter, logger) - processor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationTimerActiveQueueProcessorScope)), - ) - - timerTaskFilter := func(task tasks.Task) bool { - return taskAllocator.verifyActiveTask(namespace.ID(task.GetNamespaceID()), task) - } - taskExecutor := newTimerQueueActiveTaskExecutor( - shard, - workflowCache, - workflowDeleteManager, - processor, - logger, - metricProvider, - config, - matchingClient, - ) - ackLevel := shard.GetQueueClusterAckLevel(tasks.CategoryTimer, currentClusterName).FireTime - - // if single cursor is enabled, then this processor is responsible for both active and standby tasks - // and we need to customize some parameters for ack manager and task executable - if singleProcessor { - timerTaskFilter = nil - taskExecutor = queues.NewExecutorWrapper( - currentClusterName, - shard.GetNamespaceRegistry(), - taskExecutor, - newTimerQueueStandbyTaskExecutor( - shard, - workflowCache, - workflowDeleteManager, - xdc.NewNDCHistoryResender( - shard.GetNamespaceRegistry(), - clientBean, - func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { - engine, err := shard.GetEngine(ctx) - if err != nil { - return err - } - return engine.ReplicateEventsV2(ctx, request) - }, - shard.GetPayloadSerializer(), - config.StandbyTaskReReplicationContextTimeout, - logger, - ), - matchingClient, - logger, - metricProvider, - // note: the cluster name is for calculating time for standby tasks, - // here we are basically using current cluster time - // this field will be deprecated soon, currently exists so that - // we have the option of revert to old behavior - currentClusterName, - config, - ), - logger, - ) - ackLevel = shard.GetQueueAckLevel(tasks.CategoryTimer).FireTime - } - - timerQueueAckMgr := newTimerQueueAckMgr( - metrics.TimerActiveQueueProcessorScope, - shard, - ackLevel, - timeNow, - updateShardAckLevel, - logger, - currentClusterName, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - timerTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - config.TimerTaskMaxRetryCount, - config.NamespaceCacheRefreshInterval, - ) - }, - singleProcessor, - ) - - processor.timerQueueProcessorBase = newTimerQueueProcessorBase( - metrics.TimerActiveQueueProcessorScope, - shard, - workflowCache, - processor, - timerQueueAckMgr, - timer.NewLocalGate(shard.GetTimeSource()), - scheduler, - rescheduler, - rateLimiter, - logger, - ) - - return processor -} - -func newTimerQueueFailoverProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - workflowDeleteManager deletemanager.DeleteManager, - namespaceIDs map[string]struct{}, - standbyClusterName string, - minLevel time.Time, - maxLevel time.Time, - matchingClient matchingservice.MatchingServiceClient, - taskAllocator taskAllocator, - rateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, - logger log.Logger, - metricProvider metrics.MetricsHandler, -) (func(ackLevel tasks.Key) error, *timerQueueActiveProcessorImpl) { - currentClusterName := shard.GetClusterMetadata().GetCurrentClusterName() - timeNow := func() time.Time { - // should use current cluster's time when doing namespace failover - return shard.GetCurrentTime(currentClusterName) - } - failoverStartTime := shard.GetTimeSource().Now() - failoverUUID := uuid.New() - - updateShardAckLevel := func(ackLevel tasks.Key) error { - return shard.UpdateFailoverLevel( - tasks.CategoryTimer, - failoverUUID, - persistence.FailoverLevel{ - StartTime: failoverStartTime, - MinLevel: tasks.NewKey(minLevel, 0), - CurrentLevel: ackLevel, - MaxLevel: tasks.NewKey(maxLevel, 0), - NamespaceIDs: namespaceIDs, - }, - ) - } - timerAckMgrShutdown := func() error { - return shard.DeleteFailoverLevel(tasks.CategoryTimer, failoverUUID) - } - - logger = log.With( - logger, - tag.ClusterName(currentClusterName), - tag.WorkflowNamespaceIDs(namespaceIDs), - tag.FailoverMsg("from: "+standbyClusterName), - ) - timerTaskFilter := func(task tasks.Task) bool { - return taskAllocator.verifyFailoverActiveTask(namespaceIDs, namespace.ID(task.GetNamespaceID()), task) - } - - processor := &timerQueueActiveProcessorImpl{} - - taskExecutor := newTimerQueueActiveTaskExecutor( - shard, - workflowCache, - workflowDeleteManager, - processor, - logger, - metricProvider, - shard.GetConfig(), - matchingClient, - ) - - if scheduler == nil { - scheduler = newTimerTaskShardScheduler(shard, schedulerRateLimiter, logger) - processor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationTimerActiveQueueProcessorScope)), - ) - - timerQueueAckMgr := newTimerQueueFailoverAckMgr( - shard, - minLevel, - maxLevel, - timeNow, - updateShardAckLevel, - timerAckMgrShutdown, - logger, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - timerTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - shard.GetConfig().TimerTaskMaxRetryCount, - shard.GetConfig().NamespaceCacheRefreshInterval, - ) - }, - ) - - processor.timerQueueProcessorBase = newTimerQueueProcessorBase( - metrics.TimerActiveQueueProcessorScope, - shard, - workflowCache, - processor, - timerQueueAckMgr, - timer.NewLocalGate(shard.GetTimeSource()), - scheduler, - rescheduler, - rateLimiter, - logger, - ) - - return updateShardAckLevel, processor -} - -func (t *timerQueueActiveProcessorImpl) Start() { - if t.ownedScheduler != nil { - t.ownedScheduler.Start() - } - t.timerQueueProcessorBase.Start() -} - -func (t *timerQueueActiveProcessorImpl) Stop() { - t.timerQueueProcessorBase.Stop() - if t.ownedScheduler != nil { - t.ownedScheduler.Stop() - } -} - -func (t *timerQueueActiveProcessorImpl) getAckLevel() tasks.Key { - return t.timerQueueProcessorBase.timerQueueAckMgr.getAckLevel() -} - -func (t *timerQueueActiveProcessorImpl) getReadLevel() tasks.Key { - return t.timerQueueProcessorBase.timerQueueAckMgr.getReadLevel() -} - -// NotifyNewTimers - Notify the processor about the new active timer events arrival. -// This should be called each time new timer events arrives, otherwise timers maybe fired unexpected. -func (t *timerQueueActiveProcessorImpl) notifyNewTimers( - timerTasks []tasks.Task, -) { - t.timerQueueProcessorBase.notifyNewTimers(timerTasks) -} diff --git a/service/history/timerQueueActiveTaskExecutor.go b/service/history/timerQueueActiveTaskExecutor.go index 1bceca3558a..e68cb34a588 100644 --- a/service/history/timerQueueActiveTaskExecutor.go +++ b/service/history/timerQueueActiveTaskExecutor.go @@ -60,8 +60,6 @@ import ( type ( timerQueueActiveTaskExecutor struct { *timerQueueTaskExecutorBase - - queueProcessor *timerQueueActiveProcessorImpl } ) @@ -69,7 +67,6 @@ func newTimerQueueActiveTaskExecutor( shard shard.Context, workflowCache wcache.Cache, workflowDeleteManager deletemanager.DeleteManager, - queueProcessor *timerQueueActiveProcessorImpl, logger log.Logger, metricProvider metrics.MetricsHandler, config *configs.Config, @@ -85,7 +82,6 @@ func newTimerQueueActiveTaskExecutor( metricProvider, config, ), - queueProcessor: queueProcessor, } } @@ -606,16 +602,7 @@ func (t *timerQueueActiveTaskExecutor) updateWorkflowExecution( } now := t.shard.GetTimeSource().Now() - err = context.UpdateWorkflowExecutionAsActive(ctx, now) - if err != nil { - if shard.IsShardOwnershipLostError(err) && t.queueProcessor != nil { - // Shard is stolen. Stop timer processing to reduce duplicates - t.queueProcessor.Stop() - } - return err - } - - return nil + return context.UpdateWorkflowExecutionAsActive(ctx, now) } func (t *timerQueueActiveTaskExecutor) emitTimeoutMetricScopeWithNamespaceTag( diff --git a/service/history/timerQueueActiveTaskExecutor_test.go b/service/history/timerQueueActiveTaskExecutor_test.go index 721031546b8..1f6aa1f056f 100644 --- a/service/history/timerQueueActiveTaskExecutor_test.go +++ b/service/history/timerQueueActiveTaskExecutor_test.go @@ -54,7 +54,6 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" - "go.temporal.io/server/common/quotas" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -184,27 +183,6 @@ func (s *timerQueueActiveTaskExecutorSuite) SetupTest() { s.mockShard, s.workflowCache, s.mockDeleteManager, - newTimerQueueActiveProcessor( - s.mockShard, - s.workflowCache, - nil, - nil, - s.mockDeleteManager, - s.mockMatchingClient, - newTaskAllocator(s.mockShard), - s.mockShard.Resource.ClientBean, - quotas.NewDefaultOutgoingRateLimiter( - func() float64 { return float64(config.TimerProcessorMaxPollRPS()) }, - ), - quotas.NewRequestRateLimiterAdapter( - quotas.NewDefaultOutgoingRateLimiter( - func() float64 { return float64(config.TaskSchedulerMaxQPS()) }, - ), - ), - s.logger, - metrics.NoopMetricsHandler, - false, - ), s.logger, metrics.NoopMetricsHandler, config, diff --git a/service/history/timerQueueFactory.go b/service/history/timerQueueFactory.go index 450764646c4..7e6616d7193 100644 --- a/service/history/timerQueueFactory.go +++ b/service/history/timerQueueFactory.go @@ -68,28 +68,24 @@ type ( func NewTimerQueueFactory( params timerQueueFactoryParams, ) QueueFactory { - var hostScheduler queues.Scheduler - if params.Config.TimerProcessorEnablePriorityTaskScheduler() { - hostScheduler = queues.NewNamespacePriorityScheduler( - params.ClusterMetadata.GetCurrentClusterName(), - queues.NamespacePrioritySchedulerOptions{ - WorkerCount: params.Config.TimerProcessorSchedulerWorkerCount, - ActiveNamespaceWeights: params.Config.TimerProcessorSchedulerActiveRoundRobinWeights, - StandbyNamespaceWeights: params.Config.TimerProcessorSchedulerStandbyRoundRobinWeights, - EnableRateLimiter: params.Config.TaskSchedulerEnableRateLimiter, - MaxDispatchThrottleDuration: HostSchedulerMaxDispatchThrottleDuration, - }, - params.NamespaceRegistry, - params.SchedulerRateLimiter, - params.TimeSource, - params.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTimerQueueProcessorScope)), - params.Logger, - ) - } return &timerQueueFactory{ timerQueueFactoryParams: params, QueueFactoryBase: QueueFactoryBase{ - HostScheduler: hostScheduler, + HostScheduler: queues.NewNamespacePriorityScheduler( + params.ClusterMetadata.GetCurrentClusterName(), + queues.NamespacePrioritySchedulerOptions{ + WorkerCount: params.Config.TimerProcessorSchedulerWorkerCount, + ActiveNamespaceWeights: params.Config.TimerProcessorSchedulerActiveRoundRobinWeights, + StandbyNamespaceWeights: params.Config.TimerProcessorSchedulerStandbyRoundRobinWeights, + EnableRateLimiter: params.Config.TaskSchedulerEnableRateLimiter, + MaxDispatchThrottleDuration: HostSchedulerMaxDispatchThrottleDuration, + }, + params.NamespaceRegistry, + params.SchedulerRateLimiter, + params.TimeSource, + params.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTimerQueueProcessorScope)), + params.Logger, + ), HostPriorityAssigner: queues.NewPriorityAssigner(), HostRateLimiter: NewQueueHostRateLimiter( params.Config.TimerProcessorMaxPollHostRPS, @@ -112,107 +108,91 @@ func (f *timerQueueFactory) CreateQueue( shard shard.Context, workflowCache wcache.Cache, ) queues.Queue { - if f.HostScheduler != nil && f.Config.TimerProcessorEnableMultiCursor() { - logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) - - currentClusterName := f.ClusterMetadata.GetCurrentClusterName() - workflowDeleteManager := deletemanager.NewDeleteManager( - shard, - workflowCache, - f.Config, - f.ArchivalClient, - shard.GetTimeSource(), - ) - - activeExecutor := newTimerQueueActiveTaskExecutor( - shard, - workflowCache, - workflowDeleteManager, - nil, - logger, - f.MetricsHandler, - f.Config, - f.MatchingClient, - ) - - standbyExecutor := newTimerQueueStandbyTaskExecutor( - shard, - workflowCache, - workflowDeleteManager, - xdc.NewNDCHistoryResender( - shard.GetNamespaceRegistry(), - f.ClientBean, - func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { - engine, err := shard.GetEngine(ctx) - if err != nil { - return err - } - return engine.ReplicateEventsV2(ctx, request) - }, - shard.GetPayloadSerializer(), - f.Config.StandbyTaskReReplicationContextTimeout, - logger, - ), - f.MatchingClient, - logger, - f.MetricsHandler, - // note: the cluster name is for calculating time for standby tasks, - // here we are basically using current cluster time - // this field will be deprecated soon, currently exists so that - // we have the option of revert to old behavior - currentClusterName, - f.Config, - ) - - executor := queues.NewExecutorWrapper( - currentClusterName, - f.NamespaceRegistry, - activeExecutor, - standbyExecutor, - logger, - ) - - return queues.NewScheduledQueue( - shard, - tasks.CategoryTimer, - f.HostScheduler, - f.HostPriorityAssigner, - executor, - &queues.Options{ - ReaderOptions: queues.ReaderOptions{ - BatchSize: f.Config.TimerTaskBatchSize, - MaxPendingTasksCount: f.Config.QueuePendingTaskMaxCount, - PollBackoffInterval: f.Config.TimerProcessorPollBackoffInterval, - }, - MonitorOptions: queues.MonitorOptions{ - PendingTasksCriticalCount: f.Config.QueuePendingTaskCriticalCount, - ReaderStuckCriticalAttempts: f.Config.QueueReaderStuckCriticalAttempts, - SliceCountCriticalThreshold: f.Config.QueueCriticalSlicesCount, - }, - MaxPollRPS: f.Config.TimerProcessorMaxPollRPS, - MaxPollInterval: f.Config.TimerProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: f.Config.TimerProcessorMaxPollIntervalJitterCoefficient, - CheckpointInterval: f.Config.TimerProcessorUpdateAckInterval, - CheckpointIntervalJitterCoefficient: f.Config.TimerProcessorUpdateAckIntervalJitterCoefficient, - MaxReaderCount: f.Config.QueueMaxReaderCount, - TaskMaxRetryCount: f.Config.TimerTaskMaxRetryCount, - }, - f.HostReaderRateLimiter, - logger, - f.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTimerQueueProcessorScope)), - ) - } + logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) - return newTimerQueueProcessor( + currentClusterName := f.ClusterMetadata.GetCurrentClusterName() + workflowDeleteManager := deletemanager.NewDeleteManager( shard, workflowCache, - f.HostScheduler, - f.HostPriorityAssigner, - f.ClientBean, + f.Config, f.ArchivalClient, + shard.GetTimeSource(), + ) + + activeExecutor := newTimerQueueActiveTaskExecutor( + shard, + workflowCache, + workflowDeleteManager, + logger, + f.MetricsHandler, + f.Config, + f.MatchingClient, + ) + + standbyExecutor := newTimerQueueStandbyTaskExecutor( + shard, + workflowCache, + workflowDeleteManager, + xdc.NewNDCHistoryResender( + shard.GetNamespaceRegistry(), + f.ClientBean, + func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { + engine, err := shard.GetEngine(ctx) + if err != nil { + return err + } + return engine.ReplicateEventsV2(ctx, request) + }, + shard.GetPayloadSerializer(), + f.Config.StandbyTaskReReplicationContextTimeout, + logger, + ), f.MatchingClient, + logger, f.MetricsHandler, - f.HostRateLimiter, - f.SchedulerRateLimiter, + // note: the cluster name is for calculating time for standby tasks, + // here we are basically using current cluster time + // this field will be deprecated soon, currently exists so that + // we have the option of revert to old behavior + currentClusterName, + f.Config, + ) + + executor := queues.NewExecutorWrapper( + currentClusterName, + f.NamespaceRegistry, + activeExecutor, + standbyExecutor, + logger, + ) + + return queues.NewScheduledQueue( + shard, + tasks.CategoryTimer, + f.HostScheduler, + f.HostPriorityAssigner, + executor, + &queues.Options{ + ReaderOptions: queues.ReaderOptions{ + BatchSize: f.Config.TimerTaskBatchSize, + MaxPendingTasksCount: f.Config.QueuePendingTaskMaxCount, + PollBackoffInterval: f.Config.TimerProcessorPollBackoffInterval, + }, + MonitorOptions: queues.MonitorOptions{ + PendingTasksCriticalCount: f.Config.QueuePendingTaskCriticalCount, + ReaderStuckCriticalAttempts: f.Config.QueueReaderStuckCriticalAttempts, + SliceCountCriticalThreshold: f.Config.QueueCriticalSlicesCount, + }, + MaxPollRPS: f.Config.TimerProcessorMaxPollRPS, + MaxPollInterval: f.Config.TimerProcessorMaxPollInterval, + MaxPollIntervalJitterCoefficient: f.Config.TimerProcessorMaxPollIntervalJitterCoefficient, + CheckpointInterval: f.Config.TimerProcessorUpdateAckInterval, + CheckpointIntervalJitterCoefficient: f.Config.TimerProcessorUpdateAckIntervalJitterCoefficient, + MaxReaderCount: f.Config.QueueMaxReaderCount, + TaskMaxRetryCount: f.Config.TimerTaskMaxRetryCount, + }, + f.HostReaderRateLimiter, + logger, + f.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTimerQueueProcessorScope)), ) } diff --git a/service/history/timerQueueProcessor.go b/service/history/timerQueueProcessor.go deleted file mode 100644 index f9a7b183c4c..00000000000 --- a/service/history/timerQueueProcessor.go +++ /dev/null @@ -1,447 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "fmt" - "sync" - "sync/atomic" - "time" - - "go.temporal.io/api/serviceerror" - - "go.temporal.io/server/api/matchingservice/v1" - "go.temporal.io/server/client" - "go.temporal.io/server/common" - "go.temporal.io/server/common/backoff" - "go.temporal.io/server/common/cluster" - "go.temporal.io/server/common/dynamicconfig" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/service/history/configs" - deletemanager "go.temporal.io/server/service/history/deletemanager" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" - "go.temporal.io/server/service/worker/archiver" -) - -var errUnknownTimerTask = serviceerror.NewInternal("unknown timer task") - -type ( - timeNow func() time.Time - updateTimerAckLevel func(tasks.Key) error - timerQueueShutdown func() error - timerQueueProcessorImpl struct { - singleProcessor bool - currentClusterName string - shard shard.Context - taskAllocator taskAllocator - config *configs.Config - metricHandler metrics.MetricsHandler - workflowCache wcache.Cache - scheduler queues.Scheduler - priorityAssigner queues.PriorityAssigner - workflowDeleteManager deletemanager.DeleteManager - ackLevel tasks.Key - hostRateLimiter quotas.RateLimiter - schedulerRateLimiter queues.SchedulerRateLimiter - logger log.Logger - clientBean client.Bean - matchingClient matchingservice.MatchingServiceClient - status int32 - shutdownChan chan struct{} - shutdownWG sync.WaitGroup - activeTimerProcessor *timerQueueActiveProcessorImpl - standbyTimerProcessorsLock sync.RWMutex - standbyTimerProcessors map[string]*timerQueueStandbyProcessorImpl - } -) - -func newTimerQueueProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - clientBean client.Bean, - archivalClient archiver.Client, - matchingClient matchingservice.MatchingServiceClient, - metricHandler metrics.MetricsHandler, - hostRateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, -) queues.Queue { - - singleProcessor := !shard.GetClusterMetadata().IsGlobalNamespaceEnabled() || - shard.GetConfig().TimerProcessorEnableSingleProcessor() - - currentClusterName := shard.GetClusterMetadata().GetCurrentClusterName() - config := shard.GetConfig() - logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) - taskAllocator := newTaskAllocator(shard) - workflowDeleteManager := deletemanager.NewDeleteManager( - shard, - workflowCache, - config, - archivalClient, - shard.GetTimeSource(), - ) - - return &timerQueueProcessorImpl{ - singleProcessor: singleProcessor, - currentClusterName: currentClusterName, - shard: shard, - taskAllocator: taskAllocator, - config: config, - metricHandler: metricHandler, - workflowCache: workflowCache, - scheduler: scheduler, - priorityAssigner: priorityAssigner, - workflowDeleteManager: workflowDeleteManager, - ackLevel: shard.GetQueueAckLevel(tasks.CategoryTimer), - hostRateLimiter: hostRateLimiter, - schedulerRateLimiter: schedulerRateLimiter, - logger: logger, - clientBean: clientBean, - matchingClient: matchingClient, - status: common.DaemonStatusInitialized, - shutdownChan: make(chan struct{}), - activeTimerProcessor: newTimerQueueActiveProcessor( - shard, - workflowCache, - scheduler, - priorityAssigner, - workflowDeleteManager, - matchingClient, - taskAllocator, - clientBean, - newQueueProcessorRateLimiter( - hostRateLimiter, - config.TimerProcessorMaxPollRPS, - ), - schedulerRateLimiter, - logger, - metricHandler, - singleProcessor, - ), - standbyTimerProcessors: make(map[string]*timerQueueStandbyProcessorImpl), - } -} - -func (t *timerQueueProcessorImpl) Start() { - if !atomic.CompareAndSwapInt32(&t.status, common.DaemonStatusInitialized, common.DaemonStatusStarted) { - return - } - t.activeTimerProcessor.Start() - if !t.singleProcessor { - t.listenToClusterMetadataChange() - } - - t.shutdownWG.Add(1) - go t.completeTimersLoop() -} - -func (t *timerQueueProcessorImpl) Stop() { - if !atomic.CompareAndSwapInt32(&t.status, common.DaemonStatusStarted, common.DaemonStatusStopped) { - return - } - t.activeTimerProcessor.Stop() - if !t.singleProcessor { - t.shard.GetClusterMetadata().UnRegisterMetadataChangeCallback(t) - t.standbyTimerProcessorsLock.RLock() - for _, standbyTimerProcessor := range t.standbyTimerProcessors { - standbyTimerProcessor.Stop() - } - t.standbyTimerProcessorsLock.RUnlock() - } - close(t.shutdownChan) - common.AwaitWaitGroup(&t.shutdownWG, time.Minute) -} - -// NotifyNewTasks - Notify the processor about the new active / standby timer arrival. -// This should be called each time new timer arrives, otherwise timers maybe fired unexpected. -func (t *timerQueueProcessorImpl) NotifyNewTasks( - clusterName string, - timerTasks []tasks.Task, -) { - if clusterName == t.currentClusterName || t.singleProcessor { - t.activeTimerProcessor.notifyNewTimers(timerTasks) - return - } - - t.standbyTimerProcessorsLock.RLock() - standbyTimerProcessor, ok := t.standbyTimerProcessors[clusterName] - t.standbyTimerProcessorsLock.RUnlock() - if !ok { - t.logger.Warn(fmt.Sprintf("Cannot find timer processor for %s.", clusterName)) - return - } - standbyTimerProcessor.setCurrentTime(t.shard.GetCurrentTime(clusterName)) - standbyTimerProcessor.notifyNewTimers(timerTasks) -} - -func (t *timerQueueProcessorImpl) FailoverNamespace( - namespaceIDs map[string]struct{}, -) { - if t.singleProcessor { - // TODO: we may want to reschedule all tasks for new active namespaces in buffer - // so that they don't have to keeping waiting on the backoff timer - return - } - - // Failover queue is used to scan all inflight tasks, if queue processor is not - // started, there's no inflight task and we don't need to create a failover processor. - // Also the HandleAction will be blocked if queue processor processing loop is not running. - if atomic.LoadInt32(&t.status) != common.DaemonStatusStarted { - return - } - - minLevel := t.shard.GetQueueClusterAckLevel(tasks.CategoryTimer, t.currentClusterName).FireTime - standbyClusterName := t.currentClusterName - for clusterName, info := range t.shard.GetClusterMetadata().GetAllClusterInfo() { - if !info.Enabled { - continue - } - - ackLevel := t.shard.GetQueueClusterAckLevel(tasks.CategoryTimer, clusterName).FireTime - if ackLevel.Before(minLevel) { - minLevel = ackLevel - standbyClusterName = clusterName - } - } - // the ack manager is exclusive, so just add a cassandra min precision - maxLevel := t.activeTimerProcessor.getReadLevel().FireTime.Add(1 * time.Millisecond) - t.logger.Info("Timer Failover Triggered", - tag.WorkflowNamespaceIDs(namespaceIDs), - tag.MinLevel(minLevel.UnixNano()), - tag.MaxLevel(maxLevel.UnixNano())) - // we should consider make the failover idempotent - updateShardAckLevel, failoverTimerProcessor := newTimerQueueFailoverProcessor( - t.shard, - t.workflowCache, - t.scheduler, - t.priorityAssigner, - t.workflowDeleteManager, - namespaceIDs, - standbyClusterName, - minLevel, - maxLevel, - t.matchingClient, - t.taskAllocator, - newQueueProcessorRateLimiter( - t.hostRateLimiter, - t.config.TimerProcessorFailoverMaxPollRPS, - ), - t.schedulerRateLimiter, - t.logger, - t.metricHandler, - ) - - // NOTE: READ REF BEFORE MODIFICATION - // ref: historyEngine.go registerNamespaceFailoverCallback function - err := updateShardAckLevel(tasks.NewKey(minLevel, 0)) - if err != nil { - t.logger.Error("Error when update shard ack level", tag.Error(err)) - } - failoverTimerProcessor.Start() -} - -func (t *timerQueueProcessorImpl) LockTaskProcessing() { - if t.singleProcessor { - return - } - - t.taskAllocator.lock() -} - -func (t *timerQueueProcessorImpl) UnlockTaskProcessing() { - if t.singleProcessor { - return - } - - t.taskAllocator.unlock() -} - -func (t *timerQueueProcessorImpl) Category() tasks.Category { - return tasks.CategoryTimer -} - -func (t *timerQueueProcessorImpl) completeTimersLoop() { - defer t.shutdownWG.Done() - - timer := time.NewTimer(t.config.TimerProcessorCompleteTimerInterval()) - defer timer.Stop() - - completeTaskRetryPolicy := common.CreateCompleteTaskRetryPolicy() - - for { - select { - case <-t.shutdownChan: - // before shutdown, make sure the ack level is up-to-date - if err := t.completeTimers(); err != nil { - t.logger.Error("Failed to complete timer task", tag.Error(err)) - } - return - case <-timer.C: - // TODO: We should have a better approach to handle shard and its component lifecycle - _ = backoff.ThrottleRetry(func() error { - err := t.completeTimers() - if err != nil { - t.logger.Info("Failed to complete timer task", tag.Error(err)) - } - return err - }, completeTaskRetryPolicy, func(err error) bool { - select { - case <-t.shutdownChan: - return false - default: - } - return !shard.IsShardOwnershipLostError(err) - }) - - timer.Reset(t.config.TimerProcessorCompleteTimerInterval()) - } - } -} - -func (t *timerQueueProcessorImpl) completeTimers() error { - lowerAckLevel := t.ackLevel - upperAckLevel := t.activeTimerProcessor.getAckLevel() - - if !t.singleProcessor { - t.standbyTimerProcessorsLock.RLock() - for _, standbyTimerProcessor := range t.standbyTimerProcessors { - ackLevel := standbyTimerProcessor.getAckLevel() - if upperAckLevel.CompareTo(ackLevel) > 0 { - upperAckLevel = ackLevel - } - } - t.standbyTimerProcessorsLock.RUnlock() - - for _, failoverInfo := range t.shard.GetAllFailoverLevels(tasks.CategoryTimer) { - if !upperAckLevel.FireTime.Before(failoverInfo.MinLevel.FireTime) { - upperAckLevel = failoverInfo.MinLevel - } - } - } - - t.logger.Debug("Start completing timer task", tag.AckLevel(lowerAckLevel), tag.AckLevel(upperAckLevel)) - if lowerAckLevel.CompareTo(upperAckLevel) > 0 { - return nil - } - - t.metricHandler.Counter(metrics.TaskBatchCompleteCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(metrics.TimerQueueProcessorScope)) - - if lowerAckLevel.FireTime.Before(upperAckLevel.FireTime) { - ctx, cancel := newQueueIOContext() - defer cancel() - - err := t.shard.GetExecutionManager().RangeCompleteHistoryTasks(ctx, &persistence.RangeCompleteHistoryTasksRequest{ - ShardID: t.shard.GetShardID(), - TaskCategory: tasks.CategoryTimer, - InclusiveMinTaskKey: tasks.NewKey(lowerAckLevel.FireTime, 0), - ExclusiveMaxTaskKey: tasks.NewKey(upperAckLevel.FireTime, 0), - }) - if err != nil { - return err - } - } - - t.ackLevel = upperAckLevel - - return t.shard.UpdateQueueAckLevel(tasks.CategoryTimer, t.ackLevel) -} - -func (t *timerQueueProcessorImpl) listenToClusterMetadataChange() { - t.shard.GetClusterMetadata().RegisterMetadataChangeCallback( - t, - t.handleClusterMetadataUpdate, - ) -} - -func (t *timerQueueProcessorImpl) handleClusterMetadataUpdate( - oldClusterMetadata map[string]*cluster.ClusterInformation, - newClusterMetadata map[string]*cluster.ClusterInformation, -) { - t.standbyTimerProcessorsLock.Lock() - defer t.standbyTimerProcessorsLock.Unlock() - for clusterName := range oldClusterMetadata { - if clusterName == t.currentClusterName { - continue - } - // The metadata triggers a update when the following fields update: 1. Enabled 2. Initial Failover Version 3. Cluster address - // The callback covers three cases: - // Case 1: Remove a cluster Case 2: Add a new cluster Case 3: Refresh cluster metadata. - if processor, ok := t.standbyTimerProcessors[clusterName]; ok { - // Case 1 and Case 3 - processor.Stop() - delete(t.standbyTimerProcessors, clusterName) - } - if clusterInfo := newClusterMetadata[clusterName]; clusterInfo != nil && clusterInfo.Enabled { - // Case 2 and Case 3 - processor := newTimerQueueStandbyProcessor( - t.shard, - t.workflowCache, - t.scheduler, - t.priorityAssigner, - t.workflowDeleteManager, - t.matchingClient, - clusterName, - t.taskAllocator, - t.clientBean, - newQueueProcessorRateLimiter( - t.hostRateLimiter, - t.config.TimerProcessorMaxPollRPS, - ), - t.schedulerRateLimiter, - t.logger, - t.metricHandler, - ) - processor.Start() - t.standbyTimerProcessors[clusterName] = processor - } - } -} - -func newQueueProcessorRateLimiter( - hostRateLimiter quotas.RateLimiter, - shardMaxPollRPS dynamicconfig.IntPropertyFn, -) quotas.RateLimiter { - return quotas.NewMultiRateLimiter( - []quotas.RateLimiter{ - quotas.NewDefaultOutgoingRateLimiter( - func() float64 { - return float64(shardMaxPollRPS()) - }, - ), - hostRateLimiter, - }, - ) -} diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go deleted file mode 100644 index 0861b1d5968..00000000000 --- a/service/history/timerQueueProcessorBase.go +++ /dev/null @@ -1,395 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - "sync" - "sync/atomic" - "time" - - "go.temporal.io/server/common/timer" - "go.temporal.io/server/service/history/configs" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" - - "go.temporal.io/server/common" - "go.temporal.io/server/common/backoff" - "go.temporal.io/server/common/clock" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/quotas" -) - -var ( - emptyTime = time.Time{} - - loadTimerTaskThrottleRetryDelay = 3 * time.Second -) - -type ( - timerQueueProcessorBase struct { - operation string - shard shard.Context - cache wcache.Cache - executionManager persistence.ExecutionManager - status int32 - shutdownWG sync.WaitGroup - shutdownCh chan struct{} - config *configs.Config - logger log.Logger - metricHandler metrics.MetricsHandler - timerProcessor common.Daemon - timerQueueAckMgr timerQueueAckMgr - timerGate timer.Gate - timeSource clock.TimeSource - rateLimiter quotas.RateLimiter - lastPollTime time.Time - readTaskRetrier backoff.Retrier - scheduler queues.Scheduler - rescheduler queues.Rescheduler - - // timer notification - newTimerCh chan struct{} - newTimeLock sync.Mutex - newTime time.Time - } -) - -func newTimerQueueProcessorBase( - operation string, - shard shard.Context, - workflowCache wcache.Cache, - timerProcessor common.Daemon, - timerQueueAckMgr timerQueueAckMgr, - timerGate timer.Gate, - scheduler queues.Scheduler, - rescheduler queues.Rescheduler, - rateLimiter quotas.RateLimiter, - logger log.Logger, -) *timerQueueProcessorBase { - logger = log.With(logger, tag.ComponentTimerQueue) - config := shard.GetConfig() - - base := &timerQueueProcessorBase{ - operation: operation, - shard: shard, - timerProcessor: timerProcessor, - cache: workflowCache, - executionManager: shard.GetExecutionManager(), - status: common.DaemonStatusInitialized, - shutdownCh: make(chan struct{}), - config: config, - logger: logger, - metricHandler: shard.GetMetricsHandler(), - timerQueueAckMgr: timerQueueAckMgr, - timerGate: timerGate, - timeSource: shard.GetTimeSource(), - newTimerCh: make(chan struct{}, 1), - lastPollTime: time.Time{}, - scheduler: scheduler, - rescheduler: rescheduler, - rateLimiter: rateLimiter, - readTaskRetrier: backoff.NewRetrier( - common.CreateReadTaskRetryPolicy(), - backoff.SystemClock, - ), - } - - return base -} - -func (t *timerQueueProcessorBase) Start() { - if !atomic.CompareAndSwapInt32(&t.status, common.DaemonStatusInitialized, common.DaemonStatusStarted) { - return - } - - t.rescheduler.Start() - - t.shutdownWG.Add(1) - // notify a initial scan - t.notifyNewTimer(time.Time{}) - go t.processorPump() - - t.logger.Info("Timer queue processor started.") -} - -func (t *timerQueueProcessorBase) Stop() { - if !atomic.CompareAndSwapInt32(&t.status, common.DaemonStatusStarted, common.DaemonStatusStopped) { - return - } - - t.rescheduler.Stop() - - t.timerGate.Close() - close(t.shutdownCh) - - if success := common.AwaitWaitGroup(&t.shutdownWG, time.Minute); !success { - t.logger.Warn("Timer queue processor timedout on shutdown.") - } - - t.logger.Info("Timer queue processor stopped.") -} - -func (t *timerQueueProcessorBase) processorPump() { - defer t.shutdownWG.Done() - -RetryProcessor: - for { - select { - case <-t.shutdownCh: - break RetryProcessor - default: - err := t.internalProcessor() - if err != nil { - t.logger.Error("processor pump failed with error", tag.Error(err)) - } - } - } - - t.logger.Info("Timer queue processor pump shutting down.") - t.logger.Info("Timer processor exiting.") -} - -// 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( - timerTasks []tasks.Task, -) { - if len(timerTasks) == 0 { - return - } - - newTime := timerTasks[0].GetVisibilityTime() - for _, task := range timerTasks { - ts := task.GetVisibilityTime() - if ts.Before(newTime) { - newTime = ts - } - } - - t.notifyNewTimer(newTime) -} - -func (t *timerQueueProcessorBase) notifyNewTimer( - newTime time.Time, -) { - t.newTimeLock.Lock() - defer t.newTimeLock.Unlock() - if t.newTime.IsZero() || newTime.Before(t.newTime) { - t.newTime = newTime - select { - case t.newTimerCh <- struct{}{}: - // Notified about new time. - default: - // Channel "full" -> drop and move on, this will happen only if service is in high load. - } - } -} - -func (t *timerQueueProcessorBase) internalProcessor() error { - pollTimer := time.NewTimer(backoff.JitDuration( - t.config.TimerProcessorMaxPollInterval(), - t.config.TimerProcessorMaxPollIntervalJitterCoefficient(), - )) - defer pollTimer.Stop() - - updateAckTimer := time.NewTimer(backoff.JitDuration( - t.config.TimerProcessorUpdateAckInterval(), - t.config.TimerProcessorUpdateAckIntervalJitterCoefficient(), - )) - defer updateAckTimer.Stop() - -eventLoop: - for { - // prioritize shutdown - select { - case <-t.shutdownCh: - break eventLoop - default: - // noop - } - - // Wait until one of four things occurs: - // 1. we get notified of a new message - // 2. the timer gate fires (message scheduled to be delivered) - // 3. shutdown was triggered. - // 4. updating ack level - // - select { - case <-t.shutdownCh: - break eventLoop - case <-t.timerQueueAckMgr.getFinishedChan(): - // timer queue ack manager indicate that all task scanned - // are finished and no more tasks - // use a separate goroutine since the caller hold the shutdownWG - // stop the entire timer queue processor, not just processor base. - go t.timerProcessor.Stop() - return nil - case <-t.timerGate.FireChan(): - nextFireTime, err := t.readAndFanoutTimerTasks() - if err != nil { - return err - } - if nextFireTime != nil { - t.timerGate.Update(*nextFireTime) - } - case <-pollTimer.C: - pollTimer.Reset(backoff.JitDuration( - t.config.TimerProcessorMaxPollInterval(), - t.config.TimerProcessorMaxPollIntervalJitterCoefficient(), - )) - if t.lastPollTime.Add(t.config.TimerProcessorMaxPollInterval()).Before(t.timeSource.Now()) { - nextFireTime, err := t.readAndFanoutTimerTasks() - if err != nil { - return err - } - if nextFireTime != nil { - t.timerGate.Update(*nextFireTime) - } - } - case <-updateAckTimer.C: - updateAckTimer.Reset(backoff.JitDuration( - t.config.TimerProcessorUpdateAckInterval(), - t.config.TimerProcessorUpdateAckIntervalJitterCoefficient(), - )) - if err := t.timerQueueAckMgr.updateAckLevel(); shard.IsShardOwnershipLostError(err) { - // shard is closed, shutdown timerQProcessor and bail out - // stop the entire timer queue processor, not just processor base. - go t.timerProcessor.Stop() - return err - } - case <-t.newTimerCh: - t.newTimeLock.Lock() - newTime := t.newTime - t.newTime = emptyTime - t.newTimeLock.Unlock() - // New Timer has arrived. - t.metricHandler.Counter(metrics.NewTimerNotifyCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(t.operation)) - t.timerGate.Update(newTime) - } - } - - return nil -} - -func (t *timerQueueProcessorBase) readAndFanoutTimerTasks() (*time.Time, error) { - ctx, cancel := context.WithTimeout(context.Background(), loadTimerTaskThrottleRetryDelay) - if err := t.rateLimiter.Wait(ctx); err != nil { - deadline, _ := ctx.Deadline() - t.notifyNewTimer(deadline) // re-enqueue the event - cancel() - return nil, nil - } - cancel() - - if !t.verifyReschedulerSize() { - return nil, nil - } - - t.lastPollTime = t.timeSource.Now() - timerTasks, nextFireTime, moreTasks, err := t.timerQueueAckMgr.readTimerTasks() - if err != nil { - if common.IsResourceExhausted(err) { - t.notifyNewTimer(t.timeSource.Now().Add(loadTimerTaskThrottleRetryDelay)) - } else if err != shard.ErrShardStatusUnknown && !shard.IsShardOwnershipLostError(err) { - t.notifyNewTimer(t.timeSource.Now().Add(t.readTaskRetrier.NextBackOff())) - } - // if shard status is invalid, stopping processing and wait for the notication from shard - // after shard is re-acquired - return nil, err - } - t.readTaskRetrier.Reset() - - for _, task := range timerTasks { - t.submitTask(task) - select { - case <-t.shutdownCh: - return nil, nil - default: - } - } - - if !moreTasks { - if nextFireTime == nil { - return nil, nil - } - return nextFireTime, nil - } - - t.notifyNewTimer(time.Time{}) // re-enqueue the event - return nil, nil -} - -func (t *timerQueueProcessorBase) verifyReschedulerSize() bool { - passed := t.rescheduler.Len() < t.config.TimerProcessorMaxReschedulerSize() - if !passed { - // set backoff timer - t.notifyNewTimer(t.timeSource.Now().Add(t.config.TimerProcessorPollBackoffInterval())) - } - - return passed -} - -func (t *timerQueueProcessorBase) submitTask( - executable queues.Executable, -) { - now := t.timeSource.Now() - // Persistence layer may lose precision when persisting the task, which essentially move - // task fire time forward. Need to account for that when submitting the task. - if fireTime := executable.GetKey().FireTime.Add(persistence.ScheduledTaskMinPrecision); now.Before(fireTime) { - t.rescheduler.Add(executable, fireTime) - return - } - - executable.SetScheduledTime(now) - if !t.scheduler.TrySubmit(executable) { - executable.Reschedule() - } -} - -func newTimerTaskShardScheduler( - shard shard.Context, - rateLimiter queues.SchedulerRateLimiter, - logger log.Logger, -) queues.Scheduler { - config := shard.GetConfig() - return queues.NewPriorityScheduler( - queues.PrioritySchedulerOptions{ - WorkerCount: config.TimerTaskWorkerCount, - EnableRateLimiter: config.TaskSchedulerEnableRateLimiter, - MaxDispatchThrottleDuration: ShardSchedulerMaxDispatchThrottleDuration, - }, - rateLimiter, - shard.GetTimeSource(), - logger, - ) -} diff --git a/service/history/timerQueueStandbyProcessor.go b/service/history/timerQueueStandbyProcessor.go deleted file mode 100644 index bf5ab578a5f..00000000000 --- a/service/history/timerQueueStandbyProcessor.go +++ /dev/null @@ -1,217 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - "time" - - "go.temporal.io/server/api/historyservice/v1" - "go.temporal.io/server/api/matchingservice/v1" - "go.temporal.io/server/client" - - enumsspb "go.temporal.io/server/api/enums/v1" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/namespace" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/common/timer" - "go.temporal.io/server/common/xdc" - deletemanager "go.temporal.io/server/service/history/deletemanager" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" -) - -type ( - timerQueueStandbyProcessorImpl struct { - timerGate timer.RemoteGate - timerQueueProcessorBase *timerQueueProcessorBase - - // this is the scheduler owned by this standby queue processor - ownedScheduler queues.Scheduler - } -) - -func newTimerQueueStandbyProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - workflowDeleteManager deletemanager.DeleteManager, - matchingClient matchingservice.MatchingServiceClient, - clusterName string, - taskAllocator taskAllocator, - clientBean client.Bean, - rateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, - logger log.Logger, - metricProvider metrics.MetricsHandler, -) *timerQueueStandbyProcessorImpl { - timeNow := func() time.Time { - return shard.GetCurrentTime(clusterName) - } - updateShardAckLevel := func(ackLevel tasks.Key) error { - return shard.UpdateQueueClusterAckLevel( - tasks.CategoryTimer, - clusterName, - ackLevel, - ) - } - logger = log.With(logger, tag.ClusterName(clusterName)) - timerTaskFilter := func(task tasks.Task) bool { - switch task.GetType() { - case enumsspb.TASK_TYPE_WORKFLOW_RUN_TIMEOUT, - enumsspb.TASK_TYPE_DELETE_HISTORY_EVENT: - return true - default: - return taskAllocator.verifyStandbyTask(clusterName, namespace.ID(task.GetNamespaceID()), task) - } - } - - timerGate := timer.NewRemoteGate() - timerGate.SetCurrentTime(shard.GetCurrentTime(clusterName)) - - config := shard.GetConfig() - taskExecutor := newTimerQueueStandbyTaskExecutor( - shard, - workflowCache, - workflowDeleteManager, - xdc.NewNDCHistoryResender( - shard.GetNamespaceRegistry(), - clientBean, - func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { - engine, err := shard.GetEngine(ctx) - if err != nil { - return err - } - return engine.ReplicateEventsV2(ctx, request) - }, - shard.GetPayloadSerializer(), - config.StandbyTaskReReplicationContextTimeout, - logger, - ), - matchingClient, - logger, - metricProvider, - clusterName, - config, - ) - - processor := &timerQueueStandbyProcessorImpl{ - timerGate: timerGate, - } - - if scheduler == nil { - scheduler = newTimerTaskShardScheduler(shard, schedulerRateLimiter, logger) - processor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationTimerStandbyQueueProcessorScope)), - ) - - timerQueueAckMgr := newTimerQueueAckMgr( - metrics.TimerStandbyQueueProcessorScope, - shard, - shard.GetQueueClusterAckLevel(tasks.CategoryTimer, clusterName).FireTime, - timeNow, - updateShardAckLevel, - logger, - clusterName, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - timerTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - config.TimerTaskMaxRetryCount, - config.NamespaceCacheRefreshInterval, - ) - }, - // we are creating standby processor, - // so we know we are not in single processor mode - false, - ) - - processor.timerQueueProcessorBase = newTimerQueueProcessorBase( - metrics.TimerStandbyQueueProcessorScope, - shard, - workflowCache, - processor, - timerQueueAckMgr, - timerGate, - scheduler, - rescheduler, - rateLimiter, - logger, - ) - - return processor -} - -func (t *timerQueueStandbyProcessorImpl) Start() { - if t.ownedScheduler != nil { - t.ownedScheduler.Start() - } - t.timerQueueProcessorBase.Start() -} - -func (t *timerQueueStandbyProcessorImpl) Stop() { - t.timerQueueProcessorBase.Stop() - if t.ownedScheduler != nil { - t.ownedScheduler.Stop() - } -} - -func (t *timerQueueStandbyProcessorImpl) setCurrentTime( - currentTime time.Time, -) { - t.timerGate.SetCurrentTime(currentTime) -} - -func (t *timerQueueStandbyProcessorImpl) getAckLevel() tasks.Key { - return t.timerQueueProcessorBase.timerQueueAckMgr.getAckLevel() -} - -// NotifyNewTimers - Notify the processor about the new standby timer events arrival. -// This should be called each time new timer events arrives, otherwise timers maybe fired unexpected. -func (t *timerQueueStandbyProcessorImpl) notifyNewTimers( - timerTasks []tasks.Task, -) { - t.timerQueueProcessorBase.notifyNewTimers(timerTasks) -} diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index ce01467f31a..5c96cd68d0e 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -46,6 +46,8 @@ import ( wcache "go.temporal.io/server/service/history/workflow/cache" ) +var errUnknownTimerTask = serviceerror.NewInternal("unknown timer task") + type ( timerQueueTaskExecutorBase struct { currentClusterName string diff --git a/service/history/transferQueueActiveProcessor.go b/service/history/transferQueueActiveProcessor.go deleted file mode 100644 index cb493eaa570..00000000000 --- a/service/history/transferQueueActiveProcessor.go +++ /dev/null @@ -1,388 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - - "github.com/pborman/uuid" - - "go.temporal.io/server/api/historyservice/v1" - "go.temporal.io/server/api/matchingservice/v1" - "go.temporal.io/server/client" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/namespace" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/common/sdk" - "go.temporal.io/server/common/xdc" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" - "go.temporal.io/server/service/worker/archiver" -) - -type ( - transferQueueActiveProcessorImpl struct { - *transferQueueProcessorBase - *queueProcessorBase - queueAckMgr - - // this is the scheduler owned by this active queue processor - ownedScheduler queues.Scheduler - } -) - -func newTransferQueueActiveProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - archivalClient archiver.Client, - sdkClientFactory sdk.ClientFactory, - matchingClient matchingservice.MatchingServiceClient, - historyClient historyservice.HistoryServiceClient, - taskAllocator taskAllocator, - clientBean client.Bean, - rateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, - logger log.Logger, - metricProvider metrics.MetricsHandler, - singleProcessor bool, -) *transferQueueActiveProcessorImpl { - config := shard.GetConfig() - options := &QueueProcessorOptions{ - BatchSize: config.TransferTaskBatchSize, - MaxPollInterval: config.TransferProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: config.TransferProcessorMaxPollIntervalJitterCoefficient, - UpdateAckInterval: config.TransferProcessorUpdateAckInterval, - UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, - MaxReschdulerSize: config.TransferProcessorMaxReschedulerSize, - PollBackoffInterval: config.TransferProcessorPollBackoffInterval, - Operation: metrics.TransferActiveQueueProcessorScope, - } - currentClusterName := shard.GetClusterMetadata().GetCurrentClusterName() - logger = log.With(logger, tag.ClusterName(currentClusterName)) - - maxReadLevel := func() int64 { - return shard.GetImmediateQueueExclusiveHighReadWatermark().TaskID - } - updateTransferAckLevel := func(ackLevel int64) error { - // in single cursor mode, continue to update cluster ack level - // complete task loop will update overall ack level and - // shard.UpdateQueueAcklevel will then forward it to standby cluster ack level entries - // so that we can later disable single cursor mode without encountering tombstone issues - return shard.UpdateQueueClusterAckLevel( - tasks.CategoryTransfer, - currentClusterName, - tasks.NewImmediateKey(ackLevel), - ) - } - - transferQueueShutdown := func() error { - return nil - } - - processor := &transferQueueActiveProcessorImpl{ - transferQueueProcessorBase: newTransferQueueProcessorBase( - shard, - options, - maxReadLevel, - updateTransferAckLevel, - transferQueueShutdown, - logger, - ), - } - - if scheduler == nil { - scheduler = newTransferTaskShardScheduler(shard, schedulerRateLimiter, logger) - processor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationTransferActiveQueueProcessorScope)), - ) - - transferTaskFilter := func(task tasks.Task) bool { - return taskAllocator.verifyActiveTask(namespace.ID(task.GetNamespaceID()), task) - } - taskExecutor := newTransferQueueActiveTaskExecutor( - shard, - workflowCache, - archivalClient, - sdkClientFactory, - logger, - metricProvider, - config, - matchingClient, - ) - ackLevel := shard.GetQueueClusterAckLevel(tasks.CategoryTransfer, currentClusterName).TaskID - - // if single cursor is enabled, then this processor is responsible for both active and standby tasks - // and we need to customize some parameters for ack manager and task executable - if singleProcessor { - transferTaskFilter = nil - taskExecutor = queues.NewExecutorWrapper( - currentClusterName, - shard.GetNamespaceRegistry(), - taskExecutor, - newTransferQueueStandbyTaskExecutor( - shard, - workflowCache, - archivalClient, - xdc.NewNDCHistoryResender( - shard.GetNamespaceRegistry(), - clientBean, - func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { - engine, err := shard.GetEngine(ctx) - if err != nil { - return err - } - return engine.ReplicateEventsV2(ctx, request) - }, - shard.GetPayloadSerializer(), - config.StandbyTaskReReplicationContextTimeout, - logger, - ), - logger, - metricProvider, - // note: the cluster name is for calculating time for standby tasks, - // here we are basically using current cluster time - // this field will be deprecated soon, currently exists so that - // we have the option of revert to old behavior - currentClusterName, - matchingClient, - ), - logger, - ) - - ackLevel = shard.GetQueueAckLevel(tasks.CategoryTransfer).TaskID - } - - queueAckMgr := newQueueAckMgr( - shard, - options, - processor, - ackLevel, - logger, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - transferTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - config.TransferTaskMaxRetryCount, - shard.GetConfig().NamespaceCacheRefreshInterval, - ) - }, - ) - - queueProcessorBase := newQueueProcessorBase( - currentClusterName, - shard, - options, - processor, - queueAckMgr, - workflowCache, - scheduler, - rescheduler, - rateLimiter, - logger, - ) - processor.queueAckMgr = queueAckMgr - processor.queueProcessorBase = queueProcessorBase - - return processor -} - -func newTransferQueueFailoverProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - archivalClient archiver.Client, - sdkClientFactory sdk.ClientFactory, - matchingClient matchingservice.MatchingServiceClient, - historyClient historyservice.HistoryServiceClient, - namespaceIDs map[string]struct{}, - standbyClusterName string, - minLevel int64, - maxLevel int64, - taskAllocator taskAllocator, - rateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, - logger log.Logger, - metricProvider metrics.MetricsHandler, -) (func(ackLevel int64) error, *transferQueueActiveProcessorImpl) { - config := shard.GetConfig() - options := &QueueProcessorOptions{ - BatchSize: config.TransferTaskBatchSize, - MaxPollInterval: config.TransferProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: config.TransferProcessorMaxPollIntervalJitterCoefficient, - UpdateAckInterval: config.TransferProcessorUpdateAckInterval, - UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, - MaxReschdulerSize: config.TransferProcessorMaxReschedulerSize, - PollBackoffInterval: config.TransferProcessorPollBackoffInterval, - Operation: metrics.TransferActiveQueueProcessorScope, - } - currentClusterName := shard.GetClusterMetadata().GetCurrentClusterName() - failoverUUID := uuid.New() - logger = log.With( - logger, - tag.ClusterName(currentClusterName), - tag.WorkflowNamespaceIDs(namespaceIDs), - tag.FailoverMsg("from: "+standbyClusterName), - ) - - transferTaskFilter := func(task tasks.Task) bool { - return taskAllocator.verifyFailoverActiveTask(namespaceIDs, namespace.ID(task.GetNamespaceID()), task) - } - maxReadAckLevel := func() int64 { - return maxLevel // this is a const - } - failoverStartTime := shard.GetTimeSource().Now() - updateTransferAckLevel := func(ackLevel int64) error { - return shard.UpdateFailoverLevel( - tasks.CategoryTransfer, - failoverUUID, - persistence.FailoverLevel{ - StartTime: failoverStartTime, - MinLevel: tasks.NewImmediateKey(minLevel), - CurrentLevel: tasks.NewImmediateKey(ackLevel), - MaxLevel: tasks.NewImmediateKey(maxLevel), - NamespaceIDs: namespaceIDs, - }, - ) - } - transferQueueShutdown := func() error { - return shard.DeleteFailoverLevel(tasks.CategoryTransfer, failoverUUID) - } - - processor := &transferQueueActiveProcessorImpl{ - transferQueueProcessorBase: newTransferQueueProcessorBase( - shard, - options, - maxReadAckLevel, - updateTransferAckLevel, - transferQueueShutdown, - logger, - ), - } - - taskExecutor := newTransferQueueActiveTaskExecutor( - shard, - workflowCache, - archivalClient, - sdkClientFactory, - logger, - metricProvider, - config, - matchingClient, - ) - - if scheduler == nil { - scheduler = newTransferTaskShardScheduler(shard, schedulerRateLimiter, logger) - processor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationTransferActiveQueueProcessorScope)), - ) - - queueAckMgr := newQueueFailoverAckMgr( - shard, - options, - processor, - minLevel, - logger, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - transferTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - shard.GetConfig().TransferTaskMaxRetryCount, - shard.GetConfig().NamespaceCacheRefreshInterval, - ) - }, - ) - - queueProcessorBase := newQueueProcessorBase( - currentClusterName, - shard, - options, - processor, - queueAckMgr, - workflowCache, - scheduler, - rescheduler, - rateLimiter, - logger, - ) - processor.queueAckMgr = queueAckMgr - processor.queueProcessorBase = queueProcessorBase - return updateTransferAckLevel, processor -} - -func (t *transferQueueActiveProcessorImpl) notifyNewTask() { - t.queueProcessorBase.notifyNewTask() -} - -func (t *transferQueueActiveProcessorImpl) Start() { - if t.ownedScheduler != nil { - t.ownedScheduler.Start() - } - t.queueProcessorBase.Start() -} - -func (t *transferQueueActiveProcessorImpl) Stop() { - t.queueProcessorBase.Stop() - if t.ownedScheduler != nil { - t.ownedScheduler.Stop() - } -} diff --git a/service/history/transferQueueFactory.go b/service/history/transferQueueFactory.go index beb3186a2a2..58dc23d9731 100644 --- a/service/history/transferQueueFactory.go +++ b/service/history/transferQueueFactory.go @@ -70,28 +70,24 @@ type ( func NewTransferQueueFactory( params transferQueueFactoryParams, ) QueueFactory { - var hostScheduler queues.Scheduler - if params.Config.TransferProcessorEnablePriorityTaskScheduler() { - hostScheduler = queues.NewNamespacePriorityScheduler( - params.ClusterMetadata.GetCurrentClusterName(), - queues.NamespacePrioritySchedulerOptions{ - WorkerCount: params.Config.TransferProcessorSchedulerWorkerCount, - ActiveNamespaceWeights: params.Config.TransferProcessorSchedulerActiveRoundRobinWeights, - StandbyNamespaceWeights: params.Config.TransferProcessorSchedulerStandbyRoundRobinWeights, - EnableRateLimiter: params.Config.TaskSchedulerEnableRateLimiter, - MaxDispatchThrottleDuration: HostSchedulerMaxDispatchThrottleDuration, - }, - params.NamespaceRegistry, - params.SchedulerRateLimiter, - params.TimeSource, - params.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTransferQueueProcessorScope)), - params.Logger, - ) - } return &transferQueueFactory{ transferQueueFactoryParams: params, QueueFactoryBase: QueueFactoryBase{ - HostScheduler: hostScheduler, + HostScheduler: queues.NewNamespacePriorityScheduler( + params.ClusterMetadata.GetCurrentClusterName(), + queues.NamespacePrioritySchedulerOptions{ + WorkerCount: params.Config.TransferProcessorSchedulerWorkerCount, + ActiveNamespaceWeights: params.Config.TransferProcessorSchedulerActiveRoundRobinWeights, + StandbyNamespaceWeights: params.Config.TransferProcessorSchedulerStandbyRoundRobinWeights, + EnableRateLimiter: params.Config.TaskSchedulerEnableRateLimiter, + MaxDispatchThrottleDuration: HostSchedulerMaxDispatchThrottleDuration, + }, + params.NamespaceRegistry, + params.SchedulerRateLimiter, + params.TimeSource, + params.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTransferQueueProcessorScope)), + params.Logger, + ), HostPriorityAssigner: queues.NewPriorityAssigner(), HostRateLimiter: NewQueueHostRateLimiter( params.Config.TransferProcessorMaxPollHostRPS, @@ -114,96 +110,79 @@ func (f *transferQueueFactory) CreateQueue( shard shard.Context, workflowCache wcache.Cache, ) queues.Queue { - if f.HostScheduler != nil && f.Config.TransferProcessorEnableMultiCursor() { - logger := log.With(shard.GetLogger(), tag.ComponentTransferQueue) - - currentClusterName := f.ClusterMetadata.GetCurrentClusterName() - activeExecutor := newTransferQueueActiveTaskExecutor( - shard, - workflowCache, - f.ArchivalClient, - f.SdkClientFactory, - logger, - f.MetricsHandler, - f.Config, - f.MatchingClient, - ) + logger := log.With(shard.GetLogger(), tag.ComponentTransferQueue) - standbyExecutor := newTransferQueueStandbyTaskExecutor( - shard, - workflowCache, - f.ArchivalClient, - xdc.NewNDCHistoryResender( - f.NamespaceRegistry, - f.ClientBean, - func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { - engine, err := shard.GetEngine(ctx) - if err != nil { - return err - } - return engine.ReplicateEventsV2(ctx, request) - }, - shard.GetPayloadSerializer(), - f.Config.StandbyTaskReReplicationContextTimeout, - logger, - ), - logger, - f.MetricsHandler, - currentClusterName, - f.MatchingClient, - ) + currentClusterName := f.ClusterMetadata.GetCurrentClusterName() + activeExecutor := newTransferQueueActiveTaskExecutor( + shard, + workflowCache, + f.ArchivalClient, + f.SdkClientFactory, + logger, + f.MetricsHandler, + f.Config, + f.MatchingClient, + ) - executor := queues.NewExecutorWrapper( - currentClusterName, + standbyExecutor := newTransferQueueStandbyTaskExecutor( + shard, + workflowCache, + f.ArchivalClient, + xdc.NewNDCHistoryResender( f.NamespaceRegistry, - activeExecutor, - standbyExecutor, - logger, - ) - - return queues.NewImmediateQueue( - shard, - tasks.CategoryTransfer, - f.HostScheduler, - f.HostPriorityAssigner, - executor, - &queues.Options{ - ReaderOptions: queues.ReaderOptions{ - BatchSize: f.Config.TransferTaskBatchSize, - MaxPendingTasksCount: f.Config.QueuePendingTaskMaxCount, - PollBackoffInterval: f.Config.TransferProcessorPollBackoffInterval, - }, - MonitorOptions: queues.MonitorOptions{ - PendingTasksCriticalCount: f.Config.QueuePendingTaskCriticalCount, - ReaderStuckCriticalAttempts: f.Config.QueueReaderStuckCriticalAttempts, - SliceCountCriticalThreshold: f.Config.QueueCriticalSlicesCount, - }, - MaxPollRPS: f.Config.TransferProcessorMaxPollRPS, - MaxPollInterval: f.Config.TransferProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: f.Config.TransferProcessorMaxPollIntervalJitterCoefficient, - CheckpointInterval: f.Config.TransferProcessorUpdateAckInterval, - CheckpointIntervalJitterCoefficient: f.Config.TransferProcessorUpdateAckIntervalJitterCoefficient, - MaxReaderCount: f.Config.QueueMaxReaderCount, - TaskMaxRetryCount: f.Config.TransferTaskMaxRetryCount, + f.ClientBean, + func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { + engine, err := shard.GetEngine(ctx) + if err != nil { + return err + } + return engine.ReplicateEventsV2(ctx, request) }, - f.HostReaderRateLimiter, + shard.GetPayloadSerializer(), + f.Config.StandbyTaskReReplicationContextTimeout, logger, - f.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTransferQueueProcessorScope)), - ) - } + ), + logger, + f.MetricsHandler, + currentClusterName, + f.MatchingClient, + ) + + executor := queues.NewExecutorWrapper( + currentClusterName, + f.NamespaceRegistry, + activeExecutor, + standbyExecutor, + logger, + ) - return newTransferQueueProcessor( + return queues.NewImmediateQueue( shard, - workflowCache, + tasks.CategoryTransfer, f.HostScheduler, f.HostPriorityAssigner, - f.ClientBean, - f.ArchivalClient, - f.SdkClientFactory, - f.MatchingClient, - f.HistoryClient, - f.MetricsHandler, - f.HostRateLimiter, - f.SchedulerRateLimiter, + executor, + &queues.Options{ + ReaderOptions: queues.ReaderOptions{ + BatchSize: f.Config.TransferTaskBatchSize, + MaxPendingTasksCount: f.Config.QueuePendingTaskMaxCount, + PollBackoffInterval: f.Config.TransferProcessorPollBackoffInterval, + }, + MonitorOptions: queues.MonitorOptions{ + PendingTasksCriticalCount: f.Config.QueuePendingTaskCriticalCount, + ReaderStuckCriticalAttempts: f.Config.QueueReaderStuckCriticalAttempts, + SliceCountCriticalThreshold: f.Config.QueueCriticalSlicesCount, + }, + MaxPollRPS: f.Config.TransferProcessorMaxPollRPS, + MaxPollInterval: f.Config.TransferProcessorMaxPollInterval, + MaxPollIntervalJitterCoefficient: f.Config.TransferProcessorMaxPollIntervalJitterCoefficient, + CheckpointInterval: f.Config.TransferProcessorUpdateAckInterval, + CheckpointIntervalJitterCoefficient: f.Config.TransferProcessorUpdateAckIntervalJitterCoefficient, + MaxReaderCount: f.Config.QueueMaxReaderCount, + TaskMaxRetryCount: f.Config.TransferTaskMaxRetryCount, + }, + f.HostReaderRateLimiter, + logger, + f.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationTransferQueueProcessorScope)), ) } diff --git a/service/history/transferQueueProcessor.go b/service/history/transferQueueProcessor.go deleted file mode 100644 index 8e84977bc46..00000000000 --- a/service/history/transferQueueProcessor.go +++ /dev/null @@ -1,424 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "fmt" - "sync" - "sync/atomic" - "time" - - "go.temporal.io/api/serviceerror" - - "go.temporal.io/server/api/historyservice/v1" - "go.temporal.io/server/api/matchingservice/v1" - "go.temporal.io/server/client" - "go.temporal.io/server/common" - "go.temporal.io/server/common/backoff" - "go.temporal.io/server/common/cluster" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/common/sdk" - "go.temporal.io/server/service/history/configs" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" - "go.temporal.io/server/service/worker/archiver" -) - -var errUnknownTransferTask = serviceerror.NewInternal("Unknown transfer task") - -type ( - taskFilter func(task tasks.Task) bool - - transferQueueProcessorImpl struct { - singleProcessor bool - currentClusterName string - shard shard.Context - workflowCache wcache.Cache - archivalClient archiver.Client - sdkClientFactory sdk.ClientFactory - taskAllocator taskAllocator - config *configs.Config - metricHandler metrics.MetricsHandler - clientBean client.Bean - matchingClient matchingservice.MatchingServiceClient - historyClient historyservice.HistoryServiceClient - ackLevel int64 - hostRateLimiter quotas.RateLimiter - schedulerRateLimiter queues.SchedulerRateLimiter - logger log.Logger - isStarted int32 - isStopped int32 - shutdownChan chan struct{} - scheduler queues.Scheduler - priorityAssigner queues.PriorityAssigner - activeTaskProcessor *transferQueueActiveProcessorImpl - standbyTaskProcessorsLock sync.RWMutex - standbyTaskProcessors map[string]*transferQueueStandbyProcessorImpl - } -) - -func newTransferQueueProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - clientBean client.Bean, - archivalClient archiver.Client, - sdkClientFactory sdk.ClientFactory, - matchingClient matchingservice.MatchingServiceClient, - historyClient historyservice.HistoryServiceClient, - metricProvider metrics.MetricsHandler, - hostRateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, -) queues.Queue { - - singleProcessor := !shard.GetClusterMetadata().IsGlobalNamespaceEnabled() || - shard.GetConfig().TransferProcessorEnableSingleProcessor() - - logger := log.With(shard.GetLogger(), tag.ComponentTransferQueue) - currentClusterName := shard.GetClusterMetadata().GetCurrentClusterName() - config := shard.GetConfig() - taskAllocator := newTaskAllocator(shard) - - return &transferQueueProcessorImpl{ - singleProcessor: singleProcessor, - currentClusterName: currentClusterName, - shard: shard, - workflowCache: workflowCache, - archivalClient: archivalClient, - sdkClientFactory: sdkClientFactory, - taskAllocator: taskAllocator, - config: config, - metricHandler: metricProvider, - clientBean: clientBean, - matchingClient: matchingClient, - historyClient: historyClient, - ackLevel: shard.GetQueueAckLevel(tasks.CategoryTransfer).TaskID, - hostRateLimiter: hostRateLimiter, - schedulerRateLimiter: schedulerRateLimiter, - logger: logger, - shutdownChan: make(chan struct{}), - scheduler: scheduler, - priorityAssigner: priorityAssigner, - activeTaskProcessor: newTransferQueueActiveProcessor( - shard, - workflowCache, - scheduler, - priorityAssigner, - archivalClient, - sdkClientFactory, - matchingClient, - historyClient, - taskAllocator, - clientBean, - newQueueProcessorRateLimiter( - hostRateLimiter, - config.TransferProcessorMaxPollRPS, - ), - schedulerRateLimiter, - logger, - metricProvider, - singleProcessor, - ), - standbyTaskProcessors: make(map[string]*transferQueueStandbyProcessorImpl), - } -} - -func (t *transferQueueProcessorImpl) Start() { - if !atomic.CompareAndSwapInt32(&t.isStarted, 0, 1) { - return - } - t.activeTaskProcessor.Start() - if !t.singleProcessor { - t.listenToClusterMetadataChange() - } - - go t.completeTransferLoop() -} - -func (t *transferQueueProcessorImpl) Stop() { - if !atomic.CompareAndSwapInt32(&t.isStopped, 0, 1) { - return - } - t.activeTaskProcessor.Stop() - if !t.singleProcessor { - t.shard.GetClusterMetadata().UnRegisterMetadataChangeCallback(t) - t.standbyTaskProcessorsLock.RLock() - for _, standbyTaskProcessor := range t.standbyTaskProcessors { - standbyTaskProcessor.Stop() - } - t.standbyTaskProcessorsLock.RUnlock() - } - close(t.shutdownChan) -} - -// NotifyNewTasks - Notify the processor about the new active / standby transfer task arrival. -// This should be called each time new transfer task arrives, otherwise tasks maybe delayed. -func (t *transferQueueProcessorImpl) NotifyNewTasks( - clusterName string, - transferTasks []tasks.Task, -) { - if clusterName == t.currentClusterName || t.singleProcessor { - // we will ignore the current time passed in, since the active processor process task immediately - if len(transferTasks) != 0 { - t.activeTaskProcessor.notifyNewTask() - } - return - } - - t.standbyTaskProcessorsLock.RLock() - standbyTaskProcessor, ok := t.standbyTaskProcessors[clusterName] - t.standbyTaskProcessorsLock.RUnlock() - if !ok { - t.logger.Warn(fmt.Sprintf("Cannot find transfer processor for %s.", clusterName)) - return - } - if len(transferTasks) != 0 { - standbyTaskProcessor.notifyNewTask() - } -} - -func (t *transferQueueProcessorImpl) FailoverNamespace( - namespaceIDs map[string]struct{}, -) { - if t.singleProcessor { - // TODO: we may want to reschedule all tasks for new active namespaces in buffer - // so that they don't have to keeping waiting on the backoff timer - return - } - - minLevel := t.shard.GetQueueClusterAckLevel(tasks.CategoryTransfer, t.currentClusterName).TaskID - standbyClusterName := t.currentClusterName - for clusterName, info := range t.shard.GetClusterMetadata().GetAllClusterInfo() { - if !info.Enabled { - continue - } - ackLevel := t.shard.GetQueueClusterAckLevel(tasks.CategoryTransfer, clusterName).TaskID - if ackLevel < minLevel { - minLevel = ackLevel - standbyClusterName = clusterName - } - } - - // the ack manager is exclusive, so add 1 - maxLevel := t.activeTaskProcessor.getQueueReadLevel() + 1 - t.logger.Info("Transfer Failover Triggered", - tag.WorkflowNamespaceIDs(namespaceIDs), - tag.MinLevel(minLevel), - tag.MaxLevel(maxLevel)) - updateShardAckLevel, failoverTaskProcessor := newTransferQueueFailoverProcessor( - t.shard, - t.workflowCache, - t.scheduler, - t.priorityAssigner, - t.archivalClient, - t.sdkClientFactory, - t.matchingClient, - t.historyClient, - namespaceIDs, - standbyClusterName, - minLevel, - maxLevel, - t.taskAllocator, - newQueueProcessorRateLimiter( - t.hostRateLimiter, - t.config.TransferProcessorFailoverMaxPollRPS, - ), - t.schedulerRateLimiter, - t.logger, - t.metricHandler, - ) - - // NOTE: READ REF BEFORE MODIFICATION - // ref: historyEngine.go registerNamespaceFailoverCallback function - err := updateShardAckLevel(minLevel) - if err != nil { - t.logger.Error("Error update shard ack level", tag.Error(err)) - } - failoverTaskProcessor.Start() -} - -func (t *transferQueueProcessorImpl) LockTaskProcessing() { - if t.singleProcessor { - return - } - - t.taskAllocator.lock() -} - -func (t *transferQueueProcessorImpl) UnlockTaskProcessing() { - if t.singleProcessor { - return - } - - t.taskAllocator.unlock() -} - -func (t *transferQueueProcessorImpl) Category() tasks.Category { - return tasks.CategoryTransfer -} - -func (t *transferQueueProcessorImpl) completeTransferLoop() { - timer := time.NewTimer(t.config.TransferProcessorCompleteTransferInterval()) - defer timer.Stop() - - completeTaskRetryPolicy := common.CreateCompleteTaskRetryPolicy() - - for { - select { - case <-t.shutdownChan: - // before shutdown, make sure the ack level is up to date - if err := t.completeTransfer(); err != nil { - t.logger.Error("Failed to complete transfer task", tag.Error(err)) - } - return - case <-timer.C: - // TODO: We should have a better approach to handle shard and its component lifecycle - _ = backoff.ThrottleRetry(func() error { - err := t.completeTransfer() - if err != nil { - t.logger.Info("Failed to complete transfer task", tag.Error(err)) - } - return err - }, completeTaskRetryPolicy, func(err error) bool { - select { - case <-t.shutdownChan: - return false - default: - } - return !shard.IsShardOwnershipLostError(err) - }) - - timer.Reset(t.config.TransferProcessorCompleteTransferInterval()) - } - } -} - -func (t *transferQueueProcessorImpl) completeTransfer() error { - lowerAckLevel := t.ackLevel - upperAckLevel := t.activeTaskProcessor.queueAckMgr.getQueueAckLevel() - - if !t.singleProcessor { - t.standbyTaskProcessorsLock.RLock() - for _, standbyTaskProcessor := range t.standbyTaskProcessors { - ackLevel := standbyTaskProcessor.queueAckMgr.getQueueAckLevel() - if upperAckLevel > ackLevel { - upperAckLevel = ackLevel - } - } - t.standbyTaskProcessorsLock.RUnlock() - - for _, failoverInfo := range t.shard.GetAllFailoverLevels(tasks.CategoryTransfer) { - if upperAckLevel > failoverInfo.MinLevel.TaskID { - upperAckLevel = failoverInfo.MinLevel.TaskID - } - } - } - - t.logger.Debug("Start completing transfer task", tag.AckLevel(lowerAckLevel), tag.AckLevel(upperAckLevel)) - if lowerAckLevel >= upperAckLevel { - return nil - } - - t.metricHandler.Counter(metrics.TaskBatchCompleteCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(metrics.TransferQueueProcessorScope)) - - if lowerAckLevel < upperAckLevel { - ctx, cancel := newQueueIOContext() - defer cancel() - - err := t.shard.GetExecutionManager().RangeCompleteHistoryTasks(ctx, &persistence.RangeCompleteHistoryTasksRequest{ - ShardID: t.shard.GetShardID(), - TaskCategory: tasks.CategoryTransfer, - InclusiveMinTaskKey: tasks.NewImmediateKey(lowerAckLevel + 1), - ExclusiveMaxTaskKey: tasks.NewImmediateKey(upperAckLevel + 1), - }) - if err != nil { - return err - } - } - - t.ackLevel = upperAckLevel - - return t.shard.UpdateQueueAckLevel(tasks.CategoryTransfer, tasks.NewImmediateKey(upperAckLevel)) -} - -func (t *transferQueueProcessorImpl) listenToClusterMetadataChange() { - t.shard.GetClusterMetadata().RegisterMetadataChangeCallback( - t, - t.handleClusterMetadataUpdate, - ) -} - -func (t *transferQueueProcessorImpl) handleClusterMetadataUpdate( - oldClusterMetadata map[string]*cluster.ClusterInformation, - newClusterMetadata map[string]*cluster.ClusterInformation, -) { - t.standbyTaskProcessorsLock.Lock() - defer t.standbyTaskProcessorsLock.Unlock() - for clusterName := range oldClusterMetadata { - if clusterName == t.currentClusterName { - continue - } - // The metadata triggers a update when the following fields update: 1. Enabled 2. Initial Failover Version 3. Cluster address - // The callback covers three cases: - // Case 1: Remove a cluster Case 2: Add a new cluster Case 3: Refresh cluster metadata. - if processor, ok := t.standbyTaskProcessors[clusterName]; ok { - // Case 1 and Case 3 - processor.Stop() - delete(t.standbyTaskProcessors, clusterName) - } - if clusterInfo := newClusterMetadata[clusterName]; clusterInfo != nil && clusterInfo.Enabled { - // Case 2 and Case 3 - processor := newTransferQueueStandbyProcessor( - clusterName, - t.shard, - t.scheduler, - t.priorityAssigner, - t.workflowCache, - t.archivalClient, - t.taskAllocator, - t.clientBean, - newQueueProcessorRateLimiter( - t.hostRateLimiter, - t.config.TransferProcessorMaxPollRPS, - ), - t.schedulerRateLimiter, - t.logger, - t.metricHandler, - t.matchingClient, - ) - processor.Start() - t.standbyTaskProcessors[clusterName] = processor - } - } -} diff --git a/service/history/transferQueueProcessorBase.go b/service/history/transferQueueProcessorBase.go deleted file mode 100644 index e2150b84bdd..00000000000 --- a/service/history/transferQueueProcessorBase.go +++ /dev/null @@ -1,130 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - "time" - - "go.temporal.io/server/common/headers" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" -) - -const ( - queueIOTimeout = 5 * time.Second -) - -type ( - maxReadLevel func() int64 - - updateTransferAckLevel func(ackLevel int64) error - transferQueueShutdown func() error - - transferQueueProcessorBase struct { - shard shard.Context - options *QueueProcessorOptions - executionManager persistence.ExecutionManager - maxReadLevel maxReadLevel - updateTransferAckLevel updateTransferAckLevel - transferQueueShutdown transferQueueShutdown - logger log.Logger - } -) - -func newTransferQueueProcessorBase( - shard shard.Context, - options *QueueProcessorOptions, - maxReadLevel maxReadLevel, - updateTransferAckLevel updateTransferAckLevel, - transferQueueShutdown transferQueueShutdown, - logger log.Logger, -) *transferQueueProcessorBase { - return &transferQueueProcessorBase{ - shard: shard, - options: options, - executionManager: shard.GetExecutionManager(), - maxReadLevel: maxReadLevel, - updateTransferAckLevel: updateTransferAckLevel, - transferQueueShutdown: transferQueueShutdown, - logger: logger, - } -} - -func (t *transferQueueProcessorBase) readTasks( - readLevel int64, -) ([]tasks.Task, bool, error) { - ctx, cancel := newQueueIOContext() - defer cancel() - - response, err := t.executionManager.GetHistoryTasks(ctx, &persistence.GetHistoryTasksRequest{ - ShardID: t.shard.GetShardID(), - TaskCategory: tasks.CategoryTransfer, - InclusiveMinTaskKey: tasks.NewImmediateKey(readLevel + 1), - ExclusiveMaxTaskKey: tasks.NewImmediateKey(t.maxReadLevel()), - BatchSize: t.options.BatchSize(), - }) - if err != nil { - return nil, false, err - } - return response.Tasks, len(response.NextPageToken) != 0, nil -} - -func (t *transferQueueProcessorBase) updateAckLevel( - ackLevel int64, -) error { - return t.updateTransferAckLevel(ackLevel) -} - -func (t *transferQueueProcessorBase) queueShutdown() error { - return t.transferQueueShutdown() -} - -func newTransferTaskShardScheduler( - shard shard.Context, - rateLimiter queues.SchedulerRateLimiter, - logger log.Logger, -) queues.Scheduler { - config := shard.GetConfig() - return queues.NewPriorityScheduler( - queues.PrioritySchedulerOptions{ - WorkerCount: config.TransferTaskWorkerCount, - EnableRateLimiter: config.TaskSchedulerEnableRateLimiter, - MaxDispatchThrottleDuration: ShardSchedulerMaxDispatchThrottleDuration, - }, - rateLimiter, - shard.GetTimeSource(), - logger, - ) -} - -func newQueueIOContext() (context.Context, context.CancelFunc) { - ctx, cancel := context.WithTimeout(context.Background(), queueIOTimeout) - ctx = headers.SetCallerInfo(ctx, headers.SystemBackgroundCallerInfo) - return ctx, cancel -} diff --git a/service/history/transferQueueStandbyProcessor.go b/service/history/transferQueueStandbyProcessor.go deleted file mode 100644 index 27ca894616e..00000000000 --- a/service/history/transferQueueStandbyProcessor.go +++ /dev/null @@ -1,220 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "context" - - enumsspb "go.temporal.io/server/api/enums/v1" - "go.temporal.io/server/api/historyservice/v1" - "go.temporal.io/server/api/matchingservice/v1" - "go.temporal.io/server/client" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/namespace" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/common/xdc" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" - "go.temporal.io/server/service/worker/archiver" -) - -type ( - transferQueueStandbyProcessorImpl struct { - *transferQueueProcessorBase - *queueProcessorBase - queueAckMgr - - // this is the scheduler owned by this standby queue processor - ownedScheduler queues.Scheduler - } -) - -func newTransferQueueStandbyProcessor( - clusterName string, - shard shard.Context, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - workflowCache wcache.Cache, - archivalClient archiver.Client, - taskAllocator taskAllocator, - clientBean client.Bean, - rateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, - logger log.Logger, - metricProvider metrics.MetricsHandler, - matchingClient matchingservice.MatchingServiceClient, -) *transferQueueStandbyProcessorImpl { - config := shard.GetConfig() - options := &QueueProcessorOptions{ - BatchSize: config.TransferTaskBatchSize, - MaxPollInterval: config.TransferProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: config.TransferProcessorMaxPollIntervalJitterCoefficient, - UpdateAckInterval: config.TransferProcessorUpdateAckInterval, - UpdateAckIntervalJitterCoefficient: config.TransferProcessorUpdateAckIntervalJitterCoefficient, - MaxReschdulerSize: config.TransferProcessorMaxReschedulerSize, - PollBackoffInterval: config.TransferProcessorPollBackoffInterval, - Operation: metrics.TransferStandbyQueueProcessorScope, - } - logger = log.With(logger, tag.ClusterName(clusterName)) - - transferTaskFilter := func(task tasks.Task) bool { - switch task.GetType() { - case enumsspb.TASK_TYPE_TRANSFER_RESET_WORKFLOW: - // no reset needed for standby - return false - case enumsspb.TASK_TYPE_TRANSFER_DELETE_EXECUTION: - return true - case enumsspb.TASK_TYPE_TRANSFER_CLOSE_EXECUTION: - if shard.GetArchivalMetadata().GetVisibilityConfig().ClusterConfiguredForArchival() { - return true - } - fallthrough - default: - return taskAllocator.verifyStandbyTask(clusterName, namespace.ID(task.GetNamespaceID()), task) - } - } - maxReadLevel := func() int64 { - // we are creating standby processor, so we know we are not in single processor mode - return shard.GetImmediateQueueExclusiveHighReadWatermark().TaskID - } - updateClusterAckLevel := func(ackLevel int64) error { - return shard.UpdateQueueClusterAckLevel(tasks.CategoryTransfer, clusterName, tasks.NewImmediateKey(ackLevel)) - } - transferQueueShutdown := func() error { - return nil - } - - processor := &transferQueueStandbyProcessorImpl{ - transferQueueProcessorBase: newTransferQueueProcessorBase( - shard, - options, - maxReadLevel, - updateClusterAckLevel, - transferQueueShutdown, - logger, - ), - } - - taskExecutor := newTransferQueueStandbyTaskExecutor( - shard, - workflowCache, - archivalClient, - xdc.NewNDCHistoryResender( - shard.GetNamespaceRegistry(), - clientBean, - func(ctx context.Context, request *historyservice.ReplicateEventsV2Request) error { - engine, err := shard.GetEngine(ctx) - if err != nil { - return err - } - return engine.ReplicateEventsV2(ctx, request) - }, - shard.GetPayloadSerializer(), - config.StandbyTaskReReplicationContextTimeout, - logger, - ), - logger, - metricProvider, - clusterName, - matchingClient, - ) - - if scheduler == nil { - scheduler = newTransferTaskShardScheduler(shard, schedulerRateLimiter, logger) - processor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationTransferStandbyQueueProcessorScope)), - ) - - queueAckMgr := newQueueAckMgr( - shard, - options, - processor, - shard.GetQueueClusterAckLevel(tasks.CategoryTransfer, clusterName).TaskID, - logger, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - transferTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - shard.GetConfig().TransferTaskMaxRetryCount, - shard.GetConfig().NamespaceCacheRefreshInterval, - ) - }, - ) - - queueProcessorBase := newQueueProcessorBase( - clusterName, - shard, - options, - processor, - queueAckMgr, - workflowCache, - scheduler, - rescheduler, - rateLimiter, - logger, - ) - - processor.queueAckMgr = queueAckMgr - processor.queueProcessorBase = queueProcessorBase - - return processor -} - -func (t *transferQueueStandbyProcessorImpl) notifyNewTask() { - t.queueProcessorBase.notifyNewTask() -} - -func (t *transferQueueStandbyProcessorImpl) Start() { - if t.ownedScheduler != nil { - t.ownedScheduler.Start() - } - t.queueProcessorBase.Start() -} - -func (t *transferQueueStandbyProcessorImpl) Stop() { - t.queueProcessorBase.Stop() - if t.ownedScheduler != nil { - t.ownedScheduler.Stop() - } -} diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index b765a31f83d..0374da66851 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -60,6 +60,8 @@ const ( taskHistoryOpTimeout = 20 * time.Second ) +var errUnknownTransferTask = serviceerror.NewInternal("Unknown transfer task") + type ( transferQueueTaskExecutorBase struct { currentClusterName string diff --git a/service/history/visibilityQueueFactory.go b/service/history/visibilityQueueFactory.go index 1799c9f6ab7..20488b5a207 100644 --- a/service/history/visibilityQueueFactory.go +++ b/service/history/visibilityQueueFactory.go @@ -59,28 +59,24 @@ type ( func NewVisibilityQueueFactory( params visibilityQueueFactoryParams, ) QueueFactory { - var hostScheduler queues.Scheduler - if params.Config.VisibilityProcessorEnablePriorityTaskScheduler() { - hostScheduler = queues.NewNamespacePriorityScheduler( - params.ClusterMetadata.GetCurrentClusterName(), - queues.NamespacePrioritySchedulerOptions{ - WorkerCount: params.Config.VisibilityProcessorSchedulerWorkerCount, - ActiveNamespaceWeights: params.Config.VisibilityProcessorSchedulerActiveRoundRobinWeights, - StandbyNamespaceWeights: params.Config.VisibilityProcessorSchedulerStandbyRoundRobinWeights, - EnableRateLimiter: params.Config.TaskSchedulerEnableRateLimiter, - MaxDispatchThrottleDuration: HostSchedulerMaxDispatchThrottleDuration, - }, - params.NamespaceRegistry, - params.SchedulerRateLimiter, - params.TimeSource, - params.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationVisibilityQueueProcessorScope)), - params.Logger, - ) - } return &visibilityQueueFactory{ visibilityQueueFactoryParams: params, QueueFactoryBase: QueueFactoryBase{ - HostScheduler: hostScheduler, + HostScheduler: queues.NewNamespacePriorityScheduler( + params.ClusterMetadata.GetCurrentClusterName(), + queues.NamespacePrioritySchedulerOptions{ + WorkerCount: params.Config.VisibilityProcessorSchedulerWorkerCount, + ActiveNamespaceWeights: params.Config.VisibilityProcessorSchedulerActiveRoundRobinWeights, + StandbyNamespaceWeights: params.Config.VisibilityProcessorSchedulerStandbyRoundRobinWeights, + EnableRateLimiter: params.Config.TaskSchedulerEnableRateLimiter, + MaxDispatchThrottleDuration: HostSchedulerMaxDispatchThrottleDuration, + }, + params.NamespaceRegistry, + params.SchedulerRateLimiter, + params.TimeSource, + params.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationVisibilityQueueProcessorScope)), + params.Logger, + ), HostPriorityAssigner: queues.NewPriorityAssigner(), HostRateLimiter: NewQueueHostRateLimiter( params.Config.VisibilityProcessorMaxPollHostRPS, @@ -103,58 +99,45 @@ func (f *visibilityQueueFactory) CreateQueue( shard shard.Context, workflowCache wcache.Cache, ) queues.Queue { - if f.HostScheduler != nil && f.Config.VisibilityProcessorEnableMultiCursor() { - logger := log.With(shard.GetLogger(), tag.ComponentVisibilityQueue) - - executor := newVisibilityQueueTaskExecutor( - shard, - workflowCache, - f.VisibilityMgr, - logger, - f.MetricsHandler, - f.Config.VisibilityProcessorEnsureCloseBeforeDelete, - f.Config.VisibilityProcessorEnableCloseWorkflowCleanup, - ) - - return queues.NewImmediateQueue( - shard, - tasks.CategoryVisibility, - f.HostScheduler, - f.HostPriorityAssigner, - executor, - &queues.Options{ - ReaderOptions: queues.ReaderOptions{ - BatchSize: f.Config.VisibilityTaskBatchSize, - MaxPendingTasksCount: f.Config.QueuePendingTaskMaxCount, - PollBackoffInterval: f.Config.VisibilityProcessorPollBackoffInterval, - }, - MonitorOptions: queues.MonitorOptions{ - PendingTasksCriticalCount: f.Config.QueuePendingTaskCriticalCount, - ReaderStuckCriticalAttempts: f.Config.QueueReaderStuckCriticalAttempts, - SliceCountCriticalThreshold: f.Config.QueueCriticalSlicesCount, - }, - MaxPollRPS: f.Config.VisibilityProcessorMaxPollRPS, - MaxPollInterval: f.Config.VisibilityProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: f.Config.VisibilityProcessorMaxPollIntervalJitterCoefficient, - CheckpointInterval: f.Config.VisibilityProcessorUpdateAckInterval, - CheckpointIntervalJitterCoefficient: f.Config.VisibilityProcessorUpdateAckIntervalJitterCoefficient, - MaxReaderCount: f.Config.QueueMaxReaderCount, - TaskMaxRetryCount: f.Config.VisibilityTaskMaxRetryCount, - }, - f.HostReaderRateLimiter, - logger, - f.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationVisibilityQueueProcessorScope)), - ) - } + logger := log.With(shard.GetLogger(), tag.ComponentVisibilityQueue) - return newVisibilityQueueProcessor( + executor := newVisibilityQueueTaskExecutor( shard, workflowCache, - f.HostScheduler, - f.HostPriorityAssigner, f.VisibilityMgr, + logger, f.MetricsHandler, - f.HostRateLimiter, - f.SchedulerRateLimiter, + f.Config.VisibilityProcessorEnsureCloseBeforeDelete, + f.Config.VisibilityProcessorEnableCloseWorkflowCleanup, + ) + + return queues.NewImmediateQueue( + shard, + tasks.CategoryVisibility, + f.HostScheduler, + f.HostPriorityAssigner, + executor, + &queues.Options{ + ReaderOptions: queues.ReaderOptions{ + BatchSize: f.Config.VisibilityTaskBatchSize, + MaxPendingTasksCount: f.Config.QueuePendingTaskMaxCount, + PollBackoffInterval: f.Config.VisibilityProcessorPollBackoffInterval, + }, + MonitorOptions: queues.MonitorOptions{ + PendingTasksCriticalCount: f.Config.QueuePendingTaskCriticalCount, + ReaderStuckCriticalAttempts: f.Config.QueueReaderStuckCriticalAttempts, + SliceCountCriticalThreshold: f.Config.QueueCriticalSlicesCount, + }, + MaxPollRPS: f.Config.VisibilityProcessorMaxPollRPS, + MaxPollInterval: f.Config.VisibilityProcessorMaxPollInterval, + MaxPollIntervalJitterCoefficient: f.Config.VisibilityProcessorMaxPollIntervalJitterCoefficient, + CheckpointInterval: f.Config.VisibilityProcessorUpdateAckInterval, + CheckpointIntervalJitterCoefficient: f.Config.VisibilityProcessorUpdateAckIntervalJitterCoefficient, + MaxReaderCount: f.Config.QueueMaxReaderCount, + TaskMaxRetryCount: f.Config.VisibilityTaskMaxRetryCount, + }, + f.HostReaderRateLimiter, + logger, + f.MetricsHandler.WithTags(metrics.OperationTag(metrics.OperationVisibilityQueueProcessorScope)), ) } diff --git a/service/history/visibilityQueueProcessor.go b/service/history/visibilityQueueProcessor.go deleted file mode 100644 index 45ab93cc9e4..00000000000 --- a/service/history/visibilityQueueProcessor.go +++ /dev/null @@ -1,375 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "sync/atomic" - "time" - - "go.temporal.io/server/common" - "go.temporal.io/server/common/backoff" - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/metrics" - "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/persistence/visibility/manager" - "go.temporal.io/server/common/quotas" - "go.temporal.io/server/service/history/configs" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/tasks" - wcache "go.temporal.io/server/service/history/workflow/cache" -) - -type ( - updateVisibilityAckLevel func(ackLevel int64) error - visibilityQueueShutdown func() error - - visibilityQueueProcessorImpl struct { - // from transferQueueActiveProcessorImpl (transferQueueProcessorImpl.activeTaskProcessor) - *queueProcessorBase - queueAckMgr - shard shard.Context - options *QueueProcessorOptions - executionManager persistence.ExecutionManager - maxReadLevel maxReadLevel - updateVisibilityAckLevel updateVisibilityAckLevel - visibilityQueueShutdown visibilityQueueShutdown - visibilityTaskFilter taskFilter - ownedScheduler queues.Scheduler // this is the scheduler owned by this visibility queue processor - logger log.Logger - metricHandler metrics.MetricsHandler - - // from transferQueueProcessorImpl - config *configs.Config - ackLevel int64 - - isStarted int32 - isStopped int32 - shutdownChan chan struct{} - } -) - -func newVisibilityQueueProcessor( - shard shard.Context, - workflowCache wcache.Cache, - scheduler queues.Scheduler, - priorityAssigner queues.PriorityAssigner, - visibilityMgr manager.VisibilityManager, - metricProvider metrics.MetricsHandler, - hostRateLimiter quotas.RateLimiter, - schedulerRateLimiter queues.SchedulerRateLimiter, -) queues.Queue { - - config := shard.GetConfig() - logger := log.With(shard.GetLogger(), tag.ComponentVisibilityQueue) - metricHandler := shard.GetMetricsHandler() - - options := &QueueProcessorOptions{ - BatchSize: config.VisibilityTaskBatchSize, - MaxPollInterval: config.VisibilityProcessorMaxPollInterval, - MaxPollIntervalJitterCoefficient: config.VisibilityProcessorMaxPollIntervalJitterCoefficient, - UpdateAckInterval: config.VisibilityProcessorUpdateAckInterval, - UpdateAckIntervalJitterCoefficient: config.VisibilityProcessorUpdateAckIntervalJitterCoefficient, - MaxReschdulerSize: config.VisibilityProcessorMaxReschedulerSize, - PollBackoffInterval: config.VisibilityProcessorPollBackoffInterval, - Operation: metrics.VisibilityQueueProcessorScope, - } - visibilityTaskFilter := func(taskInfo tasks.Task) bool { - return true - } - maxReadLevel := func() int64 { - return shard.GetImmediateQueueExclusiveHighReadWatermark().TaskID - } - updateVisibilityAckLevel := func(ackLevel int64) error { - return shard.UpdateQueueAckLevel(tasks.CategoryVisibility, tasks.NewImmediateKey(ackLevel)) - } - - visibilityQueueShutdown := func() error { - return nil - } - - ackLevel := shard.GetQueueAckLevel(tasks.CategoryVisibility).TaskID - retProcessor := &visibilityQueueProcessorImpl{ - shard: shard, - options: options, - maxReadLevel: maxReadLevel, - updateVisibilityAckLevel: updateVisibilityAckLevel, - visibilityQueueShutdown: visibilityQueueShutdown, - visibilityTaskFilter: visibilityTaskFilter, - logger: logger, - metricHandler: metricHandler, - - config: config, - ackLevel: ackLevel, - shutdownChan: make(chan struct{}), - - queueAckMgr: nil, // is set bellow - queueProcessorBase: nil, // is set bellow - executionManager: shard.GetExecutionManager(), - } - - taskExecutor := newVisibilityQueueTaskExecutor( - shard, - workflowCache, - visibilityMgr, - logger, - metricProvider, - config.VisibilityProcessorEnsureCloseBeforeDelete, - config.VisibilityProcessorEnableCloseWorkflowCleanup, - ) - - if scheduler == nil { - scheduler = newVisibilityTaskShardScheduler(shard, schedulerRateLimiter, logger) - retProcessor.ownedScheduler = scheduler - } - - rescheduler := queues.NewRescheduler( - scheduler, - shard.GetTimeSource(), - logger, - metricProvider.WithTags(metrics.OperationTag(metrics.OperationVisibilityQueueProcessorScope)), - ) - - queueAckMgr := newQueueAckMgr( - shard, - options, - retProcessor, - ackLevel, - logger, - func(t tasks.Task) queues.Executable { - return queues.NewExecutable( - queues.DefaultReaderId, - t, - visibilityTaskFilter, - taskExecutor, - scheduler, - rescheduler, - priorityAssigner, - shard.GetTimeSource(), - shard.GetNamespaceRegistry(), - logger, - metricProvider, - shard.GetConfig().VisibilityTaskMaxRetryCount, - shard.GetConfig().NamespaceCacheRefreshInterval, - ) - }, - ) - - queueProcessorBase := newQueueProcessorBase( - shard.GetClusterMetadata().GetCurrentClusterName(), - shard, - options, - retProcessor, - queueAckMgr, - workflowCache, - scheduler, - rescheduler, - newQueueProcessorRateLimiter( - hostRateLimiter, - config.VisibilityProcessorMaxPollRPS, - ), - logger, - ) - retProcessor.queueAckMgr = queueAckMgr - retProcessor.queueProcessorBase = queueProcessorBase - - return retProcessor -} - -// visibilityQueueProcessor implementation -func (t *visibilityQueueProcessorImpl) Start() { - if !atomic.CompareAndSwapInt32(&t.isStarted, 0, 1) { - return - } - if t.ownedScheduler != nil { - t.ownedScheduler.Start() - } - t.queueProcessorBase.Start() - go t.completeTaskLoop() -} - -func (t *visibilityQueueProcessorImpl) Stop() { - if !atomic.CompareAndSwapInt32(&t.isStopped, 0, 1) { - return - } - t.queueProcessorBase.Stop() - if t.ownedScheduler != nil { - t.ownedScheduler.Stop() - } - close(t.shutdownChan) -} - -// NotifyNewTasks - Notify the processor about the new visibility task arrival. -// This should be called each time new visibility task arrives, otherwise tasks maybe delayed. -func (t *visibilityQueueProcessorImpl) NotifyNewTasks( - _ string, - visibilityTasks []tasks.Task, -) { - if len(visibilityTasks) != 0 { - t.notifyNewTask() - } -} - -func (t *visibilityQueueProcessorImpl) FailoverNamespace( - namespaceIDs map[string]struct{}, -) { - // no-op -} - -func (t *visibilityQueueProcessorImpl) LockTaskProcessing() { - // no-op -} - -func (t *visibilityQueueProcessorImpl) UnlockTaskProcessing() { - // no-op -} - -func (t *visibilityQueueProcessorImpl) Category() tasks.Category { - return tasks.CategoryVisibility -} - -func (t *visibilityQueueProcessorImpl) completeTaskLoop() { - timer := time.NewTimer(t.config.VisibilityProcessorCompleteTaskInterval()) - defer timer.Stop() - - completeTaskRetryPolicy := common.CreateCompleteTaskRetryPolicy() - - for { - select { - case <-t.shutdownChan: - // before shutdown, make sure the ack level is up to date - if err := t.completeTask(); err != nil { - t.logger.Error("Failed to complete visibility task", tag.Error(err)) - } - return - case <-timer.C: - // TODO: We should have a better approach to handle shard and its component lifecycle - _ = backoff.ThrottleRetry(func() error { - err := t.completeTask() - if err != nil { - t.logger.Error("Failed to complete visibility task", tag.Error(err)) - } - return err - }, completeTaskRetryPolicy, func(err error) bool { - select { - case <-t.shutdownChan: - return false - default: - } - return !shard.IsShardOwnershipLostError(err) - }) - - timer.Reset(t.config.VisibilityProcessorCompleteTaskInterval()) - } - } -} - -func (t *visibilityQueueProcessorImpl) completeTask() error { - lowerAckLevel := t.ackLevel - upperAckLevel := t.queueAckMgr.getQueueAckLevel() - - t.logger.Debug("Start completing visibility task", tag.AckLevel(lowerAckLevel), tag.AckLevel(upperAckLevel)) - if lowerAckLevel >= upperAckLevel { - return nil - } - - t.metricHandler.Counter(metrics.TaskBatchCompleteCounter.GetMetricName()).Record( - 1, - metrics.OperationTag(metrics.VisibilityQueueProcessorScope)) - - if lowerAckLevel < upperAckLevel { - ctx, cancel := newQueueIOContext() - defer cancel() - - err := t.shard.GetExecutionManager().RangeCompleteHistoryTasks(ctx, &persistence.RangeCompleteHistoryTasksRequest{ - ShardID: t.shard.GetShardID(), - TaskCategory: tasks.CategoryVisibility, - InclusiveMinTaskKey: tasks.NewImmediateKey(lowerAckLevel + 1), - ExclusiveMaxTaskKey: tasks.NewImmediateKey(upperAckLevel + 1), - }) - if err != nil { - return err - } - } - - t.ackLevel = upperAckLevel - - return t.shard.UpdateQueueAckLevel(tasks.CategoryVisibility, tasks.NewImmediateKey(upperAckLevel)) -} - -// queueProcessor interface -func (t *visibilityQueueProcessorImpl) notifyNewTask() { - t.queueProcessorBase.notifyNewTask() -} - -// processor interfaces -func (t *visibilityQueueProcessorImpl) readTasks( - readLevel int64, -) ([]tasks.Task, bool, error) { - ctx, cancel := newQueueIOContext() - defer cancel() - - response, err := t.executionManager.GetHistoryTasks(ctx, &persistence.GetHistoryTasksRequest{ - ShardID: t.shard.GetShardID(), - TaskCategory: tasks.CategoryVisibility, - InclusiveMinTaskKey: tasks.NewImmediateKey(readLevel + 1), - ExclusiveMaxTaskKey: tasks.NewImmediateKey(t.maxReadLevel()), - BatchSize: t.options.BatchSize(), - }) - if err != nil { - return nil, false, err - } - - return response.Tasks, len(response.NextPageToken) != 0, nil -} - -func (t *visibilityQueueProcessorImpl) updateAckLevel( - ackLevel int64, -) error { - return t.updateVisibilityAckLevel(ackLevel) -} - -func (t *visibilityQueueProcessorImpl) queueShutdown() error { - return t.visibilityQueueShutdown() -} - -func newVisibilityTaskShardScheduler( - shard shard.Context, - rateLimiter queues.SchedulerRateLimiter, - logger log.Logger, -) queues.Scheduler { - config := shard.GetConfig() - return queues.NewPriorityScheduler( - queues.PrioritySchedulerOptions{ - WorkerCount: config.VisibilityTaskWorkerCount, - EnableRateLimiter: config.TaskSchedulerEnableRateLimiter, - MaxDispatchThrottleDuration: ShardSchedulerMaxDispatchThrottleDuration, - }, - rateLimiter, - shard.GetTimeSource(), - logger, - ) -} From 503da0b386a827e981169cd851d742db16b5fd6a Mon Sep 17 00:00:00 2001 From: Yichao Yang Date: Thu, 15 Dec 2022 12:30:59 -0800 Subject: [PATCH 2/2] remove unused components --- service/history/historyEngineInterfaces.go | 82 ---- .../history/historyEngineInterfaces_mock.go | 446 ------------------ service/history/taskAllocator.go | 126 ----- 3 files changed, 654 deletions(-) delete mode 100644 service/history/historyEngineInterfaces.go delete mode 100644 service/history/historyEngineInterfaces_mock.go delete mode 100644 service/history/taskAllocator.go diff --git a/service/history/historyEngineInterfaces.go b/service/history/historyEngineInterfaces.go deleted file mode 100644 index ae27c89f4fe..00000000000 --- a/service/history/historyEngineInterfaces.go +++ /dev/null @@ -1,82 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -//go:generate mockgen -copyright_file ../../LICENSE -package $GOPACKAGE -source $GOFILE -destination historyEngineInterfaces_mock.go - -package history - -import ( - "context" - "time" - - replicationspb "go.temporal.io/server/api/replication/v1" - "go.temporal.io/server/common" - "go.temporal.io/server/service/history/queues" - "go.temporal.io/server/service/history/tasks" -) - -type ( - queueProcessor interface { - common.Daemon - notifyNewTask() - } - - // ReplicatorQueueProcessor is the interface for replicator queue processor - ReplicatorQueueProcessor interface { - queueProcessor - getTasks( - ctx context.Context, - pollingCluster string, - lastReadTaskID int64, - ) (*replicationspb.ReplicationMessages, error) - getTask( - ctx context.Context, - taskInfo *replicationspb.ReplicationTaskInfo, - ) (*replicationspb.ReplicationTask, error) - } - - queueAckMgr interface { - getFinishedChan() <-chan struct{} - readQueueTasks() ([]queues.Executable, bool, error) - getQueueAckLevel() int64 - getQueueReadLevel() int64 - updateQueueAckLevel() error - } - - processor interface { - readTasks(readLevel int64) ([]tasks.Task, bool, error) - updateAckLevel(taskID int64) error - queueShutdown() error - } - - timerQueueAckMgr interface { - getFinishedChan() <-chan struct{} - readTimerTasks() ([]queues.Executable, *time.Time, bool, error) - getAckLevel() tasks.Key - getReadLevel() tasks.Key - updateAckLevel() error - } - - taskExecutableInitializer func(tasks.Task) queues.Executable -) diff --git a/service/history/historyEngineInterfaces_mock.go b/service/history/historyEngineInterfaces_mock.go deleted file mode 100644 index 4c79869b135..00000000000 --- a/service/history/historyEngineInterfaces_mock.go +++ /dev/null @@ -1,446 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -// Code generated by MockGen. DO NOT EDIT. -// Source: historyEngineInterfaces.go - -// Package history is a generated GoMock package. -package history - -import ( - context "context" - reflect "reflect" - time "time" - - gomock "github.com/golang/mock/gomock" - repication "go.temporal.io/server/api/replication/v1" - queues "go.temporal.io/server/service/history/queues" - tasks "go.temporal.io/server/service/history/tasks" -) - -// MockqueueProcessor is a mock of queueProcessor interface. -type MockqueueProcessor struct { - ctrl *gomock.Controller - recorder *MockqueueProcessorMockRecorder -} - -// MockqueueProcessorMockRecorder is the mock recorder for MockqueueProcessor. -type MockqueueProcessorMockRecorder struct { - mock *MockqueueProcessor -} - -// NewMockqueueProcessor creates a new mock instance. -func NewMockqueueProcessor(ctrl *gomock.Controller) *MockqueueProcessor { - mock := &MockqueueProcessor{ctrl: ctrl} - mock.recorder = &MockqueueProcessorMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockqueueProcessor) EXPECT() *MockqueueProcessorMockRecorder { - return m.recorder -} - -// Start mocks base method. -func (m *MockqueueProcessor) Start() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Start") -} - -// Start indicates an expected call of Start. -func (mr *MockqueueProcessorMockRecorder) Start() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Start", reflect.TypeOf((*MockqueueProcessor)(nil).Start)) -} - -// Stop mocks base method. -func (m *MockqueueProcessor) Stop() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Stop") -} - -// Stop indicates an expected call of Stop. -func (mr *MockqueueProcessorMockRecorder) Stop() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockqueueProcessor)(nil).Stop)) -} - -// notifyNewTask mocks base method. -func (m *MockqueueProcessor) notifyNewTask() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "notifyNewTask") -} - -// notifyNewTask indicates an expected call of notifyNewTask. -func (mr *MockqueueProcessorMockRecorder) notifyNewTask() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "notifyNewTask", reflect.TypeOf((*MockqueueProcessor)(nil).notifyNewTask)) -} - -// MockReplicatorQueueProcessor is a mock of ReplicatorQueueProcessor interface. -type MockReplicatorQueueProcessor struct { - ctrl *gomock.Controller - recorder *MockReplicatorQueueProcessorMockRecorder -} - -// MockReplicatorQueueProcessorMockRecorder is the mock recorder for MockReplicatorQueueProcessor. -type MockReplicatorQueueProcessorMockRecorder struct { - mock *MockReplicatorQueueProcessor -} - -// NewMockReplicatorQueueProcessor creates a new mock instance. -func NewMockReplicatorQueueProcessor(ctrl *gomock.Controller) *MockReplicatorQueueProcessor { - mock := &MockReplicatorQueueProcessor{ctrl: ctrl} - mock.recorder = &MockReplicatorQueueProcessorMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockReplicatorQueueProcessor) EXPECT() *MockReplicatorQueueProcessorMockRecorder { - return m.recorder -} - -// Start mocks base method. -func (m *MockReplicatorQueueProcessor) Start() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Start") -} - -// Start indicates an expected call of Start. -func (mr *MockReplicatorQueueProcessorMockRecorder) Start() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Start", reflect.TypeOf((*MockReplicatorQueueProcessor)(nil).Start)) -} - -// Stop mocks base method. -func (m *MockReplicatorQueueProcessor) Stop() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Stop") -} - -// Stop indicates an expected call of Stop. -func (mr *MockReplicatorQueueProcessorMockRecorder) Stop() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockReplicatorQueueProcessor)(nil).Stop)) -} - -// getTask mocks base method. -func (m *MockReplicatorQueueProcessor) getTask(ctx context.Context, taskInfo *repication.ReplicationTaskInfo) (*repication.ReplicationTask, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getTask", ctx, taskInfo) - ret0, _ := ret[0].(*repication.ReplicationTask) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// getTask indicates an expected call of getTask. -func (mr *MockReplicatorQueueProcessorMockRecorder) getTask(ctx, taskInfo interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getTask", reflect.TypeOf((*MockReplicatorQueueProcessor)(nil).getTask), ctx, taskInfo) -} - -// getTasks mocks base method. -func (m *MockReplicatorQueueProcessor) getTasks(ctx context.Context, pollingCluster string, lastReadTaskID int64) (*repication.ReplicationMessages, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getTasks", ctx, pollingCluster, lastReadTaskID) - ret0, _ := ret[0].(*repication.ReplicationMessages) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// getTasks indicates an expected call of getTasks. -func (mr *MockReplicatorQueueProcessorMockRecorder) getTasks(ctx, pollingCluster, lastReadTaskID interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getTasks", reflect.TypeOf((*MockReplicatorQueueProcessor)(nil).getTasks), ctx, pollingCluster, lastReadTaskID) -} - -// notifyNewTask mocks base method. -func (m *MockReplicatorQueueProcessor) notifyNewTask() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "notifyNewTask") -} - -// notifyNewTask indicates an expected call of notifyNewTask. -func (mr *MockReplicatorQueueProcessorMockRecorder) notifyNewTask() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "notifyNewTask", reflect.TypeOf((*MockReplicatorQueueProcessor)(nil).notifyNewTask)) -} - -// MockqueueAckMgr is a mock of queueAckMgr interface. -type MockqueueAckMgr struct { - ctrl *gomock.Controller - recorder *MockqueueAckMgrMockRecorder -} - -// MockqueueAckMgrMockRecorder is the mock recorder for MockqueueAckMgr. -type MockqueueAckMgrMockRecorder struct { - mock *MockqueueAckMgr -} - -// NewMockqueueAckMgr creates a new mock instance. -func NewMockqueueAckMgr(ctrl *gomock.Controller) *MockqueueAckMgr { - mock := &MockqueueAckMgr{ctrl: ctrl} - mock.recorder = &MockqueueAckMgrMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockqueueAckMgr) EXPECT() *MockqueueAckMgrMockRecorder { - return m.recorder -} - -// getFinishedChan mocks base method. -func (m *MockqueueAckMgr) getFinishedChan() <-chan struct{} { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getFinishedChan") - ret0, _ := ret[0].(<-chan struct{}) - return ret0 -} - -// getFinishedChan indicates an expected call of getFinishedChan. -func (mr *MockqueueAckMgrMockRecorder) getFinishedChan() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getFinishedChan", reflect.TypeOf((*MockqueueAckMgr)(nil).getFinishedChan)) -} - -// getQueueAckLevel mocks base method. -func (m *MockqueueAckMgr) getQueueAckLevel() int64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getQueueAckLevel") - ret0, _ := ret[0].(int64) - return ret0 -} - -// getQueueAckLevel indicates an expected call of getQueueAckLevel. -func (mr *MockqueueAckMgrMockRecorder) getQueueAckLevel() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getQueueAckLevel", reflect.TypeOf((*MockqueueAckMgr)(nil).getQueueAckLevel)) -} - -// getQueueReadLevel mocks base method. -func (m *MockqueueAckMgr) getQueueReadLevel() int64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getQueueReadLevel") - ret0, _ := ret[0].(int64) - return ret0 -} - -// getQueueReadLevel indicates an expected call of getQueueReadLevel. -func (mr *MockqueueAckMgrMockRecorder) getQueueReadLevel() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getQueueReadLevel", reflect.TypeOf((*MockqueueAckMgr)(nil).getQueueReadLevel)) -} - -// readQueueTasks mocks base method. -func (m *MockqueueAckMgr) readQueueTasks() ([]queues.Executable, bool, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "readQueueTasks") - ret0, _ := ret[0].([]queues.Executable) - ret1, _ := ret[1].(bool) - ret2, _ := ret[2].(error) - return ret0, ret1, ret2 -} - -// readQueueTasks indicates an expected call of readQueueTasks. -func (mr *MockqueueAckMgrMockRecorder) readQueueTasks() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "readQueueTasks", reflect.TypeOf((*MockqueueAckMgr)(nil).readQueueTasks)) -} - -// updateQueueAckLevel mocks base method. -func (m *MockqueueAckMgr) updateQueueAckLevel() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "updateQueueAckLevel") - ret0, _ := ret[0].(error) - return ret0 -} - -// updateQueueAckLevel indicates an expected call of updateQueueAckLevel. -func (mr *MockqueueAckMgrMockRecorder) updateQueueAckLevel() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "updateQueueAckLevel", reflect.TypeOf((*MockqueueAckMgr)(nil).updateQueueAckLevel)) -} - -// Mockprocessor is a mock of processor interface. -type Mockprocessor struct { - ctrl *gomock.Controller - recorder *MockprocessorMockRecorder -} - -// MockprocessorMockRecorder is the mock recorder for Mockprocessor. -type MockprocessorMockRecorder struct { - mock *Mockprocessor -} - -// NewMockprocessor creates a new mock instance. -func NewMockprocessor(ctrl *gomock.Controller) *Mockprocessor { - mock := &Mockprocessor{ctrl: ctrl} - mock.recorder = &MockprocessorMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *Mockprocessor) EXPECT() *MockprocessorMockRecorder { - return m.recorder -} - -// queueShutdown mocks base method. -func (m *Mockprocessor) queueShutdown() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "queueShutdown") - ret0, _ := ret[0].(error) - return ret0 -} - -// queueShutdown indicates an expected call of queueShutdown. -func (mr *MockprocessorMockRecorder) queueShutdown() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "queueShutdown", reflect.TypeOf((*Mockprocessor)(nil).queueShutdown)) -} - -// readTasks mocks base method. -func (m *Mockprocessor) readTasks(readLevel int64) ([]tasks.Task, bool, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "readTasks", readLevel) - ret0, _ := ret[0].([]tasks.Task) - ret1, _ := ret[1].(bool) - ret2, _ := ret[2].(error) - return ret0, ret1, ret2 -} - -// readTasks indicates an expected call of readTasks. -func (mr *MockprocessorMockRecorder) readTasks(readLevel interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "readTasks", reflect.TypeOf((*Mockprocessor)(nil).readTasks), readLevel) -} - -// updateAckLevel mocks base method. -func (m *Mockprocessor) updateAckLevel(taskID int64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "updateAckLevel", taskID) - ret0, _ := ret[0].(error) - return ret0 -} - -// updateAckLevel indicates an expected call of updateAckLevel. -func (mr *MockprocessorMockRecorder) updateAckLevel(taskID interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "updateAckLevel", reflect.TypeOf((*Mockprocessor)(nil).updateAckLevel), taskID) -} - -// MocktimerQueueAckMgr is a mock of timerQueueAckMgr interface. -type MocktimerQueueAckMgr struct { - ctrl *gomock.Controller - recorder *MocktimerQueueAckMgrMockRecorder -} - -// MocktimerQueueAckMgrMockRecorder is the mock recorder for MocktimerQueueAckMgr. -type MocktimerQueueAckMgrMockRecorder struct { - mock *MocktimerQueueAckMgr -} - -// NewMocktimerQueueAckMgr creates a new mock instance. -func NewMocktimerQueueAckMgr(ctrl *gomock.Controller) *MocktimerQueueAckMgr { - mock := &MocktimerQueueAckMgr{ctrl: ctrl} - mock.recorder = &MocktimerQueueAckMgrMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MocktimerQueueAckMgr) EXPECT() *MocktimerQueueAckMgrMockRecorder { - return m.recorder -} - -// getAckLevel mocks base method. -func (m *MocktimerQueueAckMgr) getAckLevel() tasks.Key { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getAckLevel") - ret0, _ := ret[0].(tasks.Key) - return ret0 -} - -// getAckLevel indicates an expected call of getAckLevel. -func (mr *MocktimerQueueAckMgrMockRecorder) getAckLevel() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getAckLevel", reflect.TypeOf((*MocktimerQueueAckMgr)(nil).getAckLevel)) -} - -// getFinishedChan mocks base method. -func (m *MocktimerQueueAckMgr) getFinishedChan() <-chan struct{} { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getFinishedChan") - ret0, _ := ret[0].(<-chan struct{}) - return ret0 -} - -// getFinishedChan indicates an expected call of getFinishedChan. -func (mr *MocktimerQueueAckMgrMockRecorder) getFinishedChan() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getFinishedChan", reflect.TypeOf((*MocktimerQueueAckMgr)(nil).getFinishedChan)) -} - -// getReadLevel mocks base method. -func (m *MocktimerQueueAckMgr) getReadLevel() tasks.Key { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "getReadLevel") - ret0, _ := ret[0].(tasks.Key) - return ret0 -} - -// getReadLevel indicates an expected call of getReadLevel. -func (mr *MocktimerQueueAckMgrMockRecorder) getReadLevel() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "getReadLevel", reflect.TypeOf((*MocktimerQueueAckMgr)(nil).getReadLevel)) -} - -// readTimerTasks mocks base method. -func (m *MocktimerQueueAckMgr) readTimerTasks() ([]queues.Executable, *time.Time, bool, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "readTimerTasks") - ret0, _ := ret[0].([]queues.Executable) - ret1, _ := ret[1].(*time.Time) - ret2, _ := ret[2].(bool) - ret3, _ := ret[3].(error) - return ret0, ret1, ret2, ret3 -} - -// readTimerTasks indicates an expected call of readTimerTasks. -func (mr *MocktimerQueueAckMgrMockRecorder) readTimerTasks() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "readTimerTasks", reflect.TypeOf((*MocktimerQueueAckMgr)(nil).readTimerTasks)) -} - -// updateAckLevel mocks base method. -func (m *MocktimerQueueAckMgr) updateAckLevel() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "updateAckLevel") - ret0, _ := ret[0].(error) - return ret0 -} - -// updateAckLevel indicates an expected call of updateAckLevel. -func (mr *MocktimerQueueAckMgrMockRecorder) updateAckLevel() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "updateAckLevel", reflect.TypeOf((*MocktimerQueueAckMgr)(nil).updateAckLevel)) -} diff --git a/service/history/taskAllocator.go b/service/history/taskAllocator.go deleted file mode 100644 index 75cd7219c9f..00000000000 --- a/service/history/taskAllocator.go +++ /dev/null @@ -1,126 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package history - -import ( - "sync" - - "go.temporal.io/server/common/log" - "go.temporal.io/server/common/log/tag" - "go.temporal.io/server/common/namespace" - "go.temporal.io/server/service/history/shard" -) - -type ( - taskAllocator interface { - verifyActiveTask(taskNamespaceID namespace.ID, task interface{}) bool - verifyFailoverActiveTask(targetNamespaceIDs map[string]struct{}, taskNamespaceID namespace.ID, task interface{}) bool - verifyStandbyTask(standbyCluster string, taskNamespaceID namespace.ID, task interface{}) bool - lock() - unlock() - } - - taskAllocatorImpl struct { - currentClusterName string - shard shard.Context - namespaceRegistry namespace.Registry - logger log.Logger - - locker sync.RWMutex - } -) - -// newTaskAllocator create a new task allocator -func newTaskAllocator(shard shard.Context) taskAllocator { - return &taskAllocatorImpl{ - currentClusterName: shard.GetClusterMetadata().GetCurrentClusterName(), - shard: shard, - namespaceRegistry: shard.GetNamespaceRegistry(), - logger: shard.GetLogger(), - } -} - -// verifyActiveTask, will return true if task activeness check is successful -func (t *taskAllocatorImpl) verifyActiveTask(taskNamespaceID namespace.ID, task interface{}) bool { - t.locker.RLock() - defer t.locker.RUnlock() - - namespaceEntry, err := t.namespaceRegistry.GetNamespaceByID(taskNamespaceID) - if err != nil { - t.logger.Debug("Unable to find namespace, process task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return true - } - if namespaceEntry.IsGlobalNamespace() && t.currentClusterName != namespaceEntry.ActiveClusterName() { - // timer task does not belong to cluster name - t.logger.Debug("Namespace is not active, skip task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return false - } - t.logger.Debug("Namespace is active, process task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return true -} - -// verifyFailoverActiveTask, will return true if task activeness check is successful -func (t *taskAllocatorImpl) verifyFailoverActiveTask(targetNamespaceIDs map[string]struct{}, taskNamespaceID namespace.ID, task interface{}) bool { - _, ok := targetNamespaceIDs[taskNamespaceID.String()] - if ok { - t.logger.Debug("Failover Namespace is active, process task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return true - } - t.logger.Debug("Failover Namespace is not active, skip task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return false -} - -// verifyStandbyTask, will return true if task standbyness check is successful -func (t *taskAllocatorImpl) verifyStandbyTask(standbyCluster string, taskNamespaceID namespace.ID, task interface{}) bool { - t.locker.RLock() - defer t.locker.RUnlock() - - namespaceEntry, err := t.namespaceRegistry.GetNamespaceByID(taskNamespaceID) - if err != nil { - t.logger.Debug("Unable to find namespace, skip task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return false - } - if !namespaceEntry.IsGlobalNamespace() { - // non global namespace, timer task does not belong here - t.logger.Debug("Namespace is not global, skip task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return false - } else if namespaceEntry.IsGlobalNamespace() && namespaceEntry.ActiveClusterName() != standbyCluster { - // timer task does not belong here - t.logger.Debug("Namespace is not standby, skip task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return false - } - t.logger.Debug("Namespace is standby, process task.", tag.WorkflowNamespaceID(taskNamespaceID.String()), tag.Value(task)) - return true -} - -// lock block all task allocation -func (t *taskAllocatorImpl) lock() { - t.locker.Lock() -} - -// unlock resume the task allocator -func (t *taskAllocatorImpl) unlock() { - t.locker.Unlock() -}