Skip to content

Commit a026002

Browse files
authored
Remove task max retry count config (#3771)
1 parent 7215a32 commit a026002

18 files changed

+7
-43
lines changed

common/dynamicconfig/constants.go

-10
Original file line numberDiff line numberDiff line change
@@ -387,8 +387,6 @@ const (
387387

388388
// TimerTaskBatchSize is batch size for timer processor to process tasks
389389
TimerTaskBatchSize = "history.timerTaskBatchSize"
390-
// TimerTaskMaxRetryCount is max retry count for timer processor
391-
TimerTaskMaxRetryCount = "history.timerTaskMaxRetryCount"
392390
// TimerProcessorSchedulerWorkerCount is the number of workers in the host level task scheduler for timer processor
393391
TimerProcessorSchedulerWorkerCount = "history.timerProcessorSchedulerWorkerCount"
394392
// TimerProcessorSchedulerActiveRoundRobinWeights is the priority round robin weights used by timer task scheduler for active namespaces
@@ -430,8 +428,6 @@ const (
430428
TransferProcessorMaxPollRPS = "history.transferProcessorMaxPollRPS"
431429
// TransferProcessorMaxPollHostRPS is max poll rate per second for all transferQueueProcessor on a host
432430
TransferProcessorMaxPollHostRPS = "history.transferProcessorMaxPollHostRPS"
433-
// TransferTaskMaxRetryCount is max times of retry for transferQueueProcessor
434-
TransferTaskMaxRetryCount = "history.transferTaskMaxRetryCount"
435431
// TransferProcessorSchedulerWorkerCount is the number of workers in the host level task scheduler for transferQueueProcessor
436432
TransferProcessorSchedulerWorkerCount = "history.transferProcessorSchedulerWorkerCount"
437433
// TransferProcessorSchedulerActiveRoundRobinWeights is the priority round robin weights used by transfer task scheduler for active namespaces
@@ -463,8 +459,6 @@ const (
463459
VisibilityProcessorMaxPollRPS = "history.visibilityProcessorMaxPollRPS"
464460
// VisibilityProcessorMaxPollHostRPS is max poll rate per second for all visibilityQueueProcessor on a host
465461
VisibilityProcessorMaxPollHostRPS = "history.visibilityProcessorMaxPollHostRPS"
466-
// VisibilityTaskMaxRetryCount is max times of retry for visibilityQueueProcessor
467-
VisibilityTaskMaxRetryCount = "history.visibilityTaskMaxRetryCount"
468462
// VisibilityProcessorSchedulerWorkerCount is the number of workers in the host level task scheduler for visibilityQueueProcessor
469463
VisibilityProcessorSchedulerWorkerCount = "history.visibilityProcessorSchedulerWorkerCount"
470464
// VisibilityProcessorSchedulerActiveRoundRobinWeights is the priority round robin weights by visibility task scheduler for active namespaces
@@ -514,8 +508,6 @@ const (
514508
ArchivalProcessorPollBackoffInterval = "history.archivalProcessorPollBackoffInterval"
515509
// ArchivalProcessorArchiveDelay is the delay before archivalQueueProcessor starts to process archival tasks
516510
ArchivalProcessorArchiveDelay = "history.archivalProcessorArchiveDelay"
517-
// ArchivalProcessorRetryWarningLimit is the number of times an archival task may be retried before we log a warning
518-
ArchivalProcessorRetryWarningLimit = "history.archivalProcessorRetryLimitWarning"
519511
// ArchivalBackendMaxRPS is the maximum rate of requests per second to the archival backend
520512
ArchivalBackendMaxRPS = "history.archivalBackendMaxRPS"
521513
// DurableArchivalEnabled is the flag to enable durable archival
@@ -527,8 +519,6 @@ const (
527519
ReplicatorMaxSkipTaskCount = "history.replicatorMaxSkipTaskCount"
528520
// ReplicatorTaskWorkerCount is number of worker for ReplicatorProcessor
529521
ReplicatorTaskWorkerCount = "history.replicatorTaskWorkerCount"
530-
// ReplicatorTaskMaxRetryCount is max times of retry for ReplicatorProcessor
531-
ReplicatorTaskMaxRetryCount = "history.replicatorTaskMaxRetryCount"
532522
// ReplicatorProcessorMaxPollRPS is max poll rate per second for ReplicatorProcessor
533523
ReplicatorProcessorMaxPollRPS = "history.replicatorProcessorMaxPollRPS"
534524
// ReplicatorProcessorMaxPollInterval is max poll interval for ReplicatorProcessor

service/history/archival_queue_factory.go

-1
Original file line numberDiff line numberDiff line change
@@ -180,7 +180,6 @@ func (f *archivalQueueFactory) newScheduledQueue(shard shard.Context, executor q
180180
CheckpointInterval: f.Config.ArchivalProcessorUpdateAckInterval,
181181
CheckpointIntervalJitterCoefficient: f.Config.ArchivalProcessorUpdateAckIntervalJitterCoefficient,
182182
MaxReaderCount: f.Config.QueueMaxReaderCount,
183-
TaskMaxRetryCount: f.Config.ArchivalProcessorRetryWarningLimit,
184183
},
185184
f.HostReaderRateLimiter,
186185
logger,

service/history/archival_queue_task_executor_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -511,7 +511,6 @@ func TestArchivalQueueTaskExecutor(t *testing.T) {
511511
namespaceRegistry,
512512
nil,
513513
metrics.NoopMetricsHandler,
514-
nil,
515514
)
516515
err := executable.Execute()
517516
if len(p.ExpectedErrorSubstrings) > 0 {

service/history/configs/config.go

-10
Original file line numberDiff line numberDiff line change
@@ -98,7 +98,6 @@ type Config struct {
9898
// TimerQueueProcessor settings
9999
TimerTaskHighPriorityRPS dynamicconfig.IntPropertyFnWithNamespaceFilter
100100
TimerTaskBatchSize dynamicconfig.IntPropertyFn
101-
TimerTaskMaxRetryCount dynamicconfig.IntPropertyFn
102101
TimerProcessorSchedulerWorkerCount dynamicconfig.IntPropertyFn
103102
TimerProcessorSchedulerActiveRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter
104103
TimerProcessorSchedulerStandbyRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter
@@ -119,7 +118,6 @@ type Config struct {
119118
// TransferQueueProcessor settings
120119
TransferTaskHighPriorityRPS dynamicconfig.IntPropertyFnWithNamespaceFilter
121120
TransferTaskBatchSize dynamicconfig.IntPropertyFn
122-
TransferTaskMaxRetryCount dynamicconfig.IntPropertyFn
123121
TransferProcessorSchedulerWorkerCount dynamicconfig.IntPropertyFn
124122
TransferProcessorSchedulerActiveRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter
125123
TransferProcessorSchedulerStandbyRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter
@@ -139,7 +137,6 @@ type Config struct {
139137
// TODO: clean up unused replicator settings
140138
ReplicatorTaskBatchSize dynamicconfig.IntPropertyFn
141139
ReplicatorTaskWorkerCount dynamicconfig.IntPropertyFn
142-
ReplicatorTaskMaxRetryCount dynamicconfig.IntPropertyFn
143140
ReplicatorProcessorMaxPollRPS dynamicconfig.IntPropertyFn
144141
ReplicatorProcessorMaxPollInterval dynamicconfig.DurationPropertyFn
145142
ReplicatorProcessorMaxPollIntervalJitterCoefficient dynamicconfig.FloatPropertyFn
@@ -249,7 +246,6 @@ type Config struct {
249246
// VisibilityQueueProcessor settings
250247
VisibilityTaskHighPriorityRPS dynamicconfig.IntPropertyFnWithNamespaceFilter
251248
VisibilityTaskBatchSize dynamicconfig.IntPropertyFn
252-
VisibilityTaskMaxRetryCount dynamicconfig.IntPropertyFn
253249
VisibilityProcessorSchedulerWorkerCount dynamicconfig.IntPropertyFn
254250
VisibilityProcessorSchedulerActiveRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter
255251
VisibilityProcessorSchedulerStandbyRoundRobinWeights dynamicconfig.MapPropertyFnWithNamespaceFilter
@@ -290,7 +286,6 @@ type Config struct {
290286
ArchivalProcessorUpdateAckInterval dynamicconfig.DurationPropertyFn
291287
ArchivalProcessorUpdateAckIntervalJitterCoefficient dynamicconfig.FloatPropertyFn
292288
ArchivalProcessorArchiveDelay dynamicconfig.DurationPropertyFn
293-
ArchivalProcessorRetryWarningLimit dynamicconfig.IntPropertyFn
294289
ArchivalBackendMaxRPS dynamicconfig.FloatPropertyFn
295290
}
296291

@@ -350,7 +345,6 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis
350345
TaskSchedulerNamespaceMaxQPS: dc.GetIntPropertyFilteredByNamespace(dynamicconfig.TaskSchedulerNamespaceMaxQPS, 0),
351346

352347
TimerTaskBatchSize: dc.GetIntProperty(dynamicconfig.TimerTaskBatchSize, 100),
353-
TimerTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.TimerTaskMaxRetryCount, 20),
354348
TimerProcessorSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.TimerProcessorSchedulerWorkerCount, 512),
355349
TimerProcessorSchedulerActiveRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TimerProcessorSchedulerActiveRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultActiveTaskPriorityWeight)),
356350
TimerProcessorSchedulerStandbyRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TimerProcessorSchedulerStandbyRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultStandbyTaskPriorityWeight)),
@@ -369,7 +363,6 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis
369363
RetentionTimerJitterDuration: dc.GetDurationProperty(dynamicconfig.RetentionTimerJitterDuration, 30*time.Minute),
370364

371365
TransferTaskBatchSize: dc.GetIntProperty(dynamicconfig.TransferTaskBatchSize, 100),
372-
TransferTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.TransferTaskMaxRetryCount, 20),
373366
TransferProcessorSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.TransferProcessorSchedulerWorkerCount, 512),
374367
TransferProcessorSchedulerActiveRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TransferProcessorSchedulerActiveRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultActiveTaskPriorityWeight)),
375368
TransferProcessorSchedulerStandbyRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.TransferProcessorSchedulerStandbyRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultStandbyTaskPriorityWeight)),
@@ -387,7 +380,6 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis
387380

388381
ReplicatorTaskBatchSize: dc.GetIntProperty(dynamicconfig.ReplicatorTaskBatchSize, 100),
389382
ReplicatorTaskWorkerCount: dc.GetIntProperty(dynamicconfig.ReplicatorTaskWorkerCount, 10),
390-
ReplicatorTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.ReplicatorTaskMaxRetryCount, 100),
391383
ReplicatorProcessorMaxPollRPS: dc.GetIntProperty(dynamicconfig.ReplicatorProcessorMaxPollRPS, 20),
392384
ReplicatorProcessorMaxPollInterval: dc.GetDurationProperty(dynamicconfig.ReplicatorProcessorMaxPollInterval, 1*time.Minute),
393385
ReplicatorProcessorMaxPollIntervalJitterCoefficient: dc.GetFloat64Property(dynamicconfig.ReplicatorProcessorMaxPollIntervalJitterCoefficient, 0.15),
@@ -469,7 +461,6 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis
469461
VisibilityTaskBatchSize: dc.GetIntProperty(dynamicconfig.VisibilityTaskBatchSize, 100),
470462
VisibilityProcessorMaxPollRPS: dc.GetIntProperty(dynamicconfig.VisibilityProcessorMaxPollRPS, 20),
471463
VisibilityProcessorMaxPollHostRPS: dc.GetIntProperty(dynamicconfig.VisibilityProcessorMaxPollHostRPS, 0),
472-
VisibilityTaskMaxRetryCount: dc.GetIntProperty(dynamicconfig.VisibilityTaskMaxRetryCount, 20),
473464
VisibilityProcessorSchedulerWorkerCount: dc.GetIntProperty(dynamicconfig.VisibilityProcessorSchedulerWorkerCount, 512),
474465
VisibilityProcessorSchedulerActiveRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.VisibilityProcessorSchedulerActiveRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultActiveTaskPriorityWeight)),
475466
VisibilityProcessorSchedulerStandbyRoundRobinWeights: dc.GetMapPropertyFnWithNamespaceFilter(dynamicconfig.VisibilityProcessorSchedulerStandbyRoundRobinWeights, ConvertWeightsToDynamicConfigValue(DefaultStandbyTaskPriorityWeight)),
@@ -514,7 +505,6 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis
514505
ArchivalProcessorUpdateAckIntervalJitterCoefficient, 0.15),
515506
ArchivalProcessorPollBackoffInterval: dc.GetDurationProperty(dynamicconfig.ArchivalProcessorPollBackoffInterval, 5*time.Second),
516507
ArchivalProcessorArchiveDelay: dc.GetDurationProperty(dynamicconfig.ArchivalProcessorArchiveDelay, 5*time.Minute),
517-
ArchivalProcessorRetryWarningLimit: dc.GetIntProperty(dynamicconfig.ArchivalProcessorRetryWarningLimit, 100),
518508
ArchivalBackendMaxRPS: dc.GetFloat64Property(dynamicconfig.ArchivalBackendMaxRPS, 10000.0),
519509
}
520510

service/history/queues/executable.go

+5-6
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,6 @@ import (
3636
"go.temporal.io/server/common"
3737
"go.temporal.io/server/common/backoff"
3838
"go.temporal.io/server/common/clock"
39-
"go.temporal.io/server/common/dynamicconfig"
4039
"go.temporal.io/server/common/headers"
4140
"go.temporal.io/server/common/log"
4241
"go.temporal.io/server/common/log/tag"
@@ -90,6 +89,9 @@ const (
9089
// resourceExhaustedResubmitMaxAttempts is the same as resubmitMaxAttempts but only applies to resource
9190
// exhausted error
9291
resourceExhaustedResubmitMaxAttempts = 1
92+
// taskCriticalLogMetricAttempts, if exceeded, task attempts metrics and critical processing error log will be emitted
93+
// while task is retrying
94+
taskCriticalLogMetricAttempts = 30
9395
)
9496

9597
type (
@@ -118,7 +120,6 @@ type (
118120
logger log.Logger
119121
metricsHandler metrics.Handler
120122
taggedMetricsHandler metrics.Handler
121-
criticalRetryAttempt dynamicconfig.IntPropertyFn
122123
}
123124
)
124125

@@ -136,7 +137,6 @@ func NewExecutable(
136137
namespaceRegistry namespace.Registry,
137138
logger log.Logger,
138139
metricsHandler metrics.Handler,
139-
criticalRetryAttempt dynamicconfig.IntPropertyFn,
140140
) Executable {
141141
executable := &executableImpl{
142142
Task: task,
@@ -158,7 +158,6 @@ func NewExecutable(
158158
),
159159
metricsHandler: metricsHandler,
160160
taggedMetricsHandler: metricsHandler,
161-
criticalRetryAttempt: criticalRetryAttempt,
162161
}
163162
executable.updatePriority()
164163
return executable
@@ -209,9 +208,9 @@ func (e *executableImpl) HandleErr(err error) (retErr error) {
209208
defer e.Unlock()
210209

211210
e.attempt++
212-
if e.attempt > e.criticalRetryAttempt() {
211+
if e.attempt > taskCriticalLogMetricAttempts {
213212
e.taggedMetricsHandler.Histogram(metrics.TaskAttempt.GetMetricName(), metrics.TaskAttempt.GetMetricUnit()).Record(int64(e.attempt))
214-
e.logger.Error("Critical error processing task, retrying.", tag.Error(err), tag.OperationCritical)
213+
e.logger.Error("Critical error processing task, retrying.", tag.Attempt(int32(e.attempt)), tag.Error(err), tag.OperationCritical)
215214
}
216215
}
217216
}()

service/history/queues/executable_test.go

-2
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,6 @@ import (
3838

3939
"go.temporal.io/server/common/clock"
4040
"go.temporal.io/server/common/definition"
41-
"go.temporal.io/server/common/dynamicconfig"
4241
"go.temporal.io/server/common/log"
4342
"go.temporal.io/server/common/metrics"
4443
"go.temporal.io/server/common/namespace"
@@ -222,6 +221,5 @@ func (s *executableSuite) newTestExecutable() Executable {
222221
s.mockNamespaceRegistry,
223222
log.NewTestLogger(),
224223
metrics.NoopMetricsHandler,
225-
dynamicconfig.GetIntPropertyFn(100),
226224
)
227225
}

service/history/queues/queue_base.go

-2
Original file line numberDiff line numberDiff line change
@@ -116,7 +116,6 @@ type (
116116
CheckpointInterval dynamicconfig.DurationPropertyFn
117117
CheckpointIntervalJitterCoefficient dynamicconfig.FloatPropertyFn
118118
MaxReaderCount dynamicconfig.IntPropertyFn
119-
TaskMaxRetryCount dynamicconfig.IntPropertyFn
120119
}
121120
)
122121

@@ -167,7 +166,6 @@ func newQueueBase(
167166
shard.GetNamespaceRegistry(),
168167
logger,
169168
metricsHandler,
170-
options.TaskMaxRetryCount,
171169
)
172170
}
173171

service/history/queues/queue_base_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -85,7 +85,6 @@ var testQueueOptions = &Options{
8585
CheckpointInterval: dynamicconfig.GetDurationPropertyFn(100 * time.Millisecond),
8686
CheckpointIntervalJitterCoefficient: dynamicconfig.GetFloatPropertyFn(0.15),
8787
MaxReaderCount: dynamicconfig.GetIntPropertyFn(5),
88-
TaskMaxRetryCount: dynamicconfig.GetIntPropertyFn(100),
8988
}
9089

9190
func TestQueueBaseSuite(t *testing.T) {

service/history/queues/reader_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ func (s *readerSuite) SetupTest() {
7777
s.metricsHandler = metrics.NoopMetricsHandler
7878

7979
s.executableInitializer = func(readerID int32, t tasks.Task) Executable {
80-
return NewExecutable(readerID, t, nil, nil, nil, NewNoopPriorityAssigner(), clock.NewRealTimeSource(), nil, nil, metrics.NoopMetricsHandler, nil)
80+
return NewExecutable(readerID, t, nil, nil, nil, NewNoopPriorityAssigner(), clock.NewRealTimeSource(), nil, nil, metrics.NoopMetricsHandler)
8181
}
8282
s.monitor = newMonitor(tasks.CategoryTypeScheduled, &MonitorOptions{
8383
PendingTasksCriticalCount: dynamicconfig.GetIntPropertyFn(1000),

service/history/queues/slice_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ func (s *sliceSuite) SetupTest() {
6969
s.controller = gomock.NewController(s.T())
7070

7171
s.executableInitializer = func(readerID int32, t tasks.Task) Executable {
72-
return NewExecutable(readerID, t, nil, nil, nil, NewNoopPriorityAssigner(), clock.NewRealTimeSource(), nil, nil, metrics.NoopMetricsHandler, nil)
72+
return NewExecutable(readerID, t, nil, nil, nil, NewNoopPriorityAssigner(), clock.NewRealTimeSource(), nil, nil, metrics.NoopMetricsHandler)
7373
}
7474
s.monitor = newMonitor(tasks.CategoryTypeScheduled, &MonitorOptions{
7575
PendingTasksCriticalCount: dynamicconfig.GetIntPropertyFn(1000),

service/history/timerQueueActiveTaskExecutor_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -1482,6 +1482,5 @@ func (s *timerQueueActiveTaskExecutorSuite) newTaskExecutable(
14821482
nil,
14831483
nil,
14841484
metrics.NoopMetricsHandler,
1485-
nil,
14861485
)
14871486
}

service/history/timerQueueFactory.go

-1
Original file line numberDiff line numberDiff line change
@@ -193,7 +193,6 @@ func (f *timerQueueFactory) CreateQueue(
193193
CheckpointInterval: f.Config.TimerProcessorUpdateAckInterval,
194194
CheckpointIntervalJitterCoefficient: f.Config.TimerProcessorUpdateAckIntervalJitterCoefficient,
195195
MaxReaderCount: f.Config.QueueMaxReaderCount,
196-
TaskMaxRetryCount: f.Config.TimerTaskMaxRetryCount,
197196
},
198197
f.HostReaderRateLimiter,
199198
logger,

service/history/timerQueueStandbyTaskExecutor_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -1545,6 +1545,5 @@ func (s *timerQueueStandbyTaskExecutorSuite) newTaskExecutable(
15451545
nil,
15461546
nil,
15471547
metrics.NoopMetricsHandler,
1548-
nil,
15491548
)
15501549
}

service/history/transferQueueActiveTaskExecutor_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -2805,6 +2805,5 @@ func (s *transferQueueActiveTaskExecutorSuite) newTaskExecutable(
28052805
nil,
28062806
nil,
28072807
metrics.NoopMetricsHandler,
2808-
nil,
28092808
)
28102809
}

service/history/transferQueueFactory.go

-1
Original file line numberDiff line numberDiff line change
@@ -183,7 +183,6 @@ func (f *transferQueueFactory) CreateQueue(
183183
CheckpointInterval: f.Config.TransferProcessorUpdateAckInterval,
184184
CheckpointIntervalJitterCoefficient: f.Config.TransferProcessorUpdateAckIntervalJitterCoefficient,
185185
MaxReaderCount: f.Config.QueueMaxReaderCount,
186-
TaskMaxRetryCount: f.Config.TransferTaskMaxRetryCount,
187186
},
188187
f.HostReaderRateLimiter,
189188
logger,

service/history/transferQueueStandbyTaskExecutor_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -1303,6 +1303,5 @@ func (s *transferQueueStandbyTaskExecutorSuite) newTaskExecutable(
13031303
nil,
13041304
nil,
13051305
metrics.NoopMetricsHandler,
1306-
nil,
13071306
)
13081307
}

service/history/visibilityQueueFactory.go

-1
Original file line numberDiff line numberDiff line change
@@ -138,7 +138,6 @@ func (f *visibilityQueueFactory) CreateQueue(
138138
CheckpointInterval: f.Config.VisibilityProcessorUpdateAckInterval,
139139
CheckpointIntervalJitterCoefficient: f.Config.VisibilityProcessorUpdateAckIntervalJitterCoefficient,
140140
MaxReaderCount: f.Config.QueueMaxReaderCount,
141-
TaskMaxRetryCount: f.Config.VisibilityTaskMaxRetryCount,
142141
},
143142
f.HostReaderRateLimiter,
144143
logger,

service/history/visibilityQueueTaskExecutor_test.go

-1
Original file line numberDiff line numberDiff line change
@@ -641,6 +641,5 @@ func (s *visibilityQueueTaskExecutorSuite) newTaskExecutable(
641641
nil,
642642
nil,
643643
metrics.NoopMetricsHandler,
644-
nil,
645644
)
646645
}

0 commit comments

Comments
 (0)