2
0
mirror of https://github.com/hibiken/asynq.git synced 2024-12-27 16:13:40 +08:00

feature: configurable archive ttl and max archived size

This commit is contained in:
fajrifernanda 2024-12-04 13:23:04 +07:00
parent 106c07adaa
commit 9261e4318c
8 changed files with 172 additions and 35 deletions

View File

@ -30,6 +30,40 @@ type Client struct {
sharedConnection bool sharedConnection bool
} }
type ClientConfig struct {
MaxArchiveSize *int
ArchivedExpirationInDays *int
}
func validateClientConfig(cfg *ClientConfig) {
if cfg.MaxArchiveSize == nil {
value := base.DefaultMaxArchiveSize
cfg.MaxArchiveSize = &value
}
if cfg.ArchivedExpirationInDays == nil {
value := base.DefaultArchivedExpirationInDays
cfg.ArchivedExpirationInDays = &value
}
if *(cfg.ArchivedExpirationInDays) < 0 {
value := 1
cfg.ArchivedExpirationInDays = &value
}
}
// NewClientWithConfig returns a new Client instance given a redis connection option.
func NewClientWithConfig(r RedisConnOpt, cfg ClientConfig) *Client {
validateClientConfig(&cfg)
c, ok := r.MakeRedisClient().(redis.UniversalClient)
if !ok {
panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r))
}
return &Client{broker: rdb.NewRDBWithConfig(c, rdb.RDBConfig{
MaxArchiveSize: cfg.MaxArchiveSize,
ArchivedExpirationInDays: cfg.ArchivedExpirationInDays,
})}
}
// NewClient returns a new Client instance given a redis connection option. // NewClient returns a new Client instance given a redis connection option.
func NewClient(r RedisConnOpt) *Client { func NewClient(r RedisConnOpt) *Client {
redisClient, ok := r.MakeRedisClient().(redis.UniversalClient) redisClient, ok := r.MakeRedisClient().(redis.UniversalClient)

View File

@ -25,15 +25,45 @@ type Inspector struct {
sharedConnection bool sharedConnection bool
} }
// New returns a new instance of Inspector. type InspectorConfig struct {
func NewInspector(r RedisConnOpt) *Inspector { MaxArchiveSize *int
ArchivedExpirationInDays *int
}
func validateInspectorConfig(cfg *InspectorConfig) {
if cfg.MaxArchiveSize == nil {
value := base.DefaultMaxArchiveSize
cfg.MaxArchiveSize = &value
}
if cfg.ArchivedExpirationInDays == nil {
value := base.DefaultArchivedExpirationInDays
cfg.ArchivedExpirationInDays = &value
}
if *(cfg.ArchivedExpirationInDays) < 0 {
value := 1
cfg.ArchivedExpirationInDays = &value
}
}
// NewInspectorWithConfig returns a new instance of Inspector.
func NewInspectorWithConfig(r RedisConnOpt, cfg InspectorConfig) *Inspector {
validateInspectorConfig(&cfg)
c, ok := r.MakeRedisClient().(redis.UniversalClient) c, ok := r.MakeRedisClient().(redis.UniversalClient)
if !ok { if !ok {
panic(fmt.Sprintf("inspeq: unsupported RedisConnOpt type %T", r)) panic(fmt.Sprintf("inspeq: unsupported RedisConnOpt type %T", r))
} }
inspector := NewInspectorFromRedisClient(c) return &Inspector{
inspector.sharedConnection = false rdb: rdb.NewRDBWithConfig(c, rdb.RDBConfig{
return inspector MaxArchiveSize: cfg.MaxArchiveSize,
ArchivedExpirationInDays: cfg.ArchivedExpirationInDays,
}),
}
}
// NewInspector returns a new instance of Inspector.
func NewInspector(r RedisConnOpt) *Inspector {
return NewInspectorWithConfig(r, InspectorConfig{})
} }
// NewInspectorFromRedisClient returns a new instance of Inspector given a redis.UniversalClient // NewInspectorFromRedisClient returns a new instance of Inspector given a redis.UniversalClient

View File

@ -31,6 +31,11 @@ const DefaultQueueName = "default"
// DefaultQueue is the redis key for the default queue. // DefaultQueue is the redis key for the default queue.
var DefaultQueue = PendingKey(DefaultQueueName) var DefaultQueue = PendingKey(DefaultQueueName)
const (
DefaultMaxArchiveSize = 10000 // maximum number of tasks in archive
DefaultArchivedExpirationInDays = 90 // number of days before an archived task gets deleted permanently
)
// Global Redis keys. // Global Redis keys.
const ( const (
AllServers = "asynq:servers" // ZSET AllServers = "asynq:servers" // ZSET

View File

@ -1181,8 +1181,8 @@ func (r *RDB) ArchiveAllAggregatingTasks(qname, gname string) (int64, error) {
now := r.clock.Now() now := r.clock.Now()
argv := []interface{}{ argv := []interface{}{
now.Unix(), now.Unix(),
now.AddDate(0, 0, -archivedExpirationInDays).Unix(), now.AddDate(0, 0, -(*r.config.ArchivedExpirationInDays)).Unix(),
maxArchiveSize, *r.config.MaxArchiveSize,
base.TaskKeyPrefix(qname), base.TaskKeyPrefix(qname),
gname, gname,
} }
@ -1237,8 +1237,8 @@ func (r *RDB) ArchiveAllPendingTasks(qname string) (int64, error) {
now := r.clock.Now() now := r.clock.Now()
argv := []interface{}{ argv := []interface{}{
now.Unix(), now.Unix(),
now.AddDate(0, 0, -archivedExpirationInDays).Unix(), now.AddDate(0, 0, -(*r.config.ArchivedExpirationInDays)).Unix(),
maxArchiveSize, *r.config.MaxArchiveSize,
base.TaskKeyPrefix(qname), base.TaskKeyPrefix(qname),
} }
res, err := archiveAllPendingCmd.Run(context.Background(), r.client, keys, argv...).Result() res, err := archiveAllPendingCmd.Run(context.Background(), r.client, keys, argv...).Result()
@ -1328,8 +1328,8 @@ func (r *RDB) ArchiveTask(qname, id string) error {
argv := []interface{}{ argv := []interface{}{
id, id,
now.Unix(), now.Unix(),
now.AddDate(0, 0, -archivedExpirationInDays).Unix(), now.AddDate(0, 0, -(*r.config.ArchivedExpirationInDays)).Unix(),
maxArchiveSize, *r.config.MaxArchiveSize,
base.QueueKeyPrefix(qname), base.QueueKeyPrefix(qname),
base.GroupKeyPrefix(qname), base.GroupKeyPrefix(qname),
} }
@ -1393,8 +1393,8 @@ func (r *RDB) archiveAll(src, dst, qname string) (int64, error) {
now := r.clock.Now() now := r.clock.Now()
argv := []interface{}{ argv := []interface{}{
now.Unix(), now.Unix(),
now.AddDate(0, 0, -archivedExpirationInDays).Unix(), now.AddDate(0, 0, -(*r.config.ArchivedExpirationInDays)).Unix(),
maxArchiveSize, *r.config.MaxArchiveSize,
base.TaskKeyPrefix(qname), base.TaskKeyPrefix(qname),
qname, qname,
} }

View File

@ -30,14 +30,42 @@ type RDB struct {
client redis.UniversalClient client redis.UniversalClient
clock timeutil.Clock clock timeutil.Clock
queuesPublished sync.Map queuesPublished sync.Map
config RDBConfig
}
type RDBConfig struct {
MaxArchiveSize *int
ArchivedExpirationInDays *int
}
func validateRDBConfig(cfg *RDBConfig) {
if cfg.MaxArchiveSize == nil {
value := base.DefaultMaxArchiveSize
cfg.MaxArchiveSize = &value
}
if cfg.ArchivedExpirationInDays == nil {
value := base.DefaultArchivedExpirationInDays
cfg.ArchivedExpirationInDays = &value
}
if *(cfg.ArchivedExpirationInDays) < 0 {
value := 1
cfg.ArchivedExpirationInDays = &value
}
}
// NewRDBWithConfig returns a new instance of RDB.
func NewRDBWithConfig(client redis.UniversalClient, cfg RDBConfig) *RDB {
validateRDBConfig(&cfg)
return &RDB{
client: client,
clock: timeutil.NewRealClock(),
config: cfg,
}
} }
// NewRDB returns a new instance of RDB. // NewRDB returns a new instance of RDB.
func NewRDB(client redis.UniversalClient) *RDB { func NewRDB(client redis.UniversalClient) *RDB {
return &RDB{ return NewRDBWithConfig(client, RDBConfig{})
client: client,
clock: timeutil.NewRealClock(),
}
} }
// Close closes the connection with redis server. // Close closes the connection with redis server.
@ -836,11 +864,6 @@ func (r *RDB) Retry(ctx context.Context, msg *base.TaskMessage, processAt time.T
return r.runScript(ctx, op, retryCmd, keys, argv...) return r.runScript(ctx, op, retryCmd, keys, argv...)
} }
const (
maxArchiveSize = 10000 // maximum number of tasks in archive
archivedExpirationInDays = 90 // number of days before an archived task gets deleted permanently
)
// KEYS[1] -> asynq:{<qname>}:t:<task_id> // KEYS[1] -> asynq:{<qname>}:t:<task_id>
// KEYS[2] -> asynq:{<qname>}:active // KEYS[2] -> asynq:{<qname>}:active
// KEYS[3] -> asynq:{<qname>}:lease // KEYS[3] -> asynq:{<qname>}:lease
@ -904,6 +927,10 @@ return redis.status_reply("OK")`)
// Archive sends the given task to archive, attaching the error message to the task. // Archive sends the given task to archive, attaching the error message to the task.
// It also trims the archive by timestamp and set size. // It also trims the archive by timestamp and set size.
func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string) error { func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string) error {
if *(r.config.MaxArchiveSize) <= 0 {
return nil
}
var op errors.Op = "rdb.Archive" var op errors.Op = "rdb.Archive"
now := r.clock.Now() now := r.clock.Now()
modified := *msg modified := *msg
@ -913,7 +940,8 @@ func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string)
if err != nil { if err != nil {
return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode message: %v", err)) return errors.E(op, errors.Internal, fmt.Sprintf("cannot encode message: %v", err))
} }
cutoff := now.AddDate(0, 0, -archivedExpirationInDays) cutoff := now.AddDate(0, 0, -(*r.config.ArchivedExpirationInDays))
expireAt := now.Add(statsTTL) expireAt := now.Add(statsTTL)
keys := []string{ keys := []string{
base.TaskKey(msg.Queue, msg.ID), base.TaskKey(msg.Queue, msg.ID),
@ -931,7 +959,7 @@ func (r *RDB) Archive(ctx context.Context, msg *base.TaskMessage, errMsg string)
encoded, encoded,
now.Unix(), now.Unix(),
cutoff.Unix(), cutoff.Unix(),
maxArchiveSize, *r.config.MaxArchiveSize,
expireAt.Unix(), expireAt.Unix(),
int64(math.MaxInt64), int64(math.MaxInt64),
} }

View File

@ -2250,7 +2250,7 @@ func TestArchiveTrim(t *testing.T) {
errMsg := "SMTP server not responding" errMsg := "SMTP server not responding"
maxArchiveSet := make([]base.Z, 0) maxArchiveSet := make([]base.Z, 0)
for i := 0; i < maxArchiveSize-1; i++ { for i := 0; i < base.DefaultMaxArchiveSize-1; i++ {
maxArchiveSet = append(maxArchiveSet, base.Z{Message: &base.TaskMessage{ maxArchiveSet = append(maxArchiveSet, base.Z{Message: &base.TaskMessage{
ID: uuid.NewString(), ID: uuid.NewString(),
Type: "generate_csv", Type: "generate_csv",
@ -2306,7 +2306,7 @@ func TestArchiveTrim(t *testing.T) {
}, },
archived: map[string][]base.Z{ archived: map[string][]base.Z{
"default": { "default": {
{Message: t2, Score: now.Add(-time.Hour * 24 * (archivedExpirationInDays + 1)).Unix()}, {Message: t2, Score: now.Add(-time.Hour * 24 * (base.DefaultArchivedExpirationInDays + 1)).Unix()},
}, },
}, },
wantArchived: map[string][]base.Z{ wantArchived: map[string][]base.Z{

View File

@ -94,7 +94,10 @@ func NewSchedulerFromRedisClient(c redis.UniversalClient, opts *SchedulerOpts) *
heartbeatInterval: heartbeatInterval, heartbeatInterval: heartbeatInterval,
logger: logger, logger: logger,
client: NewClientFromRedisClient(c), client: NewClientFromRedisClient(c),
rdb: rdb.NewRDB(c), rdb: rdb.NewRDBWithConfig(c, rdb.RDBConfig{
MaxArchiveSize: opts.MaxArchiveSize,
ArchivedExpirationInDays: opts.ArchivedExpirationInDays,
}),
cron: cron.New(cron.WithLocation(loc)), cron: cron.New(cron.WithLocation(loc)),
location: loc, location: loc,
done: make(chan struct{}), done: make(chan struct{}),
@ -151,6 +154,16 @@ type SchedulerOpts struct {
// EnqueueErrorHandler gets called when scheduler cannot enqueue a registered task // EnqueueErrorHandler gets called when scheduler cannot enqueue a registered task
// due to an error. // due to an error.
EnqueueErrorHandler func(task *Task, opts []Option, err error) EnqueueErrorHandler func(task *Task, opts []Option, err error)
// MaxArchiveSize specifies the maximum size of the archive that can be created by the server.
//
// If unset the DefaultMaxArchiveSize is used. If set to a zero or a negative value, nothing will be archived.
MaxArchiveSize *int
// ArchivedExpirationInDays specifies the number of days after which archived tasks are deleted.
//
// If unset, DefaultArchivedExpirationInDays is used. The value must be greater than zero.
ArchivedExpirationInDays *int
} }
// enqueueJob encapsulates the job of enqueuing a task and recording the event. // enqueueJob encapsulates the job of enqueuing a task and recording the event.

View File

@ -253,6 +253,14 @@ type Config struct {
// If unset or zero, default batch size of 100 is used. // If unset or zero, default batch size of 100 is used.
// Make sure to not put a big number as the batch size to prevent a long-running script. // Make sure to not put a big number as the batch size to prevent a long-running script.
JanitorBatchSize int JanitorBatchSize int
// If unset the DefaultMaxArchiveSize is used. If set to a zero or a negative value, nothing will be archived.
MaxArchiveSize *int
// ArchivedExpirationInDays specifies the number of days after which archived tasks are deleted.
//
// If unset, DefaultArchivedExpirationInDays is used. The value must be greater than zero.
ArchivedExpirationInDays *int
} }
// GroupAggregator aggregates a group of tasks into one before the tasks are passed to the Handler. // GroupAggregator aggregates a group of tasks into one before the tasks are passed to the Handler.
@ -427,9 +435,25 @@ const (
defaultJanitorBatchSize = 100 defaultJanitorBatchSize = 100
) )
func validateConfig(cfg *Config) {
if cfg.MaxArchiveSize == nil {
value := base.DefaultMaxArchiveSize
cfg.MaxArchiveSize = &value
}
if cfg.ArchivedExpirationInDays == nil {
value := base.DefaultArchivedExpirationInDays
cfg.ArchivedExpirationInDays = &value
}
if *(cfg.ArchivedExpirationInDays) < 0 {
value := 1
cfg.ArchivedExpirationInDays = &value
}
}
// NewServer returns a new Server given a redis connection option // NewServer returns a new Server given a redis connection option
// and server configuration. // and server configuration.
func NewServer(r RedisConnOpt, cfg Config) *Server { func NewServer(r RedisConnOpt, cfg Config) *Server {
validateConfig(&cfg)
redisClient, ok := r.MakeRedisClient().(redis.UniversalClient) redisClient, ok := r.MakeRedisClient().(redis.UniversalClient)
if !ok { if !ok {
panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r)) panic(fmt.Sprintf("asynq: unsupported RedisConnOpt type %T", r))
@ -504,7 +528,10 @@ func NewServerFromRedisClient(c redis.UniversalClient, cfg Config) *Server {
} }
logger.SetLevel(toInternalLogLevel(loglevel)) logger.SetLevel(toInternalLogLevel(loglevel))
rdb := rdb.NewRDB(c) rdb := rdb.NewRDBWithConfig(c, rdb.RDBConfig{
MaxArchiveSize: cfg.MaxArchiveSize,
ArchivedExpirationInDays: cfg.ArchivedExpirationInDays,
})
starting := make(chan *workerInfo) starting := make(chan *workerInfo)
finished := make(chan *base.TaskMessage) finished := make(chan *base.TaskMessage)
syncCh := make(chan *syncRequest) syncCh := make(chan *syncRequest)