2
0
mirror of https://github.com/hibiken/asynq.git synced 2024-09-20 11:05:58 +08:00

Avoid creating script struct on every rdb method invocation

This commit is contained in:
Ken Hibino 2020-02-08 11:06:14 -08:00
parent 7e33c9ebe0
commit d9327cf24b
5 changed files with 383 additions and 313 deletions

View File

@ -12,8 +12,7 @@ import (
"time" "time"
) )
// Simple E2E Benchmark testing with no scheduled tasks and // Simple E2E Benchmark testing with no scheduled tasks and retries.
// no retries.
func BenchmarkEndToEndSimple(b *testing.B) { func BenchmarkEndToEndSimple(b *testing.B) {
const count = 100000 const count = 100000
for n := 0; n < b.N; n++ { for n := 0; n < b.N; n++ {
@ -101,3 +100,58 @@ func BenchmarkEndToEnd(b *testing.B) {
b.StartTimer() // end teardown b.StartTimer() // end teardown
} }
} }
// Simple E2E Benchmark testing with no scheduled tasks and retries with multiple queues.
func BenchmarkEndToEndMultipleQueues(b *testing.B) {
// number of tasks to create for each queue
const (
highCount = 20000
defaultCount = 20000
lowCount = 20000
)
for n := 0; n < b.N; n++ {
b.StopTimer() // begin setup
setup(b)
redis := &RedisClientOpt{
Addr: redisAddr,
DB: redisDB,
}
client := NewClient(redis)
bg := NewBackground(redis, &Config{
Concurrency: 10,
Queues: map[string]uint{
"high": 6,
"default": 3,
"low": 1,
},
})
// Create a bunch of tasks
for i := 0; i < highCount; i++ {
t := NewTask(fmt.Sprintf("task%d", i), map[string]interface{}{"data": i})
client.Schedule(t, time.Now(), Queue("high"))
}
for i := 0; i < defaultCount; i++ {
t := NewTask(fmt.Sprintf("task%d", i), map[string]interface{}{"data": i})
client.Schedule(t, time.Now())
}
for i := 0; i < lowCount; i++ {
t := NewTask(fmt.Sprintf("task%d", i), map[string]interface{}{"data": i})
client.Schedule(t, time.Now(), Queue("low"))
}
var wg sync.WaitGroup
wg.Add(highCount + defaultCount + lowCount)
handler := func(t *Task) error {
wg.Done()
return nil
}
b.StartTimer() // end setup
bg.start(HandlerFunc(handler))
wg.Wait()
b.StopTimer() // begin teardown
bg.stop()
b.StartTimer() // end teardown
}
}

View File

@ -86,49 +86,48 @@ type DeadTask struct {
Queue string Queue string
} }
// CurrentStats returns a current state of the queues. // KEYS[1] -> asynq:queues
func (r *RDB) CurrentStats() (*Stats, error) { // KEYS[2] -> asynq:in_progress
// KEYS[1] -> asynq:queues // KEYS[3] -> asynq:scheduled
// KEYS[2] -> asynq:in_progress // KEYS[4] -> asynq:retry
// KEYS[3] -> asynq:scheduled // KEYS[5] -> asynq:dead
// KEYS[4] -> asynq:retry // KEYS[6] -> asynq:processed:<yyyy-mm-dd>
// KEYS[5] -> asynq:dead // KEYS[7] -> asynq:failure:<yyyy-mm-dd>
// KEYS[6] -> asynq:processed:<yyyy-mm-dd> var currentStatsCmd = redis.NewScript(`
// KEYS[7] -> asynq:failure:<yyyy-mm-dd> local res = {}
script := redis.NewScript(` local queues = redis.call("SMEMBERS", KEYS[1])
local res = {} for _, qkey in ipairs(queues) do
local queues = redis.call("SMEMBERS", KEYS[1])
for _, qkey in ipairs(queues) do
table.insert(res, qkey) table.insert(res, qkey)
table.insert(res, redis.call("LLEN", qkey)) table.insert(res, redis.call("LLEN", qkey))
end end
table.insert(res, KEYS[2]) table.insert(res, KEYS[2])
table.insert(res, redis.call("LLEN", KEYS[2])) table.insert(res, redis.call("LLEN", KEYS[2]))
table.insert(res, KEYS[3]) table.insert(res, KEYS[3])
table.insert(res, redis.call("ZCARD", KEYS[3])) table.insert(res, redis.call("ZCARD", KEYS[3]))
table.insert(res, KEYS[4]) table.insert(res, KEYS[4])
table.insert(res, redis.call("ZCARD", KEYS[4])) table.insert(res, redis.call("ZCARD", KEYS[4]))
table.insert(res, KEYS[5]) table.insert(res, KEYS[5])
table.insert(res, redis.call("ZCARD", KEYS[5])) table.insert(res, redis.call("ZCARD", KEYS[5]))
local pcount = 0 local pcount = 0
local p = redis.call("GET", KEYS[6]) local p = redis.call("GET", KEYS[6])
if p then if p then
pcount = tonumber(p) pcount = tonumber(p)
end end
table.insert(res, "processed") table.insert(res, "processed")
table.insert(res, pcount) table.insert(res, pcount)
local fcount = 0 local fcount = 0
local f = redis.call("GET", KEYS[7]) local f = redis.call("GET", KEYS[7])
if f then if f then
fcount = tonumber(f) fcount = tonumber(f)
end end
table.insert(res, "failed") table.insert(res, "failed")
table.insert(res, fcount) table.insert(res, fcount)
return res return res`)
`)
// CurrentStats returns a current state of the queues.
func (r *RDB) CurrentStats() (*Stats, error) {
now := time.Now() now := time.Now()
res, err := script.Run(r.client, []string{ res, err := currentStatsCmd.Run(r.client, []string{
base.AllQueues, base.AllQueues,
base.InProgressQueue, base.InProgressQueue,
base.ScheduledQueue, base.ScheduledQueue,
@ -173,6 +172,17 @@ func (r *RDB) CurrentStats() (*Stats, error) {
return stats, nil return stats, nil
} }
var historicalStatsCmd = redis.NewScript(`
local res = {}
for _, key in ipairs(KEYS) do
local n = redis.call("GET", key)
if not n then
n = 0
end
table.insert(res, tonumber(n))
end
return res`)
// HistoricalStats returns a list of stats from the last n days. // HistoricalStats returns a list of stats from the last n days.
func (r *RDB) HistoricalStats(n int) ([]*DailyStats, error) { func (r *RDB) HistoricalStats(n int) ([]*DailyStats, error) {
if n < 1 { if n < 1 {
@ -188,18 +198,7 @@ func (r *RDB) HistoricalStats(n int) ([]*DailyStats, error) {
keys = append(keys, base.ProcessedKey(ts)) keys = append(keys, base.ProcessedKey(ts))
keys = append(keys, base.FailureKey(ts)) keys = append(keys, base.FailureKey(ts))
} }
script := redis.NewScript(` res, err := historicalStatsCmd.Run(r.client, keys, len(keys)).Result()
local res = {}
for _, key in ipairs(KEYS) do
local n = redis.call("GET", key)
if not n then
n = 0
end
table.insert(res, tonumber(n))
end
return res
`)
res, err := script.Run(r.client, keys, len(keys)).Result()
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -475,21 +474,21 @@ func (r *RDB) EnqueueAllDeadTasks() (int64, error) {
return r.removeAndEnqueueAll(base.DeadQueue) return r.removeAndEnqueueAll(base.DeadQueue)
} }
func (r *RDB) removeAndEnqueue(zset, id string, score float64) (int64, error) { var removeAndEnqueueCmd = redis.NewScript(`
script := redis.NewScript(` local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], ARGV[1], ARGV[1])
local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], ARGV[1], ARGV[1]) for _, msg in ipairs(msgs) do
for _, msg in ipairs(msgs) do
local decoded = cjson.decode(msg) local decoded = cjson.decode(msg)
if decoded["ID"] == ARGV[2] then if decoded["ID"] == ARGV[2] then
redis.call("ZREM", KEYS[1], msg)
local qkey = ARGV[3] .. decoded["Queue"] local qkey = ARGV[3] .. decoded["Queue"]
redis.call("LPUSH", qkey, msg) redis.call("LPUSH", qkey, msg)
redis.call("ZREM", KEYS[1], msg)
return 1 return 1
end end
end end
return 0 return 0`)
`)
res, err := script.Run(r.client, []string{zset}, score, id, base.QueuePrefix).Result() func (r *RDB) removeAndEnqueue(zset, id string, score float64) (int64, error) {
res, err := removeAndEnqueueCmd.Run(r.client, []string{zset}, score, id, base.QueuePrefix).Result()
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -500,18 +499,18 @@ func (r *RDB) removeAndEnqueue(zset, id string, score float64) (int64, error) {
return n, nil return n, nil
} }
func (r *RDB) removeAndEnqueueAll(zset string) (int64, error) { var removeAndEnqueueAllCmd = redis.NewScript(`
script := redis.NewScript(` local msgs = redis.call("ZRANGE", KEYS[1], 0, -1)
local msgs = redis.call("ZRANGE", KEYS[1], 0, -1) for _, msg in ipairs(msgs) do
for _, msg in ipairs(msgs) do
redis.call("ZREM", KEYS[1], msg)
local decoded = cjson.decode(msg) local decoded = cjson.decode(msg)
local qkey = ARGV[1] .. decoded["Queue"] local qkey = ARGV[1] .. decoded["Queue"]
redis.call("LPUSH", qkey, msg) redis.call("LPUSH", qkey, msg)
end redis.call("ZREM", KEYS[1], msg)
return table.getn(msgs) end
`) return table.getn(msgs)`)
res, err := script.Run(r.client, []string{zset}, base.QueuePrefix).Result()
func (r *RDB) removeAndEnqueueAll(zset string) (int64, error) {
res, err := removeAndEnqueueAllCmd.Run(r.client, []string{zset}, base.QueuePrefix).Result()
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -562,17 +561,16 @@ func (r *RDB) KillAllScheduledTasks() (int64, error) {
return r.removeAndKillAll(base.ScheduledQueue) return r.removeAndKillAll(base.ScheduledQueue)
} }
func (r *RDB) removeAndKill(zset, id string, score float64) (int64, error) { // KEYS[1] -> ZSET to move task from (e.g., retry queue)
// KEYS[1] -> ZSET to move task from (e.g., retry queue) // KEYS[2] -> asynq:dead
// KEYS[2] -> asynq:dead // ARGV[1] -> score of the task to kill
// ARGV[1] -> score of the task to kill // ARGV[2] -> id of the task to kill
// ARGV[2] -> id of the task to kill // ARGV[3] -> current timestamp
// ARGV[3] -> current timestamp // ARGV[4] -> cutoff timestamp (e.g., 90 days ago)
// ARGV[4] -> cutoff timestamp (e.g., 90 days ago) // ARGV[5] -> max number of tasks in dead queue (e.g., 100)
// ARGV[5] -> max number of tasks in dead queue (e.g., 100) var removeAndKillCmd = redis.NewScript(`
script := redis.NewScript(` local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], ARGV[1], ARGV[1])
local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], ARGV[1], ARGV[1]) for _, msg in ipairs(msgs) do
for _, msg in ipairs(msgs) do
local decoded = cjson.decode(msg) local decoded = cjson.decode(msg)
if decoded["ID"] == ARGV[2] then if decoded["ID"] == ARGV[2] then
redis.call("ZREM", KEYS[1], msg) redis.call("ZREM", KEYS[1], msg)
@ -581,12 +579,13 @@ func (r *RDB) removeAndKill(zset, id string, score float64) (int64, error) {
redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[5]) redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[5])
return 1 return 1
end end
end end
return 0 return 0`)
`)
func (r *RDB) removeAndKill(zset, id string, score float64) (int64, error) {
now := time.Now() now := time.Now()
limit := now.AddDate(0, 0, -deadExpirationInDays).Unix() // 90 days ago limit := now.AddDate(0, 0, -deadExpirationInDays).Unix() // 90 days ago
res, err := script.Run(r.client, res, err := removeAndKillCmd.Run(r.client,
[]string{zset, base.DeadQueue}, []string{zset, base.DeadQueue},
score, id, now.Unix(), limit, maxDeadTasks).Result() score, id, now.Unix(), limit, maxDeadTasks).Result()
if err != nil { if err != nil {
@ -599,25 +598,25 @@ func (r *RDB) removeAndKill(zset, id string, score float64) (int64, error) {
return n, nil return n, nil
} }
func (r *RDB) removeAndKillAll(zset string) (int64, error) { // KEYS[1] -> ZSET to move task from (e.g., retry queue)
// KEYS[1] -> ZSET to move task from (e.g., retry queue) // KEYS[2] -> asynq:dead
// KEYS[2] -> asynq:dead // ARGV[1] -> current timestamp
// ARGV[1] -> current timestamp // ARGV[2] -> cutoff timestamp (e.g., 90 days ago)
// ARGV[2] -> cutoff timestamp (e.g., 90 days ago) // ARGV[3] -> max number of tasks in dead queue (e.g., 100)
// ARGV[3] -> max number of tasks in dead queue (e.g., 100) var removeAndKillAllCmd = redis.NewScript(`
script := redis.NewScript(` local msgs = redis.call("ZRANGE", KEYS[1], 0, -1)
local msgs = redis.call("ZRANGE", KEYS[1], 0, -1) for _, msg in ipairs(msgs) do
for _, msg in ipairs(msgs) do
redis.call("ZREM", KEYS[1], msg)
redis.call("ZADD", KEYS[2], ARGV[1], msg) redis.call("ZADD", KEYS[2], ARGV[1], msg)
redis.call("ZREM", KEYS[1], msg)
redis.call("ZREMRANGEBYSCORE", KEYS[2], "-inf", ARGV[2]) redis.call("ZREMRANGEBYSCORE", KEYS[2], "-inf", ARGV[2])
redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[3]) redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[3])
end end
return table.getn(msgs) return table.getn(msgs)`)
`)
func (r *RDB) removeAndKillAll(zset string) (int64, error) {
now := time.Now() now := time.Now()
limit := now.AddDate(0, 0, -deadExpirationInDays).Unix() // 90 days ago limit := now.AddDate(0, 0, -deadExpirationInDays).Unix() // 90 days ago
res, err := script.Run(r.client, []string{zset, base.DeadQueue}, res, err := removeAndKillAllCmd.Run(r.client, []string{zset, base.DeadQueue},
now.Unix(), limit, maxDeadTasks).Result() now.Unix(), limit, maxDeadTasks).Result()
if err != nil { if err != nil {
return 0, err return 0, err
@ -650,19 +649,19 @@ func (r *RDB) DeleteScheduledTask(id xid.ID, score int64) error {
return r.deleteTask(base.ScheduledQueue, id.String(), float64(score)) return r.deleteTask(base.ScheduledQueue, id.String(), float64(score))
} }
func (r *RDB) deleteTask(zset, id string, score float64) error { var deleteTaskCmd = redis.NewScript(`
script := redis.NewScript(` local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], ARGV[1], ARGV[1])
local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], ARGV[1], ARGV[1]) for _, msg in ipairs(msgs) do
for _, msg in ipairs(msgs) do
local decoded = cjson.decode(msg) local decoded = cjson.decode(msg)
if decoded["ID"] == ARGV[2] then if decoded["ID"] == ARGV[2] then
redis.call("ZREM", KEYS[1], msg) redis.call("ZREM", KEYS[1], msg)
return 1 return 1
end end
end end
return 0 return 0`)
`)
res, err := script.Run(r.client, []string{zset}, score, id).Result() func (r *RDB) deleteTask(zset, id string, score float64) error {
res, err := deleteTaskCmd.Run(r.client, []string{zset}, score, id).Result()
if err != nil { if err != nil {
return err return err
} }
@ -709,6 +708,27 @@ func (e *ErrQueueNotEmpty) Error() string {
return fmt.Sprintf("queue %q is not empty", e.qname) return fmt.Sprintf("queue %q is not empty", e.qname)
} }
// Skip checking whether queue is empty before removing.
var removeQueueForceCmd = redis.NewScript(`
local n = redis.call("SREM", KEYS[1], KEYS[2])
if n == 0 then
return redis.error_reply("LIST NOT FOUND")
end
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")`)
// Checks whether queue is empty before removing.
var removeQueueCmd = redis.NewScript(`
local l = redis.call("LLEN", KEYS[2]) if l > 0 then
return redis.error_reply("LIST NOT EMPTY")
end
local n = redis.call("SREM", KEYS[1], KEYS[2])
if n == 0 then
return redis.error_reply("LIST NOT FOUND")
end
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")`)
// RemoveQueue removes the specified queue. // RemoveQueue removes the specified queue.
// //
// If force is set to true, it will remove the queue regardless // If force is set to true, it will remove the queue regardless
@ -718,27 +738,9 @@ func (e *ErrQueueNotEmpty) Error() string {
func (r *RDB) RemoveQueue(qname string, force bool) error { func (r *RDB) RemoveQueue(qname string, force bool) error {
var script *redis.Script var script *redis.Script
if force { if force {
script = redis.NewScript(` script = removeQueueForceCmd
local n = redis.call("SREM", KEYS[1], KEYS[2])
if n == 0 then
return redis.error_reply("LIST NOT FOUND")
end
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")
`)
} else { } else {
script = redis.NewScript(` script = removeQueueCmd
local l = redis.call("LLEN", KEYS[2])
if l > 0 then
return redis.error_reply("LIST NOT EMPTY")
end
local n = redis.call("SREM", KEYS[1], KEYS[2])
if n == 0 then
return redis.error_reply("LIST NOT FOUND")
end
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")
`)
} }
err := script.Run(r.client, err := script.Run(r.client,
[]string{base.AllQueues, base.QueueKey(qname)}, []string{base.AllQueues, base.QueueKey(qname)},
@ -756,23 +758,23 @@ func (r *RDB) RemoveQueue(qname string, force bool) error {
return nil return nil
} }
// ListProcesses returns the list of process statuses. // Note: Script also removes stale keys.
func (r *RDB) ListProcesses() ([]*base.ProcessInfo, error) { var listProcessesCmd = redis.NewScript(`
// Note: Script also removes stale keys. local res = {}
script := redis.NewScript(` local now = tonumber(ARGV[1])
local res = {} local keys = redis.call("ZRANGEBYSCORE", KEYS[1], now, "+inf")
local now = tonumber(ARGV[1]) for _, key in ipairs(keys) do
local keys = redis.call("ZRANGEBYSCORE", KEYS[1], now, "+inf")
for _, key in ipairs(keys) do
local ps = redis.call("GET", key) local ps = redis.call("GET", key)
if ps then if ps then
table.insert(res, ps) table.insert(res, ps)
end end
end end
redis.call("ZREMRANGEBYSCORE", KEYS[1], "-inf", now-1) redis.call("ZREMRANGEBYSCORE", KEYS[1], "-inf", now-1)
return res return res`)
`)
res, err := script.Run(r.client, // ListProcesses returns the list of process statuses.
func (r *RDB) ListProcesses() ([]*base.ProcessInfo, error) {
res, err := listProcessesCmd.Run(r.client,
[]string{base.AllProcesses}, time.Now().UTC().Unix()).Result() []string{base.AllProcesses}, time.Now().UTC().Unix()).Result()
if err != nil { if err != nil {
return nil, err return nil, err

View File

@ -41,6 +41,14 @@ func (r *RDB) Close() error {
return r.client.Close() return r.client.Close()
} }
// KEYS[1] -> asynq:queues:<qname>
// KEYS[2] -> asynq:queues
// ARGV[1] -> task message data
var enqueueCmd = redis.NewScript(`
redis.call("LPUSH", KEYS[1], ARGV[1])
redis.call("SADD", KEYS[2], KEYS[1])
return 1`)
// Enqueue inserts the given task to the tail of the queue. // Enqueue inserts the given task to the tail of the queue.
func (r *RDB) Enqueue(msg *base.TaskMessage) error { func (r *RDB) Enqueue(msg *base.TaskMessage) error {
bytes, err := json.Marshal(msg) bytes, err := json.Marshal(msg)
@ -48,23 +56,18 @@ func (r *RDB) Enqueue(msg *base.TaskMessage) error {
return err return err
} }
key := base.QueueKey(msg.Queue) key := base.QueueKey(msg.Queue)
script := redis.NewScript(` return enqueueCmd.Run(r.client, []string{key, base.AllQueues}, bytes).Err()
redis.call("LPUSH", KEYS[1], ARGV[1])
redis.call("SADD", KEYS[2], KEYS[1])
return 1
`)
return script.Run(r.client, []string{key, base.AllQueues}, string(bytes)).Err()
} }
// Dequeue queries given queues in order and pops a task message if there // Dequeue queries given queues in order and pops a task message if there is one and returns it.
// is one and returns it. If all queues are empty, ErrNoProcessableTask // If all queues are empty, ErrNoProcessableTask error is returned.
// error is returned.
func (r *RDB) Dequeue(qnames ...string) (*base.TaskMessage, error) { func (r *RDB) Dequeue(qnames ...string) (*base.TaskMessage, error) {
var data string var data string
var err error var err error
if len(qnames) == 1 { if len(qnames) == 1 {
data, err = r.dequeueSingle(base.QueueKey(qnames[0])) data, err = r.dequeueSingle(base.QueueKey(qnames[0]))
} else { } else {
// TODO(hibiken): Take keys are argument and don't compute every time
var keys []string var keys []string
for _, q := range qnames { for _, q := range qnames {
keys = append(keys, base.QueueKey(q)) keys = append(keys, base.QueueKey(q))
@ -90,72 +93,74 @@ func (r *RDB) dequeueSingle(queue string) (data string, err error) {
return r.client.BRPopLPush(queue, base.InProgressQueue, time.Second).Result() return r.client.BRPopLPush(queue, base.InProgressQueue, time.Second).Result()
} }
// KEYS[1] -> asynq:in_progress
// ARGV -> List of queues to query in order
var dequeueCmd = redis.NewScript(`
local res
for _, qkey in ipairs(ARGV) do
res = redis.call("RPOPLPUSH", qkey, KEYS[1])
if res then
return res
end
end
return res`)
func (r *RDB) dequeue(queues ...string) (data string, err error) { func (r *RDB) dequeue(queues ...string) (data string, err error) {
var args []interface{} var args []interface{}
for _, qkey := range queues { for _, qkey := range queues {
args = append(args, qkey) args = append(args, qkey)
} }
script := redis.NewScript(` res, err := dequeueCmd.Run(r.client, []string{base.InProgressQueue}, args...).Result()
local res
for _, qkey in ipairs(ARGV) do
res = redis.call("RPOPLPUSH", qkey, KEYS[1])
if res then
return res
end
end
return res
`)
res, err := script.Run(r.client, []string{base.InProgressQueue}, args...).Result()
if err != nil { if err != nil {
return "", err return "", err
} }
return cast.ToStringE(res) return cast.ToStringE(res)
} }
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:processed:<yyyy-mm-dd>
// ARGV[1] -> base.TaskMessage value
// ARGV[2] -> stats expiration timestamp
// Note: LREM count ZERO means "remove all elements equal to val"
var doneCmd = redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
local n = redis.call("INCR", KEYS[2])
if tonumber(n) == 1 then
redis.call("EXPIREAT", KEYS[2], ARGV[2])
end
return redis.status_reply("OK")
`)
// Done removes the task from in-progress queue to mark the task as done. // Done removes the task from in-progress queue to mark the task as done.
func (r *RDB) Done(msg *base.TaskMessage) error { func (r *RDB) Done(msg *base.TaskMessage) error {
bytes, err := json.Marshal(msg) bytes, err := json.Marshal(msg)
if err != nil { if err != nil {
return err return err
} }
// Note: LREM count ZERO means "remove all elements equal to val"
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:processed:<yyyy-mm-dd>
// ARGV[1] -> base.TaskMessage value
// ARGV[2] -> stats expiration timestamp
script := redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
local n = redis.call("INCR", KEYS[2])
if tonumber(n) == 1 then
redis.call("EXPIREAT", KEYS[2], ARGV[2])
end
return redis.status_reply("OK")
`)
now := time.Now() now := time.Now()
processedKey := base.ProcessedKey(now) processedKey := base.ProcessedKey(now)
expireAt := now.Add(statsTTL) expireAt := now.Add(statsTTL)
return script.Run(r.client, return doneCmd.Run(r.client,
[]string{base.InProgressQueue, processedKey}, []string{base.InProgressQueue, processedKey},
string(bytes), expireAt.Unix()).Err() bytes, expireAt.Unix()).Err()
} }
// Requeue moves the task from in-progress queue to the default // KEYS[1] -> asynq:in_progress
// queue. // KEYS[2] -> asynq:queues:<qname>
// ARGV[1] -> base.TaskMessage value
// Note: Use RPUSH to push to the head of the queue.
var requeueCmd = redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
redis.call("RPUSH", KEYS[2], ARGV[1])
return redis.status_reply("OK")`)
// Requeue moves the task from in-progress queue to the specified queue.
func (r *RDB) Requeue(msg *base.TaskMessage) error { func (r *RDB) Requeue(msg *base.TaskMessage) error {
bytes, err := json.Marshal(msg) bytes, err := json.Marshal(msg)
if err != nil { if err != nil {
return err return err
} }
// Note: Use RPUSH to push to the head of the queue. return requeueCmd.Run(r.client,
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:queues:default
// ARGV[1] -> base.TaskMessage value
script := redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
redis.call("RPUSH", KEYS[2], ARGV[1])
return redis.status_reply("OK")
`)
return script.Run(r.client,
[]string{base.InProgressQueue, base.QueueKey(msg.Queue)}, []string{base.InProgressQueue, base.QueueKey(msg.Queue)},
string(bytes)).Err() string(bytes)).Err()
} }
@ -171,6 +176,27 @@ func (r *RDB) Schedule(msg *base.TaskMessage, processAt time.Time) error {
&redis.Z{Member: string(bytes), Score: score}).Err() &redis.Z{Member: string(bytes), Score: score}).Err()
} }
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:retry
// KEYS[3] -> asynq:processed:<yyyy-mm-dd>
// KEYS[4] -> asynq:failure:<yyyy-mm-dd>
// ARGV[1] -> base.TaskMessage value to remove from base.InProgressQueue queue
// ARGV[2] -> base.TaskMessage value to add to Retry queue
// ARGV[3] -> retry_at UNIX timestamp
// ARGV[4] -> stats expiration timestamp
var retryCmd = redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
local n = redis.call("INCR", KEYS[3])
if tonumber(n) == 1 then
redis.call("EXPIREAT", KEYS[3], ARGV[4])
end
local m = redis.call("INCR", KEYS[4])
if tonumber(m) == 1 then
redis.call("EXPIREAT", KEYS[4], ARGV[4])
end
return redis.status_reply("OK")`)
// Retry moves the task from in-progress to retry queue, incrementing retry count // Retry moves the task from in-progress to retry queue, incrementing retry count
// and assigning error message to the task message. // and assigning error message to the task message.
func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string) error { func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string) error {
@ -185,32 +211,11 @@ func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string) e
if err != nil { if err != nil {
return err return err
} }
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:retry
// KEYS[3] -> asynq:processed:<yyyy-mm-dd>
// KEYS[4] -> asynq:failure:<yyyy-mm-dd>
// ARGV[1] -> base.TaskMessage value to remove from base.InProgressQueue queue
// ARGV[2] -> base.TaskMessage value to add to Retry queue
// ARGV[3] -> retry_at UNIX timestamp
// ARGV[4] -> stats expiration timestamp
script := redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
local n = redis.call("INCR", KEYS[3])
if tonumber(n) == 1 then
redis.call("EXPIREAT", KEYS[3], ARGV[4])
end
local m = redis.call("INCR", KEYS[4])
if tonumber(m) == 1 then
redis.call("EXPIREAT", KEYS[4], ARGV[4])
end
return redis.status_reply("OK")
`)
now := time.Now() now := time.Now()
processedKey := base.ProcessedKey(now) processedKey := base.ProcessedKey(now)
failureKey := base.FailureKey(now) failureKey := base.FailureKey(now)
expireAt := now.Add(statsTTL) expireAt := now.Add(statsTTL)
return script.Run(r.client, return retryCmd.Run(r.client,
[]string{base.InProgressQueue, base.RetryQueue, processedKey, failureKey}, []string{base.InProgressQueue, base.RetryQueue, processedKey, failureKey},
string(bytesToRemove), string(bytesToAdd), processAt.Unix(), expireAt.Unix()).Err() string(bytesToRemove), string(bytesToAdd), processAt.Unix(), expireAt.Unix()).Err()
} }
@ -220,6 +225,31 @@ const (
deadExpirationInDays = 90 deadExpirationInDays = 90
) )
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:dead
// KEYS[3] -> asynq:processed:<yyyy-mm-dd>
// KEYS[4] -> asynq.failure:<yyyy-mm-dd>
// ARGV[1] -> base.TaskMessage value to remove from base.InProgressQueue queue
// ARGV[2] -> base.TaskMessage value to add to Dead queue
// ARGV[3] -> died_at UNIX timestamp
// ARGV[4] -> cutoff timestamp (e.g., 90 days ago)
// ARGV[5] -> max number of tasks in dead queue (e.g., 100)
// ARGV[6] -> stats expiration timestamp
var killCmd = redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
redis.call("ZREMRANGEBYSCORE", KEYS[2], "-inf", ARGV[4])
redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[5])
local n = redis.call("INCR", KEYS[3])
if tonumber(n) == 1 then
redis.call("EXPIREAT", KEYS[3], ARGV[6])
end
local m = redis.call("INCR", KEYS[4])
if tonumber(m) == 1 then
redis.call("EXPIREAT", KEYS[4], ARGV[6])
end
return redis.status_reply("OK")`)
// Kill sends the task to "dead" queue from in-progress queue, assigning // Kill sends the task to "dead" queue from in-progress queue, assigning
// the error message to the task. // the error message to the task.
// It also trims the set by timestamp and set size. // It also trims the set by timestamp and set size.
@ -239,50 +269,27 @@ func (r *RDB) Kill(msg *base.TaskMessage, errMsg string) error {
processedKey := base.ProcessedKey(now) processedKey := base.ProcessedKey(now)
failureKey := base.FailureKey(now) failureKey := base.FailureKey(now)
expireAt := now.Add(statsTTL) expireAt := now.Add(statsTTL)
// KEYS[1] -> asynq:in_progress return killCmd.Run(r.client,
// KEYS[2] -> asynq:dead
// KEYS[3] -> asynq:processed:<yyyy-mm-dd>
// KEYS[4] -> asynq.failure:<yyyy-mm-dd>
// ARGV[1] -> base.TaskMessage value to remove from base.InProgressQueue queue
// ARGV[2] -> base.TaskMessage value to add to Dead queue
// ARGV[3] -> died_at UNIX timestamp
// ARGV[4] -> cutoff timestamp (e.g., 90 days ago)
// ARGV[5] -> max number of tasks in dead queue (e.g., 100)
// ARGV[6] -> stats expiration timestamp
script := redis.NewScript(`
redis.call("LREM", KEYS[1], 0, ARGV[1])
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
redis.call("ZREMRANGEBYSCORE", KEYS[2], "-inf", ARGV[4])
redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[5])
local n = redis.call("INCR", KEYS[3])
if tonumber(n) == 1 then
redis.call("EXPIREAT", KEYS[3], ARGV[6])
end
local m = redis.call("INCR", KEYS[4])
if tonumber(m) == 1 then
redis.call("EXPIREAT", KEYS[4], ARGV[6])
end
return redis.status_reply("OK")
`)
return script.Run(r.client,
[]string{base.InProgressQueue, base.DeadQueue, processedKey, failureKey}, []string{base.InProgressQueue, base.DeadQueue, processedKey, failureKey},
string(bytesToRemove), string(bytesToAdd), now.Unix(), limit, maxDeadTasks, expireAt.Unix()).Err() string(bytesToRemove), string(bytesToAdd), now.Unix(), limit, maxDeadTasks, expireAt.Unix()).Err()
} }
// RestoreUnfinished moves all tasks from in-progress list to the queue // KEYS[1] -> asynq:in_progress
// and reports the number of tasks restored. // ARGV[1] -> queue prefix
func (r *RDB) RestoreUnfinished() (int64, error) { var requeueAllCmd = redis.NewScript(`
script := redis.NewScript(` local msgs = redis.call("LRANGE", KEYS[1], 0, -1)
local msgs = redis.call("LRANGE", KEYS[1], 0, -1) for _, msg in ipairs(msgs) do
for _, msg in ipairs(msgs) do
local decoded = cjson.decode(msg) local decoded = cjson.decode(msg)
local qkey = ARGV[1] .. decoded["Queue"] local qkey = ARGV[1] .. decoded["Queue"]
redis.call("LREM", KEYS[1], 0, msg)
redis.call("RPUSH", qkey, msg) redis.call("RPUSH", qkey, msg)
end redis.call("LREM", KEYS[1], 0, msg)
return table.getn(msgs) end
`) return table.getn(msgs)`)
res, err := script.Run(r.client, []string{base.InProgressQueue}, base.QueuePrefix).Result()
// RequeueAll moves all tasks from in-progress list to the queue
// and reports the number of tasks restored.
func (r *RDB) RequeueAll() (int64, error) {
res, err := requeueAllCmd.Run(r.client, []string{base.InProgressQueue}, base.QueuePrefix).Result()
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -313,40 +320,55 @@ func (r *RDB) CheckAndEnqueue(qnames ...string) error {
return nil return nil
} }
// forward moves all tasks with a score less than the current unix time // KEYS[1] -> source queue (e.g. scheduled or retry queue)
// from the src zset. // ARGV[1] -> current unix time
func (r *RDB) forward(src string) error { // ARGV[2] -> queue prefix
script := redis.NewScript(` var forwardCmd = redis.NewScript(`
local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1]) local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
for _, msg in ipairs(msgs) do for _, msg in ipairs(msgs) do
redis.call("ZREM", KEYS[1], msg)
local decoded = cjson.decode(msg) local decoded = cjson.decode(msg)
local qkey = ARGV[2] .. decoded["Queue"] local qkey = ARGV[2] .. decoded["Queue"]
redis.call("LPUSH", qkey, msg) redis.call("LPUSH", qkey, msg)
end redis.call("ZREM", KEYS[1], msg)
return msgs end
`) return msgs`)
// forward moves all tasks with a score less than the current unix time
// from the src zset.
func (r *RDB) forward(src string) error {
now := float64(time.Now().Unix()) now := float64(time.Now().Unix())
return script.Run(r.client, return forwardCmd.Run(r.client,
[]string{src}, now, base.QueuePrefix).Err() []string{src}, now, base.QueuePrefix).Err()
} }
// KEYS[1] -> source queue (e.g. scheduled or retry queue)
// KEYS[2] -> destination queue
var forwardSingleCmd = redis.NewScript(`
local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
for _, msg in ipairs(msgs) do
redis.call("LPUSH", KEYS[2], msg)
redis.call("ZREM", KEYS[1], msg)
end
return msgs`)
// forwardSingle moves all tasks with a score less than the current unix time // forwardSingle moves all tasks with a score less than the current unix time
// from the src zset to dst list. // from the src zset to dst list.
func (r *RDB) forwardSingle(src, dst string) error { func (r *RDB) forwardSingle(src, dst string) error {
script := redis.NewScript(`
local msgs = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
for _, msg in ipairs(msgs) do
redis.call("ZREM", KEYS[1], msg)
redis.call("LPUSH", KEYS[2], msg)
end
return msgs
`)
now := float64(time.Now().Unix()) now := float64(time.Now().Unix())
return script.Run(r.client, return forwardSingleCmd.Run(r.client,
[]string{src, dst}, now).Err() []string{src, dst}, now).Err()
} }
// KEYS[1] -> asynq:ps
// KEYS[2] -> asynq:ps:<host:pid>
// ARGV[1] -> expiration time
// ARGV[2] -> TTL in seconds
// ARGV[3] -> process info
var writeProcessInfoCmd = redis.NewScript(`
redis.call("ZADD", KEYS[1], ARGV[1], KEYS[2])
redis.call("SETEX", KEYS[2], ARGV[2], ARGV[3])
return redis.status_reply("OK")`)
// WriteProcessInfo writes process information to redis with expiration // WriteProcessInfo writes process information to redis with expiration
// set to the value ttl. // set to the value ttl.
func (r *RDB) WriteProcessInfo(ps *base.ProcessInfo, ttl time.Duration) error { func (r *RDB) WriteProcessInfo(ps *base.ProcessInfo, ttl time.Duration) error {
@ -358,17 +380,7 @@ func (r *RDB) WriteProcessInfo(ps *base.ProcessInfo, ttl time.Duration) error {
// ref: https://github.com/antirez/redis/issues/135#issuecomment-2361996 // ref: https://github.com/antirez/redis/issues/135#issuecomment-2361996
exp := time.Now().Add(ttl).UTC() exp := time.Now().Add(ttl).UTC()
key := base.ProcessInfoKey(ps.Host, ps.PID) key := base.ProcessInfoKey(ps.Host, ps.PID)
// KEYS[1] -> asynq:ps return writeProcessInfoCmd.Run(r.client, []string{base.AllProcesses, key}, float64(exp.Unix()), ttl.Seconds(), string(bytes)).Err()
// KEYS[2] -> asynq:ps:<host:pid>
// ARGV[1] -> expiration time
// ARGV[2] -> TTL in seconds
// ARGV[3] -> process info
script := redis.NewScript(`
redis.call("ZADD", KEYS[1], ARGV[1], KEYS[2])
redis.call("SETEX", KEYS[2], ARGV[2], ARGV[3])
return redis.status_reply("OK")
`)
return script.Run(r.client, []string{base.AllProcesses, key}, float64(exp.Unix()), ttl.Seconds(), string(bytes)).Err()
} }
// ReadProcessInfo reads process information stored in redis. // ReadProcessInfo reads process information stored in redis.
@ -386,13 +398,15 @@ func (r *RDB) ReadProcessInfo(host string, pid int) (*base.ProcessInfo, error) {
return &pinfo, nil return &pinfo, nil
} }
// KEYS[1] -> asynq:ps
// KEYS[2] -> asynq:ps:<host:pid>
var clearProcessInfoCmd = redis.NewScript(`
redis.call("ZREM", KEYS[1], KEYS[2])
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")`)
// ClearProcessInfo deletes process information from redis. // ClearProcessInfo deletes process information from redis.
func (r *RDB) ClearProcessInfo(ps *base.ProcessInfo) error { func (r *RDB) ClearProcessInfo(ps *base.ProcessInfo) error {
key := base.ProcessInfoKey(ps.Host, ps.PID) key := base.ProcessInfoKey(ps.Host, ps.PID)
script := redis.NewScript(` return clearProcessInfoCmd.Run(r.client, []string{base.AllProcesses, key}).Err()
redis.call("ZREM", KEYS[1], KEYS[2])
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")
`)
return script.Run(r.client, []string{base.AllProcesses, key}).Err()
} }

View File

@ -541,7 +541,7 @@ func TestKill(t *testing.T) {
} }
} }
func TestRestoreUnfinished(t *testing.T) { func TestRequeueAll(t *testing.T) {
r := setup(t) r := setup(t)
t1 := h.NewTaskMessage("send_email", nil) t1 := h.NewTaskMessage("send_email", nil)
t2 := h.NewTaskMessage("export_csv", nil) t2 := h.NewTaskMessage("export_csv", nil)
@ -613,9 +613,9 @@ func TestRestoreUnfinished(t *testing.T) {
h.SeedEnqueuedQueue(t, r.client, msgs, qname) h.SeedEnqueuedQueue(t, r.client, msgs, qname)
} }
got, err := r.RestoreUnfinished() got, err := r.RequeueAll()
if got != tc.want || err != nil { if got != tc.want || err != nil {
t.Errorf("(*RDB).RestoreUnfinished() = %v %v, want %v nil", got, err, tc.want) t.Errorf("(*RDB).RequeueAll() = %v %v, want %v nil", got, err, tc.want)
continue continue
} }

View File

@ -191,7 +191,7 @@ func (p *processor) exec() {
// restore moves all tasks from "in-progress" back to queue // restore moves all tasks from "in-progress" back to queue
// to restore all unfinished tasks. // to restore all unfinished tasks.
func (p *processor) restore() { func (p *processor) restore() {
n, err := p.rdb.RestoreUnfinished() n, err := p.rdb.RequeueAll()
if err != nil { if err != nil {
logger.error("Could not restore unfinished tasks: %v", err) logger.error("Could not restore unfinished tasks: %v", err)
} }