mirror of
https://github.com/hibiken/asynq.git
synced 2024-11-10 11:31:58 +08:00
Merge pull request #1 from hibiken/feature/simple_rescue
Rescue dead "In Progress" tasks
This commit is contained in:
commit
847d724985
40
processor.go
40
processor.go
@ -41,6 +41,9 @@ func (p *processor) terminate() {
|
||||
}
|
||||
|
||||
func (p *processor) start() {
|
||||
// NOTE: The call to "restore" needs to complete before starting
|
||||
// the processor goroutine.
|
||||
p.restore()
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
@ -59,11 +62,10 @@ func (p *processor) start() {
|
||||
// exec pulls a task out of the queue and starts a worker goroutine to
|
||||
// process the task.
|
||||
func (p *processor) exec() {
|
||||
// NOTE: BLPOP needs to timeout to avoid blocking forever
|
||||
// NOTE: dequeue needs to timeout to avoid blocking forever
|
||||
// in case of a program shutdown or additon of a new queue.
|
||||
const timeout = 5 * time.Second
|
||||
// TODO(hibiken): sort the list of queues in order of priority
|
||||
msg, err := p.rdb.dequeue(timeout, p.rdb.listQueues()...)
|
||||
msg, err := p.rdb.dequeue(defaultQueue, timeout)
|
||||
if err != nil {
|
||||
switch err {
|
||||
case errQueuePopTimeout:
|
||||
@ -81,34 +83,24 @@ func (p *processor) exec() {
|
||||
task := &Task{Type: msg.Type, Payload: msg.Payload}
|
||||
p.sema <- struct{}{} // acquire token
|
||||
go func(task *Task) {
|
||||
quit := make(chan struct{}) // channel to signal heartbeat goroutine
|
||||
defer func() {
|
||||
quit <- struct{}{}
|
||||
if err := p.rdb.srem(inProgress, msg); err != nil {
|
||||
log.Printf("[SERVER ERROR] SREM failed: %v\n", err)
|
||||
}
|
||||
if err := p.rdb.clearHeartbeat(msg.ID); err != nil {
|
||||
log.Printf("[SERVER ERROR] DEL heartbeat failed: %v\n", err)
|
||||
if err := p.rdb.lrem(inProgress, msg); err != nil {
|
||||
log.Printf("[SERVER ERROR] LREM failed: %v\n", err)
|
||||
}
|
||||
<-p.sema // release token
|
||||
}()
|
||||
// start "heartbeat" goroutine
|
||||
go func() {
|
||||
ticker := time.NewTicker(5 * time.Second)
|
||||
for {
|
||||
select {
|
||||
case <-quit:
|
||||
return
|
||||
case t := <-ticker.C:
|
||||
if err := p.rdb.heartbeat(msg.ID, t); err != nil {
|
||||
log.Printf("[ERROR] heartbeat failed for %v at %v: %v", msg.ID, t, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
err := p.handler(task) // TODO(hibiken): maybe also handle panic?
|
||||
if err != nil {
|
||||
retryTask(p.rdb, msg, err)
|
||||
}
|
||||
}(task)
|
||||
}
|
||||
|
||||
// restore moves all tasks from "in-progress" back to queue
|
||||
// to restore all unfinished tasks.
|
||||
func (p *processor) restore() {
|
||||
err := p.rdb.moveAll(inProgress, defaultQueue)
|
||||
if err != nil {
|
||||
log.Printf("[SERVER ERROR] could not move tasks from %q to %q\n", inProgress, defaultQueue)
|
||||
}
|
||||
}
|
||||
|
71
rdb.go
71
rdb.go
@ -9,18 +9,17 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/go-redis/redis/v7"
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
// Redis keys
|
||||
const (
|
||||
queuePrefix = "asynq:queues:" // LIST - asynq:queues:<qname>
|
||||
allQueues = "asynq:queues" // SET
|
||||
scheduled = "asynq:scheduled" // ZSET
|
||||
retry = "asynq:retry" // ZSET
|
||||
dead = "asynq:dead" // ZSET
|
||||
inProgress = "asynq:in_progress" // SET
|
||||
heartbeatPrefix = "asynq:heartbeat:" // STRING - asynq:heartbeat:<taskID>
|
||||
queuePrefix = "asynq:queues:" // LIST - asynq:queues:<qname>
|
||||
defaultQueue = queuePrefix + "default" // LIST
|
||||
allQueues = "asynq:queues" // SET
|
||||
scheduled = "asynq:scheduled" // ZSET
|
||||
retry = "asynq:retry" // ZSET
|
||||
dead = "asynq:dead" // ZSET
|
||||
inProgress = "asynq:in_progress" // SET
|
||||
)
|
||||
|
||||
var (
|
||||
@ -50,7 +49,7 @@ func (r *rdb) push(msg *taskMessage) error {
|
||||
return fmt.Errorf("command SADD %q %q failed: %v",
|
||||
allQueues, qname, err)
|
||||
}
|
||||
err = r.client.RPush(qname, string(bytes)).Err()
|
||||
err = r.client.LPush(qname, string(bytes)).Err()
|
||||
if err != nil {
|
||||
return fmt.Errorf("command RPUSH %q %q failed: %v",
|
||||
qname, string(bytes), err)
|
||||
@ -59,38 +58,33 @@ func (r *rdb) push(msg *taskMessage) error {
|
||||
}
|
||||
|
||||
// dequeue blocks until there is a taskMessage available to be processed,
|
||||
// once available, it adds the task to "in progress" set and returns the task.
|
||||
func (r *rdb) dequeue(timeout time.Duration, keys ...string) (*taskMessage, error) {
|
||||
// TODO(hibiken): Make BLPOP & SADD atomic.
|
||||
res, err := r.client.BLPop(timeout, keys...).Result()
|
||||
// once available, it adds the task to "in progress" list and returns the task.
|
||||
func (r *rdb) dequeue(qname string, timeout time.Duration) (*taskMessage, error) {
|
||||
data, err := r.client.BRPopLPush(qname, inProgress, timeout).Result()
|
||||
if err != nil {
|
||||
if err != redis.Nil {
|
||||
return nil, fmt.Errorf("command BLPOP %v %v failed: %v", timeout, keys, err)
|
||||
return nil, fmt.Errorf("command BRPOPLPUSH %q %q %v failed: %v", qname, inProgress, timeout, err)
|
||||
}
|
||||
return nil, errQueuePopTimeout
|
||||
}
|
||||
q, data := res[0], res[1]
|
||||
err = r.client.SAdd(inProgress, data).Err()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("command SADD %q %v failed: %v", inProgress, data, err)
|
||||
}
|
||||
var msg taskMessage
|
||||
err = json.Unmarshal([]byte(data), &msg)
|
||||
if err != nil {
|
||||
return nil, errDeserializeTask
|
||||
}
|
||||
fmt.Printf("[DEBUG] perform task %+v from %s\n", msg, q)
|
||||
fmt.Printf("[DEBUG] perform task %+v from %s\n", msg, qname)
|
||||
return &msg, nil
|
||||
}
|
||||
|
||||
func (r *rdb) srem(key string, msg *taskMessage) error {
|
||||
func (r *rdb) lrem(key string, msg *taskMessage) error {
|
||||
bytes, err := json.Marshal(msg)
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not encode task into JSON: %v", err)
|
||||
}
|
||||
err = r.client.SRem(key, string(bytes)).Err()
|
||||
// NOTE: count ZERO means "remove all elements equal to val"
|
||||
err = r.client.LRem(key, 0, string(bytes)).Err()
|
||||
if err != nil {
|
||||
return fmt.Errorf("command SREM %s %s failed: %v", key, string(bytes), err)
|
||||
return fmt.Errorf("command LREM %s 0 %s failed: %v", key, string(bytes), err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@ -147,24 +141,6 @@ func (r *rdb) move(from string, msg *taskMessage) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *rdb) heartbeat(id uuid.UUID, timestamp time.Time) error {
|
||||
key := heartbeatPrefix + id.String()
|
||||
err := r.client.Set(key, timestamp, 0).Err() // zero expiration means no expiration
|
||||
if err != nil {
|
||||
return fmt.Errorf("command SET %s %v failed: %v", key, timestamp, err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *rdb) clearHeartbeat(id uuid.UUID) error {
|
||||
key := heartbeatPrefix + id.String()
|
||||
err := r.client.Del(key).Err()
|
||||
if err != nil {
|
||||
return fmt.Errorf("command DEL %s failed: %v", key, err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
const maxDeadTask = 100
|
||||
const deadExpirationInDays = 90
|
||||
|
||||
@ -195,3 +171,16 @@ func (r *rdb) listQueues() []string {
|
||||
}
|
||||
return queues
|
||||
}
|
||||
|
||||
// moveAll moves all tasks from src list to dst list.
|
||||
func (r *rdb) moveAll(src, dst string) error {
|
||||
// TODO(hibiken): Lua script
|
||||
txf := func(tx *redis.Tx) error {
|
||||
length := tx.LLen(src).Val()
|
||||
for i := 0; i < int(length); i++ {
|
||||
tx.RPopLPush(src, dst)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
return r.client.Watch(txf, src)
|
||||
}
|
||||
|
64
rdb_test.go
64
rdb_test.go
@ -2,6 +2,7 @@ package asynq
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"math/rand"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@ -12,6 +13,10 @@ import (
|
||||
|
||||
var client *redis.Client
|
||||
|
||||
func init() {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
}
|
||||
|
||||
// setup connects to a redis database and flush all keys
|
||||
// before returning an instance of rdb.
|
||||
func setup() *rdb {
|
||||
@ -26,14 +31,18 @@ func setup() *rdb {
|
||||
return newRDB(client)
|
||||
}
|
||||
|
||||
func randomTask(taskType, qname string) *taskMessage {
|
||||
return &taskMessage{
|
||||
ID: uuid.New(),
|
||||
Type: taskType,
|
||||
Queue: qname,
|
||||
Retry: rand.Intn(100),
|
||||
}
|
||||
}
|
||||
|
||||
func TestPush(t *testing.T) {
|
||||
r := setup()
|
||||
msg := &taskMessage{
|
||||
ID: uuid.New(),
|
||||
Type: "sendEmail",
|
||||
Queue: "default",
|
||||
Retry: 10,
|
||||
}
|
||||
msg := randomTask("send_email", "default")
|
||||
|
||||
err := r.push(msg)
|
||||
if err != nil {
|
||||
@ -55,15 +64,10 @@ func TestPush(t *testing.T) {
|
||||
|
||||
func TestDequeueImmediateReturn(t *testing.T) {
|
||||
r := setup()
|
||||
msg := &taskMessage{
|
||||
ID: uuid.New(),
|
||||
Type: "GenerateCSVExport",
|
||||
Queue: "csv",
|
||||
Retry: 10,
|
||||
}
|
||||
msg := randomTask("export_csv", "csv")
|
||||
r.push(msg)
|
||||
|
||||
res, err := r.dequeue(time.Second, "asynq:queues:csv")
|
||||
res, err := r.dequeue("asynq:queues:csv", time.Second)
|
||||
if err != nil {
|
||||
t.Fatalf("r.bpop() failed: %v", err)
|
||||
}
|
||||
@ -71,7 +75,7 @@ func TestDequeueImmediateReturn(t *testing.T) {
|
||||
if !cmp.Equal(res, msg) {
|
||||
t.Errorf("cmp.Equal(res, msg) = %t, want %t", false, true)
|
||||
}
|
||||
jobs := client.SMembers(inProgress).Val()
|
||||
jobs := client.LRange(inProgress, 0, -1).Val()
|
||||
if len(jobs) != 1 {
|
||||
t.Fatalf("len(jobs) = %d, want %d", len(jobs), 1)
|
||||
}
|
||||
@ -87,8 +91,38 @@ func TestDequeueImmediateReturn(t *testing.T) {
|
||||
func TestDequeueTimeout(t *testing.T) {
|
||||
r := setup()
|
||||
|
||||
_, err := r.dequeue(time.Second, "asynq:queues:default")
|
||||
_, err := r.dequeue("asynq:queues:default", time.Second)
|
||||
if err != errQueuePopTimeout {
|
||||
t.Errorf("err = %v, want %v", err, errQueuePopTimeout)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMoveAll(t *testing.T) {
|
||||
r := setup()
|
||||
seed := []*taskMessage{
|
||||
randomTask("send_email", "default"),
|
||||
randomTask("export_csv", "csv"),
|
||||
randomTask("sync_stuff", "sync"),
|
||||
}
|
||||
for _, task := range seed {
|
||||
bytes, err := json.Marshal(task)
|
||||
if err != nil {
|
||||
t.Errorf("json.Marhsal() failed: %v", err)
|
||||
}
|
||||
if err := client.LPush(inProgress, string(bytes)).Err(); err != nil {
|
||||
t.Errorf("LPUSH %q %s failed: %v", inProgress, string(bytes), err)
|
||||
}
|
||||
}
|
||||
|
||||
err := r.moveAll(inProgress, defaultQueue)
|
||||
if err != nil {
|
||||
t.Errorf("moveAll failed: %v", err)
|
||||
}
|
||||
|
||||
if l := client.LLen(inProgress).Val(); l != 0 {
|
||||
t.Errorf("LLEN %q = %d, want 0", inProgress, l)
|
||||
}
|
||||
if l := client.LLen(defaultQueue).Val(); int(l) != len(seed) {
|
||||
t.Errorf("LLEN %q = %d, want %d", defaultQueue, l, len(seed))
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user