From 3fa867b190bc713efe316a0afb0a72f6fc4ef031 Mon Sep 17 00:00:00 2001 From: Ken Hibino Date: Sat, 23 Nov 2019 08:21:32 -0800 Subject: [PATCH 1/4] Remove per task heartbeat --- processor.go | 19 ------------------- rdb.go | 19 ------------------- 2 files changed, 38 deletions(-) diff --git a/processor.go b/processor.go index 87a3570..71770cb 100644 --- a/processor.go +++ b/processor.go @@ -81,31 +81,12 @@ 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) - } <-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) diff --git a/rdb.go b/rdb.go index d4e00e7..5a4b692 100644 --- a/rdb.go +++ b/rdb.go @@ -9,7 +9,6 @@ import ( "time" "github.com/go-redis/redis/v7" - "github.com/google/uuid" ) // Redis keys @@ -147,24 +146,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 From 76ceb282a906a40e23e9c05a3e8852584fc5bcfe Mon Sep 17 00:00:00 2001 From: Ken Hibino Date: Sat, 23 Nov 2019 08:24:16 -0800 Subject: [PATCH 2/4] Change the direction of list push/pop operations --- rdb.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/rdb.go b/rdb.go index 5a4b692..d394c82 100644 --- a/rdb.go +++ b/rdb.go @@ -49,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) @@ -60,8 +60,8 @@ 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() + // TODO(hibiken): Make BRPOP & SADD atomic. + res, err := r.client.BRPop(timeout, keys...).Result() if err != nil { if err != redis.Nil { return nil, fmt.Errorf("command BLPOP %v %v failed: %v", timeout, keys, err) From 4a327933bdd17725bdfd4703df7ec6bdcf472954 Mon Sep 17 00:00:00 2001 From: Ken Hibino Date: Sat, 23 Nov 2019 08:43:41 -0800 Subject: [PATCH 3/4] Use command BRPUSHLPOP to move from queue to in_progress in redis --- processor.go | 9 ++++----- rdb.go | 37 ++++++++++++++++--------------------- rdb_test.go | 6 +++--- 3 files changed, 23 insertions(+), 29 deletions(-) diff --git a/processor.go b/processor.go index 71770cb..a50e474 100644 --- a/processor.go +++ b/processor.go @@ -59,11 +59,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: @@ -82,8 +81,8 @@ func (p *processor) exec() { p.sema <- struct{}{} // acquire token go func(task *Task) { defer func() { - if err := p.rdb.srem(inProgress, msg); err != nil { - log.Printf("[SERVER ERROR] SREM 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 }() diff --git a/rdb.go b/rdb.go index d394c82..ee50e43 100644 --- a/rdb.go +++ b/rdb.go @@ -13,13 +13,13 @@ import ( // Redis keys const ( - queuePrefix = "asynq:queues:" // LIST - asynq:queues: - 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: + queuePrefix = "asynq:queues:" // LIST - asynq:queues: + 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 ( @@ -58,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 BRPOP & SADD atomic. - res, err := r.client.BRPop(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 } diff --git a/rdb_test.go b/rdb_test.go index c78d69f..fd5410e 100644 --- a/rdb_test.go +++ b/rdb_test.go @@ -63,7 +63,7 @@ func TestDequeueImmediateReturn(t *testing.T) { } 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 +71,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,7 +87,7 @@ 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) } From fd80126a6756bd6cefa56557ba4a8875ee9d3784 Mon Sep 17 00:00:00 2001 From: Ken Hibino Date: Sat, 23 Nov 2019 15:09:50 -0800 Subject: [PATCH 4/4] Add logic to restore unfinished tasks back into the default queue if there are any uncompleted tasks --- processor.go | 12 +++++++++++ rdb.go | 13 ++++++++++++ rdb_test.go | 58 +++++++++++++++++++++++++++++++++++++++++----------- 3 files changed, 71 insertions(+), 12 deletions(-) diff --git a/processor.go b/processor.go index a50e474..8459f7f 100644 --- a/processor.go +++ b/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 { @@ -92,3 +95,12 @@ func (p *processor) exec() { } }(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) + } +} diff --git a/rdb.go b/rdb.go index ee50e43..3be97d5 100644 --- a/rdb.go +++ b/rdb.go @@ -171,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) +} diff --git a/rdb_test.go b/rdb_test.go index fd5410e..cec2de9 100644 --- a/rdb_test.go +++ b/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,12 +64,7 @@ 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("asynq:queues:csv", time.Second) @@ -92,3 +96,33 @@ func TestDequeueTimeout(t *testing.T) { 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)) + } +}