Skip to content

Commit 5175ab6

Browse files
authored
Single scheduled queue max read level (#3778)
1 parent 82c958d commit 5175ab6

File tree

6 files changed

+29
-91
lines changed

6 files changed

+29
-91
lines changed

service/history/queues/queue_base.go

+1-4
Original file line numberDiff line numberDiff line change
@@ -288,10 +288,7 @@ func (p *queueBase) processNewRange() error {
288288
newMaxKey = p.shard.GetImmediateQueueExclusiveHighReadWatermark()
289289
case tasks.CategoryTypeScheduled:
290290
var err error
291-
if newMaxKey, err = p.shard.UpdateScheduledQueueExclusiveHighReadWatermark(
292-
p.shard.GetClusterMetadata().GetCurrentClusterName(),
293-
true,
294-
); err != nil {
291+
if newMaxKey, err = p.shard.UpdateScheduledQueueExclusiveHighReadWatermark(); err != nil {
295292
return err
296293
}
297294
default:

service/history/shard/context.go

+1-3
Original file line numberDiff line numberDiff line change
@@ -75,9 +75,7 @@ type (
7575
GenerateTaskIDs(number int) ([]int64, error)
7676

7777
GetImmediateQueueExclusiveHighReadWatermark() tasks.Key
78-
// TODO: remove cluster and singleProcessorMode parameter after deprecating old task procesing logic
79-
// In multi-cursor world, there's only one maxReadLevel for scheduled queue for all clusters.
80-
UpdateScheduledQueueExclusiveHighReadWatermark(cluster string, singleProcessorMode bool) (tasks.Key, error)
78+
UpdateScheduledQueueExclusiveHighReadWatermark() (tasks.Key, error)
8179
GetQueueAckLevel(category tasks.Category) tasks.Key
8280
UpdateQueueAckLevel(category tasks.Category, ackLevel tasks.Key) error
8381
GetQueueClusterAckLevel(category tasks.Category, cluster string) tasks.Key

service/history/shard/context_impl.go

+19-45
Original file line numberDiff line numberDiff line change
@@ -132,7 +132,7 @@ type (
132132
taskSequenceNumber int64
133133
maxTaskSequenceNumber int64
134134
immediateTaskExclusiveMaxReadLevel int64
135-
scheduledTaskMaxReadLevelMap map[string]time.Time // cluster -> scheduledTaskMaxReadLevel
135+
scheduledTaskMaxReadLevel time.Time
136136

137137
// exist only in memory
138138
remoteClusterInfos map[string]*remoteClusterInfo
@@ -295,50 +295,26 @@ func (s *ContextImpl) getScheduledTaskMaxReadLevel(cluster string) tasks.Key {
295295
s.rLock()
296296
defer s.rUnlock()
297297

298-
if _, ok := s.scheduledTaskMaxReadLevelMap[cluster]; !ok {
299-
s.scheduledTaskMaxReadLevelMap[cluster] = tasks.DefaultFireTime
300-
}
301-
302-
return tasks.NewKey(s.scheduledTaskMaxReadLevelMap[cluster], 0)
298+
return tasks.NewKey(s.scheduledTaskMaxReadLevel, 0)
303299
}
304300

305-
func (s *ContextImpl) UpdateScheduledQueueExclusiveHighReadWatermark(
306-
cluster string,
307-
singleProcessorMode bool,
308-
) (tasks.Key, error) {
301+
func (s *ContextImpl) UpdateScheduledQueueExclusiveHighReadWatermark() (tasks.Key, error) {
309302
s.wLock()
310303
defer s.wUnlock()
311304

312-
if _, ok := s.scheduledTaskMaxReadLevelMap[cluster]; !ok {
313-
s.scheduledTaskMaxReadLevelMap[cluster] = tasks.DefaultFireTime
314-
}
315-
316305
if err := s.errorByState(); err != nil {
317-
return tasks.NewKey(s.scheduledTaskMaxReadLevelMap[cluster], 0), err
306+
return tasks.NewKey(s.scheduledTaskMaxReadLevel, 0), err
318307
}
319308

320309
currentTime := s.timeSource.Now()
321-
if cluster != "" && cluster != s.GetClusterMetadata().GetCurrentClusterName() {
322-
currentTime = s.getOrUpdateRemoteClusterInfoLocked(cluster).CurrentTime
323-
}
324310

325311
// Truncation here is just to make sure max read level has the same precision as the old logic
326312
// in case existing code can't work correctly with precision higher than 1ms.
327313
// Once we validate the rest of the code can worker correctly with higher precision, the truncation should be removed.
328314
newMaxReadLevel := currentTime.Add(s.config.TimerProcessorMaxTimeShift()).Truncate(persistence.ScheduledTaskMinPrecision)
329-
if singleProcessorMode {
330-
// When generating scheduled tasks, the task's timestamp will be compared to the namespace's active cluster's
331-
// maxReadLevel to avoid generatnig a task before maxReadLevel.
332-
// But when there's a single procssor, the queue is only using current cluster maxReadLevel.
333-
// So update the maxReadLevel map for all clusters to ensure scheduled task won't be lost.
334-
for key := range s.scheduledTaskMaxReadLevelMap {
335-
s.scheduledTaskMaxReadLevelMap[key] = util.MaxTime(s.scheduledTaskMaxReadLevelMap[key], newMaxReadLevel)
336-
}
337-
} else {
338-
s.scheduledTaskMaxReadLevelMap[cluster] = util.MaxTime(s.scheduledTaskMaxReadLevelMap[cluster], newMaxReadLevel)
339-
}
315+
s.scheduledTaskMaxReadLevel = util.MaxTime(s.scheduledTaskMaxReadLevel, newMaxReadLevel)
340316

341-
return tasks.NewKey(s.scheduledTaskMaxReadLevelMap[cluster], 0), nil
317+
return tasks.NewKey(s.scheduledTaskMaxReadLevel, 0), nil
342318
}
343319

344320
// NOTE: the ack level returned is inclusive for immediate task category (acked),
@@ -1388,7 +1364,6 @@ func (s *ContextImpl) allocateTaskIDAndTimestampLocked(
13881364
transferExclusiveMaxReadLevel *int64,
13891365
) error {
13901366
now := s.timeSource.Now()
1391-
currentCluster := s.GetClusterMetadata().GetCurrentClusterName()
13921367
for category, tasksByCategory := range newTasks {
13931368
for _, task := range tasksByCategory {
13941369
// set taskID
@@ -1409,13 +1384,7 @@ func (s *ContextImpl) allocateTaskIDAndTimestampLocked(
14091384
// if scheduled task, check if fire time is in the past
14101385
if category.Type() == tasks.CategoryTypeScheduled {
14111386
ts := task.GetVisibilityTime()
1412-
if task.GetVersion() != common.EmptyVersion && category.ID() == tasks.CategoryIDTimer {
1413-
// cannot use version to determine the corresponding cluster for timer task
1414-
// this is because during failover, timer task should be created as active
1415-
// or otherwise, failover + active processing logic may not pick up the task.
1416-
currentCluster = namespaceEntry.ActiveClusterName()
1417-
}
1418-
readCursorTS := s.scheduledTaskMaxReadLevelMap[currentCluster]
1387+
readCursorTS := s.scheduledTaskMaxReadLevel
14191388
if ts.Truncate(persistence.ScheduledTaskMinPrecision).Before(readCursorTS) {
14201389
// make sure scheduled task timestamp is higher than max read level after truncation
14211390
// as persistence layer may lose precision when persisting the task.
@@ -1427,12 +1396,14 @@ func (s *ContextImpl) allocateTaskIDAndTimestampLocked(
14271396
tag.Timestamp(ts),
14281397
tag.CursorTimestamp(readCursorTS),
14291398
tag.ValueShardAllocateTimerBeforeRead)
1430-
task.SetVisibilityTime(s.scheduledTaskMaxReadLevelMap[currentCluster].Add(persistence.ScheduledTaskMinPrecision))
1399+
task.SetVisibilityTime(s.scheduledTaskMaxReadLevel.Add(persistence.ScheduledTaskMinPrecision))
14311400
}
14321401

1433-
visibilityTs := task.GetVisibilityTime()
14341402
s.contextTaggedLogger.Debug("Assigning new timer",
1435-
tag.Timestamp(visibilityTs), tag.TaskID(task.GetTaskID()), tag.MaxQueryLevel(readCursorTS))
1403+
tag.Timestamp(task.GetVisibilityTime()),
1404+
tag.TaskID(task.GetTaskID()),
1405+
tag.MaxQueryLevel(readCursorTS),
1406+
)
14361407
}
14371408
}
14381409
}
@@ -1849,7 +1820,7 @@ func (s *ContextImpl) loadShardMetadata(ownershipChanged *bool) error {
18491820

18501821
// initialize the cluster current time to be the same as ack level
18511822
remoteClusterInfos := make(map[string]*remoteClusterInfo)
1852-
scheduledTaskMaxReadLevelMap := make(map[string]time.Time)
1823+
var scheduledTaskMaxReadLevel time.Time
18531824
currentClusterName := s.GetClusterMetadata().GetCurrentClusterName()
18541825
taskCategories := tasks.GetCategories()
18551826
for clusterName, info := range s.GetClusterMetadata().GetAllClusterInfo() {
@@ -1890,8 +1861,11 @@ func (s *ContextImpl) loadShardMetadata(ownershipChanged *bool) error {
18901861
// Add().Truncate() here is just to make sure max read level has the same precision as the old logic
18911862
// in case existing code can't work correctly with precision higher than 1ms.
18921863
// Once we validate the rest of the code can worker correctly with higher precision, the code should simply be
1893-
// scheduledTaskMaxReadLevelMap[clusterName] = maxReadTime
1894-
scheduledTaskMaxReadLevelMap[clusterName] = maxReadTime.Add(persistence.ScheduledTaskMinPrecision).Truncate(persistence.ScheduledTaskMinPrecision)
1864+
// scheduledTaskMaxReadLevel = util.MaxTime(scheduledTaskMaxReadLevel, maxReadTime)
1865+
scheduledTaskMaxReadLevel = util.MaxTime(
1866+
scheduledTaskMaxReadLevel,
1867+
maxReadTime.Add(persistence.ScheduledTaskMinPrecision).Truncate(persistence.ScheduledTaskMinPrecision),
1868+
)
18951869

18961870
if clusterName != currentClusterName {
18971871
remoteClusterInfos[clusterName] = &remoteClusterInfo{CurrentTime: maxReadTime}
@@ -1903,7 +1877,7 @@ func (s *ContextImpl) loadShardMetadata(ownershipChanged *bool) error {
19031877

19041878
s.shardInfo = updatedShardInfo
19051879
s.remoteClusterInfos = remoteClusterInfos
1906-
s.scheduledTaskMaxReadLevelMap = scheduledTaskMaxReadLevelMap
1880+
s.scheduledTaskMaxReadLevel = scheduledTaskMaxReadLevel
19071881

19081882
return nil
19091883
}

service/history/shard/context_mock.go

+4-4
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

service/history/shard/context_test.go

+4-33
Original file line numberDiff line numberDiff line change
@@ -183,43 +183,14 @@ func (s *contextSuite) TestTimerMaxReadLevelInitialization() {
183183
}
184184
}
185185

186-
func (s *contextSuite) TestTimerMaxReadLevelUpdate_MultiProcessor() {
187-
now := time.Now()
186+
func (s *contextSuite) TestTimerMaxReadLevelUpdate() {
187+
now := time.Now().Add(time.Minute)
188188
s.timeSource.Update(now)
189-
maxReadLevel, err := s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(cluster.TestCurrentClusterName, false)
190-
s.NoError(err)
191-
192-
s.timeSource.Update(now.Add(-time.Minute))
193-
newMaxReadLevel, err := s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(cluster.TestCurrentClusterName, false)
194-
s.NoError(err)
195-
s.Equal(maxReadLevel, newMaxReadLevel)
196-
197-
s.timeSource.Update(now.Add(time.Minute))
198-
newMaxReadLevel, err = s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(cluster.TestCurrentClusterName, false)
199-
s.NoError(err)
200-
s.True(newMaxReadLevel.FireTime.After(maxReadLevel.FireTime))
201-
}
202189

203-
func (s *contextSuite) TestTimerMaxReadLevelUpdate_SingleProcessor() {
204-
now := time.Now()
205-
s.timeSource.Update(now)
206-
207-
// make sure the scheduledTaskMaxReadLevelMap has value for both current cluster and alternative cluster
208-
_, err := s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(cluster.TestCurrentClusterName, false)
209-
s.NoError(err)
210-
_, err = s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(cluster.TestAlternativeClusterName, false)
190+
_, err := s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark()
211191
s.NoError(err)
212192

213-
now = time.Now().Add(time.Minute)
214-
s.timeSource.Update(now)
215-
216-
// update in single processor mode
217-
_, err = s.mockShard.UpdateScheduledQueueExclusiveHighReadWatermark(cluster.TestCurrentClusterName, true)
218-
s.NoError(err)
219-
scheduledTaskMaxReadLevelMap := s.mockShard.scheduledTaskMaxReadLevelMap
220-
s.Len(scheduledTaskMaxReadLevelMap, 2)
221-
s.True(scheduledTaskMaxReadLevelMap[cluster.TestCurrentClusterName].After(now))
222-
s.True(scheduledTaskMaxReadLevelMap[cluster.TestAlternativeClusterName].After(now))
193+
s.True(s.mockShard.scheduledTaskMaxReadLevel.After(now))
223194
}
224195

225196
func (s *contextSuite) TestDeleteWorkflowExecution_Success() {

service/history/shard/context_testutil.go

-2
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ package shard
2727
import (
2828
"context"
2929
"fmt"
30-
"time"
3130

3231
"github.com/golang/mock/gomock"
3332

@@ -100,7 +99,6 @@ func NewTestContext(
10099
taskSequenceNumber: shardInfo.RangeId << int64(config.RangeSizeBits),
101100
immediateTaskExclusiveMaxReadLevel: shardInfo.RangeId << int64(config.RangeSizeBits),
102101
maxTaskSequenceNumber: (shardInfo.RangeId + 1) << int64(config.RangeSizeBits),
103-
scheduledTaskMaxReadLevelMap: make(map[string]time.Time),
104102
remoteClusterInfos: make(map[string]*remoteClusterInfo),
105103
handoverNamespaces: make(map[namespace.Name]*namespaceHandOverInfo),
106104

0 commit comments

Comments
 (0)