2
0
mirror of https://github.com/hibiken/asynq.git synced 2025-10-20 09:16:12 +08:00

Compare commits

..

27 Commits

Author SHA1 Message Date
Ken Hibino
a19909f5f4 v0.22.0 2022-02-19 06:20:05 -08:00
Ken Hibino
cea5110d15 Add IsOrphaned field to TaskInfo 2022-02-19 06:15:44 -08:00
Ken Hibino
9b63e23274 Update log messages 2022-02-19 06:15:44 -08:00
Ken Hibino
de25201d9f Make timeutil.SimulatedClock concurrency safe 2022-02-19 06:15:44 -08:00
Ken Hibino
ec560afb01 Fix processor test 2022-02-19 06:15:44 -08:00
Ken Hibino
d4006894ad Remove base.DeadlinesKey 2022-02-19 06:15:44 -08:00
Ken Hibino
59927509d8 Remove timeout and deadline fields under task key 2022-02-19 06:15:44 -08:00
Ken Hibino
8211167de2 Update processor to create a lease and watch for expiration 2022-02-19 06:15:44 -08:00
Ken Hibino
d7169cd445 Update heartbeat to extend lease of active workers 2022-02-19 06:15:44 -08:00
Ken Hibino
dfae8638e1 Update RDB methods to work with lease 2022-02-19 06:15:44 -08:00
Ken Hibino
b9943de2ab Add Lease type to base package 2022-02-19 06:15:44 -08:00
Ken Hibino
871474f220 Update heartbeat goroutine to call ExtendLease on active tasks 2022-02-19 06:15:44 -08:00
Ken Hibino
87dc392c7f Add RDB.ExtendLease method 2022-02-19 06:15:44 -08:00
Ken Hibino
dabcb120d5 Update recoverer to use ListLeaseExpired 2022-02-19 06:15:44 -08:00
Ken Hibino
bc2f1986d7 Update ListDeadlineExceeded to ListLeaseExpired 2022-02-19 06:15:44 -08:00
Ken Hibino
b8cb579407 Update RDB methods to use lease instead of deadlines set 2022-02-19 06:15:44 -08:00
Ken Hibino
bca624792c Move task deadline compute logic to processor 2022-02-19 06:15:44 -08:00
Ken Hibino
d865d89900 Update RDB.Dequeue to insert task ID to lease set 2022-02-19 06:15:44 -08:00
Ken Hibino
852af7abd1 Add base.LeaseKey helper function 2022-02-19 06:15:44 -08:00
Ken Hibino
5490d2c625 Fix tests 2022-02-16 07:08:01 -08:00
Binaek Sarkar
ebd7a32c0f conventions 2022-02-16 06:43:08 -08:00
Binaek Sarkar
55d0610a03 test and changelog 2022-02-16 06:43:08 -08:00
Binaek Sarkar
ab8a4f5b1e review corrections 2022-02-16 06:43:08 -08:00
Binaek Sarkar
d7ceb0c090 first cut 2022-02-16 06:43:08 -08:00
Ken Hibino
8bd70c6f84 (ci): Run go (build|test) commands for each module 2022-02-01 07:00:00 -08:00
Ken Hibino
10ab4e3745 Remove replace directives in go.mod 2022-02-01 06:18:41 -08:00
Ken Hibino
349f4c50fb Add example for ResultWriter 2022-01-31 09:08:41 -08:00
34 changed files with 1474 additions and 654 deletions

View File

@@ -22,12 +22,21 @@ jobs:
with: with:
go-version: ${{ matrix.go-version }} go-version: ${{ matrix.go-version }}
- name: Build - name: Build core module
run: go build -v ./... run: go build -v ./...
- name: Test - name: Build x module
run: cd x && go build -v ./... && cd ..
- name: Build tools module
run: cd tools && go build -v ./... && cd ..
- name: Test core module
run: go test -race -v -coverprofile=coverage.txt -covermode=atomic ./... run: go test -race -v -coverprofile=coverage.txt -covermode=atomic ./...
- name: Test x module
run: cd x && go test -race -v ./... && cd ..
- name: Benchmark Test - name: Benchmark Test
run: go test -run=^$ -bench=. -loglevel=debug ./... run: go test -run=^$ -bench=. -loglevel=debug ./...

View File

@@ -7,6 +7,17 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
## [Unreleased] ## [Unreleased]
## [0.21.0] - 2022-02-19
### Added
- `BaseContext` is introduced in `Config` to specify callback hook to provide a base `context` from which `Handler` `context` is derived
- `IsOrphaned` field is added to `TaskInfo` to describe a task left in active state with no worker processing it.
### Changed
- `Server` now recovers tasks with an expired lease. Recovered tasks are retried/archived with `ErrLeaseExpired` error.
## [0.21.0] - 2022-01-22 ## [0.21.0] - 2022-01-22
### Added ### Added

View File

@@ -101,6 +101,14 @@ type TaskInfo struct {
// zero if not applicable. // zero if not applicable.
NextProcessAt time.Time NextProcessAt time.Time
// IsOrphaned describes whether the task is left in active state with no worker processing it.
// An orphaned task indicates that the worker has crashed or experienced network failures and was not able to
// extend its lease on the task.
//
// This task will be recovered by running a server against the queue the task is in.
// This field is only applicable to tasks with TaskStateActive.
IsOrphaned bool
// Retention is duration of the retention period after the task is successfully processed. // Retention is duration of the retention period after the task is successfully processed.
Retention time.Duration Retention time.Duration

View File

@@ -5,6 +5,7 @@
package asynq_test package asynq_test
import ( import (
"context"
"fmt" "fmt"
"log" "log"
"os" "os"
@@ -113,3 +114,20 @@ func ExampleParseRedisURI() {
// localhost:6379 // localhost:6379
// 10 // 10
} }
func ExampleResultWriter() {
// ResultWriter is only accessible in Handler.
h := func(ctx context.Context, task *asynq.Task) error {
// .. do task processing work
res := []byte("task result data")
n, err := task.ResultWriter().Write(res) // implements io.Writer
if err != nil {
return fmt.Errorf("failed to write task result: %v", err)
}
log.Printf(" %d bytes written", n)
return nil
}
_ = h
}

View File

@@ -70,6 +70,6 @@ func (f *forwarder) start(wg *sync.WaitGroup) {
func (f *forwarder) exec() { func (f *forwarder) exec() {
if err := f.broker.ForwardIfReady(f.queues...); err != nil { if err := f.broker.ForwardIfReady(f.queues...); err != nil {
f.logger.Errorf("Could not enqueue scheduled tasks: %v", err) f.logger.Errorf("Failed to forward scheduled tasks: %v", err)
} }
} }

2
go.mod
View File

@@ -1,6 +1,6 @@
module github.com/hibiken/asynq module github.com/hibiken/asynq
go 1.13 go 1.14
require ( require (
github.com/go-redis/redis/v8 v8.11.2 github.com/go-redis/redis/v8 v8.11.2

View File

@@ -12,6 +12,7 @@ import (
"github.com/google/uuid" "github.com/google/uuid"
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
"github.com/hibiken/asynq/internal/log" "github.com/hibiken/asynq/internal/log"
"github.com/hibiken/asynq/internal/timeutil"
) )
// heartbeater is responsible for writing process info to redis periodically to // heartbeater is responsible for writing process info to redis periodically to
@@ -19,6 +20,7 @@ import (
type heartbeater struct { type heartbeater struct {
logger *log.Logger logger *log.Logger
broker base.Broker broker base.Broker
clock timeutil.Clock
// channel to communicate back to the long running "heartbeater" goroutine. // channel to communicate back to the long running "heartbeater" goroutine.
done chan struct{} done chan struct{}
@@ -69,6 +71,7 @@ func newHeartbeater(params heartbeaterParams) *heartbeater {
return &heartbeater{ return &heartbeater{
logger: params.logger, logger: params.logger,
broker: params.broker, broker: params.broker,
clock: timeutil.NewRealClock(),
done: make(chan struct{}), done: make(chan struct{}),
interval: params.interval, interval: params.interval,
@@ -100,6 +103,8 @@ type workerInfo struct {
started time.Time started time.Time
// deadline the worker has to finish processing the task by. // deadline the worker has to finish processing the task by.
deadline time.Time deadline time.Time
// lease the worker holds for the task.
lease *base.Lease
} }
func (h *heartbeater) start(wg *sync.WaitGroup) { func (h *heartbeater) start(wg *sync.WaitGroup) {
@@ -107,7 +112,7 @@ func (h *heartbeater) start(wg *sync.WaitGroup) {
go func() { go func() {
defer wg.Done() defer wg.Done()
h.started = time.Now() h.started = h.clock.Now()
h.beat() h.beat()
@@ -134,6 +139,7 @@ func (h *heartbeater) start(wg *sync.WaitGroup) {
}() }()
} }
// beat extends lease for workers and writes server/worker info to redis.
func (h *heartbeater) beat() { func (h *heartbeater) beat() {
h.state.mu.Lock() h.state.mu.Lock()
srvStatus := h.state.value.String() srvStatus := h.state.value.String()
@@ -152,6 +158,7 @@ func (h *heartbeater) beat() {
} }
var ws []*base.WorkerInfo var ws []*base.WorkerInfo
idsByQueue := make(map[string][]string)
for id, w := range h.workers { for id, w := range h.workers {
ws = append(ws, &base.WorkerInfo{ ws = append(ws, &base.WorkerInfo{
Host: h.host, Host: h.host,
@@ -164,11 +171,30 @@ func (h *heartbeater) beat() {
Started: w.started, Started: w.started,
Deadline: w.deadline, Deadline: w.deadline,
}) })
// Check lease before adding to the set to make sure not to extend the lease if the lease is already expired.
if w.lease.IsValid() {
idsByQueue[w.msg.Queue] = append(idsByQueue[w.msg.Queue], id)
} else {
w.lease.NotifyExpiration() // notify processor if the lease is expired
}
} }
// Note: Set TTL to be long enough so that it won't expire before we write again // Note: Set TTL to be long enough so that it won't expire before we write again
// and short enough to expire quickly once the process is shut down or killed. // and short enough to expire quickly once the process is shut down or killed.
if err := h.broker.WriteServerState(&info, ws, h.interval*2); err != nil { if err := h.broker.WriteServerState(&info, ws, h.interval*2); err != nil {
h.logger.Errorf("could not write server state data: %v", err) h.logger.Errorf("Failed to write server state data: %v", err)
}
for qname, ids := range idsByQueue {
expirationTime, err := h.broker.ExtendLease(qname, ids...)
if err != nil {
h.logger.Errorf("Failed to extend lease for tasks %v: %v", ids, err)
continue
}
for _, id := range ids {
if l := h.workers[id].lease; !l.Reset(expirationTime) {
h.logger.Warnf("Lease reset failed for %s; lease deadline: %v", id, l.Deadline())
}
}
} }
} }

View File

@@ -5,6 +5,7 @@
package asynq package asynq
import ( import (
"context"
"sync" "sync"
"testing" "testing"
"time" "time"
@@ -15,21 +16,143 @@ import (
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
"github.com/hibiken/asynq/internal/rdb" "github.com/hibiken/asynq/internal/rdb"
"github.com/hibiken/asynq/internal/testbroker" "github.com/hibiken/asynq/internal/testbroker"
"github.com/hibiken/asynq/internal/timeutil"
) )
// Test goes through a few phases.
//
// Phase1: Simulate Server startup; Simulate starting tasks listed in startedWorkers
// Phase2: Simluate finishing tasks listed in finishedTasks
// Phase3: Simulate Server shutdown;
func TestHeartbeater(t *testing.T) { func TestHeartbeater(t *testing.T) {
r := setup(t) r := setup(t)
defer r.Close() defer r.Close()
rdbClient := rdb.NewRDB(r) rdbClient := rdb.NewRDB(r)
now := time.Now()
const elapsedTime = 10 * time.Second // simulated time elapsed between phase1 and phase2
clock := timeutil.NewSimulatedClock(time.Time{}) // time will be set in each test
t1 := h.NewTaskMessageWithQueue("task1", nil, "default")
t2 := h.NewTaskMessageWithQueue("task2", nil, "default")
t3 := h.NewTaskMessageWithQueue("task3", nil, "default")
t4 := h.NewTaskMessageWithQueue("task4", nil, "custom")
t5 := h.NewTaskMessageWithQueue("task5", nil, "custom")
t6 := h.NewTaskMessageWithQueue("task6", nil, "default")
// Note: intentionally set to time less than now.Add(rdb.LeaseDuration) to test lease extention is working.
lease1 := h.NewLeaseWithClock(now.Add(10*time.Second), clock)
lease2 := h.NewLeaseWithClock(now.Add(10*time.Second), clock)
lease3 := h.NewLeaseWithClock(now.Add(10*time.Second), clock)
lease4 := h.NewLeaseWithClock(now.Add(10*time.Second), clock)
lease5 := h.NewLeaseWithClock(now.Add(10*time.Second), clock)
lease6 := h.NewLeaseWithClock(now.Add(10*time.Second), clock)
tests := []struct { tests := []struct {
interval time.Duration desc string
// Interval between heartbeats.
interval time.Duration
// Server info.
host string host string
pid int pid int
queues map[string]int queues map[string]int
concurrency int concurrency int
active map[string][]*base.TaskMessage // initial active set state
lease map[string][]base.Z // initial lease set state
wantLease1 map[string][]base.Z // expected lease set state after starting all startedWorkers
wantLease2 map[string][]base.Z // expected lease set state after finishing all finishedTasks
startedWorkers []*workerInfo // workerInfo to send via the started channel
finishedTasks []*base.TaskMessage // tasks to send via the finished channel
startTime time.Time // simulated start time
elapsedTime time.Duration // simulated time elapsed between starting and finishing processing tasks
}{ }{
{2 * time.Second, "localhost", 45678, map[string]int{"default": 1}, 10}, {
desc: "With single queue",
interval: 2 * time.Second,
host: "localhost",
pid: 45678,
queues: map[string]int{"default": 1},
concurrency: 10,
active: map[string][]*base.TaskMessage{
"default": {t1, t2, t3},
},
lease: map[string][]base.Z{
"default": {
{Message: t1, Score: now.Add(10 * time.Second).Unix()},
{Message: t2, Score: now.Add(10 * time.Second).Unix()},
{Message: t3, Score: now.Add(10 * time.Second).Unix()},
},
},
startedWorkers: []*workerInfo{
{msg: t1, started: now, deadline: now.Add(2 * time.Minute), lease: lease1},
{msg: t2, started: now, deadline: now.Add(2 * time.Minute), lease: lease2},
{msg: t3, started: now, deadline: now.Add(2 * time.Minute), lease: lease3},
},
finishedTasks: []*base.TaskMessage{t1, t2},
wantLease1: map[string][]base.Z{
"default": {
{Message: t1, Score: now.Add(rdb.LeaseDuration).Unix()},
{Message: t2, Score: now.Add(rdb.LeaseDuration).Unix()},
{Message: t3, Score: now.Add(rdb.LeaseDuration).Unix()},
},
},
wantLease2: map[string][]base.Z{
"default": {
{Message: t3, Score: now.Add(elapsedTime).Add(rdb.LeaseDuration).Unix()},
},
},
startTime: now,
elapsedTime: elapsedTime,
},
{
desc: "With multiple queue",
interval: 2 * time.Second,
host: "localhost",
pid: 45678,
queues: map[string]int{"default": 1, "custom": 2},
concurrency: 10,
active: map[string][]*base.TaskMessage{
"default": {t6},
"custom": {t4, t5},
},
lease: map[string][]base.Z{
"default": {
{Message: t6, Score: now.Add(10 * time.Second).Unix()},
},
"custom": {
{Message: t4, Score: now.Add(10 * time.Second).Unix()},
{Message: t5, Score: now.Add(10 * time.Second).Unix()},
},
},
startedWorkers: []*workerInfo{
{msg: t6, started: now, deadline: now.Add(2 * time.Minute), lease: lease6},
{msg: t4, started: now, deadline: now.Add(2 * time.Minute), lease: lease4},
{msg: t5, started: now, deadline: now.Add(2 * time.Minute), lease: lease5},
},
finishedTasks: []*base.TaskMessage{t6, t5},
wantLease1: map[string][]base.Z{
"default": {
{Message: t6, Score: now.Add(rdb.LeaseDuration).Unix()},
},
"custom": {
{Message: t4, Score: now.Add(rdb.LeaseDuration).Unix()},
{Message: t5, Score: now.Add(rdb.LeaseDuration).Unix()},
},
},
wantLease2: map[string][]base.Z{
"default": {},
"custom": {
{Message: t4, Score: now.Add(elapsedTime).Add(rdb.LeaseDuration).Unix()},
},
},
startTime: now,
elapsedTime: elapsedTime,
},
} }
timeCmpOpt := cmpopts.EquateApproxTime(10 * time.Millisecond) timeCmpOpt := cmpopts.EquateApproxTime(10 * time.Millisecond)
@@ -37,8 +160,15 @@ func TestHeartbeater(t *testing.T) {
ignoreFieldOpt := cmpopts.IgnoreFields(base.ServerInfo{}, "ServerID") ignoreFieldOpt := cmpopts.IgnoreFields(base.ServerInfo{}, "ServerID")
for _, tc := range tests { for _, tc := range tests {
h.FlushDB(t, r) h.FlushDB(t, r)
h.SeedAllActiveQueues(t, r, tc.active)
h.SeedAllLease(t, r, tc.lease)
clock.SetTime(tc.startTime)
rdbClient.SetClock(clock)
srvState := &serverState{} srvState := &serverState{}
startingCh := make(chan *workerInfo)
finishedCh := make(chan *base.TaskMessage)
hb := newHeartbeater(heartbeaterParams{ hb := newHeartbeater(heartbeaterParams{
logger: testLogger, logger: testLogger,
broker: rdbClient, broker: rdbClient,
@@ -47,14 +177,19 @@ func TestHeartbeater(t *testing.T) {
queues: tc.queues, queues: tc.queues,
strictPriority: false, strictPriority: false,
state: srvState, state: srvState,
starting: make(chan *workerInfo), starting: startingCh,
finished: make(chan *base.TaskMessage), finished: finishedCh,
}) })
hb.clock = clock
// Change host and pid fields for testing purpose. // Change host and pid fields for testing purpose.
hb.host = tc.host hb.host = tc.host
hb.pid = tc.pid hb.pid = tc.pid
//===================
// Start Phase1
//===================
srvState.mu.Lock() srvState.mu.Lock()
srvState.value = srvStateActive // simulating Server.Start srvState.value = srvStateActive // simulating Server.Start
srvState.mu.Unlock() srvState.mu.Unlock()
@@ -62,61 +197,113 @@ func TestHeartbeater(t *testing.T) {
var wg sync.WaitGroup var wg sync.WaitGroup
hb.start(&wg) hb.start(&wg)
want := &base.ServerInfo{ // Simulate processor starting to work on tasks.
Host: tc.host, for _, w := range tc.startedWorkers {
PID: tc.pid, startingCh <- w
Queues: tc.queues,
Concurrency: tc.concurrency,
Started: time.Now(),
Status: "active",
} }
// allow for heartbeater to write to redis // Wait for heartbeater to write to redis
time.Sleep(tc.interval) time.Sleep(tc.interval * 2)
ss, err := rdbClient.ListServers() ss, err := rdbClient.ListServers()
if err != nil { if err != nil {
t.Errorf("could not read server info from redis: %v", err) t.Errorf("%s: could not read server info from redis: %v", tc.desc, err)
hb.shutdown() hb.shutdown()
continue continue
} }
if len(ss) != 1 { if len(ss) != 1 {
t.Errorf("(*RDB).ListServers returned %d process info, want 1", len(ss)) t.Errorf("%s: (*RDB).ListServers returned %d server info, want 1", tc.desc, len(ss))
hb.shutdown() hb.shutdown()
continue continue
} }
if diff := cmp.Diff(want, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" { wantInfo := &base.ServerInfo{
t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", ss[0], want, diff) Host: tc.host,
PID: tc.pid,
Queues: tc.queues,
Concurrency: tc.concurrency,
Started: now,
Status: "active",
ActiveWorkerCount: len(tc.startedWorkers),
}
if diff := cmp.Diff(wantInfo, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" {
t.Errorf("%s: redis stored server status %+v, want %+v; (-want, +got)\n%s", tc.desc, ss[0], wantInfo, diff)
hb.shutdown() hb.shutdown()
continue continue
} }
// server state change; simulating Server.Shutdown for qname, wantLease := range tc.wantLease1 {
gotLease := h.GetLeaseEntries(t, r, qname)
if diff := cmp.Diff(wantLease, gotLease, h.SortZSetEntryOpt); diff != "" {
t.Errorf("%s: mismatch found in %q: (-want,+got):\n%s", tc.desc, base.LeaseKey(qname), diff)
}
}
for _, w := range tc.startedWorkers {
if want := now.Add(rdb.LeaseDuration); w.lease.Deadline() != want {
t.Errorf("%s: lease deadline for %v is set to %v, want %v", tc.desc, w.msg, w.lease.Deadline(), want)
}
}
//===================
// Start Phase2
//===================
clock.AdvanceTime(tc.elapsedTime)
// Simulate processor finished processing tasks.
for _, msg := range tc.finishedTasks {
if err := rdbClient.Done(context.Background(), msg); err != nil {
t.Fatalf("RDB.Done failed: %v", err)
}
finishedCh <- msg
}
// Wait for heartbeater to write to redis
time.Sleep(tc.interval * 2)
for qname, wantLease := range tc.wantLease2 {
gotLease := h.GetLeaseEntries(t, r, qname)
if diff := cmp.Diff(wantLease, gotLease, h.SortZSetEntryOpt); diff != "" {
t.Errorf("%s: mismatch found in %q: (-want,+got):\n%s", tc.desc, base.LeaseKey(qname), diff)
}
}
//===================
// Start Phase3
//===================
// Server state change; simulating Server.Shutdown
srvState.mu.Lock() srvState.mu.Lock()
srvState.value = srvStateClosed srvState.value = srvStateClosed
srvState.mu.Unlock() srvState.mu.Unlock()
// allow for heartbeater to write to redis // Wait for heartbeater to write to redis
time.Sleep(tc.interval * 2) time.Sleep(tc.interval * 2)
want.Status = "closed" wantInfo = &base.ServerInfo{
Host: tc.host,
PID: tc.pid,
Queues: tc.queues,
Concurrency: tc.concurrency,
Started: now,
Status: "closed",
ActiveWorkerCount: len(tc.startedWorkers) - len(tc.finishedTasks),
}
ss, err = rdbClient.ListServers() ss, err = rdbClient.ListServers()
if err != nil { if err != nil {
t.Errorf("could not read process status from redis: %v", err) t.Errorf("%s: could not read server status from redis: %v", tc.desc, err)
hb.shutdown() hb.shutdown()
continue continue
} }
if len(ss) != 1 { if len(ss) != 1 {
t.Errorf("(*RDB).ListProcesses returned %d process info, want 1", len(ss)) t.Errorf("%s: (*RDB).ListServers returned %d server info, want 1", tc.desc, len(ss))
hb.shutdown() hb.shutdown()
continue continue
} }
if diff := cmp.Diff(want, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" { if diff := cmp.Diff(wantInfo, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" {
t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", ss[0], want, diff) t.Errorf("%s: redis stored process status %+v, want %+v; (-want, +got)\n%s", tc.desc, ss[0], wantInfo, diff)
hb.shutdown() hb.shutdown()
continue continue
} }

View File

@@ -308,16 +308,28 @@ func (i *Inspector) ListActiveTasks(qname string, opts ...ListOption) ([]*TaskIn
case err != nil: case err != nil:
return nil, fmt.Errorf("asynq: %v", err) return nil, fmt.Errorf("asynq: %v", err)
} }
expired, err := i.rdb.ListLeaseExpired(time.Now(), qname)
if err != nil {
return nil, fmt.Errorf("asynq: %v", err)
}
expiredSet := make(map[string]struct{}) // set of expired message IDs
for _, msg := range expired {
expiredSet[msg.ID] = struct{}{}
}
var tasks []*TaskInfo var tasks []*TaskInfo
for _, i := range infos { for _, i := range infos {
tasks = append(tasks, newTaskInfo( t := newTaskInfo(
i.Message, i.Message,
i.State, i.State,
i.NextProcessAt, i.NextProcessAt,
i.Result, i.Result,
)) )
if _, ok := expiredSet[i.Message.ID]; ok {
t.IsOrphaned = true
}
tasks = append(tasks, t)
} }
return tasks, err return tasks, nil
} }
// ListScheduledTasks retrieves scheduled tasks from the specified queue. // ListScheduledTasks retrieves scheduled tasks from the specified queue.

View File

@@ -745,6 +745,12 @@ func TestInspectorListPendingTasks(t *testing.T) {
} }
} }
func newOrphanedTaskInfo(msg *base.TaskMessage) *TaskInfo {
info := newTaskInfo(msg, base.TaskStateActive, time.Time{}, nil)
info.IsOrphaned = true
return info
}
func TestInspectorListActiveTasks(t *testing.T) { func TestInspectorListActiveTasks(t *testing.T) {
r := setup(t) r := setup(t)
defer r.Close() defer r.Close()
@@ -754,10 +760,12 @@ func TestInspectorListActiveTasks(t *testing.T) {
m4 := h.NewTaskMessageWithQueue("task4", nil, "custom") m4 := h.NewTaskMessageWithQueue("task4", nil, "custom")
inspector := NewInspector(getRedisConnOpt(t)) inspector := NewInspector(getRedisConnOpt(t))
now := time.Now()
tests := []struct { tests := []struct {
desc string desc string
active map[string][]*base.TaskMessage active map[string][]*base.TaskMessage
lease map[string][]base.Z
qname string qname string
want []*TaskInfo want []*TaskInfo
}{ }{
@@ -767,10 +775,42 @@ func TestInspectorListActiveTasks(t *testing.T) {
"default": {m1, m2}, "default": {m1, m2},
"custom": {m3, m4}, "custom": {m3, m4},
}, },
lease: map[string][]base.Z{
"default": {
{Message: m1, Score: now.Add(20 * time.Second).Unix()},
{Message: m2, Score: now.Add(20 * time.Second).Unix()},
},
"custom": {
{Message: m3, Score: now.Add(20 * time.Second).Unix()},
{Message: m4, Score: now.Add(20 * time.Second).Unix()},
},
},
qname: "custom",
want: []*TaskInfo{
newTaskInfo(m3, base.TaskStateActive, time.Time{}, nil),
newTaskInfo(m4, base.TaskStateActive, time.Time{}, nil),
},
},
{
desc: "with an orphaned task",
active: map[string][]*base.TaskMessage{
"default": {m1, m2},
"custom": {m3, m4},
},
lease: map[string][]base.Z{
"default": {
{Message: m1, Score: now.Add(20 * time.Second).Unix()},
{Message: m2, Score: now.Add(-10 * time.Second).Unix()}, // orphaned task
},
"custom": {
{Message: m3, Score: now.Add(20 * time.Second).Unix()},
{Message: m4, Score: now.Add(20 * time.Second).Unix()},
},
},
qname: "default", qname: "default",
want: []*TaskInfo{ want: []*TaskInfo{
newTaskInfo(m1, base.TaskStateActive, time.Time{}, nil), newTaskInfo(m1, base.TaskStateActive, time.Time{}, nil),
newTaskInfo(m2, base.TaskStateActive, time.Time{}, nil), newOrphanedTaskInfo(m2),
}, },
}, },
} }
@@ -778,6 +818,7 @@ func TestInspectorListActiveTasks(t *testing.T) {
for _, tc := range tests { for _, tc := range tests {
h.FlushDB(t, r) h.FlushDB(t, r)
h.SeedAllActiveQueues(t, r, tc.active) h.SeedAllActiveQueues(t, r, tc.active)
h.SeedAllLease(t, r, tc.lease)
got, err := inspector.ListActiveTasks(tc.qname) got, err := inspector.ListActiveTasks(tc.qname)
if err != nil { if err != nil {

View File

@@ -18,6 +18,7 @@ import (
"github.com/google/go-cmp/cmp/cmpopts" "github.com/google/go-cmp/cmp/cmpopts"
"github.com/google/uuid" "github.com/google/uuid"
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
"github.com/hibiken/asynq/internal/timeutil"
) )
// EquateInt64Approx returns a Comparer option that treats int64 values // EquateInt64Approx returns a Comparer option that treats int64 values
@@ -114,6 +115,13 @@ func NewTaskMessageWithQueue(taskType string, payload []byte, qname string) *bas
} }
} }
// NewLeaseWithClock returns a new lease with the given expiration time and clock.
func NewLeaseWithClock(expirationTime time.Time, clock timeutil.Clock) *base.Lease {
l := base.NewLease(expirationTime)
l.Clock = clock
return l
}
// JSON serializes the given key-value pairs into stream of bytes in JSON. // JSON serializes the given key-value pairs into stream of bytes in JSON.
func JSON(kv map[string]interface{}) []byte { func JSON(kv map[string]interface{}) []byte {
b, err := json.Marshal(kv) b, err := json.Marshal(kv)
@@ -223,11 +231,11 @@ func SeedArchivedQueue(tb testing.TB, r redis.UniversalClient, entries []base.Z,
seedRedisZSet(tb, r, base.ArchivedKey(qname), entries, base.TaskStateArchived) seedRedisZSet(tb, r, base.ArchivedKey(qname), entries, base.TaskStateArchived)
} }
// SeedDeadlines initializes the deadlines set with the given entries. // SeedLease initializes the lease set with the given entries.
func SeedDeadlines(tb testing.TB, r redis.UniversalClient, entries []base.Z, qname string) { func SeedLease(tb testing.TB, r redis.UniversalClient, entries []base.Z, qname string) {
tb.Helper() tb.Helper()
r.SAdd(context.Background(), base.AllQueues, qname) r.SAdd(context.Background(), base.AllQueues, qname)
seedRedisZSet(tb, r, base.DeadlinesKey(qname), entries, base.TaskStateActive) seedRedisZSet(tb, r, base.LeaseKey(qname), entries, base.TaskStateActive)
} }
// SeedCompletedQueue initializes the completed set witht the given entries. // SeedCompletedQueue initializes the completed set witht the given entries.
@@ -279,11 +287,11 @@ func SeedAllArchivedQueues(tb testing.TB, r redis.UniversalClient, archived map[
} }
} }
// SeedAllDeadlines initializes all of the deadlines with the given entries. // SeedAllLease initializes all of the lease sets with the given entries.
func SeedAllDeadlines(tb testing.TB, r redis.UniversalClient, deadlines map[string][]base.Z) { func SeedAllLease(tb testing.TB, r redis.UniversalClient, lease map[string][]base.Z) {
tb.Helper() tb.Helper()
for q, entries := range deadlines { for q, entries := range lease {
SeedDeadlines(tb, r, entries, q) SeedLease(tb, r, entries, q)
} }
} }
@@ -307,8 +315,6 @@ func seedRedisList(tb testing.TB, c redis.UniversalClient, key string,
data := map[string]interface{}{ data := map[string]interface{}{
"msg": encoded, "msg": encoded,
"state": state.String(), "state": state.String(),
"timeout": msg.Timeout,
"deadline": msg.Deadline,
"unique_key": msg.UniqueKey, "unique_key": msg.UniqueKey,
} }
if err := c.HSet(context.Background(), key, data).Err(); err != nil { if err := c.HSet(context.Background(), key, data).Err(); err != nil {
@@ -337,8 +343,6 @@ func seedRedisZSet(tb testing.TB, c redis.UniversalClient, key string,
data := map[string]interface{}{ data := map[string]interface{}{
"msg": encoded, "msg": encoded,
"state": state.String(), "state": state.String(),
"timeout": msg.Timeout,
"deadline": msg.Deadline,
"unique_key": msg.UniqueKey, "unique_key": msg.UniqueKey,
} }
if err := c.HSet(context.Background(), key, data).Err(); err != nil { if err := c.HSet(context.Background(), key, data).Err(); err != nil {
@@ -416,11 +420,11 @@ func GetArchivedEntries(tb testing.TB, r redis.UniversalClient, qname string) []
return getMessagesFromZSetWithScores(tb, r, qname, base.ArchivedKey, base.TaskStateArchived) return getMessagesFromZSetWithScores(tb, r, qname, base.ArchivedKey, base.TaskStateArchived)
} }
// GetDeadlinesEntries returns all task messages and its score in the deadlines set for the given queue. // GetLeaseEntries returns all task IDs and its score in the lease set for the given queue.
// It also asserts the state field of the task. // It also asserts the state field of the task.
func GetDeadlinesEntries(tb testing.TB, r redis.UniversalClient, qname string) []base.Z { func GetLeaseEntries(tb testing.TB, r redis.UniversalClient, qname string) []base.Z {
tb.Helper() tb.Helper()
return getMessagesFromZSetWithScores(tb, r, qname, base.DeadlinesKey, base.TaskStateActive) return getMessagesFromZSetWithScores(tb, r, qname, base.LeaseKey, base.TaskStateActive)
} }
// GetCompletedEntries returns all completed messages and its score in the given queue. // GetCompletedEntries returns all completed messages and its score in the given queue.

View File

@@ -18,11 +18,12 @@ import (
"github.com/golang/protobuf/ptypes" "github.com/golang/protobuf/ptypes"
"github.com/hibiken/asynq/internal/errors" "github.com/hibiken/asynq/internal/errors"
pb "github.com/hibiken/asynq/internal/proto" pb "github.com/hibiken/asynq/internal/proto"
"github.com/hibiken/asynq/internal/timeutil"
"google.golang.org/protobuf/proto" "google.golang.org/protobuf/proto"
) )
// Version of asynq library and CLI. // Version of asynq library and CLI.
const Version = "0.21.0" const Version = "0.22.0"
// DefaultQueueName is the queue name used if none are specified by user. // DefaultQueueName is the queue name used if none are specified by user.
const DefaultQueueName = "default" const DefaultQueueName = "default"
@@ -136,9 +137,9 @@ func ArchivedKey(qname string) string {
return fmt.Sprintf("%sarchived", QueueKeyPrefix(qname)) return fmt.Sprintf("%sarchived", QueueKeyPrefix(qname))
} }
// DeadlinesKey returns a redis key for the deadlines. // LeaseKey returns a redis key for the lease.
func DeadlinesKey(qname string) string { func LeaseKey(qname string) string {
return fmt.Sprintf("%sdeadlines", QueueKeyPrefix(qname)) return fmt.Sprintf("%slease", QueueKeyPrefix(qname))
} }
func CompletedKey(qname string) string { func CompletedKey(qname string) string {
@@ -603,6 +604,73 @@ func (c *Cancelations) Get(id string) (fn context.CancelFunc, ok bool) {
return fn, ok return fn, ok
} }
// Lease is a time bound lease for worker to process task.
// It provides a communication channel between lessor and lessee about lease expiration.
type Lease struct {
once sync.Once
ch chan struct{}
Clock timeutil.Clock
mu sync.Mutex
expireAt time.Time // guarded by mu
}
func NewLease(expirationTime time.Time) *Lease {
return &Lease{
ch: make(chan struct{}),
expireAt: expirationTime,
Clock: timeutil.NewRealClock(),
}
}
// Reset chanegs the lease to expire at the given time.
// It returns true if the lease is still valid and reset operation was successful, false if the lease had been expired.
func (l *Lease) Reset(expirationTime time.Time) bool {
if !l.IsValid() {
return false
}
l.mu.Lock()
defer l.mu.Unlock()
l.expireAt = expirationTime
return true
}
// Sends a notification to lessee about expired lease
// Returns true if notification was sent, returns false if the lease is still valid and notification was not sent.
func (l *Lease) NotifyExpiration() bool {
if l.IsValid() {
return false
}
l.once.Do(l.closeCh)
return true
}
func (l *Lease) closeCh() {
close(l.ch)
}
// Done returns a communication channel from which the lessee can read to get notified when lessor notifies about lease expiration.
func (l *Lease) Done() <-chan struct{} {
return l.ch
}
// Deadline returns the expiration time of the lease.
func (l *Lease) Deadline() time.Time {
l.mu.Lock()
defer l.mu.Unlock()
return l.expireAt
}
// IsValid returns true if the lease's expieration time is in the future or equals to the current time,
// returns false otherwise.
func (l *Lease) IsValid() bool {
now := l.Clock.Now()
l.mu.Lock()
defer l.mu.Unlock()
return l.expireAt.After(now) || l.expireAt.Equal(now)
}
// Broker is a message broker that supports operations to manage task queues. // Broker is a message broker that supports operations to manage task queues.
// //
// See rdb.RDB as a reference implementation. // See rdb.RDB as a reference implementation.
@@ -611,16 +679,17 @@ type Broker interface {
Enqueue(ctx context.Context, msg *TaskMessage) error Enqueue(ctx context.Context, msg *TaskMessage) error
EnqueueUnique(ctx context.Context, msg *TaskMessage, ttl time.Duration) error EnqueueUnique(ctx context.Context, msg *TaskMessage, ttl time.Duration) error
Dequeue(qnames ...string) (*TaskMessage, time.Time, error) Dequeue(qnames ...string) (*TaskMessage, time.Time, error)
Done(msg *TaskMessage) error Done(ctx context.Context, msg *TaskMessage) error
MarkAsComplete(msg *TaskMessage) error MarkAsComplete(ctx context.Context, msg *TaskMessage) error
Requeue(msg *TaskMessage) error Requeue(ctx context.Context, msg *TaskMessage) error
Schedule(ctx context.Context, msg *TaskMessage, processAt time.Time) error Schedule(ctx context.Context, msg *TaskMessage, processAt time.Time) error
ScheduleUnique(ctx context.Context, msg *TaskMessage, processAt time.Time, ttl time.Duration) error ScheduleUnique(ctx context.Context, msg *TaskMessage, processAt time.Time, ttl time.Duration) error
Retry(msg *TaskMessage, processAt time.Time, errMsg string, isFailure bool) error Retry(ctx context.Context, msg *TaskMessage, processAt time.Time, errMsg string, isFailure bool) error
Archive(msg *TaskMessage, errMsg string) error Archive(ctx context.Context, msg *TaskMessage, errMsg string) error
ForwardIfReady(qnames ...string) error ForwardIfReady(qnames ...string) error
DeleteExpiredCompletedTasks(qname string) error DeleteExpiredCompletedTasks(qname string) error
ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*TaskMessage, error) ListLeaseExpired(cutoff time.Time, qnames ...string) ([]*TaskMessage, error)
ExtendLease(qname string, ids ...string) (time.Time, error)
WriteServerState(info *ServerInfo, workers []*WorkerInfo, ttl time.Duration) error WriteServerState(info *ServerInfo, workers []*WorkerInfo, ttl time.Duration) error
ClearServerState(host string, pid int, serverID string) error ClearServerState(host string, pid int, serverID string) error
CancelationPubSub() (*redis.PubSub, error) // TODO: Need to decouple from redis to support other brokers CancelationPubSub() (*redis.PubSub, error) // TODO: Need to decouple from redis to support other brokers

View File

@@ -16,6 +16,7 @@ import (
"github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp"
"github.com/google/uuid" "github.com/google/uuid"
"github.com/hibiken/asynq/internal/timeutil"
) )
func TestTaskKey(t *testing.T) { func TestTaskKey(t *testing.T) {
@@ -71,19 +72,19 @@ func TestActiveKey(t *testing.T) {
} }
} }
func TestDeadlinesKey(t *testing.T) { func TestLeaseKey(t *testing.T) {
tests := []struct { tests := []struct {
qname string qname string
want string want string
}{ }{
{"default", "asynq:{default}:deadlines"}, {"default", "asynq:{default}:lease"},
{"custom", "asynq:{custom}:deadlines"}, {"custom", "asynq:{custom}:lease"},
} }
for _, tc := range tests { for _, tc := range tests {
got := DeadlinesKey(tc.qname) got := LeaseKey(tc.qname)
if got != tc.want { if got != tc.want {
t.Errorf("DeadlinesKey(%q) = %q, want %q", tc.qname, got, tc.want) t.Errorf("LeaseKey(%q) = %q, want %q", tc.qname, got, tc.want)
} }
} }
} }
@@ -627,3 +628,75 @@ func TestCancelationsConcurrentAccess(t *testing.T) {
t.Errorf("(*Cancelations).Get(%q) = _, true, want <nil>, false", key2) t.Errorf("(*Cancelations).Get(%q) = _, true, want <nil>, false", key2)
} }
} }
func TestLeaseReset(t *testing.T) {
now := time.Now()
clock := timeutil.NewSimulatedClock(now)
l := NewLease(now.Add(30 * time.Second))
l.Clock = clock
// Check initial state
if !l.IsValid() {
t.Errorf("lease should be valid when expiration is set to a future time")
}
if want := now.Add(30 * time.Second); l.Deadline() != want {
t.Errorf("Lease.Deadline() = %v, want %v", l.Deadline(), want)
}
// Test Reset
if !l.Reset(now.Add(45 * time.Second)) {
t.Fatalf("Lease.Reset returned false when extending")
}
if want := now.Add(45 * time.Second); l.Deadline() != want {
t.Errorf("After Reset: Lease.Deadline() = %v, want %v", l.Deadline(), want)
}
clock.AdvanceTime(1 * time.Minute) // simulate lease expiration
if l.IsValid() {
t.Errorf("lease should be invalid after expiration")
}
// Reset should return false if lease is expired.
if l.Reset(time.Now().Add(20 * time.Second)) {
t.Errorf("Lease.Reset should return false after expiration")
}
}
func TestLeaseNotifyExpiration(t *testing.T) {
now := time.Now()
clock := timeutil.NewSimulatedClock(now)
l := NewLease(now.Add(30 * time.Second))
l.Clock = clock
select {
case <-l.Done():
t.Fatalf("Lease.Done() did not block")
default:
}
if l.NotifyExpiration() {
t.Fatalf("Lease.NotifyExpiration() should return false when lease is still valid")
}
clock.AdvanceTime(1 * time.Minute) // simulate lease expiration
if l.IsValid() {
t.Errorf("Lease should be invalid after expiration")
}
if !l.NotifyExpiration() {
t.Errorf("Lease.NotifyExpiration() return return true after expiration")
}
if !l.NotifyExpiration() {
t.Errorf("It should be leagal to call Lease.NotifyExpiration multiple times")
}
select {
case <-l.Done():
// expected
default:
t.Errorf("Lease.Done() blocked after call to Lease.NotifyExpiration()")
}
}

View File

@@ -28,14 +28,14 @@ type ctxKey int
const metadataCtxKey ctxKey = 0 const metadataCtxKey ctxKey = 0
// New returns a context and cancel function for a given task message. // New returns a context and cancel function for a given task message.
func New(msg *base.TaskMessage, deadline time.Time) (context.Context, context.CancelFunc) { func New(base context.Context, msg *base.TaskMessage, deadline time.Time) (context.Context, context.CancelFunc) {
metadata := taskMetadata{ metadata := taskMetadata{
id: msg.ID, id: msg.ID,
maxRetry: msg.Retry, maxRetry: msg.Retry,
retryCount: msg.Retried, retryCount: msg.Retried,
qname: msg.Queue, qname: msg.Queue,
} }
ctx := context.WithValue(context.Background(), metadataCtxKey, metadata) ctx := context.WithValue(base, metadataCtxKey, metadata)
return context.WithDeadline(ctx, deadline) return context.WithDeadline(ctx, deadline)
} }

View File

@@ -6,6 +6,7 @@ package context
import ( import (
"context" "context"
"fmt"
"testing" "testing"
"time" "time"
@@ -28,7 +29,7 @@ func TestCreateContextWithFutureDeadline(t *testing.T) {
Payload: nil, Payload: nil,
} }
ctx, cancel := New(msg, tc.deadline) ctx, cancel := New(context.Background(), msg, tc.deadline)
select { select {
case x := <-ctx.Done(): case x := <-ctx.Done():
t.Errorf("<-ctx.Done() == %v, want nothing (it should block)", x) t.Errorf("<-ctx.Done() == %v, want nothing (it should block)", x)
@@ -53,6 +54,53 @@ func TestCreateContextWithFutureDeadline(t *testing.T) {
} }
} }
func TestCreateContextWithBaseContext(t *testing.T) {
type ctxKey string
type ctxValue string
var key ctxKey = "key"
var value ctxValue = "value"
tests := []struct {
baseCtx context.Context
validate func(ctx context.Context, t *testing.T) error
}{
{
baseCtx: context.WithValue(context.Background(), key, value),
validate: func(ctx context.Context, t *testing.T) error {
got, ok := ctx.Value(key).(ctxValue)
if !ok {
return fmt.Errorf("ctx.Value().(ctxValue) returned false, expected to be true")
}
if want := value; got != want {
return fmt.Errorf("ctx.Value().(ctxValue) returned unknown value (%v), expected to be %s", got, value)
}
return nil
},
},
}
for _, tc := range tests {
msg := &base.TaskMessage{
Type: "something",
ID: uuid.NewString(),
Payload: nil,
}
ctx, cancel := New(tc.baseCtx, msg, time.Now().Add(30*time.Minute))
defer cancel()
select {
case x := <-ctx.Done():
t.Errorf("<-ctx.Done() == %v, want nothing (it should block)", x)
default:
}
if err := tc.validate(ctx, t); err != nil {
t.Errorf("%v", err)
}
}
}
func TestCreateContextWithPastDeadline(t *testing.T) { func TestCreateContextWithPastDeadline(t *testing.T) {
tests := []struct { tests := []struct {
deadline time.Time deadline time.Time
@@ -67,7 +115,7 @@ func TestCreateContextWithPastDeadline(t *testing.T) {
Payload: nil, Payload: nil,
} }
ctx, cancel := New(msg, tc.deadline) ctx, cancel := New(context.Background(), msg, tc.deadline)
defer cancel() defer cancel()
select { select {
@@ -97,7 +145,7 @@ func TestGetTaskMetadataFromContext(t *testing.T) {
} }
for _, tc := range tests { for _, tc := range tests {
ctx, cancel := New(tc.msg, time.Now().Add(30*time.Minute)) ctx, cancel := New(context.Background(), tc.msg, time.Now().Add(30*time.Minute))
defer cancel() defer cancel()
id, ok := GetTaskID(ctx) id, ok := GetTaskID(ctx)

View File

@@ -156,16 +156,17 @@ func BenchmarkDone(b *testing.B) {
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()}, {Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()}, {Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
} }
ctx := context.Background()
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
b.StopTimer() b.StopTimer()
asynqtest.FlushDB(b, r.client) asynqtest.FlushDB(b, r.client)
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName) asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName) asynqtest.SeedLease(b, r.client, zs, base.DefaultQueueName)
b.StartTimer() b.StartTimer()
if err := r.Done(msgs[0]); err != nil { if err := r.Done(ctx, msgs[0]); err != nil {
b.Fatalf("Done failed: %v", err) b.Fatalf("Done failed: %v", err)
} }
} }
@@ -182,16 +183,17 @@ func BenchmarkRetry(b *testing.B) {
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()}, {Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()}, {Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
} }
ctx := context.Background()
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
b.StopTimer() b.StopTimer()
asynqtest.FlushDB(b, r.client) asynqtest.FlushDB(b, r.client)
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName) asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName) asynqtest.SeedLease(b, r.client, zs, base.DefaultQueueName)
b.StartTimer() b.StartTimer()
if err := r.Retry(msgs[0], time.Now().Add(1*time.Minute), "error", true /*isFailure*/); err != nil { if err := r.Retry(ctx, msgs[0], time.Now().Add(1*time.Minute), "error", true /*isFailure*/); err != nil {
b.Fatalf("Retry failed: %v", err) b.Fatalf("Retry failed: %v", err)
} }
} }
@@ -208,16 +210,17 @@ func BenchmarkArchive(b *testing.B) {
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()}, {Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()}, {Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
} }
ctx := context.Background()
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
b.StopTimer() b.StopTimer()
asynqtest.FlushDB(b, r.client) asynqtest.FlushDB(b, r.client)
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName) asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName) asynqtest.SeedLease(b, r.client, zs, base.DefaultQueueName)
b.StartTimer() b.StartTimer()
if err := r.Archive(msgs[0], "error"); err != nil { if err := r.Archive(ctx, msgs[0], "error"); err != nil {
b.Fatalf("Archive failed: %v", err) b.Fatalf("Archive failed: %v", err)
} }
} }
@@ -234,16 +237,17 @@ func BenchmarkRequeue(b *testing.B) {
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()}, {Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()}, {Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
} }
ctx := context.Background()
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
b.StopTimer() b.StopTimer()
asynqtest.FlushDB(b, r.client) asynqtest.FlushDB(b, r.client)
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName) asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName) asynqtest.SeedLease(b, r.client, zs, base.DefaultQueueName)
b.StartTimer() b.StartTimer()
if err := r.Requeue(msgs[0]); err != nil { if err := r.Requeue(ctx, msgs[0]); err != nil {
b.Fatalf("Requeue failed: %v", err) b.Fatalf("Requeue failed: %v", err)
} }
} }

View File

@@ -1387,7 +1387,7 @@ func (r *RDB) DeleteAllPendingTasks(qname string) (int64, error) {
// KEYS[3] -> asynq:{<qname>}:scheduled // KEYS[3] -> asynq:{<qname>}:scheduled
// KEYS[4] -> asynq:{<qname>}:retry // KEYS[4] -> asynq:{<qname>}:retry
// KEYS[5] -> asynq:{<qname>}:archived // KEYS[5] -> asynq:{<qname>}:archived
// KEYS[6] -> asynq:{<qname>}:deadlines // KEYS[6] -> asynq:{<qname>}:lease
// -- // --
// ARGV[1] -> task key prefix // ARGV[1] -> task key prefix
// //
@@ -1447,7 +1447,7 @@ return 1`)
// KEYS[3] -> asynq:{<qname>}:scheduled // KEYS[3] -> asynq:{<qname>}:scheduled
// KEYS[4] -> asynq:{<qname>}:retry // KEYS[4] -> asynq:{<qname>}:retry
// KEYS[5] -> asynq:{<qname>}:archived // KEYS[5] -> asynq:{<qname>}:archived
// KEYS[6] -> asynq:{<qname>}:deadlines // KEYS[6] -> asynq:{<qname>}:lease
// -- // --
// ARGV[1] -> task key prefix // ARGV[1] -> task key prefix
// //
@@ -1516,7 +1516,7 @@ func (r *RDB) RemoveQueue(qname string, force bool) error {
base.ScheduledKey(qname), base.ScheduledKey(qname),
base.RetryKey(qname), base.RetryKey(qname),
base.ArchivedKey(qname), base.ArchivedKey(qname),
base.DeadlinesKey(qname), base.LeaseKey(qname),
} }
res, err := script.Run(context.Background(), r.client, keys, base.TaskKeyPrefix(qname)).Result() res, err := script.Run(context.Background(), r.client, keys, base.TaskKeyPrefix(qname)).Result()
if err != nil { if err != nil {

View File

@@ -4036,7 +4036,7 @@ func TestRemoveQueue(t *testing.T) {
keys := []string{ keys := []string{
base.PendingKey(tc.qname), base.PendingKey(tc.qname),
base.ActiveKey(tc.qname), base.ActiveKey(tc.qname),
base.DeadlinesKey(tc.qname), base.LeaseKey(tc.qname),
base.ScheduledKey(tc.qname), base.ScheduledKey(tc.qname),
base.RetryKey(tc.qname), base.RetryKey(tc.qname),
base.ArchivedKey(tc.qname), base.ArchivedKey(tc.qname),

View File

@@ -20,6 +20,9 @@ import (
const statsTTL = 90 * 24 * time.Hour // 90 days const statsTTL = 90 * 24 * time.Hour // 90 days
// LeaseDuration is the duration used to initially create a lease and to extend it thereafter.
const LeaseDuration = 30 * time.Second
// RDB is a client interface to query and mutate task queues. // RDB is a client interface to query and mutate task queues.
type RDB struct { type RDB struct {
client redis.UniversalClient client redis.UniversalClient
@@ -84,9 +87,7 @@ func (r *RDB) runScriptWithErrorCode(ctx context.Context, op errors.Op, script *
// -- // --
// ARGV[1] -> task message data // ARGV[1] -> task message data
// ARGV[2] -> task ID // ARGV[2] -> task ID
// ARGV[3] -> task timeout in seconds (0 if not timeout) // ARGV[3] -> current unix time in nsec
// ARGV[4] -> task deadline in unix time (0 if no deadline)
// ARGV[5] -> current unix time in nsec
// //
// Output: // Output:
// Returns 1 if successfully enqueued // Returns 1 if successfully enqueued
@@ -98,9 +99,7 @@ end
redis.call("HSET", KEYS[1], redis.call("HSET", KEYS[1],
"msg", ARGV[1], "msg", ARGV[1],
"state", "pending", "state", "pending",
"timeout", ARGV[3], "pending_since", ARGV[3])
"deadline", ARGV[4],
"pending_since", ARGV[5])
redis.call("LPUSH", KEYS[2], ARGV[2]) redis.call("LPUSH", KEYS[2], ARGV[2])
return 1 return 1
`) `)
@@ -122,8 +121,6 @@ func (r *RDB) Enqueue(ctx context.Context, msg *base.TaskMessage) error {
argv := []interface{}{ argv := []interface{}{
encoded, encoded,
msg.ID, msg.ID,
msg.Timeout,
msg.Deadline,
r.clock.Now().UnixNano(), r.clock.Now().UnixNano(),
} }
n, err := r.runScriptWithErrorCode(ctx, op, enqueueCmd, keys, argv...) n, err := r.runScriptWithErrorCode(ctx, op, enqueueCmd, keys, argv...)
@@ -145,9 +142,7 @@ func (r *RDB) Enqueue(ctx context.Context, msg *base.TaskMessage) error {
// ARGV[1] -> task ID // ARGV[1] -> task ID
// ARGV[2] -> uniqueness lock TTL // ARGV[2] -> uniqueness lock TTL
// ARGV[3] -> task message data // ARGV[3] -> task message data
// ARGV[4] -> task timeout in seconds (0 if not timeout) // ARGV[4] -> current unix time in nsec
// ARGV[5] -> task deadline in unix time (0 if no deadline)
// ARGV[6] -> current unix time in nsec
// //
// Output: // Output:
// Returns 1 if successfully enqueued // Returns 1 if successfully enqueued
@@ -164,9 +159,7 @@ end
redis.call("HSET", KEYS[2], redis.call("HSET", KEYS[2],
"msg", ARGV[3], "msg", ARGV[3],
"state", "pending", "state", "pending",
"timeout", ARGV[4], "pending_since", ARGV[4],
"deadline", ARGV[5],
"pending_since", ARGV[6],
"unique_key", KEYS[1]) "unique_key", KEYS[1])
redis.call("LPUSH", KEYS[3], ARGV[1]) redis.call("LPUSH", KEYS[3], ARGV[1])
return 1 return 1
@@ -192,8 +185,6 @@ func (r *RDB) EnqueueUnique(ctx context.Context, msg *base.TaskMessage, ttl time
msg.ID, msg.ID,
int(ttl.Seconds()), int(ttl.Seconds()),
encoded, encoded,
msg.Timeout,
msg.Deadline,
r.clock.Now().UnixNano(), r.clock.Now().UnixNano(),
} }
n, err := r.runScriptWithErrorCode(ctx, op, enqueueUniqueCmd, keys, argv...) n, err := r.runScriptWithErrorCode(ctx, op, enqueueUniqueCmd, keys, argv...)
@@ -213,20 +204,17 @@ func (r *RDB) EnqueueUnique(ctx context.Context, msg *base.TaskMessage, ttl time
// KEYS[1] -> asynq:{<qname>}:pending // KEYS[1] -> asynq:{<qname>}:pending
// KEYS[2] -> asynq:{<qname>}:paused // KEYS[2] -> asynq:{<qname>}:paused
// KEYS[3] -> asynq:{<qname>}:active // KEYS[3] -> asynq:{<qname>}:active
// KEYS[4] -> asynq:{<qname>}:deadlines // KEYS[4] -> asynq:{<qname>}:lease
// -- // --
// ARGV[1] -> current time in Unix time // ARGV[1] -> initial lease expiration Unix time
// ARGV[2] -> task key prefix // ARGV[2] -> task key prefix
// //
// Output: // Output:
// Returns nil if no processable task is found in the given queue. // Returns nil if no processable task is found in the given queue.
// Returns tuple {msg , deadline} if task is found, where `msg` is the encoded // Returns an encoded TaskMessage.
// TaskMessage, and `deadline` is Unix time in seconds.
// //
// Note: dequeueCmd checks whether a queue is paused first, before // Note: dequeueCmd checks whether a queue is paused first, before
// calling RPOPLPUSH to pop a task from the queue. // calling RPOPLPUSH to pop a task from the queue.
// It computes the task deadline by inspecting Timout and Deadline fields,
// and inserts the task to the deadlines zset with the computed deadline.
var dequeueCmd = redis.NewScript(` var dequeueCmd = redis.NewScript(`
if redis.call("EXISTS", KEYS[2]) == 0 then if redis.call("EXISTS", KEYS[2]) == 0 then
local id = redis.call("RPOPLPUSH", KEYS[1], KEYS[3]) local id = redis.call("RPOPLPUSH", KEYS[1], KEYS[3])
@@ -234,41 +222,28 @@ if redis.call("EXISTS", KEYS[2]) == 0 then
local key = ARGV[2] .. id local key = ARGV[2] .. id
redis.call("HSET", key, "state", "active") redis.call("HSET", key, "state", "active")
redis.call("HDEL", key, "pending_since") redis.call("HDEL", key, "pending_since")
local data = redis.call("HMGET", key, "msg", "timeout", "deadline") redis.call("ZADD", KEYS[4], ARGV[1], id)
local msg = data[1] return redis.call("HGET", key, "msg")
local timeout = tonumber(data[2])
local deadline = tonumber(data[3])
local score
if timeout ~= 0 and deadline ~= 0 then
score = math.min(ARGV[1]+timeout, deadline)
elseif timeout ~= 0 then
score = ARGV[1] + timeout
elseif deadline ~= 0 then
score = deadline
else
return redis.error_reply("asynq internal error: both timeout and deadline are not set")
end
redis.call("ZADD", KEYS[4], score, id)
return {msg, score}
end end
end end
return nil`) return nil`)
// Dequeue queries given queues in order and pops a task message // Dequeue queries given queues in order and pops a task message
// off a queue if one exists and returns the message and deadline. // off a queue if one exists and returns the message and its lease expiration time.
// Dequeue skips a queue if the queue is paused. // Dequeue skips a queue if the queue is paused.
// If all queues are empty, ErrNoProcessableTask error is returned. // If all queues are empty, ErrNoProcessableTask error is returned.
func (r *RDB) Dequeue(qnames ...string) (msg *base.TaskMessage, deadline time.Time, err error) { func (r *RDB) Dequeue(qnames ...string) (msg *base.TaskMessage, leaseExpirationTime time.Time, err error) {
var op errors.Op = "rdb.Dequeue" var op errors.Op = "rdb.Dequeue"
for _, qname := range qnames { for _, qname := range qnames {
keys := []string{ keys := []string{
base.PendingKey(qname), base.PendingKey(qname),
base.PausedKey(qname), base.PausedKey(qname),
base.ActiveKey(qname), base.ActiveKey(qname),
base.DeadlinesKey(qname), base.LeaseKey(qname),
} }
leaseExpirationTime = r.clock.Now().Add(LeaseDuration)
argv := []interface{}{ argv := []interface{}{
r.clock.Now().Unix(), leaseExpirationTime.Unix(),
base.TaskKeyPrefix(qname), base.TaskKeyPrefix(qname),
} }
res, err := dequeueCmd.Run(context.Background(), r.client, keys, argv...).Result() res, err := dequeueCmd.Run(context.Background(), r.client, keys, argv...).Result()
@@ -277,34 +252,24 @@ func (r *RDB) Dequeue(qnames ...string) (msg *base.TaskMessage, deadline time.Ti
} else if err != nil { } else if err != nil {
return nil, time.Time{}, errors.E(op, errors.Unknown, fmt.Sprintf("redis eval error: %v", err)) return nil, time.Time{}, errors.E(op, errors.Unknown, fmt.Sprintf("redis eval error: %v", err))
} }
data, err := cast.ToSliceE(res) encoded, err := cast.ToStringE(res)
if err != nil {
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: unexpected return value from Lua script: %v", res))
}
if len(data) != 2 {
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("Lua script returned %d values; expected 2", len(data)))
}
encoded, err := cast.ToStringE(data[0])
if err != nil {
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: unexpected return value from Lua script: %v", res))
}
d, err := cast.ToInt64E(data[1])
if err != nil { if err != nil {
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: unexpected return value from Lua script: %v", res)) return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: unexpected return value from Lua script: %v", res))
} }
if msg, err = base.DecodeMessage([]byte(encoded)); err != nil { if msg, err = base.DecodeMessage([]byte(encoded)); err != nil {
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cannot decode message: %v", err)) return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cannot decode message: %v", err))
} }
return msg, time.Unix(d, 0), nil return msg, leaseExpirationTime, nil
} }
return nil, time.Time{}, errors.E(op, errors.NotFound, errors.ErrNoProcessableTask) return nil, time.Time{}, errors.E(op, errors.NotFound, errors.ErrNoProcessableTask)
} }
// KEYS[1] -> asynq:{<qname>}:active // KEYS[1] -> asynq:{<qname>}:active
// KEYS[2] -> asynq:{<qname>}:deadlines // KEYS[2] -> asynq:{<qname>}:lease
// KEYS[3] -> asynq:{<qname>}:t:<task_id> // KEYS[3] -> asynq:{<qname>}:t:<task_id>
// KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd> // KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
// KEYS[5] -> asynq:{<qname>}:processed // KEYS[5] -> asynq:{<qname>}:processed
// -------
// ARGV[1] -> task ID // ARGV[1] -> task ID
// ARGV[2] -> stats expiration timestamp // ARGV[2] -> stats expiration timestamp
// ARGV[3] -> max int64 value // ARGV[3] -> max int64 value
@@ -332,11 +297,12 @@ return redis.status_reply("OK")
`) `)
// KEYS[1] -> asynq:{<qname>}:active // KEYS[1] -> asynq:{<qname>}:active
// KEYS[2] -> asynq:{<qname>}:deadlines // KEYS[2] -> asynq:{<qname>}:lease
// KEYS[3] -> asynq:{<qname>}:t:<task_id> // KEYS[3] -> asynq:{<qname>}:t:<task_id>
// KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd> // KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
// KEYS[5] -> asynq:{<qname>}:processed // KEYS[5] -> asynq:{<qname>}:processed
// KEYS[6] -> unique key // KEYS[6] -> unique key
// -------
// ARGV[1] -> task ID // ARGV[1] -> task ID
// ARGV[2] -> stats expiration timestamp // ARGV[2] -> stats expiration timestamp
// ARGV[3] -> max int64 value // ARGV[3] -> max int64 value
@@ -368,14 +334,13 @@ return redis.status_reply("OK")
// Done removes the task from active queue and deletes the task. // Done removes the task from active queue and deletes the task.
// It removes a uniqueness lock acquired by the task, if any. // It removes a uniqueness lock acquired by the task, if any.
func (r *RDB) Done(msg *base.TaskMessage) error { func (r *RDB) Done(ctx context.Context, msg *base.TaskMessage) error {
var op errors.Op = "rdb.Done" var op errors.Op = "rdb.Done"
ctx := context.Background()
now := r.clock.Now() now := r.clock.Now()
expireAt := now.Add(statsTTL) expireAt := now.Add(statsTTL)
keys := []string{ keys := []string{
base.ActiveKey(msg.Queue), base.ActiveKey(msg.Queue),
base.DeadlinesKey(msg.Queue), base.LeaseKey(msg.Queue),
base.TaskKey(msg.Queue, msg.ID), base.TaskKey(msg.Queue, msg.ID),
base.ProcessedKey(msg.Queue, now), base.ProcessedKey(msg.Queue, now),
base.ProcessedTotalKey(msg.Queue), base.ProcessedTotalKey(msg.Queue),
@@ -394,7 +359,7 @@ func (r *RDB) Done(msg *base.TaskMessage) error {
} }
// KEYS[1] -> asynq:{<qname>}:active // KEYS[1] -> asynq:{<qname>}:active
// KEYS[2] -> asynq:{<qname>}:deadlines // KEYS[2] -> asynq:{<qname>}:lease
// KEYS[3] -> asynq:{<qname>}:completed // KEYS[3] -> asynq:{<qname>}:completed
// KEYS[4] -> asynq:{<qname>}:t:<task_id> // KEYS[4] -> asynq:{<qname>}:t:<task_id>
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd> // KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
@@ -430,7 +395,7 @@ return redis.status_reply("OK")
`) `)
// KEYS[1] -> asynq:{<qname>}:active // KEYS[1] -> asynq:{<qname>}:active
// KEYS[2] -> asynq:{<qname>}:deadlines // KEYS[2] -> asynq:{<qname>}:lease
// KEYS[3] -> asynq:{<qname>}:completed // KEYS[3] -> asynq:{<qname>}:completed
// KEYS[4] -> asynq:{<qname>}:t:<task_id> // KEYS[4] -> asynq:{<qname>}:t:<task_id>
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd> // KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
@@ -471,9 +436,8 @@ return redis.status_reply("OK")
// MarkAsComplete removes the task from active queue to mark the task as completed. // MarkAsComplete removes the task from active queue to mark the task as completed.
// It removes a uniqueness lock acquired by the task, if any. // It removes a uniqueness lock acquired by the task, if any.
func (r *RDB) MarkAsComplete(msg *base.TaskMessage) error { func (r *RDB) MarkAsComplete(ctx context.Context, msg *base.TaskMessage) error {
var op errors.Op = "rdb.MarkAsComplete" var op errors.Op = "rdb.MarkAsComplete"
ctx := context.Background()
now := r.clock.Now() now := r.clock.Now()
statsExpireAt := now.Add(statsTTL) statsExpireAt := now.Add(statsTTL)
msg.CompletedAt = now.Unix() msg.CompletedAt = now.Unix()
@@ -483,7 +447,7 @@ func (r *RDB) MarkAsComplete(msg *base.TaskMessage) error {
} }
keys := []string{ keys := []string{
base.ActiveKey(msg.Queue), base.ActiveKey(msg.Queue),
base.DeadlinesKey(msg.Queue), base.LeaseKey(msg.Queue),
base.CompletedKey(msg.Queue), base.CompletedKey(msg.Queue),
base.TaskKey(msg.Queue, msg.ID), base.TaskKey(msg.Queue, msg.ID),
base.ProcessedKey(msg.Queue, now), base.ProcessedKey(msg.Queue, now),
@@ -505,7 +469,7 @@ func (r *RDB) MarkAsComplete(msg *base.TaskMessage) error {
} }
// KEYS[1] -> asynq:{<qname>}:active // KEYS[1] -> asynq:{<qname>}:active
// KEYS[2] -> asynq:{<qname>}:deadlines // KEYS[2] -> asynq:{<qname>}:lease
// KEYS[3] -> asynq:{<qname>}:pending // KEYS[3] -> asynq:{<qname>}:pending
// KEYS[4] -> asynq:{<qname>}:t:<task_id> // KEYS[4] -> asynq:{<qname>}:t:<task_id>
// ARGV[1] -> task ID // ARGV[1] -> task ID
@@ -522,12 +486,11 @@ redis.call("HSET", KEYS[4], "state", "pending")
return redis.status_reply("OK")`) return redis.status_reply("OK")`)
// Requeue moves the task from active queue to the specified queue. // Requeue moves the task from active queue to the specified queue.
func (r *RDB) Requeue(msg *base.TaskMessage) error { func (r *RDB) Requeue(ctx context.Context, msg *base.TaskMessage) error {
var op errors.Op = "rdb.Requeue" var op errors.Op = "rdb.Requeue"
ctx := context.Background()
keys := []string{ keys := []string{
base.ActiveKey(msg.Queue), base.ActiveKey(msg.Queue),
base.DeadlinesKey(msg.Queue), base.LeaseKey(msg.Queue),
base.PendingKey(msg.Queue), base.PendingKey(msg.Queue),
base.TaskKey(msg.Queue, msg.ID), base.TaskKey(msg.Queue, msg.ID),
} }
@@ -536,11 +499,10 @@ func (r *RDB) Requeue(msg *base.TaskMessage) error {
// KEYS[1] -> asynq:{<qname>}:t:<task_id> // KEYS[1] -> asynq:{<qname>}:t:<task_id>
// KEYS[2] -> asynq:{<qname>}:scheduled // KEYS[2] -> asynq:{<qname>}:scheduled
// -------
// ARGV[1] -> task message data // ARGV[1] -> task message data
// ARGV[2] -> process_at time in Unix time // ARGV[2] -> process_at time in Unix time
// ARGV[3] -> task ID // ARGV[3] -> task ID
// ARGV[4] -> task timeout in seconds (0 if not timeout)
// ARGV[5] -> task deadline in unix time (0 if no deadline)
// //
// Output: // Output:
// Returns 1 if successfully enqueued // Returns 1 if successfully enqueued
@@ -551,9 +513,7 @@ if redis.call("EXISTS", KEYS[1]) == 1 then
end end
redis.call("HSET", KEYS[1], redis.call("HSET", KEYS[1],
"msg", ARGV[1], "msg", ARGV[1],
"state", "scheduled", "state", "scheduled")
"timeout", ARGV[4],
"deadline", ARGV[5])
redis.call("ZADD", KEYS[2], ARGV[2], ARGV[3]) redis.call("ZADD", KEYS[2], ARGV[2], ARGV[3])
return 1 return 1
`) `)
@@ -576,8 +536,6 @@ func (r *RDB) Schedule(ctx context.Context, msg *base.TaskMessage, processAt tim
encoded, encoded,
processAt.Unix(), processAt.Unix(),
msg.ID, msg.ID,
msg.Timeout,
msg.Deadline,
} }
n, err := r.runScriptWithErrorCode(ctx, op, scheduleCmd, keys, argv...) n, err := r.runScriptWithErrorCode(ctx, op, scheduleCmd, keys, argv...)
if err != nil { if err != nil {
@@ -592,12 +550,11 @@ func (r *RDB) Schedule(ctx context.Context, msg *base.TaskMessage, processAt tim
// KEYS[1] -> unique key // KEYS[1] -> unique key
// KEYS[2] -> asynq:{<qname>}:t:<task_id> // KEYS[2] -> asynq:{<qname>}:t:<task_id>
// KEYS[3] -> asynq:{<qname>}:scheduled // KEYS[3] -> asynq:{<qname>}:scheduled
// -------
// ARGV[1] -> task ID // ARGV[1] -> task ID
// ARGV[2] -> uniqueness lock TTL // ARGV[2] -> uniqueness lock TTL
// ARGV[3] -> score (process_at timestamp) // ARGV[3] -> score (process_at timestamp)
// ARGV[4] -> task message // ARGV[4] -> task message
// ARGV[5] -> task timeout in seconds (0 if not timeout)
// ARGV[6] -> task deadline in unix time (0 if no deadline)
// //
// Output: // Output:
// Returns 1 if successfully scheduled // Returns 1 if successfully scheduled
@@ -614,8 +571,6 @@ end
redis.call("HSET", KEYS[2], redis.call("HSET", KEYS[2],
"msg", ARGV[4], "msg", ARGV[4],
"state", "scheduled", "state", "scheduled",
"timeout", ARGV[5],
"deadline", ARGV[6],
"unique_key", KEYS[1]) "unique_key", KEYS[1])
redis.call("ZADD", KEYS[3], ARGV[3], ARGV[1]) redis.call("ZADD", KEYS[3], ARGV[3], ARGV[1])
return 1 return 1
@@ -642,8 +597,6 @@ func (r *RDB) ScheduleUnique(ctx context.Context, msg *base.TaskMessage, process
int(ttl.Seconds()), int(ttl.Seconds()),
processAt.Unix(), processAt.Unix(),
encoded, encoded,
msg.Timeout,
msg.Deadline,
} }
n, err := r.runScriptWithErrorCode(ctx, op, scheduleUniqueCmd, keys, argv...) n, err := r.runScriptWithErrorCode(ctx, op, scheduleUniqueCmd, keys, argv...)
if err != nil { if err != nil {
@@ -660,13 +613,13 @@ func (r *RDB) ScheduleUnique(ctx context.Context, msg *base.TaskMessage, process
// KEYS[1] -> asynq:{<qname>}:t:<task_id> // KEYS[1] -> asynq:{<qname>}:t:<task_id>
// KEYS[2] -> asynq:{<qname>}:active // KEYS[2] -> asynq:{<qname>}:active
// KEYS[3] -> asynq:{<qname>}:deadlines // KEYS[3] -> asynq:{<qname>}:lease
// KEYS[4] -> asynq:{<qname>}:retry // KEYS[4] -> asynq:{<qname>}:retry
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd> // KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
// KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd> // KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd>
// KEYS[7] -> asynq:{<qname>}:processed // KEYS[7] -> asynq:{<qname>}:processed
// KEYS[8] -> asynq:{<qname>}:failed // KEYS[8] -> asynq:{<qname>}:failed
// // -------
// ARGV[1] -> task ID // ARGV[1] -> task ID
// ARGV[2] -> updated base.TaskMessage value // ARGV[2] -> updated base.TaskMessage value
// ARGV[3] -> retry_at UNIX timestamp // ARGV[3] -> retry_at UNIX timestamp
@@ -705,9 +658,8 @@ return redis.status_reply("OK")`)
// Retry moves the task from active to retry queue. // Retry moves the task from active to retry queue.
// It also annotates the message with the given error message and // It also annotates the message with the given error message and
// if isFailure is true increments the retried counter. // if isFailure is true increments the retried counter.
func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string, isFailure bool) error { func (r *RDB) Retry(ctx context.Context, msg *base.TaskMessage, processAt time.Time, errMsg string, isFailure bool) error {
var op errors.Op = "rdb.Retry" var op errors.Op = "rdb.Retry"
ctx := context.Background()
now := r.clock.Now() now := r.clock.Now()
modified := *msg modified := *msg
if isFailure { if isFailure {
@@ -723,7 +675,7 @@ func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string, i
keys := []string{ keys := []string{
base.TaskKey(msg.Queue, msg.ID), base.TaskKey(msg.Queue, msg.ID),
base.ActiveKey(msg.Queue), base.ActiveKey(msg.Queue),
base.DeadlinesKey(msg.Queue), base.LeaseKey(msg.Queue),
base.RetryKey(msg.Queue), base.RetryKey(msg.Queue),
base.ProcessedKey(msg.Queue, now), base.ProcessedKey(msg.Queue, now),
base.FailedKey(msg.Queue, now), base.FailedKey(msg.Queue, now),
@@ -748,13 +700,13 @@ const (
// KEYS[1] -> asynq:{<qname>}:t:<task_id> // KEYS[1] -> asynq:{<qname>}:t:<task_id>
// KEYS[2] -> asynq:{<qname>}:active // KEYS[2] -> asynq:{<qname>}:active
// KEYS[3] -> asynq:{<qname>}:deadlines // KEYS[3] -> asynq:{<qname>}:lease
// KEYS[4] -> asynq:{<qname>}:archived // KEYS[4] -> asynq:{<qname>}:archived
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd> // KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
// KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd> // KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd>
// KEYS[7] -> asynq:{<qname>}:processed // KEYS[7] -> asynq:{<qname>}:processed
// KEYS[8] -> asynq:{<qname>}:failed // KEYS[8] -> asynq:{<qname>}:failed
// // -------
// ARGV[1] -> task ID // ARGV[1] -> task ID
// ARGV[2] -> updated base.TaskMessage value // ARGV[2] -> updated base.TaskMessage value
// ARGV[3] -> died_at UNIX timestamp // ARGV[3] -> died_at UNIX timestamp
@@ -793,9 +745,8 @@ return redis.status_reply("OK")`)
// Archive sends the given task to archive, attaching the error message to the task. // Archive sends the given task to archive, attaching the error message to the task.
// It also trims the archive by timestamp and set size. // It also trims the archive by timestamp and set size.
func (r *RDB) Archive(msg *base.TaskMessage, errMsg string) error { func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string) error {
var op errors.Op = "rdb.Archive" var op errors.Op = "rdb.Archive"
ctx := context.Background()
now := r.clock.Now() now := r.clock.Now()
modified := *msg modified := *msg
modified.ErrorMsg = errMsg modified.ErrorMsg = errMsg
@@ -809,7 +760,7 @@ func (r *RDB) Archive(msg *base.TaskMessage, errMsg string) error {
keys := []string{ keys := []string{
base.TaskKey(msg.Queue, msg.ID), base.TaskKey(msg.Queue, msg.ID),
base.ActiveKey(msg.Queue), base.ActiveKey(msg.Queue),
base.DeadlinesKey(msg.Queue), base.LeaseKey(msg.Queue),
base.ArchivedKey(msg.Queue), base.ArchivedKey(msg.Queue),
base.ProcessedKey(msg.Queue, now), base.ProcessedKey(msg.Queue, now),
base.FailedKey(msg.Queue, now), base.FailedKey(msg.Queue, now),
@@ -942,10 +893,10 @@ func (r *RDB) deleteExpiredCompletedTasks(qname string, batchSize int) (int64, e
return n, nil return n, nil
} }
// KEYS[1] -> asynq:{<qname>}:deadlines // KEYS[1] -> asynq:{<qname>}:lease
// ARGV[1] -> deadline in unix time // ARGV[1] -> cutoff in unix time
// ARGV[2] -> task key prefix // ARGV[2] -> task key prefix
var listDeadlineExceededCmd = redis.NewScript(` var listLeaseExpiredCmd = redis.NewScript(`
local res = {} local res = {}
local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1]) local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
for _, id in ipairs(ids) do for _, id in ipairs(ids) do
@@ -955,14 +906,14 @@ end
return res return res
`) `)
// ListDeadlineExceeded returns a list of task messages that have exceeded the deadline from the given queues. // ListLeaseExpired returns a list of task messages with an expired lease from the given queues.
func (r *RDB) ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*base.TaskMessage, error) { func (r *RDB) ListLeaseExpired(cutoff time.Time, qnames ...string) ([]*base.TaskMessage, error) {
var op errors.Op = "rdb.ListDeadlineExceeded" var op errors.Op = "rdb.ListLeaseExpired"
var msgs []*base.TaskMessage var msgs []*base.TaskMessage
for _, qname := range qnames { for _, qname := range qnames {
res, err := listDeadlineExceededCmd.Run(context.Background(), r.client, res, err := listLeaseExpiredCmd.Run(context.Background(), r.client,
[]string{base.DeadlinesKey(qname)}, []string{base.LeaseKey(qname)},
deadline.Unix(), base.TaskKeyPrefix(qname)).Result() cutoff.Unix(), base.TaskKeyPrefix(qname)).Result()
if err != nil { if err != nil {
return nil, errors.E(op, errors.Internal, fmt.Sprintf("redis eval error: %v", err)) return nil, errors.E(op, errors.Internal, fmt.Sprintf("redis eval error: %v", err))
} }
@@ -981,6 +932,22 @@ func (r *RDB) ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*bas
return msgs, nil return msgs, nil
} }
// ExtendLease extends the lease for the given tasks by LeaseDuration (30s).
// It returns a new expiration time if the operation was successful.
func (r *RDB) ExtendLease(qname string, ids ...string) (expirationTime time.Time, err error) {
expireAt := r.clock.Now().Add(LeaseDuration)
var zs []redis.Z
for _, id := range ids {
zs = append(zs, redis.Z{Member: id, Score: float64(expireAt.Unix())})
}
// Use XX option to only update elements that already exist; Don't add new elements
err = r.client.ZAddArgs(context.Background(), base.LeaseKey(qname), redis.ZAddArgs{XX: true, GT: true, Members: zs}).Err()
if err != nil {
return time.Time{}, err
}
return expireAt, nil
}
// KEYS[1] -> asynq:servers:{<host:pid:sid>} // KEYS[1] -> asynq:servers:{<host:pid:sid>}
// KEYS[2] -> asynq:workers:{<host:pid:sid>} // KEYS[2] -> asynq:workers:{<host:pid:sid>}
// ARGV[1] -> TTL in seconds // ARGV[1] -> TTL in seconds

File diff suppressed because it is too large Load Diff

View File

@@ -73,31 +73,31 @@ func (tb *TestBroker) Dequeue(qnames ...string) (*base.TaskMessage, time.Time, e
return tb.real.Dequeue(qnames...) return tb.real.Dequeue(qnames...)
} }
func (tb *TestBroker) Done(msg *base.TaskMessage) error { func (tb *TestBroker) Done(ctx context.Context, msg *base.TaskMessage) error {
tb.mu.Lock() tb.mu.Lock()
defer tb.mu.Unlock() defer tb.mu.Unlock()
if tb.sleeping { if tb.sleeping {
return errRedisDown return errRedisDown
} }
return tb.real.Done(msg) return tb.real.Done(ctx, msg)
} }
func (tb *TestBroker) MarkAsComplete(msg *base.TaskMessage) error { func (tb *TestBroker) MarkAsComplete(ctx context.Context, msg *base.TaskMessage) error {
tb.mu.Lock() tb.mu.Lock()
defer tb.mu.Unlock() defer tb.mu.Unlock()
if tb.sleeping { if tb.sleeping {
return errRedisDown return errRedisDown
} }
return tb.real.MarkAsComplete(msg) return tb.real.MarkAsComplete(ctx, msg)
} }
func (tb *TestBroker) Requeue(msg *base.TaskMessage) error { func (tb *TestBroker) Requeue(ctx context.Context, msg *base.TaskMessage) error {
tb.mu.Lock() tb.mu.Lock()
defer tb.mu.Unlock() defer tb.mu.Unlock()
if tb.sleeping { if tb.sleeping {
return errRedisDown return errRedisDown
} }
return tb.real.Requeue(msg) return tb.real.Requeue(ctx, msg)
} }
func (tb *TestBroker) Schedule(ctx context.Context, msg *base.TaskMessage, processAt time.Time) error { func (tb *TestBroker) Schedule(ctx context.Context, msg *base.TaskMessage, processAt time.Time) error {
@@ -118,22 +118,22 @@ func (tb *TestBroker) ScheduleUnique(ctx context.Context, msg *base.TaskMessage,
return tb.real.ScheduleUnique(ctx, msg, processAt, ttl) return tb.real.ScheduleUnique(ctx, msg, processAt, ttl)
} }
func (tb *TestBroker) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string, isFailure bool) error { func (tb *TestBroker) Retry(ctx context.Context, msg *base.TaskMessage, processAt time.Time, errMsg string, isFailure bool) error {
tb.mu.Lock() tb.mu.Lock()
defer tb.mu.Unlock() defer tb.mu.Unlock()
if tb.sleeping { if tb.sleeping {
return errRedisDown return errRedisDown
} }
return tb.real.Retry(msg, processAt, errMsg, isFailure) return tb.real.Retry(ctx, msg, processAt, errMsg, isFailure)
} }
func (tb *TestBroker) Archive(msg *base.TaskMessage, errMsg string) error { func (tb *TestBroker) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string) error {
tb.mu.Lock() tb.mu.Lock()
defer tb.mu.Unlock() defer tb.mu.Unlock()
if tb.sleeping { if tb.sleeping {
return errRedisDown return errRedisDown
} }
return tb.real.Archive(msg, errMsg) return tb.real.Archive(ctx, msg, errMsg)
} }
func (tb *TestBroker) ForwardIfReady(qnames ...string) error { func (tb *TestBroker) ForwardIfReady(qnames ...string) error {
@@ -154,13 +154,22 @@ func (tb *TestBroker) DeleteExpiredCompletedTasks(qname string) error {
return tb.real.DeleteExpiredCompletedTasks(qname) return tb.real.DeleteExpiredCompletedTasks(qname)
} }
func (tb *TestBroker) ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*base.TaskMessage, error) { func (tb *TestBroker) ListLeaseExpired(cutoff time.Time, qnames ...string) ([]*base.TaskMessage, error) {
tb.mu.Lock() tb.mu.Lock()
defer tb.mu.Unlock() defer tb.mu.Unlock()
if tb.sleeping { if tb.sleeping {
return nil, errRedisDown return nil, errRedisDown
} }
return tb.real.ListDeadlineExceeded(deadline, qnames...) return tb.real.ListLeaseExpired(cutoff, qnames...)
}
func (tb *TestBroker) ExtendLease(qname string, ids ...string) (time.Time, error) {
tb.mu.Lock()
defer tb.mu.Unlock()
if tb.sleeping {
return time.Time{}, errRedisDown
}
return tb.real.ExtendLease(qname, ids...)
} }
func (tb *TestBroker) WriteServerState(info *base.ServerInfo, workers []*base.WorkerInfo, ttl time.Duration) error { func (tb *TestBroker) WriteServerState(info *base.ServerInfo, workers []*base.WorkerInfo, ttl time.Duration) error {

View File

@@ -1,7 +1,14 @@
// Copyright 2022 Kentaro Hibino. All rights reserved.
// Use of this source code is governed by a MIT license
// that can be found in the LICENSE file.
// Package timeutil exports functions and types related to time and date. // Package timeutil exports functions and types related to time and date.
package timeutil package timeutil
import "time" import (
"sync"
"time"
)
// A Clock is an object that can tell you the current time. // A Clock is an object that can tell you the current time.
// //
@@ -23,16 +30,30 @@ func (_ *realTimeClock) Now() time.Time { return time.Now() }
// A SimulatedClock is a concrete Clock implementation that doesn't "tick" on its own. // A SimulatedClock is a concrete Clock implementation that doesn't "tick" on its own.
// Time is advanced by explicit call to the AdvanceTime() or SetTime() functions. // Time is advanced by explicit call to the AdvanceTime() or SetTime() functions.
// This object is concurrency safe.
type SimulatedClock struct { type SimulatedClock struct {
t time.Time mu sync.Mutex
t time.Time // guarded by mu
} }
func NewSimulatedClock(t time.Time) *SimulatedClock { func NewSimulatedClock(t time.Time) *SimulatedClock {
return &SimulatedClock{t} return &SimulatedClock{t: t}
} }
func (c *SimulatedClock) Now() time.Time { return c.t } func (c *SimulatedClock) Now() time.Time {
c.mu.Lock()
defer c.mu.Unlock()
return c.t
}
func (c *SimulatedClock) SetTime(t time.Time) { c.t = t } func (c *SimulatedClock) SetTime(t time.Time) {
c.mu.Lock()
defer c.mu.Unlock()
c.t = t
}
func (c *SimulatedClock) AdvanceTime(d time.Duration) { c.t.Add(d) } func (c *SimulatedClock) AdvanceTime(d time.Duration) {
c.mu.Lock()
defer c.mu.Unlock()
c.t = c.t.Add(d)
}

View File

@@ -0,0 +1,48 @@
// Copyright 2022 Kentaro Hibino. All rights reserved.
// Use of this source code is governed by a MIT license
// that can be found in the LICENSE file.
package timeutil
import (
"testing"
"time"
)
func TestSimulatedClock(t *testing.T) {
now := time.Now()
tests := []struct {
desc string
initTime time.Time
advanceBy time.Duration
wantTime time.Time
}{
{
desc: "advance time forward",
initTime: now,
advanceBy: 30 * time.Second,
wantTime: now.Add(30 * time.Second),
},
{
desc: "advance time backward",
initTime: now,
advanceBy: -10 * time.Second,
wantTime: now.Add(-10 * time.Second),
},
}
for _, tc := range tests {
c := NewSimulatedClock(tc.initTime)
if c.Now() != tc.initTime {
t.Errorf("%s: Before Advance; SimulatedClock.Now() = %v, want %v", tc.desc, c.Now(), tc.initTime)
}
c.AdvanceTime(tc.advanceBy)
if c.Now() != tc.wantTime {
t.Errorf("%s: After Advance; SimulatedClock.Now() = %v, want %v", tc.desc, c.Now(), tc.wantTime)
}
}
}

View File

@@ -74,7 +74,7 @@ func (j *janitor) start(wg *sync.WaitGroup) {
func (j *janitor) exec() { func (j *janitor) exec() {
for _, qname := range j.queues { for _, qname := range j.queues {
if err := j.broker.DeleteExpiredCompletedTasks(qname); err != nil { if err := j.broker.DeleteExpiredCompletedTasks(qname); err != nil {
j.logger.Errorf("Could not delete expired completed tasks from queue %q: %v", j.logger.Errorf("Failed to delete expired completed tasks from queue %q: %v",
qname, err) qname, err)
} }
} }

View File

@@ -7,6 +7,7 @@ package asynq
import ( import (
"context" "context"
"fmt" "fmt"
"math"
"math/rand" "math/rand"
"runtime" "runtime"
"runtime/debug" "runtime/debug"
@@ -19,14 +20,17 @@ import (
asynqcontext "github.com/hibiken/asynq/internal/context" asynqcontext "github.com/hibiken/asynq/internal/context"
"github.com/hibiken/asynq/internal/errors" "github.com/hibiken/asynq/internal/errors"
"github.com/hibiken/asynq/internal/log" "github.com/hibiken/asynq/internal/log"
"github.com/hibiken/asynq/internal/timeutil"
"golang.org/x/time/rate" "golang.org/x/time/rate"
) )
type processor struct { type processor struct {
logger *log.Logger logger *log.Logger
broker base.Broker broker base.Broker
clock timeutil.Clock
handler Handler handler Handler
baseCtxFn func() context.Context
queueConfig map[string]int queueConfig map[string]int
@@ -71,6 +75,7 @@ type processor struct {
type processorParams struct { type processorParams struct {
logger *log.Logger logger *log.Logger
broker base.Broker broker base.Broker
baseCtxFn func() context.Context
retryDelayFunc RetryDelayFunc retryDelayFunc RetryDelayFunc
isFailureFunc func(error) bool isFailureFunc func(error) bool
syncCh chan<- *syncRequest syncCh chan<- *syncRequest
@@ -94,6 +99,8 @@ func newProcessor(params processorParams) *processor {
return &processor{ return &processor{
logger: params.logger, logger: params.logger,
broker: params.broker, broker: params.broker,
baseCtxFn: params.baseCtxFn,
clock: timeutil.NewRealClock(),
queueConfig: queues, queueConfig: queues,
orderedQueues: orderedQueues, orderedQueues: orderedQueues,
retryDelayFunc: params.retryDelayFunc, retryDelayFunc: params.retryDelayFunc,
@@ -164,7 +171,7 @@ func (p *processor) exec() {
return return
case p.sema <- struct{}{}: // acquire token case p.sema <- struct{}{}: // acquire token
qnames := p.queues() qnames := p.queues()
msg, deadline, err := p.broker.Dequeue(qnames...) msg, leaseExpirationTime, err := p.broker.Dequeue(qnames...)
switch { switch {
case errors.Is(err, errors.ErrNoProcessableTask): case errors.Is(err, errors.ErrNoProcessableTask):
p.logger.Debug("All queues are empty") p.logger.Debug("All queues are empty")
@@ -183,14 +190,16 @@ func (p *processor) exec() {
return return
} }
p.starting <- &workerInfo{msg, time.Now(), deadline} lease := base.NewLease(leaseExpirationTime)
deadline := p.computeDeadline(msg)
p.starting <- &workerInfo{msg, time.Now(), deadline, lease}
go func() { go func() {
defer func() { defer func() {
p.finished <- msg p.finished <- msg
<-p.sema // release token <-p.sema // release token
}() }()
ctx, cancel := asynqcontext.New(msg, deadline) ctx, cancel := asynqcontext.New(p.baseCtxFn(), msg, deadline)
p.cancelations.Add(msg.ID, cancel) p.cancelations.Add(msg.ID, cancel)
defer func() { defer func() {
cancel() cancel()
@@ -201,7 +210,7 @@ func (p *processor) exec() {
select { select {
case <-ctx.Done(): case <-ctx.Done():
// already canceled (e.g. deadline exceeded). // already canceled (e.g. deadline exceeded).
p.handleFailedMessage(ctx, msg, ctx.Err()) p.handleFailedMessage(ctx, lease, msg, ctx.Err())
return return
default: default:
} }
@@ -225,24 +234,33 @@ func (p *processor) exec() {
case <-p.abort: case <-p.abort:
// time is up, push the message back to queue and quit this worker goroutine. // time is up, push the message back to queue and quit this worker goroutine.
p.logger.Warnf("Quitting worker. task id=%s", msg.ID) p.logger.Warnf("Quitting worker. task id=%s", msg.ID)
p.requeue(msg) p.requeue(lease, msg)
return
case <-lease.Done():
cancel()
p.handleFailedMessage(ctx, lease, msg, ErrLeaseExpired)
return return
case <-ctx.Done(): case <-ctx.Done():
p.handleFailedMessage(ctx, msg, ctx.Err()) p.handleFailedMessage(ctx, lease, msg, ctx.Err())
return return
case resErr := <-resCh: case resErr := <-resCh:
if resErr != nil { if resErr != nil {
p.handleFailedMessage(ctx, msg, resErr) p.handleFailedMessage(ctx, lease, msg, resErr)
return return
} }
p.handleSucceededMessage(ctx, msg) p.handleSucceededMessage(lease, msg)
} }
}() }()
} }
} }
func (p *processor) requeue(msg *base.TaskMessage) { func (p *processor) requeue(l *base.Lease, msg *base.TaskMessage) {
err := p.broker.Requeue(msg) if !l.IsValid() {
// If lease is not valid, do not write to redis; Let recoverer take care of it.
return
}
ctx, _ := context.WithDeadline(context.Background(), l.Deadline())
err := p.broker.Requeue(ctx, msg)
if err != nil { if err != nil {
p.logger.Errorf("Could not push task id=%s back to queue: %v", msg.ID, err) p.logger.Errorf("Could not push task id=%s back to queue: %v", msg.ID, err)
} else { } else {
@@ -250,49 +268,51 @@ func (p *processor) requeue(msg *base.TaskMessage) {
} }
} }
func (p *processor) handleSucceededMessage(ctx context.Context, msg *base.TaskMessage) { func (p *processor) handleSucceededMessage(l *base.Lease, msg *base.TaskMessage) {
if msg.Retention > 0 { if msg.Retention > 0 {
p.markAsComplete(ctx, msg) p.markAsComplete(l, msg)
} else { } else {
p.markAsDone(ctx, msg) p.markAsDone(l, msg)
} }
} }
func (p *processor) markAsComplete(ctx context.Context, msg *base.TaskMessage) { func (p *processor) markAsComplete(l *base.Lease, msg *base.TaskMessage) {
err := p.broker.MarkAsComplete(msg) if !l.IsValid() {
// If lease is not valid, do not write to redis; Let recoverer take care of it.
return
}
ctx, _ := context.WithDeadline(context.Background(), l.Deadline())
err := p.broker.MarkAsComplete(ctx, msg)
if err != nil { if err != nil {
errMsg := fmt.Sprintf("Could not move task id=%s type=%q from %q to %q: %+v", errMsg := fmt.Sprintf("Could not move task id=%s type=%q from %q to %q: %+v",
msg.ID, msg.Type, base.ActiveKey(msg.Queue), base.CompletedKey(msg.Queue), err) msg.ID, msg.Type, base.ActiveKey(msg.Queue), base.CompletedKey(msg.Queue), err)
deadline, ok := ctx.Deadline()
if !ok {
panic("asynq: internal error: missing deadline in context")
}
p.logger.Warnf("%s; Will retry syncing", errMsg) p.logger.Warnf("%s; Will retry syncing", errMsg)
p.syncRequestCh <- &syncRequest{ p.syncRequestCh <- &syncRequest{
fn: func() error { fn: func() error {
return p.broker.MarkAsComplete(msg) return p.broker.MarkAsComplete(ctx, msg)
}, },
errMsg: errMsg, errMsg: errMsg,
deadline: deadline, deadline: l.Deadline(),
} }
} }
} }
func (p *processor) markAsDone(ctx context.Context, msg *base.TaskMessage) { func (p *processor) markAsDone(l *base.Lease, msg *base.TaskMessage) {
err := p.broker.Done(msg) if !l.IsValid() {
// If lease is not valid, do not write to redis; Let recoverer take care of it.
return
}
ctx, _ := context.WithDeadline(context.Background(), l.Deadline())
err := p.broker.Done(ctx, msg)
if err != nil { if err != nil {
errMsg := fmt.Sprintf("Could not remove task id=%s type=%q from %q err: %+v", msg.ID, msg.Type, base.ActiveKey(msg.Queue), err) errMsg := fmt.Sprintf("Could not remove task id=%s type=%q from %q err: %+v", msg.ID, msg.Type, base.ActiveKey(msg.Queue), err)
deadline, ok := ctx.Deadline()
if !ok {
panic("asynq: internal error: missing deadline in context")
}
p.logger.Warnf("%s; Will retry syncing", errMsg) p.logger.Warnf("%s; Will retry syncing", errMsg)
p.syncRequestCh <- &syncRequest{ p.syncRequestCh <- &syncRequest{
fn: func() error { fn: func() error {
return p.broker.Done(msg) return p.broker.Done(ctx, msg)
}, },
errMsg: errMsg, errMsg: errMsg,
deadline: deadline, deadline: l.Deadline(),
} }
} }
} }
@@ -301,59 +321,61 @@ func (p *processor) markAsDone(ctx context.Context, msg *base.TaskMessage) {
// the task should not be retried and should be archived instead. // the task should not be retried and should be archived instead.
var SkipRetry = errors.New("skip retry for the task") var SkipRetry = errors.New("skip retry for the task")
func (p *processor) handleFailedMessage(ctx context.Context, msg *base.TaskMessage, err error) { func (p *processor) handleFailedMessage(ctx context.Context, l *base.Lease, msg *base.TaskMessage, err error) {
if p.errHandler != nil { if p.errHandler != nil {
p.errHandler.HandleError(ctx, NewTask(msg.Type, msg.Payload), err) p.errHandler.HandleError(ctx, NewTask(msg.Type, msg.Payload), err)
} }
if !p.isFailureFunc(err) { if !p.isFailureFunc(err) {
// retry the task without marking it as failed // retry the task without marking it as failed
p.retry(ctx, msg, err, false /*isFailure*/) p.retry(l, msg, err, false /*isFailure*/)
return return
} }
if msg.Retried >= msg.Retry || errors.Is(err, SkipRetry) { if msg.Retried >= msg.Retry || errors.Is(err, SkipRetry) {
p.logger.Warnf("Retry exhausted for task id=%s", msg.ID) p.logger.Warnf("Retry exhausted for task id=%s", msg.ID)
p.archive(ctx, msg, err) p.archive(l, msg, err)
} else { } else {
p.retry(ctx, msg, err, true /*isFailure*/) p.retry(l, msg, err, true /*isFailure*/)
} }
} }
func (p *processor) retry(ctx context.Context, msg *base.TaskMessage, e error, isFailure bool) { func (p *processor) retry(l *base.Lease, msg *base.TaskMessage, e error, isFailure bool) {
if !l.IsValid() {
// If lease is not valid, do not write to redis; Let recoverer take care of it.
return
}
ctx, _ := context.WithDeadline(context.Background(), l.Deadline())
d := p.retryDelayFunc(msg.Retried, e, NewTask(msg.Type, msg.Payload)) d := p.retryDelayFunc(msg.Retried, e, NewTask(msg.Type, msg.Payload))
retryAt := time.Now().Add(d) retryAt := time.Now().Add(d)
err := p.broker.Retry(msg, retryAt, e.Error(), isFailure) err := p.broker.Retry(ctx, msg, retryAt, e.Error(), isFailure)
if err != nil { if err != nil {
errMsg := fmt.Sprintf("Could not move task id=%s from %q to %q", msg.ID, base.ActiveKey(msg.Queue), base.RetryKey(msg.Queue)) errMsg := fmt.Sprintf("Could not move task id=%s from %q to %q", msg.ID, base.ActiveKey(msg.Queue), base.RetryKey(msg.Queue))
deadline, ok := ctx.Deadline()
if !ok {
panic("asynq: internal error: missing deadline in context")
}
p.logger.Warnf("%s; Will retry syncing", errMsg) p.logger.Warnf("%s; Will retry syncing", errMsg)
p.syncRequestCh <- &syncRequest{ p.syncRequestCh <- &syncRequest{
fn: func() error { fn: func() error {
return p.broker.Retry(msg, retryAt, e.Error(), isFailure) return p.broker.Retry(ctx, msg, retryAt, e.Error(), isFailure)
}, },
errMsg: errMsg, errMsg: errMsg,
deadline: deadline, deadline: l.Deadline(),
} }
} }
} }
func (p *processor) archive(ctx context.Context, msg *base.TaskMessage, e error) { func (p *processor) archive(l *base.Lease, msg *base.TaskMessage, e error) {
err := p.broker.Archive(msg, e.Error()) if !l.IsValid() {
// If lease is not valid, do not write to redis; Let recoverer take care of it.
return
}
ctx, _ := context.WithDeadline(context.Background(), l.Deadline())
err := p.broker.Archive(ctx, msg, e.Error())
if err != nil { if err != nil {
errMsg := fmt.Sprintf("Could not move task id=%s from %q to %q", msg.ID, base.ActiveKey(msg.Queue), base.ArchivedKey(msg.Queue)) errMsg := fmt.Sprintf("Could not move task id=%s from %q to %q", msg.ID, base.ActiveKey(msg.Queue), base.ArchivedKey(msg.Queue))
deadline, ok := ctx.Deadline()
if !ok {
panic("asynq: internal error: missing deadline in context")
}
p.logger.Warnf("%s; Will retry syncing", errMsg) p.logger.Warnf("%s; Will retry syncing", errMsg)
p.syncRequestCh <- &syncRequest{ p.syncRequestCh <- &syncRequest{
fn: func() error { fn: func() error {
return p.broker.Archive(msg, e.Error()) return p.broker.Archive(ctx, msg, e.Error())
}, },
errMsg: errMsg, errMsg: errMsg,
deadline: deadline, deadline: l.Deadline(),
} }
} }
} }
@@ -483,3 +505,19 @@ func gcd(xs ...int) int {
} }
return res return res
} }
// computeDeadline returns the given task's deadline,
func (p *processor) computeDeadline(msg *base.TaskMessage) time.Time {
if msg.Timeout == 0 && msg.Deadline == 0 {
p.logger.Errorf("asynq: internal error: both timeout and deadline are not set for the task message: %s", msg.ID)
return p.clock.Now().Add(defaultTimeout)
}
if msg.Timeout != 0 && msg.Deadline != 0 {
deadlineUnix := math.Min(float64(p.clock.Now().Unix()+msg.Timeout), float64(msg.Deadline))
return time.Unix(int64(deadlineUnix), 0)
}
if msg.Timeout != 0 {
return p.clock.Now().Add(time.Duration(msg.Timeout) * time.Second)
}
return time.Unix(msg.Deadline, 0)
}

View File

@@ -17,7 +17,10 @@ import (
"github.com/google/go-cmp/cmp/cmpopts" "github.com/google/go-cmp/cmp/cmpopts"
h "github.com/hibiken/asynq/internal/asynqtest" h "github.com/hibiken/asynq/internal/asynqtest"
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
"github.com/hibiken/asynq/internal/errors"
"github.com/hibiken/asynq/internal/log"
"github.com/hibiken/asynq/internal/rdb" "github.com/hibiken/asynq/internal/rdb"
"github.com/hibiken/asynq/internal/timeutil"
) )
var taskCmpOpts = []cmp.Option{ var taskCmpOpts = []cmp.Option{
@@ -61,6 +64,7 @@ func newProcessorForTest(t *testing.T, r *rdb.RDB, h Handler) *processor {
p := newProcessor(processorParams{ p := newProcessor(processorParams{
logger: testLogger, logger: testLogger,
broker: r, broker: r,
baseCtxFn: context.Background,
retryDelayFunc: DefaultRetryDelayFunc, retryDelayFunc: DefaultRetryDelayFunc,
isFailureFunc: defaultIsFailureFunc, isFailureFunc: defaultIsFailureFunc,
syncCh: syncCh, syncCh: syncCh,
@@ -480,6 +484,104 @@ func TestProcessorMarkAsComplete(t *testing.T) {
} }
} }
// Test a scenario where the worker server cannot communicate with redis due to a network failure
// and the lease expires
func TestProcessorWithExpiredLease(t *testing.T) {
r := setup(t)
defer r.Close()
rdbClient := rdb.NewRDB(r)
m1 := h.NewTaskMessage("task1", nil)
tests := []struct {
pending []*base.TaskMessage
handler Handler
wantErrCount int
}{
{
pending: []*base.TaskMessage{m1},
handler: HandlerFunc(func(ctx context.Context, task *Task) error {
// make sure the task processing time exceeds lease duration
// to test expired lease.
time.Sleep(rdb.LeaseDuration + 10*time.Second)
return nil
}),
wantErrCount: 1, // ErrorHandler should still be called with ErrLeaseExpired
},
}
for _, tc := range tests {
h.FlushDB(t, r)
h.SeedPendingQueue(t, r, tc.pending, base.DefaultQueueName)
starting := make(chan *workerInfo)
finished := make(chan *base.TaskMessage)
syncCh := make(chan *syncRequest)
done := make(chan struct{})
t.Cleanup(func() { close(done) })
// fake heartbeater which notifies lease expiration
go func() {
for {
select {
case w := <-starting:
// simulate expiration by resetting to some time in the past
w.lease.Reset(time.Now().Add(-5 * time.Second))
if !w.lease.NotifyExpiration() {
panic("Failed to notifiy lease expiration")
}
case <-finished:
// do nothing
case <-done:
return
}
}
}()
go fakeSyncer(syncCh, done)
p := newProcessor(processorParams{
logger: testLogger,
broker: rdbClient,
baseCtxFn: context.Background,
retryDelayFunc: DefaultRetryDelayFunc,
isFailureFunc: defaultIsFailureFunc,
syncCh: syncCh,
cancelations: base.NewCancelations(),
concurrency: 10,
queues: defaultQueueConfig,
strictPriority: false,
errHandler: nil,
shutdownTimeout: defaultShutdownTimeout,
starting: starting,
finished: finished,
})
p.handler = tc.handler
var (
mu sync.Mutex // guards n and errs
n int // number of times error handler is called
errs []error // error passed to error handler
)
p.errHandler = ErrorHandlerFunc(func(ctx context.Context, t *Task, err error) {
mu.Lock()
defer mu.Unlock()
n++
errs = append(errs, err)
})
p.start(&sync.WaitGroup{})
time.Sleep(4 * time.Second)
p.shutdown()
if n != tc.wantErrCount {
t.Errorf("Unexpected number of error count: got %d, want %d", n, tc.wantErrCount)
continue
}
for i := 0; i < tc.wantErrCount; i++ {
if !errors.Is(errs[i], ErrLeaseExpired) {
t.Errorf("Unexpected error was passed to ErrorHandler: got %v want %v", errs[i], ErrLeaseExpired)
}
}
}
}
func TestProcessorQueues(t *testing.T) { func TestProcessorQueues(t *testing.T) {
sortOpt := cmp.Transformer("SortStrings", func(in []string) []string { sortOpt := cmp.Transformer("SortStrings", func(in []string) []string {
out := append([]string(nil), in...) // Copy input to avoid mutating it out := append([]string(nil), in...) // Copy input to avoid mutating it
@@ -592,6 +694,7 @@ func TestProcessorWithStrictPriority(t *testing.T) {
p := newProcessor(processorParams{ p := newProcessor(processorParams{
logger: testLogger, logger: testLogger,
broker: rdbClient, broker: rdbClient,
baseCtxFn: context.Background,
retryDelayFunc: DefaultRetryDelayFunc, retryDelayFunc: DefaultRetryDelayFunc,
isFailureFunc: defaultIsFailureFunc, isFailureFunc: defaultIsFailureFunc,
syncCh: syncCh, syncCh: syncCh,
@@ -752,3 +855,69 @@ func TestNormalizeQueues(t *testing.T) {
} }
} }
} }
func TestProcessorComputeDeadline(t *testing.T) {
now := time.Now()
p := processor{
logger: log.NewLogger(nil),
clock: timeutil.NewSimulatedClock(now),
}
tests := []struct {
desc string
msg *base.TaskMessage
want time.Time
}{
{
desc: "message with only timeout specified",
msg: &base.TaskMessage{
Timeout: int64((30 * time.Minute).Seconds()),
},
want: now.Add(30 * time.Minute),
},
{
desc: "message with only deadline specified",
msg: &base.TaskMessage{
Deadline: now.Add(24 * time.Hour).Unix(),
},
want: now.Add(24 * time.Hour),
},
{
desc: "message with both timeout and deadline set (now+timeout < deadline)",
msg: &base.TaskMessage{
Deadline: now.Add(24 * time.Hour).Unix(),
Timeout: int64((30 * time.Minute).Seconds()),
},
want: now.Add(30 * time.Minute),
},
{
desc: "message with both timeout and deadline set (now+timeout > deadline)",
msg: &base.TaskMessage{
Deadline: now.Add(10 * time.Minute).Unix(),
Timeout: int64((30 * time.Minute).Seconds()),
},
want: now.Add(10 * time.Minute),
},
{
desc: "message with both timeout and deadline set (now+timeout == deadline)",
msg: &base.TaskMessage{
Deadline: now.Add(30 * time.Minute).Unix(),
Timeout: int64((30 * time.Minute).Seconds()),
},
want: now.Add(30 * time.Minute),
},
{
desc: "message without timeout and deadline",
msg: &base.TaskMessage{},
want: now.Add(defaultTimeout),
},
}
for _, tc := range tests {
got := p.computeDeadline(tc.msg)
// Compare the Unix epoch with seconds granularity
if got.Unix() != tc.want.Unix() {
t.Errorf("%s: got=%v, want=%v", tc.desc, got.Unix(), tc.want.Unix())
}
}
}

View File

@@ -10,6 +10,7 @@ import (
"time" "time"
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
"github.com/hibiken/asynq/internal/errors"
"github.com/hibiken/asynq/internal/log" "github.com/hibiken/asynq/internal/log"
) )
@@ -76,19 +77,23 @@ func (r *recoverer) start(wg *sync.WaitGroup) {
}() }()
} }
// ErrLeaseExpired error indicates that the task failed because the worker working on the task
// could not extend its lease due to missing heartbeats. The worker may have crashed or got cutoff from the network.
var ErrLeaseExpired = errors.New("asynq: task lease expired")
func (r *recoverer) recover() { func (r *recoverer) recover() {
// Get all tasks which have expired 30 seconds ago or earlier. // Get all tasks which have expired 30 seconds ago or earlier to accomodate certain amount of clock skew.
deadline := time.Now().Add(-30 * time.Second) cutoff := time.Now().Add(-30 * time.Second)
msgs, err := r.broker.ListDeadlineExceeded(deadline, r.queues...) msgs, err := r.broker.ListLeaseExpired(cutoff, r.queues...)
if err != nil { if err != nil {
r.logger.Warn("recoverer: could not list deadline exceeded tasks") r.logger.Warn("recoverer: could not list lease expired tasks")
return return
} }
for _, msg := range msgs { for _, msg := range msgs {
if msg.Retried >= msg.Retry { if msg.Retried >= msg.Retry {
r.archive(msg, context.DeadlineExceeded) r.archive(msg, ErrLeaseExpired)
} else { } else {
r.retry(msg, context.DeadlineExceeded) r.retry(msg, ErrLeaseExpired)
} }
} }
} }
@@ -96,13 +101,13 @@ func (r *recoverer) recover() {
func (r *recoverer) retry(msg *base.TaskMessage, err error) { func (r *recoverer) retry(msg *base.TaskMessage, err error) {
delay := r.retryDelayFunc(msg.Retried, err, NewTask(msg.Type, msg.Payload)) delay := r.retryDelayFunc(msg.Retried, err, NewTask(msg.Type, msg.Payload))
retryAt := time.Now().Add(delay) retryAt := time.Now().Add(delay)
if err := r.broker.Retry(msg, retryAt, err.Error(), r.isFailureFunc(err)); err != nil { if err := r.broker.Retry(context.Background(), msg, retryAt, err.Error(), r.isFailureFunc(err)); err != nil {
r.logger.Warnf("recoverer: could not retry deadline exceeded task: %v", err) r.logger.Warnf("recoverer: could not retry lease expired task: %v", err)
} }
} }
func (r *recoverer) archive(msg *base.TaskMessage, err error) { func (r *recoverer) archive(msg *base.TaskMessage, err error) {
if err := r.broker.Archive(msg, err.Error()); err != nil { if err := r.broker.Archive(context.Background(), msg, err.Error()); err != nil {
r.logger.Warnf("recoverer: could not move task to archive: %v", err) r.logger.Warnf("recoverer: could not move task to archive: %v", err)
} }
} }

View File

@@ -27,29 +27,25 @@ func TestRecoverer(t *testing.T) {
t4.Retried = t4.Retry // t4 has reached its max retry count t4.Retried = t4.Retry // t4 has reached its max retry count
now := time.Now() now := time.Now()
oneHourFromNow := now.Add(1 * time.Hour)
fiveMinutesFromNow := now.Add(5 * time.Minute)
fiveMinutesAgo := now.Add(-5 * time.Minute)
oneHourAgo := now.Add(-1 * time.Hour)
tests := []struct { tests := []struct {
desc string desc string
inProgress map[string][]*base.TaskMessage active map[string][]*base.TaskMessage
deadlines map[string][]base.Z lease map[string][]base.Z
retry map[string][]base.Z retry map[string][]base.Z
archived map[string][]base.Z archived map[string][]base.Z
wantActive map[string][]*base.TaskMessage wantActive map[string][]*base.TaskMessage
wantDeadlines map[string][]base.Z wantLease map[string][]base.Z
wantRetry map[string][]*base.TaskMessage wantRetry map[string][]*base.TaskMessage
wantArchived map[string][]*base.TaskMessage wantArchived map[string][]*base.TaskMessage
}{ }{
{ {
desc: "with one active task", desc: "with one active task",
inProgress: map[string][]*base.TaskMessage{ active: map[string][]*base.TaskMessage{
"default": {t1}, "default": {t1},
}, },
deadlines: map[string][]base.Z{ lease: map[string][]base.Z{
"default": {{Message: t1, Score: fiveMinutesAgo.Unix()}}, "default": {{Message: t1, Score: now.Add(-1 * time.Minute).Unix()}},
}, },
retry: map[string][]base.Z{ retry: map[string][]base.Z{
"default": {}, "default": {},
@@ -60,7 +56,7 @@ func TestRecoverer(t *testing.T) {
wantActive: map[string][]*base.TaskMessage{ wantActive: map[string][]*base.TaskMessage{
"default": {}, "default": {},
}, },
wantDeadlines: map[string][]base.Z{ wantLease: map[string][]base.Z{
"default": {}, "default": {},
}, },
wantRetry: map[string][]*base.TaskMessage{ wantRetry: map[string][]*base.TaskMessage{
@@ -72,12 +68,12 @@ func TestRecoverer(t *testing.T) {
}, },
{ {
desc: "with a task with max-retry reached", desc: "with a task with max-retry reached",
inProgress: map[string][]*base.TaskMessage{ active: map[string][]*base.TaskMessage{
"default": {t4}, "default": {t4},
"critical": {}, "critical": {},
}, },
deadlines: map[string][]base.Z{ lease: map[string][]base.Z{
"default": {{Message: t4, Score: fiveMinutesAgo.Unix()}}, "default": {{Message: t4, Score: now.Add(-40 * time.Second).Unix()}},
"critical": {}, "critical": {},
}, },
retry: map[string][]base.Z{ retry: map[string][]base.Z{
@@ -92,7 +88,7 @@ func TestRecoverer(t *testing.T) {
"default": {}, "default": {},
"critical": {}, "critical": {},
}, },
wantDeadlines: map[string][]base.Z{ wantLease: map[string][]base.Z{
"default": {}, "default": {},
"critical": {}, "critical": {},
}, },
@@ -107,17 +103,17 @@ func TestRecoverer(t *testing.T) {
}, },
{ {
desc: "with multiple active tasks, and one expired", desc: "with multiple active tasks, and one expired",
inProgress: map[string][]*base.TaskMessage{ active: map[string][]*base.TaskMessage{
"default": {t1, t2}, "default": {t1, t2},
"critical": {t3}, "critical": {t3},
}, },
deadlines: map[string][]base.Z{ lease: map[string][]base.Z{
"default": { "default": {
{Message: t1, Score: oneHourAgo.Unix()}, {Message: t1, Score: now.Add(-2 * time.Minute).Unix()},
{Message: t2, Score: fiveMinutesFromNow.Unix()}, {Message: t2, Score: now.Add(20 * time.Second).Unix()},
}, },
"critical": { "critical": {
{Message: t3, Score: oneHourFromNow.Unix()}, {Message: t3, Score: now.Add(20 * time.Second).Unix()},
}, },
}, },
retry: map[string][]base.Z{ retry: map[string][]base.Z{
@@ -132,9 +128,9 @@ func TestRecoverer(t *testing.T) {
"default": {t2}, "default": {t2},
"critical": {t3}, "critical": {t3},
}, },
wantDeadlines: map[string][]base.Z{ wantLease: map[string][]base.Z{
"default": {{Message: t2, Score: fiveMinutesFromNow.Unix()}}, "default": {{Message: t2, Score: now.Add(20 * time.Second).Unix()}},
"critical": {{Message: t3, Score: oneHourFromNow.Unix()}}, "critical": {{Message: t3, Score: now.Add(20 * time.Second).Unix()}},
}, },
wantRetry: map[string][]*base.TaskMessage{ wantRetry: map[string][]*base.TaskMessage{
"default": {t1}, "default": {t1},
@@ -147,17 +143,17 @@ func TestRecoverer(t *testing.T) {
}, },
{ {
desc: "with multiple expired active tasks", desc: "with multiple expired active tasks",
inProgress: map[string][]*base.TaskMessage{ active: map[string][]*base.TaskMessage{
"default": {t1, t2}, "default": {t1, t2},
"critical": {t3}, "critical": {t3},
}, },
deadlines: map[string][]base.Z{ lease: map[string][]base.Z{
"default": { "default": {
{Message: t1, Score: oneHourAgo.Unix()}, {Message: t1, Score: now.Add(-1 * time.Minute).Unix()},
{Message: t2, Score: oneHourFromNow.Unix()}, {Message: t2, Score: now.Add(10 * time.Second).Unix()},
}, },
"critical": { "critical": {
{Message: t3, Score: fiveMinutesAgo.Unix()}, {Message: t3, Score: now.Add(-1 * time.Minute).Unix()},
}, },
}, },
retry: map[string][]base.Z{ retry: map[string][]base.Z{
@@ -172,8 +168,8 @@ func TestRecoverer(t *testing.T) {
"default": {t2}, "default": {t2},
"critical": {}, "critical": {},
}, },
wantDeadlines: map[string][]base.Z{ wantLease: map[string][]base.Z{
"default": {{Message: t2, Score: oneHourFromNow.Unix()}}, "default": {{Message: t2, Score: now.Add(10 * time.Second).Unix()}},
}, },
wantRetry: map[string][]*base.TaskMessage{ wantRetry: map[string][]*base.TaskMessage{
"default": {t1}, "default": {t1},
@@ -186,11 +182,11 @@ func TestRecoverer(t *testing.T) {
}, },
{ {
desc: "with empty active queue", desc: "with empty active queue",
inProgress: map[string][]*base.TaskMessage{ active: map[string][]*base.TaskMessage{
"default": {}, "default": {},
"critical": {}, "critical": {},
}, },
deadlines: map[string][]base.Z{ lease: map[string][]base.Z{
"default": {}, "default": {},
"critical": {}, "critical": {},
}, },
@@ -206,7 +202,7 @@ func TestRecoverer(t *testing.T) {
"default": {}, "default": {},
"critical": {}, "critical": {},
}, },
wantDeadlines: map[string][]base.Z{ wantLease: map[string][]base.Z{
"default": {}, "default": {},
"critical": {}, "critical": {},
}, },
@@ -223,8 +219,8 @@ func TestRecoverer(t *testing.T) {
for _, tc := range tests { for _, tc := range tests {
h.FlushDB(t, r) h.FlushDB(t, r)
h.SeedAllActiveQueues(t, r, tc.inProgress) h.SeedAllActiveQueues(t, r, tc.active)
h.SeedAllDeadlines(t, r, tc.deadlines) h.SeedAllLease(t, r, tc.lease)
h.SeedAllRetryQueues(t, r, tc.retry) h.SeedAllRetryQueues(t, r, tc.retry)
h.SeedAllArchivedQueues(t, r, tc.archived) h.SeedAllArchivedQueues(t, r, tc.archived)
@@ -249,10 +245,10 @@ func TestRecoverer(t *testing.T) {
t.Errorf("%s; mismatch found in %q; (-want,+got)\n%s", tc.desc, base.ActiveKey(qname), diff) t.Errorf("%s; mismatch found in %q; (-want,+got)\n%s", tc.desc, base.ActiveKey(qname), diff)
} }
} }
for qname, want := range tc.wantDeadlines { for qname, want := range tc.wantLease {
gotDeadlines := h.GetDeadlinesEntries(t, r, qname) gotLease := h.GetLeaseEntries(t, r, qname)
if diff := cmp.Diff(want, gotDeadlines, h.SortZSetEntryOpt); diff != "" { if diff := cmp.Diff(want, gotLease, h.SortZSetEntryOpt); diff != "" {
t.Errorf("%s; mismatch found in %q; (-want,+got)\n%s", tc.desc, base.DeadlinesKey(qname), diff) t.Errorf("%s; mismatch found in %q; (-want,+got)\n%s", tc.desc, base.LeaseKey(qname), diff)
} }
} }
cmpOpt := h.EquateInt64Approx(2) // allow up to two-second difference in `LastFailedAt` cmpOpt := h.EquateInt64Approx(2) // allow up to two-second difference in `LastFailedAt`
@@ -260,7 +256,7 @@ func TestRecoverer(t *testing.T) {
gotRetry := h.GetRetryMessages(t, r, qname) gotRetry := h.GetRetryMessages(t, r, qname)
var wantRetry []*base.TaskMessage // Note: construct message here since `LastFailedAt` is relative to each test run var wantRetry []*base.TaskMessage // Note: construct message here since `LastFailedAt` is relative to each test run
for _, msg := range msgs { for _, msg := range msgs {
wantRetry = append(wantRetry, h.TaskMessageAfterRetry(*msg, "context deadline exceeded", runTime)) wantRetry = append(wantRetry, h.TaskMessageAfterRetry(*msg, ErrLeaseExpired.Error(), runTime))
} }
if diff := cmp.Diff(wantRetry, gotRetry, h.SortMsgOpt, cmpOpt); diff != "" { if diff := cmp.Diff(wantRetry, gotRetry, h.SortMsgOpt, cmpOpt); diff != "" {
t.Errorf("%s; mismatch found in %q: (-want, +got)\n%s", tc.desc, base.RetryKey(qname), diff) t.Errorf("%s; mismatch found in %q: (-want, +got)\n%s", tc.desc, base.RetryKey(qname), diff)
@@ -270,7 +266,7 @@ func TestRecoverer(t *testing.T) {
gotArchived := h.GetArchivedMessages(t, r, qname) gotArchived := h.GetArchivedMessages(t, r, qname)
var wantArchived []*base.TaskMessage var wantArchived []*base.TaskMessage
for _, msg := range msgs { for _, msg := range msgs {
wantArchived = append(wantArchived, h.TaskMessageWithError(*msg, "context deadline exceeded", runTime)) wantArchived = append(wantArchived, h.TaskMessageWithError(*msg, ErrLeaseExpired.Error(), runTime))
} }
if diff := cmp.Diff(wantArchived, gotArchived, h.SortMsgOpt, cmpOpt); diff != "" { if diff := cmp.Diff(wantArchived, gotArchived, h.SortMsgOpt, cmpOpt); diff != "" {
t.Errorf("%s; mismatch found in %q: (-want, +got)\n%s", tc.desc, base.ArchivedKey(qname), diff) t.Errorf("%s; mismatch found in %q: (-want, +got)\n%s", tc.desc, base.ArchivedKey(qname), diff)

View File

@@ -97,6 +97,12 @@ type Config struct {
// to the number of CPUs usable by the current process. // to the number of CPUs usable by the current process.
Concurrency int Concurrency int
// BaseContext optionally specifies a function that returns the base context for Handler invocations on this server.
//
// If BaseContext is nil, the default is context.Background().
// If this is defined, then it MUST return a non-nil context
BaseContext func() context.Context
// Function to calculate retry delay for a failed task. // Function to calculate retry delay for a failed task.
// //
// By default, it uses exponential backoff algorithm to calculate the delay. // By default, it uses exponential backoff algorithm to calculate the delay.
@@ -341,6 +347,10 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
if !ok { if !ok {
panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r)) panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r))
} }
baseCtxFn := cfg.BaseContext
if baseCtxFn == nil {
baseCtxFn = context.Background
}
n := cfg.Concurrency n := cfg.Concurrency
if n < 1 { if n < 1 {
n = runtime.NumCPU() n = runtime.NumCPU()
@@ -426,6 +436,7 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
logger: logger, logger: logger,
broker: rdb, broker: rdb,
retryDelayFunc: delayFunc, retryDelayFunc: delayFunc,
baseCtxFn: baseCtxFn,
isFailureFunc: isFailureFunc, isFailureFunc: isFailureFunc,
syncCh: syncCh, syncCh: syncCh,
cancelations: cancels, cancelations: cancels,

View File

@@ -5,6 +5,7 @@
package asynq package asynq
import ( import (
"context"
"fmt" "fmt"
"sync" "sync"
"testing" "testing"
@@ -41,7 +42,7 @@ func TestSyncer(t *testing.T) {
m := msg m := msg
syncRequestCh <- &syncRequest{ syncRequestCh <- &syncRequest{
fn: func() error { fn: func() error {
return rdbClient.Done(m) return rdbClient.Done(context.Background(), m)
}, },
deadline: time.Now().Add(5 * time.Minute), deadline: time.Now().Add(5 * time.Minute),
} }

View File

@@ -5,16 +5,11 @@ go 1.13
require ( require (
github.com/fatih/color v1.9.0 github.com/fatih/color v1.9.0
github.com/go-redis/redis/v8 v8.11.4 github.com/go-redis/redis/v8 v8.11.4
github.com/google/uuid v1.3.0 github.com/hibiken/asynq v0.21.0
github.com/hibiken/asynq v0.19.0 github.com/hibiken/asynq/x v0.0.0-20220131170841-349f4c50fb1d
github.com/hibiken/asynq/x v0.0.0-00010101000000-000000000000
github.com/mitchellh/go-homedir v1.1.0 github.com/mitchellh/go-homedir v1.1.0
github.com/prometheus/client_golang v1.11.0 github.com/prometheus/client_golang v1.11.0
github.com/spf13/afero v1.1.2 // indirect
github.com/spf13/cobra v1.1.1 github.com/spf13/cobra v1.1.1
github.com/spf13/viper v1.7.0 github.com/spf13/viper v1.7.0
) )
replace (
github.com/hibiken/asynq => ./..
github.com/hibiken/asynq/x => ./../x
)

View File

@@ -24,7 +24,6 @@ github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hC
github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY=
github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0=
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
@@ -33,7 +32,6 @@ github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJm
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY=
github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE=
github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
@@ -67,7 +65,6 @@ github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vb
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
github.com/go-redis/redis/v8 v8.11.2 h1:WqlSpAwz8mxDSMCvbyz1Mkiqe0LE5OY4j3lgkvu1Ts0=
github.com/go-redis/redis/v8 v8.11.2/go.mod h1:DLomh7y2e3ggQXQLd1YgmvIfecPJoFl7WU5SOQ/r06M= github.com/go-redis/redis/v8 v8.11.2/go.mod h1:DLomh7y2e3ggQXQLd1YgmvIfecPJoFl7WU5SOQ/r06M=
github.com/go-redis/redis/v8 v8.11.4 h1:kHoYkfZP6+pe04aFTnhDH6GDROa5yJdHJVNxV3F46Tg= github.com/go-redis/redis/v8 v8.11.4 h1:kHoYkfZP6+pe04aFTnhDH6GDROa5yJdHJVNxV3F46Tg=
github.com/go-redis/redis/v8 v8.11.4/go.mod h1:2Z2wHZXdQpCDXEGzqMockDpNyYvi2l4Pxt6RJr792+w= github.com/go-redis/redis/v8 v8.11.4/go.mod h1:2Z2wHZXdQpCDXEGzqMockDpNyYvi2l4Pxt6RJr792+w=
@@ -89,7 +86,6 @@ github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrU
github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0=
github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
@@ -111,7 +107,6 @@ github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXi
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/uuid v1.2.0 h1:qJYtXnJRWmpe7m/3XlyhrsLrEURqHRM2kxzoxXqyUDs=
github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
@@ -144,6 +139,11 @@ github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO
github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
github.com/hibiken/asynq v0.19.0/go.mod h1:tyc63ojaW8SJ5SBm8mvI4DDONsguP5HE85EEl4Qr5Ig=
github.com/hibiken/asynq v0.21.0 h1:uH9XogJhjq/S39E0/DEPWLZQ6hHJ73UiblZTe4RzHwA=
github.com/hibiken/asynq v0.21.0/go.mod h1:tyc63ojaW8SJ5SBm8mvI4DDONsguP5HE85EEl4Qr5Ig=
github.com/hibiken/asynq/x v0.0.0-20220131170841-349f4c50fb1d h1:Er+U+9PmnyRHRDQjSjRQ24HoWvOY7w9Pk7bUPYM3Ags=
github.com/hibiken/asynq/x v0.0.0-20220131170841-349f4c50fb1d/go.mod h1:VmxwMfMKyb6gyv8xG0oOBMXIhquWKPx+zPtbVBd2Q1s=
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM=
github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
@@ -195,19 +195,19 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78=
github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE=
github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU=
github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
github.com/onsi/ginkgo v1.15.0 h1:1V1NfVQR87RtWAgp1lv9JZJ5Jap+XFGKPi00andXGi4=
github.com/onsi/ginkgo v1.15.0/go.mod h1:hF8qUzuuC8DJGygJH3726JnCZX4MYbRB8yFfISqnKUg= github.com/onsi/ginkgo v1.15.0/go.mod h1:hF8qUzuuC8DJGygJH3726JnCZX4MYbRB8yFfISqnKUg=
github.com/onsi/ginkgo v1.16.4 h1:29JGrr5oVBm5ulCWet69zQkzWipVXIol6ygQUe/EzNc=
github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0=
github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
github.com/onsi/gomega v1.10.5 h1:7n6FEkpFmfCoo2t+YYqXH0evK+a9ICQz0xcAy9dYcaQ=
github.com/onsi/gomega v1.10.5/go.mod h1:gza4q3jKQJijlu05nKWRCW/GavJumGt8aNRxWg7mt48= github.com/onsi/gomega v1.10.5/go.mod h1:gza4q3jKQJijlu05nKWRCW/GavJumGt8aNRxWg7mt48=
github.com/onsi/gomega v1.16.0 h1:6gjqkI8iiRHMvdccRJM8rVKjCWk6ZIm6FTm3ddIe4/c=
github.com/onsi/gomega v1.16.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= github.com/onsi/gomega v1.16.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY=
github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc= github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc=
@@ -219,7 +219,6 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI=
github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
github.com/prometheus/client_golang v0.9.3 h1:9iH4JKXLzFbOAdtqv/a+j8aewx2Y8lAjAydhbaScPF8=
github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso=
github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M=
@@ -227,19 +226,16 @@ github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf
github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0=
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 h1:gQz4mCbXsO+nc9n1hCxHcGA3Zx3Eo+UHZoInFGUIXNM=
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M=
github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
github.com/prometheus/common v0.4.0 h1:7etb9YClo3a6HjLzfl6rIQaU+FDfi0VSX39io3aQ+DM=
github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo=
github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ=
github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc=
github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084 h1:sofwID9zm4tzrgykg80hfFph1mryUeLRsUfoocVVmRY=
github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
@@ -343,8 +339,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL
golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb h1:eBmm0M9fYhWpKZLjQUUKka/LtIxf46G4fxeEz5KJr9U=
golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20210428140749-89ef3d95e781 h1:DzZ89McO9/gWPsQXS/FVKAlG02ZjaQ6AlZRBimEYOd0=
golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk=
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
@@ -383,7 +379,6 @@ golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210112080510-489259a85091 h1:DMyOG0U+gKfu8JZzg2UQe9MeaC1X+xQWlAKcRnjxjCw=
golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -393,8 +388,8 @@ golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9sn
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k=
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M=
golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ=
@@ -456,7 +451,6 @@ google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzi
google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.25.0 h1:Ejskq+SyPohKW+1uil0JJMtmHCgJPJ/qWTxr8qp+R4c=
google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c=
google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk=
@@ -479,8 +473,8 @@ gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU=
gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=

View File

@@ -5,8 +5,6 @@ go 1.16
require ( require (
github.com/go-redis/redis/v8 v8.11.4 github.com/go-redis/redis/v8 v8.11.4
github.com/google/uuid v1.3.0 github.com/google/uuid v1.3.0
github.com/hibiken/asynq v0.19.0 github.com/hibiken/asynq v0.21.0
github.com/prometheus/client_golang v1.11.0 github.com/prometheus/client_golang v1.11.0
) )
replace github.com/hibiken/asynq => ./..

View File

@@ -66,6 +66,8 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/
github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/hibiken/asynq v0.21.0 h1:uH9XogJhjq/S39E0/DEPWLZQ6hHJ73UiblZTe4RzHwA=
github.com/hibiken/asynq v0.21.0/go.mod h1:tyc63ojaW8SJ5SBm8mvI4DDONsguP5HE85EEl4Qr5Ig=
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=