mirror of
https://github.com/hibiken/asynq.git
synced 2025-10-20 09:16:12 +08:00
Compare commits
63 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 |
2
.github/workflows/build.yml
vendored
2
.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:
|
||||
|
23
CHANGELOG.md
23
CHANGELOG.md
@@ -7,6 +7,23 @@ 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
|
||||
@@ -264,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()
|
||||
}
|
||||
}
|
39
asynq.go
39
asynq.go
@@ -8,6 +8,7 @@ import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"strconv"
|
||||
"strings"
|
||||
@@ -97,6 +98,14 @@ 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
|
||||
@@ -139,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,
|
||||
@@ -161,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))
|
||||
}
|
||||
@@ -188,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 {
|
||||
@@ -204,6 +219,8 @@ func (s TaskState) String() string {
|
||||
return "archived"
|
||||
case TaskStateCompleted:
|
||||
return "completed"
|
||||
case TaskStateAggregating:
|
||||
return "aggregating"
|
||||
}
|
||||
panic("asynq: unknown task state")
|
||||
}
|
||||
@@ -419,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) {
|
||||
@@ -430,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)
|
||||
@@ -444,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])
|
||||
@@ -455,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)
|
||||
}
|
||||
|
@@ -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)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
@@ -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) {
|
||||
|
@@ -12,10 +12,10 @@ 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/rdb"
|
||||
"github.com/hibiken/asynq/internal/testbroker"
|
||||
h "github.com/hibiken/asynq/internal/testutil"
|
||||
"github.com/hibiken/asynq/internal/timeutil"
|
||||
)
|
||||
|
||||
|
280
inspector.go
280
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,20 +327,20 @@ 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(), qname)
|
||||
expired, err := i.rdb.ListLeaseExpired(time.Now(), queue)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
@@ -332,17 +364,45 @@ func (i *Inspector) ListActiveTasks(qname string, opts ...ListOption) ([]*TaskIn
|
||||
return tasks, nil
|
||||
}
|
||||
|
||||
// ListScheduledTasks retrieves scheduled tasks from the specified queue.
|
||||
// Tasks are sorted by NextProcessAt in ascending order.
|
||||
// ListAggregatingTasks retrieves scheduled tasks from the specified group.
|
||||
//
|
||||
// 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) 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.ListScheduled(qname, pgn)
|
||||
infos, err := i.rdb.ListAggregating(queue, group, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
case err != nil:
|
||||
return nil, fmt.Errorf("asynq: %v", err)
|
||||
}
|
||||
var tasks []*TaskInfo
|
||||
for _, i := range infos {
|
||||
tasks = append(tasks, newTaskInfo(
|
||||
i.Message,
|
||||
i.State,
|
||||
i.NextProcessAt,
|
||||
i.Result,
|
||||
))
|
||||
}
|
||||
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(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(queue, pgn)
|
||||
switch {
|
||||
case errors.IsQueueNotFound(err):
|
||||
return nil, fmt.Errorf("asynq: %w", ErrQueueNotFound)
|
||||
@@ -365,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)
|
||||
@@ -394,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)
|
||||
@@ -423,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)
|
||||
@@ -450,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
|
||||
}
|
||||
|
||||
@@ -505,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)
|
||||
@@ -522,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
|
||||
}
|
||||
|
||||
@@ -559,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)
|
||||
@@ -577,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
|
||||
}
|
||||
|
||||
@@ -612,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)
|
||||
@@ -638,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.
|
||||
@@ -741,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.
|
||||
@@ -757,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
|
||||
}
|
||||
@@ -826,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"
|
||||
)
|
||||
|
||||
@@ -1120,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
|
||||
@@ -1221,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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3323,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)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@@ -23,7 +23,7 @@ import (
|
||||
)
|
||||
|
||||
// Version of asynq library and CLI.
|
||||
const Version = "0.22.1"
|
||||
const Version = "0.23.0"
|
||||
|
||||
// DefaultQueueName is the queue name used if none are specified by user.
|
||||
const DefaultQueueName = "default"
|
||||
@@ -50,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 {
|
||||
@@ -66,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))
|
||||
}
|
||||
@@ -84,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))
|
||||
}
|
||||
@@ -200,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 {
|
||||
@@ -250,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
|
||||
|
||||
@@ -277,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,
|
||||
})
|
||||
@@ -300,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
|
||||
@@ -676,6 +714,7 @@ func (l *Lease) IsValid() bool {
|
||||
// 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)
|
||||
@@ -687,13 +726,30 @@ type Broker interface {
|
||||
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
|
||||
|
||||
// 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
|
||||
}
|
||||
|
@@ -395,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 {
|
||||
@@ -407,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,
|
||||
@@ -418,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,
|
||||
|
@@ -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,9 +147,9 @@ 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()},
|
||||
@@ -161,9 +161,9 @@ func BenchmarkDone(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedLease(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(ctx, msgs[0]); err != nil {
|
||||
@@ -174,9 +174,9 @@ 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()},
|
||||
@@ -188,9 +188,9 @@ func BenchmarkRetry(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedLease(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(ctx, msgs[0], time.Now().Add(1*time.Minute), "error", true /*isFailure*/); err != nil {
|
||||
@@ -201,9 +201,9 @@ 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()},
|
||||
@@ -215,9 +215,9 @@ func BenchmarkArchive(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedLease(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(ctx, msgs[0], "error"); err != nil {
|
||||
@@ -228,9 +228,9 @@ 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()},
|
||||
@@ -242,9 +242,9 @@ func BenchmarkRequeue(b *testing.B) {
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer()
|
||||
asynqtest.FlushDB(b, r.client)
|
||||
asynqtest.SeedActiveQueue(b, r.client, msgs, base.DefaultQueueName)
|
||||
asynqtest.SeedLease(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(ctx, msgs[0]); err != nil {
|
||||
@@ -258,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})
|
||||
}
|
||||
@@ -266,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:
|
||||
|
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"
|
||||
@@ -348,7 +349,7 @@ func (r *RDB) Done(ctx context.Context, 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 {
|
||||
@@ -458,7 +459,7 @@ func (r *RDB) MarkAsComplete(ctx context.Context, 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 {
|
||||
@@ -497,6 +498,128 @@ func (r *RDB) Requeue(ctx context.Context, msg *base.TaskMessage) error {
|
||||
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
|
||||
// -------
|
||||
@@ -688,7 +811,7 @@ func (r *RDB) Retry(ctx context.Context, msg *base.TaskMessage, processAt time.T
|
||||
processAt.Unix(),
|
||||
expireAt.Unix(),
|
||||
isFailure,
|
||||
math.MaxInt64,
|
||||
int64(math.MaxInt64),
|
||||
}
|
||||
return r.runScript(ctx, op, retryCmd, keys, argv...)
|
||||
}
|
||||
@@ -774,7 +897,7 @@ func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string)
|
||||
cutoff.Unix(),
|
||||
maxArchiveSize,
|
||||
expireAt.Unix(),
|
||||
math.MaxInt64,
|
||||
int64(math.MaxInt64),
|
||||
}
|
||||
return r.runScript(ctx, op, archiveCmd, keys, argv...)
|
||||
}
|
||||
@@ -796,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))
|
||||
}
|
||||
@@ -825,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
|
||||
}
|
||||
@@ -842,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
|
||||
|
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.
|
||||
@@ -234,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"
|
||||
@@ -93,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")
|
||||
|
||||
@@ -245,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.
|
||||
@@ -303,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()
|
||||
@@ -311,13 +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(),
|
||||
"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 {
|
||||
@@ -339,13 +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(),
|
||||
"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 {
|
||||
@@ -434,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 {
|
||||
@@ -485,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)
|
||||
}
|
||||
}
|
||||
}
|
@@ -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 {
|
||||
|
@@ -15,11 +15,11 @@ 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"
|
||||
)
|
||||
|
||||
|
15
recoverer.go
15
recoverer.go
@@ -82,11 +82,16 @@ func (r *recoverer) start(wg *sync.WaitGroup) {
|
||||
var ErrLeaseExpired = errors.New("asynq: task lease expired")
|
||||
|
||||
func (r *recoverer) recover() {
|
||||
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 lease expired tasks")
|
||||
r.logger.Warnf("recoverer: could not list lease expired tasks: %v", err)
|
||||
return
|
||||
}
|
||||
for _, msg := range msgs {
|
||||
@@ -98,6 +103,14 @@ func (r *recoverer) recover() {
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
|
@@ -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) {
|
||||
|
@@ -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))
|
||||
}
|
||||
|
69
server.go
69
server.go
@@ -50,6 +50,7 @@ type Server struct {
|
||||
recoverer *recoverer
|
||||
healthchecker *healthchecker
|
||||
janitor *janitor
|
||||
aggregator *aggregator
|
||||
}
|
||||
|
||||
type serverState struct {
|
||||
@@ -193,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.
|
||||
@@ -338,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
|
||||
@@ -387,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 {
|
||||
@@ -468,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,
|
||||
@@ -480,6 +546,7 @@ func NewServer(r RedisConnOpt, cfg Config) *Server {
|
||||
recoverer: recoverer,
|
||||
healthchecker: healthchecker,
|
||||
janitor: janitor,
|
||||
aggregator: aggregator,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -556,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
|
||||
}
|
||||
|
||||
@@ -601,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)
|
||||
}
|
||||
|
@@ -11,9 +11,9 @@ import (
|
||||
"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) {
|
||||
|
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 {
|
||||
|
@@ -13,3 +13,5 @@ require (
|
||||
github.com/spf13/cobra v1.1.1
|
||||
github.com/spf13/viper v1.7.0
|
||||
)
|
||||
|
||||
replace github.com/hibiken/asynq => ../
|
||||
|
Reference in New Issue
Block a user