2
0
mirror of https://github.com/hibiken/asynq.git synced 2024-09-20 11:05:58 +08:00

Add ps command to asynqmon

This commit is contained in:
Ken Hibino 2020-02-01 22:22:48 -08:00
parent d03fa34eaf
commit 96f06ac89b
15 changed files with 479 additions and 144 deletions

View File

@ -33,6 +33,7 @@ type Background struct {
mu sync.Mutex mu sync.Mutex
running bool running bool
pinfo *base.ProcessInfo
rdb *rdb.RDB rdb *rdb.RDB
scheduler *scheduler scheduler *scheduler
processor *processor processor *processor
@ -108,7 +109,6 @@ func NewBackground(r RedisConnOpt, cfg *Config) *Background {
if queues == nil || len(queues) == 0 { if queues == nil || len(queues) == 0 {
queues = defaultQueueConfig queues = defaultQueueConfig
} }
qcfg := normalizeQueueCfg(queues)
host, err := os.Hostname() host, err := os.Hostname()
if err != nil { if err != nil {
@ -116,13 +116,15 @@ func NewBackground(r RedisConnOpt, cfg *Config) *Background {
} }
pid := os.Getpid() pid := os.Getpid()
pinfo := base.NewProcessInfo(host, pid, n, queues, cfg.StrictPriority)
rdb := rdb.NewRDB(createRedisClient(r)) rdb := rdb.NewRDB(createRedisClient(r))
syncRequestCh := make(chan *syncRequest) syncRequestCh := make(chan *syncRequest)
syncer := newSyncer(syncRequestCh, 5*time.Second) syncer := newSyncer(syncRequestCh, 5*time.Second)
heartbeater := newHeartbeater(rdb, 5*time.Second, host, pid, queues, n) heartbeater := newHeartbeater(rdb, pinfo, 5*time.Second)
scheduler := newScheduler(rdb, 5*time.Second, qcfg) scheduler := newScheduler(rdb, 5*time.Second, queues)
processor := newProcessor(rdb, n, qcfg, cfg.StrictPriority, delayFunc, syncRequestCh) processor := newProcessor(rdb, pinfo, delayFunc, syncRequestCh)
return &Background{ return &Background{
pinfo: pinfo,
rdb: rdb, rdb: rdb,
scheduler: scheduler, scheduler: scheduler,
processor: processor, processor: processor,
@ -174,7 +176,7 @@ func (bg *Background) Run(handler Handler) {
sig := <-sigs sig := <-sigs
if sig == syscall.SIGTSTP { if sig == syscall.SIGTSTP {
bg.processor.stop() bg.processor.stop()
bg.heartbeater.setState("stopped") bg.pinfo.SetState("stopped")
continue continue
} }
break break
@ -215,41 +217,10 @@ func (bg *Background) stop() {
bg.syncer.terminate() bg.syncer.terminate()
bg.heartbeater.terminate() bg.heartbeater.terminate()
bg.rdb.ClearProcessInfo(bg.pinfo)
bg.rdb.Close() bg.rdb.Close()
bg.processor.handler = nil bg.processor.handler = nil
bg.running = false bg.running = false
logger.info("Bye!") logger.info("Bye!")
} }
// normalizeQueueCfg divides priority numbers by their
// greatest common divisor.
func normalizeQueueCfg(queueCfg map[string]uint) map[string]uint {
var xs []uint
for _, x := range queueCfg {
xs = append(xs, x)
}
d := gcd(xs...)
res := make(map[string]uint)
for q, x := range queueCfg {
res[q] = x / d
}
return res
}
func gcd(xs ...uint) uint {
fn := func(x, y uint) uint {
for y > 0 {
x, y = y, x%y
}
return x
}
res := xs[0]
for i := 0; i < len(xs); i++ {
res = fn(xs[i], res)
if res == 1 {
return 1
}
}
return res
}

View File

@ -5,20 +5,18 @@
package asynq package asynq
import ( import (
"sync"
"time" "time"
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
"github.com/hibiken/asynq/internal/rdb" "github.com/hibiken/asynq/internal/rdb"
) )
// heartbeater is responsible for writing process status to redis periodically to // heartbeater is responsible for writing process info to redis periodically to
// indicate that the background worker process is up. // indicate that the background worker process is up.
type heartbeater struct { type heartbeater struct {
rdb *rdb.RDB rdb *rdb.RDB
mu sync.Mutex pinfo *base.ProcessInfo
ps *base.ProcessStatus
// channel to communicate back to the long running "heartbeater" goroutine. // channel to communicate back to the long running "heartbeater" goroutine.
done chan struct{} done chan struct{}
@ -27,16 +25,10 @@ type heartbeater struct {
interval time.Duration interval time.Duration
} }
func newHeartbeater(rdb *rdb.RDB, interval time.Duration, host string, pid int, queues map[string]uint, n int) *heartbeater { func newHeartbeater(rdb *rdb.RDB, pinfo *base.ProcessInfo, interval time.Duration) *heartbeater {
ps := &base.ProcessStatus{
Concurrency: n,
Queues: queues,
Host: host,
PID: pid,
}
return &heartbeater{ return &heartbeater{
rdb: rdb, rdb: rdb,
ps: ps, pinfo: pinfo,
done: make(chan struct{}), done: make(chan struct{}),
interval: interval, interval: interval,
} }
@ -48,31 +40,28 @@ func (h *heartbeater) terminate() {
h.done <- struct{}{} h.done <- struct{}{}
} }
func (h *heartbeater) setState(state string) {
h.mu.Lock()
defer h.mu.Unlock()
h.ps.State = state
}
func (h *heartbeater) start() { func (h *heartbeater) start() {
h.ps.Started = time.Now() h.pinfo.SetStarted(time.Now())
h.ps.State = "running" h.pinfo.SetState("running")
go func() { go func() {
h.beat()
for { for {
select { select {
case <-h.done: case <-h.done:
logger.info("Heartbeater done") logger.info("Heartbeater done")
return return
case <-time.After(h.interval): case <-time.After(h.interval):
// Note: Set TTL to be long enough value so that it won't expire before we write again h.beat()
// and short enough to expire quickly once process is shut down.
h.mu.Lock()
err := h.rdb.WriteProcessStatus(h.ps, h.interval*2)
h.mu.Unlock()
if err != nil {
logger.error("could not write heartbeat data: %v", err)
}
} }
} }
}() }()
} }
func (h *heartbeater) beat() {
// Note: Set TTL to be long enough so that it won't expire before we write again
// and short enough to expire quickly once the process is shut down or killed.
err := h.rdb.WriteProcessInfo(h.pinfo, h.interval*2)
if err != nil {
logger.error("could not write heartbeat data: %v", err)
}
}

View File

@ -29,12 +29,15 @@ func TestHeartbeater(t *testing.T) {
{time.Second, "some.address.ec2.aws.com", 45678, map[string]uint{"default": 1}, 10}, {time.Second, "some.address.ec2.aws.com", 45678, map[string]uint{"default": 1}, 10},
} }
timeCmpOpt := cmpopts.EquateApproxTime(10 * time.Millisecond)
ignoreOpt := cmpopts.IgnoreUnexported(base.ProcessInfo{})
for _, tc := range tests { for _, tc := range tests {
h.FlushDB(t, r) h.FlushDB(t, r)
hb := newHeartbeater(rdbClient, tc.interval, tc.host, tc.pid, tc.queues, tc.concurrency) pi := base.NewProcessInfo(tc.host, tc.pid, tc.concurrency, tc.queues, false)
hb := newHeartbeater(rdbClient, pi, tc.interval)
want := &base.ProcessStatus{ want := &base.ProcessInfo{
Host: tc.host, Host: tc.host,
PID: tc.pid, PID: tc.pid,
Queues: tc.queues, Queues: tc.queues,
@ -47,35 +50,34 @@ func TestHeartbeater(t *testing.T) {
// allow for heartbeater to write to redis // allow for heartbeater to write to redis
time.Sleep(tc.interval * 2) time.Sleep(tc.interval * 2)
got, err := rdbClient.ReadProcessStatus(tc.host, tc.pid) got, err := rdbClient.ReadProcessInfo(tc.host, tc.pid)
if err != nil { if err != nil {
t.Errorf("could not read process status from redis: %v", err) t.Errorf("could not read process status from redis: %v", err)
hb.terminate() hb.terminate()
continue continue
} }
var timeCmpOpt = cmpopts.EquateApproxTime(10 * time.Millisecond) if diff := cmp.Diff(want, got, timeCmpOpt, ignoreOpt); diff != "" {
if diff := cmp.Diff(want, got, timeCmpOpt); diff != "" {
t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", got, want, diff) t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", got, want, diff)
hb.terminate() hb.terminate()
continue continue
} }
// state change // state change
hb.setState("stopped") pi.SetState("stopped")
// allow for heartbeater to write to redis // allow for heartbeater to write to redis
time.Sleep(tc.interval * 2) time.Sleep(tc.interval * 2)
want.State = "stopped" want.State = "stopped"
got, err = rdbClient.ReadProcessStatus(tc.host, tc.pid) got, err = rdbClient.ReadProcessInfo(tc.host, tc.pid)
if err != nil { if err != nil {
t.Errorf("could not read process status from redis: %v", err) t.Errorf("could not read process status from redis: %v", err)
hb.terminate() hb.terminate()
continue continue
} }
if diff := cmp.Diff(want, got, timeCmpOpt); diff != "" { if diff := cmp.Diff(want, got, timeCmpOpt, ignoreOpt); diff != "" {
t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", got, want, diff) t.Errorf("redis stored process status %+v, want %+v; (-want, +got)\n%s", got, want, diff)
hb.terminate() hb.terminate()
continue continue

View File

@ -41,6 +41,18 @@ var SortZSetEntryOpt = cmp.Transformer("SortZSetEntries", func(in []ZSetEntry) [
return out return out
}) })
// SortProcessInfoOpt is a cmp.Option to sort base.ProcessInfo for comparing slice of process info.
var SortProcessInfoOpt = cmp.Transformer("SortProcessInfo", func(in []*base.ProcessInfo) []*base.ProcessInfo {
out := append([]*base.ProcessInfo(nil), in...) // Copy input to avoid mutating it
sort.Slice(out, func(i, j int) bool {
if out[i].Host != out[j].Host {
return out[i].Host < out[j].Host
}
return out[i].PID < out[j].PID
})
return out
})
// IgnoreIDOpt is an cmp.Option to ignore ID field in task messages when comparing. // IgnoreIDOpt is an cmp.Option to ignore ID field in task messages when comparing.
var IgnoreIDOpt = cmpopts.IgnoreFields(base.TaskMessage{}, "ID") var IgnoreIDOpt = cmpopts.IgnoreFields(base.TaskMessage{}, "ID")

View File

@ -8,6 +8,7 @@ package base
import ( import (
"fmt" "fmt"
"strings" "strings"
"sync"
"time" "time"
"github.com/rs/xid" "github.com/rs/xid"
@ -19,6 +20,7 @@ const DefaultQueueName = "default"
// Redis keys // Redis keys
const ( const (
psPrefix = "asynq:ps:" // HASH psPrefix = "asynq:ps:" // HASH
AllProcesses = "asynq:ps" // ZSET
processedPrefix = "asynq:processed:" // STRING - asynq:processed:<yyyy-mm-dd> processedPrefix = "asynq:processed:" // STRING - asynq:processed:<yyyy-mm-dd>
failurePrefix = "asynq:failure:" // STRING - asynq:failure:<yyyy-mm-dd> failurePrefix = "asynq:failure:" // STRING - asynq:failure:<yyyy-mm-dd>
QueuePrefix = "asynq:queues:" // LIST - asynq:queues:<qname> QueuePrefix = "asynq:queues:" // LIST - asynq:queues:<qname>
@ -47,8 +49,8 @@ func FailureKey(t time.Time) string {
return failurePrefix + t.UTC().Format("2006-01-02") return failurePrefix + t.UTC().Format("2006-01-02")
} }
// ProcessStatusKey returns a redis key string for process status. // ProcessInfoKey returns a redis key string for process info.
func ProcessStatusKey(hostname string, pid int) string { func ProcessInfoKey(hostname string, pid int) string {
return fmt.Sprintf("%s%s:%d", psPrefix, hostname, pid) return fmt.Sprintf("%s%s:%d", psPrefix, hostname, pid)
} }
@ -77,12 +79,47 @@ type TaskMessage struct {
ErrorMsg string ErrorMsg string
} }
// ProcessStatus holds information about running background worker process. // ProcessInfo holds information about running background worker process.
type ProcessStatus struct { type ProcessInfo struct {
Concurrency int mu sync.Mutex
Queues map[string]uint Concurrency int
PID int Queues map[string]uint
Host string StrictPriority bool
State string PID int
Started time.Time Host string
State string
Started time.Time
ActiveWorkerCount int
}
// NewProcessInfo returns a new instance of ProcessInfo.
func NewProcessInfo(host string, pid, concurrency int, queues map[string]uint, strict bool) *ProcessInfo {
return &ProcessInfo{
Host: host,
PID: pid,
Concurrency: concurrency,
Queues: queues,
StrictPriority: strict,
}
}
// SetState set the state field of the process info.
func (p *ProcessInfo) SetState(state string) {
p.mu.Lock()
defer p.mu.Unlock()
p.State = state
}
// SetStarted set the started field of the process info.
func (p *ProcessInfo) SetStarted(t time.Time) {
p.mu.Lock()
defer p.mu.Unlock()
p.Started = t
}
// IncrActiveWorkerCount increments active worker count by delta.
func (p *ProcessInfo) IncrActiveWorkerCount(delta int) {
p.mu.Lock()
defer p.mu.Unlock()
p.ActiveWorkerCount += delta
} }

View File

@ -5,6 +5,7 @@
package base package base
import ( import (
"sync"
"testing" "testing"
"time" "time"
) )
@ -61,7 +62,7 @@ func TestFailureKey(t *testing.T) {
} }
} }
func TestProcessStatusKey(t *testing.T) { func TestProcessInfoKey(t *testing.T) {
tests := []struct { tests := []struct {
hostname string hostname string
pid int pid int
@ -72,9 +73,36 @@ func TestProcessStatusKey(t *testing.T) {
} }
for _, tc := range tests { for _, tc := range tests {
got := ProcessStatusKey(tc.hostname, tc.pid) got := ProcessInfoKey(tc.hostname, tc.pid)
if got != tc.want { if got != tc.want {
t.Errorf("ProcessStatusKey(%s, %d) = %s, want %s", tc.hostname, tc.pid, got, tc.want) t.Errorf("ProcessInfoKey(%s, %d) = %s, want %s", tc.hostname, tc.pid, got, tc.want)
} }
} }
} }
// Note: Run this test with -race flag to check for data race.
func TestProcessInfoSetter(t *testing.T) {
pi := NewProcessInfo("localhost", 1234, 8, map[string]uint{"default": 1}, false)
var wg sync.WaitGroup
wg.Add(3)
go func() {
pi.SetState("runnning")
wg.Done()
}()
go func() {
pi.SetStarted(time.Now())
pi.IncrActiveWorkerCount(1)
wg.Done()
}()
go func() {
pi.SetState("stopped")
wg.Done()
}()
wg.Wait()
}

View File

@ -755,3 +755,40 @@ func (r *RDB) RemoveQueue(qname string, force bool) error {
} }
return nil return nil
} }
// ListProcesses returns the list of process statuses.
func (r *RDB) ListProcesses() ([]*base.ProcessInfo, error) {
// Note: Script also removes stale keys.
script := redis.NewScript(`
local res = {}
local now = tonumber(ARGV[1])
local keys = redis.call("ZRANGEBYSCORE", KEYS[1], now, "+inf")
for _, key in ipairs(keys) do
local ps = redis.call("GET", key)
if ps then
table.insert(res, ps)
end
end
redis.call("ZREMRANGEBYSCORE", KEYS[1], "-inf", now-1)
return res
`)
res, err := script.Run(r.client,
[]string{base.AllProcesses}, time.Now().UTC().Unix()).Result()
if err != nil {
return nil, err
}
data, err := cast.ToStringSliceE(res)
if err != nil {
return nil, err
}
var processes []*base.ProcessInfo
for _, s := range data {
var ps base.ProcessInfo
err := json.Unmarshal([]byte(s), &ps)
if err != nil {
continue // skip bad data
}
processes = append(processes, &ps)
}
return processes, nil
}

View File

@ -2050,3 +2050,56 @@ func TestRemoveQueueError(t *testing.T) {
} }
} }
} }
func TestListProcesses(t *testing.T) {
r := setup(t)
ps1 := &base.ProcessInfo{
Concurrency: 10,
Queues: map[string]uint{"default": 1},
Host: "do.droplet1",
PID: 1234,
State: "running",
Started: time.Now().Add(-time.Hour),
ActiveWorkerCount: 5,
}
ps2 := &base.ProcessInfo{
Concurrency: 20,
Queues: map[string]uint{"email": 1},
Host: "do.droplet2",
PID: 9876,
State: "stopped",
Started: time.Now().Add(-2 * time.Hour),
ActiveWorkerCount: 20,
}
tests := []struct {
processes []*base.ProcessInfo
}{
{processes: []*base.ProcessInfo{}},
{processes: []*base.ProcessInfo{ps1}},
{processes: []*base.ProcessInfo{ps1, ps2}},
}
ignoreOpt := cmpopts.IgnoreUnexported(base.ProcessInfo{})
for _, tc := range tests {
h.FlushDB(t, r.client)
for _, ps := range tc.processes {
if err := r.WriteProcessInfo(ps, 5*time.Second); err != nil {
t.Fatal(err)
}
}
got, err := r.ListProcesses()
if err != nil {
t.Errorf("r.ListProcesses returned an error: %v", err)
}
if diff := cmp.Diff(tc.processes, got, h.SortProcessInfoOpt, ignoreOpt); diff != "" {
t.Errorf("r.ListProcesses returned %v, want %v; (-want,+got)\n%s",
got, tc.processes, diff)
}
}
}

View File

@ -347,28 +347,52 @@ func (r *RDB) forwardSingle(src, dst string) error {
[]string{src, dst}, now).Err() []string{src, dst}, now).Err()
} }
// WriteProcessStatus writes process information to redis with expiration // WriteProcessInfo writes process information to redis with expiration
// set to the value ttl. // set to the value ttl.
func (r *RDB) WriteProcessStatus(ps *base.ProcessStatus, ttl time.Duration) error { func (r *RDB) WriteProcessInfo(ps *base.ProcessInfo, ttl time.Duration) error {
bytes, err := json.Marshal(ps) bytes, err := json.Marshal(ps)
if err != nil { if err != nil {
return err return err
} }
key := base.ProcessStatusKey(ps.Host, ps.PID) // Note: Add key to ZSET with expiration time as score.
return r.client.Set(key, string(bytes), ttl).Err() // ref: https://github.com/antirez/redis/issues/135#issuecomment-2361996
exp := time.Now().Add(ttl).UTC()
key := base.ProcessInfoKey(ps.Host, ps.PID)
// KEYS[1] -> asynq:ps
// KEYS[2] -> asynq:ps:<host:pid>
// ARGV[1] -> expiration time
// ARGV[2] -> TTL in seconds
// ARGV[3] -> process info
script := redis.NewScript(`
redis.call("ZADD", KEYS[1], ARGV[1], KEYS[2])
redis.call("SETEX", KEYS[2], ARGV[2], ARGV[3])
return redis.status_reply("OK")
`)
return script.Run(r.client, []string{base.AllProcesses, key}, float64(exp.Unix()), ttl.Seconds(), string(bytes)).Err()
} }
// ReadProcessStatus reads process information stored in redis. // ReadProcessInfo reads process information stored in redis.
func (r *RDB) ReadProcessStatus(host string, pid int) (*base.ProcessStatus, error) { func (r *RDB) ReadProcessInfo(host string, pid int) (*base.ProcessInfo, error) {
key := base.ProcessStatusKey(host, pid) key := base.ProcessInfoKey(host, pid)
data, err := r.client.Get(key).Result() data, err := r.client.Get(key).Result()
if err != nil { if err != nil {
return nil, err return nil, err
} }
var ps base.ProcessStatus var pinfo base.ProcessInfo
err = json.Unmarshal([]byte(data), &ps) err = json.Unmarshal([]byte(data), &pinfo)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return &ps, nil return &pinfo, nil
}
// ClearProcessInfo deletes process information from redis.
func (r *RDB) ClearProcessInfo(ps *base.ProcessInfo) error {
key := base.ProcessInfoKey(ps.Host, ps.PID)
script := redis.NewScript(`
redis.call("ZREM", KEYS[1], KEYS[2])
redis.call("DEL", KEYS[2])
return redis.status_reply("OK")
`)
return script.Run(r.client, []string{base.AllProcesses, key}).Err()
} }

View File

@ -6,11 +6,13 @@ package rdb
import ( import (
"fmt" "fmt"
"strconv"
"testing" "testing"
"time" "time"
"github.com/go-redis/redis/v7" "github.com/go-redis/redis/v7"
"github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
h "github.com/hibiken/asynq/internal/asynqtest" h "github.com/hibiken/asynq/internal/asynqtest"
"github.com/hibiken/asynq/internal/base" "github.com/hibiken/asynq/internal/base"
) )
@ -739,48 +741,81 @@ func TestCheckAndEnqueue(t *testing.T) {
} }
} }
func TestReadWriteProcessStatus(t *testing.T) { func TestReadWriteClearProcessInfo(t *testing.T) {
r := setup(t) r := setup(t)
ps1 := &base.ProcessStatus{ pinfo := &base.ProcessInfo{
Concurrency: 10, Concurrency: 10,
Queues: map[string]uint{"default": 2, "email": 5, "low": 1}, Queues: map[string]uint{"default": 2, "email": 5, "low": 1},
PID: 98765, PID: 98765,
Host: "localhost", Host: "localhost",
State: "running", State: "running",
Started: time.Now(), Started: time.Now(),
ActiveWorkerCount: 1,
} }
tests := []struct { tests := []struct {
ps *base.ProcessStatus pi *base.ProcessInfo
ttl time.Duration ttl time.Duration
}{ }{
{ps1, 5 * time.Second}, {pinfo, 5 * time.Second},
} }
for _, tc := range tests { for _, tc := range tests {
h.FlushDB(t, r.client) h.FlushDB(t, r.client)
err := r.WriteProcessStatus(tc.ps, tc.ttl) err := r.WriteProcessInfo(tc.pi, tc.ttl)
if err != nil { if err != nil {
t.Errorf("r.WriteProcessStatus returned an error: %v", err) t.Errorf("r.WriteProcessInfo returned an error: %v", err)
continue continue
} }
got, err := r.ReadProcessStatus(tc.ps.Host, tc.ps.PID) got, err := r.ReadProcessInfo(tc.pi.Host, tc.pi.PID)
if err != nil { if err != nil {
t.Errorf("r.ReadProcessStatus returned an error: %v", err) t.Errorf("r.ReadProcessInfo returned an error: %v", err)
continue continue
} }
if diff := cmp.Diff(tc.ps, got); diff != "" { ignoreOpt := cmpopts.IgnoreUnexported(base.ProcessInfo{})
t.Errorf("r.ReadProcessStatus(%q, %d) = %+v, want %+v; (-want,+got)\n%s", if diff := cmp.Diff(tc.pi, got, ignoreOpt); diff != "" {
tc.ps.Host, tc.ps.PID, got, tc.ps, diff) t.Errorf("r.ReadProcessInfo(%q, %d) = %+v, want %+v; (-want,+got)\n%s",
tc.pi.Host, tc.pi.PID, got, tc.pi, diff)
} }
key := base.ProcessStatusKey(tc.ps.Host, tc.ps.PID) key := base.ProcessInfoKey(tc.pi.Host, tc.pi.PID)
gotTTL := r.client.TTL(key).Val() gotTTL := r.client.TTL(key).Val()
if !cmp.Equal(tc.ttl, gotTTL, timeCmpOpt) { if !cmp.Equal(tc.ttl, gotTTL, timeCmpOpt) {
t.Errorf("redis TTL %q returned %v, want %v", key, gotTTL, tc.ttl) t.Errorf("redis TTL %q returned %v, want %v", key, gotTTL, tc.ttl)
} }
now := time.Now().UTC()
allKeys, err := r.client.ZRangeByScore(base.AllProcesses, &redis.ZRangeBy{
Min: strconv.Itoa(int(now.Unix())),
Max: "+inf",
}).Result()
if err != nil {
t.Errorf("redis ZRANGEBYSCORE %q %d +inf returned an error: %v",
base.AllProcesses, now.Unix(), err)
continue
}
wantAllKeys := []string{key}
if diff := cmp.Diff(wantAllKeys, allKeys); diff != "" {
t.Errorf("all keys = %v, want %v; (-want,+got)\n%s", allKeys, wantAllKeys, diff)
}
if err := r.ClearProcessInfo(tc.pi); err != nil {
t.Errorf("r.ClearProcessInfo returned an error: %v", err)
continue
}
// 1 means key exists
if r.client.Exists(key).Val() == 1 {
t.Errorf("expected %q to be deleted", key)
}
if r.client.ZCard(base.AllProcesses).Val() != 0 {
t.Errorf("expected %q to be empty", base.AllProcesses)
}
} }
} }

View File

@ -19,6 +19,8 @@ import (
type processor struct { type processor struct {
rdb *rdb.RDB rdb *rdb.RDB
pinfo *base.ProcessInfo
handler Handler handler Handler
queueConfig map[string]uint queueConfig map[string]uint
@ -53,25 +55,21 @@ type processor struct {
type retryDelayFunc func(n int, err error, task *Task) time.Duration type retryDelayFunc func(n int, err error, task *Task) time.Duration
// newProcessor constructs a new processor. // newProcessor constructs a new processor.
// func newProcessor(r *rdb.RDB, pinfo *base.ProcessInfo, fn retryDelayFunc, syncRequestCh chan<- *syncRequest) *processor {
// r is an instance of RDB used by the processor. qcfg := normalizeQueueCfg(pinfo.Queues)
// n specifies the max number of concurrenct worker goroutines.
// qfcg is a mapping of queue names to associated priority level.
// strict specifies whether queue priority should be treated strictly.
// fn is a function to compute retry delay.
func newProcessor(r *rdb.RDB, n int, qcfg map[string]uint, strict bool, fn retryDelayFunc, syncRequestCh chan<- *syncRequest) *processor {
orderedQueues := []string(nil) orderedQueues := []string(nil)
if strict { if pinfo.StrictPriority {
orderedQueues = sortByPriority(qcfg) orderedQueues = sortByPriority(qcfg)
} }
return &processor{ return &processor{
rdb: r, rdb: r,
pinfo: pinfo,
queueConfig: qcfg, queueConfig: qcfg,
orderedQueues: orderedQueues, orderedQueues: orderedQueues,
retryDelayFunc: fn, retryDelayFunc: fn,
syncRequestCh: syncRequestCh, syncRequestCh: syncRequestCh,
errLogLimiter: rate.NewLimiter(rate.Every(3*time.Second), 1), errLogLimiter: rate.NewLimiter(rate.Every(3*time.Second), 1),
sema: make(chan struct{}, n), sema: make(chan struct{}, pinfo.Concurrency),
done: make(chan struct{}), done: make(chan struct{}),
abort: make(chan struct{}), abort: make(chan struct{}),
quit: make(chan struct{}), quit: make(chan struct{}),
@ -153,8 +151,12 @@ func (p *processor) exec() {
p.requeue(msg) p.requeue(msg)
return return
case p.sema <- struct{}{}: // acquire token case p.sema <- struct{}{}: // acquire token
p.pinfo.IncrActiveWorkerCount(1)
go func() { go func() {
defer func() { <-p.sema /* release token */ }() defer func() {
<-p.sema /* release token */
p.pinfo.IncrActiveWorkerCount(-1)
}()
resCh := make(chan error, 1) resCh := make(chan error, 1)
task := NewTask(msg.Type, msg.Payload) task := NewTask(msg.Type, msg.Payload)
@ -331,3 +333,35 @@ type byPriority []*queue
func (x byPriority) Len() int { return len(x) } func (x byPriority) Len() int { return len(x) }
func (x byPriority) Less(i, j int) bool { return x[i].priority < x[j].priority } func (x byPriority) Less(i, j int) bool { return x[i].priority < x[j].priority }
func (x byPriority) Swap(i, j int) { x[i], x[j] = x[j], x[i] } func (x byPriority) Swap(i, j int) { x[i], x[j] = x[j], x[i] }
// normalizeQueueCfg divides priority numbers by their
// greatest common divisor.
func normalizeQueueCfg(queueCfg map[string]uint) map[string]uint {
var xs []uint
for _, x := range queueCfg {
xs = append(xs, x)
}
d := gcd(xs...)
res := make(map[string]uint)
for q, x := range queueCfg {
res[q] = x / d
}
return res
}
func gcd(xs ...uint) uint {
fn := func(x, y uint) uint {
for y > 0 {
x, y = y, x%y
}
return x
}
res := xs[0]
for i := 0; i < len(xs); i++ {
res = fn(xs[i], res)
if res == 1 {
return 1
}
}
return res
}

View File

@ -65,7 +65,8 @@ func TestProcessorSuccess(t *testing.T) {
processed = append(processed, task) processed = append(processed, task)
return nil return nil
} }
p := newProcessor(rdbClient, 10, defaultQueueConfig, false, defaultDelayFunc, nil) pi := base.NewProcessInfo("localhost", 1234, 10, defaultQueueConfig, false)
p := newProcessor(rdbClient, pi, defaultDelayFunc, nil)
p.handler = HandlerFunc(handler) p.handler = HandlerFunc(handler)
p.start() p.start()
@ -148,7 +149,8 @@ func TestProcessorRetry(t *testing.T) {
handler := func(task *Task) error { handler := func(task *Task) error {
return fmt.Errorf(errMsg) return fmt.Errorf(errMsg)
} }
p := newProcessor(rdbClient, 10, defaultQueueConfig, false, delayFunc, nil) pi := base.NewProcessInfo("localhost", 1234, 10, defaultQueueConfig, false)
p := newProcessor(rdbClient, pi, delayFunc, nil)
p.handler = HandlerFunc(handler) p.handler = HandlerFunc(handler)
p.start() p.start()
@ -207,7 +209,8 @@ func TestProcessorQueues(t *testing.T) {
} }
for _, tc := range tests { for _, tc := range tests {
p := newProcessor(nil, 10, tc.queueCfg, false, defaultDelayFunc, nil) pi := base.NewProcessInfo("localhost", 1234, 10, tc.queueCfg, false)
p := newProcessor(nil, pi, defaultDelayFunc, nil)
got := p.queues() got := p.queues()
if diff := cmp.Diff(tc.want, got, sortOpt); diff != "" { if diff := cmp.Diff(tc.want, got, sortOpt); diff != "" {
t.Errorf("with queue config: %v\n(*processor).queues() = %v, want %v\n(-want,+got):\n%s", t.Errorf("with queue config: %v\n(*processor).queues() = %v, want %v\n(-want,+got):\n%s",
@ -273,7 +276,8 @@ func TestProcessorWithStrictPriority(t *testing.T) {
"low": 1, "low": 1,
} }
// Note: Set concurrency to 1 to make sure tasks are processed one at a time. // Note: Set concurrency to 1 to make sure tasks are processed one at a time.
p := newProcessor(rdbClient, 1 /*concurrency */, queueCfg, true /* strict */, defaultDelayFunc, nil) pi := base.NewProcessInfo("localhost", 1234, 1 /*concurrency */, queueCfg, true /* strict */)
p := newProcessor(rdbClient, pi, defaultDelayFunc, nil)
p.handler = HandlerFunc(handler) p.handler = HandlerFunc(handler)
p.start() p.start()

View File

@ -10,7 +10,6 @@ import (
"os" "os"
"strconv" "strconv"
"strings" "strings"
"text/tabwriter"
"time" "time"
"github.com/go-redis/redis/v7" "github.com/go-redis/redis/v7"
@ -215,18 +214,3 @@ func listDead(r *rdb.RDB) {
printTable(cols, printRows) printTable(cols, printRows)
fmt.Printf("\nShowing %d tasks from page %d\n", len(tasks), pageNum) fmt.Printf("\nShowing %d tasks from page %d\n", len(tasks), pageNum)
} }
func printTable(cols []string, printRows func(w io.Writer, tmpl string)) {
format := strings.Repeat("%v\t", len(cols)) + "\n"
tw := new(tabwriter.Writer).Init(os.Stdout, 0, 8, 2, ' ', 0)
var headers []interface{}
var seps []interface{}
for _, name := range cols {
headers = append(headers, name)
seps = append(seps, strings.Repeat("-", len(name)))
}
fmt.Fprintf(tw, format, headers...)
fmt.Fprintf(tw, format, seps...)
printRows(tw, format)
tw.Flush()
}

89
tools/asynqmon/cmd/ps.go Normal file
View File

@ -0,0 +1,89 @@
// Copyright 2020 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"
"io"
"os"
"strings"
"time"
"github.com/go-redis/redis/v7"
"github.com/hibiken/asynq/internal/rdb"
"github.com/spf13/cobra"
"github.com/spf13/viper"
)
// psCmd represents the ps command
var psCmd = &cobra.Command{
Use: "ps",
Short: "Shows all background worker processes",
Long: `Ps (asynqmon ps) will show all background worker processes
backed by the specified redis instance.
The command shows the following for each process:
* Host and PID of the process
* Number of active workers out of worker pool
* Queues configuration
* State of the process ("running" | "stopped")
A "running" process is processing tasks in queues.
A "stopped" process are no longer processing new tasks.`,
Args: cobra.NoArgs,
Run: ps,
}
func init() {
rootCmd.AddCommand(psCmd)
}
func ps(cmd *cobra.Command, args []string) {
c := redis.NewClient(&redis.Options{
Addr: viper.GetString("uri"),
DB: viper.GetInt("db"),
Password: viper.GetString("password"),
})
r := rdb.NewRDB(c)
processes, err := r.ListProcesses()
if err != nil {
fmt.Println(err)
os.Exit(1)
}
if len(processes) == 0 {
fmt.Println("No processes")
return
}
cols := []string{"Host", "PID", "State", "Active Workers", "Queues", "Started"}
printRows := func(w io.Writer, tmpl string) {
for _, ps := range processes {
fmt.Fprintf(w, tmpl,
ps.Host, ps.PID, ps.State,
fmt.Sprintf("%d/%d", ps.ActiveWorkerCount, ps.Concurrency),
formatQueues(ps.Queues), timeAgo(ps.Started))
}
}
printTable(cols, printRows)
}
// timeAgo takes a time and returns a string of the format "<duration> ago".
func timeAgo(since time.Time) string {
d := time.Since(since).Round(time.Second)
return fmt.Sprintf("%v ago", d)
}
func formatQueues(queues map[string]uint) string {
var b strings.Builder
l := len(queues)
for qname, p := range queues {
fmt.Fprintf(&b, "%s:%d", qname, p)
l--
if l > 0 {
b.WriteString(" ")
}
}
return b.String()
}

View File

@ -6,7 +6,10 @@ package cmd
import ( import (
"fmt" "fmt"
"io"
"os" "os"
"strings"
"text/tabwriter"
"github.com/spf13/cobra" "github.com/spf13/cobra"
@ -81,3 +84,36 @@ func initConfig() {
fmt.Println("Using config file:", viper.ConfigFileUsed()) fmt.Println("Using config file:", viper.ConfigFileUsed())
} }
} }
// printTable is a helper function to print data in table format.
//
// cols is a list of headers and printRow specifies how to print rows.
//
// Example:
// type User struct {
// Name string
// Addr string
// Age int
// }
// data := []*User{{"user1", "addr1", 24}, {"user2", "addr2", 42}, ...}
// cols := []string{"Name", "Addr", "Age"}
// printRows := func(w io.Writer, tmpl string) {
// for _, u := range data {
// fmt.Fprintf(w, tmpl, u.Name, u.Addr, u.Age)
// }
// }
// printTable(cols, printRows)
func printTable(cols []string, printRows func(w io.Writer, tmpl string)) {
format := strings.Repeat("%v\t", len(cols)) + "\n"
tw := new(tabwriter.Writer).Init(os.Stdout, 0, 8, 2, ' ', 0)
var headers []interface{}
var seps []interface{}
for _, name := range cols {
headers = append(headers, name)
seps = append(seps, strings.Repeat("-", len(name)))
}
fmt.Fprintf(tw, format, headers...)
fmt.Fprintf(tw, format, seps...)
printRows(tw, format)
tw.Flush()
}