2
0
mirror of https://github.com/hibiken/asynq.git synced 2024-09-20 19:06:46 +08:00
asynq/internal/rdb/rdb.go

320 lines
9.7 KiB
Go
Raw Normal View History

2020-01-03 10:13:16 +08:00
// Copyright 2020 Kentaro Hibino. All rights reserved.
// Use of this source code is governed by a MIT license
// that can be found in the LICENSE file.
2019-12-04 22:25:58 +08:00
// Package rdb encapsulates the interactions with redis.
2019-12-04 13:01:26 +08:00
package rdb
2019-11-20 11:44:41 +08:00
import (
"encoding/json"
"errors"
"fmt"
"time"
"github.com/go-redis/redis/v7"
2019-12-22 23:15:45 +08:00
"github.com/hibiken/asynq/internal/base"
2019-11-20 11:44:41 +08:00
)
var (
// ErrDequeueTimeout indicates that the blocking dequeue operation timed out.
ErrDequeueTimeout = errors.New("blocking dequeue operation timed out")
// ErrTaskNotFound indicates that a task that matches the given identifier was not found.
ErrTaskNotFound = errors.New("could not find a task")
)
2019-11-20 11:44:41 +08:00
const statsTTL = 90 * 24 * time.Hour // 90 days
2019-12-04 22:25:58 +08:00
// RDB is a client interface to query and mutate task queues.
2019-12-04 13:01:26 +08:00
type RDB struct {
2019-11-20 11:44:41 +08:00
client *redis.Client
}
2019-12-04 13:01:26 +08:00
// NewRDB returns a new instance of RDB.
func NewRDB(client *redis.Client) *RDB {
return &RDB{client}
}
// Close closes the connection with redis server.
func (r *RDB) Close() error {
return r.client.Close()
2019-11-20 11:44:41 +08:00
}
2019-12-28 12:37:15 +08:00
// Enqueue inserts the given task to the tail of the queue.
2019-12-22 23:15:45 +08:00
func (r *RDB) Enqueue(msg *base.TaskMessage) error {
2019-11-20 11:44:41 +08:00
bytes, err := json.Marshal(msg)
if err != nil {
2019-12-28 12:37:15 +08:00
return err
2019-11-20 11:44:41 +08:00
}
2019-12-22 23:15:45 +08:00
qname := base.QueuePrefix + msg.Queue
2019-12-28 12:37:15 +08:00
return r.client.LPush(qname, string(bytes)).Err()
2019-11-20 11:44:41 +08:00
}
2019-12-04 13:01:26 +08:00
// Dequeue blocks until there is a task available to be processed,
2019-12-28 12:37:15 +08:00
// once a task is available, it adds the task to "in progress" queue
// and returns the task. If there are no tasks for the entire timeout
// duration, it returns ErrDequeueTimeout.
2019-12-22 23:15:45 +08:00
func (r *RDB) Dequeue(timeout time.Duration) (*base.TaskMessage, error) {
data, err := r.client.BRPopLPush(base.DefaultQueue, base.InProgressQueue, timeout).Result()
2019-11-28 11:36:56 +08:00
if err == redis.Nil {
2019-12-04 13:01:26 +08:00
return nil, ErrDequeueTimeout
2019-11-28 11:36:56 +08:00
}
2019-11-20 11:44:41 +08:00
if err != nil {
2019-12-28 12:37:15 +08:00
return nil, err
2019-11-20 11:44:41 +08:00
}
2019-12-22 23:15:45 +08:00
var msg base.TaskMessage
2019-11-20 11:44:41 +08:00
err = json.Unmarshal([]byte(data), &msg)
if err != nil {
2019-12-28 12:37:15 +08:00
return nil, err
2019-11-20 11:44:41 +08:00
}
return &msg, nil
}
2019-12-04 22:33:05 +08:00
// Done removes the task from in-progress queue to mark the task as done.
2019-12-22 23:15:45 +08:00
func (r *RDB) Done(msg *base.TaskMessage) error {
2019-11-22 13:45:27 +08:00
bytes, err := json.Marshal(msg)
if err != nil {
2019-12-28 12:37:15 +08:00
return err
2019-11-22 13:45:27 +08:00
}
// Note: LREM count ZERO means "remove all elements equal to val"
// Note: Script will try removing the message by exact match first,
2019-12-31 12:10:34 +08:00
// if the task is mutated and exact match is not found, it'll fallback
// to finding a match with ID.
// 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(`
local x = redis.call("LREM", KEYS[1], 0, ARGV[1])
if tonumber(x) == 0 then
local target = cjson.decode(ARGV[1])
local data = redis.call("LRANGE", KEYS[1], 0, -1)
for _, s in ipairs(data) do
local msg = cjson.decode(s)
if target["ID"] == msg["ID"] then
redis.call("LREM", KEYS[1], 0, s)
break
end
end
end
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()
processedKey := base.ProcessedKey(now)
expireAt := now.Add(statsTTL)
2019-12-28 12:37:15 +08:00
return script.Run(r.client,
[]string{base.InProgressQueue, processedKey},
2019-12-28 12:37:15 +08:00
string(bytes), expireAt.Unix()).Err()
2019-11-22 13:45:27 +08:00
}
// Requeue moves the task from in-progress queue to the default
// queue.
2019-12-22 23:15:45 +08:00
func (r *RDB) Requeue(msg *base.TaskMessage) error {
bytes, err := json.Marshal(msg)
if err != nil {
2019-12-28 12:37:15 +08:00
return err
}
2019-12-18 12:34:56 +08:00
// Note: Use RPUSH to push to the head of the queue.
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:queues:default
2019-12-22 23:15:45 +08:00
// 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")
`)
2019-12-28 12:37:15 +08:00
return script.Run(r.client,
[]string{base.InProgressQueue, base.DefaultQueue},
string(bytes)).Err()
}
// Schedule adds the task to the backlog queue to be processed in the future.
2019-12-22 23:15:45 +08:00
func (r *RDB) Schedule(msg *base.TaskMessage, processAt time.Time) error {
bytes, err := json.Marshal(msg)
if err != nil {
2019-12-28 12:37:15 +08:00
return err
}
score := float64(processAt.Unix())
2019-12-28 12:37:15 +08:00
return r.client.ZAdd(base.ScheduledQueue,
&redis.Z{Member: string(bytes), Score: score}).Err()
}
// Retry moves the task from in-progress to retry queue, incrementing retry count
// and assigning error message to the task message.
2019-12-22 23:15:45 +08:00
func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string) error {
bytesToRemove, err := json.Marshal(msg)
if err != nil {
2019-12-29 02:51:37 +08:00
return err
}
modified := *msg
modified.Retried++
modified.ErrorMsg = errMsg
bytesToAdd, err := json.Marshal(&modified)
if err != nil {
2019-12-29 02:51:37 +08:00
return err
}
// Note: Script will try removing the message by exact match first,
2019-12-31 12:10:34 +08:00
// if the task is mutated and exact match is not found, it'll fallback
// to finding a match with ID.
// 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(`
local x = redis.call("LREM", KEYS[1], 0, ARGV[1])
if tonumber(x) == 0 then
local target = cjson.decode(ARGV[1])
local data = redis.call("LRANGE", KEYS[1], 0, -1)
for _, s in ipairs(data) do
local msg = cjson.decode(s)
if target["ID"] == msg["ID"] then
redis.call("LREM", KEYS[1], 0, s)
break
end
end
end
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()
processedKey := base.ProcessedKey(now)
failureKey := base.FailureKey(now)
expireAt := now.Add(statsTTL)
2019-12-29 02:51:37 +08:00
return script.Run(r.client,
[]string{base.InProgressQueue, base.RetryQueue, processedKey, failureKey},
2019-12-29 02:51:37 +08:00
string(bytesToRemove), string(bytesToAdd), processAt.Unix(), expireAt.Unix()).Err()
}
const (
maxDeadTasks = 10000
deadExpirationInDays = 90
)
// Kill sends the task to "dead" queue from in-progress queue, assigning
// the error message to the task.
2019-12-04 22:50:52 +08:00
// It also trims the set by timestamp and set size.
2019-12-22 23:15:45 +08:00
func (r *RDB) Kill(msg *base.TaskMessage, errMsg string) error {
bytesToRemove, err := json.Marshal(msg)
2019-11-20 11:44:41 +08:00
if err != nil {
2019-12-29 02:51:37 +08:00
return err
2019-11-20 11:44:41 +08:00
}
modified := *msg
modified.ErrorMsg = errMsg
bytesToAdd, err := json.Marshal(&modified)
if err != nil {
2019-12-29 02:51:37 +08:00
return err
}
2019-11-20 11:44:41 +08:00
now := time.Now()
limit := now.AddDate(0, 0, -deadExpirationInDays).Unix() // 90 days ago
processedKey := base.ProcessedKey(now)
failureKey := base.FailureKey(now)
expireAt := now.Add(statsTTL)
// Note: Script will try removing the message by exact match first,
2019-12-31 12:10:34 +08:00
// if the task is mutated and exact match is not found, it'll fallback
// to finding a match with ID.
// KEYS[1] -> asynq:in_progress
// KEYS[2] -> asynq:dead
// KEYS[3] -> asynq:processed:<yyyy-mm-dd>
// KEYS[4] -> asynq.failure:<yyyy-mm-dd>
2019-12-22 23:15:45 +08:00
// 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(`
local x = redis.call("LREM", KEYS[1], 0, ARGV[1])
if tonumber(x) == 0 then
local target = cjson.decode(ARGV[1])
local data = redis.call("LRANGE", KEYS[1], 0, -1)
for _, s in ipairs(data) do
local msg = cjson.decode(s)
if target["ID"] == msg["ID"] then
redis.call("LREM", KEYS[1], 0, s)
break
end
end
end
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")
`)
2019-12-29 02:51:37 +08:00
return script.Run(r.client,
[]string{base.InProgressQueue, base.DeadQueue, processedKey, failureKey},
2019-12-29 02:51:37 +08:00
string(bytesToRemove), string(bytesToAdd), now.Unix(), limit, maxDeadTasks, expireAt.Unix()).Err()
2019-11-20 11:44:41 +08:00
}
// RestoreUnfinished moves all tasks from in-progress list to the queue
// and reports the number of tasks restored.
func (r *RDB) RestoreUnfinished() (int64, error) {
script := redis.NewScript(`
local len = redis.call("LLEN", KEYS[1])
for i = len, 1, -1 do
redis.call("RPOPLPUSH", KEYS[1], KEYS[2])
end
return len
`)
2019-12-22 23:15:45 +08:00
res, err := script.Run(r.client, []string{base.InProgressQueue, base.DefaultQueue}).Result()
if err != nil {
return 0, err
}
n, ok := res.(int64)
if !ok {
return 0, fmt.Errorf("could not cast %v to int64", res)
}
return n, nil
}
2019-12-04 23:28:57 +08:00
// CheckAndEnqueue checks for all scheduled tasks and enqueues any tasks that
// have to be processed.
func (r *RDB) CheckAndEnqueue() error {
2019-12-22 23:15:45 +08:00
delayed := []string{base.ScheduledQueue, base.RetryQueue}
2019-12-04 23:14:37 +08:00
for _, zset := range delayed {
if err := r.forward(zset); err != nil {
return err
}
}
return nil
}
// forward moves all tasks with a score less than the current unix time
// from the given zset to the default queue.
2019-12-04 23:14:37 +08:00
func (r *RDB) forward(from 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())
2019-12-29 02:51:37 +08:00
return script.Run(r.client,
[]string{from, base.DefaultQueue}, now).Err()
2019-12-07 14:29:40 +08:00
}