mirror of
https://github.com/hibiken/asynq.git
synced 2025-10-21 21:46:12 +08:00
Compare commits
93 Commits
Author | SHA1 | Date | |
---|---|---|---|
|
245d4fe663 | ||
|
94719e325c | ||
|
8b2a787759 | ||
|
451be7e50f | ||
|
578321f226 | ||
|
2c783566f3 | ||
|
39718f8bea | ||
|
829f64fd38 | ||
|
a369443955 | ||
|
de139cc18e | ||
|
74db013ab9 | ||
|
725105ca03 | ||
|
d8f31e45f1 | ||
|
9023cbf4be | ||
|
9279c09125 | ||
|
bc27126670 | ||
|
0cfa7f47ba | ||
|
8a4fb71dd5 | ||
|
7fb5b25944 | ||
|
71bd8f0535 | ||
|
4c8432e0ce | ||
|
e939b5d166 | ||
|
1acd62c760 | ||
|
0149396bae | ||
|
45ed560708 | ||
|
01eeb8756e | ||
|
47af17cfb4 | ||
|
eb064c2bab | ||
|
652939dd3a | ||
|
efe3c74037 | ||
|
74d2eea4e0 | ||
|
60a4dc1401 | ||
|
4b716780ef | ||
|
e63f41fb24 | ||
|
1c388baf06 | ||
|
47a66231b3 | ||
|
3551d3334c | ||
|
8b16ede8bc | ||
|
c8658a53e6 | ||
|
562506c7ba | ||
|
888b5590fb | ||
|
196db64d4d | ||
|
4b35eb0e1a | ||
|
b29fe58434 | ||
|
7849b1114c | ||
|
99c00bffeb | ||
|
4542b52da8 | ||
|
d841dc2f8d | ||
|
ab28234767 | ||
|
eb27b0fe1e | ||
|
088be63ee4 | ||
|
ed69667e86 | ||
|
4e8885276c | ||
|
401f7fb4fe | ||
|
61854ea1dc | ||
|
f17c157b0f | ||
|
8b582899ad | ||
|
e3d2939a4c | ||
|
2ce71e83b0 | ||
|
1608366032 | ||
|
3f4f0c1daa | ||
|
f94a65dc9f | ||
|
04d7c8c38c | ||
|
c04fd41653 | ||
|
7e5efb0e30 | ||
|
cabf8d3627 | ||
|
a19909f5f4 | ||
|
cea5110d15 | ||
|
9b63e23274 | ||
|
de25201d9f | ||
|
ec560afb01 | ||
|
d4006894ad | ||
|
59927509d8 | ||
|
8211167de2 | ||
|
d7169cd445 | ||
|
dfae8638e1 | ||
|
b9943de2ab | ||
|
871474f220 | ||
|
87dc392c7f | ||
|
dabcb120d5 | ||
|
bc2f1986d7 | ||
|
b8cb579407 | ||
|
bca624792c | ||
|
d865d89900 | ||
|
852af7abd1 | ||
|
5490d2c625 | ||
|
ebd7a32c0f | ||
|
55d0610a03 | ||
|
ab8a4f5b1e | ||
|
d7ceb0c090 | ||
|
8bd70c6f84 | ||
|
10ab4e3745 | ||
|
349f4c50fb |
15
.github/workflows/build.yml
vendored
15
.github/workflows/build.yml
vendored
@@ -7,7 +7,7 @@ jobs:
|
||||
strategy:
|
||||
matrix:
|
||||
os: [ubuntu-latest]
|
||||
go-version: [1.14.x, 1.15.x, 1.16.x, 1.17.x]
|
||||
go-version: [1.14.x, 1.15.x, 1.16.x, 1.17.x, 1.18.x]
|
||||
runs-on: ${{ matrix.os }}
|
||||
services:
|
||||
redis:
|
||||
@@ -22,12 +22,21 @@ jobs:
|
||||
with:
|
||||
go-version: ${{ matrix.go-version }}
|
||||
|
||||
- name: Build
|
||||
- name: Build core module
|
||||
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 ./...
|
||||
|
||||
- name: Test x module
|
||||
run: cd x && go test -race -v ./... && cd ..
|
||||
|
||||
- name: Benchmark Test
|
||||
run: go test -run=^$ -bench=. -loglevel=debug ./...
|
||||
|
||||
|
40
CHANGELOG.md
40
CHANGELOG.md
@@ -7,6 +7,40 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
|
||||
|
||||
## [Unreleased]
|
||||
|
||||
## [0.23.0] - 2022-03-11
|
||||
|
||||
### Added
|
||||
|
||||
- `Group` option is introduced to enqueue task in a group.
|
||||
- `GroupAggregator` and related types are introduced for task aggregation feature.
|
||||
- `GroupGracePeriod`, `GroupMaxSize`, `GroupMaxDelay`, and `GroupAggregator` fields are added to `Config`.
|
||||
- `Inspector` has new methods related to "aggregating tasks".
|
||||
- `Group` field is added to `TaskInfo`.
|
||||
- (CLI): `group ls` command is added
|
||||
- (CLI): `task ls` supports listing aggregating tasks via `--state=aggregating --group=<GROUP>` flags
|
||||
- Enable rediss url parsing support
|
||||
|
||||
### Fixed
|
||||
|
||||
- Fixed overflow issue with 32-bit systems (For details, see https://github.com/hibiken/asynq/pull/426)
|
||||
|
||||
## [0.22.1] - 2022-02-20
|
||||
|
||||
### Fixed
|
||||
|
||||
- Fixed Redis version compatibility: Keep support for redis v4.0+
|
||||
|
||||
## [0.22.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
|
||||
|
||||
### Added
|
||||
@@ -247,9 +281,9 @@ Use `ProcessIn` or `ProcessAt` option to schedule a task instead of `EnqueueIn`
|
||||
|
||||
#### `Inspector`
|
||||
|
||||
All Inspector methods are scoped to a queue, and the methods take `qname (string)` as the first argument.
|
||||
`EnqueuedTask` is renamed to `PendingTask` and its corresponding methods.
|
||||
`InProgressTask` is renamed to `ActiveTask` and its corresponding methods.
|
||||
All Inspector methods are scoped to a queue, and the methods take `qname (string)` as the first argument.
|
||||
`EnqueuedTask` is renamed to `PendingTask` and its corresponding methods.
|
||||
`InProgressTask` is renamed to `ActiveTask` and its corresponding methods.
|
||||
Command "Enqueue" is replaced by the verb "Run" (e.g. `EnqueueAllScheduledTasks` --> `RunAllScheduledTasks`)
|
||||
|
||||
#### `CLI`
|
||||
|
176
aggregator.go
Normal file
176
aggregator.go
Normal file
@@ -0,0 +1,176 @@
|
||||
// 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 asynq
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/log"
|
||||
)
|
||||
|
||||
// An aggregator is responsible for checking groups and aggregate into one task
|
||||
// if any of the grouping condition is met.
|
||||
type aggregator struct {
|
||||
logger *log.Logger
|
||||
broker base.Broker
|
||||
client *Client
|
||||
|
||||
// channel to communicate back to the long running "aggregator" goroutine.
|
||||
done chan struct{}
|
||||
|
||||
// list of queue names to check and aggregate.
|
||||
queues []string
|
||||
|
||||
// Group configurations
|
||||
gracePeriod time.Duration
|
||||
maxDelay time.Duration
|
||||
maxSize int
|
||||
|
||||
// User provided group aggregator.
|
||||
ga GroupAggregator
|
||||
|
||||
// interval used to check for aggregation
|
||||
interval time.Duration
|
||||
|
||||
// sema is a counting semaphore to ensure the number of active aggregating function
|
||||
// does not exceed the limit.
|
||||
sema chan struct{}
|
||||
}
|
||||
|
||||
type aggregatorParams struct {
|
||||
logger *log.Logger
|
||||
broker base.Broker
|
||||
queues []string
|
||||
gracePeriod time.Duration
|
||||
maxDelay time.Duration
|
||||
maxSize int
|
||||
groupAggregator GroupAggregator
|
||||
}
|
||||
|
||||
const (
|
||||
// Maximum number of aggregation checks in flight concurrently.
|
||||
maxConcurrentAggregationChecks = 3
|
||||
|
||||
// Default interval used for aggregation checks. If the provided gracePeriod is less than
|
||||
// the default, use the gracePeriod.
|
||||
defaultAggregationCheckInterval = 7 * time.Second
|
||||
)
|
||||
|
||||
func newAggregator(params aggregatorParams) *aggregator {
|
||||
interval := defaultAggregationCheckInterval
|
||||
if params.gracePeriod < interval {
|
||||
interval = params.gracePeriod
|
||||
}
|
||||
return &aggregator{
|
||||
logger: params.logger,
|
||||
broker: params.broker,
|
||||
client: &Client{broker: params.broker},
|
||||
done: make(chan struct{}),
|
||||
queues: params.queues,
|
||||
gracePeriod: params.gracePeriod,
|
||||
maxDelay: params.maxDelay,
|
||||
maxSize: params.maxSize,
|
||||
ga: params.groupAggregator,
|
||||
sema: make(chan struct{}, maxConcurrentAggregationChecks),
|
||||
interval: interval,
|
||||
}
|
||||
}
|
||||
|
||||
func (a *aggregator) shutdown() {
|
||||
if a.ga == nil {
|
||||
return
|
||||
}
|
||||
a.logger.Debug("Aggregator shutting down...")
|
||||
// Signal the aggregator goroutine to stop.
|
||||
a.done <- struct{}{}
|
||||
}
|
||||
|
||||
func (a *aggregator) start(wg *sync.WaitGroup) {
|
||||
if a.ga == nil {
|
||||
return
|
||||
}
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
ticker := time.NewTicker(a.interval)
|
||||
for {
|
||||
select {
|
||||
case <-a.done:
|
||||
a.logger.Debug("Waiting for all aggregation checks to finish...")
|
||||
// block until all aggregation checks released the token
|
||||
for i := 0; i < cap(a.sema); i++ {
|
||||
a.sema <- struct{}{}
|
||||
}
|
||||
a.logger.Debug("Aggregator done")
|
||||
ticker.Stop()
|
||||
return
|
||||
case t := <-ticker.C:
|
||||
a.exec(t)
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (a *aggregator) exec(t time.Time) {
|
||||
select {
|
||||
case a.sema <- struct{}{}: // acquire token
|
||||
go a.aggregate(t)
|
||||
default:
|
||||
// If the semaphore blocks, then we are currently running max number of
|
||||
// aggregation checks. Skip this round and log warning.
|
||||
a.logger.Warnf("Max number of aggregation checks in flight. Skipping")
|
||||
}
|
||||
}
|
||||
|
||||
func (a *aggregator) aggregate(t time.Time) {
|
||||
defer func() { <-a.sema /* release token */ }()
|
||||
for _, qname := range a.queues {
|
||||
groups, err := a.broker.ListGroups(qname)
|
||||
if err != nil {
|
||||
a.logger.Errorf("Failed to list groups in queue: %q", qname)
|
||||
continue
|
||||
}
|
||||
for _, gname := range groups {
|
||||
aggregationSetID, err := a.broker.AggregationCheck(
|
||||
qname, gname, t, a.gracePeriod, a.maxDelay, a.maxSize)
|
||||
if err != nil {
|
||||
a.logger.Errorf("Failed to run aggregation check: queue=%q group=%q", qname, gname)
|
||||
continue
|
||||
}
|
||||
if aggregationSetID == "" {
|
||||
a.logger.Debugf("No aggregation needed at this time: queue=%q group=%q", qname, gname)
|
||||
continue
|
||||
}
|
||||
|
||||
// Aggregate and enqueue.
|
||||
msgs, deadline, err := a.broker.ReadAggregationSet(qname, gname, aggregationSetID)
|
||||
if err != nil {
|
||||
a.logger.Errorf("Failed to read aggregation set: queue=%q, group=%q, setID=%q",
|
||||
qname, gname, aggregationSetID)
|
||||
continue
|
||||
}
|
||||
tasks := make([]*Task, len(msgs))
|
||||
for i, m := range msgs {
|
||||
tasks[i] = NewTask(m.Type, m.Payload)
|
||||
}
|
||||
aggregatedTask := a.ga.Aggregate(gname, tasks)
|
||||
ctx, cancel := context.WithDeadline(context.Background(), deadline)
|
||||
if _, err := a.client.EnqueueContext(ctx, aggregatedTask, Queue(qname)); err != nil {
|
||||
a.logger.Errorf("Failed to enqueue aggregated task (queue=%q, group=%q, setID=%q): %v",
|
||||
qname, gname, aggregationSetID, err)
|
||||
cancel()
|
||||
continue
|
||||
}
|
||||
if err := a.broker.DeleteAggregationSet(ctx, qname, gname, aggregationSetID); err != nil {
|
||||
a.logger.Warnf("Failed to delete aggregation set: queue=%q, group=%q, setID=%q",
|
||||
qname, gname, aggregationSetID)
|
||||
}
|
||||
cancel()
|
||||
}
|
||||
}
|
||||
}
|
165
aggregator_test.go
Normal file
165
aggregator_test.go
Normal file
@@ -0,0 +1,165 @@
|
||||
// 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 asynq
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func TestAggregator(t *testing.T) {
|
||||
r := setup(t)
|
||||
defer r.Close()
|
||||
rdbClient := rdb.NewRDB(r)
|
||||
client := Client{broker: rdbClient}
|
||||
|
||||
tests := []struct {
|
||||
desc string
|
||||
gracePeriod time.Duration
|
||||
maxDelay time.Duration
|
||||
maxSize int
|
||||
aggregateFunc func(gname string, tasks []*Task) *Task
|
||||
tasks []*Task // tasks to enqueue
|
||||
enqueueFrequency time.Duration // time between one enqueue event to another
|
||||
waitTime time.Duration // time to wait
|
||||
wantGroups map[string]map[string][]base.Z
|
||||
wantPending map[string][]*base.TaskMessage
|
||||
}{
|
||||
{
|
||||
desc: "group older than the grace period should be aggregated",
|
||||
gracePeriod: 1 * time.Second,
|
||||
maxDelay: 0, // no maxdelay limit
|
||||
maxSize: 0, // no maxsize limit
|
||||
aggregateFunc: func(gname string, tasks []*Task) *Task {
|
||||
return NewTask(gname, nil, MaxRetry(len(tasks))) // use max retry to see how many tasks were aggregated
|
||||
},
|
||||
tasks: []*Task{
|
||||
NewTask("task1", nil, Group("mygroup")),
|
||||
NewTask("task2", nil, Group("mygroup")),
|
||||
NewTask("task3", nil, Group("mygroup")),
|
||||
},
|
||||
enqueueFrequency: 300 * time.Millisecond,
|
||||
waitTime: 3 * time.Second,
|
||||
wantGroups: map[string]map[string][]base.Z{
|
||||
"default": {
|
||||
"mygroup": {},
|
||||
},
|
||||
},
|
||||
wantPending: map[string][]*base.TaskMessage{
|
||||
"default": {
|
||||
h.NewTaskMessageBuilder().SetType("mygroup").SetRetry(3).Build(),
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "group older than the max-delay should be aggregated",
|
||||
gracePeriod: 2 * time.Second,
|
||||
maxDelay: 4 * time.Second,
|
||||
maxSize: 0, // no maxsize limit
|
||||
aggregateFunc: func(gname string, tasks []*Task) *Task {
|
||||
return NewTask(gname, nil, MaxRetry(len(tasks))) // use max retry to see how many tasks were aggregated
|
||||
},
|
||||
tasks: []*Task{
|
||||
NewTask("task1", nil, Group("mygroup")), // time 0
|
||||
NewTask("task2", nil, Group("mygroup")), // time 1s
|
||||
NewTask("task3", nil, Group("mygroup")), // time 2s
|
||||
NewTask("task4", nil, Group("mygroup")), // time 3s
|
||||
},
|
||||
enqueueFrequency: 1 * time.Second,
|
||||
waitTime: 4 * time.Second,
|
||||
wantGroups: map[string]map[string][]base.Z{
|
||||
"default": {
|
||||
"mygroup": {},
|
||||
},
|
||||
},
|
||||
wantPending: map[string][]*base.TaskMessage{
|
||||
"default": {
|
||||
h.NewTaskMessageBuilder().SetType("mygroup").SetRetry(4).Build(),
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "group reached the max-size should be aggregated",
|
||||
gracePeriod: 1 * time.Minute,
|
||||
maxDelay: 0, // no maxdelay limit
|
||||
maxSize: 5,
|
||||
aggregateFunc: func(gname string, tasks []*Task) *Task {
|
||||
return NewTask(gname, nil, MaxRetry(len(tasks))) // use max retry to see how many tasks were aggregated
|
||||
},
|
||||
tasks: []*Task{
|
||||
NewTask("task1", nil, Group("mygroup")),
|
||||
NewTask("task2", nil, Group("mygroup")),
|
||||
NewTask("task3", nil, Group("mygroup")),
|
||||
NewTask("task4", nil, Group("mygroup")),
|
||||
NewTask("task5", nil, Group("mygroup")),
|
||||
},
|
||||
enqueueFrequency: 300 * time.Millisecond,
|
||||
waitTime: defaultAggregationCheckInterval * 2,
|
||||
wantGroups: map[string]map[string][]base.Z{
|
||||
"default": {
|
||||
"mygroup": {},
|
||||
},
|
||||
},
|
||||
wantPending: map[string][]*base.TaskMessage{
|
||||
"default": {
|
||||
h.NewTaskMessageBuilder().SetType("mygroup").SetRetry(5).Build(),
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
h.FlushDB(t, r)
|
||||
|
||||
aggregator := newAggregator(aggregatorParams{
|
||||
logger: testLogger,
|
||||
broker: rdbClient,
|
||||
queues: []string{"default"},
|
||||
gracePeriod: tc.gracePeriod,
|
||||
maxDelay: tc.maxDelay,
|
||||
maxSize: tc.maxSize,
|
||||
groupAggregator: GroupAggregatorFunc(tc.aggregateFunc),
|
||||
})
|
||||
|
||||
var wg sync.WaitGroup
|
||||
aggregator.start(&wg)
|
||||
|
||||
for _, task := range tc.tasks {
|
||||
if _, err := client.Enqueue(task); err != nil {
|
||||
t.Errorf("%s: Client Enqueue failed: %v", tc.desc, err)
|
||||
aggregator.shutdown()
|
||||
wg.Wait()
|
||||
continue
|
||||
}
|
||||
time.Sleep(tc.enqueueFrequency)
|
||||
}
|
||||
|
||||
time.Sleep(tc.waitTime)
|
||||
|
||||
for qname, groups := range tc.wantGroups {
|
||||
for gname, want := range groups {
|
||||
gotGroup := h.GetGroupEntries(t, r, qname, gname)
|
||||
if diff := cmp.Diff(want, gotGroup, h.SortZSetEntryOpt); diff != "" {
|
||||
t.Errorf("%s: mismatch found in %q; (-want,+got)\n%s", tc.desc, base.GroupKey(qname, gname), diff)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for qname, want := range tc.wantPending {
|
||||
gotPending := h.GetPendingMessages(t, r, qname)
|
||||
if diff := cmp.Diff(want, gotPending, h.SortMsgOpt, h.IgnoreIDOpt); diff != "" {
|
||||
t.Errorf("%s: mismatch found in %q; (-want,+got)\n%s", tc.desc, base.PendingKey(qname), diff)
|
||||
}
|
||||
}
|
||||
aggregator.shutdown()
|
||||
wg.Wait()
|
||||
}
|
||||
}
|
47
asynq.go
47
asynq.go
@@ -8,6 +8,7 @@ import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"strconv"
|
||||
"strings"
|
||||
@@ -97,10 +98,26 @@ type TaskInfo struct {
|
||||
// Deadline is the deadline for the task, zero value if not specified.
|
||||
Deadline time.Time
|
||||
|
||||
// Group is the name of the group in which the task belongs.
|
||||
//
|
||||
// Tasks in the same queue can be grouped together by Group name and will be aggregated into one task
|
||||
// by a Server processing the queue.
|
||||
//
|
||||
// Empty string (default) indicates task does not belong to any groups, and no aggregation will be applied to the task.
|
||||
Group string
|
||||
|
||||
// NextProcessAt is the time the task is scheduled to be processed,
|
||||
// zero if not applicable.
|
||||
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 time.Duration
|
||||
|
||||
@@ -131,6 +148,7 @@ func newTaskInfo(msg *base.TaskMessage, state base.TaskState, nextProcessAt time
|
||||
MaxRetry: msg.Retry,
|
||||
Retried: msg.Retried,
|
||||
LastErr: msg.ErrorMsg,
|
||||
Group: msg.GroupKey,
|
||||
Timeout: time.Duration(msg.Timeout) * time.Second,
|
||||
Deadline: fromUnixTimeOrZero(msg.Deadline),
|
||||
Retention: time.Duration(msg.Retention) * time.Second,
|
||||
@@ -153,6 +171,8 @@ func newTaskInfo(msg *base.TaskMessage, state base.TaskState, nextProcessAt time
|
||||
info.State = TaskStateArchived
|
||||
case base.TaskStateCompleted:
|
||||
info.State = TaskStateCompleted
|
||||
case base.TaskStateAggregating:
|
||||
info.State = TaskStateAggregating
|
||||
default:
|
||||
panic(fmt.Sprintf("internal error: unknown state: %d", state))
|
||||
}
|
||||
@@ -180,6 +200,9 @@ const (
|
||||
|
||||
// Indicates that the task is processed successfully and retained until the retention TTL expires.
|
||||
TaskStateCompleted
|
||||
|
||||
// Indicates that the task is waiting in a group to be aggreated into one task.
|
||||
TaskStateAggregating
|
||||
)
|
||||
|
||||
func (s TaskState) String() string {
|
||||
@@ -196,6 +219,8 @@ func (s TaskState) String() string {
|
||||
return "archived"
|
||||
case TaskStateCompleted:
|
||||
return "completed"
|
||||
case TaskStateAggregating:
|
||||
return "aggregating"
|
||||
}
|
||||
panic("asynq: unknown task state")
|
||||
}
|
||||
@@ -411,9 +436,10 @@ func (opt RedisClusterClientOpt) MakeRedisClient() interface{} {
|
||||
// ParseRedisURI parses redis uri string and returns RedisConnOpt if uri is valid.
|
||||
// It returns a non-nil error if uri cannot be parsed.
|
||||
//
|
||||
// Three URI schemes are supported, which are redis:, redis-socket:, and redis-sentinel:.
|
||||
// Three URI schemes are supported, which are redis:, rediss:, redis-socket:, and redis-sentinel:.
|
||||
// Supported formats are:
|
||||
// redis://[:password@]host[:port][/dbnumber]
|
||||
// rediss://[:password@]host[:port][/dbnumber]
|
||||
// redis-socket://[:password@]path[?db=dbnumber]
|
||||
// redis-sentinel://[:password@]host1[:port][,host2:[:port]][,hostN:[:port]][?master=masterName]
|
||||
func ParseRedisURI(uri string) (RedisConnOpt, error) {
|
||||
@@ -422,7 +448,7 @@ func ParseRedisURI(uri string) (RedisConnOpt, error) {
|
||||
return nil, fmt.Errorf("asynq: could not parse redis uri: %v", err)
|
||||
}
|
||||
switch u.Scheme {
|
||||
case "redis":
|
||||
case "redis", "rediss":
|
||||
return parseRedisURI(u)
|
||||
case "redis-socket":
|
||||
return parseRedisSocketURI(u)
|
||||
@@ -436,6 +462,8 @@ func ParseRedisURI(uri string) (RedisConnOpt, error) {
|
||||
func parseRedisURI(u *url.URL) (RedisConnOpt, error) {
|
||||
var db int
|
||||
var err error
|
||||
var redisConnOpt RedisClientOpt
|
||||
|
||||
if len(u.Path) > 0 {
|
||||
xs := strings.Split(strings.Trim(u.Path, "/"), "/")
|
||||
db, err = strconv.Atoi(xs[0])
|
||||
@@ -447,7 +475,20 @@ func parseRedisURI(u *url.URL) (RedisConnOpt, error) {
|
||||
if v, ok := u.User.Password(); ok {
|
||||
password = v
|
||||
}
|
||||
return RedisClientOpt{Addr: u.Host, DB: db, Password: password}, nil
|
||||
|
||||
if u.Scheme == "rediss" {
|
||||
h, _, err := net.SplitHostPort(u.Host)
|
||||
if err != nil {
|
||||
h = u.Host
|
||||
}
|
||||
redisConnOpt.TLSConfig = &tls.Config{ServerName: h}
|
||||
}
|
||||
|
||||
redisConnOpt.Addr = u.Host
|
||||
redisConnOpt.Password = password
|
||||
redisConnOpt.DB = db
|
||||
|
||||
return redisConnOpt, nil
|
||||
}
|
||||
|
||||
func parseRedisSocketURI(u *url.URL) (RedisConnOpt, error) {
|
||||
|
@@ -5,6 +5,7 @@
|
||||
package asynq
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
"flag"
|
||||
"sort"
|
||||
"strings"
|
||||
@@ -12,8 +13,9 @@ import (
|
||||
|
||||
"github.com/go-redis/redis/v8"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/hibiken/asynq/internal/log"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
//============================================================================
|
||||
@@ -99,6 +101,10 @@ func TestParseRedisURI(t *testing.T) {
|
||||
"redis://localhost:6379",
|
||||
RedisClientOpt{Addr: "localhost:6379"},
|
||||
},
|
||||
{
|
||||
"rediss://localhost:6379",
|
||||
RedisClientOpt{Addr: "localhost:6379", TLSConfig: &tls.Config{ServerName: "localhost"}},
|
||||
},
|
||||
{
|
||||
"redis://localhost:6379/3",
|
||||
RedisClientOpt{Addr: "localhost:6379", DB: 3},
|
||||
@@ -151,7 +157,7 @@ func TestParseRedisURI(t *testing.T) {
|
||||
continue
|
||||
}
|
||||
|
||||
if diff := cmp.Diff(tc.want, got); diff != "" {
|
||||
if diff := cmp.Diff(tc.want, got, cmpopts.IgnoreUnexported(tls.Config{})); diff != "" {
|
||||
t.Errorf("ParseRedisURI(%q) = %+v, want %+v\n(-want,+got)\n%s", tc.uri, got, tc.want, diff)
|
||||
}
|
||||
}
|
||||
|
@@ -12,7 +12,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
// Creates a new task of type "task<n>" with payload {"data": n}.
|
||||
|
83
client.go
83
client.go
@@ -24,7 +24,7 @@ import (
|
||||
//
|
||||
// Clients are safe for concurrent use by multiple goroutines.
|
||||
type Client struct {
|
||||
rdb *rdb.RDB
|
||||
broker base.Broker
|
||||
}
|
||||
|
||||
// NewClient returns a new Client instance given a redis connection option.
|
||||
@@ -33,7 +33,7 @@ func NewClient(r RedisConnOpt) *Client {
|
||||
if !ok {
|
||||
panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r))
|
||||
}
|
||||
return &Client{rdb: rdb.NewRDB(c)}
|
||||
return &Client{broker: rdb.NewRDB(c)}
|
||||
}
|
||||
|
||||
type OptionType int
|
||||
@@ -48,6 +48,7 @@ const (
|
||||
ProcessInOpt
|
||||
TaskIDOpt
|
||||
RetentionOpt
|
||||
GroupOpt
|
||||
)
|
||||
|
||||
// Option specifies the task processing behavior.
|
||||
@@ -73,6 +74,7 @@ type (
|
||||
processAtOption time.Time
|
||||
processInOption time.Duration
|
||||
retentionOption time.Duration
|
||||
groupOption string
|
||||
)
|
||||
|
||||
// MaxRetry returns an option to specify the max number of times
|
||||
@@ -91,13 +93,13 @@ func (n retryOption) Type() OptionType { return MaxRetryOpt }
|
||||
func (n retryOption) Value() interface{} { return int(n) }
|
||||
|
||||
// Queue returns an option to specify the queue to enqueue the task into.
|
||||
func Queue(qname string) Option {
|
||||
return queueOption(qname)
|
||||
func Queue(name string) Option {
|
||||
return queueOption(name)
|
||||
}
|
||||
|
||||
func (qname queueOption) String() string { return fmt.Sprintf("Queue(%q)", string(qname)) }
|
||||
func (qname queueOption) Type() OptionType { return QueueOpt }
|
||||
func (qname queueOption) Value() interface{} { return string(qname) }
|
||||
func (name queueOption) String() string { return fmt.Sprintf("Queue(%q)", string(name)) }
|
||||
func (name queueOption) Type() OptionType { return QueueOpt }
|
||||
func (name queueOption) Value() interface{} { return string(name) }
|
||||
|
||||
// TaskID returns an option to specify the task ID.
|
||||
func TaskID(id string) Option {
|
||||
@@ -142,7 +144,8 @@ func (t deadlineOption) Value() interface{} { return time.Time(t) }
|
||||
|
||||
// Unique returns an option to enqueue a task only if the given task is unique.
|
||||
// Task enqueued with this option is guaranteed to be unique within the given ttl.
|
||||
// Once the task gets processed successfully or once the TTL has expired, another task with the same uniqueness may be enqueued.
|
||||
// Once the task gets processed successfully or once the TTL has expired,
|
||||
// another task with the same uniqueness may be enqueued.
|
||||
// ErrDuplicateTask error is returned when enqueueing a duplicate task.
|
||||
// TTL duration must be greater than or equal to 1 second.
|
||||
//
|
||||
@@ -193,6 +196,16 @@ func (ttl retentionOption) String() string { return fmt.Sprintf("Retention(%
|
||||
func (ttl retentionOption) Type() OptionType { return RetentionOpt }
|
||||
func (ttl retentionOption) Value() interface{} { return time.Duration(ttl) }
|
||||
|
||||
// Group returns an option to specify the group used for the task.
|
||||
// Tasks in a given queue with the same group will be aggregated into one task before passed to Handler.
|
||||
func Group(name string) Option {
|
||||
return groupOption(name)
|
||||
}
|
||||
|
||||
func (name groupOption) String() string { return fmt.Sprintf("Group(%q)", string(name)) }
|
||||
func (name groupOption) Type() OptionType { return GroupOpt }
|
||||
func (name groupOption) Value() interface{} { return string(name) }
|
||||
|
||||
// ErrDuplicateTask indicates that the given task could not be enqueued since it's a duplicate of another task.
|
||||
//
|
||||
// ErrDuplicateTask error only applies to tasks enqueued with a Unique option.
|
||||
@@ -212,6 +225,7 @@ type option struct {
|
||||
uniqueTTL time.Duration
|
||||
processAt time.Time
|
||||
retention time.Duration
|
||||
group string
|
||||
}
|
||||
|
||||
// composeOptions merges user provided options into the default options
|
||||
@@ -239,8 +253,8 @@ func composeOptions(opts ...Option) (option, error) {
|
||||
res.queue = qname
|
||||
case taskIDOption:
|
||||
id := string(opt)
|
||||
if err := validateTaskID(id); err != nil {
|
||||
return option{}, err
|
||||
if isBlank(id) {
|
||||
return option{}, errors.New("task ID cannot be empty")
|
||||
}
|
||||
res.taskID = id
|
||||
case timeoutOption:
|
||||
@@ -259,6 +273,12 @@ func composeOptions(opts ...Option) (option, error) {
|
||||
res.processAt = time.Now().Add(time.Duration(opt))
|
||||
case retentionOption:
|
||||
res.retention = time.Duration(opt)
|
||||
case groupOption:
|
||||
key := string(opt)
|
||||
if isBlank(key) {
|
||||
return option{}, errors.New("group key cannot be empty")
|
||||
}
|
||||
res.group = key
|
||||
default:
|
||||
// ignore unexpected option
|
||||
}
|
||||
@@ -266,12 +286,9 @@ func composeOptions(opts ...Option) (option, error) {
|
||||
return res, nil
|
||||
}
|
||||
|
||||
// validates user provided task ID string.
|
||||
func validateTaskID(id string) error {
|
||||
if strings.TrimSpace(id) == "" {
|
||||
return errors.New("task ID cannot be empty")
|
||||
}
|
||||
return nil
|
||||
// isBlank returns true if the given s is empty or consist of all whitespaces.
|
||||
func isBlank(s string) bool {
|
||||
return strings.TrimSpace(s) == ""
|
||||
}
|
||||
|
||||
const (
|
||||
@@ -290,7 +307,7 @@ var (
|
||||
|
||||
// Close closes the connection with redis.
|
||||
func (c *Client) Close() error {
|
||||
return c.rdb.Close()
|
||||
return c.broker.Close()
|
||||
}
|
||||
|
||||
// Enqueue enqueues the given task to a queue.
|
||||
@@ -322,6 +339,9 @@ func (c *Client) Enqueue(task *Task, opts ...Option) (*TaskInfo, error) {
|
||||
//
|
||||
// The first argument context applies to the enqueue operation. To specify task timeout and deadline, use Timeout and Deadline option instead.
|
||||
func (c *Client) EnqueueContext(ctx context.Context, task *Task, opts ...Option) (*TaskInfo, error) {
|
||||
if task == nil {
|
||||
return nil, fmt.Errorf("task cannot be nil")
|
||||
}
|
||||
if strings.TrimSpace(task.Type()) == "" {
|
||||
return nil, fmt.Errorf("task typename cannot be empty")
|
||||
}
|
||||
@@ -356,17 +376,23 @@ func (c *Client) EnqueueContext(ctx context.Context, task *Task, opts ...Option)
|
||||
Deadline: deadline.Unix(),
|
||||
Timeout: int64(timeout.Seconds()),
|
||||
UniqueKey: uniqueKey,
|
||||
GroupKey: opt.group,
|
||||
Retention: int64(opt.retention.Seconds()),
|
||||
}
|
||||
now := time.Now()
|
||||
var state base.TaskState
|
||||
if opt.processAt.Before(now) || opt.processAt.Equal(now) {
|
||||
if opt.processAt.After(now) {
|
||||
err = c.schedule(ctx, msg, opt.processAt, opt.uniqueTTL)
|
||||
state = base.TaskStateScheduled
|
||||
} else if opt.group != "" {
|
||||
// Use zero value for processAt since we don't know when the task will be aggregated and processed.
|
||||
opt.processAt = time.Time{}
|
||||
err = c.addToGroup(ctx, msg, opt.group, opt.uniqueTTL)
|
||||
state = base.TaskStateAggregating
|
||||
} else {
|
||||
opt.processAt = now
|
||||
err = c.enqueue(ctx, msg, opt.uniqueTTL)
|
||||
state = base.TaskStatePending
|
||||
} else {
|
||||
err = c.schedule(ctx, msg, opt.processAt, opt.uniqueTTL)
|
||||
state = base.TaskStateScheduled
|
||||
}
|
||||
switch {
|
||||
case errors.Is(err, errors.ErrDuplicateTask):
|
||||
@@ -381,15 +407,22 @@ func (c *Client) EnqueueContext(ctx context.Context, task *Task, opts ...Option)
|
||||
|
||||
func (c *Client) enqueue(ctx context.Context, msg *base.TaskMessage, uniqueTTL time.Duration) error {
|
||||
if uniqueTTL > 0 {
|
||||
return c.rdb.EnqueueUnique(ctx, msg, uniqueTTL)
|
||||
return c.broker.EnqueueUnique(ctx, msg, uniqueTTL)
|
||||
}
|
||||
return c.rdb.Enqueue(ctx, msg)
|
||||
return c.broker.Enqueue(ctx, msg)
|
||||
}
|
||||
|
||||
func (c *Client) schedule(ctx context.Context, msg *base.TaskMessage, t time.Time, uniqueTTL time.Duration) error {
|
||||
if uniqueTTL > 0 {
|
||||
ttl := t.Add(uniqueTTL).Sub(time.Now())
|
||||
return c.rdb.ScheduleUnique(ctx, msg, t, ttl)
|
||||
return c.broker.ScheduleUnique(ctx, msg, t, ttl)
|
||||
}
|
||||
return c.rdb.Schedule(ctx, msg, t)
|
||||
return c.broker.Schedule(ctx, msg, t)
|
||||
}
|
||||
|
||||
func (c *Client) addToGroup(ctx context.Context, msg *base.TaskMessage, group string, uniqueTTL time.Duration) error {
|
||||
if uniqueTTL > 0 {
|
||||
return c.broker.AddToGroupUnique(ctx, msg, group, uniqueTTL)
|
||||
}
|
||||
return c.broker.AddToGroup(ctx, msg, group)
|
||||
}
|
||||
|
159
client_test.go
159
client_test.go
@@ -12,8 +12,8 @@ import (
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func TestClientEnqueueWithProcessAtOption(t *testing.T) {
|
||||
@@ -478,6 +478,158 @@ func TestClientEnqueue(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestClientEnqueueWithGroupOption(t *testing.T) {
|
||||
r := setup(t)
|
||||
client := NewClient(getRedisConnOpt(t))
|
||||
defer client.Close()
|
||||
|
||||
task := NewTask("mytask", []byte("foo"))
|
||||
now := time.Now()
|
||||
|
||||
tests := []struct {
|
||||
desc string
|
||||
task *Task
|
||||
opts []Option
|
||||
wantInfo *TaskInfo
|
||||
wantPending map[string][]*base.TaskMessage
|
||||
wantGroups map[string]map[string][]base.Z // map queue name to a set of groups
|
||||
wantScheduled map[string][]base.Z
|
||||
}{
|
||||
{
|
||||
desc: "With only Group option",
|
||||
task: task,
|
||||
opts: []Option{
|
||||
Group("mygroup"),
|
||||
},
|
||||
wantInfo: &TaskInfo{
|
||||
Queue: "default",
|
||||
Group: "mygroup",
|
||||
Type: task.Type(),
|
||||
Payload: task.Payload(),
|
||||
State: TaskStateAggregating,
|
||||
MaxRetry: defaultMaxRetry,
|
||||
Retried: 0,
|
||||
LastErr: "",
|
||||
LastFailedAt: time.Time{},
|
||||
Timeout: defaultTimeout,
|
||||
Deadline: time.Time{},
|
||||
NextProcessAt: time.Time{},
|
||||
},
|
||||
wantPending: map[string][]*base.TaskMessage{
|
||||
"default": {}, // should not be pending
|
||||
},
|
||||
wantGroups: map[string]map[string][]base.Z{
|
||||
"default": {
|
||||
"mygroup": {
|
||||
{
|
||||
Message: &base.TaskMessage{
|
||||
Type: task.Type(),
|
||||
Payload: task.Payload(),
|
||||
Retry: defaultMaxRetry,
|
||||
Queue: "default",
|
||||
Timeout: int64(defaultTimeout.Seconds()),
|
||||
Deadline: noDeadline.Unix(),
|
||||
GroupKey: "mygroup",
|
||||
},
|
||||
Score: now.Unix(),
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
wantScheduled: map[string][]base.Z{
|
||||
"default": {},
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "With Group and ProcessIn options",
|
||||
task: task,
|
||||
opts: []Option{
|
||||
Group("mygroup"),
|
||||
ProcessIn(30 * time.Minute),
|
||||
},
|
||||
wantInfo: &TaskInfo{
|
||||
Queue: "default",
|
||||
Group: "mygroup",
|
||||
Type: task.Type(),
|
||||
Payload: task.Payload(),
|
||||
State: TaskStateScheduled,
|
||||
MaxRetry: defaultMaxRetry,
|
||||
Retried: 0,
|
||||
LastErr: "",
|
||||
LastFailedAt: time.Time{},
|
||||
Timeout: defaultTimeout,
|
||||
Deadline: time.Time{},
|
||||
NextProcessAt: now.Add(30 * time.Minute),
|
||||
},
|
||||
wantPending: map[string][]*base.TaskMessage{
|
||||
"default": {}, // should not be pending
|
||||
},
|
||||
wantGroups: map[string]map[string][]base.Z{
|
||||
"default": {
|
||||
"mygroup": {}, // should not be added to the group yet
|
||||
},
|
||||
},
|
||||
wantScheduled: map[string][]base.Z{
|
||||
"default": {
|
||||
{
|
||||
Message: &base.TaskMessage{
|
||||
Type: task.Type(),
|
||||
Payload: task.Payload(),
|
||||
Retry: defaultMaxRetry,
|
||||
Queue: "default",
|
||||
Timeout: int64(defaultTimeout.Seconds()),
|
||||
Deadline: noDeadline.Unix(),
|
||||
GroupKey: "mygroup",
|
||||
},
|
||||
Score: now.Add(30 * time.Minute).Unix(),
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
h.FlushDB(t, r) // clean up db before each test case.
|
||||
|
||||
gotInfo, err := client.Enqueue(tc.task, tc.opts...)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
continue
|
||||
}
|
||||
cmpOptions := []cmp.Option{
|
||||
cmpopts.IgnoreFields(TaskInfo{}, "ID"),
|
||||
cmpopts.EquateApproxTime(500 * time.Millisecond),
|
||||
}
|
||||
if diff := cmp.Diff(tc.wantInfo, gotInfo, cmpOptions...); diff != "" {
|
||||
t.Errorf("%s;\nEnqueue(task) returned %v, want %v; (-want,+got)\n%s",
|
||||
tc.desc, gotInfo, tc.wantInfo, diff)
|
||||
}
|
||||
|
||||
for qname, want := range tc.wantPending {
|
||||
got := h.GetPendingMessages(t, r, qname)
|
||||
if diff := cmp.Diff(want, got, h.IgnoreIDOpt, cmpopts.EquateEmpty()); diff != "" {
|
||||
t.Errorf("%s;\nmismatch found in %q; (-want,+got)\n%s", tc.desc, base.PendingKey(qname), diff)
|
||||
}
|
||||
}
|
||||
|
||||
for qname, groups := range tc.wantGroups {
|
||||
for groupKey, want := range groups {
|
||||
got := h.GetGroupEntries(t, r, qname, groupKey)
|
||||
if diff := cmp.Diff(want, got, h.IgnoreIDOpt, cmpopts.EquateEmpty()); diff != "" {
|
||||
t.Errorf("%s;\nmismatch found in %q; (-want,+got)\n%s", tc.desc, base.GroupKey(qname, groupKey), diff)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for qname, want := range tc.wantScheduled {
|
||||
gotScheduled := h.GetScheduledEntries(t, r, qname)
|
||||
if diff := cmp.Diff(want, gotScheduled, h.IgnoreIDOpt, cmpopts.EquateEmpty()); diff != "" {
|
||||
t.Errorf("%s;\nmismatch found in %q; (-want,+got)\n%s", tc.desc, base.ScheduledKey(qname), diff)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestClientEnqueueWithTaskIDOption(t *testing.T) {
|
||||
r := setup(t)
|
||||
client := NewClient(getRedisConnOpt(t))
|
||||
@@ -707,6 +859,11 @@ func TestClientEnqueueError(t *testing.T) {
|
||||
task *Task
|
||||
opts []Option
|
||||
}{
|
||||
{
|
||||
desc: "With nil task",
|
||||
task: nil,
|
||||
opts: []Option{},
|
||||
},
|
||||
{
|
||||
desc: "With empty queue name",
|
||||
task: task,
|
||||
|
@@ -36,7 +36,7 @@ func GetMaxRetry(ctx context.Context) (n int, ok bool) {
|
||||
|
||||
// GetQueueName extracts queue name from a context, if any.
|
||||
//
|
||||
// Return value qname indicates which queue the task was pulled from.
|
||||
func GetQueueName(ctx context.Context) (qname string, ok bool) {
|
||||
// Return value queue indicates which queue the task was pulled from.
|
||||
func GetQueueName(ctx context.Context) (queue string, ok bool) {
|
||||
return asynqcontext.GetQueueName(ctx)
|
||||
}
|
||||
|
@@ -5,6 +5,7 @@
|
||||
package asynq_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
@@ -113,3 +114,20 @@ func ExampleParseRedisURI() {
|
||||
// localhost:6379
|
||||
// 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
|
||||
}
|
||||
|
@@ -56,13 +56,15 @@ func (f *forwarder) start(wg *sync.WaitGroup) {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
timer := time.NewTimer(f.avgInterval)
|
||||
for {
|
||||
select {
|
||||
case <-f.done:
|
||||
f.logger.Debug("Forwarder done")
|
||||
return
|
||||
case <-time.After(f.avgInterval):
|
||||
case <-timer.C:
|
||||
f.exec()
|
||||
timer.Reset(f.avgInterval)
|
||||
}
|
||||
}
|
||||
}()
|
||||
@@ -70,6 +72,6 @@ func (f *forwarder) start(wg *sync.WaitGroup) {
|
||||
|
||||
func (f *forwarder) exec() {
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@@ -10,9 +10,9 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func TestForwarder(t *testing.T) {
|
||||
|
2
go.mod
2
go.mod
@@ -1,6 +1,6 @@
|
||||
module github.com/hibiken/asynq
|
||||
|
||||
go 1.13
|
||||
go 1.14
|
||||
|
||||
require (
|
||||
github.com/go-redis/redis/v8 v8.11.2
|
||||
|
30
heartbeat.go
30
heartbeat.go
@@ -12,6 +12,7 @@ import (
|
||||
"github.com/google/uuid"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/log"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
)
|
||||
|
||||
// heartbeater is responsible for writing process info to redis periodically to
|
||||
@@ -19,6 +20,7 @@ import (
|
||||
type heartbeater struct {
|
||||
logger *log.Logger
|
||||
broker base.Broker
|
||||
clock timeutil.Clock
|
||||
|
||||
// channel to communicate back to the long running "heartbeater" goroutine.
|
||||
done chan struct{}
|
||||
@@ -69,6 +71,7 @@ func newHeartbeater(params heartbeaterParams) *heartbeater {
|
||||
return &heartbeater{
|
||||
logger: params.logger,
|
||||
broker: params.broker,
|
||||
clock: timeutil.NewRealClock(),
|
||||
done: make(chan struct{}),
|
||||
interval: params.interval,
|
||||
|
||||
@@ -100,6 +103,8 @@ type workerInfo struct {
|
||||
started time.Time
|
||||
// deadline the worker has to finish processing the task by.
|
||||
deadline time.Time
|
||||
// lease the worker holds for the task.
|
||||
lease *base.Lease
|
||||
}
|
||||
|
||||
func (h *heartbeater) start(wg *sync.WaitGroup) {
|
||||
@@ -107,7 +112,7 @@ func (h *heartbeater) start(wg *sync.WaitGroup) {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
h.started = time.Now()
|
||||
h.started = h.clock.Now()
|
||||
|
||||
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() {
|
||||
h.state.mu.Lock()
|
||||
srvStatus := h.state.value.String()
|
||||
@@ -152,6 +158,7 @@ func (h *heartbeater) beat() {
|
||||
}
|
||||
|
||||
var ws []*base.WorkerInfo
|
||||
idsByQueue := make(map[string][]string)
|
||||
for id, w := range h.workers {
|
||||
ws = append(ws, &base.WorkerInfo{
|
||||
Host: h.host,
|
||||
@@ -164,11 +171,30 @@ func (h *heartbeater) beat() {
|
||||
Started: w.started,
|
||||
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
|
||||
// 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 {
|
||||
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())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@@ -5,31 +5,154 @@
|
||||
package asynq
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
"github.com/hibiken/asynq/internal/testbroker"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
"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) {
|
||||
r := setup(t)
|
||||
defer r.Close()
|
||||
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 {
|
||||
interval time.Duration
|
||||
desc string
|
||||
|
||||
// Interval between heartbeats.
|
||||
interval time.Duration
|
||||
|
||||
// Server info.
|
||||
host string
|
||||
pid int
|
||||
queues map[string]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)
|
||||
@@ -37,8 +160,15 @@ func TestHeartbeater(t *testing.T) {
|
||||
ignoreFieldOpt := cmpopts.IgnoreFields(base.ServerInfo{}, "ServerID")
|
||||
for _, tc := range tests {
|
||||
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{}
|
||||
startingCh := make(chan *workerInfo)
|
||||
finishedCh := make(chan *base.TaskMessage)
|
||||
hb := newHeartbeater(heartbeaterParams{
|
||||
logger: testLogger,
|
||||
broker: rdbClient,
|
||||
@@ -47,14 +177,19 @@ func TestHeartbeater(t *testing.T) {
|
||||
queues: tc.queues,
|
||||
strictPriority: false,
|
||||
state: srvState,
|
||||
starting: make(chan *workerInfo),
|
||||
finished: make(chan *base.TaskMessage),
|
||||
starting: startingCh,
|
||||
finished: finishedCh,
|
||||
})
|
||||
hb.clock = clock
|
||||
|
||||
// Change host and pid fields for testing purpose.
|
||||
hb.host = tc.host
|
||||
hb.pid = tc.pid
|
||||
|
||||
//===================
|
||||
// Start Phase1
|
||||
//===================
|
||||
|
||||
srvState.mu.Lock()
|
||||
srvState.value = srvStateActive // simulating Server.Start
|
||||
srvState.mu.Unlock()
|
||||
@@ -62,61 +197,113 @@ func TestHeartbeater(t *testing.T) {
|
||||
var wg sync.WaitGroup
|
||||
hb.start(&wg)
|
||||
|
||||
want := &base.ServerInfo{
|
||||
Host: tc.host,
|
||||
PID: tc.pid,
|
||||
Queues: tc.queues,
|
||||
Concurrency: tc.concurrency,
|
||||
Started: time.Now(),
|
||||
Status: "active",
|
||||
// Simulate processor starting to work on tasks.
|
||||
for _, w := range tc.startedWorkers {
|
||||
startingCh <- w
|
||||
}
|
||||
|
||||
// allow for heartbeater to write to redis
|
||||
time.Sleep(tc.interval)
|
||||
// Wait for heartbeater to write to redis
|
||||
time.Sleep(tc.interval * 2)
|
||||
|
||||
ss, err := rdbClient.ListServers()
|
||||
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()
|
||||
continue
|
||||
}
|
||||
|
||||
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()
|
||||
continue
|
||||
}
|
||||
|
||||
if diff := cmp.Diff(want, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" {
|
||||
t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", ss[0], want, diff)
|
||||
wantInfo := &base.ServerInfo{
|
||||
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()
|
||||
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.value = srvStateClosed
|
||||
srvState.mu.Unlock()
|
||||
|
||||
// allow for heartbeater to write to redis
|
||||
// Wait for heartbeater to write to redis
|
||||
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()
|
||||
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()
|
||||
continue
|
||||
}
|
||||
|
||||
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()
|
||||
continue
|
||||
}
|
||||
|
||||
if diff := cmp.Diff(want, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" {
|
||||
t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", ss[0], want, diff)
|
||||
if diff := cmp.Diff(wantInfo, ss[0], timeCmpOpt, ignoreOpt, ignoreFieldOpt); diff != "" {
|
||||
t.Errorf("%s: redis stored process status %+v, want %+v; (-want, +got)\n%s", tc.desc, ss[0], wantInfo, diff)
|
||||
hb.shutdown()
|
||||
continue
|
||||
}
|
||||
|
288
inspector.go
288
inspector.go
@@ -43,7 +43,32 @@ func (i *Inspector) Queues() ([]string, error) {
|
||||
return i.rdb.AllQueues()
|
||||
}
|
||||
|
||||
// QueueInfo represents a state of queues at a certain time.
|
||||
// Groups returns a list of all groups within the given queue.
|
||||
func (i *Inspector) Groups(queue string) ([]*GroupInfo, error) {
|
||||
stats, err := i.rdb.GroupStats(queue)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var res []*GroupInfo
|
||||
for _, s := range stats {
|
||||
res = append(res, &GroupInfo{
|
||||
Group: s.Group,
|
||||
Size: s.Size,
|
||||
})
|
||||
}
|
||||
return res, nil
|
||||
}
|
||||
|
||||
// GroupInfo represents a state of a group at a cerntain time.
|
||||
type GroupInfo struct {
|
||||
// Name of the group.
|
||||
Group string
|
||||
|
||||
// Size is the total number of tasks in the group.
|
||||
Size int
|
||||
}
|
||||
|
||||
// QueueInfo represents a state of a queue at a certain time.
|
||||
type QueueInfo struct {
|
||||
// Name of the queue.
|
||||
Queue string
|
||||
@@ -56,9 +81,12 @@ type QueueInfo struct {
|
||||
Latency time.Duration
|
||||
|
||||
// Size is the total number of tasks in the queue.
|
||||
// The value is the sum of Pending, Active, Scheduled, Retry, and Archived.
|
||||
// The value is the sum of Pending, Active, Scheduled, Retry, Aggregating and Archived.
|
||||
Size int
|
||||
|
||||
// Groups is the total number of groups in the queue.
|
||||
Groups int
|
||||
|
||||
// Number of pending tasks.
|
||||
Pending int
|
||||
// Number of active tasks.
|
||||
@@ -71,6 +99,8 @@ type QueueInfo struct {
|
||||
Archived int
|
||||
// Number of stored completed tasks.
|
||||
Completed int
|
||||
// Number of aggregating tasks.
|
||||
Aggregating int
|
||||
|
||||
// Total number of tasks being processed within the given date (counter resets daily).
|
||||
// The number includes both succeeded and failed tasks.
|
||||
@@ -92,11 +122,11 @@ type QueueInfo struct {
|
||||
}
|
||||
|
||||
// GetQueueInfo returns current information of the given queue.
|
||||
func (i *Inspector) GetQueueInfo(qname string) (*QueueInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) GetQueueInfo(queue string) (*QueueInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
stats, err := i.rdb.CurrentStats(qname)
|
||||
stats, err := i.rdb.CurrentStats(queue)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -105,12 +135,14 @@ func (i *Inspector) GetQueueInfo(qname string) (*QueueInfo, error) {
|
||||
MemoryUsage: stats.MemoryUsage,
|
||||
Latency: stats.Latency,
|
||||
Size: stats.Size,
|
||||
Groups: stats.Groups,
|
||||
Pending: stats.Pending,
|
||||
Active: stats.Active,
|
||||
Scheduled: stats.Scheduled,
|
||||
Retry: stats.Retry,
|
||||
Archived: stats.Archived,
|
||||
Completed: stats.Completed,
|
||||
Aggregating: stats.Aggregating,
|
||||
Processed: stats.Processed,
|
||||
Failed: stats.Failed,
|
||||
ProcessedTotal: stats.ProcessedTotal,
|
||||
@@ -134,11 +166,11 @@ type DailyStats struct {
|
||||
}
|
||||
|
||||
// History returns a list of stats from the last n days.
|
||||
func (i *Inspector) History(qname string, n int) ([]*DailyStats, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) History(queue string, n int) ([]*DailyStats, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
stats, err := i.rdb.HistoricalStats(qname, n)
|
||||
stats, err := i.rdb.HistoricalStats(queue, n)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -175,13 +207,13 @@ var (
|
||||
// If the specified queue does not exist, DeleteQueue returns ErrQueueNotFound.
|
||||
// If force is set to false and the specified queue is not empty, DeleteQueue
|
||||
// returns ErrQueueNotEmpty.
|
||||
func (i *Inspector) DeleteQueue(qname string, force bool) error {
|
||||
err := i.rdb.RemoveQueue(qname, force)
|
||||
func (i *Inspector) DeleteQueue(queue string, force bool) error {
|
||||
err := i.rdb.RemoveQueue(queue, force)
|
||||
if errors.IsQueueNotFound(err) {
|
||||
return fmt.Errorf("%w: queue=%q", ErrQueueNotFound, qname)
|
||||
return fmt.Errorf("%w: queue=%q", ErrQueueNotFound, queue)
|
||||
}
|
||||
if errors.IsQueueNotEmpty(err) {
|
||||
return fmt.Errorf("%w: queue=%q", ErrQueueNotEmpty, qname)
|
||||
return fmt.Errorf("%w: queue=%q", ErrQueueNotEmpty, queue)
|
||||
}
|
||||
return err
|
||||
}
|
||||
@@ -190,8 +222,8 @@ func (i *Inspector) DeleteQueue(qname string, force bool) error {
|
||||
//
|
||||
// Returns an error wrapping ErrQueueNotFound if a queue with the given name doesn't exist.
|
||||
// Returns an error wrapping ErrTaskNotFound if a task with the given id doesn't exist in the queue.
|
||||
func (i *Inspector) GetTaskInfo(qname, id string) (*TaskInfo, error) {
|
||||
info, err := i.rdb.GetTaskInfo(qname, id)
|
||||
func (i *Inspector) GetTaskInfo(queue, id string) (*TaskInfo, error) {
|
||||
info, err := i.rdb.GetTaskInfo(queue, id)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -267,13 +299,13 @@ func Page(n int) ListOption {
|
||||
// ListPendingTasks retrieves pending tasks from the specified queue.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListPendingTasks(qname string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ListPendingTasks(queue string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListPending(qname, pgn)
|
||||
infos, err := i.rdb.ListPending(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -295,13 +327,53 @@ func (i *Inspector) ListPendingTasks(qname string, opts ...ListOption) ([]*TaskI
|
||||
// ListActiveTasks retrieves active tasks from the specified queue.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListActiveTasks(qname string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ListActiveTasks(queue string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListActive(qname, pgn)
|
||||
infos, err := i.rdb.ListActive(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
case err != nil:
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
expired, err := i.rdb.ListLeaseExpired(time.Now(), queue)
|
||||
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
|
||||
for _, i := range infos {
|
||||
t := newTaskInfo(
|
||||
i.Message,
|
||||
i.State,
|
||||
i.NextProcessAt,
|
||||
i.Result,
|
||||
)
|
||||
if _, ok := expiredSet[i.Message.ID]; ok {
|
||||
t.IsOrphaned = true
|
||||
}
|
||||
tasks = append(tasks, t)
|
||||
}
|
||||
return tasks, nil
|
||||
}
|
||||
|
||||
// ListAggregatingTasks retrieves scheduled tasks from the specified group.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListAggregatingTasks(queue, group string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListAggregating(queue, group, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -317,20 +389,20 @@ func (i *Inspector) ListActiveTasks(qname string, opts ...ListOption) ([]*TaskIn
|
||||
i.Result,
|
||||
))
|
||||
}
|
||||
return tasks, err
|
||||
return tasks, nil
|
||||
}
|
||||
|
||||
// ListScheduledTasks retrieves scheduled tasks from the specified queue.
|
||||
// Tasks are sorted by NextProcessAt in ascending order.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListScheduledTasks(qname string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ListScheduledTasks(queue string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListScheduled(qname, pgn)
|
||||
infos, err := i.rdb.ListScheduled(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -353,13 +425,13 @@ func (i *Inspector) ListScheduledTasks(qname string, opts ...ListOption) ([]*Tas
|
||||
// Tasks are sorted by NextProcessAt in ascending order.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListRetryTasks(qname string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ListRetryTasks(queue string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListRetry(qname, pgn)
|
||||
infos, err := i.rdb.ListRetry(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -382,13 +454,13 @@ func (i *Inspector) ListRetryTasks(qname string, opts ...ListOption) ([]*TaskInf
|
||||
// Tasks are sorted by LastFailedAt in descending order.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListArchivedTasks(qname string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ListArchivedTasks(queue string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListArchived(qname, pgn)
|
||||
infos, err := i.rdb.ListArchived(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -411,13 +483,13 @@ func (i *Inspector) ListArchivedTasks(qname string, opts ...ListOption) ([]*Task
|
||||
// Tasks are sorted by expiration time (i.e. CompletedAt + Retention) in descending order.
|
||||
//
|
||||
// By default, it retrieves the first 30 tasks.
|
||||
func (i *Inspector) ListCompletedTasks(qname string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ListCompletedTasks(queue string, opts ...ListOption) ([]*TaskInfo, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
opt := composeListOptions(opts...)
|
||||
pgn := rdb.Pagination{Size: opt.pageSize, Page: opt.pageNum - 1}
|
||||
infos, err := i.rdb.ListCompleted(qname, pgn)
|
||||
infos, err := i.rdb.ListCompleted(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -438,51 +510,61 @@ func (i *Inspector) ListCompletedTasks(qname string, opts ...ListOption) ([]*Tas
|
||||
|
||||
// DeleteAllPendingTasks deletes all pending tasks from the specified queue,
|
||||
// and reports the number tasks deleted.
|
||||
func (i *Inspector) DeleteAllPendingTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) DeleteAllPendingTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.DeleteAllPendingTasks(qname)
|
||||
n, err := i.rdb.DeleteAllPendingTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// DeleteAllScheduledTasks deletes all scheduled tasks from the specified queue,
|
||||
// and reports the number tasks deleted.
|
||||
func (i *Inspector) DeleteAllScheduledTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) DeleteAllScheduledTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.DeleteAllScheduledTasks(qname)
|
||||
n, err := i.rdb.DeleteAllScheduledTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// DeleteAllRetryTasks deletes all retry tasks from the specified queue,
|
||||
// and reports the number tasks deleted.
|
||||
func (i *Inspector) DeleteAllRetryTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) DeleteAllRetryTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.DeleteAllRetryTasks(qname)
|
||||
n, err := i.rdb.DeleteAllRetryTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// DeleteAllArchivedTasks deletes all archived tasks from the specified queue,
|
||||
// and reports the number tasks deleted.
|
||||
func (i *Inspector) DeleteAllArchivedTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) DeleteAllArchivedTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.DeleteAllArchivedTasks(qname)
|
||||
n, err := i.rdb.DeleteAllArchivedTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// DeleteAllCompletedTasks deletes all completed tasks from the specified queue,
|
||||
// and reports the number tasks deleted.
|
||||
func (i *Inspector) DeleteAllCompletedTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) DeleteAllCompletedTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.DeleteAllCompletedTasks(qname)
|
||||
n, err := i.rdb.DeleteAllCompletedTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// DeleteAllAggregatingTasks deletes all tasks from the specified group,
|
||||
// and reports the number of tasks deleted.
|
||||
func (i *Inspector) DeleteAllAggregatingTasks(queue, group string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.DeleteAllAggregatingTasks(queue, group)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
@@ -493,11 +575,11 @@ func (i *Inspector) DeleteAllCompletedTasks(qname string) (int, error) {
|
||||
// If a queue with the given name doesn't exist, it returns an error wrapping ErrQueueNotFound.
|
||||
// If a task with the given id doesn't exist in the queue, it returns an error wrapping ErrTaskNotFound.
|
||||
// If the task is in active state, it returns a non-nil error.
|
||||
func (i *Inspector) DeleteTask(qname, id string) error {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) DeleteTask(queue, id string) error {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
err := i.rdb.DeleteTask(qname, id)
|
||||
err := i.rdb.DeleteTask(queue, id)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -510,33 +592,43 @@ func (i *Inspector) DeleteTask(qname, id string) error {
|
||||
|
||||
}
|
||||
|
||||
// RunAllScheduledTasks transition all scheduled tasks to pending state from the given queue,
|
||||
// and reports the number of tasks transitioned.
|
||||
func (i *Inspector) RunAllScheduledTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
// RunAllScheduledTasks schedules all scheduled tasks from the given queue to run,
|
||||
// and reports the number of tasks scheduled to run.
|
||||
func (i *Inspector) RunAllScheduledTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.RunAllScheduledTasks(qname)
|
||||
n, err := i.rdb.RunAllScheduledTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// RunAllRetryTasks transition all retry tasks to pending state from the given queue,
|
||||
// and reports the number of tasks transitioned.
|
||||
func (i *Inspector) RunAllRetryTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
// RunAllRetryTasks schedules all retry tasks from the given queue to run,
|
||||
// and reports the number of tasks scheduled to run.
|
||||
func (i *Inspector) RunAllRetryTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.RunAllRetryTasks(qname)
|
||||
n, err := i.rdb.RunAllRetryTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// RunAllArchivedTasks transition all archived tasks to pending state from the given queue,
|
||||
// and reports the number of tasks transitioned.
|
||||
func (i *Inspector) RunAllArchivedTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
// RunAllArchivedTasks schedules all archived tasks from the given queue to run,
|
||||
// and reports the number of tasks scheduled to run.
|
||||
func (i *Inspector) RunAllArchivedTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.RunAllArchivedTasks(qname)
|
||||
n, err := i.rdb.RunAllArchivedTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// RunAllAggregatingTasks schedules all tasks from the given grou to run.
|
||||
// and reports the number of tasks scheduled to run.
|
||||
func (i *Inspector) RunAllAggregatingTasks(queue, group string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.RunAllAggregatingTasks(queue, group)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
@@ -547,11 +639,11 @@ func (i *Inspector) RunAllArchivedTasks(qname string) (int, error) {
|
||||
// If a queue with the given name doesn't exist, it returns an error wrapping ErrQueueNotFound.
|
||||
// If a task with the given id doesn't exist in the queue, it returns an error wrapping ErrTaskNotFound.
|
||||
// If the task is in pending or active state, it returns a non-nil error.
|
||||
func (i *Inspector) RunTask(qname, id string) error {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) RunTask(queue, id string) error {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
err := i.rdb.RunTask(qname, id)
|
||||
err := i.rdb.RunTask(queue, id)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -565,31 +657,41 @@ func (i *Inspector) RunTask(qname, id string) error {
|
||||
|
||||
// ArchiveAllPendingTasks archives all pending tasks from the given queue,
|
||||
// and reports the number of tasks archived.
|
||||
func (i *Inspector) ArchiveAllPendingTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ArchiveAllPendingTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.ArchiveAllPendingTasks(qname)
|
||||
n, err := i.rdb.ArchiveAllPendingTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// ArchiveAllScheduledTasks archives all scheduled tasks from the given queue,
|
||||
// and reports the number of tasks archiveed.
|
||||
func (i *Inspector) ArchiveAllScheduledTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ArchiveAllScheduledTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.ArchiveAllScheduledTasks(qname)
|
||||
n, err := i.rdb.ArchiveAllScheduledTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// ArchiveAllRetryTasks archives all retry tasks from the given queue,
|
||||
// and reports the number of tasks archiveed.
|
||||
func (i *Inspector) ArchiveAllRetryTasks(qname string) (int, error) {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ArchiveAllRetryTasks(queue string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.ArchiveAllRetryTasks(qname)
|
||||
n, err := i.rdb.ArchiveAllRetryTasks(queue)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
// ArchiveAllAggregatingTasks archives all tasks from the given group,
|
||||
// and reports the number of tasks archived.
|
||||
func (i *Inspector) ArchiveAllAggregatingTasks(queue, group string) (int, error) {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
n, err := i.rdb.ArchiveAllAggregatingTasks(queue, group)
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
@@ -600,11 +702,11 @@ func (i *Inspector) ArchiveAllRetryTasks(qname string) (int, error) {
|
||||
// If a queue with the given name doesn't exist, it returns an error wrapping ErrQueueNotFound.
|
||||
// If a task with the given id doesn't exist in the queue, it returns an error wrapping ErrTaskNotFound.
|
||||
// If the task is in already archived, it returns a non-nil error.
|
||||
func (i *Inspector) ArchiveTask(qname, id string) error {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) ArchiveTask(queue, id string) error {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return fmt.Errorf("asynq: err")
|
||||
}
|
||||
err := i.rdb.ArchiveTask(qname, id)
|
||||
err := i.rdb.ArchiveTask(queue, id)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -626,20 +728,20 @@ func (i *Inspector) CancelProcessing(id string) error {
|
||||
|
||||
// PauseQueue pauses task processing on the specified queue.
|
||||
// If the queue is already paused, it will return a non-nil error.
|
||||
func (i *Inspector) PauseQueue(qname string) error {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) PauseQueue(queue string) error {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return err
|
||||
}
|
||||
return i.rdb.Pause(qname)
|
||||
return i.rdb.Pause(queue)
|
||||
}
|
||||
|
||||
// UnpauseQueue resumes task processing on the specified queue.
|
||||
// If the queue is not paused, it will return a non-nil error.
|
||||
func (i *Inspector) UnpauseQueue(qname string) error {
|
||||
if err := base.ValidateQueueName(qname); err != nil {
|
||||
func (i *Inspector) UnpauseQueue(queue string) error {
|
||||
if err := base.ValidateQueueName(queue); err != nil {
|
||||
return err
|
||||
}
|
||||
return i.rdb.Unpause(qname)
|
||||
return i.rdb.Unpause(queue)
|
||||
}
|
||||
|
||||
// Servers return a list of running servers' information.
|
||||
@@ -729,8 +831,8 @@ type WorkerInfo struct {
|
||||
}
|
||||
|
||||
// ClusterKeySlot returns an integer identifying the hash slot the given queue hashes to.
|
||||
func (i *Inspector) ClusterKeySlot(qname string) (int64, error) {
|
||||
return i.rdb.ClusterKeySlot(qname)
|
||||
func (i *Inspector) ClusterKeySlot(queue string) (int64, error) {
|
||||
return i.rdb.ClusterKeySlot(queue)
|
||||
}
|
||||
|
||||
// ClusterNode describes a node in redis cluster.
|
||||
@@ -745,8 +847,8 @@ type ClusterNode struct {
|
||||
// ClusterNodes returns a list of nodes the given queue belongs to.
|
||||
//
|
||||
// Only relevant if task queues are stored in redis cluster.
|
||||
func (i *Inspector) ClusterNodes(qname string) ([]*ClusterNode, error) {
|
||||
nodes, err := i.rdb.ClusterNodes(qname)
|
||||
func (i *Inspector) ClusterNodes(queue string) ([]*ClusterNode, error) {
|
||||
nodes, err := i.rdb.ClusterNodes(queue)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -814,11 +916,11 @@ func parseOption(s string) (Option, error) {
|
||||
fn, arg := parseOptionFunc(s), parseOptionArg(s)
|
||||
switch fn {
|
||||
case "Queue":
|
||||
qname, err := strconv.Unquote(arg)
|
||||
queue, err := strconv.Unquote(arg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return Queue(qname), nil
|
||||
return Queue(queue), nil
|
||||
case "MaxRetry":
|
||||
n, err := strconv.Atoi(arg)
|
||||
if err != nil {
|
||||
|
@@ -12,12 +12,13 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/go-redis/redis/v8"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
)
|
||||
|
||||
@@ -745,6 +746,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) {
|
||||
r := setup(t)
|
||||
defer r.Close()
|
||||
@@ -754,10 +761,12 @@ func TestInspectorListActiveTasks(t *testing.T) {
|
||||
m4 := h.NewTaskMessageWithQueue("task4", nil, "custom")
|
||||
|
||||
inspector := NewInspector(getRedisConnOpt(t))
|
||||
now := time.Now()
|
||||
|
||||
tests := []struct {
|
||||
desc string
|
||||
active map[string][]*base.TaskMessage
|
||||
lease map[string][]base.Z
|
||||
qname string
|
||||
want []*TaskInfo
|
||||
}{
|
||||
@@ -767,10 +776,42 @@ func TestInspectorListActiveTasks(t *testing.T) {
|
||||
"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(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",
|
||||
want: []*TaskInfo{
|
||||
newTaskInfo(m1, base.TaskStateActive, time.Time{}, nil),
|
||||
newTaskInfo(m2, base.TaskStateActive, time.Time{}, nil),
|
||||
newOrphanedTaskInfo(m2),
|
||||
},
|
||||
},
|
||||
}
|
||||
@@ -778,6 +819,7 @@ func TestInspectorListActiveTasks(t *testing.T) {
|
||||
for _, tc := range tests {
|
||||
h.FlushDB(t, r)
|
||||
h.SeedAllActiveQueues(t, r, tc.active)
|
||||
h.SeedAllLease(t, r, tc.lease)
|
||||
|
||||
got, err := inspector.ListActiveTasks(tc.qname)
|
||||
if err != nil {
|
||||
@@ -1079,6 +1121,106 @@ func TestInspectorListCompletedTasks(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestInspectorListAggregatingTasks(t *testing.T) {
|
||||
r := setup(t)
|
||||
defer r.Close()
|
||||
now := time.Now()
|
||||
|
||||
m1 := h.NewTaskMessageBuilder().SetType("task1").SetQueue("default").SetGroup("group1").Build()
|
||||
m2 := h.NewTaskMessageBuilder().SetType("task2").SetQueue("default").SetGroup("group1").Build()
|
||||
m3 := h.NewTaskMessageBuilder().SetType("task3").SetQueue("default").SetGroup("group1").Build()
|
||||
m4 := h.NewTaskMessageBuilder().SetType("task4").SetQueue("default").SetGroup("group2").Build()
|
||||
m5 := h.NewTaskMessageBuilder().SetType("task5").SetQueue("custom").SetGroup("group1").Build()
|
||||
|
||||
inspector := NewInspector(getRedisConnOpt(t))
|
||||
|
||||
fxt := struct {
|
||||
tasks []*h.TaskSeedData
|
||||
allQueues []string
|
||||
allGroups map[string][]string
|
||||
groups map[string][]*redis.Z
|
||||
}{
|
||||
tasks: []*h.TaskSeedData{
|
||||
{Msg: m1, State: base.TaskStateAggregating},
|
||||
{Msg: m2, State: base.TaskStateAggregating},
|
||||
{Msg: m3, State: base.TaskStateAggregating},
|
||||
{Msg: m4, State: base.TaskStateAggregating},
|
||||
{Msg: m5, State: base.TaskStateAggregating},
|
||||
},
|
||||
allQueues: []string{"default", "custom"},
|
||||
allGroups: map[string][]string{
|
||||
base.AllGroups("default"): {"group1", "group2"},
|
||||
base.AllGroups("custom"): {"group1"},
|
||||
},
|
||||
groups: map[string][]*redis.Z{
|
||||
base.GroupKey("default", "group1"): {
|
||||
{Member: m1.ID, Score: float64(now.Add(-30 * time.Second).Unix())},
|
||||
{Member: m2.ID, Score: float64(now.Add(-20 * time.Second).Unix())},
|
||||
{Member: m3.ID, Score: float64(now.Add(-10 * time.Second).Unix())},
|
||||
},
|
||||
base.GroupKey("default", "group2"): {
|
||||
{Member: m4.ID, Score: float64(now.Add(-30 * time.Second).Unix())},
|
||||
},
|
||||
base.GroupKey("custom", "group1"): {
|
||||
{Member: m5.ID, Score: float64(now.Add(-30 * time.Second).Unix())},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
desc string
|
||||
qname string
|
||||
gname string
|
||||
want []*TaskInfo
|
||||
}{
|
||||
{
|
||||
desc: "default queue group1",
|
||||
qname: "default",
|
||||
gname: "group1",
|
||||
want: []*TaskInfo{
|
||||
createAggregatingTaskInfo(m1),
|
||||
createAggregatingTaskInfo(m2),
|
||||
createAggregatingTaskInfo(m3),
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "custom queue group1",
|
||||
qname: "custom",
|
||||
gname: "group1",
|
||||
want: []*TaskInfo{
|
||||
createAggregatingTaskInfo(m5),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
h.FlushDB(t, r)
|
||||
h.SeedTasks(t, r, fxt.tasks)
|
||||
h.SeedRedisSet(t, r, base.AllQueues, fxt.allQueues)
|
||||
h.SeedRedisSets(t, r, fxt.allGroups)
|
||||
h.SeedRedisZSets(t, r, fxt.groups)
|
||||
|
||||
t.Run(tc.desc, func(t *testing.T) {
|
||||
got, err := inspector.ListAggregatingTasks(tc.qname, tc.gname)
|
||||
if err != nil {
|
||||
t.Fatalf("ListAggregatingTasks returned error: %v", err)
|
||||
}
|
||||
|
||||
cmpOpts := []cmp.Option{
|
||||
cmpopts.EquateApproxTime(2 * time.Second),
|
||||
cmp.AllowUnexported(TaskInfo{}),
|
||||
}
|
||||
if diff := cmp.Diff(tc.want, got, cmpOpts...); diff != "" {
|
||||
t.Errorf("ListAggregatingTasks = %v, want = %v; (-want,+got)\n%s", got, tc.want, diff)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func createAggregatingTaskInfo(msg *base.TaskMessage) *TaskInfo {
|
||||
return newTaskInfo(msg, base.TaskStateAggregating, time.Time{}, nil)
|
||||
}
|
||||
|
||||
func TestInspectorListPagination(t *testing.T) {
|
||||
// Create 100 tasks.
|
||||
var msgs []*base.TaskMessage
|
||||
@@ -1180,6 +1322,9 @@ func TestInspectorListTasksQueueNotFoundError(t *testing.T) {
|
||||
if _, err := inspector.ListCompletedTasks(tc.qname); !errors.Is(err, tc.wantErr) {
|
||||
t.Errorf("ListCompletedTasks(%q) returned error %v, want %v", tc.qname, err, tc.wantErr)
|
||||
}
|
||||
if _, err := inspector.ListAggregatingTasks(tc.qname, "mygroup"); !errors.Is(err, tc.wantErr) {
|
||||
t.Errorf("ListAggregatingTasks(%q, \"mygroup\") returned error %v, want %v", tc.qname, err, tc.wantErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3282,3 +3427,99 @@ func TestParseOption(t *testing.T) {
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestInspectorGroups(t *testing.T) {
|
||||
r := setup(t)
|
||||
defer r.Close()
|
||||
inspector := NewInspector(getRedisConnOpt(t))
|
||||
|
||||
m1 := h.NewTaskMessageBuilder().SetGroup("group1").Build()
|
||||
m2 := h.NewTaskMessageBuilder().SetGroup("group1").Build()
|
||||
m3 := h.NewTaskMessageBuilder().SetGroup("group1").Build()
|
||||
m4 := h.NewTaskMessageBuilder().SetGroup("group2").Build()
|
||||
m5 := h.NewTaskMessageBuilder().SetQueue("custom").SetGroup("group1").Build()
|
||||
m6 := h.NewTaskMessageBuilder().SetQueue("custom").SetGroup("group1").Build()
|
||||
|
||||
now := time.Now()
|
||||
|
||||
fixtures := struct {
|
||||
tasks []*h.TaskSeedData
|
||||
allGroups map[string][]string
|
||||
groups map[string][]*redis.Z
|
||||
}{
|
||||
tasks: []*h.TaskSeedData{
|
||||
{Msg: m1, State: base.TaskStateAggregating},
|
||||
{Msg: m2, State: base.TaskStateAggregating},
|
||||
{Msg: m3, State: base.TaskStateAggregating},
|
||||
{Msg: m4, State: base.TaskStateAggregating},
|
||||
{Msg: m5, State: base.TaskStateAggregating},
|
||||
},
|
||||
allGroups: map[string][]string{
|
||||
base.AllGroups("default"): {"group1", "group2"},
|
||||
base.AllGroups("custom"): {"group1"},
|
||||
},
|
||||
groups: map[string][]*redis.Z{
|
||||
base.GroupKey("default", "group1"): {
|
||||
{Member: m1.ID, Score: float64(now.Add(-10 * time.Second).Unix())},
|
||||
{Member: m2.ID, Score: float64(now.Add(-20 * time.Second).Unix())},
|
||||
{Member: m3.ID, Score: float64(now.Add(-30 * time.Second).Unix())},
|
||||
},
|
||||
base.GroupKey("default", "group2"): {
|
||||
{Member: m4.ID, Score: float64(now.Add(-20 * time.Second).Unix())},
|
||||
},
|
||||
base.GroupKey("custom", "group1"): {
|
||||
{Member: m5.ID, Score: float64(now.Add(-10 * time.Second).Unix())},
|
||||
{Member: m6.ID, Score: float64(now.Add(-20 * time.Second).Unix())},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
desc string
|
||||
qname string
|
||||
want []*GroupInfo
|
||||
}{
|
||||
{
|
||||
desc: "default queue groups",
|
||||
qname: "default",
|
||||
want: []*GroupInfo{
|
||||
{Group: "group1", Size: 3},
|
||||
{Group: "group2", Size: 1},
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "custom queue groups",
|
||||
qname: "custom",
|
||||
want: []*GroupInfo{
|
||||
{Group: "group1", Size: 2},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
var sortGroupInfosOpt = cmp.Transformer(
|
||||
"SortGroupInfos",
|
||||
func(in []*GroupInfo) []*GroupInfo {
|
||||
out := append([]*GroupInfo(nil), in...)
|
||||
sort.Slice(out, func(i, j int) bool {
|
||||
return out[i].Group < out[j].Group
|
||||
})
|
||||
return out
|
||||
})
|
||||
|
||||
for _, tc := range tests {
|
||||
h.FlushDB(t, r)
|
||||
h.SeedTasks(t, r, fixtures.tasks)
|
||||
h.SeedRedisSets(t, r, fixtures.allGroups)
|
||||
h.SeedRedisZSets(t, r, fixtures.groups)
|
||||
|
||||
t.Run(tc.desc, func(t *testing.T) {
|
||||
got, err := inspector.Groups(tc.qname)
|
||||
if err != nil {
|
||||
t.Fatalf("Groups returned error: %v", err)
|
||||
}
|
||||
if diff := cmp.Diff(tc.want, got, sortGroupInfosOpt); diff != "" {
|
||||
t.Errorf("Groups = %v, want %v; (-want,+got)\n%s", got, tc.want, diff)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@@ -18,11 +18,12 @@ import (
|
||||
"github.com/golang/protobuf/ptypes"
|
||||
"github.com/hibiken/asynq/internal/errors"
|
||||
pb "github.com/hibiken/asynq/internal/proto"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
// Version of asynq library and CLI.
|
||||
const Version = "0.21.0"
|
||||
const Version = "0.23.0"
|
||||
|
||||
// DefaultQueueName is the queue name used if none are specified by user.
|
||||
const DefaultQueueName = "default"
|
||||
@@ -49,6 +50,7 @@ const (
|
||||
TaskStateRetry
|
||||
TaskStateArchived
|
||||
TaskStateCompleted
|
||||
TaskStateAggregating // describes a state where task is waiting in a group to be aggregated
|
||||
)
|
||||
|
||||
func (s TaskState) String() string {
|
||||
@@ -65,6 +67,8 @@ func (s TaskState) String() string {
|
||||
return "archived"
|
||||
case TaskStateCompleted:
|
||||
return "completed"
|
||||
case TaskStateAggregating:
|
||||
return "aggregating"
|
||||
}
|
||||
panic(fmt.Sprintf("internal error: unknown task state %d", s))
|
||||
}
|
||||
@@ -83,6 +87,8 @@ func TaskStateFromString(s string) (TaskState, error) {
|
||||
return TaskStateArchived, nil
|
||||
case "completed":
|
||||
return TaskStateCompleted, nil
|
||||
case "aggregating":
|
||||
return TaskStateAggregating, nil
|
||||
}
|
||||
return 0, errors.E(errors.FailedPrecondition, fmt.Sprintf("%q is not supported task state", s))
|
||||
}
|
||||
@@ -136,9 +142,9 @@ func ArchivedKey(qname string) string {
|
||||
return fmt.Sprintf("%sarchived", QueueKeyPrefix(qname))
|
||||
}
|
||||
|
||||
// DeadlinesKey returns a redis key for the deadlines.
|
||||
func DeadlinesKey(qname string) string {
|
||||
return fmt.Sprintf("%sdeadlines", QueueKeyPrefix(qname))
|
||||
// LeaseKey returns a redis key for the lease.
|
||||
func LeaseKey(qname string) string {
|
||||
return fmt.Sprintf("%slease", QueueKeyPrefix(qname))
|
||||
}
|
||||
|
||||
func CompletedKey(qname string) string {
|
||||
@@ -199,6 +205,32 @@ func UniqueKey(qname, tasktype string, payload []byte) string {
|
||||
return fmt.Sprintf("%sunique:%s:%s", QueueKeyPrefix(qname), tasktype, hex.EncodeToString(checksum[:]))
|
||||
}
|
||||
|
||||
// GroupKeyPrefix returns a prefix for group key.
|
||||
func GroupKeyPrefix(qname string) string {
|
||||
return fmt.Sprintf("%sg:", QueueKeyPrefix(qname))
|
||||
}
|
||||
|
||||
// GroupKey returns a redis key used to group tasks belong in the same group.
|
||||
func GroupKey(qname, gkey string) string {
|
||||
return fmt.Sprintf("%s%s", GroupKeyPrefix(qname), gkey)
|
||||
}
|
||||
|
||||
// AggregationSetKey returns a redis key used for an aggregation set.
|
||||
func AggregationSetKey(qname, gname, setID string) string {
|
||||
return fmt.Sprintf("%s:%s", GroupKey(qname, gname), setID)
|
||||
}
|
||||
|
||||
// AllGroups return a redis key used to store all group keys used in a given queue.
|
||||
func AllGroups(qname string) string {
|
||||
return fmt.Sprintf("%sgroups", QueueKeyPrefix(qname))
|
||||
}
|
||||
|
||||
// AllAggregationSets returns a redis key used to store all aggregation sets (set of tasks staged to be aggregated)
|
||||
// in a given queue.
|
||||
func AllAggregationSets(qname string) string {
|
||||
return fmt.Sprintf("%saggregation_sets", QueueKeyPrefix(qname))
|
||||
}
|
||||
|
||||
// TaskMessage is the internal representation of a task with additional metadata fields.
|
||||
// Serialized data of this type gets written to redis.
|
||||
type TaskMessage struct {
|
||||
@@ -249,6 +281,11 @@ type TaskMessage struct {
|
||||
// Empty string indicates that no uniqueness lock was used.
|
||||
UniqueKey string
|
||||
|
||||
// GroupKey holds the group key used for task aggregation.
|
||||
//
|
||||
// Empty string indicates no aggregation is used for this task.
|
||||
GroupKey string
|
||||
|
||||
// Retention specifies the number of seconds the task should be retained after completion.
|
||||
Retention int64
|
||||
|
||||
@@ -276,6 +313,7 @@ func EncodeMessage(msg *TaskMessage) ([]byte, error) {
|
||||
Timeout: msg.Timeout,
|
||||
Deadline: msg.Deadline,
|
||||
UniqueKey: msg.UniqueKey,
|
||||
GroupKey: msg.GroupKey,
|
||||
Retention: msg.Retention,
|
||||
CompletedAt: msg.CompletedAt,
|
||||
})
|
||||
@@ -299,6 +337,7 @@ func DecodeMessage(data []byte) (*TaskMessage, error) {
|
||||
Timeout: pbmsg.GetTimeout(),
|
||||
Deadline: pbmsg.GetDeadline(),
|
||||
UniqueKey: pbmsg.GetUniqueKey(),
|
||||
GroupKey: pbmsg.GetGroupKey(),
|
||||
Retention: pbmsg.GetRetention(),
|
||||
CompletedAt: pbmsg.GetCompletedAt(),
|
||||
}, nil
|
||||
@@ -603,28 +642,114 @@ func (c *Cancelations) Get(id string) (fn context.CancelFunc, ok bool) {
|
||||
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.
|
||||
//
|
||||
// See rdb.RDB as a reference implementation.
|
||||
type Broker interface {
|
||||
Ping() error
|
||||
Close() error
|
||||
Enqueue(ctx context.Context, msg *TaskMessage) error
|
||||
EnqueueUnique(ctx context.Context, msg *TaskMessage, ttl time.Duration) error
|
||||
Dequeue(qnames ...string) (*TaskMessage, time.Time, error)
|
||||
Done(msg *TaskMessage) error
|
||||
MarkAsComplete(msg *TaskMessage) error
|
||||
Requeue(msg *TaskMessage) error
|
||||
Done(ctx context.Context, msg *TaskMessage) error
|
||||
MarkAsComplete(ctx context.Context, msg *TaskMessage) error
|
||||
Requeue(ctx context.Context, msg *TaskMessage) error
|
||||
Schedule(ctx context.Context, msg *TaskMessage, processAt time.Time) 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
|
||||
Archive(msg *TaskMessage, errMsg string) error
|
||||
Retry(ctx context.Context, msg *TaskMessage, processAt time.Time, errMsg string, isFailure bool) error
|
||||
Archive(ctx context.Context, msg *TaskMessage, errMsg string) error
|
||||
ForwardIfReady(qnames ...string) error
|
||||
|
||||
// Group aggregation related methods
|
||||
AddToGroup(ctx context.Context, msg *TaskMessage, gname string) error
|
||||
AddToGroupUnique(ctx context.Context, msg *TaskMessage, groupKey string, ttl time.Duration) error
|
||||
ListGroups(qname string) ([]string, error)
|
||||
AggregationCheck(qname, gname string, t time.Time, gracePeriod, maxDelay time.Duration, maxSize int) (aggregationSetID string, err error)
|
||||
ReadAggregationSet(qname, gname, aggregationSetID string) ([]*TaskMessage, time.Time, error)
|
||||
DeleteAggregationSet(ctx context.Context, qname, gname, aggregationSetID string) error
|
||||
ReclaimStaleAggregationSets(qname string) error
|
||||
|
||||
// Task retention related method
|
||||
DeleteExpiredCompletedTasks(qname string) error
|
||||
ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*TaskMessage, error)
|
||||
|
||||
// Lease related methods
|
||||
ListLeaseExpired(cutoff time.Time, qnames ...string) ([]*TaskMessage, error)
|
||||
ExtendLease(qname string, ids ...string) (time.Time, error)
|
||||
|
||||
// State snapshot related methods
|
||||
WriteServerState(info *ServerInfo, workers []*WorkerInfo, ttl time.Duration) error
|
||||
ClearServerState(host string, pid int, serverID string) error
|
||||
|
||||
// Cancelation related methods
|
||||
CancelationPubSub() (*redis.PubSub, error) // TODO: Need to decouple from redis to support other brokers
|
||||
PublishCancelation(id string) error
|
||||
|
||||
WriteResult(qname, id string, data []byte) (n int, err error)
|
||||
Close() error
|
||||
}
|
||||
|
@@ -16,6 +16,7 @@ import (
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/uuid"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
)
|
||||
|
||||
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 {
|
||||
qname string
|
||||
want string
|
||||
}{
|
||||
{"default", "asynq:{default}:deadlines"},
|
||||
{"custom", "asynq:{custom}:deadlines"},
|
||||
{"default", "asynq:{default}:lease"},
|
||||
{"custom", "asynq:{custom}:lease"},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
got := DeadlinesKey(tc.qname)
|
||||
got := LeaseKey(tc.qname)
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -394,6 +395,107 @@ func TestUniqueKey(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestGroupKey(t *testing.T) {
|
||||
tests := []struct {
|
||||
qname string
|
||||
gkey string
|
||||
want string
|
||||
}{
|
||||
{
|
||||
qname: "default",
|
||||
gkey: "mygroup",
|
||||
want: "asynq:{default}:g:mygroup",
|
||||
},
|
||||
{
|
||||
qname: "custom",
|
||||
gkey: "foo",
|
||||
want: "asynq:{custom}:g:foo",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
got := GroupKey(tc.qname, tc.gkey)
|
||||
if got != tc.want {
|
||||
t.Errorf("GroupKey(%q, %q) = %q, want %q", tc.qname, tc.gkey, got, tc.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestAggregationSetKey(t *testing.T) {
|
||||
tests := []struct {
|
||||
qname string
|
||||
gname string
|
||||
setID string
|
||||
want string
|
||||
}{
|
||||
{
|
||||
qname: "default",
|
||||
gname: "mygroup",
|
||||
setID: "12345",
|
||||
want: "asynq:{default}:g:mygroup:12345",
|
||||
},
|
||||
{
|
||||
qname: "custom",
|
||||
gname: "foo",
|
||||
setID: "98765",
|
||||
want: "asynq:{custom}:g:foo:98765",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
got := AggregationSetKey(tc.qname, tc.gname, tc.setID)
|
||||
if got != tc.want {
|
||||
t.Errorf("AggregationSetKey(%q, %q, %q) = %q, want %q", tc.qname, tc.gname, tc.setID, got, tc.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestAllGroups(t *testing.T) {
|
||||
tests := []struct {
|
||||
qname string
|
||||
want string
|
||||
}{
|
||||
{
|
||||
qname: "default",
|
||||
want: "asynq:{default}:groups",
|
||||
},
|
||||
{
|
||||
qname: "custom",
|
||||
want: "asynq:{custom}:groups",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
got := AllGroups(tc.qname)
|
||||
if got != tc.want {
|
||||
t.Errorf("AllGroups(%q) = %q, want %q", tc.qname, got, tc.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestAllAggregationSets(t *testing.T) {
|
||||
tests := []struct {
|
||||
qname string
|
||||
want string
|
||||
}{
|
||||
{
|
||||
qname: "default",
|
||||
want: "asynq:{default}:aggregation_sets",
|
||||
},
|
||||
{
|
||||
qname: "custom",
|
||||
want: "asynq:{custom}:aggregation_sets",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
got := AllAggregationSets(tc.qname)
|
||||
if got != tc.want {
|
||||
t.Errorf("AllAggregationSets(%q) = %q, want %q", tc.qname, got, tc.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestMessageEncoding(t *testing.T) {
|
||||
id := uuid.NewString()
|
||||
tests := []struct {
|
||||
@@ -406,6 +508,7 @@ func TestMessageEncoding(t *testing.T) {
|
||||
Payload: toBytes(map[string]interface{}{"a": 1, "b": "hello!", "c": true}),
|
||||
ID: id,
|
||||
Queue: "default",
|
||||
GroupKey: "mygroup",
|
||||
Retry: 10,
|
||||
Retried: 0,
|
||||
Timeout: 1800,
|
||||
@@ -417,6 +520,7 @@ func TestMessageEncoding(t *testing.T) {
|
||||
Payload: toBytes(map[string]interface{}{"a": json.Number("1"), "b": "hello!", "c": true}),
|
||||
ID: id,
|
||||
Queue: "default",
|
||||
GroupKey: "mygroup",
|
||||
Retry: 10,
|
||||
Retried: 0,
|
||||
Timeout: 1800,
|
||||
@@ -627,3 +731,75 @@ func TestCancelationsConcurrentAccess(t *testing.T) {
|
||||
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()")
|
||||
}
|
||||
}
|
||||
|
@@ -28,14 +28,14 @@ type ctxKey int
|
||||
const metadataCtxKey ctxKey = 0
|
||||
|
||||
// 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{
|
||||
id: msg.ID,
|
||||
maxRetry: msg.Retry,
|
||||
retryCount: msg.Retried,
|
||||
qname: msg.Queue,
|
||||
}
|
||||
ctx := context.WithValue(context.Background(), metadataCtxKey, metadata)
|
||||
ctx := context.WithValue(base, metadataCtxKey, metadata)
|
||||
return context.WithDeadline(ctx, deadline)
|
||||
}
|
||||
|
||||
|
@@ -6,6 +6,7 @@ package context
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@@ -28,7 +29,7 @@ func TestCreateContextWithFutureDeadline(t *testing.T) {
|
||||
Payload: nil,
|
||||
}
|
||||
|
||||
ctx, cancel := New(msg, tc.deadline)
|
||||
ctx, cancel := New(context.Background(), msg, tc.deadline)
|
||||
select {
|
||||
case x := <-ctx.Done():
|
||||
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) {
|
||||
tests := []struct {
|
||||
deadline time.Time
|
||||
@@ -67,7 +115,7 @@ func TestCreateContextWithPastDeadline(t *testing.T) {
|
||||
Payload: nil,
|
||||
}
|
||||
|
||||
ctx, cancel := New(msg, tc.deadline)
|
||||
ctx, cancel := New(context.Background(), msg, tc.deadline)
|
||||
defer cancel()
|
||||
|
||||
select {
|
||||
@@ -97,7 +145,7 @@ func TestGetTaskMetadataFromContext(t *testing.T) {
|
||||
}
|
||||
|
||||
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()
|
||||
|
||||
id, ok := GetTaskID(ctx)
|
||||
|
@@ -32,6 +32,7 @@ const _ = proto.ProtoPackageIsVersion4
|
||||
|
||||
// TaskMessage is the internal representation of a task with additional
|
||||
// metadata fields.
|
||||
// Next ID: 15
|
||||
type TaskMessage struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@@ -65,6 +66,9 @@ type TaskMessage struct {
|
||||
// UniqueKey holds the redis key used for uniqueness lock for this task.
|
||||
// Empty string indicates that no uniqueness lock was used.
|
||||
UniqueKey string `protobuf:"bytes,10,opt,name=unique_key,json=uniqueKey,proto3" json:"unique_key,omitempty"`
|
||||
// GroupKey is a name of the group used for task aggregation.
|
||||
// This field is optional and empty value means no aggregation for the task.
|
||||
GroupKey string `protobuf:"bytes,14,opt,name=group_key,json=groupKey,proto3" json:"group_key,omitempty"`
|
||||
// Retention period specified in a number of seconds.
|
||||
// The task will be stored in redis as a completed task until the TTL
|
||||
// expires.
|
||||
@@ -184,6 +188,13 @@ func (x *TaskMessage) GetUniqueKey() string {
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *TaskMessage) GetGroupKey() string {
|
||||
if x != nil {
|
||||
return x.GroupKey
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *TaskMessage) GetRetention() int64 {
|
||||
if x != nil {
|
||||
return x.Retention
|
||||
@@ -614,7 +625,7 @@ var file_asynq_proto_rawDesc = []byte{
|
||||
0x0a, 0x0b, 0x61, 0x73, 0x79, 0x6e, 0x71, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x05, 0x61,
|
||||
0x73, 0x79, 0x6e, 0x71, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xea, 0x02, 0x0a, 0x0b, 0x54, 0x61, 0x73, 0x6b, 0x4d, 0x65,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x87, 0x03, 0x0a, 0x0b, 0x54, 0x61, 0x73, 0x6b, 0x4d, 0x65,
|
||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79,
|
||||
0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c,
|
||||
@@ -633,84 +644,86 @@ var file_asynq_proto_rawDesc = []byte{
|
||||
0x6e, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x64, 0x65, 0x61, 0x64, 0x6c, 0x69,
|
||||
0x6e, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6b, 0x65, 0x79,
|
||||
0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65,
|
||||
0x79, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0c,
|
||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x12,
|
||||
0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18,
|
||||
0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64,
|
||||
0x41, 0x74, 0x22, 0x8f, 0x03, 0x0a, 0x0a, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x6e, 0x66,
|
||||
0x6f, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x05, 0x52, 0x03, 0x70, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65,
|
||||
0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76,
|
||||
0x65, 0x72, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65,
|
||||
0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75,
|
||||
0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x35, 0x0a, 0x06, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73,
|
||||
0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x61, 0x73, 0x79, 0x6e, 0x71, 0x2e, 0x53,
|
||||
0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x51, 0x75, 0x65, 0x75, 0x65, 0x73,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x12, 0x27, 0x0a,
|
||||
0x0f, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79,
|
||||
0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x50, 0x72,
|
||||
0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73,
|
||||
0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x39,
|
||||
0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09,
|
||||
0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x2e, 0x0a, 0x13, 0x61, 0x63, 0x74,
|
||||
0x69, 0x76, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74,
|
||||
0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x57, 0x6f,
|
||||
0x72, 0x6b, 0x65, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x1a, 0x39, 0x0a, 0x0b, 0x51, 0x75, 0x65,
|
||||
0x75, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,
|
||||
0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x3a, 0x02, 0x38, 0x01, 0x22, 0xb1, 0x02, 0x0a, 0x0a, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49,
|
||||
0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x69, 0x64, 0x18, 0x02,
|
||||
0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x70, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72,
|
||||
0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65,
|
||||
0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69,
|
||||
0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12,
|
||||
0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, 0x01,
|
||||
0x28, 0x0c, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
|
||||
0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,
|
||||
0x71, 0x75, 0x65, 0x75, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74,
|
||||
0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
|
||||
0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65,
|
||||
0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65,
|
||||
0x12, 0x36, 0x0a, 0x08, 0x64, 0x65, 0x61, 0x64, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x09, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x08,
|
||||
0x64, 0x65, 0x61, 0x64, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0xad, 0x02, 0x0a, 0x0e, 0x53, 0x63, 0x68,
|
||||
0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x0e, 0x0a, 0x02, 0x69,
|
||||
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73,
|
||||
0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12,
|
||||
0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x0c, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
|
||||
0x27, 0x0a, 0x0f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75,
|
||||
0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, 0x0a, 0x11, 0x6e, 0x65, 0x78, 0x74,
|
||||
0x5f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52,
|
||||
0x0f, 0x6e, 0x65, 0x78, 0x74, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x54, 0x69, 0x6d, 0x65,
|
||||
0x12, 0x46, 0x0a, 0x11, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65,
|
||||
0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f,
|
||||
0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69,
|
||||
0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0f, 0x70, 0x72, 0x65, 0x76, 0x45, 0x6e, 0x71,
|
||||
0x75, 0x65, 0x75, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x6f, 0x0a, 0x15, 0x53, 0x63, 0x68, 0x65,
|
||||
0x64, 0x75, 0x6c, 0x65, 0x72, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x45, 0x76, 0x65, 0x6e,
|
||||
0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x3d, 0x0a, 0x0c, 0x65, 0x6e,
|
||||
0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x79, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x0e,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x12, 0x1c,
|
||||
0x0a, 0x09, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28,
|
||||
0x03, 0x52, 0x09, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0d, 0x20, 0x01,
|
||||
0x28, 0x03, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x22,
|
||||
0x8f, 0x03, 0x0a, 0x0a, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12,
|
||||
0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f,
|
||||
0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52,
|
||||
0x03, 0x70, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69,
|
||||
0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49,
|
||||
0x64, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79,
|
||||
0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65,
|
||||
0x6e, 0x63, 0x79, 0x12, 0x35, 0x0a, 0x06, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x18, 0x05, 0x20,
|
||||
0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x61, 0x73, 0x79, 0x6e, 0x71, 0x2e, 0x53, 0x65, 0x72, 0x76,
|
||||
0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x51, 0x75, 0x65, 0x75, 0x65, 0x73, 0x45, 0x6e, 0x74,
|
||||
0x72, 0x79, 0x52, 0x06, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74,
|
||||
0x72, 0x69, 0x63, 0x74, 0x5f, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x06, 0x20,
|
||||
0x01, 0x28, 0x08, 0x52, 0x0e, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x50, 0x72, 0x69, 0x6f, 0x72,
|
||||
0x69, 0x74, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x07, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x73,
|
||||
0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
||||
0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
|
||||
0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61,
|
||||
0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x2e, 0x0a, 0x13, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65,
|
||||
0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x09, 0x20,
|
||||
0x01, 0x28, 0x05, 0x52, 0x11, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65,
|
||||
0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x1a, 0x39, 0x0a, 0x0b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x73,
|
||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
|
||||
0x01, 0x22, 0xb1, 0x02, 0x0a, 0x0a, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f,
|
||||
0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
|
||||
0x68, 0x6f, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x05, 0x52, 0x03, 0x70, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,
|
||||
0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65,
|
||||
0x72, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x08, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
||||
0x0b, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05,
|
||||
0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65,
|
||||
0x75, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65,
|
||||
0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
|
||||
0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x36, 0x0a,
|
||||
0x08, 0x64, 0x65, 0x61, 0x64, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32,
|
||||
0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
|
||||
0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x08, 0x64, 0x65, 0x61,
|
||||
0x64, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0xad, 0x02, 0x0a, 0x0e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75,
|
||||
0x6c, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x1b, 0x0a, 0x09,
|
||||
0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x08, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x73,
|
||||
0x6b, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
||||
0x0b, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x27, 0x0a, 0x0f,
|
||||
0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18,
|
||||
0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x4f, 0x70,
|
||||
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, 0x0a, 0x11, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x65, 0x6e,
|
||||
0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
|
||||
0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x65, 0x6e,
|
||||
0x71, 0x75, 0x65, 0x75, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x29, 0x5a, 0x27, 0x67, 0x69, 0x74,
|
||||
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x69, 0x62, 0x69, 0x6b, 0x65, 0x6e, 0x2f,
|
||||
0x61, 0x73, 0x79, 0x6e, 0x71, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0f, 0x6e, 0x65,
|
||||
0x78, 0x74, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x46, 0x0a,
|
||||
0x11, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x74, 0x69,
|
||||
0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
|
||||
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73,
|
||||
0x74, 0x61, 0x6d, 0x70, 0x52, 0x0f, 0x70, 0x72, 0x65, 0x76, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75,
|
||||
0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x6f, 0x0a, 0x15, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c,
|
||||
0x65, 0x72, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x17,
|
||||
0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x3d, 0x0a, 0x0c, 0x65, 0x6e, 0x71, 0x75, 0x65,
|
||||
0x75, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e,
|
||||
0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
|
||||
0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x65, 0x6e, 0x71, 0x75, 0x65,
|
||||
0x75, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x29, 0x5a, 0x27, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
|
||||
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x69, 0x62, 0x69, 0x6b, 0x65, 0x6e, 0x2f, 0x61, 0x73, 0x79,
|
||||
0x6e, 0x71, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
@@ -11,6 +11,7 @@ option go_package = "github.com/hibiken/asynq/internal/proto";
|
||||
|
||||
// TaskMessage is the internal representation of a task with additional
|
||||
// metadata fields.
|
||||
// Next ID: 15
|
||||
message TaskMessage {
|
||||
// Type indicates the kind of the task to be performed.
|
||||
string type = 1;
|
||||
@@ -51,6 +52,10 @@ message TaskMessage {
|
||||
// Empty string indicates that no uniqueness lock was used.
|
||||
string unique_key = 10;
|
||||
|
||||
// GroupKey is a name of the group used for task aggregation.
|
||||
// This field is optional and empty value means no aggregation for the task.
|
||||
string group_key = 14;
|
||||
|
||||
// Retention period specified in a number of seconds.
|
||||
// The task will be stored in redis as a completed task until the TTL
|
||||
// expires.
|
||||
|
@@ -10,19 +10,19 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func BenchmarkEnqueue(b *testing.B) {
|
||||
r := setup(b)
|
||||
ctx := context.Background()
|
||||
msg := asynqtest.NewTaskMessage("task1", nil)
|
||||
msg := testutil.NewTaskMessage("task1", nil)
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
testutil.FlushDB(b, r.client)
|
||||
b.StartTimer()
|
||||
|
||||
if err := r.Enqueue(ctx, msg); err != nil {
|
||||
@@ -45,7 +45,7 @@ func BenchmarkEnqueueUnique(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
testutil.FlushDB(b, r.client)
|
||||
b.StartTimer()
|
||||
|
||||
if err := r.EnqueueUnique(ctx, msg, uniqueTTL); err != nil {
|
||||
@@ -57,13 +57,13 @@ func BenchmarkEnqueueUnique(b *testing.B) {
|
||||
func BenchmarkSchedule(b *testing.B) {
|
||||
r := setup(b)
|
||||
ctx := context.Background()
|
||||
msg := asynqtest.NewTaskMessage("task1", nil)
|
||||
msg := testutil.NewTaskMessage("task1", nil)
|
||||
processAt := time.Now().Add(3 * time.Minute)
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
testutil.FlushDB(b, r.client)
|
||||
b.StartTimer()
|
||||
|
||||
if err := r.Schedule(ctx, msg, processAt); err != nil {
|
||||
@@ -87,7 +87,7 @@ func BenchmarkScheduleUnique(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
testutil.FlushDB(b, r.client)
|
||||
b.StartTimer()
|
||||
|
||||
if err := r.ScheduleUnique(ctx, msg, processAt, uniqueTTL); err != nil {
|
||||
@@ -103,9 +103,9 @@ func BenchmarkDequeueSingleQueue(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
testutil.FlushDB(b, r.client)
|
||||
for i := 0; i < 10; i++ {
|
||||
m := asynqtest.NewTaskMessageWithQueue(
|
||||
m := testutil.NewTaskMessageWithQueue(
|
||||
fmt.Sprintf("task%d", i), nil, base.DefaultQueueName)
|
||||
if err := r.Enqueue(ctx, m); err != nil {
|
||||
b.Fatalf("Enqueue failed: %v", err)
|
||||
@@ -127,10 +127,10 @@ func BenchmarkDequeueMultipleQueues(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
testutil.FlushDB(b, r.client)
|
||||
for i := 0; i < 10; i++ {
|
||||
for _, qname := range qnames {
|
||||
m := asynqtest.NewTaskMessageWithQueue(
|
||||
m := testutil.NewTaskMessageWithQueue(
|
||||
fmt.Sprintf("%s_task%d", qname, i), nil, qname)
|
||||
if err := r.Enqueue(ctx, m); err != nil {
|
||||
b.Fatalf("Enqueue failed: %v", err)
|
||||
@@ -147,25 +147,26 @@ func BenchmarkDequeueMultipleQueues(b *testing.B) {
|
||||
|
||||
func BenchmarkDone(b *testing.B) {
|
||||
r := setup(b)
|
||||
m1 := asynqtest.NewTaskMessage("task1", nil)
|
||||
m2 := asynqtest.NewTaskMessage("task2", nil)
|
||||
m3 := asynqtest.NewTaskMessage("task3", nil)
|
||||
m1 := testutil.NewTaskMessage("task1", nil)
|
||||
m2 := testutil.NewTaskMessage("task2", nil)
|
||||
m3 := testutil.NewTaskMessage("task3", nil)
|
||||
msgs := []*base.TaskMessage{m1, m2, m3}
|
||||
zs := []base.Z{
|
||||
{Message: m1, Score: time.Now().Add(10 * time.Second).Unix()},
|
||||
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
|
||||
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
|
||||
}
|
||||
ctx := context.Background()
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName)
|
||||
testutil.FlushDB(b, r.client)
|
||||
testutil.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
testutil.SeedLease(b, r.client, zs, base.DefaultQueueName)
|
||||
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)
|
||||
}
|
||||
}
|
||||
@@ -173,25 +174,26 @@ func BenchmarkDone(b *testing.B) {
|
||||
|
||||
func BenchmarkRetry(b *testing.B) {
|
||||
r := setup(b)
|
||||
m1 := asynqtest.NewTaskMessage("task1", nil)
|
||||
m2 := asynqtest.NewTaskMessage("task2", nil)
|
||||
m3 := asynqtest.NewTaskMessage("task3", nil)
|
||||
m1 := testutil.NewTaskMessage("task1", nil)
|
||||
m2 := testutil.NewTaskMessage("task2", nil)
|
||||
m3 := testutil.NewTaskMessage("task3", nil)
|
||||
msgs := []*base.TaskMessage{m1, m2, m3}
|
||||
zs := []base.Z{
|
||||
{Message: m1, Score: time.Now().Add(10 * time.Second).Unix()},
|
||||
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
|
||||
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
|
||||
}
|
||||
ctx := context.Background()
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName)
|
||||
testutil.FlushDB(b, r.client)
|
||||
testutil.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
testutil.SeedLease(b, r.client, zs, base.DefaultQueueName)
|
||||
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)
|
||||
}
|
||||
}
|
||||
@@ -199,25 +201,26 @@ func BenchmarkRetry(b *testing.B) {
|
||||
|
||||
func BenchmarkArchive(b *testing.B) {
|
||||
r := setup(b)
|
||||
m1 := asynqtest.NewTaskMessage("task1", nil)
|
||||
m2 := asynqtest.NewTaskMessage("task2", nil)
|
||||
m3 := asynqtest.NewTaskMessage("task3", nil)
|
||||
m1 := testutil.NewTaskMessage("task1", nil)
|
||||
m2 := testutil.NewTaskMessage("task2", nil)
|
||||
m3 := testutil.NewTaskMessage("task3", nil)
|
||||
msgs := []*base.TaskMessage{m1, m2, m3}
|
||||
zs := []base.Z{
|
||||
{Message: m1, Score: time.Now().Add(10 * time.Second).Unix()},
|
||||
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
|
||||
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
|
||||
}
|
||||
ctx := context.Background()
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName)
|
||||
testutil.FlushDB(b, r.client)
|
||||
testutil.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
testutil.SeedLease(b, r.client, zs, base.DefaultQueueName)
|
||||
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)
|
||||
}
|
||||
}
|
||||
@@ -225,25 +228,26 @@ func BenchmarkArchive(b *testing.B) {
|
||||
|
||||
func BenchmarkRequeue(b *testing.B) {
|
||||
r := setup(b)
|
||||
m1 := asynqtest.NewTaskMessage("task1", nil)
|
||||
m2 := asynqtest.NewTaskMessage("task2", nil)
|
||||
m3 := asynqtest.NewTaskMessage("task3", nil)
|
||||
m1 := testutil.NewTaskMessage("task1", nil)
|
||||
m2 := testutil.NewTaskMessage("task2", nil)
|
||||
m3 := testutil.NewTaskMessage("task3", nil)
|
||||
msgs := []*base.TaskMessage{m1, m2, m3}
|
||||
zs := []base.Z{
|
||||
{Message: m1, Score: time.Now().Add(10 * time.Second).Unix()},
|
||||
{Message: m2, Score: time.Now().Add(20 * time.Second).Unix()},
|
||||
{Message: m3, Score: time.Now().Add(30 * time.Second).Unix()},
|
||||
}
|
||||
ctx := context.Background()
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedDeadlines(b, r.client, zs, base.DefaultQueueName)
|
||||
testutil.FlushDB(b, r.client)
|
||||
testutil.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
testutil.SeedLease(b, r.client, zs, base.DefaultQueueName)
|
||||
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)
|
||||
}
|
||||
}
|
||||
@@ -254,7 +258,7 @@ func BenchmarkCheckAndEnqueue(b *testing.B) {
|
||||
now := time.Now()
|
||||
var zs []base.Z
|
||||
for i := -100; i < 100; i++ {
|
||||
msg := asynqtest.NewTaskMessage(fmt.Sprintf("task%d", i), nil)
|
||||
msg := testutil.NewTaskMessage(fmt.Sprintf("task%d", i), nil)
|
||||
score := now.Add(time.Duration(i) * time.Second).Unix()
|
||||
zs = append(zs, base.Z{Message: msg, Score: score})
|
||||
}
|
||||
@@ -262,8 +266,8 @@ func BenchmarkCheckAndEnqueue(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedScheduledQueue(b, r.client, zs, base.DefaultQueueName)
|
||||
testutil.FlushDB(b, r.client)
|
||||
testutil.SeedScheduledQueue(b, r.client, zs, base.DefaultQueueName)
|
||||
b.StartTimer()
|
||||
|
||||
if err := r.ForwardIfReady(base.DefaultQueueName); err != nil {
|
||||
|
@@ -34,13 +34,18 @@ type Stats struct {
|
||||
Paused bool
|
||||
// Size is the total number of tasks in the queue.
|
||||
Size int
|
||||
|
||||
// Groups is the total number of groups in the queue.
|
||||
Groups int
|
||||
|
||||
// Number of tasks in each state.
|
||||
Pending int
|
||||
Active int
|
||||
Scheduled int
|
||||
Retry int
|
||||
Archived int
|
||||
Completed int
|
||||
Pending int
|
||||
Active int
|
||||
Scheduled int
|
||||
Retry int
|
||||
Archived int
|
||||
Completed int
|
||||
Aggregating int
|
||||
|
||||
// Number of tasks processed within the current date.
|
||||
// The number includes both succeeded and failed tasks.
|
||||
@@ -83,8 +88,10 @@ type DailyStats struct {
|
||||
// KEYS[9] -> asynq:<qname>:processed
|
||||
// KEYS[10] -> asynq:<qname>:failed
|
||||
// KEYS[11] -> asynq:<qname>:paused
|
||||
//
|
||||
// KEYS[12] -> asynq:<qname>:groups
|
||||
// --------
|
||||
// ARGV[1] -> task key prefix
|
||||
// ARGV[2] -> group key prefix
|
||||
var currentStatsCmd = redis.NewScript(`
|
||||
local res = {}
|
||||
local pendingTaskCount = redis.call("LLEN", KEYS[1])
|
||||
@@ -118,6 +125,15 @@ if pendingTaskCount > 0 then
|
||||
else
|
||||
table.insert(res, 0)
|
||||
end
|
||||
local group_names = redis.call("SMEMBERS", KEYS[12])
|
||||
table.insert(res, "group_size")
|
||||
table.insert(res, table.getn(group_names))
|
||||
local aggregating_count = 0
|
||||
for _, gname in ipairs(group_names) do
|
||||
aggregating_count = aggregating_count + redis.call("ZCARD", ARGV[2] .. gname)
|
||||
end
|
||||
table.insert(res, "aggregating_count")
|
||||
table.insert(res, aggregating_count)
|
||||
return res`)
|
||||
|
||||
// CurrentStats returns a current state of the queues.
|
||||
@@ -131,7 +147,7 @@ func (r *RDB) CurrentStats(qname string) (*Stats, error) {
|
||||
return nil, errors.E(op, errors.NotFound, &errors.QueueNotFoundError{Queue: qname})
|
||||
}
|
||||
now := r.clock.Now()
|
||||
res, err := currentStatsCmd.Run(context.Background(), r.client, []string{
|
||||
keys := []string{
|
||||
base.PendingKey(qname),
|
||||
base.ActiveKey(qname),
|
||||
base.ScheduledKey(qname),
|
||||
@@ -143,7 +159,13 @@ func (r *RDB) CurrentStats(qname string) (*Stats, error) {
|
||||
base.ProcessedTotalKey(qname),
|
||||
base.FailedTotalKey(qname),
|
||||
base.PausedKey(qname),
|
||||
}, base.TaskKeyPrefix(qname)).Result()
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
base.TaskKeyPrefix(qname),
|
||||
base.GroupKeyPrefix(qname),
|
||||
}
|
||||
res, err := currentStatsCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.Unknown, err)
|
||||
}
|
||||
@@ -198,6 +220,11 @@ func (r *RDB) CurrentStats(qname string) (*Stats, error) {
|
||||
} else {
|
||||
stats.Latency = r.clock.Now().Sub(time.Unix(0, int64(val)))
|
||||
}
|
||||
case "group_size":
|
||||
stats.Groups = val
|
||||
case "aggregating_count":
|
||||
stats.Aggregating = val
|
||||
size += val
|
||||
}
|
||||
}
|
||||
stats.Size = size
|
||||
@@ -219,9 +246,12 @@ func (r *RDB) CurrentStats(qname string) (*Stats, error) {
|
||||
// KEYS[4] -> asynq:{qname}:retry
|
||||
// KEYS[5] -> asynq:{qname}:archived
|
||||
// KEYS[6] -> asynq:{qname}:completed
|
||||
//
|
||||
// ARGV[1] -> asynq:{qname}:t:
|
||||
// ARGV[2] -> sample_size (e.g 20)
|
||||
// KEYS[7] -> asynq:{qname}:groups
|
||||
// -------
|
||||
// ARGV[1] -> asynq:{qname}:t: (task key prefix)
|
||||
// ARGV[2] -> task sample size per redis list/zset (e.g 20)
|
||||
// ARGV[3] -> group sample size
|
||||
// ARGV[4] -> asynq:{qname}:g: (group key prefix)
|
||||
var memoryUsageCmd = redis.NewScript(`
|
||||
local sample_size = tonumber(ARGV[2])
|
||||
if sample_size <= 0 then
|
||||
@@ -262,12 +292,36 @@ for i=3,6 do
|
||||
memusg = memusg + m
|
||||
end
|
||||
end
|
||||
local groups = redis.call("SMEMBERS", KEYS[7])
|
||||
if table.getn(groups) > 0 then
|
||||
local agg_task_count = 0
|
||||
local agg_task_sample_total = 0
|
||||
local agg_task_sample_size = 0
|
||||
for i, gname in ipairs(groups) do
|
||||
local group_key = ARGV[4] .. gname
|
||||
agg_task_count = agg_task_count + redis.call("ZCARD", group_key)
|
||||
if i <= tonumber(ARGV[3]) then
|
||||
local ids = redis.call("ZRANGE", group_key, 0, sample_size - 1)
|
||||
for _, id in ipairs(ids) do
|
||||
local bytes = redis.call("MEMORY", "USAGE", ARGV[1] .. id)
|
||||
agg_task_sample_total = agg_task_sample_total + bytes
|
||||
agg_task_sample_size = agg_task_sample_size + 1
|
||||
end
|
||||
end
|
||||
end
|
||||
local avg = agg_task_sample_total / agg_task_sample_size
|
||||
memusg = memusg + (avg * agg_task_count)
|
||||
end
|
||||
return memusg
|
||||
`)
|
||||
|
||||
func (r *RDB) memoryUsage(qname string) (int64, error) {
|
||||
var op errors.Op = "rdb.memoryUsage"
|
||||
const sampleSize = 20
|
||||
const (
|
||||
taskSampleSize = 20
|
||||
groupSampleSize = 5
|
||||
)
|
||||
|
||||
keys := []string{
|
||||
base.ActiveKey(qname),
|
||||
base.PendingKey(qname),
|
||||
@@ -275,10 +329,13 @@ func (r *RDB) memoryUsage(qname string) (int64, error) {
|
||||
base.RetryKey(qname),
|
||||
base.ArchivedKey(qname),
|
||||
base.CompletedKey(qname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
base.TaskKeyPrefix(qname),
|
||||
sampleSize,
|
||||
taskSampleSize,
|
||||
groupSampleSize,
|
||||
base.GroupKeyPrefix(qname),
|
||||
}
|
||||
res, err := memoryUsageCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
@@ -490,6 +547,56 @@ func (r *RDB) GetTaskInfo(qname, id string) (*base.TaskInfo, error) {
|
||||
}, nil
|
||||
}
|
||||
|
||||
type GroupStat struct {
|
||||
// Name of the group.
|
||||
Group string
|
||||
|
||||
// Size of the group.
|
||||
Size int
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:groups
|
||||
// -------
|
||||
// ARGV[1] -> group key prefix
|
||||
//
|
||||
// Output:
|
||||
// list of group name and size (e.g. group1 size1 group2 size2 ...)
|
||||
//
|
||||
// Time Complexity:
|
||||
// O(N) where N being the number of groups in the given queue.
|
||||
var groupStatsCmd = redis.NewScript(`
|
||||
local res = {}
|
||||
local group_names = redis.call("SMEMBERS", KEYS[1])
|
||||
for _, gname in ipairs(group_names) do
|
||||
local size = redis.call("ZCARD", ARGV[1] .. gname)
|
||||
table.insert(res, gname)
|
||||
table.insert(res, size)
|
||||
end
|
||||
return res
|
||||
`)
|
||||
|
||||
func (r *RDB) GroupStats(qname string) ([]*GroupStat, error) {
|
||||
var op errors.Op = "RDB.GroupStats"
|
||||
keys := []string{base.AllGroups(qname)}
|
||||
argv := []interface{}{base.GroupKeyPrefix(qname)}
|
||||
res, err := groupStatsCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.Unknown, err)
|
||||
}
|
||||
data, err := cast.ToSliceE(res)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.Internal, "cast error: unexpected return value from Lua script")
|
||||
}
|
||||
var stats []*GroupStat
|
||||
for i := 0; i < len(data); i += 2 {
|
||||
stats = append(stats, &GroupStat{
|
||||
Group: data[i].(string),
|
||||
Size: int(data[i+1].(int64)),
|
||||
})
|
||||
}
|
||||
return stats, nil
|
||||
}
|
||||
|
||||
// Pagination specifies the page size and page number
|
||||
// for the list operation.
|
||||
type Pagination struct {
|
||||
@@ -619,7 +726,7 @@ func (r *RDB) ListScheduled(qname string, pgn Pagination) ([]*base.TaskInfo, err
|
||||
if !exists {
|
||||
return nil, errors.E(op, errors.NotFound, &errors.QueueNotFoundError{Queue: qname})
|
||||
}
|
||||
res, err := r.listZSetEntries(qname, base.TaskStateScheduled, pgn)
|
||||
res, err := r.listZSetEntries(qname, base.TaskStateScheduled, base.ScheduledKey(qname), pgn)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
@@ -637,7 +744,7 @@ func (r *RDB) ListRetry(qname string, pgn Pagination) ([]*base.TaskInfo, error)
|
||||
if !exists {
|
||||
return nil, errors.E(op, errors.NotFound, &errors.QueueNotFoundError{Queue: qname})
|
||||
}
|
||||
res, err := r.listZSetEntries(qname, base.TaskStateRetry, pgn)
|
||||
res, err := r.listZSetEntries(qname, base.TaskStateRetry, base.RetryKey(qname), pgn)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
@@ -654,7 +761,7 @@ func (r *RDB) ListArchived(qname string, pgn Pagination) ([]*base.TaskInfo, erro
|
||||
if !exists {
|
||||
return nil, errors.E(op, errors.NotFound, &errors.QueueNotFoundError{Queue: qname})
|
||||
}
|
||||
zs, err := r.listZSetEntries(qname, base.TaskStateArchived, pgn)
|
||||
zs, err := r.listZSetEntries(qname, base.TaskStateArchived, base.ArchivedKey(qname), pgn)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
@@ -671,7 +778,24 @@ func (r *RDB) ListCompleted(qname string, pgn Pagination) ([]*base.TaskInfo, err
|
||||
if !exists {
|
||||
return nil, errors.E(op, errors.NotFound, &errors.QueueNotFoundError{Queue: qname})
|
||||
}
|
||||
zs, err := r.listZSetEntries(qname, base.TaskStateCompleted, pgn)
|
||||
zs, err := r.listZSetEntries(qname, base.TaskStateCompleted, base.CompletedKey(qname), pgn)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
return zs, nil
|
||||
}
|
||||
|
||||
// ListAggregating returns all tasks from the given group.
|
||||
func (r *RDB) ListAggregating(qname, gname string, pgn Pagination) ([]*base.TaskInfo, error) {
|
||||
var op errors.Op = "rdb.ListAggregating"
|
||||
exists, err := r.queueExists(qname)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sismember", Err: err})
|
||||
}
|
||||
if !exists {
|
||||
return nil, errors.E(op, errors.NotFound, &errors.QueueNotFoundError{Queue: qname})
|
||||
}
|
||||
zs, err := r.listZSetEntries(qname, base.TaskStateAggregating, base.GroupKey(qname, gname), pgn)
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
@@ -707,20 +831,7 @@ return data
|
||||
|
||||
// listZSetEntries returns a list of message and score pairs in Redis sorted-set
|
||||
// with the given key.
|
||||
func (r *RDB) listZSetEntries(qname string, state base.TaskState, pgn Pagination) ([]*base.TaskInfo, error) {
|
||||
var key string
|
||||
switch state {
|
||||
case base.TaskStateScheduled:
|
||||
key = base.ScheduledKey(qname)
|
||||
case base.TaskStateRetry:
|
||||
key = base.RetryKey(qname)
|
||||
case base.TaskStateArchived:
|
||||
key = base.ArchivedKey(qname)
|
||||
case base.TaskStateCompleted:
|
||||
key = base.CompletedKey(qname)
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported task state: %v", state))
|
||||
}
|
||||
func (r *RDB) listZSetEntries(qname string, state base.TaskState, key string, pgn Pagination) ([]*base.TaskInfo, error) {
|
||||
res, err := listZSetEntriesCmd.Run(context.Background(), r.client, []string{key},
|
||||
pgn.start(), pgn.stop(), base.TaskKeyPrefix(qname)).Result()
|
||||
if err != nil {
|
||||
@@ -811,14 +922,67 @@ func (r *RDB) RunAllArchivedTasks(qname string) (int64, error) {
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// runAllAggregatingCmd schedules all tasks in the group to run individually.
|
||||
//
|
||||
// Input:
|
||||
// KEYS[1] -> asynq:{<qname>}:g:<gname>
|
||||
// KEYS[2] -> asynq:{<qname>}:pending
|
||||
// KEYS[3] -> asynq:{<qname>}:groups
|
||||
// -------
|
||||
// ARGV[1] -> task key prefix
|
||||
// ARGV[2] -> group name
|
||||
//
|
||||
// Output:
|
||||
// integer: number of tasks scheduled to run
|
||||
var runAllAggregatingCmd = redis.NewScript(`
|
||||
local ids = redis.call("ZRANGE", KEYS[1], 0, -1)
|
||||
for _, id in ipairs(ids) do
|
||||
redis.call("LPUSH", KEYS[2], id)
|
||||
redis.call("HSET", ARGV[1] .. id, "state", "pending")
|
||||
end
|
||||
redis.call("DEL", KEYS[1])
|
||||
redis.call("SREM", KEYS[3], ARGV[2])
|
||||
return table.getn(ids)
|
||||
`)
|
||||
|
||||
// RunAllAggregatingTasks schedules all tasks from the given queue to run
|
||||
// and returns the number of tasks scheduled to run.
|
||||
// If a queue with the given name doesn't exist, it returns QueueNotFoundError.
|
||||
func (r *RDB) RunAllAggregatingTasks(qname, gname string) (int64, error) {
|
||||
var op errors.Op = "rdb.RunAllAggregatingTasks"
|
||||
if err := r.checkQueueExists(qname); err != nil {
|
||||
return 0, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
keys := []string{
|
||||
base.GroupKey(qname, gname),
|
||||
base.PendingKey(qname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
base.TaskKeyPrefix(qname),
|
||||
gname,
|
||||
}
|
||||
res, err := runAllAggregatingCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
return 0, errors.E(op, errors.Internal, err)
|
||||
}
|
||||
n, ok := res.(int64)
|
||||
if !ok {
|
||||
return 0, errors.E(op, errors.Internal, fmt.Sprintf("unexpected return value from script %v", res))
|
||||
}
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// runTaskCmd is a Lua script that updates the given task to pending state.
|
||||
//
|
||||
// Input:
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:pending
|
||||
// KEYS[3] -> asynq:{<qname>}:groups
|
||||
// --
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> queue key prefix; asynq:{<qname>}:
|
||||
// ARGV[3] -> group key prefix
|
||||
//
|
||||
// Output:
|
||||
// Numeric code indicating the status:
|
||||
@@ -831,15 +995,24 @@ var runTaskCmd = redis.NewScript(`
|
||||
if redis.call("EXISTS", KEYS[1]) == 0 then
|
||||
return 0
|
||||
end
|
||||
local state = redis.call("HGET", KEYS[1], "state")
|
||||
local state, group = unpack(redis.call("HMGET", KEYS[1], "state", "group"))
|
||||
if state == "active" then
|
||||
return -1
|
||||
elseif state == "pending" then
|
||||
return -2
|
||||
end
|
||||
local n = redis.call("ZREM", ARGV[2] .. state, ARGV[1])
|
||||
if n == 0 then
|
||||
return redis.error_reply("internal error: task id not found in zset " .. tostring(state))
|
||||
elseif state == "aggregating" then
|
||||
local n = redis.call("ZREM", ARGV[3] .. group, ARGV[1])
|
||||
if n == 0 then
|
||||
return redis.error_reply("internal error: task id not found in zset " .. tostring(ARGV[3] .. group))
|
||||
end
|
||||
if redis.call("ZCARD", ARGV[3] .. group) == 0 then
|
||||
redis.call("SREM", KEYS[3], group)
|
||||
end
|
||||
else
|
||||
local n = redis.call("ZREM", ARGV[2] .. state, ARGV[1])
|
||||
if n == 0 then
|
||||
return redis.error_reply("internal error: task id not found in zset " .. tostring(ARGV[2] .. state))
|
||||
end
|
||||
end
|
||||
redis.call("LPUSH", KEYS[2], ARGV[1])
|
||||
redis.call("HSET", KEYS[1], "state", "pending")
|
||||
@@ -860,10 +1033,12 @@ func (r *RDB) RunTask(qname, id string) error {
|
||||
keys := []string{
|
||||
base.TaskKey(qname, id),
|
||||
base.PendingKey(qname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
id,
|
||||
base.QueueKeyPrefix(qname),
|
||||
base.GroupKeyPrefix(qname),
|
||||
}
|
||||
res, err := runTaskCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
@@ -962,6 +1137,66 @@ func (r *RDB) ArchiveAllScheduledTasks(qname string) (int64, error) {
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// archiveAllAggregatingCmd archives all tasks in the given group.
|
||||
//
|
||||
// Input:
|
||||
// KEYS[1] -> asynq:{<qname>}:g:<gname>
|
||||
// KEYS[2] -> asynq:{<qname>}:archived
|
||||
// KEYS[3] -> asynq:{<qname>}:groups
|
||||
// -------
|
||||
// ARGV[1] -> current timestamp
|
||||
// ARGV[2] -> cutoff timestamp (e.g., 90 days ago)
|
||||
// ARGV[3] -> max number of tasks in archive (e.g., 100)
|
||||
// ARGV[4] -> task key prefix (asynq:{<qname>}:t:)
|
||||
// ARGV[5] -> group name
|
||||
//
|
||||
// Output:
|
||||
// integer: Number of tasks archived
|
||||
var archiveAllAggregatingCmd = redis.NewScript(`
|
||||
local ids = redis.call("ZRANGE", KEYS[1], 0, -1)
|
||||
for _, id in ipairs(ids) do
|
||||
redis.call("ZADD", KEYS[2], ARGV[1], id)
|
||||
redis.call("HSET", ARGV[4] .. id, "state", "archived")
|
||||
end
|
||||
redis.call("ZREMRANGEBYSCORE", KEYS[2], "-inf", ARGV[2])
|
||||
redis.call("ZREMRANGEBYRANK", KEYS[2], 0, -ARGV[3])
|
||||
redis.call("DEL", KEYS[1])
|
||||
redis.call("SREM", KEYS[3], ARGV[5])
|
||||
return table.getn(ids)
|
||||
`)
|
||||
|
||||
// ArchiveAllAggregatingTasks archives all aggregating tasks from the given group
|
||||
// and returns the number of tasks archived.
|
||||
// If a queue with the given name doesn't exist, it returns QueueNotFoundError.
|
||||
func (r *RDB) ArchiveAllAggregatingTasks(qname, gname string) (int64, error) {
|
||||
var op errors.Op = "rdb.ArchiveAllAggregatingTasks"
|
||||
if err := r.checkQueueExists(qname); err != nil {
|
||||
return 0, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
keys := []string{
|
||||
base.GroupKey(qname, gname),
|
||||
base.ArchivedKey(qname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
now := r.clock.Now()
|
||||
argv := []interface{}{
|
||||
now.Unix(),
|
||||
now.AddDate(0, 0, -archivedExpirationInDays).Unix(),
|
||||
maxArchiveSize,
|
||||
base.TaskKeyPrefix(qname),
|
||||
gname,
|
||||
}
|
||||
res, err := archiveAllAggregatingCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
return 0, errors.E(op, errors.Internal, err)
|
||||
}
|
||||
n, ok := res.(int64)
|
||||
if !ok {
|
||||
return 0, errors.E(op, errors.Internal, fmt.Sprintf("unexpected return value from script %v", res))
|
||||
}
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// archiveAllPendingCmd is a Lua script that moves all pending tasks from
|
||||
// the given queue to archived state.
|
||||
//
|
||||
@@ -1022,12 +1257,14 @@ func (r *RDB) ArchiveAllPendingTasks(qname string) (int64, error) {
|
||||
// Input:
|
||||
// KEYS[1] -> task key (asynq:{<qname>}:t:<task_id>)
|
||||
// KEYS[2] -> archived key (asynq:{<qname>}:archived)
|
||||
// KEYS[3] -> all groups key (asynq:{<qname>}:groups)
|
||||
// --
|
||||
// ARGV[1] -> id of the task to archive
|
||||
// ARGV[2] -> current timestamp
|
||||
// ARGV[3] -> cutoff timestamp (e.g., 90 days ago)
|
||||
// ARGV[4] -> max number of tasks in archived state (e.g., 100)
|
||||
// ARGV[5] -> queue key prefix (asynq:{<qname>}:)
|
||||
// ARGV[6] -> group key prefix (asynq:{<qname>}:g:)
|
||||
//
|
||||
// Output:
|
||||
// Numeric code indicating the status:
|
||||
@@ -1040,7 +1277,7 @@ var archiveTaskCmd = redis.NewScript(`
|
||||
if redis.call("EXISTS", KEYS[1]) == 0 then
|
||||
return 0
|
||||
end
|
||||
local state = redis.call("HGET", KEYS[1], "state")
|
||||
local state, group = unpack(redis.call("HMGET", KEYS[1], "state", "group"))
|
||||
if state == "active" then
|
||||
return -2
|
||||
end
|
||||
@@ -1049,11 +1286,18 @@ if state == "archived" then
|
||||
end
|
||||
if state == "pending" then
|
||||
if redis.call("LREM", ARGV[5] .. state, 1, ARGV[1]) == 0 then
|
||||
return redis.error_reply("task id not found in list " .. tostring(state))
|
||||
return redis.error_reply("task id not found in list " .. tostring(ARGV[5] .. state))
|
||||
end
|
||||
elseif state == "aggregating" then
|
||||
if redis.call("ZREM", ARGV[6] .. group, ARGV[1]) == 0 then
|
||||
return redis.error_reply("task id not found in zset " .. tostring(ARGV[6] .. group))
|
||||
end
|
||||
if redis.call("ZCARD", ARGV[6] .. group) == 0 then
|
||||
redis.call("SREM", KEYS[3], group)
|
||||
end
|
||||
else
|
||||
if redis.call("ZREM", ARGV[5] .. state, ARGV[1]) == 0 then
|
||||
return redis.error_reply("task id not found in zset " .. tostring(state))
|
||||
return redis.error_reply("task id not found in zset " .. tostring(ARGV[5] .. state))
|
||||
end
|
||||
end
|
||||
redis.call("ZADD", KEYS[2], ARGV[2], ARGV[1])
|
||||
@@ -1078,6 +1322,7 @@ func (r *RDB) ArchiveTask(qname, id string) error {
|
||||
keys := []string{
|
||||
base.TaskKey(qname, id),
|
||||
base.ArchivedKey(qname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
now := r.clock.Now()
|
||||
argv := []interface{}{
|
||||
@@ -1086,6 +1331,7 @@ func (r *RDB) ArchiveTask(qname, id string) error {
|
||||
now.AddDate(0, 0, -archivedExpirationInDays).Unix(),
|
||||
maxArchiveSize,
|
||||
base.QueueKeyPrefix(qname),
|
||||
base.GroupKeyPrefix(qname),
|
||||
}
|
||||
res, err := archiveTaskCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
@@ -1168,9 +1414,11 @@ func (r *RDB) archiveAll(src, dst, qname string) (int64, error) {
|
||||
|
||||
// Input:
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:groups
|
||||
// --
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> queue key prefix
|
||||
// ARGV[3] -> group key prefix
|
||||
//
|
||||
// Output:
|
||||
// Numeric code indicating the status:
|
||||
@@ -1181,17 +1429,24 @@ var deleteTaskCmd = redis.NewScript(`
|
||||
if redis.call("EXISTS", KEYS[1]) == 0 then
|
||||
return 0
|
||||
end
|
||||
local state = redis.call("HGET", KEYS[1], "state")
|
||||
local state, group = unpack(redis.call("HMGET", KEYS[1], "state", "group"))
|
||||
if state == "active" then
|
||||
return -1
|
||||
end
|
||||
if state == "pending" then
|
||||
if redis.call("LREM", ARGV[2] .. state, 0, ARGV[1]) == 0 then
|
||||
return redis.error_reply("task is not found in list: " .. tostring(state))
|
||||
return redis.error_reply("task is not found in list: " .. tostring(ARGV[2] .. state))
|
||||
end
|
||||
elseif state == "aggregating" then
|
||||
if redis.call("ZREM", ARGV[3] .. group, ARGV[1]) == 0 then
|
||||
return redis.error_reply("task is not found in zset: " .. tostring(ARGV[3] .. group))
|
||||
end
|
||||
if redis.call("ZCARD", ARGV[3] .. group) == 0 then
|
||||
redis.call("SREM", KEYS[2], group)
|
||||
end
|
||||
else
|
||||
if redis.call("ZREM", ARGV[2] .. state, ARGV[1]) == 0 then
|
||||
return redis.error_reply("task is not found in zset: " .. tostring(state))
|
||||
return redis.error_reply("task is not found in zset: " .. tostring(ARGV[2] .. state))
|
||||
end
|
||||
end
|
||||
local unique_key = redis.call("HGET", KEYS[1], "unique_key")
|
||||
@@ -1214,10 +1469,12 @@ func (r *RDB) DeleteTask(qname, id string) error {
|
||||
}
|
||||
keys := []string{
|
||||
base.TaskKey(qname, id),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
id,
|
||||
base.QueueKeyPrefix(qname),
|
||||
base.GroupKeyPrefix(qname),
|
||||
}
|
||||
res, err := deleteTaskCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
@@ -1336,6 +1593,50 @@ func (r *RDB) deleteAll(key, qname string) (int64, error) {
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// deleteAllAggregatingCmd deletes all tasks from the given group.
|
||||
//
|
||||
// Input:
|
||||
// KEYS[1] -> asynq:{<qname>}:g:<gname>
|
||||
// KEYS[2] -> asynq:{<qname>}:groups
|
||||
// -------
|
||||
// ARGV[1] -> task key prefix
|
||||
// ARGV[2] -> group name
|
||||
var deleteAllAggregatingCmd = redis.NewScript(`
|
||||
local ids = redis.call("ZRANGE", KEYS[1], 0, -1)
|
||||
for _, id in ipairs(ids) do
|
||||
redis.call("DEL", ARGV[1] .. id)
|
||||
end
|
||||
redis.call("SREM", KEYS[2], ARGV[2])
|
||||
redis.call("DEL", KEYS[1])
|
||||
return table.getn(ids)
|
||||
`)
|
||||
|
||||
// DeleteAllAggregatingTasks deletes all aggregating tasks from the given group
|
||||
// and returns the number of tasks deleted.
|
||||
func (r *RDB) DeleteAllAggregatingTasks(qname, gname string) (int64, error) {
|
||||
var op errors.Op = "rdb.DeleteAllAggregatingTasks"
|
||||
if err := r.checkQueueExists(qname); err != nil {
|
||||
return 0, errors.E(op, errors.CanonicalCode(err), err)
|
||||
}
|
||||
keys := []string{
|
||||
base.GroupKey(qname, gname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
base.TaskKeyPrefix(qname),
|
||||
gname,
|
||||
}
|
||||
res, err := deleteAllAggregatingCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
return 0, errors.E(op, errors.Unknown, err)
|
||||
}
|
||||
n, ok := res.(int64)
|
||||
if !ok {
|
||||
return 0, errors.E(op, errors.Internal, "command error: unexpected return value %v", res)
|
||||
}
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// deleteAllPendingCmd deletes all pending tasks from the given queue.
|
||||
//
|
||||
// Input:
|
||||
@@ -1387,7 +1688,7 @@ func (r *RDB) DeleteAllPendingTasks(qname string) (int64, error) {
|
||||
// KEYS[3] -> asynq:{<qname>}:scheduled
|
||||
// KEYS[4] -> asynq:{<qname>}:retry
|
||||
// KEYS[5] -> asynq:{<qname>}:archived
|
||||
// KEYS[6] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[6] -> asynq:{<qname>}:lease
|
||||
// --
|
||||
// ARGV[1] -> task key prefix
|
||||
//
|
||||
@@ -1447,7 +1748,7 @@ return 1`)
|
||||
// KEYS[3] -> asynq:{<qname>}:scheduled
|
||||
// KEYS[4] -> asynq:{<qname>}:retry
|
||||
// KEYS[5] -> asynq:{<qname>}:archived
|
||||
// KEYS[6] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[6] -> asynq:{<qname>}:lease
|
||||
// --
|
||||
// ARGV[1] -> task key prefix
|
||||
//
|
||||
@@ -1516,7 +1817,7 @@ func (r *RDB) RemoveQueue(qname string, force bool) error {
|
||||
base.ScheduledKey(qname),
|
||||
base.RetryKey(qname),
|
||||
base.ArchivedKey(qname),
|
||||
base.DeadlinesKey(qname),
|
||||
base.LeaseKey(qname),
|
||||
}
|
||||
res, err := script.Run(context.Background(), r.client, keys, base.TaskKeyPrefix(qname)).Result()
|
||||
if err != nil {
|
||||
|
File diff suppressed because it is too large
Load Diff
@@ -12,6 +12,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/go-redis/redis/v8"
|
||||
"github.com/google/uuid"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/errors"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
@@ -20,6 +21,9 @@ import (
|
||||
|
||||
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.
|
||||
type RDB struct {
|
||||
client redis.UniversalClient
|
||||
@@ -84,9 +88,7 @@ func (r *RDB) runScriptWithErrorCode(ctx context.Context, op errors.Op, script *
|
||||
// --
|
||||
// ARGV[1] -> task message data
|
||||
// ARGV[2] -> task ID
|
||||
// ARGV[3] -> task timeout in seconds (0 if not timeout)
|
||||
// ARGV[4] -> task deadline in unix time (0 if no deadline)
|
||||
// ARGV[5] -> current unix time in nsec
|
||||
// ARGV[3] -> current unix time in nsec
|
||||
//
|
||||
// Output:
|
||||
// Returns 1 if successfully enqueued
|
||||
@@ -98,9 +100,7 @@ end
|
||||
redis.call("HSET", KEYS[1],
|
||||
"msg", ARGV[1],
|
||||
"state", "pending",
|
||||
"timeout", ARGV[3],
|
||||
"deadline", ARGV[4],
|
||||
"pending_since", ARGV[5])
|
||||
"pending_since", ARGV[3])
|
||||
redis.call("LPUSH", KEYS[2], ARGV[2])
|
||||
return 1
|
||||
`)
|
||||
@@ -122,8 +122,6 @@ func (r *RDB) Enqueue(ctx context.Context, msg *base.TaskMessage) error {
|
||||
argv := []interface{}{
|
||||
encoded,
|
||||
msg.ID,
|
||||
msg.Timeout,
|
||||
msg.Deadline,
|
||||
r.clock.Now().UnixNano(),
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(ctx, op, enqueueCmd, keys, argv...)
|
||||
@@ -145,9 +143,7 @@ func (r *RDB) Enqueue(ctx context.Context, msg *base.TaskMessage) error {
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> uniqueness lock TTL
|
||||
// ARGV[3] -> task message data
|
||||
// ARGV[4] -> task timeout in seconds (0 if not timeout)
|
||||
// ARGV[5] -> task deadline in unix time (0 if no deadline)
|
||||
// ARGV[6] -> current unix time in nsec
|
||||
// ARGV[4] -> current unix time in nsec
|
||||
//
|
||||
// Output:
|
||||
// Returns 1 if successfully enqueued
|
||||
@@ -164,9 +160,7 @@ end
|
||||
redis.call("HSET", KEYS[2],
|
||||
"msg", ARGV[3],
|
||||
"state", "pending",
|
||||
"timeout", ARGV[4],
|
||||
"deadline", ARGV[5],
|
||||
"pending_since", ARGV[6],
|
||||
"pending_since", ARGV[4],
|
||||
"unique_key", KEYS[1])
|
||||
redis.call("LPUSH", KEYS[3], ARGV[1])
|
||||
return 1
|
||||
@@ -192,8 +186,6 @@ func (r *RDB) EnqueueUnique(ctx context.Context, msg *base.TaskMessage, ttl time
|
||||
msg.ID,
|
||||
int(ttl.Seconds()),
|
||||
encoded,
|
||||
msg.Timeout,
|
||||
msg.Deadline,
|
||||
r.clock.Now().UnixNano(),
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(ctx, op, enqueueUniqueCmd, keys, argv...)
|
||||
@@ -213,20 +205,17 @@ func (r *RDB) EnqueueUnique(ctx context.Context, msg *base.TaskMessage, ttl time
|
||||
// KEYS[1] -> asynq:{<qname>}:pending
|
||||
// KEYS[2] -> asynq:{<qname>}:paused
|
||||
// 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
|
||||
//
|
||||
// Output:
|
||||
// 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
|
||||
// TaskMessage, and `deadline` is Unix time in seconds.
|
||||
// Returns an encoded TaskMessage.
|
||||
//
|
||||
// Note: dequeueCmd checks whether a queue is paused first, before
|
||||
// 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(`
|
||||
if redis.call("EXISTS", KEYS[2]) == 0 then
|
||||
local id = redis.call("RPOPLPUSH", KEYS[1], KEYS[3])
|
||||
@@ -234,41 +223,28 @@ if redis.call("EXISTS", KEYS[2]) == 0 then
|
||||
local key = ARGV[2] .. id
|
||||
redis.call("HSET", key, "state", "active")
|
||||
redis.call("HDEL", key, "pending_since")
|
||||
local data = redis.call("HMGET", key, "msg", "timeout", "deadline")
|
||||
local msg = data[1]
|
||||
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}
|
||||
redis.call("ZADD", KEYS[4], ARGV[1], id)
|
||||
return redis.call("HGET", key, "msg")
|
||||
end
|
||||
end
|
||||
return nil`)
|
||||
|
||||
// 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.
|
||||
// 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"
|
||||
for _, qname := range qnames {
|
||||
keys := []string{
|
||||
base.PendingKey(qname),
|
||||
base.PausedKey(qname),
|
||||
base.ActiveKey(qname),
|
||||
base.DeadlinesKey(qname),
|
||||
base.LeaseKey(qname),
|
||||
}
|
||||
leaseExpirationTime = r.clock.Now().Add(LeaseDuration)
|
||||
argv := []interface{}{
|
||||
r.clock.Now().Unix(),
|
||||
leaseExpirationTime.Unix(),
|
||||
base.TaskKeyPrefix(qname),
|
||||
}
|
||||
res, err := dequeueCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
@@ -277,34 +253,24 @@ func (r *RDB) Dequeue(qnames ...string) (msg *base.TaskMessage, deadline time.Ti
|
||||
} else if err != nil {
|
||||
return nil, time.Time{}, errors.E(op, errors.Unknown, fmt.Sprintf("redis eval error: %v", err))
|
||||
}
|
||||
data, err := cast.ToSliceE(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])
|
||||
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 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 msg, time.Unix(d, 0), nil
|
||||
return msg, leaseExpirationTime, nil
|
||||
}
|
||||
return nil, time.Time{}, errors.E(op, errors.NotFound, errors.ErrNoProcessableTask)
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:active
|
||||
// KEYS[2] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[2] -> asynq:{<qname>}:lease
|
||||
// KEYS[3] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
||||
// KEYS[5] -> asynq:{<qname>}:processed
|
||||
// -------
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> stats expiration timestamp
|
||||
// ARGV[3] -> max int64 value
|
||||
@@ -332,11 +298,12 @@ return redis.status_reply("OK")
|
||||
`)
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:active
|
||||
// KEYS[2] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[2] -> asynq:{<qname>}:lease
|
||||
// KEYS[3] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[4] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
||||
// KEYS[5] -> asynq:{<qname>}:processed
|
||||
// KEYS[6] -> unique key
|
||||
// -------
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> stats expiration timestamp
|
||||
// ARGV[3] -> max int64 value
|
||||
@@ -368,14 +335,13 @@ return redis.status_reply("OK")
|
||||
|
||||
// Done removes the task from active queue and deletes the task.
|
||||
// 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"
|
||||
ctx := context.Background()
|
||||
now := r.clock.Now()
|
||||
expireAt := now.Add(statsTTL)
|
||||
keys := []string{
|
||||
base.ActiveKey(msg.Queue),
|
||||
base.DeadlinesKey(msg.Queue),
|
||||
base.LeaseKey(msg.Queue),
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
base.ProcessedKey(msg.Queue, now),
|
||||
base.ProcessedTotalKey(msg.Queue),
|
||||
@@ -383,7 +349,7 @@ func (r *RDB) Done(msg *base.TaskMessage) error {
|
||||
argv := []interface{}{
|
||||
msg.ID,
|
||||
expireAt.Unix(),
|
||||
math.MaxInt64,
|
||||
int64(math.MaxInt64),
|
||||
}
|
||||
// Note: We cannot pass empty unique key when running this script in redis-cluster.
|
||||
if len(msg.UniqueKey) > 0 {
|
||||
@@ -394,7 +360,7 @@ func (r *RDB) Done(msg *base.TaskMessage) error {
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:active
|
||||
// KEYS[2] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[2] -> asynq:{<qname>}:lease
|
||||
// KEYS[3] -> asynq:{<qname>}:completed
|
||||
// KEYS[4] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
||||
@@ -430,7 +396,7 @@ return redis.status_reply("OK")
|
||||
`)
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:active
|
||||
// KEYS[2] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[2] -> asynq:{<qname>}:lease
|
||||
// KEYS[3] -> asynq:{<qname>}:completed
|
||||
// KEYS[4] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
||||
@@ -471,9 +437,8 @@ return redis.status_reply("OK")
|
||||
|
||||
// MarkAsComplete removes the task from active queue to mark the task as completed.
|
||||
// It removes a uniqueness lock acquired by the task, if any.
|
||||
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"
|
||||
ctx := context.Background()
|
||||
now := r.clock.Now()
|
||||
statsExpireAt := now.Add(statsTTL)
|
||||
msg.CompletedAt = now.Unix()
|
||||
@@ -483,7 +448,7 @@ func (r *RDB) MarkAsComplete(msg *base.TaskMessage) error {
|
||||
}
|
||||
keys := []string{
|
||||
base.ActiveKey(msg.Queue),
|
||||
base.DeadlinesKey(msg.Queue),
|
||||
base.LeaseKey(msg.Queue),
|
||||
base.CompletedKey(msg.Queue),
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
base.ProcessedKey(msg.Queue, now),
|
||||
@@ -494,7 +459,7 @@ func (r *RDB) MarkAsComplete(msg *base.TaskMessage) error {
|
||||
statsExpireAt.Unix(),
|
||||
now.Unix() + msg.Retention,
|
||||
encoded,
|
||||
math.MaxInt64,
|
||||
int64(math.MaxInt64),
|
||||
}
|
||||
// Note: We cannot pass empty unique key when running this script in redis-cluster.
|
||||
if len(msg.UniqueKey) > 0 {
|
||||
@@ -505,7 +470,7 @@ func (r *RDB) MarkAsComplete(msg *base.TaskMessage) error {
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:active
|
||||
// KEYS[2] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[2] -> asynq:{<qname>}:lease
|
||||
// KEYS[3] -> asynq:{<qname>}:pending
|
||||
// KEYS[4] -> asynq:{<qname>}:t:<task_id>
|
||||
// ARGV[1] -> task ID
|
||||
@@ -522,25 +487,145 @@ redis.call("HSET", KEYS[4], "state", "pending")
|
||||
return redis.status_reply("OK")`)
|
||||
|
||||
// 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"
|
||||
ctx := context.Background()
|
||||
keys := []string{
|
||||
base.ActiveKey(msg.Queue),
|
||||
base.DeadlinesKey(msg.Queue),
|
||||
base.LeaseKey(msg.Queue),
|
||||
base.PendingKey(msg.Queue),
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
}
|
||||
return r.runScript(ctx, op, requeueCmd, keys, msg.ID)
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:g:<group_key>
|
||||
// KEYS[3] -> asynq:{<qname>}:groups
|
||||
// -------
|
||||
// ARGV[1] -> task message data
|
||||
// ARGV[2] -> task ID
|
||||
// ARGV[3] -> current time in Unix time
|
||||
// ARGV[4] -> group key
|
||||
//
|
||||
// Output:
|
||||
// Returns 1 if successfully added
|
||||
// Returns 0 if task ID already exists
|
||||
var addToGroupCmd = redis.NewScript(`
|
||||
if redis.call("EXISTS", KEYS[1]) == 1 then
|
||||
return 0
|
||||
end
|
||||
redis.call("HSET", KEYS[1],
|
||||
"msg", ARGV[1],
|
||||
"state", "aggregating",
|
||||
"group", ARGV[4])
|
||||
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
|
||||
redis.call("SADD", KEYS[3], ARGV[4])
|
||||
return 1
|
||||
`)
|
||||
|
||||
func (r *RDB) AddToGroup(ctx context.Context, msg *base.TaskMessage, groupKey string) error {
|
||||
var op errors.Op = "rdb.AddToGroup"
|
||||
encoded, err := base.EncodeMessage(msg)
|
||||
if err != nil {
|
||||
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
||||
}
|
||||
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
||||
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
||||
}
|
||||
keys := []string{
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
base.GroupKey(msg.Queue, groupKey),
|
||||
base.AllGroups(msg.Queue),
|
||||
}
|
||||
argv := []interface{}{
|
||||
encoded,
|
||||
msg.ID,
|
||||
r.clock.Now().Unix(),
|
||||
groupKey,
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(ctx, op, addToGroupCmd, keys, argv...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if n == 0 {
|
||||
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:g:<group_key>
|
||||
// KEYS[3] -> asynq:{<qname>}:groups
|
||||
// KEYS[4] -> unique key
|
||||
// -------
|
||||
// ARGV[1] -> task message data
|
||||
// ARGV[2] -> task ID
|
||||
// ARGV[3] -> current time in Unix time
|
||||
// ARGV[4] -> group key
|
||||
// ARGV[5] -> uniqueness lock TTL
|
||||
//
|
||||
// Output:
|
||||
// Returns 1 if successfully added
|
||||
// Returns 0 if task ID already exists
|
||||
// Returns -1 if task unique key already exists
|
||||
var addToGroupUniqueCmd = redis.NewScript(`
|
||||
local ok = redis.call("SET", KEYS[4], ARGV[2], "NX", "EX", ARGV[5])
|
||||
if not ok then
|
||||
return -1
|
||||
end
|
||||
if redis.call("EXISTS", KEYS[1]) == 1 then
|
||||
return 0
|
||||
end
|
||||
redis.call("HSET", KEYS[1],
|
||||
"msg", ARGV[1],
|
||||
"state", "aggregating",
|
||||
"group", ARGV[4])
|
||||
redis.call("ZADD", KEYS[2], ARGV[3], ARGV[2])
|
||||
redis.call("SADD", KEYS[3], ARGV[4])
|
||||
return 1
|
||||
`)
|
||||
|
||||
func (r *RDB) AddToGroupUnique(ctx context.Context, msg *base.TaskMessage, groupKey string, ttl time.Duration) error {
|
||||
var op errors.Op = "rdb.AddToGroupUnique"
|
||||
encoded, err := base.EncodeMessage(msg)
|
||||
if err != nil {
|
||||
return errors.E(op, errors.Unknown, fmt.Sprintf("cannot encode message: %v", err))
|
||||
}
|
||||
if err := r.client.SAdd(ctx, base.AllQueues, msg.Queue).Err(); err != nil {
|
||||
return errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "sadd", Err: err})
|
||||
}
|
||||
keys := []string{
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
base.GroupKey(msg.Queue, groupKey),
|
||||
base.AllGroups(msg.Queue),
|
||||
base.UniqueKey(msg.Queue, msg.Type, msg.Payload),
|
||||
}
|
||||
argv := []interface{}{
|
||||
encoded,
|
||||
msg.ID,
|
||||
r.clock.Now().Unix(),
|
||||
groupKey,
|
||||
int(ttl.Seconds()),
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(ctx, op, addToGroupUniqueCmd, keys, argv...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if n == -1 {
|
||||
return errors.E(op, errors.AlreadyExists, errors.ErrDuplicateTask)
|
||||
}
|
||||
if n == 0 {
|
||||
return errors.E(op, errors.AlreadyExists, errors.ErrTaskIdConflict)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:scheduled
|
||||
// -------
|
||||
// ARGV[1] -> task message data
|
||||
// ARGV[2] -> process_at time in Unix time
|
||||
// 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:
|
||||
// Returns 1 if successfully enqueued
|
||||
@@ -551,9 +636,7 @@ if redis.call("EXISTS", KEYS[1]) == 1 then
|
||||
end
|
||||
redis.call("HSET", KEYS[1],
|
||||
"msg", ARGV[1],
|
||||
"state", "scheduled",
|
||||
"timeout", ARGV[4],
|
||||
"deadline", ARGV[5])
|
||||
"state", "scheduled")
|
||||
redis.call("ZADD", KEYS[2], ARGV[2], ARGV[3])
|
||||
return 1
|
||||
`)
|
||||
@@ -576,8 +659,6 @@ func (r *RDB) Schedule(ctx context.Context, msg *base.TaskMessage, processAt tim
|
||||
encoded,
|
||||
processAt.Unix(),
|
||||
msg.ID,
|
||||
msg.Timeout,
|
||||
msg.Deadline,
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(ctx, op, scheduleCmd, keys, argv...)
|
||||
if err != nil {
|
||||
@@ -592,12 +673,11 @@ func (r *RDB) Schedule(ctx context.Context, msg *base.TaskMessage, processAt tim
|
||||
// KEYS[1] -> unique key
|
||||
// KEYS[2] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[3] -> asynq:{<qname>}:scheduled
|
||||
// -------
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> uniqueness lock TTL
|
||||
// ARGV[3] -> score (process_at timestamp)
|
||||
// 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:
|
||||
// Returns 1 if successfully scheduled
|
||||
@@ -614,8 +694,6 @@ end
|
||||
redis.call("HSET", KEYS[2],
|
||||
"msg", ARGV[4],
|
||||
"state", "scheduled",
|
||||
"timeout", ARGV[5],
|
||||
"deadline", ARGV[6],
|
||||
"unique_key", KEYS[1])
|
||||
redis.call("ZADD", KEYS[3], ARGV[3], ARGV[1])
|
||||
return 1
|
||||
@@ -642,8 +720,6 @@ func (r *RDB) ScheduleUnique(ctx context.Context, msg *base.TaskMessage, process
|
||||
int(ttl.Seconds()),
|
||||
processAt.Unix(),
|
||||
encoded,
|
||||
msg.Timeout,
|
||||
msg.Deadline,
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(ctx, op, scheduleUniqueCmd, keys, argv...)
|
||||
if err != nil {
|
||||
@@ -660,13 +736,13 @@ func (r *RDB) ScheduleUnique(ctx context.Context, msg *base.TaskMessage, process
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:active
|
||||
// KEYS[3] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[3] -> asynq:{<qname>}:lease
|
||||
// KEYS[4] -> asynq:{<qname>}:retry
|
||||
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
||||
// KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd>
|
||||
// KEYS[7] -> asynq:{<qname>}:processed
|
||||
// KEYS[8] -> asynq:{<qname>}:failed
|
||||
//
|
||||
// -------
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> updated base.TaskMessage value
|
||||
// ARGV[3] -> retry_at UNIX timestamp
|
||||
@@ -705,9 +781,8 @@ return redis.status_reply("OK")`)
|
||||
// Retry moves the task from active to retry queue.
|
||||
// It also annotates the message with the given error message and
|
||||
// if isFailure is true increments the retried counter.
|
||||
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"
|
||||
ctx := context.Background()
|
||||
now := r.clock.Now()
|
||||
modified := *msg
|
||||
if isFailure {
|
||||
@@ -723,7 +798,7 @@ func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string, i
|
||||
keys := []string{
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
base.ActiveKey(msg.Queue),
|
||||
base.DeadlinesKey(msg.Queue),
|
||||
base.LeaseKey(msg.Queue),
|
||||
base.RetryKey(msg.Queue),
|
||||
base.ProcessedKey(msg.Queue, now),
|
||||
base.FailedKey(msg.Queue, now),
|
||||
@@ -736,7 +811,7 @@ func (r *RDB) Retry(msg *base.TaskMessage, processAt time.Time, errMsg string, i
|
||||
processAt.Unix(),
|
||||
expireAt.Unix(),
|
||||
isFailure,
|
||||
math.MaxInt64,
|
||||
int64(math.MaxInt64),
|
||||
}
|
||||
return r.runScript(ctx, op, retryCmd, keys, argv...)
|
||||
}
|
||||
@@ -748,13 +823,13 @@ const (
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:t:<task_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:active
|
||||
// KEYS[3] -> asynq:{<qname>}:deadlines
|
||||
// KEYS[3] -> asynq:{<qname>}:lease
|
||||
// KEYS[4] -> asynq:{<qname>}:archived
|
||||
// KEYS[5] -> asynq:{<qname>}:processed:<yyyy-mm-dd>
|
||||
// KEYS[6] -> asynq:{<qname>}:failed:<yyyy-mm-dd>
|
||||
// KEYS[7] -> asynq:{<qname>}:processed
|
||||
// KEYS[8] -> asynq:{<qname>}:failed
|
||||
//
|
||||
// -------
|
||||
// ARGV[1] -> task ID
|
||||
// ARGV[2] -> updated base.TaskMessage value
|
||||
// ARGV[3] -> died_at UNIX timestamp
|
||||
@@ -793,9 +868,8 @@ return redis.status_reply("OK")`)
|
||||
|
||||
// Archive sends the given task to archive, attaching the error message to the task.
|
||||
// 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"
|
||||
ctx := context.Background()
|
||||
now := r.clock.Now()
|
||||
modified := *msg
|
||||
modified.ErrorMsg = errMsg
|
||||
@@ -809,7 +883,7 @@ func (r *RDB) Archive(msg *base.TaskMessage, errMsg string) error {
|
||||
keys := []string{
|
||||
base.TaskKey(msg.Queue, msg.ID),
|
||||
base.ActiveKey(msg.Queue),
|
||||
base.DeadlinesKey(msg.Queue),
|
||||
base.LeaseKey(msg.Queue),
|
||||
base.ArchivedKey(msg.Queue),
|
||||
base.ProcessedKey(msg.Queue, now),
|
||||
base.FailedKey(msg.Queue, now),
|
||||
@@ -823,7 +897,7 @@ func (r *RDB) Archive(msg *base.TaskMessage, errMsg string) error {
|
||||
cutoff.Unix(),
|
||||
maxArchiveSize,
|
||||
expireAt.Unix(),
|
||||
math.MaxInt64,
|
||||
int64(math.MaxInt64),
|
||||
}
|
||||
return r.runScript(ctx, op, archiveCmd, keys, argv...)
|
||||
}
|
||||
@@ -845,24 +919,41 @@ func (r *RDB) ForwardIfReady(qnames ...string) error {
|
||||
// ARGV[1] -> current unix time in seconds
|
||||
// ARGV[2] -> task key prefix
|
||||
// ARGV[3] -> current unix time in nsec
|
||||
// ARGV[4] -> group key prefix
|
||||
// Note: Script moves tasks up to 100 at a time to keep the runtime of script short.
|
||||
var forwardCmd = redis.NewScript(`
|
||||
local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1], "LIMIT", 0, 100)
|
||||
for _, id in ipairs(ids) do
|
||||
redis.call("LPUSH", KEYS[2], id)
|
||||
redis.call("ZREM", KEYS[1], id)
|
||||
redis.call("HSET", ARGV[2] .. id,
|
||||
"state", "pending",
|
||||
"pending_since", ARGV[3])
|
||||
local taskKey = ARGV[2] .. id
|
||||
local group = redis.call("HGET", taskKey, "group")
|
||||
if group and group ~= '' then
|
||||
redis.call("ZADD", ARGV[4] .. group, ARGV[1], id)
|
||||
redis.call("ZREM", KEYS[1], id)
|
||||
redis.call("HSET", taskKey,
|
||||
"state", "aggregating")
|
||||
else
|
||||
redis.call("LPUSH", KEYS[2], id)
|
||||
redis.call("ZREM", KEYS[1], id)
|
||||
redis.call("HSET", taskKey,
|
||||
"state", "pending",
|
||||
"pending_since", ARGV[3])
|
||||
end
|
||||
end
|
||||
return table.getn(ids)`)
|
||||
|
||||
// forward moves tasks with a score less than the current unix time
|
||||
// from the src zset to the dst list. It returns the number of tasks moved.
|
||||
func (r *RDB) forward(src, dst, taskKeyPrefix string) (int, error) {
|
||||
// forward moves tasks with a score less than the current unix time from the delayed (i.e. scheduled | retry) zset
|
||||
// to the pending list or group set.
|
||||
// It returns the number of tasks moved.
|
||||
func (r *RDB) forward(delayedKey, pendingKey, taskKeyPrefix, groupKeyPrefix string) (int, error) {
|
||||
now := r.clock.Now()
|
||||
res, err := forwardCmd.Run(context.Background(), r.client,
|
||||
[]string{src, dst}, now.Unix(), taskKeyPrefix, now.UnixNano()).Result()
|
||||
keys := []string{delayedKey, pendingKey}
|
||||
argv := []interface{}{
|
||||
now.Unix(),
|
||||
taskKeyPrefix,
|
||||
now.UnixNano(),
|
||||
groupKeyPrefix,
|
||||
}
|
||||
res, err := forwardCmd.Run(context.Background(), r.client, keys, argv...).Result()
|
||||
if err != nil {
|
||||
return 0, errors.E(errors.Internal, fmt.Sprintf("redis eval error: %v", err))
|
||||
}
|
||||
@@ -874,15 +965,16 @@ func (r *RDB) forward(src, dst, taskKeyPrefix string) (int, error) {
|
||||
}
|
||||
|
||||
// forwardAll checks for tasks in scheduled/retry state that are ready to be run, and updates
|
||||
// their state to "pending".
|
||||
// their state to "pending" or "aggregating".
|
||||
func (r *RDB) forwardAll(qname string) (err error) {
|
||||
sources := []string{base.ScheduledKey(qname), base.RetryKey(qname)}
|
||||
dst := base.PendingKey(qname)
|
||||
delayedKeys := []string{base.ScheduledKey(qname), base.RetryKey(qname)}
|
||||
pendingKey := base.PendingKey(qname)
|
||||
taskKeyPrefix := base.TaskKeyPrefix(qname)
|
||||
for _, src := range sources {
|
||||
groupKeyPrefix := base.GroupKeyPrefix(qname)
|
||||
for _, delayedKey := range delayedKeys {
|
||||
n := 1
|
||||
for n != 0 {
|
||||
n, err = r.forward(src, dst, taskKeyPrefix)
|
||||
n, err = r.forward(delayedKey, pendingKey, taskKeyPrefix, groupKeyPrefix)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -891,6 +983,248 @@ func (r *RDB) forwardAll(qname string) (err error) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// ListGroups returns a list of all known groups in the given queue.
|
||||
func (r *RDB) ListGroups(qname string) ([]string, error) {
|
||||
var op errors.Op = "RDB.ListGroups"
|
||||
groups, err := r.client.SMembers(context.Background(), base.AllGroups(qname)).Result()
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "smembers", Err: err})
|
||||
}
|
||||
return groups, nil
|
||||
}
|
||||
|
||||
// aggregationCheckCmd checks the given group for whether to create an aggregation set.
|
||||
// An aggregation set is created if one of the aggregation criteria is met:
|
||||
// 1) group has reached or exceeded its max size
|
||||
// 2) group's oldest task has reached or exceeded its max delay
|
||||
// 3) group's latest task has reached or exceeded its grace period
|
||||
// if aggreation criteria is met, the command moves those tasks from the group
|
||||
// and put them in an aggregation set. Additionally, if the creation of aggregation set
|
||||
// empties the group, it will clear the group name from the all groups set.
|
||||
//
|
||||
// KEYS[1] -> asynq:{<qname>}:g:<gname>
|
||||
// KEYS[2] -> asynq:{<qname>}:g:<gname>:<aggregation_set_id>
|
||||
// KEYS[3] -> asynq:{<qname>}:aggregation_sets
|
||||
// KEYS[4] -> asynq:{<qname>}:groups
|
||||
// -------
|
||||
// ARGV[1] -> max group size
|
||||
// ARGV[2] -> max group delay in unix time
|
||||
// ARGV[3] -> start time of the grace period
|
||||
// ARGV[4] -> aggregation set expire time
|
||||
// ARGV[5] -> current time in unix time
|
||||
// ARGV[6] -> group name
|
||||
//
|
||||
// Output:
|
||||
// Returns 0 if no aggregation set was created
|
||||
// Returns 1 if an aggregation set was created
|
||||
//
|
||||
// Time Complexity:
|
||||
// O(log(N) + M) with N being the number tasks in the group zset
|
||||
// and M being the max size.
|
||||
var aggregationCheckCmd = redis.NewScript(`
|
||||
local size = redis.call("ZCARD", KEYS[1])
|
||||
if size == 0 then
|
||||
return 0
|
||||
end
|
||||
local maxSize = tonumber(ARGV[1])
|
||||
if maxSize ~= 0 and size >= maxSize then
|
||||
local res = redis.call("ZRANGE", KEYS[1], 0, maxSize-1, "WITHSCORES")
|
||||
for i=1, table.getn(res)-1, 2 do
|
||||
redis.call("ZADD", KEYS[2], tonumber(res[i+1]), res[i])
|
||||
end
|
||||
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, maxSize-1)
|
||||
redis.call("ZADD", KEYS[3], ARGV[4], KEYS[2])
|
||||
if size == maxSize then
|
||||
redis.call("SREM", KEYS[4], ARGV[6])
|
||||
end
|
||||
return 1
|
||||
end
|
||||
local maxDelay = tonumber(ARGV[2])
|
||||
local currentTime = tonumber(ARGV[5])
|
||||
if maxDelay ~= 0 then
|
||||
local oldestEntry = redis.call("ZRANGE", KEYS[1], 0, 0, "WITHSCORES")
|
||||
local oldestEntryScore = tonumber(oldestEntry[2])
|
||||
local maxDelayTime = currentTime - maxDelay
|
||||
if oldestEntryScore <= maxDelayTime then
|
||||
local res = redis.call("ZRANGE", KEYS[1], 0, maxSize-1, "WITHSCORES")
|
||||
for i=1, table.getn(res)-1, 2 do
|
||||
redis.call("ZADD", KEYS[2], tonumber(res[i+1]), res[i])
|
||||
end
|
||||
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, maxSize-1)
|
||||
redis.call("ZADD", KEYS[3], ARGV[4], KEYS[2])
|
||||
if size <= maxSize or maxSize == 0 then
|
||||
redis.call("SREM", KEYS[4], ARGV[6])
|
||||
end
|
||||
return 1
|
||||
end
|
||||
end
|
||||
local latestEntry = redis.call("ZREVRANGE", KEYS[1], 0, 0, "WITHSCORES")
|
||||
local latestEntryScore = tonumber(latestEntry[2])
|
||||
local gracePeriodStartTime = currentTime - tonumber(ARGV[3])
|
||||
if latestEntryScore <= gracePeriodStartTime then
|
||||
local res = redis.call("ZRANGE", KEYS[1], 0, maxSize-1, "WITHSCORES")
|
||||
for i=1, table.getn(res)-1, 2 do
|
||||
redis.call("ZADD", KEYS[2], tonumber(res[i+1]), res[i])
|
||||
end
|
||||
redis.call("ZREMRANGEBYRANK", KEYS[1], 0, maxSize-1)
|
||||
redis.call("ZADD", KEYS[3], ARGV[4], KEYS[2])
|
||||
if size <= maxSize or maxSize == 0 then
|
||||
redis.call("SREM", KEYS[4], ARGV[6])
|
||||
end
|
||||
return 1
|
||||
end
|
||||
return 0
|
||||
`)
|
||||
|
||||
// Task aggregation should finish within this timeout.
|
||||
// Otherwise an aggregation set should be reclaimed by the recoverer.
|
||||
const aggregationTimeout = 2 * time.Minute
|
||||
|
||||
// AggregationCheck checks the group identified by the given queue and group name to see if the tasks in the
|
||||
// group are ready to be aggregated. If so, it moves the tasks to be aggregated to a aggregation set and returns
|
||||
// the set ID. If not, it returns an empty string for the set ID.
|
||||
// The time for gracePeriod and maxDelay is computed relative to the time t.
|
||||
//
|
||||
// Note: It assumes that this function is called at frequency less than or equal to the gracePeriod. In other words,
|
||||
// the function only checks the most recently added task aganist the given gracePeriod.
|
||||
func (r *RDB) AggregationCheck(qname, gname string, t time.Time, gracePeriod, maxDelay time.Duration, maxSize int) (string, error) {
|
||||
var op errors.Op = "RDB.AggregationCheck"
|
||||
aggregationSetID := uuid.NewString()
|
||||
expireTime := r.clock.Now().Add(aggregationTimeout)
|
||||
keys := []string{
|
||||
base.GroupKey(qname, gname),
|
||||
base.AggregationSetKey(qname, gname, aggregationSetID),
|
||||
base.AllAggregationSets(qname),
|
||||
base.AllGroups(qname),
|
||||
}
|
||||
argv := []interface{}{
|
||||
maxSize,
|
||||
int64(maxDelay.Seconds()),
|
||||
int64(gracePeriod.Seconds()),
|
||||
expireTime.Unix(),
|
||||
t.Unix(),
|
||||
gname,
|
||||
}
|
||||
n, err := r.runScriptWithErrorCode(context.Background(), op, aggregationCheckCmd, keys, argv...)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
switch n {
|
||||
case 0:
|
||||
return "", nil
|
||||
case 1:
|
||||
return aggregationSetID, nil
|
||||
default:
|
||||
return "", errors.E(op, errors.Internal, fmt.Sprintf("unexpected return value from lua script: %d", n))
|
||||
}
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:g:<gname>:<aggregation_set_id>
|
||||
// ------
|
||||
// ARGV[1] -> task key prefix
|
||||
//
|
||||
// Output:
|
||||
// Array of encoded task messages
|
||||
//
|
||||
// Time Complexity:
|
||||
// O(N) with N being the number of tasks in the aggregation set.
|
||||
var readAggregationSetCmd = redis.NewScript(`
|
||||
local msgs = {}
|
||||
local ids = redis.call("ZRANGE", KEYS[1], 0, -1)
|
||||
for _, id in ipairs(ids) do
|
||||
local key = ARGV[1] .. id
|
||||
table.insert(msgs, redis.call("HGET", key, "msg"))
|
||||
end
|
||||
return msgs
|
||||
`)
|
||||
|
||||
// ReadAggregationSet retrieves members of an aggregation set and returns a list of tasks in the set and
|
||||
// the deadline for aggregating those tasks.
|
||||
func (r *RDB) ReadAggregationSet(qname, gname, setID string) ([]*base.TaskMessage, time.Time, error) {
|
||||
var op errors.Op = "RDB.ReadAggregationSet"
|
||||
ctx := context.Background()
|
||||
aggSetKey := base.AggregationSetKey(qname, gname, setID)
|
||||
res, err := readAggregationSetCmd.Run(ctx, r.client,
|
||||
[]string{aggSetKey}, base.TaskKeyPrefix(qname)).Result()
|
||||
if err != nil {
|
||||
return nil, time.Time{}, errors.E(op, errors.Unknown, fmt.Sprintf("redis eval error: %v", err))
|
||||
}
|
||||
data, err := cast.ToStringSliceE(res)
|
||||
if err != nil {
|
||||
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cast error: Lua script returned unexpected value: %v", res))
|
||||
}
|
||||
var msgs []*base.TaskMessage
|
||||
for _, s := range data {
|
||||
msg, err := base.DecodeMessage([]byte(s))
|
||||
if err != nil {
|
||||
return nil, time.Time{}, errors.E(op, errors.Internal, fmt.Sprintf("cannot decode message: %v", err))
|
||||
}
|
||||
msgs = append(msgs, msg)
|
||||
}
|
||||
deadlineUnix, err := r.client.ZScore(ctx, base.AllAggregationSets(qname), aggSetKey).Result()
|
||||
if err != nil {
|
||||
return nil, time.Time{}, errors.E(op, errors.Unknown, &errors.RedisCommandError{Command: "zscore", Err: err})
|
||||
}
|
||||
return msgs, time.Unix(int64(deadlineUnix), 0), nil
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:g:<gname>:<aggregation_set_id>
|
||||
// KEYS[2] -> asynq:{<qname>}:aggregation_sets
|
||||
// -------
|
||||
// ARGV[1] -> task key prefix
|
||||
//
|
||||
// Output:
|
||||
// Redis status reply
|
||||
//
|
||||
// Time Complexity:
|
||||
// max(O(N), O(log(M))) with N being the number of tasks in the aggregation set
|
||||
// and M being the number of elements in the all-aggregation-sets list.
|
||||
var deleteAggregationSetCmd = redis.NewScript(`
|
||||
local ids = redis.call("ZRANGE", KEYS[1], 0, -1)
|
||||
for _, id in ipairs(ids) do
|
||||
redis.call("DEL", ARGV[1] .. id)
|
||||
end
|
||||
redis.call("DEL", KEYS[1])
|
||||
redis.call("ZREM", KEYS[2], KEYS[1])
|
||||
return redis.status_reply("OK")
|
||||
`)
|
||||
|
||||
// DeleteAggregationSet deletes the aggregation set and its members identified by the parameters.
|
||||
func (r *RDB) DeleteAggregationSet(ctx context.Context, qname, gname, setID string) error {
|
||||
var op errors.Op = "RDB.DeleteAggregationSet"
|
||||
keys := []string{
|
||||
base.AggregationSetKey(qname, gname, setID),
|
||||
base.AllAggregationSets(qname),
|
||||
}
|
||||
return r.runScript(ctx, op, deleteAggregationSetCmd, keys, base.TaskKeyPrefix(qname))
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:aggregation_sets
|
||||
// -------
|
||||
// ARGV[1] -> current time in unix time
|
||||
var reclaimStateAggregationSetsCmd = redis.NewScript(`
|
||||
local staleSetKeys = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
|
||||
for _, key in ipairs(staleSetKeys) do
|
||||
local idx = string.find(key, ":[^:]*$")
|
||||
local groupKey = string.sub(key, 1, idx-1)
|
||||
local res = redis.call("ZRANGE", key, 0, -1, "WITHSCORES")
|
||||
for i=1, table.getn(res)-1, 2 do
|
||||
redis.call("ZADD", groupKey, tonumber(res[i+1]), res[i])
|
||||
end
|
||||
redis.call("DEL", key)
|
||||
end
|
||||
redis.call("ZREMRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
|
||||
return redis.status_reply("OK")
|
||||
`)
|
||||
|
||||
// ReclaimStateAggregationSets checks for any stale aggregation sets in the given queue, and
|
||||
// reclaim tasks in the stale aggregation set by putting them back in the group.
|
||||
func (r *RDB) ReclaimStaleAggregationSets(qname string) error {
|
||||
var op errors.Op = "RDB.ReclaimStaleAggregationSets"
|
||||
return r.runScript(context.Background(), op, reclaimStateAggregationSetsCmd,
|
||||
[]string{base.AllAggregationSets(qname)}, r.clock.Now().Unix())
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:completed
|
||||
// ARGV[1] -> current time in unix time
|
||||
// ARGV[2] -> task key prefix
|
||||
@@ -942,10 +1276,10 @@ func (r *RDB) deleteExpiredCompletedTasks(qname string, batchSize int) (int64, e
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:{<qname>}:deadlines
|
||||
// ARGV[1] -> deadline in unix time
|
||||
// KEYS[1] -> asynq:{<qname>}:lease
|
||||
// ARGV[1] -> cutoff in unix time
|
||||
// ARGV[2] -> task key prefix
|
||||
var listDeadlineExceededCmd = redis.NewScript(`
|
||||
var listLeaseExpiredCmd = redis.NewScript(`
|
||||
local res = {}
|
||||
local ids = redis.call("ZRANGEBYSCORE", KEYS[1], "-inf", ARGV[1])
|
||||
for _, id in ipairs(ids) do
|
||||
@@ -955,14 +1289,14 @@ end
|
||||
return res
|
||||
`)
|
||||
|
||||
// ListDeadlineExceeded returns a list of task messages that have exceeded the deadline from the given queues.
|
||||
func (r *RDB) ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*base.TaskMessage, error) {
|
||||
var op errors.Op = "rdb.ListDeadlineExceeded"
|
||||
// ListLeaseExpired returns a list of task messages with an expired lease from the given queues.
|
||||
func (r *RDB) ListLeaseExpired(cutoff time.Time, qnames ...string) ([]*base.TaskMessage, error) {
|
||||
var op errors.Op = "rdb.ListLeaseExpired"
|
||||
var msgs []*base.TaskMessage
|
||||
for _, qname := range qnames {
|
||||
res, err := listDeadlineExceededCmd.Run(context.Background(), r.client,
|
||||
[]string{base.DeadlinesKey(qname)},
|
||||
deadline.Unix(), base.TaskKeyPrefix(qname)).Result()
|
||||
res, err := listLeaseExpiredCmd.Run(context.Background(), r.client,
|
||||
[]string{base.LeaseKey(qname)},
|
||||
cutoff.Unix(), base.TaskKeyPrefix(qname)).Result()
|
||||
if err != nil {
|
||||
return nil, errors.E(op, errors.Internal, fmt.Sprintf("redis eval error: %v", err))
|
||||
}
|
||||
@@ -981,6 +1315,23 @@ func (r *RDB) ListDeadlineExceeded(deadline time.Time, qnames ...string) ([]*bas
|
||||
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
|
||||
// TODO: Consider adding GT option to ensure we only "extend" the lease. Ceveat is that GT is supported from redis v6.2.0 or above.
|
||||
err = r.client.ZAddXX(context.Background(), base.LeaseKey(qname), zs...).Err()
|
||||
if err != nil {
|
||||
return time.Time{}, err
|
||||
}
|
||||
return expireAt, nil
|
||||
}
|
||||
|
||||
// KEYS[1] -> asynq:servers:{<host:pid:sid>}
|
||||
// KEYS[2] -> asynq:workers:{<host:pid:sid>}
|
||||
// ARGV[1] -> TTL in seconds
|
||||
|
File diff suppressed because it is too large
Load Diff
@@ -15,7 +15,7 @@ import (
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
)
|
||||
|
||||
var errRedisDown = errors.New("asynqtest: redis is down")
|
||||
var errRedisDown = errors.New("testutil: redis is down")
|
||||
|
||||
// TestBroker is a broker implementation which enables
|
||||
// to simulate Redis failure in tests.
|
||||
@@ -73,31 +73,31 @@ func (tb *TestBroker) Dequeue(qnames ...string) (*base.TaskMessage, time.Time, e
|
||||
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()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
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()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
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()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
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 {
|
||||
@@ -118,22 +118,22 @@ func (tb *TestBroker) ScheduleUnique(ctx context.Context, msg *base.TaskMessage,
|
||||
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()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
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()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return errRedisDown
|
||||
}
|
||||
return tb.real.Archive(msg, errMsg)
|
||||
return tb.real.Archive(ctx, msg, errMsg)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) ForwardIfReady(qnames ...string) error {
|
||||
@@ -154,13 +154,22 @@ func (tb *TestBroker) DeleteExpiredCompletedTasks(qname string) error {
|
||||
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()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
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 {
|
||||
@@ -225,3 +234,66 @@ func (tb *TestBroker) Close() error {
|
||||
}
|
||||
return tb.real.Close()
|
||||
}
|
||||
|
||||
func (tb *TestBroker) AddToGroup(ctx context.Context, msg *base.TaskMessage, gname string) error {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return errRedisDown
|
||||
}
|
||||
return tb.real.AddToGroup(ctx, msg, gname)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) AddToGroupUnique(ctx context.Context, msg *base.TaskMessage, gname string, ttl time.Duration) error {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return errRedisDown
|
||||
}
|
||||
return tb.real.AddToGroupUnique(ctx, msg, gname, ttl)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) ListGroups(qname string) ([]string, error) {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return nil, errRedisDown
|
||||
}
|
||||
return tb.real.ListGroups(qname)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) AggregationCheck(qname, gname string, t time.Time, gracePeriod, maxDelay time.Duration, maxSize int) (aggregationSetID string, err error) {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return "", errRedisDown
|
||||
}
|
||||
return tb.real.AggregationCheck(qname, gname, t, gracePeriod, maxDelay, maxSize)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) ReadAggregationSet(qname, gname, aggregationSetID string) ([]*base.TaskMessage, time.Time, error) {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return nil, time.Time{}, errRedisDown
|
||||
}
|
||||
return tb.real.ReadAggregationSet(qname, gname, aggregationSetID)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) DeleteAggregationSet(ctx context.Context, qname, gname, aggregationSetID string) error {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return errRedisDown
|
||||
}
|
||||
return tb.real.DeleteAggregationSet(ctx, qname, gname, aggregationSetID)
|
||||
}
|
||||
|
||||
func (tb *TestBroker) ReclaimStaleAggregationSets(qname string) error {
|
||||
tb.mu.Lock()
|
||||
defer tb.mu.Unlock()
|
||||
if tb.sleeping {
|
||||
return errRedisDown
|
||||
}
|
||||
return tb.real.ReclaimStaleAggregationSets(qname)
|
||||
}
|
||||
|
84
internal/testutil/builder.go
Normal file
84
internal/testutil/builder.go
Normal file
@@ -0,0 +1,84 @@
|
||||
// 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 testutil
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
)
|
||||
|
||||
func makeDefaultTaskMessage() *base.TaskMessage {
|
||||
return &base.TaskMessage{
|
||||
ID: uuid.NewString(),
|
||||
Type: "default_task",
|
||||
Queue: "default",
|
||||
Retry: 25,
|
||||
Timeout: 1800, // default timeout of 30 mins
|
||||
Deadline: 0, // no deadline
|
||||
}
|
||||
}
|
||||
|
||||
type TaskMessageBuilder struct {
|
||||
msg *base.TaskMessage
|
||||
}
|
||||
|
||||
func NewTaskMessageBuilder() *TaskMessageBuilder {
|
||||
return &TaskMessageBuilder{}
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) lazyInit() {
|
||||
if b.msg == nil {
|
||||
b.msg = makeDefaultTaskMessage()
|
||||
}
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) Build() *base.TaskMessage {
|
||||
b.lazyInit()
|
||||
return b.msg
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetType(typename string) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.Type = typename
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetPayload(payload []byte) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.Payload = payload
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetQueue(qname string) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.Queue = qname
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetRetry(n int) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.Retry = n
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetTimeout(timeout time.Duration) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.Timeout = int64(timeout.Seconds())
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetDeadline(deadline time.Time) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.Deadline = deadline.Unix()
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *TaskMessageBuilder) SetGroup(gname string) *TaskMessageBuilder {
|
||||
b.lazyInit()
|
||||
b.msg.GroupKey = gname
|
||||
return b
|
||||
}
|
94
internal/testutil/builder_test.go
Normal file
94
internal/testutil/builder_test.go
Normal file
@@ -0,0 +1,94 @@
|
||||
// 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 testutil
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
)
|
||||
|
||||
func TestTaskMessageBuilder(t *testing.T) {
|
||||
tests := []struct {
|
||||
desc string
|
||||
ops func(b *TaskMessageBuilder) // operations to perform on the builder
|
||||
want *base.TaskMessage
|
||||
}{
|
||||
{
|
||||
desc: "zero value and build",
|
||||
ops: nil,
|
||||
want: &base.TaskMessage{
|
||||
Type: "default_task",
|
||||
Queue: "default",
|
||||
Payload: nil,
|
||||
Retry: 25,
|
||||
Timeout: 1800, // 30m
|
||||
Deadline: 0,
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "with type, payload, and queue",
|
||||
ops: func(b *TaskMessageBuilder) {
|
||||
b.SetType("foo").SetPayload([]byte("hello")).SetQueue("myqueue")
|
||||
},
|
||||
want: &base.TaskMessage{
|
||||
Type: "foo",
|
||||
Queue: "myqueue",
|
||||
Payload: []byte("hello"),
|
||||
Retry: 25,
|
||||
Timeout: 1800, // 30m
|
||||
Deadline: 0,
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "with retry, timeout, and deadline",
|
||||
ops: func(b *TaskMessageBuilder) {
|
||||
b.SetRetry(1).
|
||||
SetTimeout(20 * time.Second).
|
||||
SetDeadline(time.Date(2017, 3, 6, 0, 0, 0, 0, time.UTC))
|
||||
},
|
||||
want: &base.TaskMessage{
|
||||
Type: "default_task",
|
||||
Queue: "default",
|
||||
Payload: nil,
|
||||
Retry: 1,
|
||||
Timeout: 20,
|
||||
Deadline: time.Date(2017, 3, 6, 0, 0, 0, 0, time.UTC).Unix(),
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "with group",
|
||||
ops: func(b *TaskMessageBuilder) {
|
||||
b.SetGroup("mygroup")
|
||||
},
|
||||
want: &base.TaskMessage{
|
||||
Type: "default_task",
|
||||
Queue: "default",
|
||||
Payload: nil,
|
||||
Retry: 25,
|
||||
Timeout: 1800,
|
||||
Deadline: 0,
|
||||
GroupKey: "mygroup",
|
||||
},
|
||||
},
|
||||
}
|
||||
cmpOpts := []cmp.Option{cmpopts.IgnoreFields(base.TaskMessage{}, "ID")}
|
||||
|
||||
for _, tc := range tests {
|
||||
var b TaskMessageBuilder
|
||||
if tc.ops != nil {
|
||||
tc.ops(&b)
|
||||
}
|
||||
|
||||
got := b.Build()
|
||||
if diff := cmp.Diff(tc.want, got, cmpOpts...); diff != "" {
|
||||
t.Errorf("%s: TaskMessageBuilder.Build() = %+v, want %+v;\n(-want,+got)\n%s",
|
||||
tc.desc, got, tc.want, diff)
|
||||
}
|
||||
}
|
||||
}
|
@@ -2,8 +2,8 @@
|
||||
// Use of this source code is governed by a MIT license
|
||||
// that can be found in the LICENSE file.
|
||||
|
||||
// Package asynqtest defines test helpers for asynq and its internal packages.
|
||||
package asynqtest
|
||||
// Package testutil defines test helpers for asynq and its internal packages.
|
||||
package testutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
@@ -18,6 +18,7 @@ import (
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
)
|
||||
|
||||
// EquateInt64Approx returns a Comparer option that treats int64 values
|
||||
@@ -92,6 +93,20 @@ var SortStringSliceOpt = cmp.Transformer("SortStringSlice", func(in []string) []
|
||||
return out
|
||||
})
|
||||
|
||||
var SortRedisZSetEntryOpt = cmp.Transformer("SortZSetEntries", func(in []redis.Z) []redis.Z {
|
||||
out := append([]redis.Z(nil), in...) // Copy input to avoid mutating it
|
||||
sort.Slice(out, func(i, j int) bool {
|
||||
// TODO: If member is a comparable type (int, string, etc) compare by the member
|
||||
// Use generic comparable type here once update to go1.18
|
||||
if _, ok := out[i].Member.(string); ok {
|
||||
// If member is a string, compare the member
|
||||
return out[i].Member.(string) < out[j].Member.(string)
|
||||
}
|
||||
return out[i].Score < out[j].Score
|
||||
})
|
||||
return out
|
||||
})
|
||||
|
||||
// IgnoreIDOpt is an cmp.Option to ignore ID field in task messages when comparing.
|
||||
var IgnoreIDOpt = cmpopts.IgnoreFields(base.TaskMessage{}, "ID")
|
||||
|
||||
@@ -114,6 +129,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.
|
||||
func JSON(kv map[string]interface{}) []byte {
|
||||
b, err := json.Marshal(kv)
|
||||
@@ -223,11 +245,11 @@ func SeedArchivedQueue(tb testing.TB, r redis.UniversalClient, entries []base.Z,
|
||||
seedRedisZSet(tb, r, base.ArchivedKey(qname), entries, base.TaskStateArchived)
|
||||
}
|
||||
|
||||
// SeedDeadlines initializes the deadlines set with the given entries.
|
||||
func SeedDeadlines(tb testing.TB, r redis.UniversalClient, entries []base.Z, qname string) {
|
||||
// SeedLease initializes the lease set with the given entries.
|
||||
func SeedLease(tb testing.TB, r redis.UniversalClient, entries []base.Z, qname string) {
|
||||
tb.Helper()
|
||||
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.
|
||||
@@ -237,6 +259,21 @@ func SeedCompletedQueue(tb testing.TB, r redis.UniversalClient, entries []base.Z
|
||||
seedRedisZSet(tb, r, base.CompletedKey(qname), entries, base.TaskStateCompleted)
|
||||
}
|
||||
|
||||
// SeedGroup initializes the group with the given entries.
|
||||
func SeedGroup(tb testing.TB, r redis.UniversalClient, entries []base.Z, qname, gname string) {
|
||||
tb.Helper()
|
||||
ctx := context.Background()
|
||||
r.SAdd(ctx, base.AllQueues, qname)
|
||||
r.SAdd(ctx, base.AllGroups(qname), gname)
|
||||
seedRedisZSet(tb, r, base.GroupKey(qname, gname), entries, base.TaskStateAggregating)
|
||||
}
|
||||
|
||||
func SeedAggregationSet(tb testing.TB, r redis.UniversalClient, entries []base.Z, qname, gname, setID string) {
|
||||
tb.Helper()
|
||||
r.SAdd(context.Background(), base.AllQueues, qname)
|
||||
seedRedisZSet(tb, r, base.AggregationSetKey(qname, gname, setID), entries, base.TaskStateAggregating)
|
||||
}
|
||||
|
||||
// SeedAllPendingQueues initializes all of the specified queues with the given messages.
|
||||
//
|
||||
// pending maps a queue name to a list of messages.
|
||||
@@ -279,11 +316,11 @@ func SeedAllArchivedQueues(tb testing.TB, r redis.UniversalClient, archived map[
|
||||
}
|
||||
}
|
||||
|
||||
// SeedAllDeadlines initializes all of the deadlines with the given entries.
|
||||
func SeedAllDeadlines(tb testing.TB, r redis.UniversalClient, deadlines map[string][]base.Z) {
|
||||
// SeedAllLease initializes all of the lease sets with the given entries.
|
||||
func SeedAllLease(tb testing.TB, r redis.UniversalClient, lease map[string][]base.Z) {
|
||||
tb.Helper()
|
||||
for q, entries := range deadlines {
|
||||
SeedDeadlines(tb, r, entries, q)
|
||||
for q, entries := range lease {
|
||||
SeedLease(tb, r, entries, q)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -295,6 +332,18 @@ func SeedAllCompletedQueues(tb testing.TB, r redis.UniversalClient, completed ma
|
||||
}
|
||||
}
|
||||
|
||||
// SeedAllGroups initializes all groups in all queues.
|
||||
// The map maps queue names to group names which maps to a list of task messages and the time it was
|
||||
// added to the group.
|
||||
func SeedAllGroups(tb testing.TB, r redis.UniversalClient, groups map[string]map[string][]base.Z) {
|
||||
tb.Helper()
|
||||
for qname, g := range groups {
|
||||
for gname, entries := range g {
|
||||
SeedGroup(tb, r, entries, qname, gname)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func seedRedisList(tb testing.TB, c redis.UniversalClient, key string,
|
||||
msgs []*base.TaskMessage, state base.TaskState) {
|
||||
tb.Helper()
|
||||
@@ -303,15 +352,14 @@ func seedRedisList(tb testing.TB, c redis.UniversalClient, key string,
|
||||
if err := c.LPush(context.Background(), key, msg.ID).Err(); err != nil {
|
||||
tb.Fatal(err)
|
||||
}
|
||||
key := base.TaskKey(msg.Queue, msg.ID)
|
||||
taskKey := base.TaskKey(msg.Queue, msg.ID)
|
||||
data := map[string]interface{}{
|
||||
"msg": encoded,
|
||||
"state": state.String(),
|
||||
"timeout": msg.Timeout,
|
||||
"deadline": msg.Deadline,
|
||||
"unique_key": msg.UniqueKey,
|
||||
"group": msg.GroupKey,
|
||||
}
|
||||
if err := c.HSet(context.Background(), key, data).Err(); err != nil {
|
||||
if err := c.HSet(context.Background(), taskKey, data).Err(); err != nil {
|
||||
tb.Fatal(err)
|
||||
}
|
||||
if len(msg.UniqueKey) > 0 {
|
||||
@@ -333,15 +381,14 @@ func seedRedisZSet(tb testing.TB, c redis.UniversalClient, key string,
|
||||
if err := c.ZAdd(context.Background(), key, z).Err(); err != nil {
|
||||
tb.Fatal(err)
|
||||
}
|
||||
key := base.TaskKey(msg.Queue, msg.ID)
|
||||
taskKey := base.TaskKey(msg.Queue, msg.ID)
|
||||
data := map[string]interface{}{
|
||||
"msg": encoded,
|
||||
"state": state.String(),
|
||||
"timeout": msg.Timeout,
|
||||
"deadline": msg.Deadline,
|
||||
"unique_key": msg.UniqueKey,
|
||||
"group": msg.GroupKey,
|
||||
}
|
||||
if err := c.HSet(context.Background(), key, data).Err(); err != nil {
|
||||
if err := c.HSet(context.Background(), taskKey, data).Err(); err != nil {
|
||||
tb.Fatal(err)
|
||||
}
|
||||
if len(msg.UniqueKey) > 0 {
|
||||
@@ -416,11 +463,11 @@ func GetArchivedEntries(tb testing.TB, r redis.UniversalClient, qname string) []
|
||||
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.
|
||||
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()
|
||||
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.
|
||||
@@ -430,6 +477,14 @@ func GetCompletedEntries(tb testing.TB, r redis.UniversalClient, qname string) [
|
||||
return getMessagesFromZSetWithScores(tb, r, qname, base.CompletedKey, base.TaskStateCompleted)
|
||||
}
|
||||
|
||||
// GetGroupEntries returns all scheduled messages and its score in the given queue.
|
||||
// It also asserts the state field of the task.
|
||||
func GetGroupEntries(tb testing.TB, r redis.UniversalClient, qname, groupKey string) []base.Z {
|
||||
tb.Helper()
|
||||
return getMessagesFromZSetWithScores(tb, r, qname,
|
||||
func(qname string) string { return base.GroupKey(qname, groupKey) }, base.TaskStateAggregating)
|
||||
}
|
||||
|
||||
// Retrieves all messages stored under `keyFn(qname)` key in redis list.
|
||||
func getMessagesFromList(tb testing.TB, r redis.UniversalClient, qname string,
|
||||
keyFn func(qname string) string, state base.TaskState) []*base.TaskMessage {
|
||||
@@ -481,3 +536,107 @@ func getMessagesFromZSetWithScores(tb testing.TB, r redis.UniversalClient,
|
||||
}
|
||||
return res
|
||||
}
|
||||
|
||||
// TaskSeedData holds the data required to seed tasks under the task key in test.
|
||||
type TaskSeedData struct {
|
||||
Msg *base.TaskMessage
|
||||
State base.TaskState
|
||||
PendingSince time.Time
|
||||
}
|
||||
|
||||
func SeedTasks(tb testing.TB, r redis.UniversalClient, taskData []*TaskSeedData) {
|
||||
for _, data := range taskData {
|
||||
msg := data.Msg
|
||||
ctx := context.Background()
|
||||
key := base.TaskKey(msg.Queue, msg.ID)
|
||||
v := map[string]interface{}{
|
||||
"msg": MustMarshal(tb, msg),
|
||||
"state": data.State.String(),
|
||||
"unique_key": msg.UniqueKey,
|
||||
"group": msg.GroupKey,
|
||||
}
|
||||
if !data.PendingSince.IsZero() {
|
||||
v["pending_since"] = data.PendingSince.Unix()
|
||||
}
|
||||
if err := r.HSet(ctx, key, v).Err(); err != nil {
|
||||
tb.Fatalf("Failed to write task data in redis: %v", err)
|
||||
}
|
||||
if len(msg.UniqueKey) > 0 {
|
||||
err := r.SetNX(ctx, msg.UniqueKey, msg.ID, 1*time.Minute).Err()
|
||||
if err != nil {
|
||||
tb.Fatalf("Failed to set unique lock in redis: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func SeedRedisZSets(tb testing.TB, r redis.UniversalClient, zsets map[string][]*redis.Z) {
|
||||
for key, zs := range zsets {
|
||||
// FIXME: How come we can't simply do ZAdd(ctx, key, zs...) here?
|
||||
for _, z := range zs {
|
||||
if err := r.ZAdd(context.Background(), key, z).Err(); err != nil {
|
||||
tb.Fatalf("Failed to seed zset (key=%q): %v", key, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func SeedRedisSets(tb testing.TB, r redis.UniversalClient, sets map[string][]string) {
|
||||
for key, set := range sets {
|
||||
SeedRedisSet(tb, r, key, set)
|
||||
}
|
||||
}
|
||||
|
||||
func SeedRedisSet(tb testing.TB, r redis.UniversalClient, key string, members []string) {
|
||||
for _, mem := range members {
|
||||
if err := r.SAdd(context.Background(), key, mem).Err(); err != nil {
|
||||
tb.Fatalf("Failed to seed set (key=%q): %v", key, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func SeedRedisLists(tb testing.TB, r redis.UniversalClient, lists map[string][]string) {
|
||||
for key, vals := range lists {
|
||||
for _, v := range vals {
|
||||
if err := r.LPush(context.Background(), key, v).Err(); err != nil {
|
||||
tb.Fatalf("Failed to seed list (key=%q): %v", key, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func AssertRedisLists(t *testing.T, r redis.UniversalClient, wantLists map[string][]string) {
|
||||
for key, want := range wantLists {
|
||||
got, err := r.LRange(context.Background(), key, 0, -1).Result()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to read list (key=%q): %v", key, err)
|
||||
}
|
||||
if diff := cmp.Diff(want, got, SortStringSliceOpt); diff != "" {
|
||||
t.Errorf("mismatch found in list (key=%q): (-want,+got)\n%s", key, diff)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func AssertRedisSets(t *testing.T, r redis.UniversalClient, wantSets map[string][]string) {
|
||||
for key, want := range wantSets {
|
||||
got, err := r.SMembers(context.Background(), key).Result()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to read set (key=%q): %v", key, err)
|
||||
}
|
||||
if diff := cmp.Diff(want, got, SortStringSliceOpt); diff != "" {
|
||||
t.Errorf("mismatch found in set (key=%q): (-want,+got)\n%s", key, diff)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func AssertRedisZSets(t *testing.T, r redis.UniversalClient, wantZSets map[string][]redis.Z) {
|
||||
for key, want := range wantZSets {
|
||||
got, err := r.ZRangeWithScores(context.Background(), key, 0, -1).Result()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to read zset (key=%q): %v", key, err)
|
||||
}
|
||||
if diff := cmp.Diff(want, got, SortRedisZSetEntryOpt); diff != "" {
|
||||
t.Errorf("mismatch found in zset (key=%q): (-want,+got)\n%s", key, diff)
|
||||
}
|
||||
}
|
||||
}
|
@@ -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
|
||||
|
||||
import "time"
|
||||
import (
|
||||
"sync"
|
||||
"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.
|
||||
// Time is advanced by explicit call to the AdvanceTime() or SetTime() functions.
|
||||
// This object is concurrency safe.
|
||||
type SimulatedClock struct {
|
||||
t time.Time
|
||||
mu sync.Mutex
|
||||
t time.Time // guarded by mu
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
48
internal/timeutil/timeutil_test.go
Normal file
48
internal/timeutil/timeutil_test.go
Normal 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)
|
||||
}
|
||||
}
|
||||
}
|
@@ -74,7 +74,7 @@ func (j *janitor) start(wg *sync.WaitGroup) {
|
||||
func (j *janitor) exec() {
|
||||
for _, qname := range j.queues {
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@@ -10,9 +10,9 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func newCompletedTask(qname, tasktype string, payload []byte, completedAt time.Time) *base.TaskMessage {
|
||||
|
138
processor.go
138
processor.go
@@ -7,6 +7,7 @@ package asynq
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"math/rand"
|
||||
"runtime"
|
||||
"runtime/debug"
|
||||
@@ -19,14 +20,17 @@ import (
|
||||
asynqcontext "github.com/hibiken/asynq/internal/context"
|
||||
"github.com/hibiken/asynq/internal/errors"
|
||||
"github.com/hibiken/asynq/internal/log"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
"golang.org/x/time/rate"
|
||||
)
|
||||
|
||||
type processor struct {
|
||||
logger *log.Logger
|
||||
broker base.Broker
|
||||
clock timeutil.Clock
|
||||
|
||||
handler Handler
|
||||
handler Handler
|
||||
baseCtxFn func() context.Context
|
||||
|
||||
queueConfig map[string]int
|
||||
|
||||
@@ -71,6 +75,7 @@ type processor struct {
|
||||
type processorParams struct {
|
||||
logger *log.Logger
|
||||
broker base.Broker
|
||||
baseCtxFn func() context.Context
|
||||
retryDelayFunc RetryDelayFunc
|
||||
isFailureFunc func(error) bool
|
||||
syncCh chan<- *syncRequest
|
||||
@@ -94,6 +99,8 @@ func newProcessor(params processorParams) *processor {
|
||||
return &processor{
|
||||
logger: params.logger,
|
||||
broker: params.broker,
|
||||
baseCtxFn: params.baseCtxFn,
|
||||
clock: timeutil.NewRealClock(),
|
||||
queueConfig: queues,
|
||||
orderedQueues: orderedQueues,
|
||||
retryDelayFunc: params.retryDelayFunc,
|
||||
@@ -164,7 +171,7 @@ func (p *processor) exec() {
|
||||
return
|
||||
case p.sema <- struct{}{}: // acquire token
|
||||
qnames := p.queues()
|
||||
msg, deadline, err := p.broker.Dequeue(qnames...)
|
||||
msg, leaseExpirationTime, err := p.broker.Dequeue(qnames...)
|
||||
switch {
|
||||
case errors.Is(err, errors.ErrNoProcessableTask):
|
||||
p.logger.Debug("All queues are empty")
|
||||
@@ -183,14 +190,16 @@ func (p *processor) exec() {
|
||||
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() {
|
||||
defer func() {
|
||||
p.finished <- msg
|
||||
<-p.sema // release token
|
||||
}()
|
||||
|
||||
ctx, cancel := asynqcontext.New(msg, deadline)
|
||||
ctx, cancel := asynqcontext.New(p.baseCtxFn(), msg, deadline)
|
||||
p.cancelations.Add(msg.ID, cancel)
|
||||
defer func() {
|
||||
cancel()
|
||||
@@ -201,7 +210,7 @@ func (p *processor) exec() {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
// already canceled (e.g. deadline exceeded).
|
||||
p.handleFailedMessage(ctx, msg, ctx.Err())
|
||||
p.handleFailedMessage(ctx, lease, msg, ctx.Err())
|
||||
return
|
||||
default:
|
||||
}
|
||||
@@ -225,24 +234,33 @@ func (p *processor) exec() {
|
||||
case <-p.abort:
|
||||
// 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.requeue(msg)
|
||||
p.requeue(lease, msg)
|
||||
return
|
||||
case <-lease.Done():
|
||||
cancel()
|
||||
p.handleFailedMessage(ctx, lease, msg, ErrLeaseExpired)
|
||||
return
|
||||
case <-ctx.Done():
|
||||
p.handleFailedMessage(ctx, msg, ctx.Err())
|
||||
p.handleFailedMessage(ctx, lease, msg, ctx.Err())
|
||||
return
|
||||
case resErr := <-resCh:
|
||||
if resErr != nil {
|
||||
p.handleFailedMessage(ctx, msg, resErr)
|
||||
p.handleFailedMessage(ctx, lease, msg, resErr)
|
||||
return
|
||||
}
|
||||
p.handleSucceededMessage(ctx, msg)
|
||||
p.handleSucceededMessage(lease, msg)
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
func (p *processor) requeue(msg *base.TaskMessage) {
|
||||
err := p.broker.Requeue(msg)
|
||||
func (p *processor) requeue(l *base.Lease, msg *base.TaskMessage) {
|
||||
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 {
|
||||
p.logger.Errorf("Could not push task id=%s back to queue: %v", msg.ID, err)
|
||||
} 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 {
|
||||
p.markAsComplete(ctx, msg)
|
||||
p.markAsComplete(l, msg)
|
||||
} else {
|
||||
p.markAsDone(ctx, msg)
|
||||
p.markAsDone(l, msg)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *processor) markAsComplete(ctx context.Context, msg *base.TaskMessage) {
|
||||
err := p.broker.MarkAsComplete(msg)
|
||||
func (p *processor) markAsComplete(l *base.Lease, msg *base.TaskMessage) {
|
||||
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 {
|
||||
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)
|
||||
deadline, ok := ctx.Deadline()
|
||||
if !ok {
|
||||
panic("asynq: internal error: missing deadline in context")
|
||||
}
|
||||
p.logger.Warnf("%s; Will retry syncing", errMsg)
|
||||
p.syncRequestCh <- &syncRequest{
|
||||
fn: func() error {
|
||||
return p.broker.MarkAsComplete(msg)
|
||||
return p.broker.MarkAsComplete(ctx, msg)
|
||||
},
|
||||
errMsg: errMsg,
|
||||
deadline: deadline,
|
||||
deadline: l.Deadline(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (p *processor) markAsDone(ctx context.Context, msg *base.TaskMessage) {
|
||||
err := p.broker.Done(msg)
|
||||
func (p *processor) markAsDone(l *base.Lease, msg *base.TaskMessage) {
|
||||
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 {
|
||||
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.syncRequestCh <- &syncRequest{
|
||||
fn: func() error {
|
||||
return p.broker.Done(msg)
|
||||
return p.broker.Done(ctx, msg)
|
||||
},
|
||||
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.
|
||||
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 {
|
||||
p.errHandler.HandleError(ctx, NewTask(msg.Type, msg.Payload), err)
|
||||
}
|
||||
if !p.isFailureFunc(err) {
|
||||
// retry the task without marking it as failed
|
||||
p.retry(ctx, msg, err, false /*isFailure*/)
|
||||
p.retry(l, msg, err, false /*isFailure*/)
|
||||
return
|
||||
}
|
||||
if msg.Retried >= msg.Retry || errors.Is(err, SkipRetry) {
|
||||
p.logger.Warnf("Retry exhausted for task id=%s", msg.ID)
|
||||
p.archive(ctx, msg, err)
|
||||
p.archive(l, msg, err)
|
||||
} 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))
|
||||
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 {
|
||||
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.syncRequestCh <- &syncRequest{
|
||||
fn: func() error {
|
||||
return p.broker.Retry(msg, retryAt, e.Error(), isFailure)
|
||||
return p.broker.Retry(ctx, msg, retryAt, e.Error(), isFailure)
|
||||
},
|
||||
errMsg: errMsg,
|
||||
deadline: deadline,
|
||||
deadline: l.Deadline(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (p *processor) archive(ctx context.Context, msg *base.TaskMessage, e error) {
|
||||
err := p.broker.Archive(msg, e.Error())
|
||||
func (p *processor) archive(l *base.Lease, msg *base.TaskMessage, 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 {
|
||||
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.syncRequestCh <- &syncRequest{
|
||||
fn: func() error {
|
||||
return p.broker.Archive(msg, e.Error())
|
||||
return p.broker.Archive(ctx, msg, e.Error())
|
||||
},
|
||||
errMsg: errMsg,
|
||||
deadline: deadline,
|
||||
deadline: l.Deadline(),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -483,3 +505,19 @@ func gcd(xs ...int) int {
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
@@ -15,9 +15,12 @@ import (
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/errors"
|
||||
"github.com/hibiken/asynq/internal/log"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
)
|
||||
|
||||
var taskCmpOpts = []cmp.Option{
|
||||
@@ -61,6 +64,7 @@ func newProcessorForTest(t *testing.T, r *rdb.RDB, h Handler) *processor {
|
||||
p := newProcessor(processorParams{
|
||||
logger: testLogger,
|
||||
broker: r,
|
||||
baseCtxFn: context.Background,
|
||||
retryDelayFunc: DefaultRetryDelayFunc,
|
||||
isFailureFunc: defaultIsFailureFunc,
|
||||
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) {
|
||||
sortOpt := cmp.Transformer("SortStrings", func(in []string) []string {
|
||||
out := append([]string(nil), in...) // Copy input to avoid mutating it
|
||||
@@ -592,6 +694,7 @@ func TestProcessorWithStrictPriority(t *testing.T) {
|
||||
p := newProcessor(processorParams{
|
||||
logger: testLogger,
|
||||
broker: rdbClient,
|
||||
baseCtxFn: context.Background,
|
||||
retryDelayFunc: DefaultRetryDelayFunc,
|
||||
isFailureFunc: defaultIsFailureFunc,
|
||||
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())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
36
recoverer.go
36
recoverer.go
@@ -10,6 +10,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/errors"
|
||||
"github.com/hibiken/asynq/internal/log"
|
||||
)
|
||||
|
||||
@@ -76,19 +77,36 @@ 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() {
|
||||
// Get all tasks which have expired 30 seconds ago or earlier.
|
||||
deadline := time.Now().Add(-30 * time.Second)
|
||||
msgs, err := r.broker.ListDeadlineExceeded(deadline, r.queues...)
|
||||
r.recoverLeaseExpiredTasks()
|
||||
r.recoverStaleAggregationSets()
|
||||
}
|
||||
|
||||
func (r *recoverer) recoverLeaseExpiredTasks() {
|
||||
// Get all tasks which have expired 30 seconds ago or earlier to accomodate certain amount of clock skew.
|
||||
cutoff := time.Now().Add(-30 * time.Second)
|
||||
msgs, err := r.broker.ListLeaseExpired(cutoff, r.queues...)
|
||||
if err != nil {
|
||||
r.logger.Warn("recoverer: could not list deadline exceeded tasks")
|
||||
r.logger.Warnf("recoverer: could not list lease expired tasks: %v", err)
|
||||
return
|
||||
}
|
||||
for _, msg := range msgs {
|
||||
if msg.Retried >= msg.Retry {
|
||||
r.archive(msg, context.DeadlineExceeded)
|
||||
r.archive(msg, ErrLeaseExpired)
|
||||
} else {
|
||||
r.retry(msg, context.DeadlineExceeded)
|
||||
r.retry(msg, ErrLeaseExpired)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (r *recoverer) recoverStaleAggregationSets() {
|
||||
for _, qname := range r.queues {
|
||||
if err := r.broker.ReclaimStaleAggregationSets(qname); err != nil {
|
||||
r.logger.Warnf("recoverer: could not reclaim stale aggregation sets in queue %q: %v", qname, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -96,13 +114,13 @@ func (r *recoverer) recover() {
|
||||
func (r *recoverer) retry(msg *base.TaskMessage, err error) {
|
||||
delay := r.retryDelayFunc(msg.Retried, err, NewTask(msg.Type, msg.Payload))
|
||||
retryAt := time.Now().Add(delay)
|
||||
if err := r.broker.Retry(msg, retryAt, err.Error(), r.isFailureFunc(err)); err != nil {
|
||||
r.logger.Warnf("recoverer: could not retry deadline exceeded task: %v", err)
|
||||
if err := r.broker.Retry(context.Background(), msg, retryAt, err.Error(), r.isFailureFunc(err)); err != nil {
|
||||
r.logger.Warnf("recoverer: could not retry lease expired task: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@@ -10,9 +10,9 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func TestRecoverer(t *testing.T) {
|
||||
@@ -27,29 +27,25 @@ func TestRecoverer(t *testing.T) {
|
||||
t4.Retried = t4.Retry // t4 has reached its max retry count
|
||||
|
||||
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 {
|
||||
desc string
|
||||
inProgress map[string][]*base.TaskMessage
|
||||
deadlines map[string][]base.Z
|
||||
retry map[string][]base.Z
|
||||
archived map[string][]base.Z
|
||||
wantActive map[string][]*base.TaskMessage
|
||||
wantDeadlines map[string][]base.Z
|
||||
wantRetry map[string][]*base.TaskMessage
|
||||
wantArchived map[string][]*base.TaskMessage
|
||||
desc string
|
||||
active map[string][]*base.TaskMessage
|
||||
lease map[string][]base.Z
|
||||
retry map[string][]base.Z
|
||||
archived map[string][]base.Z
|
||||
wantActive map[string][]*base.TaskMessage
|
||||
wantLease map[string][]base.Z
|
||||
wantRetry map[string][]*base.TaskMessage
|
||||
wantArchived map[string][]*base.TaskMessage
|
||||
}{
|
||||
{
|
||||
desc: "with one active task",
|
||||
inProgress: map[string][]*base.TaskMessage{
|
||||
active: map[string][]*base.TaskMessage{
|
||||
"default": {t1},
|
||||
},
|
||||
deadlines: map[string][]base.Z{
|
||||
"default": {{Message: t1, Score: fiveMinutesAgo.Unix()}},
|
||||
lease: map[string][]base.Z{
|
||||
"default": {{Message: t1, Score: now.Add(-1 * time.Minute).Unix()}},
|
||||
},
|
||||
retry: map[string][]base.Z{
|
||||
"default": {},
|
||||
@@ -60,7 +56,7 @@ func TestRecoverer(t *testing.T) {
|
||||
wantActive: map[string][]*base.TaskMessage{
|
||||
"default": {},
|
||||
},
|
||||
wantDeadlines: map[string][]base.Z{
|
||||
wantLease: map[string][]base.Z{
|
||||
"default": {},
|
||||
},
|
||||
wantRetry: map[string][]*base.TaskMessage{
|
||||
@@ -72,12 +68,12 @@ func TestRecoverer(t *testing.T) {
|
||||
},
|
||||
{
|
||||
desc: "with a task with max-retry reached",
|
||||
inProgress: map[string][]*base.TaskMessage{
|
||||
active: map[string][]*base.TaskMessage{
|
||||
"default": {t4},
|
||||
"critical": {},
|
||||
},
|
||||
deadlines: map[string][]base.Z{
|
||||
"default": {{Message: t4, Score: fiveMinutesAgo.Unix()}},
|
||||
lease: map[string][]base.Z{
|
||||
"default": {{Message: t4, Score: now.Add(-40 * time.Second).Unix()}},
|
||||
"critical": {},
|
||||
},
|
||||
retry: map[string][]base.Z{
|
||||
@@ -92,7 +88,7 @@ func TestRecoverer(t *testing.T) {
|
||||
"default": {},
|
||||
"critical": {},
|
||||
},
|
||||
wantDeadlines: map[string][]base.Z{
|
||||
wantLease: map[string][]base.Z{
|
||||
"default": {},
|
||||
"critical": {},
|
||||
},
|
||||
@@ -107,17 +103,17 @@ func TestRecoverer(t *testing.T) {
|
||||
},
|
||||
{
|
||||
desc: "with multiple active tasks, and one expired",
|
||||
inProgress: map[string][]*base.TaskMessage{
|
||||
active: map[string][]*base.TaskMessage{
|
||||
"default": {t1, t2},
|
||||
"critical": {t3},
|
||||
},
|
||||
deadlines: map[string][]base.Z{
|
||||
lease: map[string][]base.Z{
|
||||
"default": {
|
||||
{Message: t1, Score: oneHourAgo.Unix()},
|
||||
{Message: t2, Score: fiveMinutesFromNow.Unix()},
|
||||
{Message: t1, Score: now.Add(-2 * time.Minute).Unix()},
|
||||
{Message: t2, Score: now.Add(20 * time.Second).Unix()},
|
||||
},
|
||||
"critical": {
|
||||
{Message: t3, Score: oneHourFromNow.Unix()},
|
||||
{Message: t3, Score: now.Add(20 * time.Second).Unix()},
|
||||
},
|
||||
},
|
||||
retry: map[string][]base.Z{
|
||||
@@ -132,9 +128,9 @@ func TestRecoverer(t *testing.T) {
|
||||
"default": {t2},
|
||||
"critical": {t3},
|
||||
},
|
||||
wantDeadlines: map[string][]base.Z{
|
||||
"default": {{Message: t2, Score: fiveMinutesFromNow.Unix()}},
|
||||
"critical": {{Message: t3, Score: oneHourFromNow.Unix()}},
|
||||
wantLease: map[string][]base.Z{
|
||||
"default": {{Message: t2, Score: now.Add(20 * time.Second).Unix()}},
|
||||
"critical": {{Message: t3, Score: now.Add(20 * time.Second).Unix()}},
|
||||
},
|
||||
wantRetry: map[string][]*base.TaskMessage{
|
||||
"default": {t1},
|
||||
@@ -147,17 +143,17 @@ func TestRecoverer(t *testing.T) {
|
||||
},
|
||||
{
|
||||
desc: "with multiple expired active tasks",
|
||||
inProgress: map[string][]*base.TaskMessage{
|
||||
active: map[string][]*base.TaskMessage{
|
||||
"default": {t1, t2},
|
||||
"critical": {t3},
|
||||
},
|
||||
deadlines: map[string][]base.Z{
|
||||
lease: map[string][]base.Z{
|
||||
"default": {
|
||||
{Message: t1, Score: oneHourAgo.Unix()},
|
||||
{Message: t2, Score: oneHourFromNow.Unix()},
|
||||
{Message: t1, Score: now.Add(-1 * time.Minute).Unix()},
|
||||
{Message: t2, Score: now.Add(10 * time.Second).Unix()},
|
||||
},
|
||||
"critical": {
|
||||
{Message: t3, Score: fiveMinutesAgo.Unix()},
|
||||
{Message: t3, Score: now.Add(-1 * time.Minute).Unix()},
|
||||
},
|
||||
},
|
||||
retry: map[string][]base.Z{
|
||||
@@ -172,8 +168,8 @@ func TestRecoverer(t *testing.T) {
|
||||
"default": {t2},
|
||||
"critical": {},
|
||||
},
|
||||
wantDeadlines: map[string][]base.Z{
|
||||
"default": {{Message: t2, Score: oneHourFromNow.Unix()}},
|
||||
wantLease: map[string][]base.Z{
|
||||
"default": {{Message: t2, Score: now.Add(10 * time.Second).Unix()}},
|
||||
},
|
||||
wantRetry: map[string][]*base.TaskMessage{
|
||||
"default": {t1},
|
||||
@@ -186,11 +182,11 @@ func TestRecoverer(t *testing.T) {
|
||||
},
|
||||
{
|
||||
desc: "with empty active queue",
|
||||
inProgress: map[string][]*base.TaskMessage{
|
||||
active: map[string][]*base.TaskMessage{
|
||||
"default": {},
|
||||
"critical": {},
|
||||
},
|
||||
deadlines: map[string][]base.Z{
|
||||
lease: map[string][]base.Z{
|
||||
"default": {},
|
||||
"critical": {},
|
||||
},
|
||||
@@ -206,7 +202,7 @@ func TestRecoverer(t *testing.T) {
|
||||
"default": {},
|
||||
"critical": {},
|
||||
},
|
||||
wantDeadlines: map[string][]base.Z{
|
||||
wantLease: map[string][]base.Z{
|
||||
"default": {},
|
||||
"critical": {},
|
||||
},
|
||||
@@ -223,8 +219,8 @@ func TestRecoverer(t *testing.T) {
|
||||
|
||||
for _, tc := range tests {
|
||||
h.FlushDB(t, r)
|
||||
h.SeedAllActiveQueues(t, r, tc.inProgress)
|
||||
h.SeedAllDeadlines(t, r, tc.deadlines)
|
||||
h.SeedAllActiveQueues(t, r, tc.active)
|
||||
h.SeedAllLease(t, r, tc.lease)
|
||||
h.SeedAllRetryQueues(t, r, tc.retry)
|
||||
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)
|
||||
}
|
||||
}
|
||||
for qname, want := range tc.wantDeadlines {
|
||||
gotDeadlines := h.GetDeadlinesEntries(t, r, qname)
|
||||
if diff := cmp.Diff(want, gotDeadlines, h.SortZSetEntryOpt); diff != "" {
|
||||
t.Errorf("%s; mismatch found in %q; (-want,+got)\n%s", tc.desc, base.DeadlinesKey(qname), diff)
|
||||
for qname, want := range tc.wantLease {
|
||||
gotLease := h.GetLeaseEntries(t, r, qname)
|
||||
if diff := cmp.Diff(want, gotLease, h.SortZSetEntryOpt); 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`
|
||||
@@ -260,7 +256,7 @@ func TestRecoverer(t *testing.T) {
|
||||
gotRetry := h.GetRetryMessages(t, r, qname)
|
||||
var wantRetry []*base.TaskMessage // Note: construct message here since `LastFailedAt` is relative to each test run
|
||||
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 != "" {
|
||||
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)
|
||||
var wantArchived []*base.TaskMessage
|
||||
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 != "" {
|
||||
t.Errorf("%s; mismatch found in %q: (-want, +got)\n%s", tc.desc, base.ArchivedKey(qname), diff)
|
||||
|
@@ -10,8 +10,8 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func TestSchedulerRegister(t *testing.T) {
|
||||
@@ -69,8 +69,8 @@ func TestSchedulerRegister(t *testing.T) {
|
||||
time.Sleep(tc.wait)
|
||||
scheduler.Shutdown()
|
||||
|
||||
got := asynqtest.GetPendingMessages(t, r, tc.queue)
|
||||
if diff := cmp.Diff(tc.want, got, asynqtest.IgnoreIDOpt); diff != "" {
|
||||
got := testutil.GetPendingMessages(t, r, tc.queue)
|
||||
if diff := cmp.Diff(tc.want, got, testutil.IgnoreIDOpt); diff != "" {
|
||||
t.Errorf("mismatch found in queue %q: (-want,+got)\n%s", tc.queue, diff)
|
||||
}
|
||||
}
|
||||
@@ -148,7 +148,7 @@ func TestSchedulerUnregister(t *testing.T) {
|
||||
time.Sleep(tc.wait)
|
||||
scheduler.Shutdown()
|
||||
|
||||
got := asynqtest.GetPendingMessages(t, r, tc.queue)
|
||||
got := testutil.GetPendingMessages(t, r, tc.queue)
|
||||
if len(got) != 0 {
|
||||
t.Errorf("%d tasks were enqueued, want zero", len(got))
|
||||
}
|
||||
|
80
server.go
80
server.go
@@ -50,6 +50,7 @@ type Server struct {
|
||||
recoverer *recoverer
|
||||
healthchecker *healthchecker
|
||||
janitor *janitor
|
||||
aggregator *aggregator
|
||||
}
|
||||
|
||||
type serverState struct {
|
||||
@@ -97,6 +98,12 @@ type Config struct {
|
||||
// to the number of CPUs usable by the current process.
|
||||
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.
|
||||
//
|
||||
// By default, it uses exponential backoff algorithm to calculate the delay.
|
||||
@@ -187,6 +194,52 @@ type Config struct {
|
||||
//
|
||||
// If unset or zero, the interval is set to 5 seconds.
|
||||
DelayedTaskCheckInterval time.Duration
|
||||
|
||||
// GroupGracePeriod specifies the amount of time the server will wait for an incoming task before aggregating
|
||||
// the tasks in a group. If an incoming task is received within this period, the server will wait for another
|
||||
// period of the same length, up to GroupMaxDelay if specified.
|
||||
//
|
||||
// If unset or zero, the grace period is set to 1 minute.
|
||||
// Minimum duration for GroupGracePeriod is 1 second. If value specified is less than a second, the call to
|
||||
// NewServer will panic.
|
||||
GroupGracePeriod time.Duration
|
||||
|
||||
// GroupMaxDelay specifies the maximum amount of time the server will wait for incoming tasks before aggregating
|
||||
// the tasks in a group.
|
||||
//
|
||||
// If unset or zero, no delay limit is used.
|
||||
GroupMaxDelay time.Duration
|
||||
|
||||
// GroupMaxSize specifies the maximum number of tasks that can be aggregated into a single task within a group.
|
||||
// If GroupMaxSize is reached, the server will aggregate the tasks into one immediately.
|
||||
//
|
||||
// If unset or zero, no size limit is used.
|
||||
GroupMaxSize int
|
||||
|
||||
// GroupAggregator specifies the aggregation function used to aggregate multiple tasks in a group into one task.
|
||||
//
|
||||
// If unset or nil, the group aggregation feature will be disabled on the server.
|
||||
GroupAggregator GroupAggregator
|
||||
}
|
||||
|
||||
// GroupAggregator aggregates a group of tasks into one before the tasks are passed to the Handler.
|
||||
type GroupAggregator interface {
|
||||
// Aggregate aggregates the given tasks in a group with the given group name,
|
||||
// and returns a new task which is the aggregation of those tasks.
|
||||
//
|
||||
// Use NewTask(typename, payload, opts...) to set any options for the aggregated task.
|
||||
// The Queue option, if provided, will be ignored and the aggregated task will always be enqueued
|
||||
// to the same queue the group belonged.
|
||||
Aggregate(group string, tasks []*Task) *Task
|
||||
}
|
||||
|
||||
// The GroupAggregatorFunc type is an adapter to allow the use of ordinary functions as a GroupAggregator.
|
||||
// If f is a function with the appropriate signature, GroupAggregatorFunc(f) is a GroupAggregator that calls f.
|
||||
type GroupAggregatorFunc func(group string, tasks []*Task) *Task
|
||||
|
||||
// Aggregate calls fn(group, tasks)
|
||||
func (fn GroupAggregatorFunc) Aggregate(group string, tasks []*Task) *Task {
|
||||
return fn(group, tasks)
|
||||
}
|
||||
|
||||
// An ErrorHandler handles an error occured during task processing.
|
||||
@@ -332,6 +385,8 @@ const (
|
||||
defaultHealthCheckInterval = 15 * time.Second
|
||||
|
||||
defaultDelayedTaskCheckInterval = 5 * time.Second
|
||||
|
||||
defaultGroupGracePeriod = 1 * time.Minute
|
||||
)
|
||||
|
||||
// NewServer returns a new Server given a redis connection option
|
||||
@@ -341,6 +396,10 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
|
||||
if !ok {
|
||||
panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r))
|
||||
}
|
||||
baseCtxFn := cfg.BaseContext
|
||||
if baseCtxFn == nil {
|
||||
baseCtxFn = context.Background
|
||||
}
|
||||
n := cfg.Concurrency
|
||||
if n < 1 {
|
||||
n = runtime.NumCPU()
|
||||
@@ -377,6 +436,14 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
|
||||
if healthcheckInterval == 0 {
|
||||
healthcheckInterval = defaultHealthCheckInterval
|
||||
}
|
||||
// TODO: Create a helper to check for zero value and fall back to default (e.g. getDurationOrDefault())
|
||||
groupGracePeriod := cfg.GroupGracePeriod
|
||||
if groupGracePeriod == 0 {
|
||||
groupGracePeriod = defaultGroupGracePeriod
|
||||
}
|
||||
if groupGracePeriod < time.Second {
|
||||
panic("GroupGracePeriod cannot be less than a second")
|
||||
}
|
||||
logger := log.NewLogger(cfg.Logger)
|
||||
loglevel := cfg.LogLevel
|
||||
if loglevel == level_unspecified {
|
||||
@@ -426,6 +493,7 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
|
||||
logger: logger,
|
||||
broker: rdb,
|
||||
retryDelayFunc: delayFunc,
|
||||
baseCtxFn: baseCtxFn,
|
||||
isFailureFunc: isFailureFunc,
|
||||
syncCh: syncCh,
|
||||
cancelations: cancels,
|
||||
@@ -457,6 +525,15 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
|
||||
queues: qnames,
|
||||
interval: 8 * time.Second,
|
||||
})
|
||||
aggregator := newAggregator(aggregatorParams{
|
||||
logger: logger,
|
||||
broker: rdb,
|
||||
queues: qnames,
|
||||
gracePeriod: groupGracePeriod,
|
||||
maxDelay: cfg.GroupMaxDelay,
|
||||
maxSize: cfg.GroupMaxSize,
|
||||
groupAggregator: cfg.GroupAggregator,
|
||||
})
|
||||
return &Server{
|
||||
logger: logger,
|
||||
broker: rdb,
|
||||
@@ -469,6 +546,7 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
|
||||
recoverer: recoverer,
|
||||
healthchecker: healthchecker,
|
||||
janitor: janitor,
|
||||
aggregator: aggregator,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -545,6 +623,7 @@ func (srv *Server) Start(handler Handler) error {
|
||||
srv.forwarder.start(&srv.wg)
|
||||
srv.processor.start(&srv.wg)
|
||||
srv.janitor.start(&srv.wg)
|
||||
srv.aggregator.start(&srv.wg)
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -590,6 +669,7 @@ func (srv *Server) Shutdown() {
|
||||
srv.syncer.shutdown()
|
||||
srv.subscriber.shutdown()
|
||||
srv.janitor.shutdown()
|
||||
srv.aggregator.shutdown()
|
||||
srv.healthchecker.shutdown()
|
||||
srv.heartbeater.shutdown()
|
||||
srv.wg.Wait()
|
||||
|
@@ -11,9 +11,9 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
"github.com/hibiken/asynq/internal/testbroker"
|
||||
"github.com/hibiken/asynq/internal/testutil"
|
||||
"go.uber.org/goleak"
|
||||
)
|
||||
|
||||
@@ -40,12 +40,12 @@ func TestServer(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
_, err = c.Enqueue(NewTask("send_email", asynqtest.JSON(map[string]interface{}{"recipient_id": 123})))
|
||||
_, err = c.Enqueue(NewTask("send_email", testutil.JSON(map[string]interface{}{"recipient_id": 123})))
|
||||
if err != nil {
|
||||
t.Errorf("could not enqueue a task: %v", err)
|
||||
}
|
||||
|
||||
_, err = c.Enqueue(NewTask("send_email", asynqtest.JSON(map[string]interface{}{"recipient_id": 456})), ProcessIn(1*time.Hour))
|
||||
_, err = c.Enqueue(NewTask("send_email", testutil.JSON(map[string]interface{}{"recipient_id": 456})), ProcessIn(1*time.Hour))
|
||||
if err != nil {
|
||||
t.Errorf("could not enqueue a task: %v", err)
|
||||
}
|
||||
|
@@ -5,14 +5,15 @@
|
||||
package asynq
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
h "github.com/hibiken/asynq/internal/asynqtest"
|
||||
"github.com/hibiken/asynq/internal/base"
|
||||
"github.com/hibiken/asynq/internal/rdb"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
)
|
||||
|
||||
func TestSyncer(t *testing.T) {
|
||||
@@ -41,7 +42,7 @@ func TestSyncer(t *testing.T) {
|
||||
m := msg
|
||||
syncRequestCh <- &syncRequest{
|
||||
fn: func() error {
|
||||
return rdbClient.Done(m)
|
||||
return rdbClient.Done(context.Background(), m)
|
||||
},
|
||||
deadline: time.Now().Add(5 * time.Minute),
|
||||
}
|
||||
|
48
tools/asynq/cmd/group.go
Normal file
48
tools/asynq/cmd/group.go
Normal 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 cmd
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
func init() {
|
||||
rootCmd.AddCommand(groupCmd)
|
||||
groupCmd.AddCommand(groupListCmd)
|
||||
groupListCmd.Flags().StringP("queue", "q", "", "queue to inspect")
|
||||
groupListCmd.MarkFlagRequired("queue")
|
||||
}
|
||||
|
||||
var groupCmd = &cobra.Command{
|
||||
Use: "group",
|
||||
Short: "Manage groups",
|
||||
}
|
||||
|
||||
var groupListCmd = &cobra.Command{
|
||||
Use: "ls",
|
||||
Short: "List groups",
|
||||
Args: cobra.NoArgs,
|
||||
Run: groupLists,
|
||||
}
|
||||
|
||||
func groupLists(cmd *cobra.Command, args []string) {
|
||||
qname, err := cmd.Flags().GetString("queue")
|
||||
if err != nil {
|
||||
fmt.Println(err)
|
||||
os.Exit(1)
|
||||
}
|
||||
inspector := createInspector()
|
||||
groups, err := inspector.Groups(qname)
|
||||
if len(groups) == 0 {
|
||||
fmt.Printf("No groups found in queue %q\n", qname)
|
||||
return
|
||||
}
|
||||
for _, g := range groups {
|
||||
fmt.Println(g.Group)
|
||||
}
|
||||
}
|
@@ -145,12 +145,13 @@ func printQueueInfo(info *asynq.QueueInfo) {
|
||||
bold.Println("Queue Info")
|
||||
fmt.Printf("Name: %s\n", info.Queue)
|
||||
fmt.Printf("Size: %d\n", info.Size)
|
||||
fmt.Printf("Groups: %d\n", info.Groups)
|
||||
fmt.Printf("Paused: %t\n\n", info.Paused)
|
||||
bold.Println("Task Count by State")
|
||||
printTable(
|
||||
[]string{"active", "pending", "scheduled", "retry", "archived", "completed"},
|
||||
[]string{"active", "pending", "aggregating", "scheduled", "retry", "archived", "completed"},
|
||||
func(w io.Writer, tmpl string) {
|
||||
fmt.Fprintf(w, tmpl, info.Active, info.Pending, info.Scheduled, info.Retry, info.Archived, info.Completed)
|
||||
fmt.Fprintf(w, tmpl, info.Active, info.Pending, info.Aggregating, info.Scheduled, info.Retry, info.Archived, info.Completed)
|
||||
},
|
||||
)
|
||||
fmt.Println()
|
||||
|
@@ -59,15 +59,16 @@ func init() {
|
||||
}
|
||||
|
||||
type AggregateStats struct {
|
||||
Active int `json:"active"`
|
||||
Pending int `json:"pending"`
|
||||
Scheduled int `json:"scheduled"`
|
||||
Retry int `json:"retry"`
|
||||
Archived int `json:"archived"`
|
||||
Completed int `json:"completed"`
|
||||
Processed int `json:"processed"`
|
||||
Failed int `json:"failed"`
|
||||
Timestamp time.Time `json:"timestamp"`
|
||||
Active int `json:"active"`
|
||||
Pending int `json:"pending"`
|
||||
Aggregating int `json:"aggregating"`
|
||||
Scheduled int `json:"scheduled"`
|
||||
Retry int `json:"retry"`
|
||||
Archived int `json:"archived"`
|
||||
Completed int `json:"completed"`
|
||||
Processed int `json:"processed"`
|
||||
Failed int `json:"failed"`
|
||||
Timestamp time.Time `json:"timestamp"`
|
||||
}
|
||||
|
||||
type FullStats struct {
|
||||
@@ -95,6 +96,7 @@ func stats(cmd *cobra.Command, args []string) {
|
||||
}
|
||||
aggStats.Active += s.Active
|
||||
aggStats.Pending += s.Pending
|
||||
aggStats.Aggregating += s.Aggregating
|
||||
aggStats.Scheduled += s.Scheduled
|
||||
aggStats.Retry += s.Retry
|
||||
aggStats.Archived += s.Archived
|
||||
@@ -155,13 +157,13 @@ func stats(cmd *cobra.Command, args []string) {
|
||||
}
|
||||
|
||||
func printStatsByState(s *AggregateStats) {
|
||||
format := strings.Repeat("%v\t", 6) + "\n"
|
||||
format := strings.Repeat("%v\t", 7) + "\n"
|
||||
tw := new(tabwriter.Writer).Init(os.Stdout, 0, 8, 2, ' ', 0)
|
||||
fmt.Fprintf(tw, format, "active", "pending", "scheduled", "retry", "archived", "completed")
|
||||
width := maxInt(9 /* defaultWidth */, maxWidthOf(s.Active, s.Pending, s.Scheduled, s.Retry, s.Archived, s.Completed)) // length of widest column
|
||||
fmt.Fprintf(tw, format, "active", "pending", "aggregating", "scheduled", "retry", "archived", "completed")
|
||||
width := maxInt(9 /* defaultWidth */, maxWidthOf(s.Active, s.Pending, s.Aggregating, s.Scheduled, s.Retry, s.Archived, s.Completed)) // length of widest column
|
||||
sep := strings.Repeat("-", width)
|
||||
fmt.Fprintf(tw, format, sep, sep, sep, sep, sep, sep)
|
||||
fmt.Fprintf(tw, format, s.Active, s.Pending, s.Scheduled, s.Retry, s.Archived, s.Completed)
|
||||
fmt.Fprintf(tw, format, sep, sep, sep, sep, sep, sep, sep)
|
||||
fmt.Fprintf(tw, format, s.Active, s.Pending, s.Aggregating, s.Scheduled, s.Retry, s.Archived, s.Completed)
|
||||
tw.Flush()
|
||||
}
|
||||
|
||||
|
@@ -22,6 +22,7 @@ func init() {
|
||||
taskListCmd.Flags().StringP("state", "s", "", "state of the tasks to inspect")
|
||||
taskListCmd.Flags().Int("page", 1, "page number")
|
||||
taskListCmd.Flags().Int("size", 30, "page size")
|
||||
taskListCmd.Flags().StringP("group", "g", "", "group to inspect (required for listing aggregating tasks)")
|
||||
taskListCmd.MarkFlagRequired("queue")
|
||||
taskListCmd.MarkFlagRequired("state")
|
||||
|
||||
@@ -83,6 +84,7 @@ var taskListCmd = &cobra.Command{
|
||||
The value for the state flag should be one of:
|
||||
- active
|
||||
- pending
|
||||
- aggregating
|
||||
- scheduled
|
||||
- retry
|
||||
- archived
|
||||
@@ -92,12 +94,19 @@ List opeartion paginates the result set.
|
||||
By default, the command fetches the first 30 tasks.
|
||||
Use --page and --size flags to specify the page number and size.
|
||||
|
||||
|
||||
Example:
|
||||
To list pending tasks from "default" queue, run
|
||||
asynq task ls --queue=default --state=pending
|
||||
|
||||
To list the tasks from the second page, run
|
||||
asynq task ls --queue=default --state=pending --page=1`,
|
||||
asynq task ls --queue=default --state=pending --page=1
|
||||
|
||||
For aggregating tasks, additional --group flag is required.
|
||||
|
||||
Example:
|
||||
asynq task ls --queue=default --state=aggregating --group=mygroup
|
||||
`,
|
||||
Run: taskList,
|
||||
}
|
||||
|
||||
@@ -192,6 +201,17 @@ func taskList(cmd *cobra.Command, args []string) {
|
||||
listArchivedTasks(qname, pageNum, pageSize)
|
||||
case "completed":
|
||||
listCompletedTasks(qname, pageNum, pageSize)
|
||||
case "aggregating":
|
||||
group, err := cmd.Flags().GetString("group")
|
||||
if err != nil {
|
||||
fmt.Println(err)
|
||||
os.Exit(1)
|
||||
}
|
||||
if group == "" {
|
||||
fmt.Println("Flag --group is required for listing aggregating tasks")
|
||||
os.Exit(1)
|
||||
}
|
||||
listAggregatingTasks(qname, group, pageNum, pageSize)
|
||||
default:
|
||||
fmt.Printf("error: state=%q is not supported\n", state)
|
||||
os.Exit(1)
|
||||
@@ -334,6 +354,27 @@ func listCompletedTasks(qname string, pageNum, pageSize int) {
|
||||
})
|
||||
}
|
||||
|
||||
func listAggregatingTasks(qname, group string, pageNum, pageSize int) {
|
||||
i := createInspector()
|
||||
tasks, err := i.ListAggregatingTasks(qname, group, asynq.PageSize(pageSize), asynq.Page(pageNum))
|
||||
if err != nil {
|
||||
fmt.Println(err)
|
||||
os.Exit(1)
|
||||
}
|
||||
if len(tasks) == 0 {
|
||||
fmt.Printf("No aggregating tasks in group %q \n", group)
|
||||
return
|
||||
}
|
||||
printTable(
|
||||
[]string{"ID", "Type", "Payload", "Group"},
|
||||
func(w io.Writer, tmpl string) {
|
||||
for _, t := range tasks {
|
||||
fmt.Fprintf(w, tmpl, t.ID, t.Type, sprintBytes(t.Payload), t.Group)
|
||||
}
|
||||
},
|
||||
)
|
||||
}
|
||||
|
||||
func taskCancel(cmd *cobra.Command, args []string) {
|
||||
i := createInspector()
|
||||
for _, id := range args {
|
||||
|
11
tools/go.mod
11
tools/go.mod
@@ -5,16 +5,13 @@ go 1.13
|
||||
require (
|
||||
github.com/fatih/color v1.9.0
|
||||
github.com/go-redis/redis/v8 v8.11.4
|
||||
github.com/google/uuid v1.3.0
|
||||
github.com/hibiken/asynq v0.19.0
|
||||
github.com/hibiken/asynq/x v0.0.0-00010101000000-000000000000
|
||||
github.com/hibiken/asynq v0.21.0
|
||||
github.com/hibiken/asynq/x v0.0.0-20220131170841-349f4c50fb1d
|
||||
github.com/mitchellh/go-homedir v1.1.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/viper v1.7.0
|
||||
)
|
||||
|
||||
replace (
|
||||
github.com/hibiken/asynq => ./..
|
||||
github.com/hibiken/asynq/x => ./../x
|
||||
)
|
||||
replace github.com/hibiken/asynq => ../
|
||||
|
28
tools/go.sum
28
tools/go.sum
@@ -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-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 v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0=
|
||||
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/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/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
|
||||
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.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE=
|
||||
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.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
|
||||
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.4 h1:kHoYkfZP6+pe04aFTnhDH6GDROa5yJdHJVNxV3F46Tg=
|
||||
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/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
|
||||
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.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
|
||||
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-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
|
||||
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.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
|
||||
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/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
|
||||
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/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM=
|
||||
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/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/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78=
|
||||
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/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.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.16.4 h1:29JGrr5oVBm5ulCWet69zQkzWipVXIol6ygQUe/EzNc=
|
||||
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.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.16.0 h1:6gjqkI8iiRHMvdccRJM8rVKjCWk6ZIm6FTm3ddIe4/c=
|
||||
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/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/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.3 h1:9iH4JKXLzFbOAdtqv/a+j8aewx2Y8lAjAydhbaScPF8=
|
||||
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.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_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-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.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M=
|
||||
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.4.0 h1:7etb9YClo3a6HjLzfl6rIQaU+FDfi0VSX39io3aQ+DM=
|
||||
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.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/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-20190507164030-5867b95ac084 h1:sofwID9zm4tzrgykg80hfFph1mryUeLRsUfoocVVmRY=
|
||||
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.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-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-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-20210428140749-89ef3d95e781 h1:DzZ89McO9/gWPsQXS/FVKAlG02ZjaQ6AlZRBimEYOd0=
|
||||
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-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-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-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-20210124154548-22da62e12c0c/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.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.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k=
|
||||
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/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
|
||||
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.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.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.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
|
||||
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.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.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU=
|
||||
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.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo=
|
||||
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
||||
|
4
x/go.mod
4
x/go.mod
@@ -5,8 +5,6 @@ go 1.16
|
||||
require (
|
||||
github.com/go-redis/redis/v8 v8.11.4
|
||||
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
|
||||
)
|
||||
|
||||
replace github.com/hibiken/asynq => ./..
|
||||
|
2
x/go.sum
2
x/go.sum
@@ -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.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
|
||||
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/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
|
||||
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
|
||||
|
Reference in New Issue
Block a user