mirror of
https://github.com/hibiken/asynq.git
synced 2024-11-10 11:31:58 +08:00
Allow user to specify retry delay duration
This commit is contained in:
parent
9af14d9a6d
commit
22b21df884
@ -3,6 +3,8 @@ package asynq
|
|||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"log"
|
"log"
|
||||||
|
"math"
|
||||||
|
"math/rand"
|
||||||
"os"
|
"os"
|
||||||
"os/signal"
|
"os/signal"
|
||||||
"sync"
|
"sync"
|
||||||
@ -43,7 +45,21 @@ type Config struct {
|
|||||||
// TODO(hibiken): Add ShutdownTimeout
|
// TODO(hibiken): Add ShutdownTimeout
|
||||||
// ShutdownTimeout time.Duration
|
// ShutdownTimeout time.Duration
|
||||||
|
|
||||||
// TODO(hibiken): Add RetryDelayFunc
|
// Function to calculate retry delay for a failed task.
|
||||||
|
//
|
||||||
|
// By default, it uses exponential backoff algorithm to calculate the delay.
|
||||||
|
//
|
||||||
|
// n is the number of times the task has been retried.
|
||||||
|
// e is the error returned by the task handler.
|
||||||
|
// t is the task in question. t is read-only, the function should not mutate t.
|
||||||
|
RetryDelayFunc func(n int, e error, t *Task) time.Duration
|
||||||
|
}
|
||||||
|
|
||||||
|
// Formula taken from https://github.com/mperham/sidekiq.
|
||||||
|
func defaultDelayFunc(n int, e error, t *Task) time.Duration {
|
||||||
|
r := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||||
|
s := int(math.Pow(float64(n), 4)) + 15 + (r.Intn(30) * (n + 1))
|
||||||
|
return time.Duration(s) * time.Second
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewBackground returns a new Background instance given a redis client
|
// NewBackground returns a new Background instance given a redis client
|
||||||
@ -53,9 +69,13 @@ func NewBackground(r *redis.Client, cfg *Config) *Background {
|
|||||||
if n < 1 {
|
if n < 1 {
|
||||||
n = 1
|
n = 1
|
||||||
}
|
}
|
||||||
|
delayFunc := cfg.RetryDelayFunc
|
||||||
|
if delayFunc == nil {
|
||||||
|
delayFunc = defaultDelayFunc
|
||||||
|
}
|
||||||
rdb := rdb.NewRDB(r)
|
rdb := rdb.NewRDB(r)
|
||||||
scheduler := newScheduler(rdb, 5*time.Second)
|
scheduler := newScheduler(rdb, 5*time.Second)
|
||||||
processor := newProcessor(rdb, n, nil)
|
processor := newProcessor(rdb, n, delayFunc)
|
||||||
return &Background{
|
return &Background{
|
||||||
rdb: rdb,
|
rdb: rdb,
|
||||||
scheduler: scheduler,
|
scheduler: scheduler,
|
||||||
@ -70,6 +90,9 @@ func NewBackground(r *redis.Client, cfg *Config) *Background {
|
|||||||
//
|
//
|
||||||
// If ProcessTask return a non-nil error or panics, the task
|
// If ProcessTask return a non-nil error or panics, the task
|
||||||
// will be retried after delay.
|
// will be retried after delay.
|
||||||
|
//
|
||||||
|
// Note: The argument task is ready only, ProcessTask should
|
||||||
|
// not mutate the task.
|
||||||
type Handler interface {
|
type Handler interface {
|
||||||
ProcessTask(*Task) error
|
ProcessTask(*Task) error
|
||||||
}
|
}
|
||||||
|
36
processor.go
36
processor.go
@ -3,8 +3,6 @@ package asynq
|
|||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"log"
|
"log"
|
||||||
"math"
|
|
||||||
"math/rand"
|
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
@ -17,6 +15,8 @@ type processor struct {
|
|||||||
|
|
||||||
handler Handler
|
handler Handler
|
||||||
|
|
||||||
|
retryDelayFunc retryDelayFunc
|
||||||
|
|
||||||
// timeout for blocking dequeue operation.
|
// timeout for blocking dequeue operation.
|
||||||
// dequeue needs to timeout to avoid blocking forever
|
// dequeue needs to timeout to avoid blocking forever
|
||||||
// in case of a program shutdown or additon of a new queue.
|
// in case of a program shutdown or additon of a new queue.
|
||||||
@ -38,15 +38,18 @@ type processor struct {
|
|||||||
quit chan struct{}
|
quit chan struct{}
|
||||||
}
|
}
|
||||||
|
|
||||||
func newProcessor(r *rdb.RDB, numWorkers int, handler Handler) *processor {
|
type retryDelayFunc func(n int, err error, task *Task) time.Duration
|
||||||
|
|
||||||
|
func newProcessor(r *rdb.RDB, n int, fn retryDelayFunc) *processor {
|
||||||
return &processor{
|
return &processor{
|
||||||
rdb: r,
|
rdb: r,
|
||||||
handler: handler,
|
retryDelayFunc: fn,
|
||||||
dequeueTimeout: 2 * time.Second,
|
dequeueTimeout: 2 * time.Second,
|
||||||
sema: make(chan struct{}, numWorkers),
|
sema: make(chan struct{}, n),
|
||||||
done: make(chan struct{}),
|
done: make(chan struct{}),
|
||||||
abort: make(chan struct{}),
|
abort: make(chan struct{}),
|
||||||
quit: make(chan struct{}),
|
quit: make(chan struct{}),
|
||||||
|
handler: HandlerFunc(func(t *Task) error { return fmt.Errorf("handler not set") }),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -136,9 +139,9 @@ func (p *processor) exec() {
|
|||||||
// 3) Kill -> Removes the message from InProgress & Adds the message to Dead
|
// 3) Kill -> Removes the message from InProgress & Adds the message to Dead
|
||||||
if resErr != nil {
|
if resErr != nil {
|
||||||
if msg.Retried >= msg.Retry {
|
if msg.Retried >= msg.Retry {
|
||||||
p.kill(msg, resErr.Error())
|
p.kill(msg, resErr)
|
||||||
} else {
|
} else {
|
||||||
p.retry(msg, resErr.Error())
|
p.retry(msg, resErr)
|
||||||
}
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -174,17 +177,18 @@ func (p *processor) markAsDone(msg *base.TaskMessage) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *processor) retry(msg *base.TaskMessage, errMsg string) {
|
func (p *processor) retry(msg *base.TaskMessage, e error) {
|
||||||
retryAt := time.Now().Add(delaySeconds(msg.Retried))
|
d := p.retryDelayFunc(msg.Retried, e, &Task{Type: msg.Type, Payload: msg.Payload})
|
||||||
err := p.rdb.Retry(msg, retryAt, errMsg)
|
retryAt := time.Now().Add(d)
|
||||||
|
err := p.rdb.Retry(msg, retryAt, e.Error())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("[ERROR] Could not send task %+v to Retry queue: %v\n", msg, err)
|
log.Printf("[ERROR] Could not send task %+v to Retry queue: %v\n", msg, err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *processor) kill(msg *base.TaskMessage, errMsg string) {
|
func (p *processor) kill(msg *base.TaskMessage, e error) {
|
||||||
log.Printf("[WARN] Retry exhausted for task(Type: %q, ID: %v)\n", msg.Type, msg.ID)
|
log.Printf("[WARN] Retry exhausted for task(Type: %q, ID: %v)\n", msg.Type, msg.ID)
|
||||||
err := p.rdb.Kill(msg, errMsg)
|
err := p.rdb.Kill(msg, e.Error())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("[ERROR] Could not send task %+v to Dead queue: %v\n", msg, err)
|
log.Printf("[ERROR] Could not send task %+v to Dead queue: %v\n", msg, err)
|
||||||
}
|
}
|
||||||
@ -201,11 +205,3 @@ func perform(h Handler, task *Task) (err error) {
|
|||||||
}()
|
}()
|
||||||
return h.ProcessTask(task)
|
return h.ProcessTask(task)
|
||||||
}
|
}
|
||||||
|
|
||||||
// delaySeconds returns a number seconds to delay before retrying.
|
|
||||||
// Formula taken from https://github.com/mperham/sidekiq.
|
|
||||||
func delaySeconds(count int) time.Duration {
|
|
||||||
r := rand.New(rand.NewSource(time.Now().UnixNano()))
|
|
||||||
s := int(math.Pow(float64(count), 4)) + 15 + (r.Intn(30) * (count + 1))
|
|
||||||
return time.Duration(s) * time.Second
|
|
||||||
}
|
|
||||||
|
@ -59,7 +59,8 @@ func TestProcessorSuccess(t *testing.T) {
|
|||||||
processed = append(processed, task)
|
processed = append(processed, task)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
p := newProcessor(rdbClient, 10, HandlerFunc(handler))
|
p := newProcessor(rdbClient, 10, defaultDelayFunc)
|
||||||
|
p.handler = HandlerFunc(handler)
|
||||||
p.dequeueTimeout = time.Second // short time out for test purpose
|
p.dequeueTimeout = time.Second // short time out for test purpose
|
||||||
|
|
||||||
p.start()
|
p.start()
|
||||||
@ -107,19 +108,27 @@ func TestProcessorRetry(t *testing.T) {
|
|||||||
r4.ErrorMsg = errMsg
|
r4.ErrorMsg = errMsg
|
||||||
r4.Retried = m4.Retried + 1
|
r4.Retried = m4.Retried + 1
|
||||||
|
|
||||||
|
now := time.Now()
|
||||||
|
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
enqueued []*base.TaskMessage // initial default queue state
|
enqueued []*base.TaskMessage // initial default queue state
|
||||||
incoming []*base.TaskMessage // tasks to be enqueued during run
|
incoming []*base.TaskMessage // tasks to be enqueued during run
|
||||||
|
delay time.Duration // retry delay duration
|
||||||
wait time.Duration // wait duration between starting and stopping processor for this test case
|
wait time.Duration // wait duration between starting and stopping processor for this test case
|
||||||
wantRetry []*base.TaskMessage // tasks in retry queue at the end
|
wantRetry []h.ZSetEntry // tasks in retry queue at the end
|
||||||
wantDead []*base.TaskMessage // tasks in dead queue at the end
|
wantDead []*base.TaskMessage // tasks in dead queue at the end
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
enqueued: []*base.TaskMessage{m1, m2},
|
enqueued: []*base.TaskMessage{m1, m2},
|
||||||
incoming: []*base.TaskMessage{m3, m4},
|
incoming: []*base.TaskMessage{m3, m4},
|
||||||
wait: time.Second,
|
delay: time.Minute,
|
||||||
wantRetry: []*base.TaskMessage{&r2, &r3, &r4},
|
wait: time.Second,
|
||||||
wantDead: []*base.TaskMessage{&r1},
|
wantRetry: []h.ZSetEntry{
|
||||||
|
{Msg: &r2, Score: now.Add(time.Minute).Unix()},
|
||||||
|
{Msg: &r3, Score: now.Add(time.Minute).Unix()},
|
||||||
|
{Msg: &r4, Score: now.Add(time.Minute).Unix()},
|
||||||
|
},
|
||||||
|
wantDead: []*base.TaskMessage{&r1},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -128,10 +137,14 @@ func TestProcessorRetry(t *testing.T) {
|
|||||||
h.SeedDefaultQueue(t, r, tc.enqueued) // initialize default queue.
|
h.SeedDefaultQueue(t, r, tc.enqueued) // initialize default queue.
|
||||||
|
|
||||||
// instantiate a new processor
|
// instantiate a new processor
|
||||||
|
delayFunc := func(n int, e error, t *Task) time.Duration {
|
||||||
|
return tc.delay
|
||||||
|
}
|
||||||
handler := func(task *Task) error {
|
handler := func(task *Task) error {
|
||||||
return fmt.Errorf(errMsg)
|
return fmt.Errorf(errMsg)
|
||||||
}
|
}
|
||||||
p := newProcessor(rdbClient, 10, HandlerFunc(handler))
|
p := newProcessor(rdbClient, 10, delayFunc)
|
||||||
|
p.handler = HandlerFunc(handler)
|
||||||
p.dequeueTimeout = time.Second // short time out for test purpose
|
p.dequeueTimeout = time.Second // short time out for test purpose
|
||||||
|
|
||||||
p.start()
|
p.start()
|
||||||
@ -145,8 +158,8 @@ func TestProcessorRetry(t *testing.T) {
|
|||||||
time.Sleep(tc.wait)
|
time.Sleep(tc.wait)
|
||||||
p.terminate()
|
p.terminate()
|
||||||
|
|
||||||
gotRetry := h.GetRetryMessages(t, r)
|
gotRetry := h.GetRetryEntries(t, r)
|
||||||
if diff := cmp.Diff(tc.wantRetry, gotRetry, h.SortMsgOpt); diff != "" {
|
if diff := cmp.Diff(tc.wantRetry, gotRetry, h.SortZSetEntryOpt); diff != "" {
|
||||||
t.Errorf("mismatch found in %q after running processor; (-want, +got)\n%s", base.RetryQueue, diff)
|
t.Errorf("mismatch found in %q after running processor; (-want, +got)\n%s", base.RetryQueue, diff)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user