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 (
|
2021-09-02 20:56:02 +08:00
|
|
|
"context"
|
2019-11-20 11:44:41 +08:00
|
|
|
"fmt"
|
2021-12-19 22:58:12 +08:00
|
|
|
"math"
|
2019-11-20 11:44:41 +08:00
|
|
|
"time"
|
|
|
|
|
2021-09-02 20:56:02 +08:00
|
|
|
"github.com/go-redis/redis/v8"
|
2022-03-10 09:05:16 +08:00
|
|
|
"github.com/google/uuid"
|
2019-12-22 23:15:45 +08:00
|
|
|
"github.com/hibiken/asynq/internal/base"
|
2021-05-09 22:39:01 +08:00
|
|
|
"github.com/hibiken/asynq/internal/errors"
|
2021-12-11 01:07:41 +08:00
|
|
|
"github.com/hibiken/asynq/internal/timeutil"
|
2020-01-07 22:28:34 +08:00
|
|
|
"github.com/spf13/cast"
|
2019-11-20 11:44:41 +08:00
|
|
|
)
|
|
|
|
|
2019-12-23 21:33:48 +08:00
|
|
|
const statsTTL = 90 * 24 * time.Hour // 90 days
|
|
|
|
|
2022-02-12 22:49:53 +08:00
|
|
|
// LeaseDuration is the duration used to initially create a lease and to extend it thereafter.
|
|
|
|
const LeaseDuration = 30 * time.Second
|
2022-02-08 22:47:31 +08:00
|
|
|
|
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 {
|
2020-08-27 22:19:21 +08:00
|
|
|
client redis.UniversalClient
|
2021-12-11 01:07:41 +08:00
|
|
|
clock timeutil.Clock
|
2019-11-20 11:44:41 +08:00
|
|
|
}
|
|
|
|
|
2019-12-04 13:01:26 +08:00
|
|
|
// NewRDB returns a new instance of RDB.
|
2020-08-27 22:19:21 +08:00
|
|
|
func NewRDB(client redis.UniversalClient) *RDB {
|
2021-12-11 01:07:41 +08:00
|
|
|
return &RDB{
|
|
|
|
client: client,
|
|
|
|
clock: timeutil.NewRealClock(),
|
|
|
|
}
|
2019-12-04 13:01:26 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Close closes the connection with redis server.
|
|
|
|
func (r *RDB) Close() error {
|
|
|
|
return r.client.Close()
|
2019-11-20 11:44:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-25 21:37:58 +08:00
|
|
|
// Client returns the reference to underlying redis client.
|
|
|
|
func (r *RDB) Client() redis.UniversalClient {
|
|
|
|
return r.client
|
|
|
|
}
|
|
|
|
|
2021-12-11 01:07:41 +08:00
|
|
|
// SetClock sets the clock used by RDB to the given clock.
|
|
|
|
//
|
|
|
|
// Use this function to set the clock to SimulatedClock in tests.
|
|
|
|
func (r *RDB) SetClock(c timeutil.Clock) {
|
|
|
|
r.clock = c
|
|
|
|
}
|
|
|
|
|
2020-07-26 09:49:27 +08:00
|
|
|
// Ping checks the connection with redis server.
|
|
|
|
func (r *RDB) Ping() error {
|
2021-09-02 20:56:02 +08:00
|
|
|
return r.client.Ping(context.Background()).Err()
|
2020-07-26 09:49:27 +08:00
|
|
|
}
|
|
|
|
|
2021-11-16 08:34:26 +08:00
|
|
|
func (r *RDB) runScript(ctx context.Context, op errors.Op, script *redis.Script, keys []string, args ...interface{}) error {
|
|
|
|
if err := script.Run(ctx, r.client, keys, args...).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, fmt.Sprintf("redis eval error: %v", err))
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-09-11 07:47:00 +08:00
|
|
|
// Runs the given script with keys and args and retuns the script's return value as int64.
|
2021-11-16 08:34:26 +08:00
|
|
|
func (r *RDB) runScriptWithErrorCode(ctx context.Context, op errors.Op, script *redis.Script, keys []string, args ...interface{}) (int64, error) {
|
|
|
|
res, err := script.Run(ctx, r.client, keys, args...).Result()
|
2021-09-11 07:47:00 +08:00
|
|
|
if err != nil {
|
|
|
|
return 0, errors.E(op, errors.Unknown, fmt.Sprintf("redis eval error: %v", err))
|
|
|
|
}
|
|
|
|
n, ok := res.(int64)
|
|
|
|
if !ok {
|
|
|
|
return 0, errors.E(op, errors.Internal, fmt.Sprintf("unexpected return value from Lua script: %v", res))
|
|
|
|
}
|
|
|
|
return n, nil
|
|
|
|
}
|
|
|
|
|
2021-05-11 00:16:13 +08:00
|
|
|
// enqueueCmd enqueues a given task message.
|
|
|
|
//
|
|
|
|
// Input:
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:pending
|
2021-05-11 00:16:13 +08:00
|
|
|
// --
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task message data
|
|
|
|
// ARGV[2] -> task ID
|
2022-02-16 22:17:08 +08:00
|
|
|
// ARGV[3] -> current unix time in nsec
|
2021-05-11 00:16:13 +08:00
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 1 if successfully enqueued
|
2021-09-11 07:47:00 +08:00
|
|
|
// Returns 0 if task ID already exists
|
2021-03-13 08:23:08 +08:00
|
|
|
var enqueueCmd = redis.NewScript(`
|
2021-09-11 07:47:00 +08:00
|
|
|
if redis.call("EXISTS", KEYS[1]) == 1 then
|
|
|
|
return 0
|
|
|
|
end
|
2021-04-18 22:19:19 +08:00
|
|
|
redis.call("HSET", KEYS[1],
|
|
|
|
"msg", ARGV[1],
|
|
|
|
"state", "pending",
|
2022-02-16 22:17:08 +08:00
|
|
|
"pending_since", ARGV[3])
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("LPUSH", KEYS[2], ARGV[2])
|
|
|
|
return 1
|
|
|
|
`)
|
|
|
|
|
|
|
|
// Enqueue adds the given task to the pending list of the queue.
|
2021-11-16 08:34:26 +08:00
|
|
|
func (r *RDB) Enqueue(ctx context.Context, msg *base.TaskMessage) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.Enqueue"
|
2020-06-12 11:58:27 +08:00
|
|
|
encoded, err := base.EncodeMessage(msg)
|
2019-11-20 11:44:41 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
2019-11-20 11:44:41 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
2020-08-07 21:31:02 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
keys := []string{
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.PendingKey(msg.Queue),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
|
|
|
encoded,
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-12-11 22:27:44 +08:00
|
|
|
r.clock.Now().UnixNano(),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
n, err := r.runScriptWithErrorCode(ctx, op, enqueueCmd, keys, argv...)
|
2021-09-11 07:47:00 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if n == 0 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
|
|
|
}
|
|
|
|
return nil
|
2019-11-20 11:44:41 +08:00
|
|
|
}
|
|
|
|
|
2021-05-11 00:16:13 +08:00
|
|
|
// enqueueUniqueCmd enqueues the task message if the task is unique.
|
|
|
|
//
|
2020-08-07 21:31:02 +08:00
|
|
|
// KEYS[1] -> unique key
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:t:<taskid>
|
|
|
|
// KEYS[3] -> asynq:{<qname>}:pending
|
2021-05-11 00:16:13 +08:00
|
|
|
// --
|
2020-03-18 21:49:39 +08:00
|
|
|
// ARGV[1] -> task ID
|
|
|
|
// ARGV[2] -> uniqueness lock TTL
|
|
|
|
// ARGV[3] -> task message data
|
2022-02-16 22:17:08 +08:00
|
|
|
// ARGV[4] -> current unix time in nsec
|
2021-05-11 00:16:13 +08:00
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 1 if successfully enqueued
|
2021-09-11 07:47:00 +08:00
|
|
|
// Returns 0 if task ID conflicts with another task
|
|
|
|
// Returns -1 if task unique key already exists
|
2020-03-18 21:49:39 +08:00
|
|
|
var enqueueUniqueCmd = redis.NewScript(`
|
|
|
|
local ok = redis.call("SET", KEYS[1], ARGV[1], "NX", "EX", ARGV[2])
|
|
|
|
if not ok then
|
2021-09-11 07:47:00 +08:00
|
|
|
return -1
|
|
|
|
end
|
|
|
|
if redis.call("EXISTS", KEYS[2]) == 1 then
|
2020-03-18 21:49:39 +08:00
|
|
|
return 0
|
|
|
|
end
|
2021-04-19 21:28:58 +08:00
|
|
|
redis.call("HSET", KEYS[2],
|
|
|
|
"msg", ARGV[3],
|
2021-05-10 10:20:54 +08:00
|
|
|
"state", "pending",
|
2022-02-16 22:17:08 +08:00
|
|
|
"pending_since", ARGV[4],
|
2021-06-09 21:06:43 +08:00
|
|
|
"unique_key", KEYS[1])
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("LPUSH", KEYS[3], ARGV[1])
|
2020-03-18 21:49:39 +08:00
|
|
|
return 1
|
|
|
|
`)
|
|
|
|
|
|
|
|
// EnqueueUnique inserts the given task if the task's uniqueness lock can be acquired.
|
|
|
|
// It returns ErrDuplicateTask if the lock cannot be acquired.
|
2021-11-16 08:34:26 +08:00
|
|
|
func (r *RDB) EnqueueUnique(ctx context.Context, msg *base.TaskMessage, ttl time.Duration) error {
|
2021-05-09 22:39:01 +08:00
|
|
|
var op errors.Op = "rdb.EnqueueUnique"
|
2020-06-12 11:58:27 +08:00
|
|
|
encoded, err := base.EncodeMessage(msg)
|
2020-03-18 21:49:39 +08:00
|
|
|
if err != nil {
|
2021-05-09 22:39:01 +08:00
|
|
|
return errors.E(op, errors.Internal, "cannot encode task message: %v", err)
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
2021-05-09 22:39:01 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
2020-08-07 21:31:02 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
keys := []string{
|
|
|
|
msg.UniqueKey,
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.PendingKey(msg.Queue),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-03-13 08:23:08 +08:00
|
|
|
int(ttl.Seconds()),
|
|
|
|
encoded,
|
2021-12-11 22:27:44 +08:00
|
|
|
r.clock.Now().UnixNano(),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
n, err := r.runScriptWithErrorCode(ctx, op, enqueueUniqueCmd, keys, argv...)
|
2020-03-18 21:49:39 +08:00
|
|
|
if err != nil {
|
2021-09-11 07:47:00 +08:00
|
|
|
return err
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
2021-09-11 07:47:00 +08:00
|
|
|
if n == -1 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrDuplicateTask)
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
|
|
|
if n == 0 {
|
2021-09-11 07:47:00 +08:00
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-05-10 21:04:42 +08:00
|
|
|
// Input:
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:pending
|
2020-08-27 22:19:21 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:paused
|
2020-09-06 03:43:15 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:active
|
2022-02-08 22:47:31 +08:00
|
|
|
// KEYS[4] -> asynq:{<qname>}:lease
|
2021-05-10 21:04:42 +08:00
|
|
|
// --
|
2022-02-08 22:47:31 +08:00
|
|
|
// ARGV[1] -> initial lease expiration Unix time
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[2] -> task key prefix
|
2020-06-04 20:37:17 +08:00
|
|
|
//
|
2021-05-10 21:04:42 +08:00
|
|
|
// Output:
|
|
|
|
// Returns nil if no processable task is found in the given queue.
|
2022-02-08 22:47:31 +08:00
|
|
|
// Returns an encoded TaskMessage.
|
2021-05-10 21:04:42 +08:00
|
|
|
//
|
|
|
|
// Note: dequeueCmd checks whether a queue is paused first, before
|
2020-06-04 20:37:17 +08:00
|
|
|
// calling RPOPLPUSH to pop a task from the queue.
|
2020-02-09 03:06:14 +08:00
|
|
|
var dequeueCmd = redis.NewScript(`
|
2020-08-27 22:19:21 +08:00
|
|
|
if redis.call("EXISTS", KEYS[2]) == 0 then
|
2021-03-13 08:23:08 +08:00
|
|
|
local id = redis.call("RPOPLPUSH", KEYS[1], KEYS[3])
|
|
|
|
if id then
|
|
|
|
local key = ARGV[2] .. id
|
2021-04-26 22:07:12 +08:00
|
|
|
redis.call("HSET", key, "state", "active")
|
2021-12-11 22:42:47 +08:00
|
|
|
redis.call("HDEL", key, "pending_since")
|
2022-02-08 22:47:31 +08:00
|
|
|
redis.call("ZADD", KEYS[4], ARGV[1], id)
|
|
|
|
return redis.call("HGET", key, "msg")
|
2020-02-09 03:06:14 +08:00
|
|
|
end
|
|
|
|
end
|
2020-06-04 20:37:17 +08:00
|
|
|
return nil`)
|
2020-02-09 03:06:14 +08:00
|
|
|
|
2021-05-10 21:04:42 +08:00
|
|
|
// Dequeue queries given queues in order and pops a task message
|
2022-02-14 06:34:38 +08:00
|
|
|
// off a queue if one exists and returns the message and its lease expiration time.
|
2021-05-10 21:04:42 +08:00
|
|
|
// Dequeue skips a queue if the queue is paused.
|
|
|
|
// If all queues are empty, ErrNoProcessableTask error is returned.
|
2022-02-14 06:34:38 +08:00
|
|
|
func (r *RDB) Dequeue(qnames ...string) (msg *base.TaskMessage, leaseExpirationTime time.Time, err error) {
|
2021-05-10 21:04:42 +08:00
|
|
|
var op errors.Op = "rdb.Dequeue"
|
2020-08-27 22:19:21 +08:00
|
|
|
for _, qname := range qnames {
|
|
|
|
keys := []string{
|
2021-03-13 08:23:08 +08:00
|
|
|
base.PendingKey(qname),
|
2020-08-27 22:19:21 +08:00
|
|
|
base.PausedKey(qname),
|
2020-09-06 03:43:15 +08:00
|
|
|
base.ActiveKey(qname),
|
2022-02-08 22:47:31 +08:00
|
|
|
base.LeaseKey(qname),
|
2020-08-27 22:19:21 +08:00
|
|
|
}
|
2022-02-14 06:34:38 +08:00
|
|
|
leaseExpirationTime = r.clock.Now().Add(LeaseDuration)
|
2021-03-13 08:23:08 +08:00
|
|
|
argv := []interface{}{
|
2022-02-14 06:34:38 +08:00
|
|
|
leaseExpirationTime.Unix(),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.TaskKeyPrefix(qname),
|
|
|
|
}
|
2021-09-02 20:56:02 +08:00
|
|
|
res, err := dequeueCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
2020-08-27 22:19:21 +08:00
|
|
|
if err == redis.Nil {
|
|
|
|
continue
|
|
|
|
} else if err != nil {
|
2022-02-14 06:34:38 +08:00
|
|
|
return nil, time.Time{}, errors.E(op, errors.Unknown, fmt.Sprintf("redis eval error: %v", err))
|
2021-05-10 21:04:42 +08:00
|
|
|
}
|
2022-02-08 22:47:31 +08:00
|
|
|
encoded, err := cast.ToStringE(res)
|
2021-05-10 21:04:42 +08:00
|
|
|
if err != nil {
|
2022-02-14 06:34:38 +08:00
|
|
|
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: unexpected return value from Lua script: %v", res))
|
2020-08-27 22:19:21 +08:00
|
|
|
}
|
2021-05-10 21:04:42 +08:00
|
|
|
if msg, err = base.DecodeMessage([]byte(encoded)); err != nil {
|
2022-02-14 06:34:38 +08:00
|
|
|
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cannot decode message: %v", err))
|
2020-08-27 22:19:21 +08:00
|
|
|
}
|
2022-02-14 06:34:38 +08:00
|
|
|
return msg, leaseExpirationTime, nil
|
2020-08-27 22:19:21 +08:00
|
|
|
}
|
2022-02-14 06:34:38 +08:00
|
|
|
return nil, time.Time{}, errors.E(op, errors.NotFound, errors.ErrNoProcessableTask)
|
2020-01-07 22:28:34 +08:00
|
|
|
}
|
|
|
|
|
2020-09-06 03:43:15 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:lease
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
2021-12-17 08:53:02 +08:00
|
|
|
// KEYS[5] -> asynq:{<qname>}:processed
|
2022-02-11 11:01:05 +08:00
|
|
|
// -------
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task ID
|
2020-08-28 20:37:40 +08:00
|
|
|
// ARGV[2] -> stats expiration timestamp
|
2021-12-17 08:53:02 +08:00
|
|
|
// ARGV[3] -> max int64 value
|
2020-08-28 20:37:40 +08:00
|
|
|
var doneCmd = redis.NewScript(`
|
|
|
|
if redis.call("LREM", KEYS[1], 0, ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
if redis.call("ZREM", KEYS[2], ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
if redis.call("DEL", KEYS[3]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
local n = redis.call("INCR", KEYS[4])
|
2020-08-28 20:37:40 +08:00
|
|
|
if tonumber(n) == 1 then
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("EXPIREAT", KEYS[4], ARGV[2])
|
2020-08-28 20:37:40 +08:00
|
|
|
end
|
2021-12-17 08:53:02 +08:00
|
|
|
local total = redis.call("GET", KEYS[5])
|
|
|
|
if tonumber(total) == tonumber(ARGV[3]) then
|
|
|
|
redis.call("SET", KEYS[5], 1)
|
|
|
|
else
|
|
|
|
redis.call("INCR", KEYS[5])
|
|
|
|
end
|
2020-08-28 20:37:40 +08:00
|
|
|
return redis.status_reply("OK")
|
|
|
|
`)
|
|
|
|
|
2020-09-06 03:43:15 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:lease
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
2021-12-17 08:53:02 +08:00
|
|
|
// KEYS[5] -> asynq:{<qname>}:processed
|
|
|
|
// KEYS[6] -> unique key
|
2022-02-11 11:01:05 +08:00
|
|
|
// -------
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task ID
|
2020-02-09 03:06:14 +08:00
|
|
|
// ARGV[2] -> stats expiration timestamp
|
2021-12-17 08:53:02 +08:00
|
|
|
// ARGV[3] -> max int64 value
|
2020-08-28 20:37:40 +08:00
|
|
|
var doneUniqueCmd = redis.NewScript(`
|
2020-06-19 03:12:29 +08:00
|
|
|
if redis.call("LREM", KEYS[1], 0, ARGV[1]) == 0 then
|
2020-06-08 04:04:27 +08:00
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2020-06-19 03:12:29 +08:00
|
|
|
if redis.call("ZREM", KEYS[2], ARGV[1]) == 0 then
|
2020-06-18 22:10:57 +08:00
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
if redis.call("DEL", KEYS[3]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
local n = redis.call("INCR", KEYS[4])
|
2020-02-09 03:06:14 +08:00
|
|
|
if tonumber(n) == 1 then
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("EXPIREAT", KEYS[4], ARGV[2])
|
2020-02-09 03:06:14 +08:00
|
|
|
end
|
2021-12-17 08:53:02 +08:00
|
|
|
local total = redis.call("GET", KEYS[5])
|
|
|
|
if tonumber(total) == tonumber(ARGV[3]) then
|
|
|
|
redis.call("SET", KEYS[5], 1)
|
|
|
|
else
|
|
|
|
redis.call("INCR", KEYS[5])
|
|
|
|
end
|
|
|
|
if redis.call("GET", KEYS[6]) == ARGV[1] then
|
|
|
|
redis.call("DEL", KEYS[6])
|
2020-03-18 21:49:39 +08:00
|
|
|
end
|
2020-02-09 03:06:14 +08:00
|
|
|
return redis.status_reply("OK")
|
|
|
|
`)
|
|
|
|
|
2021-11-06 07:52:54 +08:00
|
|
|
// Done removes the task from active queue and deletes the task.
|
2020-03-18 21:49:39 +08:00
|
|
|
// It removes a uniqueness lock acquired by the task, if any.
|
2022-02-14 06:34:38 +08:00
|
|
|
func (r *RDB) Done(ctx context.Context, msg *base.TaskMessage) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.Done"
|
2021-12-11 01:07:41 +08:00
|
|
|
now := r.clock.Now()
|
2019-12-23 21:33:48 +08:00
|
|
|
expireAt := now.Add(statsTTL)
|
2020-08-28 20:37:40 +08:00
|
|
|
keys := []string{
|
2020-09-06 03:43:15 +08:00
|
|
|
base.ActiveKey(msg.Queue),
|
2022-02-11 11:01:05 +08:00
|
|
|
base.LeaseKey(msg.Queue),
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2020-08-28 20:37:40 +08:00
|
|
|
base.ProcessedKey(msg.Queue, now),
|
2021-12-17 08:53:02 +08:00
|
|
|
base.ProcessedTotalKey(msg.Queue),
|
2020-08-28 20:37:40 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
argv := []interface{}{
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-03-13 08:23:08 +08:00
|
|
|
expireAt.Unix(),
|
2022-03-29 21:30:10 +08:00
|
|
|
int64(math.MaxInt64),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
2021-11-06 07:52:54 +08:00
|
|
|
// Note: We cannot pass empty unique key when running this script in redis-cluster.
|
2020-08-28 20:37:40 +08:00
|
|
|
if len(msg.UniqueKey) > 0 {
|
|
|
|
keys = append(keys, msg.UniqueKey)
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, doneUniqueCmd, keys, argv...)
|
2020-08-28 20:37:40 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, doneCmd, keys, argv...)
|
2019-11-22 13:45:27 +08:00
|
|
|
}
|
|
|
|
|
2021-11-06 07:52:54 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:lease
|
2021-11-06 07:52:54 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:completed
|
|
|
|
// KEYS[4] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
2021-12-17 08:53:02 +08:00
|
|
|
// KEYS[6] -> asynq:{<qname>}:processed
|
|
|
|
//
|
2021-11-06 07:52:54 +08:00
|
|
|
// ARGV[1] -> task ID
|
|
|
|
// ARGV[2] -> stats expiration timestamp
|
|
|
|
// ARGV[3] -> task exipration time in unix time
|
|
|
|
// ARGV[4] -> task message data
|
2021-12-17 08:53:02 +08:00
|
|
|
// ARGV[5] -> max int64 value
|
2021-11-06 07:52:54 +08:00
|
|
|
var markAsCompleteCmd = redis.NewScript(`
|
|
|
|
if redis.call("LREM", KEYS[1], 0, ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
if redis.call("ZREM", KEYS[2], ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
if redis.call("ZADD", KEYS[3], ARGV[3], ARGV[1]) ~= 1 then
|
|
|
|
redis.redis.error_reply("INTERNAL")
|
|
|
|
end
|
|
|
|
redis.call("HSET", KEYS[4], "msg", ARGV[4], "state", "completed")
|
|
|
|
local n = redis.call("INCR", KEYS[5])
|
|
|
|
if tonumber(n) == 1 then
|
|
|
|
redis.call("EXPIREAT", KEYS[5], ARGV[2])
|
|
|
|
end
|
2021-12-17 08:53:02 +08:00
|
|
|
local total = redis.call("GET", KEYS[6])
|
|
|
|
if tonumber(total) == tonumber(ARGV[5]) then
|
|
|
|
redis.call("SET", KEYS[6], 1)
|
|
|
|
else
|
|
|
|
redis.call("INCR", KEYS[6])
|
|
|
|
end
|
2021-11-06 07:52:54 +08:00
|
|
|
return redis.status_reply("OK")
|
|
|
|
`)
|
|
|
|
|
|
|
|
// KEYS[1] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:lease
|
2021-11-06 07:52:54 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:completed
|
|
|
|
// KEYS[4] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
2021-12-17 08:53:02 +08:00
|
|
|
// KEYS[6] -> asynq:{<qname>}:processed
|
|
|
|
// KEYS[7] -> asynq:{<qname>}:unique:{<checksum>}
|
|
|
|
//
|
2021-11-06 07:52:54 +08:00
|
|
|
// ARGV[1] -> task ID
|
|
|
|
// ARGV[2] -> stats expiration timestamp
|
|
|
|
// ARGV[3] -> task exipration time in unix time
|
|
|
|
// ARGV[4] -> task message data
|
2021-12-17 08:53:02 +08:00
|
|
|
// ARGV[5] -> max int64 value
|
2021-11-06 07:52:54 +08:00
|
|
|
var markAsCompleteUniqueCmd = redis.NewScript(`
|
|
|
|
if redis.call("LREM", KEYS[1], 0, ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
if redis.call("ZREM", KEYS[2], ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
if redis.call("ZADD", KEYS[3], ARGV[3], ARGV[1]) ~= 1 then
|
|
|
|
redis.redis.error_reply("INTERNAL")
|
|
|
|
end
|
|
|
|
redis.call("HSET", KEYS[4], "msg", ARGV[4], "state", "completed")
|
|
|
|
local n = redis.call("INCR", KEYS[5])
|
|
|
|
if tonumber(n) == 1 then
|
|
|
|
redis.call("EXPIREAT", KEYS[5], ARGV[2])
|
|
|
|
end
|
2021-12-17 08:53:02 +08:00
|
|
|
local total = redis.call("GET", KEYS[6])
|
|
|
|
if tonumber(total) == tonumber(ARGV[5]) then
|
|
|
|
redis.call("SET", KEYS[6], 1)
|
|
|
|
else
|
|
|
|
redis.call("INCR", KEYS[6])
|
|
|
|
end
|
|
|
|
if redis.call("GET", KEYS[7]) == ARGV[1] then
|
|
|
|
redis.call("DEL", KEYS[7])
|
2021-11-06 07:52:54 +08:00
|
|
|
end
|
|
|
|
return redis.status_reply("OK")
|
|
|
|
`)
|
|
|
|
|
|
|
|
// MarkAsComplete removes the task from active queue to mark the task as completed.
|
|
|
|
// It removes a uniqueness lock acquired by the task, if any.
|
2022-02-14 06:34:38 +08:00
|
|
|
func (r *RDB) MarkAsComplete(ctx context.Context, msg *base.TaskMessage) error {
|
2021-11-06 07:52:54 +08:00
|
|
|
var op errors.Op = "rdb.MarkAsComplete"
|
2021-12-11 01:07:41 +08:00
|
|
|
now := r.clock.Now()
|
2021-11-06 07:52:54 +08:00
|
|
|
statsExpireAt := now.Add(statsTTL)
|
|
|
|
msg.CompletedAt = now.Unix()
|
|
|
|
encoded, err := base.EncodeMessage(msg)
|
|
|
|
if err != nil {
|
|
|
|
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
|
|
|
}
|
|
|
|
keys := []string{
|
|
|
|
base.ActiveKey(msg.Queue),
|
2022-02-11 11:01:05 +08:00
|
|
|
base.LeaseKey(msg.Queue),
|
2021-11-06 07:52:54 +08:00
|
|
|
base.CompletedKey(msg.Queue),
|
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
|
|
|
base.ProcessedKey(msg.Queue, now),
|
2021-12-17 08:53:02 +08:00
|
|
|
base.ProcessedTotalKey(msg.Queue),
|
2021-11-06 07:52:54 +08:00
|
|
|
}
|
|
|
|
argv := []interface{}{
|
|
|
|
msg.ID,
|
|
|
|
statsExpireAt.Unix(),
|
|
|
|
now.Unix() + msg.Retention,
|
|
|
|
encoded,
|
2022-03-29 21:30:10 +08:00
|
|
|
int64(math.MaxInt64),
|
2021-11-06 07:52:54 +08:00
|
|
|
}
|
|
|
|
// Note: We cannot pass empty unique key when running this script in redis-cluster.
|
|
|
|
if len(msg.UniqueKey) > 0 {
|
|
|
|
keys = append(keys, msg.UniqueKey)
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, markAsCompleteUniqueCmd, keys, argv...)
|
2021-11-06 07:52:54 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, markAsCompleteCmd, keys, argv...)
|
2021-11-06 07:52:54 +08:00
|
|
|
}
|
|
|
|
|
2020-09-06 03:43:15 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:lease
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:pending
|
2021-04-26 22:13:48 +08:00
|
|
|
// KEYS[4] -> asynq:{<qname>}:t:<task_id>
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task ID
|
2020-02-09 03:06:14 +08:00
|
|
|
// Note: Use RPUSH to push to the head of the queue.
|
|
|
|
var requeueCmd = redis.NewScript(`
|
2020-06-19 03:12:29 +08:00
|
|
|
if redis.call("LREM", KEYS[1], 0, ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
if redis.call("ZREM", KEYS[2], ARGV[1]) == 0 then
|
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
|
|
|
redis.call("RPUSH", KEYS[3], ARGV[1])
|
2021-04-26 22:13:48 +08:00
|
|
|
redis.call("HSET", KEYS[4], "state", "pending")
|
2020-02-09 03:06:14 +08:00
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
2020-09-06 03:43:15 +08:00
|
|
|
// Requeue moves the task from active queue to the specified queue.
|
2022-02-14 06:34:38 +08:00
|
|
|
func (r *RDB) Requeue(ctx context.Context, msg *base.TaskMessage) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.Requeue"
|
2021-04-26 22:13:48 +08:00
|
|
|
keys := []string{
|
|
|
|
base.ActiveKey(msg.Queue),
|
2022-02-11 11:01:05 +08:00
|
|
|
base.LeaseKey(msg.Queue),
|
2021-04-26 22:13:48 +08:00
|
|
|
base.PendingKey(msg.Queue),
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-04-26 22:13:48 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, requeueCmd, keys, msg.ID)
|
2019-12-18 12:07:17 +08:00
|
|
|
}
|
|
|
|
|
2022-03-05 22:08:43 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:g:<group_key>
|
|
|
|
// KEYS[3] -> asynq:{<qname>}:groups
|
|
|
|
// -------
|
|
|
|
// ARGV[1] -> task message data
|
|
|
|
// ARGV[2] -> task ID
|
|
|
|
// ARGV[3] -> current time in Unix time
|
|
|
|
// ARGV[4] -> group key
|
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 1 if successfully added
|
|
|
|
// Returns 0 if task ID already exists
|
|
|
|
var addToGroupCmd = redis.NewScript(`
|
|
|
|
if redis.call("EXISTS", KEYS[1]) == 1 then
|
|
|
|
return 0
|
|
|
|
end
|
|
|
|
redis.call("HSET", KEYS[1],
|
|
|
|
"msg", ARGV[1],
|
2022-03-06 21:45:33 +08:00
|
|
|
"state", "aggregating",
|
|
|
|
"group", ARGV[4])
|
2022-03-05 22:08:43 +08:00
|
|
|
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
|
|
|
|
redis.call("SADD", KEYS[3], ARGV[4])
|
|
|
|
return 1
|
|
|
|
`)
|
|
|
|
|
|
|
|
func (r *RDB) AddToGroup(ctx context.Context, msg *base.TaskMessage, groupKey string) error {
|
|
|
|
var op errors.Op = "rdb.AddToGroup"
|
|
|
|
encoded, err := base.EncodeMessage(msg)
|
|
|
|
if err != nil {
|
|
|
|
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
|
|
|
}
|
|
|
|
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
|
|
|
}
|
|
|
|
keys := []string{
|
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
|
|
|
base.GroupKey(msg.Queue, groupKey),
|
|
|
|
base.AllGroups(msg.Queue),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
|
|
|
encoded,
|
|
|
|
msg.ID,
|
|
|
|
r.clock.Now().Unix(),
|
|
|
|
groupKey,
|
|
|
|
}
|
|
|
|
n, err := r.runScriptWithErrorCode(ctx, op, addToGroupCmd, keys, argv...)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if n == 0 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:g:<group_key>
|
|
|
|
// KEYS[3] -> asynq:{<qname>}:groups
|
|
|
|
// KEYS[4] -> unique key
|
|
|
|
// -------
|
|
|
|
// ARGV[1] -> task message data
|
|
|
|
// ARGV[2] -> task ID
|
|
|
|
// ARGV[3] -> current time in Unix time
|
|
|
|
// ARGV[4] -> group key
|
|
|
|
// ARGV[5] -> uniqueness lock TTL
|
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 1 if successfully added
|
|
|
|
// Returns 0 if task ID already exists
|
|
|
|
// Returns -1 if task unique key already exists
|
|
|
|
var addToGroupUniqueCmd = redis.NewScript(`
|
|
|
|
local ok = redis.call("SET", KEYS[4], ARGV[2], "NX", "EX", ARGV[5])
|
|
|
|
if not ok then
|
|
|
|
return -1
|
|
|
|
end
|
|
|
|
if redis.call("EXISTS", KEYS[1]) == 1 then
|
|
|
|
return 0
|
|
|
|
end
|
|
|
|
redis.call("HSET", KEYS[1],
|
|
|
|
"msg", ARGV[1],
|
2022-03-06 21:45:33 +08:00
|
|
|
"state", "aggregating",
|
|
|
|
"group", ARGV[4])
|
2022-03-05 22:08:43 +08:00
|
|
|
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
|
|
|
|
redis.call("SADD", KEYS[3], ARGV[4])
|
|
|
|
return 1
|
|
|
|
`)
|
|
|
|
|
|
|
|
func (r *RDB) AddToGroupUnique(ctx context.Context, msg *base.TaskMessage, groupKey string, ttl time.Duration) error {
|
|
|
|
var op errors.Op = "rdb.AddToGroupUnique"
|
|
|
|
encoded, err := base.EncodeMessage(msg)
|
|
|
|
if err != nil {
|
|
|
|
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
|
|
|
}
|
|
|
|
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
|
|
|
}
|
|
|
|
keys := []string{
|
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
|
|
|
base.GroupKey(msg.Queue, groupKey),
|
|
|
|
base.AllGroups(msg.Queue),
|
|
|
|
base.UniqueKey(msg.Queue, msg.Type, msg.Payload),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
|
|
|
encoded,
|
|
|
|
msg.ID,
|
|
|
|
r.clock.Now().Unix(),
|
|
|
|
groupKey,
|
|
|
|
int(ttl.Seconds()),
|
|
|
|
}
|
|
|
|
n, err := r.runScriptWithErrorCode(ctx, op, addToGroupUniqueCmd, keys, argv...)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if n == -1 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrDuplicateTask)
|
|
|
|
}
|
|
|
|
if n == 0 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:scheduled
|
2022-02-16 22:17:08 +08:00
|
|
|
// -------
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task message data
|
|
|
|
// ARGV[2] -> process_at time in Unix time
|
|
|
|
// ARGV[3] -> task ID
|
2021-09-11 07:47:00 +08:00
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 1 if successfully enqueued
|
|
|
|
// Returns 0 if task ID already exists
|
2021-03-13 08:23:08 +08:00
|
|
|
var scheduleCmd = redis.NewScript(`
|
2021-09-11 07:47:00 +08:00
|
|
|
if redis.call("EXISTS", KEYS[1]) == 1 then
|
|
|
|
return 0
|
|
|
|
end
|
2021-04-24 21:44:44 +08:00
|
|
|
redis.call("HSET", KEYS[1],
|
|
|
|
"msg", ARGV[1],
|
2022-02-16 22:17:08 +08:00
|
|
|
"state", "scheduled")
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("ZADD", KEYS[2], ARGV[2], ARGV[3])
|
|
|
|
return 1
|
|
|
|
`)
|
|
|
|
|
|
|
|
// Schedule adds the task to the scheduled set to be processed in the future.
|
2021-11-16 08:34:26 +08:00
|
|
|
func (r *RDB) Schedule(ctx context.Context, msg *base.TaskMessage, processAt time.Time) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.Schedule"
|
2020-06-12 11:58:27 +08:00
|
|
|
encoded, err := base.EncodeMessage(msg)
|
2019-12-16 09:18:43 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
2019-12-16 09:18:43 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
2020-08-07 21:31:02 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
keys := []string{
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.ScheduledKey(msg.Queue),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
|
|
|
encoded,
|
|
|
|
processAt.Unix(),
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
n, err := r.runScriptWithErrorCode(ctx, op, scheduleCmd, keys, argv...)
|
2021-09-11 07:47:00 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if n == 0 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
|
|
|
}
|
|
|
|
return nil
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
|
|
|
|
2020-08-07 21:31:02 +08:00
|
|
|
// KEYS[1] -> unique key
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[3] -> asynq:{<qname>}:scheduled
|
2022-02-16 22:17:08 +08:00
|
|
|
// -------
|
2020-03-18 21:49:39 +08:00
|
|
|
// ARGV[1] -> task ID
|
|
|
|
// ARGV[2] -> uniqueness lock TTL
|
|
|
|
// ARGV[3] -> score (process_at timestamp)
|
|
|
|
// ARGV[4] -> task message
|
2021-09-11 07:47:00 +08:00
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 1 if successfully scheduled
|
|
|
|
// Returns 0 if task ID already exists
|
|
|
|
// Returns -1 if task unique key already exists
|
2020-03-18 21:49:39 +08:00
|
|
|
var scheduleUniqueCmd = redis.NewScript(`
|
|
|
|
local ok = redis.call("SET", KEYS[1], ARGV[1], "NX", "EX", ARGV[2])
|
|
|
|
if not ok then
|
2021-09-11 07:47:00 +08:00
|
|
|
return -1
|
|
|
|
end
|
|
|
|
if redis.call("EXISTS", KEYS[2]) == 1 then
|
2020-03-18 21:49:39 +08:00
|
|
|
return 0
|
|
|
|
end
|
2021-04-25 11:41:44 +08:00
|
|
|
redis.call("HSET", KEYS[2],
|
|
|
|
"msg", ARGV[4],
|
|
|
|
"state", "scheduled",
|
2021-06-09 21:06:43 +08:00
|
|
|
"unique_key", KEYS[1])
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("ZADD", KEYS[3], ARGV[3], ARGV[1])
|
2020-03-18 21:49:39 +08:00
|
|
|
return 1
|
|
|
|
`)
|
|
|
|
|
2020-03-22 02:44:26 +08:00
|
|
|
// ScheduleUnique adds the task to the backlog queue to be processed in the future if the uniqueness lock can be acquired.
|
2020-03-18 21:49:39 +08:00
|
|
|
// It returns ErrDuplicateTask if the lock cannot be acquired.
|
2021-11-16 08:34:26 +08:00
|
|
|
func (r *RDB) ScheduleUnique(ctx context.Context, msg *base.TaskMessage, processAt time.Time, ttl time.Duration) error {
|
2021-05-09 22:39:01 +08:00
|
|
|
var op errors.Op = "rdb.ScheduleUnique"
|
2020-06-12 11:58:27 +08:00
|
|
|
encoded, err := base.EncodeMessage(msg)
|
2020-03-18 21:49:39 +08:00
|
|
|
if err != nil {
|
2021-05-09 22:39:01 +08:00
|
|
|
return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode task message: %v", err))
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
2021-05-09 22:39:01 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
2020-08-07 21:31:02 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
keys := []string{
|
|
|
|
msg.UniqueKey,
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.ScheduledKey(msg.Queue),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-03-13 08:23:08 +08:00
|
|
|
int(ttl.Seconds()),
|
|
|
|
processAt.Unix(),
|
|
|
|
encoded,
|
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
n, err := r.runScriptWithErrorCode(ctx, op, scheduleUniqueCmd, keys, argv...)
|
2020-03-18 21:49:39 +08:00
|
|
|
if err != nil {
|
2021-09-11 07:47:00 +08:00
|
|
|
return err
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
2021-09-11 07:47:00 +08:00
|
|
|
if n == -1 {
|
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrDuplicateTask)
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
|
|
|
if n == 0 {
|
2021-09-11 07:47:00 +08:00
|
|
|
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
2020-03-18 21:49:39 +08:00
|
|
|
}
|
|
|
|
return nil
|
2019-12-04 22:45:30 +08:00
|
|
|
}
|
|
|
|
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:lease
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[4] -> asynq:{<qname>}:retry
|
|
|
|
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
|
|
|
// KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd>
|
2021-12-17 08:53:02 +08:00
|
|
|
// KEYS[7] -> asynq:{<qname>}:processed
|
|
|
|
// KEYS[8] -> asynq:{<qname>}:failed
|
2022-02-11 11:01:05 +08:00
|
|
|
// -------
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task ID
|
|
|
|
// ARGV[2] -> updated base.TaskMessage value
|
2020-02-09 03:06:14 +08:00
|
|
|
// ARGV[3] -> retry_at UNIX timestamp
|
|
|
|
// ARGV[4] -> stats expiration timestamp
|
2021-09-01 21:00:54 +08:00
|
|
|
// ARGV[5] -> is_failure (bool)
|
2021-12-17 08:53:02 +08:00
|
|
|
// ARGV[6] -> max int64 value
|
2020-02-09 03:06:14 +08:00
|
|
|
var retryCmd = redis.NewScript(`
|
2021-03-13 08:23:08 +08:00
|
|
|
if redis.call("LREM", KEYS[2], 0, ARGV[1]) == 0 then
|
2020-06-08 04:04:27 +08:00
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
if redis.call("ZREM", KEYS[3], ARGV[1]) == 0 then
|
2020-06-19 01:25:01 +08:00
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("ZADD", KEYS[4], ARGV[3], ARGV[1])
|
2021-04-26 22:10:29 +08:00
|
|
|
redis.call("HSET", KEYS[1], "msg", ARGV[2], "state", "retry")
|
2021-09-01 21:00:54 +08:00
|
|
|
if tonumber(ARGV[5]) == 1 then
|
|
|
|
local n = redis.call("INCR", KEYS[5])
|
|
|
|
if tonumber(n) == 1 then
|
|
|
|
redis.call("EXPIREAT", KEYS[5], ARGV[4])
|
|
|
|
end
|
|
|
|
local m = redis.call("INCR", KEYS[6])
|
|
|
|
if tonumber(m) == 1 then
|
|
|
|
redis.call("EXPIREAT", KEYS[6], ARGV[4])
|
|
|
|
end
|
2021-12-17 08:53:02 +08:00
|
|
|
local total = redis.call("GET", KEYS[7])
|
|
|
|
if tonumber(total) == tonumber(ARGV[6]) then
|
|
|
|
redis.call("SET", KEYS[7], 1)
|
|
|
|
redis.call("SET", KEYS[8], 1)
|
|
|
|
else
|
|
|
|
redis.call("INCR", KEYS[7])
|
|
|
|
redis.call("INCR", KEYS[8])
|
|
|
|
end
|
2020-02-09 03:06:14 +08:00
|
|
|
end
|
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
2021-09-01 21:00:54 +08:00
|
|
|
// Retry moves the task from active to retry queue.
|
|
|
|
// It also annotates the message with the given error message and
|
|
|
|
// if isFailure is true increments the retried counter.
|
2022-02-14 06:34:38 +08:00
|
|
|
func (r *RDB) Retry(ctx context.Context, msg *base.TaskMessage, processAt time.Time, errMsg string, isFailure bool) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.Retry"
|
2021-12-11 01:07:41 +08:00
|
|
|
now := r.clock.Now()
|
2019-12-16 12:05:56 +08:00
|
|
|
modified := *msg
|
2021-09-01 21:00:54 +08:00
|
|
|
if isFailure {
|
|
|
|
modified.Retried++
|
|
|
|
}
|
2019-12-16 12:05:56 +08:00
|
|
|
modified.ErrorMsg = errMsg
|
2021-05-19 12:00:53 +08:00
|
|
|
modified.LastFailedAt = now.Unix()
|
2021-03-13 08:23:08 +08:00
|
|
|
encoded, err := base.EncodeMessage(&modified)
|
2019-12-16 12:05:56 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode message: %v", err))
|
2019-12-16 12:05:56 +08:00
|
|
|
}
|
2019-12-23 21:33:48 +08:00
|
|
|
expireAt := now.Add(statsTTL)
|
2021-03-13 08:23:08 +08:00
|
|
|
keys := []string{
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.ActiveKey(msg.Queue),
|
2022-02-11 11:01:05 +08:00
|
|
|
base.LeaseKey(msg.Queue),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.RetryKey(msg.Queue),
|
|
|
|
base.ProcessedKey(msg.Queue, now),
|
|
|
|
base.FailedKey(msg.Queue, now),
|
2021-12-17 08:53:02 +08:00
|
|
|
base.ProcessedTotalKey(msg.Queue),
|
|
|
|
base.FailedTotalKey(msg.Queue),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
|
|
|
argv := []interface{}{
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-03-13 08:23:08 +08:00
|
|
|
encoded,
|
|
|
|
processAt.Unix(),
|
|
|
|
expireAt.Unix(),
|
2021-09-01 21:00:54 +08:00
|
|
|
isFailure,
|
2022-03-29 21:30:10 +08:00
|
|
|
int64(math.MaxInt64),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, retryCmd, keys, argv...)
|
2019-12-16 08:15:07 +08:00
|
|
|
}
|
|
|
|
|
2019-12-26 23:17:26 +08:00
|
|
|
const (
|
2021-01-13 03:01:21 +08:00
|
|
|
maxArchiveSize = 10000 // maximum number of tasks in archive
|
|
|
|
archivedExpirationInDays = 90 // number of days before an archived task gets deleted permanently
|
2019-12-26 23:17:26 +08:00
|
|
|
)
|
|
|
|
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:active
|
2022-02-11 11:01:05 +08:00
|
|
|
// KEYS[3] -> asynq:{<qname>}:lease
|
2021-03-13 08:23:08 +08:00
|
|
|
// KEYS[4] -> asynq:{<qname>}:archived
|
|
|
|
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
|
|
|
// KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd>
|
2021-12-17 08:53:02 +08:00
|
|
|
// KEYS[7] -> asynq:{<qname>}:processed
|
|
|
|
// KEYS[8] -> asynq:{<qname>}:failed
|
2022-02-11 11:01:05 +08:00
|
|
|
// -------
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[1] -> task ID
|
|
|
|
// ARGV[2] -> updated base.TaskMessage value
|
2020-02-09 03:06:14 +08:00
|
|
|
// ARGV[3] -> died_at UNIX timestamp
|
|
|
|
// ARGV[4] -> cutoff timestamp (e.g., 90 days ago)
|
2021-01-13 03:01:21 +08:00
|
|
|
// ARGV[5] -> max number of tasks in archive (e.g., 100)
|
2020-02-09 03:06:14 +08:00
|
|
|
// ARGV[6] -> stats expiration timestamp
|
2021-12-17 08:53:02 +08:00
|
|
|
// ARGV[7] -> max int64 value
|
2021-01-13 03:01:21 +08:00
|
|
|
var archiveCmd = redis.NewScript(`
|
2021-03-13 08:23:08 +08:00
|
|
|
if redis.call("LREM", KEYS[2], 0, ARGV[1]) == 0 then
|
2020-06-08 04:04:27 +08:00
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
if redis.call("ZREM", KEYS[3], ARGV[1]) == 0 then
|
2020-06-19 01:53:58 +08:00
|
|
|
return redis.error_reply("NOT FOUND")
|
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("ZADD", KEYS[4], ARGV[3], ARGV[1])
|
|
|
|
redis.call("ZREMRANGEBYSCORE", KEYS[4], "-inf", ARGV[4])
|
|
|
|
redis.call("ZREMRANGEBYRANK", KEYS[4], 0, -ARGV[5])
|
2021-04-26 22:10:29 +08:00
|
|
|
redis.call("HSET", KEYS[1], "msg", ARGV[2], "state", "archived")
|
2021-03-13 08:23:08 +08:00
|
|
|
local n = redis.call("INCR", KEYS[5])
|
2020-02-09 03:06:14 +08:00
|
|
|
if tonumber(n) == 1 then
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("EXPIREAT", KEYS[5], ARGV[6])
|
2020-02-09 03:06:14 +08:00
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
local m = redis.call("INCR", KEYS[6])
|
2020-02-09 03:06:14 +08:00
|
|
|
if tonumber(m) == 1 then
|
2021-03-13 08:23:08 +08:00
|
|
|
redis.call("EXPIREAT", KEYS[6], ARGV[6])
|
2020-02-09 03:06:14 +08:00
|
|
|
end
|
2021-12-17 08:53:02 +08:00
|
|
|
local total = redis.call("GET", KEYS[7])
|
|
|
|
if tonumber(total) == tonumber(ARGV[7]) then
|
|
|
|
redis.call("SET", KEYS[7], 1)
|
|
|
|
redis.call("SET", KEYS[8], 1)
|
|
|
|
else
|
|
|
|
redis.call("INCR", KEYS[7])
|
|
|
|
redis.call("INCR", KEYS[8])
|
|
|
|
end
|
2020-02-09 03:06:14 +08:00
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
2021-01-13 03:01:21 +08:00
|
|
|
// Archive sends the given task to archive, attaching the error message to the task.
|
|
|
|
// It also trims the archive by timestamp and set size.
|
2022-02-14 06:34:38 +08:00
|
|
|
func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.Archive"
|
2021-12-11 01:07:41 +08:00
|
|
|
now := r.clock.Now()
|
2019-12-16 12:05:56 +08:00
|
|
|
modified := *msg
|
|
|
|
modified.ErrorMsg = errMsg
|
2021-05-19 12:00:53 +08:00
|
|
|
modified.LastFailedAt = now.Unix()
|
2021-03-13 08:23:08 +08:00
|
|
|
encoded, err := base.EncodeMessage(&modified)
|
2019-12-16 12:05:56 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode message: %v", err))
|
2019-12-16 12:05:56 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
cutoff := now.AddDate(0, 0, -archivedExpirationInDays)
|
2019-12-23 21:33:48 +08:00
|
|
|
expireAt := now.Add(statsTTL)
|
2021-03-13 08:23:08 +08:00
|
|
|
keys := []string{
|
2021-09-10 21:29:37 +08:00
|
|
|
base.TaskKey(msg.Queue, msg.ID),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.ActiveKey(msg.Queue),
|
2022-02-11 11:01:05 +08:00
|
|
|
base.LeaseKey(msg.Queue),
|
2021-03-13 08:23:08 +08:00
|
|
|
base.ArchivedKey(msg.Queue),
|
|
|
|
base.ProcessedKey(msg.Queue, now),
|
|
|
|
base.FailedKey(msg.Queue, now),
|
2021-12-17 08:53:02 +08:00
|
|
|
base.ProcessedTotalKey(msg.Queue),
|
|
|
|
base.FailedTotalKey(msg.Queue),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
|
|
|
argv := []interface{}{
|
2021-09-10 21:29:37 +08:00
|
|
|
msg.ID,
|
2021-03-13 08:23:08 +08:00
|
|
|
encoded,
|
|
|
|
now.Unix(),
|
|
|
|
cutoff.Unix(),
|
|
|
|
maxArchiveSize,
|
|
|
|
expireAt.Unix(),
|
2022-03-29 21:30:10 +08:00
|
|
|
int64(math.MaxInt64),
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, archiveCmd, keys, argv...)
|
2019-11-20 11:44:41 +08:00
|
|
|
}
|
|
|
|
|
2021-03-13 08:23:08 +08:00
|
|
|
// ForwardIfReady checks scheduled and retry sets of the given queues
|
|
|
|
// and move any tasks that are ready to be processed to the pending set.
|
|
|
|
func (r *RDB) ForwardIfReady(qnames ...string) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.ForwardIfReady"
|
2020-08-09 21:26:14 +08:00
|
|
|
for _, qname := range qnames {
|
2021-04-26 21:58:33 +08:00
|
|
|
if err := r.forwardAll(qname); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.CanonicalCode(err), err)
|
2019-12-04 23:14:37 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-08-09 21:26:14 +08:00
|
|
|
// KEYS[1] -> source queue (e.g. asynq:{<qname>:scheduled or asynq:{<qname>}:retry})
|
2021-04-26 21:58:33 +08:00
|
|
|
// KEYS[2] -> asynq:{<qname>}:pending
|
2021-12-11 01:07:41 +08:00
|
|
|
// ARGV[1] -> current unix time in seconds
|
2021-04-26 21:58:33 +08:00
|
|
|
// ARGV[2] -> task key prefix
|
2021-12-11 22:27:44 +08:00
|
|
|
// ARGV[3] -> current unix time in nsec
|
2022-03-06 21:45:33 +08:00
|
|
|
// ARGV[4] -> group key prefix
|
2020-06-08 04:04:27 +08:00
|
|
|
// Note: Script moves tasks up to 100 at a time to keep the runtime of script short.
|
2020-02-09 03:06:14 +08:00
|
|
|
var forwardCmd = redis.NewScript(`
|
2021-03-13 08:23:08 +08:00
|
|
|
local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1], "LIMIT", 0, 100)
|
|
|
|
for _, id in ipairs(ids) do
|
2022-03-06 21:45:33 +08:00
|
|
|
local taskKey = ARGV[2] .. id
|
|
|
|
local group = redis.call("HGET", taskKey, "group")
|
2022-03-06 22:04:56 +08:00
|
|
|
if group and group ~= '' then
|
2022-03-06 21:45:33 +08:00
|
|
|
redis.call("ZADD", ARGV[4] .. group, ARGV[1], id)
|
|
|
|
redis.call("ZREM", KEYS[1], id)
|
|
|
|
redis.call("HSET", taskKey,
|
|
|
|
"state", "aggregating")
|
|
|
|
else
|
|
|
|
redis.call("LPUSH", KEYS[2], id)
|
|
|
|
redis.call("ZREM", KEYS[1], id)
|
|
|
|
redis.call("HSET", taskKey,
|
|
|
|
"state", "pending",
|
|
|
|
"pending_since", ARGV[3])
|
|
|
|
end
|
2020-02-09 03:06:14 +08:00
|
|
|
end
|
2021-03-13 08:23:08 +08:00
|
|
|
return table.getn(ids)`)
|
2020-02-09 03:06:14 +08:00
|
|
|
|
2022-03-06 21:45:33 +08:00
|
|
|
// forward moves tasks with a score less than the current unix time from the delayed (i.e. scheduled | retry) zset
|
|
|
|
// to the pending list or group set.
|
|
|
|
// It returns the number of tasks moved.
|
|
|
|
func (r *RDB) forward(delayedKey, pendingKey, taskKeyPrefix, groupKeyPrefix string) (int, error) {
|
2021-12-11 01:07:41 +08:00
|
|
|
now := r.clock.Now()
|
2022-03-06 21:45:33 +08:00
|
|
|
keys := []string{delayedKey, pendingKey}
|
|
|
|
argv := []interface{}{
|
|
|
|
now.Unix(),
|
|
|
|
taskKeyPrefix,
|
|
|
|
now.UnixNano(),
|
|
|
|
groupKeyPrefix,
|
|
|
|
}
|
|
|
|
res, err := forwardCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
2020-06-08 04:04:27 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return 0, errors.E(errors.Internal, fmt.Sprintf("redis eval error: %v", err))
|
2020-06-08 04:04:27 +08:00
|
|
|
}
|
2021-05-11 00:16:13 +08:00
|
|
|
n, err := cast.ToIntE(res)
|
|
|
|
if err != nil {
|
|
|
|
return 0, errors.E(errors.Internal, fmt.Sprintf("cast error: Lua script returned unexpected value: %v", res))
|
|
|
|
}
|
|
|
|
return n, nil
|
2019-12-07 14:29:40 +08:00
|
|
|
}
|
2020-01-31 22:48:58 +08:00
|
|
|
|
2021-04-26 21:58:33 +08:00
|
|
|
// forwardAll checks for tasks in scheduled/retry state that are ready to be run, and updates
|
2022-03-06 21:45:33 +08:00
|
|
|
// their state to "pending" or "aggregating".
|
2021-04-26 21:58:33 +08:00
|
|
|
func (r *RDB) forwardAll(qname string) (err error) {
|
2022-03-06 21:45:33 +08:00
|
|
|
delayedKeys := []string{base.ScheduledKey(qname), base.RetryKey(qname)}
|
|
|
|
pendingKey := base.PendingKey(qname)
|
2021-04-26 21:58:33 +08:00
|
|
|
taskKeyPrefix := base.TaskKeyPrefix(qname)
|
2022-03-06 21:45:33 +08:00
|
|
|
groupKeyPrefix := base.GroupKeyPrefix(qname)
|
|
|
|
for _, delayedKey := range delayedKeys {
|
2021-04-26 21:58:33 +08:00
|
|
|
n := 1
|
|
|
|
for n != 0 {
|
2022-03-06 21:45:33 +08:00
|
|
|
n, err = r.forward(delayedKey, pendingKey, taskKeyPrefix, groupKeyPrefix)
|
2021-04-26 21:58:33 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-08-09 21:26:14 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2022-03-08 22:38:35 +08:00
|
|
|
// ListGroups returns a list of all known groups in the given queue.
|
|
|
|
func (r *RDB) ListGroups(qname string) ([]string, error) {
|
2022-03-11 03:06:48 +08:00
|
|
|
var op errors.Op = "RDB.ListGroups"
|
|
|
|
groups, err := r.client.SMembers(context.Background(), base.AllGroups(qname)).Result()
|
|
|
|
if err != nil {
|
|
|
|
return nil, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "smembers", Err: err})
|
|
|
|
}
|
|
|
|
return groups, nil
|
2022-03-08 22:38:35 +08:00
|
|
|
}
|
|
|
|
|
2022-03-10 09:05:16 +08:00
|
|
|
// TODO: Add comment describing what the script does.
|
|
|
|
// KEYS[1] -> asynq:{<qname>}:g:<gname>
|
|
|
|
// KEYS[2] -> asynq:{<qname>}:g:<gname>:<aggregation_set_id>
|
|
|
|
// KEYS[3] -> asynq:{<qname>}:aggregation_sets
|
|
|
|
// -------
|
|
|
|
// ARGV[1] -> max group size
|
|
|
|
// ARGV[2] -> max group delay in unix time
|
|
|
|
// ARGV[3] -> start time of the grace period
|
|
|
|
// ARGV[4] -> aggregation set ID
|
|
|
|
// ARGV[5] -> aggregation set expire time
|
2022-03-11 07:48:02 +08:00
|
|
|
// ARGV[6] -> current time in unix time
|
2022-03-10 09:05:16 +08:00
|
|
|
//
|
|
|
|
// Output:
|
|
|
|
// Returns 0 if no aggregation set was created
|
|
|
|
// Returns 1 if an aggregation set was created
|
|
|
|
var aggregationCheckCmd = redis.NewScript(`
|
|
|
|
local size = redis.call("ZCARD", KEYS[1])
|
2022-03-11 03:54:47 +08:00
|
|
|
if size == 0 then
|
|
|
|
return 0
|
|
|
|
end
|
2022-03-10 09:05:16 +08:00
|
|
|
local maxSize = tonumber(ARGV[1])
|
2022-03-11 07:48:02 +08:00
|
|
|
if maxSize ~= 0 and size >= maxSize then
|
2022-03-10 09:05:16 +08:00
|
|
|
local msgs = redis.call("ZRANGE", KEYS[1], 0, maxSize-1)
|
|
|
|
for _, msg in ipairs(msgs) do
|
|
|
|
redis.call("SADD", KEYS[2], msg)
|
|
|
|
end
|
|
|
|
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, maxSize-1)
|
|
|
|
redis.call("ZADD", KEYS[3], ARGV[5], ARGV[4])
|
|
|
|
return 1
|
|
|
|
end
|
2022-03-11 07:48:02 +08:00
|
|
|
local maxDelay = tonumber(ARGV[2])
|
|
|
|
local currentTime = tonumber(ARGV[6])
|
|
|
|
if maxDelay ~= 0 then
|
|
|
|
local oldestEntry = redis.call("ZRANGE", KEYS[1], 0, 0, "WITHSCORES")
|
|
|
|
local oldestEntryScore = tonumber(oldestEntry[2])
|
|
|
|
local maxDelayTime = currentTime - maxDelay
|
|
|
|
if oldestEntryScore <= maxDelayTime then
|
|
|
|
local msgs = redis.call("ZRANGE", KEYS[1], 0, maxSize-1)
|
|
|
|
for _, msg in ipairs(msgs) do
|
|
|
|
redis.call("SADD", KEYS[2], msg)
|
|
|
|
end
|
|
|
|
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, maxSize-1)
|
|
|
|
redis.call("ZADD", KEYS[3], ARGV[5], ARGV[4])
|
|
|
|
return 1
|
2022-03-10 09:05:16 +08:00
|
|
|
end
|
|
|
|
end
|
|
|
|
local latestEntry = redis.call("ZREVRANGE", KEYS[1], 0, 0, "WITHSCORES")
|
|
|
|
local latestEntryScore = tonumber(latestEntry[2])
|
2022-03-11 07:48:02 +08:00
|
|
|
local gracePeriodStartTime = currentTime - tonumber(ARGV[3])
|
2022-03-10 09:05:16 +08:00
|
|
|
if latestEntryScore <= gracePeriodStartTime then
|
|
|
|
local msgs = redis.call("ZRANGE", KEYS[1], 0, maxSize-1)
|
|
|
|
for _, msg in ipairs(msgs) do
|
|
|
|
redis.call("SADD", KEYS[2], msg)
|
|
|
|
end
|
|
|
|
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, maxSize-1)
|
|
|
|
redis.call("ZADD", KEYS[3], ARGV[5], ARGV[4])
|
|
|
|
return 1
|
|
|
|
end
|
|
|
|
return 0
|
|
|
|
`)
|
|
|
|
|
|
|
|
// Task aggregation should finish within this timeout.
|
|
|
|
// Otherwise an aggregation set should be reclaimed by the recoverer.
|
|
|
|
const aggregationTimeout = 2 * time.Minute
|
|
|
|
|
2022-03-08 22:38:35 +08:00
|
|
|
// AggregationCheck checks the group identified by the given queue and group name to see if the tasks in the
|
|
|
|
// group are ready to be aggregated. If so, it moves the tasks to be aggregated to a aggregation set and returns
|
2022-03-11 07:48:02 +08:00
|
|
|
// the set ID. If not, it returns an empty string for the set ID.
|
|
|
|
// The time for gracePeriod and maxDelay is computed relative to the time t.
|
2022-03-10 09:05:16 +08:00
|
|
|
//
|
|
|
|
// Note: It assumes that this function is called at frequency less than or equal to the gracePeriod. In other words,
|
|
|
|
// the function only checks the most recently added task aganist the given gracePeriod.
|
2022-03-11 07:48:02 +08:00
|
|
|
func (r *RDB) AggregationCheck(qname, gname string, t time.Time, gracePeriod, maxDelay time.Duration, maxSize int) (string, error) {
|
2022-03-10 09:05:16 +08:00
|
|
|
var op errors.Op = "RDB.AggregationCheck"
|
|
|
|
aggregationSetID := uuid.NewString()
|
|
|
|
expireTime := r.clock.Now().Add(aggregationTimeout)
|
|
|
|
keys := []string{
|
|
|
|
base.GroupKey(qname, gname),
|
|
|
|
base.AggregationSetKey(qname, gname, aggregationSetID),
|
|
|
|
base.AllAggregationSets(qname),
|
|
|
|
}
|
|
|
|
argv := []interface{}{
|
|
|
|
maxSize,
|
2022-03-11 07:48:02 +08:00
|
|
|
int64(maxDelay.Seconds()),
|
|
|
|
int64(gracePeriod.Seconds()),
|
2022-03-10 09:05:16 +08:00
|
|
|
aggregationSetID,
|
|
|
|
expireTime.Unix(),
|
2022-03-11 07:48:02 +08:00
|
|
|
t.Unix(),
|
2022-03-10 09:05:16 +08:00
|
|
|
}
|
|
|
|
n, err := r.runScriptWithErrorCode(context.Background(), op, aggregationCheckCmd, keys, argv...)
|
|
|
|
if err != nil {
|
|
|
|
return "", err
|
|
|
|
}
|
|
|
|
switch n {
|
|
|
|
case 0:
|
|
|
|
return "", nil
|
|
|
|
case 1:
|
|
|
|
return aggregationSetID, nil
|
|
|
|
default:
|
|
|
|
return "", errors.E(op, errors.Internal, fmt.Sprintf("unexpected return value from lua script: %d", n))
|
|
|
|
}
|
2022-03-08 22:38:35 +08:00
|
|
|
}
|
|
|
|
|
2022-03-10 09:05:16 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:g:<gname>:<aggregation_set_id>
|
|
|
|
// ------
|
|
|
|
// ARGV[1] -> task key prefix
|
|
|
|
var readAggregationSetCmd = redis.NewScript(`
|
|
|
|
local msgs = {}
|
|
|
|
local ids = redis.call("SMEMBERS", KEYS[1])
|
|
|
|
for _, id in ipairs(ids) do
|
|
|
|
local key = ARGV[1] .. id
|
|
|
|
table.insert(msgs, redis.call("HGET", key, "msg"))
|
|
|
|
end
|
|
|
|
return msgs
|
|
|
|
`)
|
|
|
|
|
|
|
|
// ReadAggregationSet retrieves members of an aggregation set and returns a list of tasks in the set and
|
2022-03-08 22:38:35 +08:00
|
|
|
// the deadline for aggregating those tasks.
|
|
|
|
func (r *RDB) ReadAggregationSet(qname, gname, setID string) ([]*base.TaskMessage, time.Time, error) {
|
2022-03-10 09:05:16 +08:00
|
|
|
var op errors.Op = "RDB.ReadAggregationSet"
|
|
|
|
ctx := context.Background()
|
|
|
|
res, err := readAggregationSetCmd.Run(ctx, r.client,
|
|
|
|
[]string{base.AggregationSetKey(qname, gname, setID)}, base.TaskKeyPrefix(qname)).Result()
|
|
|
|
if err != nil {
|
|
|
|
return nil, time.Time{}, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "smembers", Err: err})
|
|
|
|
}
|
|
|
|
data, err := cast.ToStringSliceE(res)
|
|
|
|
if err != nil {
|
|
|
|
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: Lua script returned unexpected value: %v", res))
|
|
|
|
}
|
|
|
|
var msgs []*base.TaskMessage
|
|
|
|
for _, s := range data {
|
|
|
|
msg, err := base.DecodeMessage([]byte(s))
|
|
|
|
if err != nil {
|
|
|
|
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cannot decode message: %v", err))
|
|
|
|
}
|
|
|
|
msgs = append(msgs, msg)
|
|
|
|
}
|
|
|
|
deadlineUnix, err := r.client.ZScore(ctx, base.AllAggregationSets(qname), setID).Result()
|
|
|
|
if err != nil {
|
|
|
|
return nil, time.Time{}, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "zscore", Err: err})
|
|
|
|
}
|
|
|
|
return msgs, time.Unix(int64(deadlineUnix), 0), nil
|
2022-03-08 22:38:35 +08:00
|
|
|
}
|
|
|
|
|
2022-03-11 03:00:28 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:g:<gname>:<aggregation_set_id>
|
|
|
|
// -------
|
|
|
|
// ARGV[1] -> task key prefix
|
|
|
|
var deleteAggregationSetCmd = redis.NewScript(`
|
|
|
|
local ids = redis.call("SMEMBERS", KEYS[1])
|
|
|
|
for _, id in ipairs(ids) do
|
|
|
|
redis.call("DEL", ARGV[1] .. id)
|
|
|
|
end
|
|
|
|
redis.call("DEL", KEYS[1])
|
|
|
|
return redis.status_reply("OK")
|
|
|
|
`)
|
|
|
|
|
|
|
|
// DeleteAggregationSet deletes the aggregation set and its members identified by the parameters.
|
2022-03-08 22:38:35 +08:00
|
|
|
func (r *RDB) DeleteAggregationSet(ctx context.Context, qname, gname, setID string) error {
|
2022-03-11 03:00:28 +08:00
|
|
|
var op errors.Op = "RDB.DeleteAggregationSet"
|
|
|
|
return r.runScript(ctx, op, deleteAggregationSetCmd, []string{base.AggregationSetKey(qname, gname, setID)}, base.TaskKeyPrefix(qname))
|
2022-03-08 22:38:35 +08:00
|
|
|
}
|
|
|
|
|
2021-11-06 07:52:54 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:completed
|
|
|
|
// ARGV[1] -> current time in unix time
|
|
|
|
// ARGV[2] -> task key prefix
|
|
|
|
// ARGV[3] -> batch size (i.e. maximum number of tasks to delete)
|
|
|
|
//
|
|
|
|
// Returns the number of tasks deleted.
|
|
|
|
var deleteExpiredCompletedTasksCmd = redis.NewScript(`
|
|
|
|
local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1], "LIMIT", 0, tonumber(ARGV[3]))
|
|
|
|
for _, id in ipairs(ids) do
|
|
|
|
redis.call("DEL", ARGV[2] .. id)
|
|
|
|
redis.call("ZREM", KEYS[1], id)
|
|
|
|
end
|
|
|
|
return table.getn(ids)`)
|
|
|
|
|
|
|
|
// DeleteExpiredCompletedTasks checks for any expired tasks in the given queue's completed set,
|
|
|
|
// and delete all expired tasks.
|
|
|
|
func (r *RDB) DeleteExpiredCompletedTasks(qname string) error {
|
|
|
|
// Note: Do this operation in fix batches to prevent long running script.
|
|
|
|
const batchSize = 100
|
|
|
|
for {
|
|
|
|
n, err := r.deleteExpiredCompletedTasks(qname, batchSize)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if n == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// deleteExpiredCompletedTasks runs the lua script to delete expired deleted task with the specified
|
|
|
|
// batch size. It reports the number of tasks deleted.
|
|
|
|
func (r *RDB) deleteExpiredCompletedTasks(qname string, batchSize int) (int64, error) {
|
|
|
|
var op errors.Op = "rdb.DeleteExpiredCompletedTasks"
|
|
|
|
keys := []string{base.CompletedKey(qname)}
|
|
|
|
argv := []interface{}{
|
2021-12-11 01:07:41 +08:00
|
|
|
r.clock.Now().Unix(),
|
2021-11-06 07:52:54 +08:00
|
|
|
base.TaskKeyPrefix(qname),
|
|
|
|
batchSize,
|
|
|
|
}
|
|
|
|
res, err := deleteExpiredCompletedTasksCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
|
|
|
if err != nil {
|
|
|
|
return 0, errors.E(op, errors.Internal, fmt.Sprintf("redis eval error: %v", err))
|
|
|
|
}
|
|
|
|
n, ok := res.(int64)
|
|
|
|
if !ok {
|
|
|
|
return 0, errors.E(op, errors.Internal, fmt.Sprintf("unexpected return value from Lua script: %v", res))
|
|
|
|
}
|
|
|
|
return n, nil
|
|
|
|
}
|
|
|
|
|
2022-02-11 21:54:56 +08:00
|
|
|
// KEYS[1] -> asynq:{<qname>}:lease
|
|
|
|
// ARGV[1] -> cutoff in unix time
|
2021-03-13 08:23:08 +08:00
|
|
|
// ARGV[2] -> task key prefix
|
2022-02-11 21:54:56 +08:00
|
|
|
var listLeaseExpiredCmd = redis.NewScript(`
|
2021-03-13 08:23:08 +08:00
|
|
|
local res = {}
|
|
|
|
local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
|
|
|
|
for _, id in ipairs(ids) do
|
|
|
|
local key = ARGV[2] .. id
|
|
|
|
table.insert(res, redis.call("HGET", key, "msg"))
|
|
|
|
end
|
|
|
|
return res
|
|
|
|
`)
|
|
|
|
|
2022-02-11 21:54:56 +08:00
|
|
|
// ListLeaseExpired returns a list of task messages with an expired lease from the given queues.
|
|
|
|
func (r *RDB) ListLeaseExpired(cutoff time.Time, qnames ...string) ([]*base.TaskMessage, error) {
|
|
|
|
var op errors.Op = "rdb.ListLeaseExpired"
|
2020-06-20 21:29:58 +08:00
|
|
|
var msgs []*base.TaskMessage
|
2020-08-10 20:37:49 +08:00
|
|
|
for _, qname := range qnames {
|
2022-02-11 21:54:56 +08:00
|
|
|
res, err := listLeaseExpiredCmd.Run(context.Background(), r.client,
|
|
|
|
[]string{base.LeaseKey(qname)},
|
|
|
|
cutoff.Unix(), base.TaskKeyPrefix(qname)).Result()
|
2021-03-13 08:23:08 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return nil, errors.E(op, errors.Internal, fmt.Sprintf("redis eval error: %v", err))
|
2021-03-13 08:23:08 +08:00
|
|
|
}
|
|
|
|
data, err := cast.ToStringSliceE(res)
|
2020-06-20 21:29:58 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return nil, errors.E(op, errors.Internal, fmt.Sprintf("cast error: Lua script returned unexpected value: %v", res))
|
2020-06-20 21:29:58 +08:00
|
|
|
}
|
2021-03-13 08:23:08 +08:00
|
|
|
for _, s := range data {
|
|
|
|
msg, err := base.DecodeMessage([]byte(s))
|
2020-08-10 20:37:49 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return nil, errors.E(op, errors.Internal, fmt.Sprintf("cannot decode message: %v", err))
|
2020-08-10 20:37:49 +08:00
|
|
|
}
|
|
|
|
msgs = append(msgs, msg)
|
|
|
|
}
|
2020-06-20 21:29:58 +08:00
|
|
|
}
|
|
|
|
return msgs, nil
|
|
|
|
}
|
|
|
|
|
2022-02-12 22:49:53 +08:00
|
|
|
// ExtendLease extends the lease for the given tasks by LeaseDuration (30s).
|
2022-02-14 06:34:38 +08:00
|
|
|
// It returns a new expiration time if the operation was successful.
|
|
|
|
func (r *RDB) ExtendLease(qname string, ids ...string) (expirationTime time.Time, err error) {
|
2022-02-12 22:49:53 +08:00
|
|
|
expireAt := r.clock.Now().Add(LeaseDuration)
|
2022-02-20 21:40:09 +08:00
|
|
|
var zs []*redis.Z
|
2022-02-12 22:49:53 +08:00
|
|
|
for _, id := range ids {
|
2022-02-20 21:40:09 +08:00
|
|
|
zs = append(zs, &redis.Z{Member: id, Score: float64(expireAt.Unix())})
|
2022-02-12 22:49:53 +08:00
|
|
|
}
|
|
|
|
// Use XX option to only update elements that already exist; Don't add new elements
|
2022-02-20 21:40:09 +08:00
|
|
|
// TODO: Consider adding GT option to ensure we only "extend" the lease. Ceveat is that GT is supported from redis v6.2.0 or above.
|
|
|
|
err = r.client.ZAddXX(context.Background(), base.LeaseKey(qname), zs...).Err()
|
2022-02-14 06:34:38 +08:00
|
|
|
if err != nil {
|
|
|
|
return time.Time{}, err
|
|
|
|
}
|
|
|
|
return expireAt, nil
|
2022-02-12 22:49:53 +08:00
|
|
|
}
|
|
|
|
|
2020-08-11 12:49:12 +08:00
|
|
|
// KEYS[1] -> asynq:servers:{<host:pid:sid>}
|
|
|
|
// KEYS[2] -> asynq:workers:{<host:pid:sid>}
|
|
|
|
// ARGV[1] -> TTL in seconds
|
|
|
|
// ARGV[2] -> server info
|
|
|
|
// ARGV[3:] -> alternate key-value pair of (worker id, worker data)
|
2020-02-21 23:18:22 +08:00
|
|
|
// Note: Add key to ZSET with expiration time as score.
|
|
|
|
// ref: https://github.com/antirez/redis/issues/135#issuecomment-2361996
|
2020-05-19 11:47:35 +08:00
|
|
|
var writeServerStateCmd = redis.NewScript(`
|
2020-08-11 12:49:12 +08:00
|
|
|
redis.call("SETEX", KEYS[1], ARGV[1], ARGV[2])
|
|
|
|
redis.call("DEL", KEYS[2])
|
|
|
|
for i = 3, table.getn(ARGV)-1, 2 do
|
|
|
|
redis.call("HSET", KEYS[2], ARGV[i], ARGV[i+1])
|
2020-02-21 23:18:22 +08:00
|
|
|
end
|
2020-08-11 12:49:12 +08:00
|
|
|
redis.call("EXPIRE", KEYS[2], ARGV[1])
|
2020-02-09 03:06:14 +08:00
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
2020-05-19 11:47:35 +08:00
|
|
|
// WriteServerState writes server state data to redis with expiration set to the value ttl.
|
|
|
|
func (r *RDB) WriteServerState(info *base.ServerInfo, workers []*base.WorkerInfo, ttl time.Duration) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.WriteServerState"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2021-03-13 08:23:08 +08:00
|
|
|
bytes, err := base.EncodeServerInfo(info)
|
2020-01-31 22:48:58 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode server info: %v", err))
|
2020-01-31 22:48:58 +08:00
|
|
|
}
|
2021-12-11 01:07:41 +08:00
|
|
|
exp := r.clock.Now().Add(ttl).UTC()
|
2020-08-11 12:49:12 +08:00
|
|
|
args := []interface{}{ttl.Seconds(), bytes} // args to the lua script
|
2020-02-21 23:18:22 +08:00
|
|
|
for _, w := range workers {
|
2021-03-13 08:23:08 +08:00
|
|
|
bytes, err := base.EncodeWorkerInfo(w)
|
2020-02-21 23:18:22 +08:00
|
|
|
if err != nil {
|
|
|
|
continue // skip bad data
|
|
|
|
}
|
2020-05-19 11:47:35 +08:00
|
|
|
args = append(args, w.ID, bytes)
|
2020-01-31 22:48:58 +08:00
|
|
|
}
|
2020-04-13 07:42:11 +08:00
|
|
|
skey := base.ServerInfoKey(info.Host, info.PID, info.ServerID)
|
|
|
|
wkey := base.WorkersKey(info.Host, info.PID, info.ServerID)
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.ZAdd(ctx, base.AllServers, &redis.Z{Score: float64(exp.Unix()), Member: skey}).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
2020-08-11 12:49:12 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.ZAdd(ctx, base.AllWorkers, &redis.Z{Score: float64(exp.Unix()), Member: wkey}).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "zadd", Err: err})
|
2020-08-11 12:49:12 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, writeServerStateCmd, []string{skey, wkey}, args...)
|
2020-02-02 14:22:48 +08:00
|
|
|
}
|
|
|
|
|
2020-08-11 12:49:12 +08:00
|
|
|
// KEYS[1] -> asynq:servers:{<host:pid:sid>}
|
|
|
|
// KEYS[2] -> asynq:workers:{<host:pid:sid>}
|
2020-06-20 21:29:58 +08:00
|
|
|
var clearServerStateCmd = redis.NewScript(`
|
2020-08-11 12:49:12 +08:00
|
|
|
redis.call("DEL", KEYS[1])
|
2020-02-09 03:06:14 +08:00
|
|
|
redis.call("DEL", KEYS[2])
|
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
2020-04-13 02:41:50 +08:00
|
|
|
// ClearServerState deletes server state data from redis.
|
2020-05-19 11:47:35 +08:00
|
|
|
func (r *RDB) ClearServerState(host string, pid int, serverID string) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.ClearServerState"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2020-05-19 11:47:35 +08:00
|
|
|
skey := base.ServerInfoKey(host, pid, serverID)
|
|
|
|
wkey := base.WorkersKey(host, pid, serverID)
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.ZRem(ctx, base.AllServers, skey).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, &errors.RedisCommandError{Command: "zrem", Err: err})
|
2020-08-11 12:49:12 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.ZRem(ctx, base.AllWorkers, wkey).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, &errors.RedisCommandError{Command: "zrem", Err: err})
|
2020-08-11 12:49:12 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, clearServerStateCmd, []string{skey, wkey})
|
2020-01-31 22:48:58 +08:00
|
|
|
}
|
2020-02-13 09:12:09 +08:00
|
|
|
|
2020-09-27 08:33:29 +08:00
|
|
|
// KEYS[1] -> asynq:schedulers:{<schedulerID>}
|
|
|
|
// ARGV[1] -> TTL in seconds
|
|
|
|
// ARGV[2:] -> schedler entries
|
|
|
|
var writeSchedulerEntriesCmd = redis.NewScript(`
|
|
|
|
redis.call("DEL", KEYS[1])
|
|
|
|
for i = 2, #ARGV do
|
|
|
|
redis.call("LPUSH", KEYS[1], ARGV[i])
|
|
|
|
end
|
|
|
|
redis.call("EXPIRE", KEYS[1], ARGV[1])
|
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
|
|
|
// WriteSchedulerEntries writes scheduler entries data to redis with expiration set to the value ttl.
|
|
|
|
func (r *RDB) WriteSchedulerEntries(schedulerID string, entries []*base.SchedulerEntry, ttl time.Duration) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.WriteSchedulerEntries"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2020-09-27 08:33:29 +08:00
|
|
|
args := []interface{}{ttl.Seconds()}
|
|
|
|
for _, e := range entries {
|
2021-03-13 08:23:08 +08:00
|
|
|
bytes, err := base.EncodeSchedulerEntry(e)
|
2020-09-27 08:33:29 +08:00
|
|
|
if err != nil {
|
|
|
|
continue // skip bad data
|
|
|
|
}
|
|
|
|
args = append(args, bytes)
|
|
|
|
}
|
2021-12-11 01:07:41 +08:00
|
|
|
exp := r.clock.Now().Add(ttl).UTC()
|
2020-09-27 08:33:29 +08:00
|
|
|
key := base.SchedulerEntriesKey(schedulerID)
|
2021-11-16 08:34:26 +08:00
|
|
|
err := r.client.ZAdd(ctx, base.AllSchedulers, &redis.Z{Score: float64(exp.Unix()), Member: key}).Err()
|
2020-09-27 08:33:29 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "zadd", Err: err})
|
2020-09-27 08:33:29 +08:00
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, writeSchedulerEntriesCmd, []string{key}, args...)
|
2020-09-27 08:33:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// ClearSchedulerEntries deletes scheduler entries data from redis.
|
|
|
|
func (r *RDB) ClearSchedulerEntries(scheduelrID string) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.ClearSchedulerEntries"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2020-09-27 08:33:29 +08:00
|
|
|
key := base.SchedulerEntriesKey(scheduelrID)
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.ZRem(ctx, base.AllSchedulers, key).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "zrem", Err: err})
|
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.Del(ctx, key).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "del", Err: err})
|
2020-09-27 08:33:29 +08:00
|
|
|
}
|
2021-05-11 00:16:13 +08:00
|
|
|
return nil
|
2020-09-27 08:33:29 +08:00
|
|
|
}
|
|
|
|
|
2020-02-13 09:12:09 +08:00
|
|
|
// CancelationPubSub returns a pubsub for cancelation messages.
|
|
|
|
func (r *RDB) CancelationPubSub() (*redis.PubSub, error) {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.CancelationPubSub"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
|
|
|
pubsub := r.client.Subscribe(ctx, base.CancelChannel)
|
|
|
|
_, err := pubsub.Receive(ctx)
|
2020-02-13 09:12:09 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return nil, errors.E(op, errors.Unknown, fmt.Sprintf("redis pubsub receive error: %v", err))
|
2020-02-13 09:12:09 +08:00
|
|
|
}
|
|
|
|
return pubsub, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// PublishCancelation publish cancelation message to all subscribers.
|
2020-02-13 09:33:41 +08:00
|
|
|
// The message is the ID for the task to be canceled.
|
2020-02-13 09:12:09 +08:00
|
|
|
func (r *RDB) PublishCancelation(id string) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.PublishCancelation"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
|
|
|
if err := r.client.Publish(ctx, base.CancelChannel, id).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, fmt.Sprintf("redis pubsub publish error: %v", err))
|
|
|
|
}
|
|
|
|
return nil
|
2020-02-13 09:12:09 +08:00
|
|
|
}
|
2020-09-27 08:33:29 +08:00
|
|
|
|
|
|
|
// KEYS[1] -> asynq:scheduler_history:<entryID>
|
|
|
|
// ARGV[1] -> enqueued_at timestamp
|
|
|
|
// ARGV[2] -> serialized SchedulerEnqueueEvent data
|
|
|
|
// ARGV[3] -> max number of events to be persisted
|
|
|
|
var recordSchedulerEnqueueEventCmd = redis.NewScript(`
|
2021-01-26 14:32:37 +08:00
|
|
|
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, -ARGV[3])
|
2020-09-27 08:33:29 +08:00
|
|
|
redis.call("ZADD", KEYS[1], ARGV[1], ARGV[2])
|
|
|
|
return redis.status_reply("OK")`)
|
|
|
|
|
|
|
|
// Maximum number of enqueue events to store per entry.
|
2021-01-26 14:32:37 +08:00
|
|
|
const maxEvents = 1000
|
2020-09-27 08:33:29 +08:00
|
|
|
|
|
|
|
// RecordSchedulerEnqueueEvent records the time when the given task was enqueued.
|
|
|
|
func (r *RDB) RecordSchedulerEnqueueEvent(entryID string, event *base.SchedulerEnqueueEvent) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.RecordSchedulerEnqueueEvent"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2021-03-13 08:23:08 +08:00
|
|
|
data, err := base.EncodeSchedulerEnqueueEvent(event)
|
2020-09-27 08:33:29 +08:00
|
|
|
if err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode scheduler enqueue event: %v", err))
|
|
|
|
}
|
|
|
|
keys := []string{
|
|
|
|
base.SchedulerHistoryKey(entryID),
|
2020-09-27 08:33:29 +08:00
|
|
|
}
|
2021-05-11 00:16:13 +08:00
|
|
|
argv := []interface{}{
|
|
|
|
event.EnqueuedAt.Unix(),
|
|
|
|
data,
|
|
|
|
maxEvents,
|
|
|
|
}
|
2021-11-16 08:34:26 +08:00
|
|
|
return r.runScript(ctx, op, recordSchedulerEnqueueEventCmd, keys, argv...)
|
2020-09-27 08:33:29 +08:00
|
|
|
}
|
2021-01-26 14:32:37 +08:00
|
|
|
|
|
|
|
// ClearSchedulerHistory deletes the enqueue event history for the given scheduler entry.
|
|
|
|
func (r *RDB) ClearSchedulerHistory(entryID string) error {
|
2021-05-11 00:16:13 +08:00
|
|
|
var op errors.Op = "rdb.ClearSchedulerHistory"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2021-01-26 14:32:37 +08:00
|
|
|
key := base.SchedulerHistoryKey(entryID)
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.Del(ctx, key).Err(); err != nil {
|
2021-05-11 00:16:13 +08:00
|
|
|
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "del", Err: err})
|
|
|
|
}
|
|
|
|
return nil
|
2021-01-26 14:32:37 +08:00
|
|
|
}
|
2021-11-06 07:52:54 +08:00
|
|
|
|
|
|
|
// WriteResult writes the given result data for the specified task.
|
|
|
|
func (r *RDB) WriteResult(qname, taskID string, data []byte) (int, error) {
|
|
|
|
var op errors.Op = "rdb.WriteResult"
|
2021-11-16 08:34:26 +08:00
|
|
|
ctx := context.Background()
|
2021-11-06 07:52:54 +08:00
|
|
|
taskKey := base.TaskKey(qname, taskID)
|
2021-11-16 08:34:26 +08:00
|
|
|
if err := r.client.HSet(ctx, taskKey, "result", data).Err(); err != nil {
|
2021-11-06 07:52:54 +08:00
|
|
|
return 0, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "hset", Err: err})
|
|
|
|
}
|
|
|
|
return len(data), nil
|
|
|
|
}
|