mirror of https://github.com/go-gitea/gitea
Rewrite queue (#24505)
# ⚠️ Breaking Many deprecated queue config options are removed (actually, they should have been removed in 1.18/1.19). If you see the fatal message when starting Gitea: "Please update your app.ini to remove deprecated config options", please follow the error messages to remove these options from your app.ini. Example: ``` 2023/05/06 19:39:22 [E] Removed queue option: `[indexer].ISSUE_INDEXER_QUEUE_TYPE`. Use new options in `[queue.issue_indexer]` 2023/05/06 19:39:22 [E] Removed queue option: `[indexer].UPDATE_BUFFER_LEN`. Use new options in `[queue.issue_indexer]` 2023/05/06 19:39:22 [F] Please update your app.ini to remove deprecated config options ``` Many options in `[queue]` are are dropped, including: `WRAP_IF_NECESSARY`, `MAX_ATTEMPTS`, `TIMEOUT`, `WORKERS`, `BLOCK_TIMEOUT`, `BOOST_TIMEOUT`, `BOOST_WORKERS`, they can be removed from app.ini. # The problem The old queue package has some legacy problems: * complexity: I doubt few people could tell how it works. * maintainability: Too many channels and mutex/cond are mixed together, too many different structs/interfaces depends each other. * stability: due to the complexity & maintainability, sometimes there are strange bugs and difficult to debug, and some code doesn't have test (indeed some code is difficult to test because a lot of things are mixed together). * general applicability: although it is called "queue", its behavior is not a well-known queue. * scalability: it doesn't seem easy to make it work with a cluster without breaking its behaviors. It came from some very old code to "avoid breaking", however, its technical debt is too heavy now. It's a good time to introduce a better "queue" package. # The new queue package It keeps using old config and concept as much as possible. * It only contains two major kinds of concepts: * The "base queue": channel, levelqueue, redis * They have the same abstraction, the same interface, and they are tested by the same testing code. * The "WokerPoolQueue", it uses the "base queue" to provide "worker pool" function, calls the "handler" to process the data in the base queue. * The new code doesn't do "PushBack" * Think about a queue with many workers, the "PushBack" can't guarantee the order for re-queued unhandled items, so in new code it just does "normal push" * The new code doesn't do "pause/resume" * The "pause/resume" was designed to handle some handler's failure: eg: document indexer (elasticsearch) is down * If a queue is paused for long time, either the producers blocks or the new items are dropped. * The new code doesn't do such "pause/resume" trick, it's not a common queue's behavior and it doesn't help much. * If there are unhandled items, the "push" function just blocks for a few seconds and then re-queue them and retry. * The new code doesn't do "worker booster" * Gitea's queue's handlers are light functions, the cost is only the go-routine, so it doesn't make sense to "boost" them. * The new code only use "max worker number" to limit the concurrent workers. * The new "Push" never blocks forever * Instead of creating more and more blocking goroutines, return an error is more friendly to the server and to the end user. There are more details in code comments: eg: the "Flush" problem, the strange "code.index" hanging problem, the "immediate" queue problem. Almost ready for review. TODO: * [x] add some necessary comments during review * [x] add some more tests if necessary * [x] update documents and config options * [x] test max worker / active worker * [x] re-run the CI tasks to see whether any test is flaky * [x] improve the `handleOldLengthConfiguration` to provide more friendly messages * [x] fine tune default config values (eg: length?) ## Code coverage: ![image](https://user-images.githubusercontent.com/2114189/236620635-55576955-f95d-4810-b12f-879026a3afdf.png)pull/24588/head^2
parent
cb700aedd1
commit
6f9c278559
@ -1,180 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package base |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"os" |
|
||||||
"runtime" |
|
||||||
"strings" |
|
||||||
"sync" |
|
||||||
"testing" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/json" |
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
"code.gitea.io/gitea/modules/queue" |
|
||||||
) |
|
||||||
|
|
||||||
var ( |
|
||||||
prefix string |
|
||||||
slowTest = 10 * time.Second |
|
||||||
slowFlush = 5 * time.Second |
|
||||||
) |
|
||||||
|
|
||||||
// TestLogger is a logger which will write to the testing log
|
|
||||||
type TestLogger struct { |
|
||||||
log.WriterLogger |
|
||||||
} |
|
||||||
|
|
||||||
var writerCloser = &testLoggerWriterCloser{} |
|
||||||
|
|
||||||
type testLoggerWriterCloser struct { |
|
||||||
sync.RWMutex |
|
||||||
t []*testing.TB |
|
||||||
} |
|
||||||
|
|
||||||
func (w *testLoggerWriterCloser) setT(t *testing.TB) { |
|
||||||
w.Lock() |
|
||||||
w.t = append(w.t, t) |
|
||||||
w.Unlock() |
|
||||||
} |
|
||||||
|
|
||||||
func (w *testLoggerWriterCloser) Write(p []byte) (int, error) { |
|
||||||
w.RLock() |
|
||||||
var t *testing.TB |
|
||||||
if len(w.t) > 0 { |
|
||||||
t = w.t[len(w.t)-1] |
|
||||||
} |
|
||||||
w.RUnlock() |
|
||||||
if t != nil && *t != nil { |
|
||||||
if len(p) > 0 && p[len(p)-1] == '\n' { |
|
||||||
p = p[:len(p)-1] |
|
||||||
} |
|
||||||
|
|
||||||
defer func() { |
|
||||||
err := recover() |
|
||||||
if err == nil { |
|
||||||
return |
|
||||||
} |
|
||||||
var errString string |
|
||||||
errErr, ok := err.(error) |
|
||||||
if ok { |
|
||||||
errString = errErr.Error() |
|
||||||
} else { |
|
||||||
errString, ok = err.(string) |
|
||||||
} |
|
||||||
if !ok { |
|
||||||
panic(err) |
|
||||||
} |
|
||||||
if !strings.HasPrefix(errString, "Log in goroutine after ") { |
|
||||||
panic(err) |
|
||||||
} |
|
||||||
}() |
|
||||||
|
|
||||||
(*t).Log(string(p)) |
|
||||||
return len(p), nil |
|
||||||
} |
|
||||||
return len(p), nil |
|
||||||
} |
|
||||||
|
|
||||||
func (w *testLoggerWriterCloser) Close() error { |
|
||||||
w.Lock() |
|
||||||
if len(w.t) > 0 { |
|
||||||
w.t = w.t[:len(w.t)-1] |
|
||||||
} |
|
||||||
w.Unlock() |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// PrintCurrentTest prints the current test to os.Stdout
|
|
||||||
func PrintCurrentTest(t testing.TB, skip ...int) func() { |
|
||||||
start := time.Now() |
|
||||||
actualSkip := 1 |
|
||||||
if len(skip) > 0 { |
|
||||||
actualSkip = skip[0] |
|
||||||
} |
|
||||||
_, filename, line, _ := runtime.Caller(actualSkip) |
|
||||||
|
|
||||||
if log.CanColorStdout { |
|
||||||
fmt.Fprintf(os.Stdout, "=== %s (%s:%d)\n", fmt.Formatter(log.NewColoredValue(t.Name())), strings.TrimPrefix(filename, prefix), line) |
|
||||||
} else { |
|
||||||
fmt.Fprintf(os.Stdout, "=== %s (%s:%d)\n", t.Name(), strings.TrimPrefix(filename, prefix), line) |
|
||||||
} |
|
||||||
writerCloser.setT(&t) |
|
||||||
return func() { |
|
||||||
took := time.Since(start) |
|
||||||
if took > slowTest { |
|
||||||
if log.CanColorStdout { |
|
||||||
fmt.Fprintf(os.Stdout, "+++ %s is a slow test (took %v)\n", fmt.Formatter(log.NewColoredValue(t.Name(), log.Bold, log.FgYellow)), fmt.Formatter(log.NewColoredValue(took, log.Bold, log.FgYellow))) |
|
||||||
} else { |
|
||||||
fmt.Fprintf(os.Stdout, "+++ %s is a slow test (took %v)\n", t.Name(), took) |
|
||||||
} |
|
||||||
} |
|
||||||
timer := time.AfterFunc(slowFlush, func() { |
|
||||||
if log.CanColorStdout { |
|
||||||
fmt.Fprintf(os.Stdout, "+++ %s ... still flushing after %v ...\n", fmt.Formatter(log.NewColoredValue(t.Name(), log.Bold, log.FgRed)), slowFlush) |
|
||||||
} else { |
|
||||||
fmt.Fprintf(os.Stdout, "+++ %s ... still flushing after %v ...\n", t.Name(), slowFlush) |
|
||||||
} |
|
||||||
}) |
|
||||||
if err := queue.GetManager().FlushAll(context.Background(), -1); err != nil { |
|
||||||
t.Errorf("Flushing queues failed with error %v", err) |
|
||||||
} |
|
||||||
timer.Stop() |
|
||||||
flushTook := time.Since(start) - took |
|
||||||
if flushTook > slowFlush { |
|
||||||
if log.CanColorStdout { |
|
||||||
fmt.Fprintf(os.Stdout, "+++ %s had a slow clean-up flush (took %v)\n", fmt.Formatter(log.NewColoredValue(t.Name(), log.Bold, log.FgRed)), fmt.Formatter(log.NewColoredValue(flushTook, log.Bold, log.FgRed))) |
|
||||||
} else { |
|
||||||
fmt.Fprintf(os.Stdout, "+++ %s had a slow clean-up flush (took %v)\n", t.Name(), flushTook) |
|
||||||
} |
|
||||||
} |
|
||||||
_ = writerCloser.Close() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Printf takes a format and args and prints the string to os.Stdout
|
|
||||||
func Printf(format string, args ...interface{}) { |
|
||||||
if log.CanColorStdout { |
|
||||||
for i := 0; i < len(args); i++ { |
|
||||||
args[i] = log.NewColoredValue(args[i]) |
|
||||||
} |
|
||||||
} |
|
||||||
fmt.Fprintf(os.Stdout, "\t"+format, args...) |
|
||||||
} |
|
||||||
|
|
||||||
// NewTestLogger creates a TestLogger as a log.LoggerProvider
|
|
||||||
func NewTestLogger() log.LoggerProvider { |
|
||||||
logger := &TestLogger{} |
|
||||||
logger.Colorize = log.CanColorStdout |
|
||||||
logger.Level = log.TRACE |
|
||||||
return logger |
|
||||||
} |
|
||||||
|
|
||||||
// Init inits connection writer with json config.
|
|
||||||
// json config only need key "level".
|
|
||||||
func (log *TestLogger) Init(config string) error { |
|
||||||
err := json.Unmarshal([]byte(config), log) |
|
||||||
if err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
log.NewWriterLogger(writerCloser) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Flush when log should be flushed
|
|
||||||
func (log *TestLogger) Flush() { |
|
||||||
} |
|
||||||
|
|
||||||
// ReleaseReopen does nothing
|
|
||||||
func (log *TestLogger) ReleaseReopen() error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// GetName returns the default name for this implementation
|
|
||||||
func (log *TestLogger) GetName() string { |
|
||||||
return "test" |
|
||||||
} |
|
@ -0,0 +1,63 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"time" |
||||||
|
) |
||||||
|
|
||||||
|
const ( |
||||||
|
backoffBegin = 50 * time.Millisecond |
||||||
|
backoffUpper = 2 * time.Second |
||||||
|
) |
||||||
|
|
||||||
|
type ( |
||||||
|
backoffFuncRetErr[T any] func() (retry bool, ret T, err error) |
||||||
|
backoffFuncErr func() (retry bool, err error) |
||||||
|
) |
||||||
|
|
||||||
|
func backoffRetErr[T any](ctx context.Context, begin, upper time.Duration, end <-chan time.Time, fn backoffFuncRetErr[T]) (ret T, err error) { |
||||||
|
d := begin |
||||||
|
for { |
||||||
|
// check whether the context has been cancelled or has reached the deadline, return early
|
||||||
|
select { |
||||||
|
case <-ctx.Done(): |
||||||
|
return ret, ctx.Err() |
||||||
|
case <-end: |
||||||
|
return ret, context.DeadlineExceeded |
||||||
|
default: |
||||||
|
} |
||||||
|
|
||||||
|
// call the target function
|
||||||
|
retry, ret, err := fn() |
||||||
|
if err != nil { |
||||||
|
return ret, err |
||||||
|
} |
||||||
|
if !retry { |
||||||
|
return ret, nil |
||||||
|
} |
||||||
|
|
||||||
|
// wait for a while before retrying, and also respect the context & deadline
|
||||||
|
select { |
||||||
|
case <-ctx.Done(): |
||||||
|
return ret, ctx.Err() |
||||||
|
case <-time.After(d): |
||||||
|
d *= 2 |
||||||
|
if d > upper { |
||||||
|
d = upper |
||||||
|
} |
||||||
|
case <-end: |
||||||
|
return ret, context.DeadlineExceeded |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func backoffErr(ctx context.Context, begin, upper time.Duration, end <-chan time.Time, fn backoffFuncErr) error { |
||||||
|
_, err := backoffRetErr(ctx, begin, upper, end, func() (retry bool, ret any, err error) { |
||||||
|
retry, err = fn() |
||||||
|
return retry, nil, err |
||||||
|
}) |
||||||
|
return err |
||||||
|
} |
@ -0,0 +1,42 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"time" |
||||||
|
) |
||||||
|
|
||||||
|
var pushBlockTime = 5 * time.Second |
||||||
|
|
||||||
|
type baseQueue interface { |
||||||
|
PushItem(ctx context.Context, data []byte) error |
||||||
|
PopItem(ctx context.Context) ([]byte, error) |
||||||
|
HasItem(ctx context.Context, data []byte) (bool, error) |
||||||
|
Len(ctx context.Context) (int, error) |
||||||
|
Close() error |
||||||
|
RemoveAll(ctx context.Context) error |
||||||
|
} |
||||||
|
|
||||||
|
func popItemByChan(ctx context.Context, popItemFn func(ctx context.Context) ([]byte, error)) (chanItem chan []byte, chanErr chan error) { |
||||||
|
chanItem = make(chan []byte) |
||||||
|
chanErr = make(chan error) |
||||||
|
go func() { |
||||||
|
for { |
||||||
|
it, err := popItemFn(ctx) |
||||||
|
if err != nil { |
||||||
|
close(chanItem) |
||||||
|
chanErr <- err |
||||||
|
return |
||||||
|
} |
||||||
|
if it == nil { |
||||||
|
close(chanItem) |
||||||
|
close(chanErr) |
||||||
|
return |
||||||
|
} |
||||||
|
chanItem <- it |
||||||
|
} |
||||||
|
}() |
||||||
|
return chanItem, chanErr |
||||||
|
} |
@ -0,0 +1,123 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"errors" |
||||||
|
"sync" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/container" |
||||||
|
) |
||||||
|
|
||||||
|
var errChannelClosed = errors.New("channel is closed") |
||||||
|
|
||||||
|
type baseChannel struct { |
||||||
|
c chan []byte |
||||||
|
set container.Set[string] |
||||||
|
mu sync.Mutex |
||||||
|
|
||||||
|
isUnique bool |
||||||
|
} |
||||||
|
|
||||||
|
var _ baseQueue = (*baseChannel)(nil) |
||||||
|
|
||||||
|
func newBaseChannelGeneric(cfg *BaseConfig, unique bool) (baseQueue, error) { |
||||||
|
q := &baseChannel{c: make(chan []byte, cfg.Length), isUnique: unique} |
||||||
|
if unique { |
||||||
|
q.set = container.Set[string]{} |
||||||
|
} |
||||||
|
return q, nil |
||||||
|
} |
||||||
|
|
||||||
|
func newBaseChannelSimple(cfg *BaseConfig) (baseQueue, error) { |
||||||
|
return newBaseChannelGeneric(cfg, false) |
||||||
|
} |
||||||
|
|
||||||
|
func newBaseChannelUnique(cfg *BaseConfig) (baseQueue, error) { |
||||||
|
return newBaseChannelGeneric(cfg, true) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseChannel) PushItem(ctx context.Context, data []byte) error { |
||||||
|
if q.c == nil { |
||||||
|
return errChannelClosed |
||||||
|
} |
||||||
|
|
||||||
|
if q.isUnique { |
||||||
|
q.mu.Lock() |
||||||
|
has := q.set.Contains(string(data)) |
||||||
|
q.mu.Unlock() |
||||||
|
if has { |
||||||
|
return ErrAlreadyInQueue |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
select { |
||||||
|
case q.c <- data: |
||||||
|
if q.isUnique { |
||||||
|
q.mu.Lock() |
||||||
|
q.set.Add(string(data)) |
||||||
|
q.mu.Unlock() |
||||||
|
} |
||||||
|
return nil |
||||||
|
case <-time.After(pushBlockTime): |
||||||
|
return context.DeadlineExceeded |
||||||
|
case <-ctx.Done(): |
||||||
|
return ctx.Err() |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseChannel) PopItem(ctx context.Context) ([]byte, error) { |
||||||
|
select { |
||||||
|
case data, ok := <-q.c: |
||||||
|
if !ok { |
||||||
|
return nil, errChannelClosed |
||||||
|
} |
||||||
|
q.mu.Lock() |
||||||
|
q.set.Remove(string(data)) |
||||||
|
q.mu.Unlock() |
||||||
|
return data, nil |
||||||
|
case <-ctx.Done(): |
||||||
|
return nil, ctx.Err() |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseChannel) HasItem(ctx context.Context, data []byte) (bool, error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
return q.set.Contains(string(data)), nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseChannel) Len(ctx context.Context) (int, error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
if q.c == nil { |
||||||
|
return 0, errChannelClosed |
||||||
|
} |
||||||
|
|
||||||
|
return len(q.c), nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseChannel) Close() error { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
close(q.c) |
||||||
|
q.set = container.Set[string]{} |
||||||
|
|
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseChannel) RemoveAll(ctx context.Context) error { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
for q.c != nil && len(q.c) > 0 { |
||||||
|
<-q.c |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
@ -0,0 +1,11 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import "testing" |
||||||
|
|
||||||
|
func TestBaseChannel(t *testing.T) { |
||||||
|
testQueueBasic(t, newBaseChannelSimple, &BaseConfig{ManagedName: "baseChannel", Length: 10}, false) |
||||||
|
testQueueBasic(t, newBaseChannelUnique, &BaseConfig{ManagedName: "baseChannel", Length: 10}, true) |
||||||
|
} |
@ -0,0 +1,38 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import "context" |
||||||
|
|
||||||
|
type baseDummy struct{} |
||||||
|
|
||||||
|
var _ baseQueue = (*baseDummy)(nil) |
||||||
|
|
||||||
|
func newBaseDummy(cfg *BaseConfig, unique bool) (baseQueue, error) { |
||||||
|
return &baseDummy{}, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseDummy) PushItem(ctx context.Context, data []byte) error { |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseDummy) PopItem(ctx context.Context) ([]byte, error) { |
||||||
|
return nil, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseDummy) Len(ctx context.Context) (int, error) { |
||||||
|
return 0, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseDummy) HasItem(ctx context.Context, data []byte) (bool, error) { |
||||||
|
return false, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseDummy) Close() error { |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseDummy) RemoveAll(ctx context.Context) error { |
||||||
|
return nil |
||||||
|
} |
@ -0,0 +1,72 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/nosql" |
||||||
|
|
||||||
|
"gitea.com/lunny/levelqueue" |
||||||
|
) |
||||||
|
|
||||||
|
type baseLevelQueue struct { |
||||||
|
internal *levelqueue.Queue |
||||||
|
conn string |
||||||
|
cfg *BaseConfig |
||||||
|
} |
||||||
|
|
||||||
|
var _ baseQueue = (*baseLevelQueue)(nil) |
||||||
|
|
||||||
|
func newBaseLevelQueueGeneric(cfg *BaseConfig, unique bool) (baseQueue, error) { |
||||||
|
if unique { |
||||||
|
return newBaseLevelQueueUnique(cfg) |
||||||
|
} |
||||||
|
return newBaseLevelQueueSimple(cfg) |
||||||
|
} |
||||||
|
|
||||||
|
func newBaseLevelQueueSimple(cfg *BaseConfig) (baseQueue, error) { |
||||||
|
conn, db, err := prepareLevelDB(cfg) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
q := &baseLevelQueue{conn: conn, cfg: cfg} |
||||||
|
q.internal, err = levelqueue.NewQueue(db, []byte(cfg.QueueFullName), false) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
|
||||||
|
return q, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueue) PushItem(ctx context.Context, data []byte) error { |
||||||
|
return baseLevelQueueCommon(q.cfg, q.internal, nil).PushItem(ctx, data) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueue) PopItem(ctx context.Context) ([]byte, error) { |
||||||
|
return baseLevelQueueCommon(q.cfg, q.internal, nil).PopItem(ctx) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueue) HasItem(ctx context.Context, data []byte) (bool, error) { |
||||||
|
return false, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueue) Len(ctx context.Context) (int, error) { |
||||||
|
return int(q.internal.Len()), nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueue) Close() error { |
||||||
|
err := q.internal.Close() |
||||||
|
_ = nosql.GetManager().CloseLevelDB(q.conn) |
||||||
|
return err |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueue) RemoveAll(ctx context.Context) error { |
||||||
|
for q.internal.Len() > 0 { |
||||||
|
if _, err := q.internal.LPop(); err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
@ -0,0 +1,92 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"fmt" |
||||||
|
"path/filepath" |
||||||
|
"strings" |
||||||
|
"sync" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/nosql" |
||||||
|
|
||||||
|
"gitea.com/lunny/levelqueue" |
||||||
|
"github.com/syndtr/goleveldb/leveldb" |
||||||
|
) |
||||||
|
|
||||||
|
type baseLevelQueuePushPoper interface { |
||||||
|
RPush(data []byte) error |
||||||
|
LPop() ([]byte, error) |
||||||
|
Len() int64 |
||||||
|
} |
||||||
|
|
||||||
|
type baseLevelQueueCommonImpl struct { |
||||||
|
length int |
||||||
|
internal baseLevelQueuePushPoper |
||||||
|
mu *sync.Mutex |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueCommonImpl) PushItem(ctx context.Context, data []byte) error { |
||||||
|
return backoffErr(ctx, backoffBegin, backoffUpper, time.After(pushBlockTime), func() (retry bool, err error) { |
||||||
|
if q.mu != nil { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
cnt := int(q.internal.Len()) |
||||||
|
if cnt >= q.length { |
||||||
|
return true, nil |
||||||
|
} |
||||||
|
retry, err = false, q.internal.RPush(data) |
||||||
|
if err == levelqueue.ErrAlreadyInQueue { |
||||||
|
err = ErrAlreadyInQueue |
||||||
|
} |
||||||
|
return retry, err |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueCommonImpl) PopItem(ctx context.Context) ([]byte, error) { |
||||||
|
return backoffRetErr(ctx, backoffBegin, backoffUpper, infiniteTimerC, func() (retry bool, data []byte, err error) { |
||||||
|
if q.mu != nil { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
data, err = q.internal.LPop() |
||||||
|
if err == levelqueue.ErrNotFound { |
||||||
|
return true, nil, nil |
||||||
|
} |
||||||
|
if err != nil { |
||||||
|
return false, nil, err |
||||||
|
} |
||||||
|
return false, data, nil |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func baseLevelQueueCommon(cfg *BaseConfig, internal baseLevelQueuePushPoper, mu *sync.Mutex) *baseLevelQueueCommonImpl { |
||||||
|
return &baseLevelQueueCommonImpl{length: cfg.Length, internal: internal} |
||||||
|
} |
||||||
|
|
||||||
|
func prepareLevelDB(cfg *BaseConfig) (conn string, db *leveldb.DB, err error) { |
||||||
|
if cfg.ConnStr == "" { // use data dir as conn str
|
||||||
|
if !filepath.IsAbs(cfg.DataFullDir) { |
||||||
|
return "", nil, fmt.Errorf("invalid leveldb data dir (not absolute): %q", cfg.DataFullDir) |
||||||
|
} |
||||||
|
conn = cfg.DataFullDir |
||||||
|
} else { |
||||||
|
if !strings.HasPrefix(cfg.ConnStr, "leveldb://") { |
||||||
|
return "", nil, fmt.Errorf("invalid leveldb connection string: %q", cfg.ConnStr) |
||||||
|
} |
||||||
|
conn = cfg.ConnStr |
||||||
|
} |
||||||
|
for i := 0; i < 10; i++ { |
||||||
|
if db, err = nosql.GetManager().GetLevelDB(conn); err == nil { |
||||||
|
break |
||||||
|
} |
||||||
|
time.Sleep(1 * time.Second) |
||||||
|
} |
||||||
|
return conn, db, err |
||||||
|
} |
@ -0,0 +1,23 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"testing" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert" |
||||||
|
) |
||||||
|
|
||||||
|
func TestBaseLevelDB(t *testing.T) { |
||||||
|
_, err := newBaseLevelQueueGeneric(&BaseConfig{ConnStr: "redis://"}, false) |
||||||
|
assert.ErrorContains(t, err, "invalid leveldb connection string") |
||||||
|
|
||||||
|
_, err = newBaseLevelQueueGeneric(&BaseConfig{DataFullDir: "relative"}, false) |
||||||
|
assert.ErrorContains(t, err, "invalid leveldb data dir") |
||||||
|
|
||||||
|
testQueueBasic(t, newBaseLevelQueueSimple, toBaseConfig("baseLevelQueue", setting.QueueSettings{Datadir: t.TempDir() + "/queue-test", Length: 10}), false) |
||||||
|
testQueueBasic(t, newBaseLevelQueueUnique, toBaseConfig("baseLevelQueueUnique", setting.QueueSettings{ConnStr: "leveldb://" + t.TempDir() + "/queue-test", Length: 10}), true) |
||||||
|
} |
@ -0,0 +1,93 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"sync" |
||||||
|
"unsafe" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/nosql" |
||||||
|
|
||||||
|
"gitea.com/lunny/levelqueue" |
||||||
|
"github.com/syndtr/goleveldb/leveldb" |
||||||
|
) |
||||||
|
|
||||||
|
type baseLevelQueueUnique struct { |
||||||
|
internal *levelqueue.UniqueQueue |
||||||
|
conn string |
||||||
|
cfg *BaseConfig |
||||||
|
|
||||||
|
mu sync.Mutex // the levelqueue.UniqueQueue is not thread-safe, there is no mutex protecting the underlying queue&set together
|
||||||
|
} |
||||||
|
|
||||||
|
var _ baseQueue = (*baseLevelQueueUnique)(nil) |
||||||
|
|
||||||
|
func newBaseLevelQueueUnique(cfg *BaseConfig) (baseQueue, error) { |
||||||
|
conn, db, err := prepareLevelDB(cfg) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
q := &baseLevelQueueUnique{conn: conn, cfg: cfg} |
||||||
|
q.internal, err = levelqueue.NewUniqueQueue(db, []byte(cfg.QueueFullName), []byte(cfg.SetFullName), false) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
|
||||||
|
return q, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueUnique) PushItem(ctx context.Context, data []byte) error { |
||||||
|
return baseLevelQueueCommon(q.cfg, q.internal, &q.mu).PushItem(ctx, data) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueUnique) PopItem(ctx context.Context) ([]byte, error) { |
||||||
|
return baseLevelQueueCommon(q.cfg, q.internal, &q.mu).PopItem(ctx) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueUnique) HasItem(ctx context.Context, data []byte) (bool, error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
return q.internal.Has(data) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueUnique) Len(ctx context.Context) (int, error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
return int(q.internal.Len()), nil |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueUnique) Close() error { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
err := q.internal.Close() |
||||||
|
_ = nosql.GetManager().CloseLevelDB(q.conn) |
||||||
|
return err |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseLevelQueueUnique) RemoveAll(ctx context.Context) error { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
type levelUniqueQueue struct { |
||||||
|
q *levelqueue.Queue |
||||||
|
set *levelqueue.Set |
||||||
|
db *leveldb.DB |
||||||
|
} |
||||||
|
lq := (*levelUniqueQueue)(unsafe.Pointer(q.internal)) |
||||||
|
|
||||||
|
members, err := lq.set.Members() |
||||||
|
if err != nil { |
||||||
|
return err // seriously corrupted
|
||||||
|
} |
||||||
|
for _, v := range members { |
||||||
|
_, _ = lq.set.Remove(v) |
||||||
|
} |
||||||
|
for lq.q.Len() > 0 { |
||||||
|
if _, err = lq.q.LPop(); err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
@ -0,0 +1,135 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"sync" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/graceful" |
||||||
|
"code.gitea.io/gitea/modules/log" |
||||||
|
"code.gitea.io/gitea/modules/nosql" |
||||||
|
|
||||||
|
"github.com/redis/go-redis/v9" |
||||||
|
) |
||||||
|
|
||||||
|
type baseRedis struct { |
||||||
|
client redis.UniversalClient |
||||||
|
isUnique bool |
||||||
|
cfg *BaseConfig |
||||||
|
|
||||||
|
mu sync.Mutex // the old implementation is not thread-safe, the queue operation and set operation should be protected together
|
||||||
|
} |
||||||
|
|
||||||
|
var _ baseQueue = (*baseRedis)(nil) |
||||||
|
|
||||||
|
func newBaseRedisGeneric(cfg *BaseConfig, unique bool) (baseQueue, error) { |
||||||
|
client := nosql.GetManager().GetRedisClient(cfg.ConnStr) |
||||||
|
|
||||||
|
var err error |
||||||
|
for i := 0; i < 10; i++ { |
||||||
|
err = client.Ping(graceful.GetManager().ShutdownContext()).Err() |
||||||
|
if err == nil { |
||||||
|
break |
||||||
|
} |
||||||
|
log.Warn("Redis is not ready, waiting for 1 second to retry: %v", err) |
||||||
|
time.Sleep(time.Second) |
||||||
|
} |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
|
||||||
|
return &baseRedis{cfg: cfg, client: client, isUnique: unique}, nil |
||||||
|
} |
||||||
|
|
||||||
|
func newBaseRedisSimple(cfg *BaseConfig) (baseQueue, error) { |
||||||
|
return newBaseRedisGeneric(cfg, false) |
||||||
|
} |
||||||
|
|
||||||
|
func newBaseRedisUnique(cfg *BaseConfig) (baseQueue, error) { |
||||||
|
return newBaseRedisGeneric(cfg, true) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseRedis) PushItem(ctx context.Context, data []byte) error { |
||||||
|
return backoffErr(ctx, backoffBegin, backoffUpper, time.After(pushBlockTime), func() (retry bool, err error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
cnt, err := q.client.LLen(ctx, q.cfg.QueueFullName).Result() |
||||||
|
if err != nil { |
||||||
|
return false, err |
||||||
|
} |
||||||
|
if int(cnt) >= q.cfg.Length { |
||||||
|
return true, nil |
||||||
|
} |
||||||
|
|
||||||
|
if q.isUnique { |
||||||
|
added, err := q.client.SAdd(ctx, q.cfg.SetFullName, data).Result() |
||||||
|
if err != nil { |
||||||
|
return false, err |
||||||
|
} |
||||||
|
if added == 0 { |
||||||
|
return false, ErrAlreadyInQueue |
||||||
|
} |
||||||
|
} |
||||||
|
return false, q.client.RPush(ctx, q.cfg.QueueFullName, data).Err() |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseRedis) PopItem(ctx context.Context) ([]byte, error) { |
||||||
|
return backoffRetErr(ctx, backoffBegin, backoffUpper, infiniteTimerC, func() (retry bool, data []byte, err error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
|
||||||
|
data, err = q.client.LPop(ctx, q.cfg.QueueFullName).Bytes() |
||||||
|
if err == redis.Nil { |
||||||
|
return true, nil, nil |
||||||
|
} |
||||||
|
if err != nil { |
||||||
|
return true, nil, nil |
||||||
|
} |
||||||
|
if q.isUnique { |
||||||
|
// the data has been popped, even if there is any error we can't do anything
|
||||||
|
_ = q.client.SRem(ctx, q.cfg.SetFullName, data).Err() |
||||||
|
} |
||||||
|
return false, data, err |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseRedis) HasItem(ctx context.Context, data []byte) (bool, error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
if !q.isUnique { |
||||||
|
return false, nil |
||||||
|
} |
||||||
|
return q.client.SIsMember(ctx, q.cfg.SetFullName, data).Result() |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseRedis) Len(ctx context.Context) (int, error) { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
cnt, err := q.client.LLen(ctx, q.cfg.QueueFullName).Result() |
||||||
|
return int(cnt), err |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseRedis) Close() error { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
return q.client.Close() |
||||||
|
} |
||||||
|
|
||||||
|
func (q *baseRedis) RemoveAll(ctx context.Context) error { |
||||||
|
q.mu.Lock() |
||||||
|
defer q.mu.Unlock() |
||||||
|
c1 := q.client.Del(ctx, q.cfg.QueueFullName) |
||||||
|
c2 := q.client.Del(ctx, q.cfg.SetFullName) |
||||||
|
if c1.Err() != nil { |
||||||
|
return c1.Err() |
||||||
|
} |
||||||
|
if c2.Err() != nil { |
||||||
|
return c2.Err() |
||||||
|
} |
||||||
|
return nil // actually, checking errors doesn't make sense here because the state could be out-of-sync
|
||||||
|
} |
@ -0,0 +1,71 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"os" |
||||||
|
"os/exec" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/nosql" |
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert" |
||||||
|
) |
||||||
|
|
||||||
|
func waitRedisReady(conn string, dur time.Duration) (ready bool) { |
||||||
|
ctxTimed, cancel := context.WithTimeout(context.Background(), time.Second*5) |
||||||
|
defer cancel() |
||||||
|
for t := time.Now(); ; time.Sleep(50 * time.Millisecond) { |
||||||
|
ret := nosql.GetManager().GetRedisClient(conn).Ping(ctxTimed) |
||||||
|
if ret.Err() == nil { |
||||||
|
return true |
||||||
|
} |
||||||
|
if time.Since(t) > dur { |
||||||
|
return false |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func redisServerCmd(t *testing.T) *exec.Cmd { |
||||||
|
redisServerProg, err := exec.LookPath("redis-server") |
||||||
|
if err != nil { |
||||||
|
return nil |
||||||
|
} |
||||||
|
c := &exec.Cmd{ |
||||||
|
Path: redisServerProg, |
||||||
|
Args: []string{redisServerProg, "--bind", "127.0.0.1", "--port", "6379"}, |
||||||
|
Dir: t.TempDir(), |
||||||
|
Stdin: os.Stdin, |
||||||
|
Stdout: os.Stdout, |
||||||
|
Stderr: os.Stderr, |
||||||
|
} |
||||||
|
return c |
||||||
|
} |
||||||
|
|
||||||
|
func TestBaseRedis(t *testing.T) { |
||||||
|
var redisServer *exec.Cmd |
||||||
|
defer func() { |
||||||
|
if redisServer != nil { |
||||||
|
_ = redisServer.Process.Signal(os.Interrupt) |
||||||
|
_ = redisServer.Wait() |
||||||
|
} |
||||||
|
}() |
||||||
|
if !waitRedisReady("redis://127.0.0.1:6379/0", 0) { |
||||||
|
redisServer = redisServerCmd(t) |
||||||
|
if redisServer == nil && os.Getenv("CI") != "" { |
||||||
|
t.Skip("redis-server not found") |
||||||
|
return |
||||||
|
} |
||||||
|
assert.NoError(t, redisServer.Start()) |
||||||
|
if !assert.True(t, waitRedisReady("redis://127.0.0.1:6379/0", 5*time.Second), "start redis-server") { |
||||||
|
return |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
testQueueBasic(t, newBaseRedisSimple, toBaseConfig("baseRedis", setting.QueueSettings{Length: 10}), false) |
||||||
|
testQueueBasic(t, newBaseRedisUnique, toBaseConfig("baseRedisUnique", setting.QueueSettings{Length: 10}), true) |
||||||
|
} |
@ -0,0 +1,140 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"fmt" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert" |
||||||
|
) |
||||||
|
|
||||||
|
func testQueueBasic(t *testing.T, newFn func(cfg *BaseConfig) (baseQueue, error), cfg *BaseConfig, isUnique bool) { |
||||||
|
t.Run(fmt.Sprintf("testQueueBasic-%s-unique:%v", cfg.ManagedName, isUnique), func(t *testing.T) { |
||||||
|
q, err := newFn(cfg) |
||||||
|
assert.NoError(t, err) |
||||||
|
|
||||||
|
ctx := context.Background() |
||||||
|
_ = q.RemoveAll(ctx) |
||||||
|
cnt, err := q.Len(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, 0, cnt) |
||||||
|
|
||||||
|
// push the first item
|
||||||
|
err = q.PushItem(ctx, []byte("foo")) |
||||||
|
assert.NoError(t, err) |
||||||
|
|
||||||
|
cnt, err = q.Len(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, 1, cnt) |
||||||
|
|
||||||
|
// push a duplicate item
|
||||||
|
err = q.PushItem(ctx, []byte("foo")) |
||||||
|
if !isUnique { |
||||||
|
assert.NoError(t, err) |
||||||
|
} else { |
||||||
|
assert.ErrorIs(t, err, ErrAlreadyInQueue) |
||||||
|
} |
||||||
|
|
||||||
|
// check the duplicate item
|
||||||
|
cnt, err = q.Len(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
has, err := q.HasItem(ctx, []byte("foo")) |
||||||
|
assert.NoError(t, err) |
||||||
|
if !isUnique { |
||||||
|
assert.EqualValues(t, 2, cnt) |
||||||
|
assert.EqualValues(t, false, has) // non-unique queues don't check for duplicates
|
||||||
|
} else { |
||||||
|
assert.EqualValues(t, 1, cnt) |
||||||
|
assert.EqualValues(t, true, has) |
||||||
|
} |
||||||
|
|
||||||
|
// push another item
|
||||||
|
err = q.PushItem(ctx, []byte("bar")) |
||||||
|
assert.NoError(t, err) |
||||||
|
|
||||||
|
// pop the first item (and the duplicate if non-unique)
|
||||||
|
it, err := q.PopItem(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, "foo", string(it)) |
||||||
|
|
||||||
|
if !isUnique { |
||||||
|
it, err = q.PopItem(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, "foo", string(it)) |
||||||
|
} |
||||||
|
|
||||||
|
// pop another item
|
||||||
|
it, err = q.PopItem(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, "bar", string(it)) |
||||||
|
|
||||||
|
// pop an empty queue (timeout, cancel)
|
||||||
|
ctxTimed, cancel := context.WithTimeout(ctx, 10*time.Millisecond) |
||||||
|
it, err = q.PopItem(ctxTimed) |
||||||
|
assert.ErrorIs(t, err, context.DeadlineExceeded) |
||||||
|
assert.Nil(t, it) |
||||||
|
cancel() |
||||||
|
|
||||||
|
ctxTimed, cancel = context.WithTimeout(ctx, 10*time.Millisecond) |
||||||
|
cancel() |
||||||
|
it, err = q.PopItem(ctxTimed) |
||||||
|
assert.ErrorIs(t, err, context.Canceled) |
||||||
|
assert.Nil(t, it) |
||||||
|
|
||||||
|
// test blocking push if queue is full
|
||||||
|
for i := 0; i < cfg.Length; i++ { |
||||||
|
err = q.PushItem(ctx, []byte(fmt.Sprintf("item-%d", i))) |
||||||
|
assert.NoError(t, err) |
||||||
|
} |
||||||
|
ctxTimed, cancel = context.WithTimeout(ctx, 10*time.Millisecond) |
||||||
|
err = q.PushItem(ctxTimed, []byte("item-full")) |
||||||
|
assert.ErrorIs(t, err, context.DeadlineExceeded) |
||||||
|
cancel() |
||||||
|
|
||||||
|
// test blocking push if queue is full (with custom pushBlockTime)
|
||||||
|
oldPushBlockTime := pushBlockTime |
||||||
|
timeStart := time.Now() |
||||||
|
pushBlockTime = 30 * time.Millisecond |
||||||
|
err = q.PushItem(ctx, []byte("item-full")) |
||||||
|
assert.ErrorIs(t, err, context.DeadlineExceeded) |
||||||
|
assert.True(t, time.Since(timeStart) >= pushBlockTime*2/3) |
||||||
|
pushBlockTime = oldPushBlockTime |
||||||
|
|
||||||
|
// remove all
|
||||||
|
cnt, err = q.Len(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, cfg.Length, cnt) |
||||||
|
|
||||||
|
_ = q.RemoveAll(ctx) |
||||||
|
|
||||||
|
cnt, err = q.Len(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, 0, cnt) |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func TestBaseDummy(t *testing.T) { |
||||||
|
q, err := newBaseDummy(&BaseConfig{}, true) |
||||||
|
assert.NoError(t, err) |
||||||
|
|
||||||
|
ctx := context.Background() |
||||||
|
assert.NoError(t, q.PushItem(ctx, []byte("foo"))) |
||||||
|
|
||||||
|
cnt, err := q.Len(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.EqualValues(t, 0, cnt) |
||||||
|
|
||||||
|
has, err := q.HasItem(ctx, []byte("foo")) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.False(t, has) |
||||||
|
|
||||||
|
it, err := q.PopItem(ctx) |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.Nil(t, it) |
||||||
|
|
||||||
|
assert.NoError(t, q.RemoveAll(ctx)) |
||||||
|
} |
@ -1,69 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import "context" |
|
||||||
|
|
||||||
// ByteFIFO defines a FIFO that takes a byte array
|
|
||||||
type ByteFIFO interface { |
|
||||||
// Len returns the length of the fifo
|
|
||||||
Len(ctx context.Context) int64 |
|
||||||
// PushFunc pushes data to the end of the fifo and calls the callback if it is added
|
|
||||||
PushFunc(ctx context.Context, data []byte, fn func() error) error |
|
||||||
// Pop pops data from the start of the fifo
|
|
||||||
Pop(ctx context.Context) ([]byte, error) |
|
||||||
// Close this fifo
|
|
||||||
Close() error |
|
||||||
// PushBack pushes data back to the top of the fifo
|
|
||||||
PushBack(ctx context.Context, data []byte) error |
|
||||||
} |
|
||||||
|
|
||||||
// UniqueByteFIFO defines a FIFO that Uniques its contents
|
|
||||||
type UniqueByteFIFO interface { |
|
||||||
ByteFIFO |
|
||||||
// Has returns whether the fifo contains this data
|
|
||||||
Has(ctx context.Context, data []byte) (bool, error) |
|
||||||
} |
|
||||||
|
|
||||||
var _ ByteFIFO = &DummyByteFIFO{} |
|
||||||
|
|
||||||
// DummyByteFIFO represents a dummy fifo
|
|
||||||
type DummyByteFIFO struct{} |
|
||||||
|
|
||||||
// PushFunc returns nil
|
|
||||||
func (*DummyByteFIFO) PushFunc(ctx context.Context, data []byte, fn func() error) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Pop returns nil
|
|
||||||
func (*DummyByteFIFO) Pop(ctx context.Context) ([]byte, error) { |
|
||||||
return []byte{}, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Close returns nil
|
|
||||||
func (*DummyByteFIFO) Close() error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Len is always 0
|
|
||||||
func (*DummyByteFIFO) Len(ctx context.Context) int64 { |
|
||||||
return 0 |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack pushes data back to the top of the fifo
|
|
||||||
func (*DummyByteFIFO) PushBack(ctx context.Context, data []byte) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
var _ UniqueByteFIFO = &DummyUniqueByteFIFO{} |
|
||||||
|
|
||||||
// DummyUniqueByteFIFO represents a dummy unique fifo
|
|
||||||
type DummyUniqueByteFIFO struct { |
|
||||||
DummyByteFIFO |
|
||||||
} |
|
||||||
|
|
||||||
// Has always returns false
|
|
||||||
func (*DummyUniqueByteFIFO) Has(ctx context.Context, data []byte) (bool, error) { |
|
||||||
return false, nil |
|
||||||
} |
|
@ -0,0 +1,36 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
) |
||||||
|
|
||||||
|
type BaseConfig struct { |
||||||
|
ManagedName string |
||||||
|
DataFullDir string // the caller must prepare an absolute path
|
||||||
|
|
||||||
|
ConnStr string |
||||||
|
Length int |
||||||
|
|
||||||
|
QueueFullName, SetFullName string |
||||||
|
} |
||||||
|
|
||||||
|
func toBaseConfig(managedName string, queueSetting setting.QueueSettings) *BaseConfig { |
||||||
|
baseConfig := &BaseConfig{ |
||||||
|
ManagedName: managedName, |
||||||
|
DataFullDir: queueSetting.Datadir, |
||||||
|
|
||||||
|
ConnStr: queueSetting.ConnStr, |
||||||
|
Length: queueSetting.Length, |
||||||
|
} |
||||||
|
|
||||||
|
// queue name and set name
|
||||||
|
baseConfig.QueueFullName = managedName + queueSetting.QueueName |
||||||
|
baseConfig.SetFullName = baseConfig.QueueFullName + queueSetting.SetName |
||||||
|
if baseConfig.SetFullName == baseConfig.QueueFullName { |
||||||
|
baseConfig.SetFullName += "_unique" |
||||||
|
} |
||||||
|
return baseConfig |
||||||
|
} |
@ -1,91 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"reflect" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/json" |
|
||||||
) |
|
||||||
|
|
||||||
// Mappable represents an interface that can MapTo another interface
|
|
||||||
type Mappable interface { |
|
||||||
MapTo(v interface{}) error |
|
||||||
} |
|
||||||
|
|
||||||
// toConfig will attempt to convert a given configuration cfg into the provided exemplar type.
|
|
||||||
//
|
|
||||||
// It will tolerate the cfg being passed as a []byte or string of a json representation of the
|
|
||||||
// exemplar or the correct type of the exemplar itself
|
|
||||||
func toConfig(exemplar, cfg interface{}) (interface{}, error) { |
|
||||||
// First of all check if we've got the same type as the exemplar - if so it's all fine.
|
|
||||||
if reflect.TypeOf(cfg).AssignableTo(reflect.TypeOf(exemplar)) { |
|
||||||
return cfg, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Now if not - does it provide a MapTo function we can try?
|
|
||||||
if mappable, ok := cfg.(Mappable); ok { |
|
||||||
newVal := reflect.New(reflect.TypeOf(exemplar)) |
|
||||||
if err := mappable.MapTo(newVal.Interface()); err == nil { |
|
||||||
return newVal.Elem().Interface(), nil |
|
||||||
} |
|
||||||
// MapTo has failed us ... let's try the json route ...
|
|
||||||
} |
|
||||||
|
|
||||||
// OK we've been passed a byte array right?
|
|
||||||
configBytes, ok := cfg.([]byte) |
|
||||||
if !ok { |
|
||||||
// oh ... it's a string then?
|
|
||||||
var configStr string |
|
||||||
|
|
||||||
configStr, ok = cfg.(string) |
|
||||||
configBytes = []byte(configStr) |
|
||||||
} |
|
||||||
if !ok { |
|
||||||
// hmm ... can we marshal it to json?
|
|
||||||
var err error |
|
||||||
configBytes, err = json.Marshal(cfg) |
|
||||||
ok = err == nil |
|
||||||
} |
|
||||||
if !ok { |
|
||||||
// no ... we've tried hard enough at this point - throw an error!
|
|
||||||
return nil, ErrInvalidConfiguration{cfg: cfg} |
|
||||||
} |
|
||||||
|
|
||||||
// OK unmarshal the byte array into a new copy of the exemplar
|
|
||||||
newVal := reflect.New(reflect.TypeOf(exemplar)) |
|
||||||
if err := json.Unmarshal(configBytes, newVal.Interface()); err != nil { |
|
||||||
// If we can't unmarshal it then return an error!
|
|
||||||
return nil, ErrInvalidConfiguration{cfg: cfg, err: err} |
|
||||||
} |
|
||||||
return newVal.Elem().Interface(), nil |
|
||||||
} |
|
||||||
|
|
||||||
// unmarshalAs will attempt to unmarshal provided bytes as the provided exemplar
|
|
||||||
func unmarshalAs(bs []byte, exemplar interface{}) (data Data, err error) { |
|
||||||
if exemplar != nil { |
|
||||||
t := reflect.TypeOf(exemplar) |
|
||||||
n := reflect.New(t) |
|
||||||
ne := n.Elem() |
|
||||||
err = json.Unmarshal(bs, ne.Addr().Interface()) |
|
||||||
data = ne.Interface().(Data) |
|
||||||
} else { |
|
||||||
err = json.Unmarshal(bs, &data) |
|
||||||
} |
|
||||||
return data, err |
|
||||||
} |
|
||||||
|
|
||||||
// assignableTo will check if provided data is assignable to the same type as the exemplar
|
|
||||||
// if the provided exemplar is nil then it will always return true
|
|
||||||
func assignableTo(data Data, exemplar interface{}) bool { |
|
||||||
if exemplar == nil { |
|
||||||
return true |
|
||||||
} |
|
||||||
|
|
||||||
// Assert data is of same type as exemplar
|
|
||||||
t := reflect.TypeOf(data) |
|
||||||
exemplarType := reflect.TypeOf(exemplar) |
|
||||||
|
|
||||||
return t.AssignableTo(exemplarType) && data != nil |
|
||||||
} |
|
@ -0,0 +1,124 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"path/filepath" |
||||||
|
"testing" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert" |
||||||
|
) |
||||||
|
|
||||||
|
func TestManager(t *testing.T) { |
||||||
|
oldAppDataPath := setting.AppDataPath |
||||||
|
setting.AppDataPath = t.TempDir() |
||||||
|
defer func() { |
||||||
|
setting.AppDataPath = oldAppDataPath |
||||||
|
}() |
||||||
|
|
||||||
|
newQueueFromConfig := func(name, cfg string) (*WorkerPoolQueue[int], error) { |
||||||
|
cfgProvider, err := setting.NewConfigProviderFromData(cfg) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
qs, err := setting.GetQueueSettings(cfgProvider, name) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
return NewWorkerPoolQueueBySetting(name, qs, func(s ...int) (unhandled []int) { return nil }, false) |
||||||
|
} |
||||||
|
|
||||||
|
// test invalid CONN_STR
|
||||||
|
_, err := newQueueFromConfig("default", ` |
||||||
|
[queue] |
||||||
|
DATADIR = temp-dir |
||||||
|
CONN_STR = redis://
|
||||||
|
`) |
||||||
|
assert.ErrorContains(t, err, "invalid leveldb connection string") |
||||||
|
|
||||||
|
// test default config
|
||||||
|
q, err := newQueueFromConfig("default", "") |
||||||
|
assert.NoError(t, err) |
||||||
|
assert.Equal(t, "default", q.GetName()) |
||||||
|
assert.Equal(t, "level", q.GetType()) |
||||||
|
assert.Equal(t, filepath.Join(setting.AppDataPath, "queues/common"), q.baseConfig.DataFullDir) |
||||||
|
assert.Equal(t, 100, q.baseConfig.Length) |
||||||
|
assert.Equal(t, 20, q.batchLength) |
||||||
|
assert.Equal(t, "", q.baseConfig.ConnStr) |
||||||
|
assert.Equal(t, "default_queue", q.baseConfig.QueueFullName) |
||||||
|
assert.Equal(t, "default_queue_unique", q.baseConfig.SetFullName) |
||||||
|
assert.Equal(t, 10, q.GetWorkerMaxNumber()) |
||||||
|
assert.Equal(t, 0, q.GetWorkerNumber()) |
||||||
|
assert.Equal(t, 0, q.GetWorkerActiveNumber()) |
||||||
|
assert.Equal(t, 0, q.GetQueueItemNumber()) |
||||||
|
assert.Equal(t, "int", q.GetItemTypeName()) |
||||||
|
|
||||||
|
// test inherited config
|
||||||
|
cfgProvider, err := setting.NewConfigProviderFromData(` |
||||||
|
[queue] |
||||||
|
TYPE = channel |
||||||
|
DATADIR = queues/dir1 |
||||||
|
LENGTH = 100 |
||||||
|
BATCH_LENGTH = 20 |
||||||
|
CONN_STR = "addrs=127.0.0.1:6379 db=0" |
||||||
|
QUEUE_NAME = _queue1 |
||||||
|
|
||||||
|
[queue.sub] |
||||||
|
TYPE = level |
||||||
|
DATADIR = queues/dir2 |
||||||
|
LENGTH = 102 |
||||||
|
BATCH_LENGTH = 22 |
||||||
|
CONN_STR = |
||||||
|
QUEUE_NAME = _q2 |
||||||
|
SET_NAME = _u2 |
||||||
|
MAX_WORKERS = 2 |
||||||
|
`) |
||||||
|
|
||||||
|
assert.NoError(t, err) |
||||||
|
|
||||||
|
q1 := createWorkerPoolQueue[string]("no-such", cfgProvider, nil, false) |
||||||
|
assert.Equal(t, "no-such", q1.GetName()) |
||||||
|
assert.Equal(t, "dummy", q1.GetType()) // no handler, so it becomes dummy
|
||||||
|
assert.Equal(t, filepath.Join(setting.AppDataPath, "queues/dir1"), q1.baseConfig.DataFullDir) |
||||||
|
assert.Equal(t, 100, q1.baseConfig.Length) |
||||||
|
assert.Equal(t, 20, q1.batchLength) |
||||||
|
assert.Equal(t, "addrs=127.0.0.1:6379 db=0", q1.baseConfig.ConnStr) |
||||||
|
assert.Equal(t, "no-such_queue1", q1.baseConfig.QueueFullName) |
||||||
|
assert.Equal(t, "no-such_queue1_unique", q1.baseConfig.SetFullName) |
||||||
|
assert.Equal(t, 10, q1.GetWorkerMaxNumber()) |
||||||
|
assert.Equal(t, 0, q1.GetWorkerNumber()) |
||||||
|
assert.Equal(t, 0, q1.GetWorkerActiveNumber()) |
||||||
|
assert.Equal(t, 0, q1.GetQueueItemNumber()) |
||||||
|
assert.Equal(t, "string", q1.GetItemTypeName()) |
||||||
|
qid1 := GetManager().qidCounter |
||||||
|
|
||||||
|
q2 := createWorkerPoolQueue("sub", cfgProvider, func(s ...int) (unhandled []int) { return nil }, false) |
||||||
|
assert.Equal(t, "sub", q2.GetName()) |
||||||
|
assert.Equal(t, "level", q2.GetType()) |
||||||
|
assert.Equal(t, filepath.Join(setting.AppDataPath, "queues/dir2"), q2.baseConfig.DataFullDir) |
||||||
|
assert.Equal(t, 102, q2.baseConfig.Length) |
||||||
|
assert.Equal(t, 22, q2.batchLength) |
||||||
|
assert.Equal(t, "", q2.baseConfig.ConnStr) |
||||||
|
assert.Equal(t, "sub_q2", q2.baseConfig.QueueFullName) |
||||||
|
assert.Equal(t, "sub_q2_u2", q2.baseConfig.SetFullName) |
||||||
|
assert.Equal(t, 2, q2.GetWorkerMaxNumber()) |
||||||
|
assert.Equal(t, 0, q2.GetWorkerNumber()) |
||||||
|
assert.Equal(t, 0, q2.GetWorkerActiveNumber()) |
||||||
|
assert.Equal(t, 0, q2.GetQueueItemNumber()) |
||||||
|
assert.Equal(t, "int", q2.GetItemTypeName()) |
||||||
|
qid2 := GetManager().qidCounter |
||||||
|
|
||||||
|
assert.Equal(t, q1, GetManager().ManagedQueues()[qid1]) |
||||||
|
|
||||||
|
GetManager().GetManagedQueue(qid1).SetWorkerMaxNumber(120) |
||||||
|
assert.Equal(t, 120, q1.workerMaxNum) |
||||||
|
|
||||||
|
stop := runWorkerPoolQueue(q2) |
||||||
|
assert.NoError(t, GetManager().GetManagedQueue(qid2).FlushWithContext(context.Background(), 0)) |
||||||
|
assert.NoError(t, GetManager().FlushAll(context.Background(), 0)) |
||||||
|
stop() |
||||||
|
} |
@ -1,201 +1,31 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
// SPDX-License-Identifier: MIT
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
package queue |
// Package queue implements a specialized queue system for Gitea.
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"time" |
|
||||||
) |
|
||||||
|
|
||||||
// ErrInvalidConfiguration is called when there is invalid configuration for a queue
|
|
||||||
type ErrInvalidConfiguration struct { |
|
||||||
cfg interface{} |
|
||||||
err error |
|
||||||
} |
|
||||||
|
|
||||||
func (err ErrInvalidConfiguration) Error() string { |
|
||||||
if err.err != nil { |
|
||||||
return fmt.Sprintf("Invalid Configuration Argument: %v: Error: %v", err.cfg, err.err) |
|
||||||
} |
|
||||||
return fmt.Sprintf("Invalid Configuration Argument: %v", err.cfg) |
|
||||||
} |
|
||||||
|
|
||||||
// IsErrInvalidConfiguration checks if an error is an ErrInvalidConfiguration
|
|
||||||
func IsErrInvalidConfiguration(err error) bool { |
|
||||||
_, ok := err.(ErrInvalidConfiguration) |
|
||||||
return ok |
|
||||||
} |
|
||||||
|
|
||||||
// Type is a type of Queue
|
|
||||||
type Type string |
|
||||||
|
|
||||||
// Data defines an type of queuable data
|
|
||||||
type Data interface{} |
|
||||||
|
|
||||||
// HandlerFunc is a function that takes a variable amount of data and processes it
|
|
||||||
type HandlerFunc func(...Data) (unhandled []Data) |
|
||||||
|
|
||||||
// NewQueueFunc is a function that creates a queue
|
|
||||||
type NewQueueFunc func(handler HandlerFunc, config, exemplar interface{}) (Queue, error) |
|
||||||
|
|
||||||
// Shutdownable represents a queue that can be shutdown
|
|
||||||
type Shutdownable interface { |
|
||||||
Shutdown() |
|
||||||
Terminate() |
|
||||||
} |
|
||||||
|
|
||||||
// Named represents a queue with a name
|
|
||||||
type Named interface { |
|
||||||
Name() string |
|
||||||
} |
|
||||||
|
|
||||||
// Queue defines an interface of a queue-like item
|
|
||||||
//
|
//
|
||||||
// Queues will handle their own contents in the Run method
|
// There are two major kinds of concepts:
|
||||||
type Queue interface { |
//
|
||||||
Flushable |
// * The "base queue": channel, level, redis:
|
||||||
Run(atShutdown, atTerminate func(func())) |
// - They have the same abstraction, the same interface, and they are tested by the same testing code.
|
||||||
Push(Data) error |
// - The dummy(immediate) queue is special, it's not a real queue, it's only used as a no-op queue or a testing queue.
|
||||||
} |
//
|
||||||
|
// * The WorkerPoolQueue: it uses the "base queue" to provide "worker pool" function.
|
||||||
// PushBackable queues can be pushed back to
|
// - It calls the "handler" to process the data in the base queue.
|
||||||
type PushBackable interface { |
// - Its "Push" function doesn't block forever,
|
||||||
// PushBack pushes data back to the top of the fifo
|
// it will return an error if the queue is full after the timeout.
|
||||||
PushBack(Data) error |
//
|
||||||
} |
// A queue can be "simple" or "unique". A unique queue will try to avoid duplicate items.
|
||||||
|
// Unique queue's "Has" function can be used to check whether an item is already in the queue,
|
||||||
// DummyQueueType is the type for the dummy queue
|
// although it's not 100% reliable due to there is no proper transaction support.
|
||||||
const DummyQueueType Type = "dummy" |
// Simple queue's "Has" function always returns "has=false".
|
||||||
|
//
|
||||||
// NewDummyQueue creates a new DummyQueue
|
// The HandlerFuncT function is called by the WorkerPoolQueue to process the data in the base queue.
|
||||||
func NewDummyQueue(handler HandlerFunc, opts, exemplar interface{}) (Queue, error) { |
// If the handler returns "unhandled" items, they will be re-queued to the base queue after a slight delay,
|
||||||
return &DummyQueue{}, nil |
// in case the item processor (eg: document indexer) is not available.
|
||||||
} |
package queue |
||||||
|
|
||||||
// DummyQueue represents an empty queue
|
|
||||||
type DummyQueue struct{} |
|
||||||
|
|
||||||
// Run does nothing
|
|
||||||
func (*DummyQueue) Run(_, _ func(func())) {} |
|
||||||
|
|
||||||
// Push fakes a push of data to the queue
|
|
||||||
func (*DummyQueue) Push(Data) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc fakes a push of data to the queue with a function. The function is never run.
|
|
||||||
func (*DummyQueue) PushFunc(Data, func() error) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Has always returns false as this queue never does anything
|
|
||||||
func (*DummyQueue) Has(Data) (bool, error) { |
|
||||||
return false, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Flush always returns nil
|
|
||||||
func (*DummyQueue) Flush(time.Duration) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// FlushWithContext always returns nil
|
|
||||||
func (*DummyQueue) FlushWithContext(context.Context) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty asserts that the queue is empty
|
|
||||||
func (*DummyQueue) IsEmpty() bool { |
|
||||||
return true |
|
||||||
} |
|
||||||
|
|
||||||
// ImmediateType is the type to execute the function when push
|
|
||||||
const ImmediateType Type = "immediate" |
|
||||||
|
|
||||||
// NewImmediate creates a new false queue to execute the function when push
|
|
||||||
func NewImmediate(handler HandlerFunc, opts, exemplar interface{}) (Queue, error) { |
|
||||||
return &Immediate{ |
|
||||||
handler: handler, |
|
||||||
}, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Immediate represents an direct execution queue
|
|
||||||
type Immediate struct { |
|
||||||
handler HandlerFunc |
|
||||||
} |
|
||||||
|
|
||||||
// Run does nothing
|
|
||||||
func (*Immediate) Run(_, _ func(func())) {} |
|
||||||
|
|
||||||
// Push fakes a push of data to the queue
|
|
||||||
func (q *Immediate) Push(data Data) error { |
|
||||||
return q.PushFunc(data, nil) |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc fakes a push of data to the queue with a function. The function is never run.
|
|
||||||
func (q *Immediate) PushFunc(data Data, f func() error) error { |
|
||||||
if f != nil { |
|
||||||
if err := f(); err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
} |
|
||||||
q.handler(data) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Has always returns false as this queue never does anything
|
|
||||||
func (*Immediate) Has(Data) (bool, error) { |
|
||||||
return false, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Flush always returns nil
|
|
||||||
func (*Immediate) Flush(time.Duration) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// FlushWithContext always returns nil
|
|
||||||
func (*Immediate) FlushWithContext(context.Context) error { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty asserts that the queue is empty
|
|
||||||
func (*Immediate) IsEmpty() bool { |
|
||||||
return true |
|
||||||
} |
|
||||||
|
|
||||||
var queuesMap = map[Type]NewQueueFunc{ |
|
||||||
DummyQueueType: NewDummyQueue, |
|
||||||
ImmediateType: NewImmediate, |
|
||||||
} |
|
||||||
|
|
||||||
// RegisteredTypes provides the list of requested types of queues
|
import "code.gitea.io/gitea/modules/util" |
||||||
func RegisteredTypes() []Type { |
|
||||||
types := make([]Type, len(queuesMap)) |
|
||||||
i := 0 |
|
||||||
for key := range queuesMap { |
|
||||||
types[i] = key |
|
||||||
i++ |
|
||||||
} |
|
||||||
return types |
|
||||||
} |
|
||||||
|
|
||||||
// RegisteredTypesAsString provides the list of requested types of queues
|
type HandlerFuncT[T any] func(...T) (unhandled []T) |
||||||
func RegisteredTypesAsString() []string { |
|
||||||
types := make([]string, len(queuesMap)) |
|
||||||
i := 0 |
|
||||||
for key := range queuesMap { |
|
||||||
types[i] = string(key) |
|
||||||
i++ |
|
||||||
} |
|
||||||
return types |
|
||||||
} |
|
||||||
|
|
||||||
// NewQueue takes a queue Type, HandlerFunc, some options and possibly an exemplar and returns a Queue or an error
|
var ErrAlreadyInQueue = util.NewAlreadyExistErrorf("already in queue") |
||||||
func NewQueue(queueType Type, handlerFunc HandlerFunc, opts, exemplar interface{}) (Queue, error) { |
|
||||||
newFn, ok := queuesMap[queueType] |
|
||||||
if !ok { |
|
||||||
return nil, fmt.Errorf("unsupported queue type: %v", queueType) |
|
||||||
} |
|
||||||
return newFn(handlerFunc, opts, exemplar) |
|
||||||
} |
|
||||||
|
@ -1,419 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"runtime/pprof" |
|
||||||
"sync" |
|
||||||
"sync/atomic" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/json" |
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
"code.gitea.io/gitea/modules/util" |
|
||||||
) |
|
||||||
|
|
||||||
// ByteFIFOQueueConfiguration is the configuration for a ByteFIFOQueue
|
|
||||||
type ByteFIFOQueueConfiguration struct { |
|
||||||
WorkerPoolConfiguration |
|
||||||
Workers int |
|
||||||
WaitOnEmpty bool |
|
||||||
} |
|
||||||
|
|
||||||
var _ Queue = &ByteFIFOQueue{} |
|
||||||
|
|
||||||
// ByteFIFOQueue is a Queue formed from a ByteFIFO and WorkerPool
|
|
||||||
type ByteFIFOQueue struct { |
|
||||||
*WorkerPool |
|
||||||
byteFIFO ByteFIFO |
|
||||||
typ Type |
|
||||||
shutdownCtx context.Context |
|
||||||
shutdownCtxCancel context.CancelFunc |
|
||||||
terminateCtx context.Context |
|
||||||
terminateCtxCancel context.CancelFunc |
|
||||||
exemplar interface{} |
|
||||||
workers int |
|
||||||
name string |
|
||||||
lock sync.Mutex |
|
||||||
waitOnEmpty bool |
|
||||||
pushed chan struct{} |
|
||||||
} |
|
||||||
|
|
||||||
// NewByteFIFOQueue creates a new ByteFIFOQueue
|
|
||||||
func NewByteFIFOQueue(typ Type, byteFIFO ByteFIFO, handle HandlerFunc, cfg, exemplar interface{}) (*ByteFIFOQueue, error) { |
|
||||||
configInterface, err := toConfig(ByteFIFOQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(ByteFIFOQueueConfiguration) |
|
||||||
|
|
||||||
terminateCtx, terminateCtxCancel := context.WithCancel(context.Background()) |
|
||||||
shutdownCtx, shutdownCtxCancel := context.WithCancel(terminateCtx) |
|
||||||
|
|
||||||
q := &ByteFIFOQueue{ |
|
||||||
byteFIFO: byteFIFO, |
|
||||||
typ: typ, |
|
||||||
shutdownCtx: shutdownCtx, |
|
||||||
shutdownCtxCancel: shutdownCtxCancel, |
|
||||||
terminateCtx: terminateCtx, |
|
||||||
terminateCtxCancel: terminateCtxCancel, |
|
||||||
exemplar: exemplar, |
|
||||||
workers: config.Workers, |
|
||||||
name: config.Name, |
|
||||||
waitOnEmpty: config.WaitOnEmpty, |
|
||||||
pushed: make(chan struct{}, 1), |
|
||||||
} |
|
||||||
q.WorkerPool = NewWorkerPool(func(data ...Data) (failed []Data) { |
|
||||||
for _, unhandled := range handle(data...) { |
|
||||||
if fail := q.PushBack(unhandled); fail != nil { |
|
||||||
failed = append(failed, fail) |
|
||||||
} |
|
||||||
} |
|
||||||
return failed |
|
||||||
}, config.WorkerPoolConfiguration) |
|
||||||
|
|
||||||
return q, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Name returns the name of this queue
|
|
||||||
func (q *ByteFIFOQueue) Name() string { |
|
||||||
return q.name |
|
||||||
} |
|
||||||
|
|
||||||
// Push pushes data to the fifo
|
|
||||||
func (q *ByteFIFOQueue) Push(data Data) error { |
|
||||||
return q.PushFunc(data, nil) |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack pushes data to the fifo
|
|
||||||
func (q *ByteFIFOQueue) PushBack(data Data) error { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
bs, err := json.Marshal(data) |
|
||||||
if err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
defer func() { |
|
||||||
select { |
|
||||||
case q.pushed <- struct{}{}: |
|
||||||
default: |
|
||||||
} |
|
||||||
}() |
|
||||||
return q.byteFIFO.PushBack(q.terminateCtx, bs) |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc pushes data to the fifo
|
|
||||||
func (q *ByteFIFOQueue) PushFunc(data Data, fn func() error) error { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
bs, err := json.Marshal(data) |
|
||||||
if err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
defer func() { |
|
||||||
select { |
|
||||||
case q.pushed <- struct{}{}: |
|
||||||
default: |
|
||||||
} |
|
||||||
}() |
|
||||||
return q.byteFIFO.PushFunc(q.terminateCtx, bs, fn) |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty checks if the queue is empty
|
|
||||||
func (q *ByteFIFOQueue) IsEmpty() bool { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if !q.WorkerPool.IsEmpty() { |
|
||||||
return false |
|
||||||
} |
|
||||||
return q.byteFIFO.Len(q.terminateCtx) == 0 |
|
||||||
} |
|
||||||
|
|
||||||
// NumberInQueue returns the number in the queue
|
|
||||||
func (q *ByteFIFOQueue) NumberInQueue() int64 { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
return q.byteFIFO.Len(q.terminateCtx) + q.WorkerPool.NumberInQueue() |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the ByteFIFOQueue
|
|
||||||
func (q *ByteFIFOQueue) Flush(timeout time.Duration) error { |
|
||||||
select { |
|
||||||
case q.pushed <- struct{}{}: |
|
||||||
default: |
|
||||||
} |
|
||||||
return q.WorkerPool.Flush(timeout) |
|
||||||
} |
|
||||||
|
|
||||||
// Run runs the bytefifo queue
|
|
||||||
func (q *ByteFIFOQueue) Run(atShutdown, atTerminate func(func())) { |
|
||||||
pprof.SetGoroutineLabels(q.baseCtx) |
|
||||||
atShutdown(q.Shutdown) |
|
||||||
atTerminate(q.Terminate) |
|
||||||
log.Debug("%s: %s Starting", q.typ, q.name) |
|
||||||
|
|
||||||
_ = q.AddWorkers(q.workers, 0) |
|
||||||
|
|
||||||
log.Trace("%s: %s Now running", q.typ, q.name) |
|
||||||
q.readToChan() |
|
||||||
|
|
||||||
<-q.shutdownCtx.Done() |
|
||||||
log.Trace("%s: %s Waiting til done", q.typ, q.name) |
|
||||||
q.Wait() |
|
||||||
|
|
||||||
log.Trace("%s: %s Waiting til cleaned", q.typ, q.name) |
|
||||||
q.CleanUp(q.terminateCtx) |
|
||||||
q.terminateCtxCancel() |
|
||||||
} |
|
||||||
|
|
||||||
const maxBackOffTime = time.Second * 3 |
|
||||||
|
|
||||||
func (q *ByteFIFOQueue) readToChan() { |
|
||||||
// handle quick cancels
|
|
||||||
select { |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
// tell the pool to shutdown.
|
|
||||||
q.baseCtxCancel() |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
// Default backoff values
|
|
||||||
backOffTime := time.Millisecond * 100 |
|
||||||
backOffTimer := time.NewTimer(0) |
|
||||||
util.StopTimer(backOffTimer) |
|
||||||
|
|
||||||
paused, _ := q.IsPausedIsResumed() |
|
||||||
|
|
||||||
loop: |
|
||||||
for { |
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
log.Trace("Queue %s pausing", q.name) |
|
||||||
_, resumed := q.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
paused, _ = q.IsPausedIsResumed() |
|
||||||
log.Trace("Queue %s resuming", q.name) |
|
||||||
if q.HasNoWorkerScaling() { |
|
||||||
log.Warn( |
|
||||||
"Queue: %s is configured to be non-scaling and has no workers - this configuration is likely incorrect.\n"+ |
|
||||||
"The queue will be paused to prevent data-loss with the assumption that you will add workers and unpause as required.", q.name) |
|
||||||
q.Pause() |
|
||||||
continue loop |
|
||||||
} |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
// tell the pool to shutdown.
|
|
||||||
q.baseCtxCancel() |
|
||||||
return |
|
||||||
case data, ok := <-q.dataChan: |
|
||||||
if !ok { |
|
||||||
return |
|
||||||
} |
|
||||||
if err := q.PushBack(data); err != nil { |
|
||||||
log.Error("Unable to push back data into queue %s", q.name) |
|
||||||
} |
|
||||||
atomic.AddInt64(&q.numInQueue, -1) |
|
||||||
} |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
// empty the pushed channel
|
|
||||||
select { |
|
||||||
case <-q.pushed: |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
err := q.doPop() |
|
||||||
|
|
||||||
util.StopTimer(backOffTimer) |
|
||||||
|
|
||||||
if err != nil { |
|
||||||
if err == errQueueEmpty && q.waitOnEmpty { |
|
||||||
log.Trace("%s: %s Waiting on Empty", q.typ, q.name) |
|
||||||
|
|
||||||
// reset the backoff time but don't set the timer
|
|
||||||
backOffTime = 100 * time.Millisecond |
|
||||||
} else if err == errUnmarshal { |
|
||||||
// reset the timer and backoff
|
|
||||||
backOffTime = 100 * time.Millisecond |
|
||||||
backOffTimer.Reset(backOffTime) |
|
||||||
} else { |
|
||||||
// backoff
|
|
||||||
backOffTimer.Reset(backOffTime) |
|
||||||
} |
|
||||||
|
|
||||||
// Need to Backoff
|
|
||||||
select { |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
// Oops we've been shutdown whilst backing off
|
|
||||||
// Make sure the worker pool is shutdown too
|
|
||||||
q.baseCtxCancel() |
|
||||||
return |
|
||||||
case <-q.pushed: |
|
||||||
// Data has been pushed to the fifo (or flush has been called)
|
|
||||||
// reset the backoff time
|
|
||||||
backOffTime = 100 * time.Millisecond |
|
||||||
continue loop |
|
||||||
case <-backOffTimer.C: |
|
||||||
// Calculate the next backoff time
|
|
||||||
backOffTime += backOffTime / 2 |
|
||||||
if backOffTime > maxBackOffTime { |
|
||||||
backOffTime = maxBackOffTime |
|
||||||
} |
|
||||||
continue loop |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Reset the backoff time
|
|
||||||
backOffTime = 100 * time.Millisecond |
|
||||||
|
|
||||||
select { |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
// Oops we've been shutdown
|
|
||||||
// Make sure the worker pool is shutdown too
|
|
||||||
q.baseCtxCancel() |
|
||||||
return |
|
||||||
default: |
|
||||||
continue loop |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
var ( |
|
||||||
errQueueEmpty = fmt.Errorf("empty queue") |
|
||||||
errEmptyBytes = fmt.Errorf("empty bytes") |
|
||||||
errUnmarshal = fmt.Errorf("failed to unmarshal") |
|
||||||
) |
|
||||||
|
|
||||||
func (q *ByteFIFOQueue) doPop() error { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
bs, err := q.byteFIFO.Pop(q.shutdownCtx) |
|
||||||
if err != nil { |
|
||||||
if err == context.Canceled { |
|
||||||
q.baseCtxCancel() |
|
||||||
return err |
|
||||||
} |
|
||||||
log.Error("%s: %s Error on Pop: %v", q.typ, q.name, err) |
|
||||||
return err |
|
||||||
} |
|
||||||
if len(bs) == 0 { |
|
||||||
if q.waitOnEmpty && q.byteFIFO.Len(q.shutdownCtx) == 0 { |
|
||||||
return errQueueEmpty |
|
||||||
} |
|
||||||
return errEmptyBytes |
|
||||||
} |
|
||||||
|
|
||||||
data, err := unmarshalAs(bs, q.exemplar) |
|
||||||
if err != nil { |
|
||||||
log.Error("%s: %s Failed to unmarshal with error: %v", q.typ, q.name, err) |
|
||||||
return errUnmarshal |
|
||||||
} |
|
||||||
|
|
||||||
log.Trace("%s %s: Task found: %#v", q.typ, q.name, data) |
|
||||||
q.WorkerPool.Push(data) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Shutdown processing from this queue
|
|
||||||
func (q *ByteFIFOQueue) Shutdown() { |
|
||||||
log.Trace("%s: %s Shutting down", q.typ, q.name) |
|
||||||
select { |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
q.shutdownCtxCancel() |
|
||||||
log.Debug("%s: %s Shutdown", q.typ, q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// IsShutdown returns a channel which is closed when this Queue is shutdown
|
|
||||||
func (q *ByteFIFOQueue) IsShutdown() <-chan struct{} { |
|
||||||
return q.shutdownCtx.Done() |
|
||||||
} |
|
||||||
|
|
||||||
// Terminate this queue and close the queue
|
|
||||||
func (q *ByteFIFOQueue) Terminate() { |
|
||||||
log.Trace("%s: %s Terminating", q.typ, q.name) |
|
||||||
q.Shutdown() |
|
||||||
select { |
|
||||||
case <-q.terminateCtx.Done(): |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
if log.IsDebug() { |
|
||||||
log.Debug("%s: %s Closing with %d tasks left in queue", q.typ, q.name, q.byteFIFO.Len(q.terminateCtx)) |
|
||||||
} |
|
||||||
q.terminateCtxCancel() |
|
||||||
if err := q.byteFIFO.Close(); err != nil { |
|
||||||
log.Error("Error whilst closing internal byte fifo in %s: %s: %v", q.typ, q.name, err) |
|
||||||
} |
|
||||||
q.baseCtxFinished() |
|
||||||
log.Debug("%s: %s Terminated", q.typ, q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// IsTerminated returns a channel which is closed when this Queue is terminated
|
|
||||||
func (q *ByteFIFOQueue) IsTerminated() <-chan struct{} { |
|
||||||
return q.terminateCtx.Done() |
|
||||||
} |
|
||||||
|
|
||||||
var _ UniqueQueue = &ByteFIFOUniqueQueue{} |
|
||||||
|
|
||||||
// ByteFIFOUniqueQueue represents a UniqueQueue formed from a UniqueByteFifo
|
|
||||||
type ByteFIFOUniqueQueue struct { |
|
||||||
ByteFIFOQueue |
|
||||||
} |
|
||||||
|
|
||||||
// NewByteFIFOUniqueQueue creates a new ByteFIFOUniqueQueue
|
|
||||||
func NewByteFIFOUniqueQueue(typ Type, byteFIFO UniqueByteFIFO, handle HandlerFunc, cfg, exemplar interface{}) (*ByteFIFOUniqueQueue, error) { |
|
||||||
configInterface, err := toConfig(ByteFIFOQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(ByteFIFOQueueConfiguration) |
|
||||||
terminateCtx, terminateCtxCancel := context.WithCancel(context.Background()) |
|
||||||
shutdownCtx, shutdownCtxCancel := context.WithCancel(terminateCtx) |
|
||||||
|
|
||||||
q := &ByteFIFOUniqueQueue{ |
|
||||||
ByteFIFOQueue: ByteFIFOQueue{ |
|
||||||
byteFIFO: byteFIFO, |
|
||||||
typ: typ, |
|
||||||
shutdownCtx: shutdownCtx, |
|
||||||
shutdownCtxCancel: shutdownCtxCancel, |
|
||||||
terminateCtx: terminateCtx, |
|
||||||
terminateCtxCancel: terminateCtxCancel, |
|
||||||
exemplar: exemplar, |
|
||||||
workers: config.Workers, |
|
||||||
name: config.Name, |
|
||||||
}, |
|
||||||
} |
|
||||||
q.WorkerPool = NewWorkerPool(func(data ...Data) (failed []Data) { |
|
||||||
for _, unhandled := range handle(data...) { |
|
||||||
if fail := q.PushBack(unhandled); fail != nil { |
|
||||||
failed = append(failed, fail) |
|
||||||
} |
|
||||||
} |
|
||||||
return failed |
|
||||||
}, config.WorkerPoolConfiguration) |
|
||||||
|
|
||||||
return q, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Has checks if the provided data is in the queue
|
|
||||||
func (q *ByteFIFOUniqueQueue) Has(data Data) (bool, error) { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return false, fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
bs, err := json.Marshal(data) |
|
||||||
if err != nil { |
|
||||||
return false, err |
|
||||||
} |
|
||||||
return q.byteFIFO.(UniqueByteFIFO).Has(q.terminateCtx, bs) |
|
||||||
} |
|
@ -1,160 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"runtime/pprof" |
|
||||||
"sync/atomic" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
) |
|
||||||
|
|
||||||
// ChannelQueueType is the type for channel queue
|
|
||||||
const ChannelQueueType Type = "channel" |
|
||||||
|
|
||||||
// ChannelQueueConfiguration is the configuration for a ChannelQueue
|
|
||||||
type ChannelQueueConfiguration struct { |
|
||||||
WorkerPoolConfiguration |
|
||||||
Workers int |
|
||||||
} |
|
||||||
|
|
||||||
// ChannelQueue implements Queue
|
|
||||||
//
|
|
||||||
// A channel queue is not persistable and does not shutdown or terminate cleanly
|
|
||||||
// It is basically a very thin wrapper around a WorkerPool
|
|
||||||
type ChannelQueue struct { |
|
||||||
*WorkerPool |
|
||||||
shutdownCtx context.Context |
|
||||||
shutdownCtxCancel context.CancelFunc |
|
||||||
terminateCtx context.Context |
|
||||||
terminateCtxCancel context.CancelFunc |
|
||||||
exemplar interface{} |
|
||||||
workers int |
|
||||||
name string |
|
||||||
} |
|
||||||
|
|
||||||
// NewChannelQueue creates a memory channel queue
|
|
||||||
func NewChannelQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(ChannelQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(ChannelQueueConfiguration) |
|
||||||
if config.BatchLength == 0 { |
|
||||||
config.BatchLength = 1 |
|
||||||
} |
|
||||||
|
|
||||||
terminateCtx, terminateCtxCancel := context.WithCancel(context.Background()) |
|
||||||
shutdownCtx, shutdownCtxCancel := context.WithCancel(terminateCtx) |
|
||||||
|
|
||||||
queue := &ChannelQueue{ |
|
||||||
shutdownCtx: shutdownCtx, |
|
||||||
shutdownCtxCancel: shutdownCtxCancel, |
|
||||||
terminateCtx: terminateCtx, |
|
||||||
terminateCtxCancel: terminateCtxCancel, |
|
||||||
exemplar: exemplar, |
|
||||||
workers: config.Workers, |
|
||||||
name: config.Name, |
|
||||||
} |
|
||||||
queue.WorkerPool = NewWorkerPool(func(data ...Data) []Data { |
|
||||||
unhandled := handle(data...) |
|
||||||
if len(unhandled) > 0 { |
|
||||||
// We can only pushback to the channel if we're paused.
|
|
||||||
if queue.IsPaused() { |
|
||||||
atomic.AddInt64(&queue.numInQueue, int64(len(unhandled))) |
|
||||||
go func() { |
|
||||||
for _, datum := range data { |
|
||||||
queue.dataChan <- datum |
|
||||||
} |
|
||||||
}() |
|
||||||
return nil |
|
||||||
} |
|
||||||
} |
|
||||||
return unhandled |
|
||||||
}, config.WorkerPoolConfiguration) |
|
||||||
|
|
||||||
queue.qid = GetManager().Add(queue, ChannelQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Run starts to run the queue
|
|
||||||
func (q *ChannelQueue) Run(atShutdown, atTerminate func(func())) { |
|
||||||
pprof.SetGoroutineLabels(q.baseCtx) |
|
||||||
atShutdown(q.Shutdown) |
|
||||||
atTerminate(q.Terminate) |
|
||||||
log.Debug("ChannelQueue: %s Starting", q.name) |
|
||||||
_ = q.AddWorkers(q.workers, 0) |
|
||||||
} |
|
||||||
|
|
||||||
// Push will push data into the queue
|
|
||||||
func (q *ChannelQueue) Push(data Data) error { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in queue: %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
q.WorkerPool.Push(data) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the channel with a timeout - the Flush worker will be registered as a flush worker with the manager
|
|
||||||
func (q *ChannelQueue) Flush(timeout time.Duration) error { |
|
||||||
if q.IsPaused() { |
|
||||||
return nil |
|
||||||
} |
|
||||||
ctx, cancel := q.commonRegisterWorkers(1, timeout, true) |
|
||||||
defer cancel() |
|
||||||
return q.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// Shutdown processing from this queue
|
|
||||||
func (q *ChannelQueue) Shutdown() { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
select { |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
log.Trace("ChannelQueue: %s Already Shutting down", q.name) |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
log.Trace("ChannelQueue: %s Shutting down", q.name) |
|
||||||
go func() { |
|
||||||
log.Trace("ChannelQueue: %s Flushing", q.name) |
|
||||||
// We can't use Cleanup here because that will close the channel
|
|
||||||
if err := q.FlushWithContext(q.terminateCtx); err != nil { |
|
||||||
count := atomic.LoadInt64(&q.numInQueue) |
|
||||||
if count > 0 { |
|
||||||
log.Warn("ChannelQueue: %s Terminated before completed flushing", q.name) |
|
||||||
} |
|
||||||
return |
|
||||||
} |
|
||||||
log.Debug("ChannelQueue: %s Flushed", q.name) |
|
||||||
}() |
|
||||||
q.shutdownCtxCancel() |
|
||||||
log.Debug("ChannelQueue: %s Shutdown", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Terminate this queue and close the queue
|
|
||||||
func (q *ChannelQueue) Terminate() { |
|
||||||
log.Trace("ChannelQueue: %s Terminating", q.name) |
|
||||||
q.Shutdown() |
|
||||||
select { |
|
||||||
case <-q.terminateCtx.Done(): |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
q.terminateCtxCancel() |
|
||||||
q.baseCtxFinished() |
|
||||||
log.Debug("ChannelQueue: %s Terminated", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Name returns the name of this queue
|
|
||||||
func (q *ChannelQueue) Name() string { |
|
||||||
return q.name |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[ChannelQueueType] = NewChannelQueue |
|
||||||
} |
|
@ -1,315 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"os" |
|
||||||
"sync" |
|
||||||
"testing" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert" |
|
||||||
) |
|
||||||
|
|
||||||
func TestChannelQueue(t *testing.T) { |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
nilFn := func(_ func()) {} |
|
||||||
|
|
||||||
queue, err := NewChannelQueue(handle, |
|
||||||
ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 5, |
|
||||||
Name: "TestChannelQueue", |
|
||||||
}, |
|
||||||
Workers: 0, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
assert.Equal(t, 5, queue.(*ChannelQueue).WorkerPool.boostWorkers) |
|
||||||
|
|
||||||
go queue.Run(nilFn, nilFn) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
go queue.Push(&test1) |
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
err = queue.Push(test1) |
|
||||||
assert.Error(t, err) |
|
||||||
} |
|
||||||
|
|
||||||
func TestChannelQueue_Batch(t *testing.T) { |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
assert.True(t, len(data) == 2) |
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
nilFn := func(_ func()) {} |
|
||||||
|
|
||||||
queue, err := NewChannelQueue(handle, |
|
||||||
ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 20, |
|
||||||
BatchLength: 2, |
|
||||||
BlockTimeout: 0, |
|
||||||
BoostTimeout: 0, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
}, |
|
||||||
Workers: 1, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go queue.Run(nilFn, nilFn) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
|
|
||||||
queue.Push(&test1) |
|
||||||
go queue.Push(&test2) |
|
||||||
|
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
result2 := <-handleChan |
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
err = queue.Push(test1) |
|
||||||
assert.Error(t, err) |
|
||||||
} |
|
||||||
|
|
||||||
func TestChannelQueue_Pause(t *testing.T) { |
|
||||||
if os.Getenv("CI") != "" { |
|
||||||
t.Skip("Skipping because test is flaky on CI") |
|
||||||
} |
|
||||||
lock := sync.Mutex{} |
|
||||||
var queue Queue |
|
||||||
var err error |
|
||||||
pushBack := false |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
lock.Lock() |
|
||||||
if pushBack { |
|
||||||
if pausable, ok := queue.(Pausable); ok { |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
return data |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
queueShutdown := []func(){} |
|
||||||
queueTerminate := []func(){} |
|
||||||
|
|
||||||
terminated := make(chan struct{}) |
|
||||||
|
|
||||||
queue, err = NewChannelQueue(handle, |
|
||||||
ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 20, |
|
||||||
BatchLength: 1, |
|
||||||
BlockTimeout: 0, |
|
||||||
BoostTimeout: 0, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
}, |
|
||||||
Workers: 1, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go func() { |
|
||||||
queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
}) |
|
||||||
close(terminated) |
|
||||||
}() |
|
||||||
|
|
||||||
// Shutdown and Terminate in defer
|
|
||||||
defer func() { |
|
||||||
lock.Lock() |
|
||||||
callbacks := make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
lock.Lock() |
|
||||||
log.Info("Finally terminating") |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
}() |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
queue.Push(&test1) |
|
||||||
|
|
||||||
pausable, ok := queue.(Pausable) |
|
||||||
if !assert.True(t, ok) { |
|
||||||
return |
|
||||||
} |
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
pausable.Pause() |
|
||||||
|
|
||||||
paused, _ := pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue is not paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
queue.Push(&test2) |
|
||||||
|
|
||||||
var result2 *testData |
|
||||||
select { |
|
||||||
case result2 = <-handleChan: |
|
||||||
assert.Fail(t, "handler chan should be empty") |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
} |
|
||||||
|
|
||||||
assert.Nil(t, result2) |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
_, resumed := pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case result2 = <-handleChan: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "handler chan should contain test2") |
|
||||||
} |
|
||||||
|
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
pushBack = true |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
_, resumed = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue is not resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
queue.Push(&test1) |
|
||||||
paused, _ = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "handler chan should not contain test1") |
|
||||||
return |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "queue should be paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
pushBack = false |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
paused, _ = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue is not paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
_, resumed = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case result1 = <-handleChan: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "handler chan should contain test1") |
|
||||||
} |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
callbacks := make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
queueShutdown = queueShutdown[:0] |
|
||||||
lock.Unlock() |
|
||||||
// Now shutdown the queue
|
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
// terminate the queue
|
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
queueShutdown = queueTerminate[:0] |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
select { |
|
||||||
case <-terminated: |
|
||||||
case <-time.After(10 * time.Second): |
|
||||||
assert.Fail(t, "Queue should have terminated") |
|
||||||
return |
|
||||||
} |
|
||||||
} |
|
@ -1,124 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/nosql" |
|
||||||
|
|
||||||
"gitea.com/lunny/levelqueue" |
|
||||||
) |
|
||||||
|
|
||||||
// LevelQueueType is the type for level queue
|
|
||||||
const LevelQueueType Type = "level" |
|
||||||
|
|
||||||
// LevelQueueConfiguration is the configuration for a LevelQueue
|
|
||||||
type LevelQueueConfiguration struct { |
|
||||||
ByteFIFOQueueConfiguration |
|
||||||
DataDir string |
|
||||||
ConnectionString string |
|
||||||
QueueName string |
|
||||||
} |
|
||||||
|
|
||||||
// LevelQueue implements a disk library queue
|
|
||||||
type LevelQueue struct { |
|
||||||
*ByteFIFOQueue |
|
||||||
} |
|
||||||
|
|
||||||
// NewLevelQueue creates a ledis local queue
|
|
||||||
func NewLevelQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(LevelQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(LevelQueueConfiguration) |
|
||||||
|
|
||||||
if len(config.ConnectionString) == 0 { |
|
||||||
config.ConnectionString = config.DataDir |
|
||||||
} |
|
||||||
config.WaitOnEmpty = true |
|
||||||
|
|
||||||
byteFIFO, err := NewLevelQueueByteFIFO(config.ConnectionString, config.QueueName) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
byteFIFOQueue, err := NewByteFIFOQueue(LevelQueueType, byteFIFO, handle, config.ByteFIFOQueueConfiguration, exemplar) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
queue := &LevelQueue{ |
|
||||||
ByteFIFOQueue: byteFIFOQueue, |
|
||||||
} |
|
||||||
queue.qid = GetManager().Add(queue, LevelQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
var _ ByteFIFO = &LevelQueueByteFIFO{} |
|
||||||
|
|
||||||
// LevelQueueByteFIFO represents a ByteFIFO formed from a LevelQueue
|
|
||||||
type LevelQueueByteFIFO struct { |
|
||||||
internal *levelqueue.Queue |
|
||||||
connection string |
|
||||||
} |
|
||||||
|
|
||||||
// NewLevelQueueByteFIFO creates a ByteFIFO formed from a LevelQueue
|
|
||||||
func NewLevelQueueByteFIFO(connection, prefix string) (*LevelQueueByteFIFO, error) { |
|
||||||
db, err := nosql.GetManager().GetLevelDB(connection) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
internal, err := levelqueue.NewQueue(db, []byte(prefix), false) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
return &LevelQueueByteFIFO{ |
|
||||||
connection: connection, |
|
||||||
internal: internal, |
|
||||||
}, nil |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc will push data into the fifo
|
|
||||||
func (fifo *LevelQueueByteFIFO) PushFunc(ctx context.Context, data []byte, fn func() error) error { |
|
||||||
if fn != nil { |
|
||||||
if err := fn(); err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
} |
|
||||||
return fifo.internal.LPush(data) |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack pushes data to the top of the fifo
|
|
||||||
func (fifo *LevelQueueByteFIFO) PushBack(ctx context.Context, data []byte) error { |
|
||||||
return fifo.internal.RPush(data) |
|
||||||
} |
|
||||||
|
|
||||||
// Pop pops data from the start of the fifo
|
|
||||||
func (fifo *LevelQueueByteFIFO) Pop(ctx context.Context) ([]byte, error) { |
|
||||||
data, err := fifo.internal.RPop() |
|
||||||
if err != nil && err != levelqueue.ErrNotFound { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
return data, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Close this fifo
|
|
||||||
func (fifo *LevelQueueByteFIFO) Close() error { |
|
||||||
err := fifo.internal.Close() |
|
||||||
_ = nosql.GetManager().CloseLevelDB(fifo.connection) |
|
||||||
return err |
|
||||||
} |
|
||||||
|
|
||||||
// Len returns the length of the fifo
|
|
||||||
func (fifo *LevelQueueByteFIFO) Len(ctx context.Context) int64 { |
|
||||||
return fifo.internal.Len() |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[LevelQueueType] = NewLevelQueue |
|
||||||
} |
|
@ -1,358 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"runtime/pprof" |
|
||||||
"sync" |
|
||||||
"sync/atomic" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
) |
|
||||||
|
|
||||||
// PersistableChannelQueueType is the type for persistable queue
|
|
||||||
const PersistableChannelQueueType Type = "persistable-channel" |
|
||||||
|
|
||||||
// PersistableChannelQueueConfiguration is the configuration for a PersistableChannelQueue
|
|
||||||
type PersistableChannelQueueConfiguration struct { |
|
||||||
Name string |
|
||||||
DataDir string |
|
||||||
BatchLength int |
|
||||||
QueueLength int |
|
||||||
Timeout time.Duration |
|
||||||
MaxAttempts int |
|
||||||
Workers int |
|
||||||
MaxWorkers int |
|
||||||
BlockTimeout time.Duration |
|
||||||
BoostTimeout time.Duration |
|
||||||
BoostWorkers int |
|
||||||
} |
|
||||||
|
|
||||||
// PersistableChannelQueue wraps a channel queue and level queue together
|
|
||||||
// The disk level queue will be used to store data at shutdown and terminate - and will be restored
|
|
||||||
// on start up.
|
|
||||||
type PersistableChannelQueue struct { |
|
||||||
channelQueue *ChannelQueue |
|
||||||
delayedStarter |
|
||||||
lock sync.Mutex |
|
||||||
closed chan struct{} |
|
||||||
} |
|
||||||
|
|
||||||
// NewPersistableChannelQueue creates a wrapped batched channel queue with persistable level queue backend when shutting down
|
|
||||||
// This differs from a wrapped queue in that the persistent queue is only used to persist at shutdown/terminate
|
|
||||||
func NewPersistableChannelQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(PersistableChannelQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(PersistableChannelQueueConfiguration) |
|
||||||
|
|
||||||
queue := &PersistableChannelQueue{ |
|
||||||
closed: make(chan struct{}), |
|
||||||
} |
|
||||||
|
|
||||||
wrappedHandle := func(data ...Data) (failed []Data) { |
|
||||||
for _, unhandled := range handle(data...) { |
|
||||||
if fail := queue.PushBack(unhandled); fail != nil { |
|
||||||
failed = append(failed, fail) |
|
||||||
} |
|
||||||
} |
|
||||||
return failed |
|
||||||
} |
|
||||||
|
|
||||||
channelQueue, err := NewChannelQueue(wrappedHandle, ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: config.QueueLength, |
|
||||||
BatchLength: config.BatchLength, |
|
||||||
BlockTimeout: config.BlockTimeout, |
|
||||||
BoostTimeout: config.BoostTimeout, |
|
||||||
BoostWorkers: config.BoostWorkers, |
|
||||||
MaxWorkers: config.MaxWorkers, |
|
||||||
Name: config.Name + "-channel", |
|
||||||
}, |
|
||||||
Workers: config.Workers, |
|
||||||
}, exemplar) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
// the level backend only needs temporary workers to catch up with the previously dropped work
|
|
||||||
levelCfg := LevelQueueConfiguration{ |
|
||||||
ByteFIFOQueueConfiguration: ByteFIFOQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: config.QueueLength, |
|
||||||
BatchLength: config.BatchLength, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 1, |
|
||||||
MaxWorkers: 5, |
|
||||||
Name: config.Name + "-level", |
|
||||||
}, |
|
||||||
Workers: 0, |
|
||||||
}, |
|
||||||
DataDir: config.DataDir, |
|
||||||
QueueName: config.Name + "-level", |
|
||||||
} |
|
||||||
|
|
||||||
levelQueue, err := NewLevelQueue(wrappedHandle, levelCfg, exemplar) |
|
||||||
if err == nil { |
|
||||||
queue.channelQueue = channelQueue.(*ChannelQueue) |
|
||||||
queue.delayedStarter = delayedStarter{ |
|
||||||
internal: levelQueue.(*LevelQueue), |
|
||||||
name: config.Name, |
|
||||||
} |
|
||||||
_ = GetManager().Add(queue, PersistableChannelQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
if IsErrInvalidConfiguration(err) { |
|
||||||
// Retrying ain't gonna make this any better...
|
|
||||||
return nil, ErrInvalidConfiguration{cfg: cfg} |
|
||||||
} |
|
||||||
|
|
||||||
queue.channelQueue = channelQueue.(*ChannelQueue) |
|
||||||
queue.delayedStarter = delayedStarter{ |
|
||||||
cfg: levelCfg, |
|
||||||
underlying: LevelQueueType, |
|
||||||
timeout: config.Timeout, |
|
||||||
maxAttempts: config.MaxAttempts, |
|
||||||
name: config.Name, |
|
||||||
} |
|
||||||
_ = GetManager().Add(queue, PersistableChannelQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Name returns the name of this queue
|
|
||||||
func (q *PersistableChannelQueue) Name() string { |
|
||||||
return q.delayedStarter.name |
|
||||||
} |
|
||||||
|
|
||||||
// Push will push the indexer data to queue
|
|
||||||
func (q *PersistableChannelQueue) Push(data Data) error { |
|
||||||
select { |
|
||||||
case <-q.closed: |
|
||||||
return q.internal.Push(data) |
|
||||||
default: |
|
||||||
return q.channelQueue.Push(data) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack will push the indexer data to queue
|
|
||||||
func (q *PersistableChannelQueue) PushBack(data Data) error { |
|
||||||
select { |
|
||||||
case <-q.closed: |
|
||||||
if pbr, ok := q.internal.(PushBackable); ok { |
|
||||||
return pbr.PushBack(data) |
|
||||||
} |
|
||||||
return q.internal.Push(data) |
|
||||||
default: |
|
||||||
return q.channelQueue.Push(data) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Run starts to run the queue
|
|
||||||
func (q *PersistableChannelQueue) Run(atShutdown, atTerminate func(func())) { |
|
||||||
pprof.SetGoroutineLabels(q.channelQueue.baseCtx) |
|
||||||
log.Debug("PersistableChannelQueue: %s Starting", q.delayedStarter.name) |
|
||||||
_ = q.channelQueue.AddWorkers(q.channelQueue.workers, 0) |
|
||||||
|
|
||||||
q.lock.Lock() |
|
||||||
if q.internal == nil { |
|
||||||
err := q.setInternal(atShutdown, q.channelQueue.handle, q.channelQueue.exemplar) |
|
||||||
q.lock.Unlock() |
|
||||||
if err != nil { |
|
||||||
log.Fatal("Unable to create internal queue for %s Error: %v", q.Name(), err) |
|
||||||
return |
|
||||||
} |
|
||||||
} else { |
|
||||||
q.lock.Unlock() |
|
||||||
} |
|
||||||
atShutdown(q.Shutdown) |
|
||||||
atTerminate(q.Terminate) |
|
||||||
|
|
||||||
if lq, ok := q.internal.(*LevelQueue); ok && lq.byteFIFO.Len(lq.terminateCtx) != 0 { |
|
||||||
// Just run the level queue - we shut it down once it's flushed
|
|
||||||
go q.internal.Run(func(_ func()) {}, func(_ func()) {}) |
|
||||||
go func() { |
|
||||||
for !lq.IsEmpty() { |
|
||||||
_ = lq.Flush(0) |
|
||||||
select { |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
case <-lq.shutdownCtx.Done(): |
|
||||||
if lq.byteFIFO.Len(lq.terminateCtx) > 0 { |
|
||||||
log.Warn("LevelQueue: %s shut down before completely flushed", q.internal.(*LevelQueue).Name()) |
|
||||||
} |
|
||||||
return |
|
||||||
} |
|
||||||
} |
|
||||||
log.Debug("LevelQueue: %s flushed so shutting down", q.internal.(*LevelQueue).Name()) |
|
||||||
q.internal.(*LevelQueue).Shutdown() |
|
||||||
GetManager().Remove(q.internal.(*LevelQueue).qid) |
|
||||||
}() |
|
||||||
} else { |
|
||||||
log.Debug("PersistableChannelQueue: %s Skipping running the empty level queue", q.delayedStarter.name) |
|
||||||
q.internal.(*LevelQueue).Shutdown() |
|
||||||
GetManager().Remove(q.internal.(*LevelQueue).qid) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the queue and blocks till the queue is empty
|
|
||||||
func (q *PersistableChannelQueue) Flush(timeout time.Duration) error { |
|
||||||
var ctx context.Context |
|
||||||
var cancel context.CancelFunc |
|
||||||
if timeout > 0 { |
|
||||||
ctx, cancel = context.WithTimeout(context.Background(), timeout) |
|
||||||
} else { |
|
||||||
ctx, cancel = context.WithCancel(context.Background()) |
|
||||||
} |
|
||||||
defer cancel() |
|
||||||
return q.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// FlushWithContext flushes the queue and blocks till the queue is empty
|
|
||||||
func (q *PersistableChannelQueue) FlushWithContext(ctx context.Context) error { |
|
||||||
errChan := make(chan error, 1) |
|
||||||
go func() { |
|
||||||
errChan <- q.channelQueue.FlushWithContext(ctx) |
|
||||||
}() |
|
||||||
go func() { |
|
||||||
q.lock.Lock() |
|
||||||
if q.internal == nil { |
|
||||||
q.lock.Unlock() |
|
||||||
errChan <- fmt.Errorf("not ready to flush internal queue %s yet", q.Name()) |
|
||||||
return |
|
||||||
} |
|
||||||
q.lock.Unlock() |
|
||||||
errChan <- q.internal.FlushWithContext(ctx) |
|
||||||
}() |
|
||||||
err1 := <-errChan |
|
||||||
err2 := <-errChan |
|
||||||
|
|
||||||
if err1 != nil { |
|
||||||
return err1 |
|
||||||
} |
|
||||||
return err2 |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty checks if a queue is empty
|
|
||||||
func (q *PersistableChannelQueue) IsEmpty() bool { |
|
||||||
if !q.channelQueue.IsEmpty() { |
|
||||||
return false |
|
||||||
} |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return false |
|
||||||
} |
|
||||||
return q.internal.IsEmpty() |
|
||||||
} |
|
||||||
|
|
||||||
// IsPaused returns if the pool is paused
|
|
||||||
func (q *PersistableChannelQueue) IsPaused() bool { |
|
||||||
return q.channelQueue.IsPaused() |
|
||||||
} |
|
||||||
|
|
||||||
// IsPausedIsResumed returns if the pool is paused and a channel that is closed when it is resumed
|
|
||||||
func (q *PersistableChannelQueue) IsPausedIsResumed() (<-chan struct{}, <-chan struct{}) { |
|
||||||
return q.channelQueue.IsPausedIsResumed() |
|
||||||
} |
|
||||||
|
|
||||||
// Pause pauses the WorkerPool
|
|
||||||
func (q *PersistableChannelQueue) Pause() { |
|
||||||
q.channelQueue.Pause() |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
pausable, ok := q.internal.(Pausable) |
|
||||||
if !ok { |
|
||||||
return |
|
||||||
} |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
|
|
||||||
// Resume resumes the WorkerPool
|
|
||||||
func (q *PersistableChannelQueue) Resume() { |
|
||||||
q.channelQueue.Resume() |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
pausable, ok := q.internal.(Pausable) |
|
||||||
if !ok { |
|
||||||
return |
|
||||||
} |
|
||||||
pausable.Resume() |
|
||||||
} |
|
||||||
|
|
||||||
// Shutdown processing this queue
|
|
||||||
func (q *PersistableChannelQueue) Shutdown() { |
|
||||||
log.Trace("PersistableChannelQueue: %s Shutting down", q.delayedStarter.name) |
|
||||||
q.lock.Lock() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-q.closed: |
|
||||||
q.lock.Unlock() |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
q.channelQueue.Shutdown() |
|
||||||
if q.internal != nil { |
|
||||||
q.internal.(*LevelQueue).Shutdown() |
|
||||||
} |
|
||||||
close(q.closed) |
|
||||||
q.lock.Unlock() |
|
||||||
|
|
||||||
log.Trace("PersistableChannelQueue: %s Cancelling pools", q.delayedStarter.name) |
|
||||||
q.channelQueue.baseCtxCancel() |
|
||||||
q.internal.(*LevelQueue).baseCtxCancel() |
|
||||||
log.Trace("PersistableChannelQueue: %s Waiting til done", q.delayedStarter.name) |
|
||||||
q.channelQueue.Wait() |
|
||||||
q.internal.(*LevelQueue).Wait() |
|
||||||
// Redirect all remaining data in the chan to the internal channel
|
|
||||||
log.Trace("PersistableChannelQueue: %s Redirecting remaining data", q.delayedStarter.name) |
|
||||||
close(q.channelQueue.dataChan) |
|
||||||
countOK, countLost := 0, 0 |
|
||||||
for data := range q.channelQueue.dataChan { |
|
||||||
err := q.internal.Push(data) |
|
||||||
if err != nil { |
|
||||||
log.Error("PersistableChannelQueue: %s Unable redirect %v due to: %v", q.delayedStarter.name, data, err) |
|
||||||
countLost++ |
|
||||||
} else { |
|
||||||
countOK++ |
|
||||||
} |
|
||||||
atomic.AddInt64(&q.channelQueue.numInQueue, -1) |
|
||||||
} |
|
||||||
if countLost > 0 { |
|
||||||
log.Warn("PersistableChannelQueue: %s %d will be restored on restart, %d lost", q.delayedStarter.name, countOK, countLost) |
|
||||||
} else if countOK > 0 { |
|
||||||
log.Warn("PersistableChannelQueue: %s %d will be restored on restart", q.delayedStarter.name, countOK) |
|
||||||
} |
|
||||||
log.Trace("PersistableChannelQueue: %s Done Redirecting remaining data", q.delayedStarter.name) |
|
||||||
|
|
||||||
log.Debug("PersistableChannelQueue: %s Shutdown", q.delayedStarter.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Terminate this queue and close the queue
|
|
||||||
func (q *PersistableChannelQueue) Terminate() { |
|
||||||
log.Trace("PersistableChannelQueue: %s Terminating", q.delayedStarter.name) |
|
||||||
q.Shutdown() |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
q.channelQueue.Terminate() |
|
||||||
if q.internal != nil { |
|
||||||
q.internal.(*LevelQueue).Terminate() |
|
||||||
} |
|
||||||
log.Debug("PersistableChannelQueue: %s Terminated", q.delayedStarter.name) |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[PersistableChannelQueueType] = NewPersistableChannelQueue |
|
||||||
} |
|
@ -1,544 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"sync" |
|
||||||
"testing" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert" |
|
||||||
) |
|
||||||
|
|
||||||
func TestPersistableChannelQueue(t *testing.T) { |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
for _, datum := range data { |
|
||||||
if datum == nil { |
|
||||||
continue |
|
||||||
} |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
lock := sync.Mutex{} |
|
||||||
queueShutdown := []func(){} |
|
||||||
queueTerminate := []func(){} |
|
||||||
|
|
||||||
tmpDir := t.TempDir() |
|
||||||
|
|
||||||
queue, err := NewPersistableChannelQueue(handle, PersistableChannelQueueConfiguration{ |
|
||||||
DataDir: tmpDir, |
|
||||||
BatchLength: 2, |
|
||||||
QueueLength: 20, |
|
||||||
Workers: 1, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
Name: "test-queue", |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
readyForShutdown := make(chan struct{}) |
|
||||||
readyForTerminate := make(chan struct{}) |
|
||||||
|
|
||||||
go queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
select { |
|
||||||
case <-readyForShutdown: |
|
||||||
default: |
|
||||||
close(readyForShutdown) |
|
||||||
} |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
select { |
|
||||||
case <-readyForTerminate: |
|
||||||
default: |
|
||||||
close(readyForTerminate) |
|
||||||
} |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
}) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
|
|
||||||
err = queue.Push(&test1) |
|
||||||
assert.NoError(t, err) |
|
||||||
go func() { |
|
||||||
err := queue.Push(&test2) |
|
||||||
assert.NoError(t, err) |
|
||||||
}() |
|
||||||
|
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
result2 := <-handleChan |
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
// test1 is a testData not a *testData so will be rejected
|
|
||||||
err = queue.Push(test1) |
|
||||||
assert.Error(t, err) |
|
||||||
|
|
||||||
<-readyForShutdown |
|
||||||
// Now shutdown the queue
|
|
||||||
lock.Lock() |
|
||||||
callbacks := make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
// Wait til it is closed
|
|
||||||
<-queue.(*PersistableChannelQueue).closed |
|
||||||
|
|
||||||
err = queue.Push(&test1) |
|
||||||
assert.NoError(t, err) |
|
||||||
err = queue.Push(&test2) |
|
||||||
assert.NoError(t, err) |
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
// terminate the queue
|
|
||||||
<-readyForTerminate |
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
// Reopen queue
|
|
||||||
queue, err = NewPersistableChannelQueue(handle, PersistableChannelQueueConfiguration{ |
|
||||||
DataDir: tmpDir, |
|
||||||
BatchLength: 2, |
|
||||||
QueueLength: 20, |
|
||||||
Workers: 1, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
Name: "test-queue", |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
readyForShutdown = make(chan struct{}) |
|
||||||
readyForTerminate = make(chan struct{}) |
|
||||||
|
|
||||||
go queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
select { |
|
||||||
case <-readyForShutdown: |
|
||||||
default: |
|
||||||
close(readyForShutdown) |
|
||||||
} |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
select { |
|
||||||
case <-readyForTerminate: |
|
||||||
default: |
|
||||||
close(readyForTerminate) |
|
||||||
} |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
}) |
|
||||||
|
|
||||||
result3 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result3.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result3.TestInt) |
|
||||||
|
|
||||||
result4 := <-handleChan |
|
||||||
assert.Equal(t, test2.TestString, result4.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result4.TestInt) |
|
||||||
|
|
||||||
<-readyForShutdown |
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
<-readyForTerminate |
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
func TestPersistableChannelQueue_Pause(t *testing.T) { |
|
||||||
lock := sync.Mutex{} |
|
||||||
var queue Queue |
|
||||||
var err error |
|
||||||
pushBack := false |
|
||||||
|
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
lock.Lock() |
|
||||||
if pushBack { |
|
||||||
if pausable, ok := queue.(Pausable); ok { |
|
||||||
log.Info("pausing") |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
return data |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
queueShutdown := []func(){} |
|
||||||
queueTerminate := []func(){} |
|
||||||
terminated := make(chan struct{}) |
|
||||||
|
|
||||||
tmpDir := t.TempDir() |
|
||||||
|
|
||||||
queue, err = NewPersistableChannelQueue(handle, PersistableChannelQueueConfiguration{ |
|
||||||
DataDir: tmpDir, |
|
||||||
BatchLength: 2, |
|
||||||
QueueLength: 20, |
|
||||||
Workers: 1, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
Name: "test-queue", |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go func() { |
|
||||||
queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
}) |
|
||||||
close(terminated) |
|
||||||
}() |
|
||||||
|
|
||||||
// Shutdown and Terminate in defer
|
|
||||||
defer func() { |
|
||||||
lock.Lock() |
|
||||||
callbacks := make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
lock.Lock() |
|
||||||
log.Info("Finally terminating") |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
}() |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
|
|
||||||
err = queue.Push(&test1) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
pausable, ok := queue.(Pausable) |
|
||||||
if !assert.True(t, ok) { |
|
||||||
return |
|
||||||
} |
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
pausable.Pause() |
|
||||||
paused, _ := pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue is not paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
queue.Push(&test2) |
|
||||||
|
|
||||||
var result2 *testData |
|
||||||
select { |
|
||||||
case result2 = <-handleChan: |
|
||||||
assert.Fail(t, "handler chan should be empty") |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
} |
|
||||||
|
|
||||||
assert.Nil(t, result2) |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
_, resumed := pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case result2 = <-handleChan: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "handler chan should contain test2") |
|
||||||
} |
|
||||||
|
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
// Set pushBack to so that the next handle will result in a Pause
|
|
||||||
lock.Lock() |
|
||||||
pushBack = true |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
// Ensure that we're still resumed
|
|
||||||
_, resumed = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue is not resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
// push test1
|
|
||||||
queue.Push(&test1) |
|
||||||
|
|
||||||
// Now as this is handled it should pause
|
|
||||||
paused, _ = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "handler chan should not contain test1") |
|
||||||
return |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "queue should be paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
pushBack = false |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
|
|
||||||
_, resumed = pausable.IsPausedIsResumed() |
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case result1 = <-handleChan: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "handler chan should contain test1") |
|
||||||
return |
|
||||||
} |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
callbacks := make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
queueShutdown = queueShutdown[:0] |
|
||||||
lock.Unlock() |
|
||||||
// Now shutdown the queue
|
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
// Wait til it is closed
|
|
||||||
select { |
|
||||||
case <-queue.(*PersistableChannelQueue).closed: |
|
||||||
case <-time.After(5 * time.Second): |
|
||||||
assert.Fail(t, "queue should close") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
err = queue.Push(&test1) |
|
||||||
assert.NoError(t, err) |
|
||||||
err = queue.Push(&test2) |
|
||||||
assert.NoError(t, err) |
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
// terminate the queue
|
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
queueShutdown = queueTerminate[:0] |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
return |
|
||||||
case <-terminated: |
|
||||||
case <-time.After(10 * time.Second): |
|
||||||
assert.Fail(t, "Queue should have terminated") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
pushBack = true |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
// Reopen queue
|
|
||||||
terminated = make(chan struct{}) |
|
||||||
queue, err = NewPersistableChannelQueue(handle, PersistableChannelQueueConfiguration{ |
|
||||||
DataDir: tmpDir, |
|
||||||
BatchLength: 1, |
|
||||||
QueueLength: 20, |
|
||||||
Workers: 1, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
Name: "test-queue", |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
pausable, ok = queue.(Pausable) |
|
||||||
if !assert.True(t, ok) { |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
paused, _ = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
go func() { |
|
||||||
queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
}) |
|
||||||
close(terminated) |
|
||||||
}() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
return |
|
||||||
case <-paused: |
|
||||||
} |
|
||||||
|
|
||||||
paused, _ = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue is not paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
pushBack = false |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
_, resumed = pausable.IsPausedIsResumed() |
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
var result3, result4 *testData |
|
||||||
|
|
||||||
select { |
|
||||||
case result3 = <-handleChan: |
|
||||||
case <-time.After(1 * time.Second): |
|
||||||
assert.Fail(t, "Handler processing should have resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
select { |
|
||||||
case result4 = <-handleChan: |
|
||||||
case <-time.After(1 * time.Second): |
|
||||||
assert.Fail(t, "Handler processing should have resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
if result4.TestString == test1.TestString { |
|
||||||
result3, result4 = result4, result3 |
|
||||||
} |
|
||||||
assert.Equal(t, test1.TestString, result3.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result3.TestInt) |
|
||||||
|
|
||||||
assert.Equal(t, test2.TestString, result4.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result4.TestInt) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueShutdown)) |
|
||||||
copy(callbacks, queueShutdown) |
|
||||||
queueShutdown = queueShutdown[:0] |
|
||||||
lock.Unlock() |
|
||||||
// Now shutdown the queue
|
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
// terminate the queue
|
|
||||||
lock.Lock() |
|
||||||
callbacks = make([]func(), len(queueTerminate)) |
|
||||||
copy(callbacks, queueTerminate) |
|
||||||
queueShutdown = queueTerminate[:0] |
|
||||||
lock.Unlock() |
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case <-time.After(10 * time.Second): |
|
||||||
assert.Fail(t, "Queue should have terminated") |
|
||||||
return |
|
||||||
case <-terminated: |
|
||||||
} |
|
||||||
} |
|
@ -1,147 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"sync" |
|
||||||
"testing" |
|
||||||
"time" |
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert" |
|
||||||
) |
|
||||||
|
|
||||||
func TestLevelQueue(t *testing.T) { |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
assert.True(t, len(data) == 2) |
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
var lock sync.Mutex |
|
||||||
queueShutdown := []func(){} |
|
||||||
queueTerminate := []func(){} |
|
||||||
|
|
||||||
tmpDir := t.TempDir() |
|
||||||
|
|
||||||
queue, err := NewLevelQueue(handle, LevelQueueConfiguration{ |
|
||||||
ByteFIFOQueueConfiguration: ByteFIFOQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 20, |
|
||||||
BatchLength: 2, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 5, |
|
||||||
MaxWorkers: 10, |
|
||||||
}, |
|
||||||
Workers: 1, |
|
||||||
}, |
|
||||||
DataDir: tmpDir, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
lock.Unlock() |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
lock.Unlock() |
|
||||||
}) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
|
|
||||||
err = queue.Push(&test1) |
|
||||||
assert.NoError(t, err) |
|
||||||
go func() { |
|
||||||
err := queue.Push(&test2) |
|
||||||
assert.NoError(t, err) |
|
||||||
}() |
|
||||||
|
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
result2 := <-handleChan |
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
err = queue.Push(test1) |
|
||||||
assert.Error(t, err) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
for _, callback := range queueShutdown { |
|
||||||
callback() |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
time.Sleep(200 * time.Millisecond) |
|
||||||
err = queue.Push(&test1) |
|
||||||
assert.NoError(t, err) |
|
||||||
err = queue.Push(&test2) |
|
||||||
assert.NoError(t, err) |
|
||||||
select { |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "Handler processing should have stopped") |
|
||||||
default: |
|
||||||
} |
|
||||||
lock.Lock() |
|
||||||
for _, callback := range queueTerminate { |
|
||||||
callback() |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
// Reopen queue
|
|
||||||
queue, err = NewWrappedQueue(handle, |
|
||||||
WrappedQueueConfiguration{ |
|
||||||
Underlying: LevelQueueType, |
|
||||||
Config: LevelQueueConfiguration{ |
|
||||||
ByteFIFOQueueConfiguration: ByteFIFOQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 20, |
|
||||||
BatchLength: 2, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 5, |
|
||||||
MaxWorkers: 10, |
|
||||||
}, |
|
||||||
Workers: 1, |
|
||||||
}, |
|
||||||
DataDir: tmpDir, |
|
||||||
}, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go queue.Run(func(shutdown func()) { |
|
||||||
lock.Lock() |
|
||||||
queueShutdown = append(queueShutdown, shutdown) |
|
||||||
lock.Unlock() |
|
||||||
}, func(terminate func()) { |
|
||||||
lock.Lock() |
|
||||||
queueTerminate = append(queueTerminate, terminate) |
|
||||||
lock.Unlock() |
|
||||||
}) |
|
||||||
|
|
||||||
result3 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result3.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result3.TestInt) |
|
||||||
|
|
||||||
result4 := <-handleChan |
|
||||||
assert.Equal(t, test2.TestString, result4.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result4.TestInt) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
for _, callback := range queueShutdown { |
|
||||||
callback() |
|
||||||
} |
|
||||||
for _, callback := range queueTerminate { |
|
||||||
callback() |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
} |
|
@ -1,137 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/graceful" |
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
"code.gitea.io/gitea/modules/nosql" |
|
||||||
|
|
||||||
"github.com/redis/go-redis/v9" |
|
||||||
) |
|
||||||
|
|
||||||
// RedisQueueType is the type for redis queue
|
|
||||||
const RedisQueueType Type = "redis" |
|
||||||
|
|
||||||
// RedisQueueConfiguration is the configuration for the redis queue
|
|
||||||
type RedisQueueConfiguration struct { |
|
||||||
ByteFIFOQueueConfiguration |
|
||||||
RedisByteFIFOConfiguration |
|
||||||
} |
|
||||||
|
|
||||||
// RedisQueue redis queue
|
|
||||||
type RedisQueue struct { |
|
||||||
*ByteFIFOQueue |
|
||||||
} |
|
||||||
|
|
||||||
// NewRedisQueue creates single redis or cluster redis queue
|
|
||||||
func NewRedisQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(RedisQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(RedisQueueConfiguration) |
|
||||||
|
|
||||||
byteFIFO, err := NewRedisByteFIFO(config.RedisByteFIFOConfiguration) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
byteFIFOQueue, err := NewByteFIFOQueue(RedisQueueType, byteFIFO, handle, config.ByteFIFOQueueConfiguration, exemplar) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
queue := &RedisQueue{ |
|
||||||
ByteFIFOQueue: byteFIFOQueue, |
|
||||||
} |
|
||||||
|
|
||||||
queue.qid = GetManager().Add(queue, RedisQueueType, config, exemplar) |
|
||||||
|
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
type redisClient interface { |
|
||||||
RPush(ctx context.Context, key string, args ...interface{}) *redis.IntCmd |
|
||||||
LPush(ctx context.Context, key string, args ...interface{}) *redis.IntCmd |
|
||||||
LPop(ctx context.Context, key string) *redis.StringCmd |
|
||||||
LLen(ctx context.Context, key string) *redis.IntCmd |
|
||||||
SAdd(ctx context.Context, key string, members ...interface{}) *redis.IntCmd |
|
||||||
SRem(ctx context.Context, key string, members ...interface{}) *redis.IntCmd |
|
||||||
SIsMember(ctx context.Context, key string, member interface{}) *redis.BoolCmd |
|
||||||
Ping(ctx context.Context) *redis.StatusCmd |
|
||||||
Close() error |
|
||||||
} |
|
||||||
|
|
||||||
var _ ByteFIFO = &RedisByteFIFO{} |
|
||||||
|
|
||||||
// RedisByteFIFO represents a ByteFIFO formed from a redisClient
|
|
||||||
type RedisByteFIFO struct { |
|
||||||
client redisClient |
|
||||||
|
|
||||||
queueName string |
|
||||||
} |
|
||||||
|
|
||||||
// RedisByteFIFOConfiguration is the configuration for the RedisByteFIFO
|
|
||||||
type RedisByteFIFOConfiguration struct { |
|
||||||
ConnectionString string |
|
||||||
QueueName string |
|
||||||
} |
|
||||||
|
|
||||||
// NewRedisByteFIFO creates a ByteFIFO formed from a redisClient
|
|
||||||
func NewRedisByteFIFO(config RedisByteFIFOConfiguration) (*RedisByteFIFO, error) { |
|
||||||
fifo := &RedisByteFIFO{ |
|
||||||
queueName: config.QueueName, |
|
||||||
} |
|
||||||
fifo.client = nosql.GetManager().GetRedisClient(config.ConnectionString) |
|
||||||
if err := fifo.client.Ping(graceful.GetManager().ShutdownContext()).Err(); err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
return fifo, nil |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc pushes data to the end of the fifo and calls the callback if it is added
|
|
||||||
func (fifo *RedisByteFIFO) PushFunc(ctx context.Context, data []byte, fn func() error) error { |
|
||||||
if fn != nil { |
|
||||||
if err := fn(); err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
} |
|
||||||
return fifo.client.RPush(ctx, fifo.queueName, data).Err() |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack pushes data to the top of the fifo
|
|
||||||
func (fifo *RedisByteFIFO) PushBack(ctx context.Context, data []byte) error { |
|
||||||
return fifo.client.LPush(ctx, fifo.queueName, data).Err() |
|
||||||
} |
|
||||||
|
|
||||||
// Pop pops data from the start of the fifo
|
|
||||||
func (fifo *RedisByteFIFO) Pop(ctx context.Context) ([]byte, error) { |
|
||||||
data, err := fifo.client.LPop(ctx, fifo.queueName).Bytes() |
|
||||||
if err == nil || err == redis.Nil { |
|
||||||
return data, nil |
|
||||||
} |
|
||||||
return data, err |
|
||||||
} |
|
||||||
|
|
||||||
// Close this fifo
|
|
||||||
func (fifo *RedisByteFIFO) Close() error { |
|
||||||
return fifo.client.Close() |
|
||||||
} |
|
||||||
|
|
||||||
// Len returns the length of the fifo
|
|
||||||
func (fifo *RedisByteFIFO) Len(ctx context.Context) int64 { |
|
||||||
val, err := fifo.client.LLen(ctx, fifo.queueName).Result() |
|
||||||
if err != nil { |
|
||||||
log.Error("Error whilst getting length of redis queue %s: Error: %v", fifo.queueName, err) |
|
||||||
return -1 |
|
||||||
} |
|
||||||
return val |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[RedisQueueType] = NewRedisQueue |
|
||||||
} |
|
@ -1,42 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"testing" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/json" |
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert" |
|
||||||
) |
|
||||||
|
|
||||||
type testData struct { |
|
||||||
TestString string |
|
||||||
TestInt int |
|
||||||
} |
|
||||||
|
|
||||||
func TestToConfig(t *testing.T) { |
|
||||||
cfg := testData{ |
|
||||||
TestString: "Config", |
|
||||||
TestInt: 10, |
|
||||||
} |
|
||||||
exemplar := testData{} |
|
||||||
|
|
||||||
cfg2I, err := toConfig(exemplar, cfg) |
|
||||||
assert.NoError(t, err) |
|
||||||
cfg2, ok := (cfg2I).(testData) |
|
||||||
assert.True(t, ok) |
|
||||||
assert.NotEqual(t, cfg2, exemplar) |
|
||||||
assert.Equal(t, &cfg, &cfg2) |
|
||||||
cfgString, err := json.Marshal(cfg) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
cfg3I, err := toConfig(exemplar, cfgString) |
|
||||||
assert.NoError(t, err) |
|
||||||
cfg3, ok := (cfg3I).(testData) |
|
||||||
assert.True(t, ok) |
|
||||||
assert.Equal(t, cfg.TestString, cfg3.TestString) |
|
||||||
assert.Equal(t, cfg.TestInt, cfg3.TestInt) |
|
||||||
assert.NotEqual(t, cfg3, exemplar) |
|
||||||
} |
|
@ -1,315 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"sync" |
|
||||||
"sync/atomic" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
"code.gitea.io/gitea/modules/util" |
|
||||||
) |
|
||||||
|
|
||||||
// WrappedQueueType is the type for a wrapped delayed starting queue
|
|
||||||
const WrappedQueueType Type = "wrapped" |
|
||||||
|
|
||||||
// WrappedQueueConfiguration is the configuration for a WrappedQueue
|
|
||||||
type WrappedQueueConfiguration struct { |
|
||||||
Underlying Type |
|
||||||
Timeout time.Duration |
|
||||||
MaxAttempts int |
|
||||||
Config interface{} |
|
||||||
QueueLength int |
|
||||||
Name string |
|
||||||
} |
|
||||||
|
|
||||||
type delayedStarter struct { |
|
||||||
internal Queue |
|
||||||
underlying Type |
|
||||||
cfg interface{} |
|
||||||
timeout time.Duration |
|
||||||
maxAttempts int |
|
||||||
name string |
|
||||||
} |
|
||||||
|
|
||||||
// setInternal must be called with the lock locked.
|
|
||||||
func (q *delayedStarter) setInternal(atShutdown func(func()), handle HandlerFunc, exemplar interface{}) error { |
|
||||||
var ctx context.Context |
|
||||||
var cancel context.CancelFunc |
|
||||||
if q.timeout > 0 { |
|
||||||
ctx, cancel = context.WithTimeout(context.Background(), q.timeout) |
|
||||||
} else { |
|
||||||
ctx, cancel = context.WithCancel(context.Background()) |
|
||||||
} |
|
||||||
|
|
||||||
defer cancel() |
|
||||||
// Ensure we also stop at shutdown
|
|
||||||
atShutdown(cancel) |
|
||||||
|
|
||||||
i := 1 |
|
||||||
for q.internal == nil { |
|
||||||
select { |
|
||||||
case <-ctx.Done(): |
|
||||||
cfg := q.cfg |
|
||||||
if s, ok := cfg.([]byte); ok { |
|
||||||
cfg = string(s) |
|
||||||
} |
|
||||||
return fmt.Errorf("timedout creating queue %v with cfg %#v in %s", q.underlying, cfg, q.name) |
|
||||||
default: |
|
||||||
queue, err := NewQueue(q.underlying, handle, q.cfg, exemplar) |
|
||||||
if err == nil { |
|
||||||
q.internal = queue |
|
||||||
break |
|
||||||
} |
|
||||||
if err.Error() != "resource temporarily unavailable" { |
|
||||||
if bs, ok := q.cfg.([]byte); ok { |
|
||||||
log.Warn("[Attempt: %d] Failed to create queue: %v for %s cfg: %s error: %v", i, q.underlying, q.name, string(bs), err) |
|
||||||
} else { |
|
||||||
log.Warn("[Attempt: %d] Failed to create queue: %v for %s cfg: %#v error: %v", i, q.underlying, q.name, q.cfg, err) |
|
||||||
} |
|
||||||
} |
|
||||||
i++ |
|
||||||
if q.maxAttempts > 0 && i > q.maxAttempts { |
|
||||||
if bs, ok := q.cfg.([]byte); ok { |
|
||||||
return fmt.Errorf("unable to create queue %v for %s with cfg %s by max attempts: error: %w", q.underlying, q.name, string(bs), err) |
|
||||||
} |
|
||||||
return fmt.Errorf("unable to create queue %v for %s with cfg %#v by max attempts: error: %w", q.underlying, q.name, q.cfg, err) |
|
||||||
} |
|
||||||
sleepTime := 100 * time.Millisecond |
|
||||||
if q.timeout > 0 && q.maxAttempts > 0 { |
|
||||||
sleepTime = (q.timeout - 200*time.Millisecond) / time.Duration(q.maxAttempts) |
|
||||||
} |
|
||||||
t := time.NewTimer(sleepTime) |
|
||||||
select { |
|
||||||
case <-ctx.Done(): |
|
||||||
util.StopTimer(t) |
|
||||||
case <-t.C: |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// WrappedQueue wraps a delayed starting queue
|
|
||||||
type WrappedQueue struct { |
|
||||||
delayedStarter |
|
||||||
lock sync.Mutex |
|
||||||
handle HandlerFunc |
|
||||||
exemplar interface{} |
|
||||||
channel chan Data |
|
||||||
numInQueue int64 |
|
||||||
} |
|
||||||
|
|
||||||
// NewWrappedQueue will attempt to create a queue of the provided type,
|
|
||||||
// but if there is a problem creating this queue it will instead create
|
|
||||||
// a WrappedQueue with delayed startup of the queue instead and a
|
|
||||||
// channel which will be redirected to the queue
|
|
||||||
func NewWrappedQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(WrappedQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(WrappedQueueConfiguration) |
|
||||||
|
|
||||||
queue, err := NewQueue(config.Underlying, handle, config.Config, exemplar) |
|
||||||
if err == nil { |
|
||||||
// Just return the queue there is no need to wrap
|
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
if IsErrInvalidConfiguration(err) { |
|
||||||
// Retrying ain't gonna make this any better...
|
|
||||||
return nil, ErrInvalidConfiguration{cfg: cfg} |
|
||||||
} |
|
||||||
|
|
||||||
queue = &WrappedQueue{ |
|
||||||
handle: handle, |
|
||||||
channel: make(chan Data, config.QueueLength), |
|
||||||
exemplar: exemplar, |
|
||||||
delayedStarter: delayedStarter{ |
|
||||||
cfg: config.Config, |
|
||||||
underlying: config.Underlying, |
|
||||||
timeout: config.Timeout, |
|
||||||
maxAttempts: config.MaxAttempts, |
|
||||||
name: config.Name, |
|
||||||
}, |
|
||||||
} |
|
||||||
_ = GetManager().Add(queue, WrappedQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Name returns the name of the queue
|
|
||||||
func (q *WrappedQueue) Name() string { |
|
||||||
return q.name + "-wrapper" |
|
||||||
} |
|
||||||
|
|
||||||
// Push will push the data to the internal channel checking it against the exemplar
|
|
||||||
func (q *WrappedQueue) Push(data Data) error { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
atomic.AddInt64(&q.numInQueue, 1) |
|
||||||
q.channel <- data |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
func (q *WrappedQueue) flushInternalWithContext(ctx context.Context) error { |
|
||||||
q.lock.Lock() |
|
||||||
if q.internal == nil { |
|
||||||
q.lock.Unlock() |
|
||||||
return fmt.Errorf("not ready to flush wrapped queue %s yet", q.Name()) |
|
||||||
} |
|
||||||
q.lock.Unlock() |
|
||||||
select { |
|
||||||
case <-ctx.Done(): |
|
||||||
return ctx.Err() |
|
||||||
default: |
|
||||||
} |
|
||||||
return q.internal.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the queue and blocks till the queue is empty
|
|
||||||
func (q *WrappedQueue) Flush(timeout time.Duration) error { |
|
||||||
var ctx context.Context |
|
||||||
var cancel context.CancelFunc |
|
||||||
if timeout > 0 { |
|
||||||
ctx, cancel = context.WithTimeout(context.Background(), timeout) |
|
||||||
} else { |
|
||||||
ctx, cancel = context.WithCancel(context.Background()) |
|
||||||
} |
|
||||||
defer cancel() |
|
||||||
return q.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// FlushWithContext implements the final part of Flushable
|
|
||||||
func (q *WrappedQueue) FlushWithContext(ctx context.Context) error { |
|
||||||
log.Trace("WrappedQueue: %s FlushWithContext", q.Name()) |
|
||||||
errChan := make(chan error, 1) |
|
||||||
go func() { |
|
||||||
errChan <- q.flushInternalWithContext(ctx) |
|
||||||
close(errChan) |
|
||||||
}() |
|
||||||
|
|
||||||
select { |
|
||||||
case err := <-errChan: |
|
||||||
return err |
|
||||||
case <-ctx.Done(): |
|
||||||
go func() { |
|
||||||
<-errChan |
|
||||||
}() |
|
||||||
return ctx.Err() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty checks whether the queue is empty
|
|
||||||
func (q *WrappedQueue) IsEmpty() bool { |
|
||||||
if atomic.LoadInt64(&q.numInQueue) != 0 { |
|
||||||
return false |
|
||||||
} |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return false |
|
||||||
} |
|
||||||
return q.internal.IsEmpty() |
|
||||||
} |
|
||||||
|
|
||||||
// Run starts to run the queue and attempts to create the internal queue
|
|
||||||
func (q *WrappedQueue) Run(atShutdown, atTerminate func(func())) { |
|
||||||
log.Debug("WrappedQueue: %s Starting", q.name) |
|
||||||
q.lock.Lock() |
|
||||||
if q.internal == nil { |
|
||||||
err := q.setInternal(atShutdown, q.handle, q.exemplar) |
|
||||||
q.lock.Unlock() |
|
||||||
if err != nil { |
|
||||||
log.Fatal("Unable to set the internal queue for %s Error: %v", q.Name(), err) |
|
||||||
return |
|
||||||
} |
|
||||||
go func() { |
|
||||||
for data := range q.channel { |
|
||||||
_ = q.internal.Push(data) |
|
||||||
atomic.AddInt64(&q.numInQueue, -1) |
|
||||||
} |
|
||||||
}() |
|
||||||
} else { |
|
||||||
q.lock.Unlock() |
|
||||||
} |
|
||||||
|
|
||||||
q.internal.Run(atShutdown, atTerminate) |
|
||||||
log.Trace("WrappedQueue: %s Done", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Shutdown this queue and stop processing
|
|
||||||
func (q *WrappedQueue) Shutdown() { |
|
||||||
log.Trace("WrappedQueue: %s Shutting down", q.name) |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return |
|
||||||
} |
|
||||||
if shutdownable, ok := q.internal.(Shutdownable); ok { |
|
||||||
shutdownable.Shutdown() |
|
||||||
} |
|
||||||
log.Debug("WrappedQueue: %s Shutdown", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Terminate this queue and close the queue
|
|
||||||
func (q *WrappedQueue) Terminate() { |
|
||||||
log.Trace("WrappedQueue: %s Terminating", q.name) |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return |
|
||||||
} |
|
||||||
if shutdownable, ok := q.internal.(Shutdownable); ok { |
|
||||||
shutdownable.Terminate() |
|
||||||
} |
|
||||||
log.Debug("WrappedQueue: %s Terminated", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// IsPaused will return if the pool or queue is paused
|
|
||||||
func (q *WrappedQueue) IsPaused() bool { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
pausable, ok := q.internal.(Pausable) |
|
||||||
return ok && pausable.IsPaused() |
|
||||||
} |
|
||||||
|
|
||||||
// Pause will pause the pool or queue
|
|
||||||
func (q *WrappedQueue) Pause() { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if pausable, ok := q.internal.(Pausable); ok { |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Resume will resume the pool or queue
|
|
||||||
func (q *WrappedQueue) Resume() { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if pausable, ok := q.internal.(Pausable); ok { |
|
||||||
pausable.Resume() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// IsPausedIsResumed will return a bool indicating if the pool or queue is paused and a channel that will be closed when it is resumed
|
|
||||||
func (q *WrappedQueue) IsPausedIsResumed() (paused, resumed <-chan struct{}) { |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if pausable, ok := q.internal.(Pausable); ok { |
|
||||||
return pausable.IsPausedIsResumed() |
|
||||||
} |
|
||||||
return context.Background().Done(), closedChan |
|
||||||
} |
|
||||||
|
|
||||||
var closedChan chan struct{} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[WrappedQueueType] = NewWrappedQueue |
|
||||||
closedChan = make(chan struct{}) |
|
||||||
close(closedChan) |
|
||||||
} |
|
@ -1,126 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"fmt" |
|
||||||
"strings" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/json" |
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
"code.gitea.io/gitea/modules/setting" |
|
||||||
) |
|
||||||
|
|
||||||
func validType(t string) (Type, error) { |
|
||||||
if len(t) == 0 { |
|
||||||
return PersistableChannelQueueType, nil |
|
||||||
} |
|
||||||
for _, typ := range RegisteredTypes() { |
|
||||||
if t == string(typ) { |
|
||||||
return typ, nil |
|
||||||
} |
|
||||||
} |
|
||||||
return PersistableChannelQueueType, fmt.Errorf("unknown queue type: %s defaulting to %s", t, string(PersistableChannelQueueType)) |
|
||||||
} |
|
||||||
|
|
||||||
func getQueueSettings(name string) (setting.QueueSettings, []byte) { |
|
||||||
q := setting.GetQueueSettings(name) |
|
||||||
cfg, err := json.Marshal(q) |
|
||||||
if err != nil { |
|
||||||
log.Error("Unable to marshall generic options: %v Error: %v", q, err) |
|
||||||
log.Error("Unable to create queue for %s", name, err) |
|
||||||
return q, []byte{} |
|
||||||
} |
|
||||||
return q, cfg |
|
||||||
} |
|
||||||
|
|
||||||
// CreateQueue for name with provided handler and exemplar
|
|
||||||
func CreateQueue(name string, handle HandlerFunc, exemplar interface{}) Queue { |
|
||||||
q, cfg := getQueueSettings(name) |
|
||||||
if len(cfg) == 0 { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
typ, err := validType(q.Type) |
|
||||||
if err != nil { |
|
||||||
log.Error("Invalid type %s provided for queue named %s defaulting to %s", q.Type, name, string(typ)) |
|
||||||
} |
|
||||||
|
|
||||||
returnable, err := NewQueue(typ, handle, cfg, exemplar) |
|
||||||
if q.WrapIfNecessary && err != nil { |
|
||||||
log.Warn("Unable to create queue for %s: %v", name, err) |
|
||||||
log.Warn("Attempting to create wrapped queue") |
|
||||||
returnable, err = NewQueue(WrappedQueueType, handle, WrappedQueueConfiguration{ |
|
||||||
Underlying: typ, |
|
||||||
Timeout: q.Timeout, |
|
||||||
MaxAttempts: q.MaxAttempts, |
|
||||||
Config: cfg, |
|
||||||
QueueLength: q.QueueLength, |
|
||||||
Name: name, |
|
||||||
}, exemplar) |
|
||||||
} |
|
||||||
if err != nil { |
|
||||||
log.Error("Unable to create queue for %s: %v", name, err) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Sanity check configuration
|
|
||||||
if q.Workers == 0 && (q.BoostTimeout == 0 || q.BoostWorkers == 0 || q.MaxWorkers == 0) { |
|
||||||
log.Warn("Queue: %s is configured to be non-scaling and have no workers\n - this configuration is likely incorrect and could cause Gitea to block", q.Name) |
|
||||||
if pausable, ok := returnable.(Pausable); ok { |
|
||||||
log.Warn("Queue: %s is being paused to prevent data-loss, add workers manually and unpause.", q.Name) |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
return returnable |
|
||||||
} |
|
||||||
|
|
||||||
// CreateUniqueQueue for name with provided handler and exemplar
|
|
||||||
func CreateUniqueQueue(name string, handle HandlerFunc, exemplar interface{}) UniqueQueue { |
|
||||||
q, cfg := getQueueSettings(name) |
|
||||||
if len(cfg) == 0 { |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
if len(q.Type) > 0 && q.Type != "dummy" && q.Type != "immediate" && !strings.HasPrefix(q.Type, "unique-") { |
|
||||||
q.Type = "unique-" + q.Type |
|
||||||
} |
|
||||||
|
|
||||||
typ, err := validType(q.Type) |
|
||||||
if err != nil || typ == PersistableChannelQueueType { |
|
||||||
typ = PersistableChannelUniqueQueueType |
|
||||||
if err != nil { |
|
||||||
log.Error("Invalid type %s provided for queue named %s defaulting to %s", q.Type, name, string(typ)) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
returnable, err := NewQueue(typ, handle, cfg, exemplar) |
|
||||||
if q.WrapIfNecessary && err != nil { |
|
||||||
log.Warn("Unable to create unique queue for %s: %v", name, err) |
|
||||||
log.Warn("Attempting to create wrapped queue") |
|
||||||
returnable, err = NewQueue(WrappedUniqueQueueType, handle, WrappedUniqueQueueConfiguration{ |
|
||||||
Underlying: typ, |
|
||||||
Timeout: q.Timeout, |
|
||||||
MaxAttempts: q.MaxAttempts, |
|
||||||
Config: cfg, |
|
||||||
QueueLength: q.QueueLength, |
|
||||||
}, exemplar) |
|
||||||
} |
|
||||||
if err != nil { |
|
||||||
log.Error("Unable to create unique queue for %s: %v", name, err) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Sanity check configuration
|
|
||||||
if q.Workers == 0 && (q.BoostTimeout == 0 || q.BoostWorkers == 0 || q.MaxWorkers == 0) { |
|
||||||
log.Warn("Queue: %s is configured to be non-scaling and have no workers\n - this configuration is likely incorrect and could cause Gitea to block", q.Name) |
|
||||||
if pausable, ok := returnable.(Pausable); ok { |
|
||||||
log.Warn("Queue: %s is being paused to prevent data-loss, add workers manually and unpause.", q.Name) |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
return returnable.(UniqueQueue) |
|
||||||
} |
|
@ -0,0 +1,40 @@ |
|||||||
|
// Copyright 2019 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"fmt" |
||||||
|
"sync" |
||||||
|
) |
||||||
|
|
||||||
|
// testStateRecorder is used to record state changes for testing, to help debug async behaviors
|
||||||
|
type testStateRecorder struct { |
||||||
|
records []string |
||||||
|
mu sync.Mutex |
||||||
|
} |
||||||
|
|
||||||
|
var testRecorder = &testStateRecorder{} |
||||||
|
|
||||||
|
func (t *testStateRecorder) Record(format string, args ...any) { |
||||||
|
t.mu.Lock() |
||||||
|
t.records = append(t.records, fmt.Sprintf(format, args...)) |
||||||
|
if len(t.records) > 1000 { |
||||||
|
t.records = t.records[len(t.records)-1000:] |
||||||
|
} |
||||||
|
t.mu.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
func (t *testStateRecorder) Records() []string { |
||||||
|
t.mu.Lock() |
||||||
|
r := make([]string, len(t.records)) |
||||||
|
copy(r, t.records) |
||||||
|
t.mu.Unlock() |
||||||
|
return r |
||||||
|
} |
||||||
|
|
||||||
|
func (t *testStateRecorder) Reset() { |
||||||
|
t.mu.Lock() |
||||||
|
t.records = nil |
||||||
|
t.mu.Unlock() |
||||||
|
} |
@ -1,28 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"fmt" |
|
||||||
) |
|
||||||
|
|
||||||
// UniqueQueue defines a queue which guarantees only one instance of same
|
|
||||||
// data is in the queue. Instances with same identity will be
|
|
||||||
// discarded if there is already one in the line.
|
|
||||||
//
|
|
||||||
// This queue is particularly useful for preventing duplicated task
|
|
||||||
// of same purpose - please note that this does not guarantee that a particular
|
|
||||||
// task cannot be processed twice or more at the same time. Uniqueness is
|
|
||||||
// only guaranteed whilst the task is waiting in the queue.
|
|
||||||
//
|
|
||||||
// Users of this queue should be careful to push only the identifier of the
|
|
||||||
// data
|
|
||||||
type UniqueQueue interface { |
|
||||||
Queue |
|
||||||
PushFunc(Data, func() error) error |
|
||||||
Has(Data) (bool, error) |
|
||||||
} |
|
||||||
|
|
||||||
// ErrAlreadyInQueue is returned when trying to push data to the queue that is already in the queue
|
|
||||||
var ErrAlreadyInQueue = fmt.Errorf("already in queue") |
|
@ -1,212 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"runtime/pprof" |
|
||||||
"sync" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/container" |
|
||||||
"code.gitea.io/gitea/modules/json" |
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
) |
|
||||||
|
|
||||||
// ChannelUniqueQueueType is the type for channel queue
|
|
||||||
const ChannelUniqueQueueType Type = "unique-channel" |
|
||||||
|
|
||||||
// ChannelUniqueQueueConfiguration is the configuration for a ChannelUniqueQueue
|
|
||||||
type ChannelUniqueQueueConfiguration ChannelQueueConfiguration |
|
||||||
|
|
||||||
// ChannelUniqueQueue implements UniqueQueue
|
|
||||||
//
|
|
||||||
// It is basically a thin wrapper around a WorkerPool but keeps a store of
|
|
||||||
// what has been pushed within a table.
|
|
||||||
//
|
|
||||||
// Please note that this Queue does not guarantee that a particular
|
|
||||||
// task cannot be processed twice or more at the same time. Uniqueness is
|
|
||||||
// only guaranteed whilst the task is waiting in the queue.
|
|
||||||
type ChannelUniqueQueue struct { |
|
||||||
*WorkerPool |
|
||||||
lock sync.Mutex |
|
||||||
table container.Set[string] |
|
||||||
shutdownCtx context.Context |
|
||||||
shutdownCtxCancel context.CancelFunc |
|
||||||
terminateCtx context.Context |
|
||||||
terminateCtxCancel context.CancelFunc |
|
||||||
exemplar interface{} |
|
||||||
workers int |
|
||||||
name string |
|
||||||
} |
|
||||||
|
|
||||||
// NewChannelUniqueQueue create a memory channel queue
|
|
||||||
func NewChannelUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(ChannelUniqueQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(ChannelUniqueQueueConfiguration) |
|
||||||
if config.BatchLength == 0 { |
|
||||||
config.BatchLength = 1 |
|
||||||
} |
|
||||||
|
|
||||||
terminateCtx, terminateCtxCancel := context.WithCancel(context.Background()) |
|
||||||
shutdownCtx, shutdownCtxCancel := context.WithCancel(terminateCtx) |
|
||||||
|
|
||||||
queue := &ChannelUniqueQueue{ |
|
||||||
table: make(container.Set[string]), |
|
||||||
shutdownCtx: shutdownCtx, |
|
||||||
shutdownCtxCancel: shutdownCtxCancel, |
|
||||||
terminateCtx: terminateCtx, |
|
||||||
terminateCtxCancel: terminateCtxCancel, |
|
||||||
exemplar: exemplar, |
|
||||||
workers: config.Workers, |
|
||||||
name: config.Name, |
|
||||||
} |
|
||||||
queue.WorkerPool = NewWorkerPool(func(data ...Data) (unhandled []Data) { |
|
||||||
for _, datum := range data { |
|
||||||
// No error is possible here because PushFunc ensures that this can be marshalled
|
|
||||||
bs, _ := json.Marshal(datum) |
|
||||||
|
|
||||||
queue.lock.Lock() |
|
||||||
queue.table.Remove(string(bs)) |
|
||||||
queue.lock.Unlock() |
|
||||||
|
|
||||||
if u := handle(datum); u != nil { |
|
||||||
if queue.IsPaused() { |
|
||||||
// We can only pushback to the channel if we're paused.
|
|
||||||
go func() { |
|
||||||
if err := queue.Push(u[0]); err != nil { |
|
||||||
log.Error("Unable to push back to queue %d. Error: %v", queue.qid, err) |
|
||||||
} |
|
||||||
}() |
|
||||||
} else { |
|
||||||
unhandled = append(unhandled, u...) |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
return unhandled |
|
||||||
}, config.WorkerPoolConfiguration) |
|
||||||
|
|
||||||
queue.qid = GetManager().Add(queue, ChannelUniqueQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Run starts to run the queue
|
|
||||||
func (q *ChannelUniqueQueue) Run(atShutdown, atTerminate func(func())) { |
|
||||||
pprof.SetGoroutineLabels(q.baseCtx) |
|
||||||
atShutdown(q.Shutdown) |
|
||||||
atTerminate(q.Terminate) |
|
||||||
log.Debug("ChannelUniqueQueue: %s Starting", q.name) |
|
||||||
_ = q.AddWorkers(q.workers, 0) |
|
||||||
} |
|
||||||
|
|
||||||
// Push will push data into the queue if the data is not already in the queue
|
|
||||||
func (q *ChannelUniqueQueue) Push(data Data) error { |
|
||||||
return q.PushFunc(data, nil) |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc will push data into the queue
|
|
||||||
func (q *ChannelUniqueQueue) PushFunc(data Data, fn func() error) error { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in queue: %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
|
|
||||||
bs, err := json.Marshal(data) |
|
||||||
if err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
q.lock.Lock() |
|
||||||
locked := true |
|
||||||
defer func() { |
|
||||||
if locked { |
|
||||||
q.lock.Unlock() |
|
||||||
} |
|
||||||
}() |
|
||||||
if !q.table.Add(string(bs)) { |
|
||||||
return ErrAlreadyInQueue |
|
||||||
} |
|
||||||
// FIXME: We probably need to implement some sort of limit here
|
|
||||||
// If the downstream queue blocks this table will grow without limit
|
|
||||||
if fn != nil { |
|
||||||
err := fn() |
|
||||||
if err != nil { |
|
||||||
q.table.Remove(string(bs)) |
|
||||||
return err |
|
||||||
} |
|
||||||
} |
|
||||||
locked = false |
|
||||||
q.lock.Unlock() |
|
||||||
q.WorkerPool.Push(data) |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Has checks if the data is in the queue
|
|
||||||
func (q *ChannelUniqueQueue) Has(data Data) (bool, error) { |
|
||||||
bs, err := json.Marshal(data) |
|
||||||
if err != nil { |
|
||||||
return false, err |
|
||||||
} |
|
||||||
|
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
return q.table.Contains(string(bs)), nil |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the channel with a timeout - the Flush worker will be registered as a flush worker with the manager
|
|
||||||
func (q *ChannelUniqueQueue) Flush(timeout time.Duration) error { |
|
||||||
if q.IsPaused() { |
|
||||||
return nil |
|
||||||
} |
|
||||||
ctx, cancel := q.commonRegisterWorkers(1, timeout, true) |
|
||||||
defer cancel() |
|
||||||
return q.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// Shutdown processing from this queue
|
|
||||||
func (q *ChannelUniqueQueue) Shutdown() { |
|
||||||
log.Trace("ChannelUniqueQueue: %s Shutting down", q.name) |
|
||||||
select { |
|
||||||
case <-q.shutdownCtx.Done(): |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
go func() { |
|
||||||
log.Trace("ChannelUniqueQueue: %s Flushing", q.name) |
|
||||||
if err := q.FlushWithContext(q.terminateCtx); err != nil { |
|
||||||
if !q.IsEmpty() { |
|
||||||
log.Warn("ChannelUniqueQueue: %s Terminated before completed flushing", q.name) |
|
||||||
} |
|
||||||
return |
|
||||||
} |
|
||||||
log.Debug("ChannelUniqueQueue: %s Flushed", q.name) |
|
||||||
}() |
|
||||||
q.shutdownCtxCancel() |
|
||||||
log.Debug("ChannelUniqueQueue: %s Shutdown", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Terminate this queue and close the queue
|
|
||||||
func (q *ChannelUniqueQueue) Terminate() { |
|
||||||
log.Trace("ChannelUniqueQueue: %s Terminating", q.name) |
|
||||||
q.Shutdown() |
|
||||||
select { |
|
||||||
case <-q.terminateCtx.Done(): |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
q.terminateCtxCancel() |
|
||||||
q.baseCtxFinished() |
|
||||||
log.Debug("ChannelUniqueQueue: %s Terminated", q.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Name returns the name of this queue
|
|
||||||
func (q *ChannelUniqueQueue) Name() string { |
|
||||||
return q.name |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[ChannelUniqueQueueType] = NewChannelUniqueQueue |
|
||||||
} |
|
@ -1,258 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"sync" |
|
||||||
"testing" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert" |
|
||||||
) |
|
||||||
|
|
||||||
func TestChannelUniqueQueue(t *testing.T) { |
|
||||||
_ = log.NewLogger(1000, "console", "console", `{"level":"warn","stacktracelevel":"NONE","stderr":true}`) |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
nilFn := func(_ func()) {} |
|
||||||
|
|
||||||
queue, err := NewChannelUniqueQueue(handle, |
|
||||||
ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 5, |
|
||||||
Name: "TestChannelQueue", |
|
||||||
}, |
|
||||||
Workers: 0, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
assert.Equal(t, queue.(*ChannelUniqueQueue).WorkerPool.boostWorkers, 5) |
|
||||||
|
|
||||||
go queue.Run(nilFn, nilFn) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
go queue.Push(&test1) |
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
err = queue.Push(test1) |
|
||||||
assert.Error(t, err) |
|
||||||
} |
|
||||||
|
|
||||||
func TestChannelUniqueQueue_Batch(t *testing.T) { |
|
||||||
_ = log.NewLogger(1000, "console", "console", `{"level":"warn","stacktracelevel":"NONE","stderr":true}`) |
|
||||||
|
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
nilFn := func(_ func()) {} |
|
||||||
|
|
||||||
queue, err := NewChannelUniqueQueue(handle, |
|
||||||
ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 20, |
|
||||||
BatchLength: 2, |
|
||||||
BlockTimeout: 0, |
|
||||||
BoostTimeout: 0, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
}, |
|
||||||
Workers: 1, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go queue.Run(nilFn, nilFn) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
|
|
||||||
queue.Push(&test1) |
|
||||||
go queue.Push(&test2) |
|
||||||
|
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
result2 := <-handleChan |
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
err = queue.Push(test1) |
|
||||||
assert.Error(t, err) |
|
||||||
} |
|
||||||
|
|
||||||
func TestChannelUniqueQueue_Pause(t *testing.T) { |
|
||||||
_ = log.NewLogger(1000, "console", "console", `{"level":"warn","stacktracelevel":"NONE","stderr":true}`) |
|
||||||
|
|
||||||
lock := sync.Mutex{} |
|
||||||
var queue Queue |
|
||||||
var err error |
|
||||||
pushBack := false |
|
||||||
handleChan := make(chan *testData) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
lock.Lock() |
|
||||||
if pushBack { |
|
||||||
if pausable, ok := queue.(Pausable); ok { |
|
||||||
pausable.Pause() |
|
||||||
} |
|
||||||
pushBack = false |
|
||||||
lock.Unlock() |
|
||||||
return data |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
for _, datum := range data { |
|
||||||
testDatum := datum.(*testData) |
|
||||||
handleChan <- testDatum |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
nilFn := func(_ func()) {} |
|
||||||
|
|
||||||
queue, err = NewChannelUniqueQueue(handle, |
|
||||||
ChannelQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: 20, |
|
||||||
BatchLength: 1, |
|
||||||
BlockTimeout: 0, |
|
||||||
BoostTimeout: 0, |
|
||||||
BoostWorkers: 0, |
|
||||||
MaxWorkers: 10, |
|
||||||
}, |
|
||||||
Workers: 1, |
|
||||||
}, &testData{}) |
|
||||||
assert.NoError(t, err) |
|
||||||
|
|
||||||
go queue.Run(nilFn, nilFn) |
|
||||||
|
|
||||||
test1 := testData{"A", 1} |
|
||||||
test2 := testData{"B", 2} |
|
||||||
queue.Push(&test1) |
|
||||||
|
|
||||||
pausable, ok := queue.(Pausable) |
|
||||||
if !assert.True(t, ok) { |
|
||||||
return |
|
||||||
} |
|
||||||
result1 := <-handleChan |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
|
|
||||||
pausable.Pause() |
|
||||||
|
|
||||||
paused, resumed := pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-resumed: |
|
||||||
assert.Fail(t, "Queue should not be resumed") |
|
||||||
return |
|
||||||
default: |
|
||||||
assert.Fail(t, "Queue is not paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
queue.Push(&test2) |
|
||||||
|
|
||||||
var result2 *testData |
|
||||||
select { |
|
||||||
case result2 = <-handleChan: |
|
||||||
assert.Fail(t, "handler chan should be empty") |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
} |
|
||||||
|
|
||||||
assert.Nil(t, result2) |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
default: |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case result2 = <-handleChan: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "handler chan should contain test2") |
|
||||||
} |
|
||||||
|
|
||||||
assert.Equal(t, test2.TestString, result2.TestString) |
|
||||||
assert.Equal(t, test2.TestInt, result2.TestInt) |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
pushBack = true |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
paused, resumed = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
assert.Fail(t, "Queue should not be paused") |
|
||||||
return |
|
||||||
case <-resumed: |
|
||||||
default: |
|
||||||
assert.Fail(t, "Queue is not resumed") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
queue.Push(&test1) |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-handleChan: |
|
||||||
assert.Fail(t, "handler chan should not contain test1") |
|
||||||
return |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "queue should be paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
paused, resumed = pausable.IsPausedIsResumed() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
case <-resumed: |
|
||||||
assert.Fail(t, "Queue should not be resumed") |
|
||||||
return |
|
||||||
default: |
|
||||||
assert.Fail(t, "Queue is not paused") |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
pausable.Resume() |
|
||||||
|
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
default: |
|
||||||
assert.Fail(t, "Queue should be resumed") |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case result1 = <-handleChan: |
|
||||||
case <-time.After(500 * time.Millisecond): |
|
||||||
assert.Fail(t, "handler chan should contain test1") |
|
||||||
} |
|
||||||
assert.Equal(t, test1.TestString, result1.TestString) |
|
||||||
assert.Equal(t, test1.TestInt, result1.TestInt) |
|
||||||
} |
|
@ -1,128 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/nosql" |
|
||||||
|
|
||||||
"gitea.com/lunny/levelqueue" |
|
||||||
) |
|
||||||
|
|
||||||
// LevelUniqueQueueType is the type for level queue
|
|
||||||
const LevelUniqueQueueType Type = "unique-level" |
|
||||||
|
|
||||||
// LevelUniqueQueueConfiguration is the configuration for a LevelUniqueQueue
|
|
||||||
type LevelUniqueQueueConfiguration struct { |
|
||||||
ByteFIFOQueueConfiguration |
|
||||||
DataDir string |
|
||||||
ConnectionString string |
|
||||||
QueueName string |
|
||||||
} |
|
||||||
|
|
||||||
// LevelUniqueQueue implements a disk library queue
|
|
||||||
type LevelUniqueQueue struct { |
|
||||||
*ByteFIFOUniqueQueue |
|
||||||
} |
|
||||||
|
|
||||||
// NewLevelUniqueQueue creates a ledis local queue
|
|
||||||
//
|
|
||||||
// Please note that this Queue does not guarantee that a particular
|
|
||||||
// task cannot be processed twice or more at the same time. Uniqueness is
|
|
||||||
// only guaranteed whilst the task is waiting in the queue.
|
|
||||||
func NewLevelUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(LevelUniqueQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(LevelUniqueQueueConfiguration) |
|
||||||
|
|
||||||
if len(config.ConnectionString) == 0 { |
|
||||||
config.ConnectionString = config.DataDir |
|
||||||
} |
|
||||||
config.WaitOnEmpty = true |
|
||||||
|
|
||||||
byteFIFO, err := NewLevelUniqueQueueByteFIFO(config.ConnectionString, config.QueueName) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
byteFIFOQueue, err := NewByteFIFOUniqueQueue(LevelUniqueQueueType, byteFIFO, handle, config.ByteFIFOQueueConfiguration, exemplar) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
queue := &LevelUniqueQueue{ |
|
||||||
ByteFIFOUniqueQueue: byteFIFOQueue, |
|
||||||
} |
|
||||||
queue.qid = GetManager().Add(queue, LevelUniqueQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
var _ UniqueByteFIFO = &LevelUniqueQueueByteFIFO{} |
|
||||||
|
|
||||||
// LevelUniqueQueueByteFIFO represents a ByteFIFO formed from a LevelUniqueQueue
|
|
||||||
type LevelUniqueQueueByteFIFO struct { |
|
||||||
internal *levelqueue.UniqueQueue |
|
||||||
connection string |
|
||||||
} |
|
||||||
|
|
||||||
// NewLevelUniqueQueueByteFIFO creates a new ByteFIFO formed from a LevelUniqueQueue
|
|
||||||
func NewLevelUniqueQueueByteFIFO(connection, prefix string) (*LevelUniqueQueueByteFIFO, error) { |
|
||||||
db, err := nosql.GetManager().GetLevelDB(connection) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
internal, err := levelqueue.NewUniqueQueue(db, []byte(prefix), []byte(prefix+"-unique"), false) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
return &LevelUniqueQueueByteFIFO{ |
|
||||||
connection: connection, |
|
||||||
internal: internal, |
|
||||||
}, nil |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc pushes data to the end of the fifo and calls the callback if it is added
|
|
||||||
func (fifo *LevelUniqueQueueByteFIFO) PushFunc(ctx context.Context, data []byte, fn func() error) error { |
|
||||||
return fifo.internal.LPushFunc(data, fn) |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack pushes data to the top of the fifo
|
|
||||||
func (fifo *LevelUniqueQueueByteFIFO) PushBack(ctx context.Context, data []byte) error { |
|
||||||
return fifo.internal.RPush(data) |
|
||||||
} |
|
||||||
|
|
||||||
// Pop pops data from the start of the fifo
|
|
||||||
func (fifo *LevelUniqueQueueByteFIFO) Pop(ctx context.Context) ([]byte, error) { |
|
||||||
data, err := fifo.internal.RPop() |
|
||||||
if err != nil && err != levelqueue.ErrNotFound { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
return data, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Len returns the length of the fifo
|
|
||||||
func (fifo *LevelUniqueQueueByteFIFO) Len(ctx context.Context) int64 { |
|
||||||
return fifo.internal.Len() |
|
||||||
} |
|
||||||
|
|
||||||
// Has returns whether the fifo contains this data
|
|
||||||
func (fifo *LevelUniqueQueueByteFIFO) Has(ctx context.Context, data []byte) (bool, error) { |
|
||||||
return fifo.internal.Has(data) |
|
||||||
} |
|
||||||
|
|
||||||
// Close this fifo
|
|
||||||
func (fifo *LevelUniqueQueueByteFIFO) Close() error { |
|
||||||
err := fifo.internal.Close() |
|
||||||
_ = nosql.GetManager().CloseLevelDB(fifo.connection) |
|
||||||
return err |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[LevelUniqueQueueType] = NewLevelUniqueQueue |
|
||||||
} |
|
@ -1,336 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"runtime/pprof" |
|
||||||
"sync" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
) |
|
||||||
|
|
||||||
// PersistableChannelUniqueQueueType is the type for persistable queue
|
|
||||||
const PersistableChannelUniqueQueueType Type = "unique-persistable-channel" |
|
||||||
|
|
||||||
// PersistableChannelUniqueQueueConfiguration is the configuration for a PersistableChannelUniqueQueue
|
|
||||||
type PersistableChannelUniqueQueueConfiguration struct { |
|
||||||
Name string |
|
||||||
DataDir string |
|
||||||
BatchLength int |
|
||||||
QueueLength int |
|
||||||
Timeout time.Duration |
|
||||||
MaxAttempts int |
|
||||||
Workers int |
|
||||||
MaxWorkers int |
|
||||||
BlockTimeout time.Duration |
|
||||||
BoostTimeout time.Duration |
|
||||||
BoostWorkers int |
|
||||||
} |
|
||||||
|
|
||||||
// PersistableChannelUniqueQueue wraps a channel queue and level queue together
|
|
||||||
//
|
|
||||||
// Please note that this Queue does not guarantee that a particular
|
|
||||||
// task cannot be processed twice or more at the same time. Uniqueness is
|
|
||||||
// only guaranteed whilst the task is waiting in the queue.
|
|
||||||
type PersistableChannelUniqueQueue struct { |
|
||||||
channelQueue *ChannelUniqueQueue |
|
||||||
delayedStarter |
|
||||||
lock sync.Mutex |
|
||||||
closed chan struct{} |
|
||||||
} |
|
||||||
|
|
||||||
// NewPersistableChannelUniqueQueue creates a wrapped batched channel queue with persistable level queue backend when shutting down
|
|
||||||
// This differs from a wrapped queue in that the persistent queue is only used to persist at shutdown/terminate
|
|
||||||
func NewPersistableChannelUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(PersistableChannelUniqueQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(PersistableChannelUniqueQueueConfiguration) |
|
||||||
|
|
||||||
queue := &PersistableChannelUniqueQueue{ |
|
||||||
closed: make(chan struct{}), |
|
||||||
} |
|
||||||
|
|
||||||
wrappedHandle := func(data ...Data) (failed []Data) { |
|
||||||
for _, unhandled := range handle(data...) { |
|
||||||
if fail := queue.PushBack(unhandled); fail != nil { |
|
||||||
failed = append(failed, fail) |
|
||||||
} |
|
||||||
} |
|
||||||
return failed |
|
||||||
} |
|
||||||
|
|
||||||
channelUniqueQueue, err := NewChannelUniqueQueue(wrappedHandle, ChannelUniqueQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: config.QueueLength, |
|
||||||
BatchLength: config.BatchLength, |
|
||||||
BlockTimeout: config.BlockTimeout, |
|
||||||
BoostTimeout: config.BoostTimeout, |
|
||||||
BoostWorkers: config.BoostWorkers, |
|
||||||
MaxWorkers: config.MaxWorkers, |
|
||||||
Name: config.Name + "-channel", |
|
||||||
}, |
|
||||||
Workers: config.Workers, |
|
||||||
}, exemplar) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
// the level backend only needs temporary workers to catch up with the previously dropped work
|
|
||||||
levelCfg := LevelUniqueQueueConfiguration{ |
|
||||||
ByteFIFOQueueConfiguration: ByteFIFOQueueConfiguration{ |
|
||||||
WorkerPoolConfiguration: WorkerPoolConfiguration{ |
|
||||||
QueueLength: config.QueueLength, |
|
||||||
BatchLength: config.BatchLength, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 1, |
|
||||||
MaxWorkers: 5, |
|
||||||
Name: config.Name + "-level", |
|
||||||
}, |
|
||||||
Workers: 0, |
|
||||||
}, |
|
||||||
DataDir: config.DataDir, |
|
||||||
QueueName: config.Name + "-level", |
|
||||||
} |
|
||||||
|
|
||||||
queue.channelQueue = channelUniqueQueue.(*ChannelUniqueQueue) |
|
||||||
|
|
||||||
levelQueue, err := NewLevelUniqueQueue(func(data ...Data) []Data { |
|
||||||
for _, datum := range data { |
|
||||||
err := queue.Push(datum) |
|
||||||
if err != nil && err != ErrAlreadyInQueue { |
|
||||||
log.Error("Unable push to channelled queue: %v", err) |
|
||||||
} |
|
||||||
} |
|
||||||
return nil |
|
||||||
}, levelCfg, exemplar) |
|
||||||
if err == nil { |
|
||||||
queue.delayedStarter = delayedStarter{ |
|
||||||
internal: levelQueue.(*LevelUniqueQueue), |
|
||||||
name: config.Name, |
|
||||||
} |
|
||||||
|
|
||||||
_ = GetManager().Add(queue, PersistableChannelUniqueQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
if IsErrInvalidConfiguration(err) { |
|
||||||
// Retrying ain't gonna make this any better...
|
|
||||||
return nil, ErrInvalidConfiguration{cfg: cfg} |
|
||||||
} |
|
||||||
|
|
||||||
queue.delayedStarter = delayedStarter{ |
|
||||||
cfg: levelCfg, |
|
||||||
underlying: LevelUniqueQueueType, |
|
||||||
timeout: config.Timeout, |
|
||||||
maxAttempts: config.MaxAttempts, |
|
||||||
name: config.Name, |
|
||||||
} |
|
||||||
_ = GetManager().Add(queue, PersistableChannelUniqueQueueType, config, exemplar) |
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Name returns the name of this queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Name() string { |
|
||||||
return q.delayedStarter.name |
|
||||||
} |
|
||||||
|
|
||||||
// Push will push the indexer data to queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Push(data Data) error { |
|
||||||
return q.PushFunc(data, nil) |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc will push the indexer data to queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) PushFunc(data Data, fn func() error) error { |
|
||||||
select { |
|
||||||
case <-q.closed: |
|
||||||
return q.internal.(UniqueQueue).PushFunc(data, fn) |
|
||||||
default: |
|
||||||
return q.channelQueue.PushFunc(data, fn) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack will push the indexer data to queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) PushBack(data Data) error { |
|
||||||
select { |
|
||||||
case <-q.closed: |
|
||||||
if pbr, ok := q.internal.(PushBackable); ok { |
|
||||||
return pbr.PushBack(data) |
|
||||||
} |
|
||||||
return q.internal.Push(data) |
|
||||||
default: |
|
||||||
return q.channelQueue.Push(data) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Has will test if the queue has the data
|
|
||||||
func (q *PersistableChannelUniqueQueue) Has(data Data) (bool, error) { |
|
||||||
// This is more difficult...
|
|
||||||
has, err := q.channelQueue.Has(data) |
|
||||||
if err != nil || has { |
|
||||||
return has, err |
|
||||||
} |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal == nil { |
|
||||||
return false, nil |
|
||||||
} |
|
||||||
return q.internal.(UniqueQueue).Has(data) |
|
||||||
} |
|
||||||
|
|
||||||
// Run starts to run the queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Run(atShutdown, atTerminate func(func())) { |
|
||||||
pprof.SetGoroutineLabels(q.channelQueue.baseCtx) |
|
||||||
log.Debug("PersistableChannelUniqueQueue: %s Starting", q.delayedStarter.name) |
|
||||||
|
|
||||||
q.lock.Lock() |
|
||||||
if q.internal == nil { |
|
||||||
err := q.setInternal(atShutdown, func(data ...Data) []Data { |
|
||||||
for _, datum := range data { |
|
||||||
err := q.Push(datum) |
|
||||||
if err != nil && err != ErrAlreadyInQueue { |
|
||||||
log.Error("Unable push to channelled queue: %v", err) |
|
||||||
} |
|
||||||
} |
|
||||||
return nil |
|
||||||
}, q.channelQueue.exemplar) |
|
||||||
q.lock.Unlock() |
|
||||||
if err != nil { |
|
||||||
log.Fatal("Unable to create internal queue for %s Error: %v", q.Name(), err) |
|
||||||
return |
|
||||||
} |
|
||||||
} else { |
|
||||||
q.lock.Unlock() |
|
||||||
} |
|
||||||
atShutdown(q.Shutdown) |
|
||||||
atTerminate(q.Terminate) |
|
||||||
_ = q.channelQueue.AddWorkers(q.channelQueue.workers, 0) |
|
||||||
|
|
||||||
if luq, ok := q.internal.(*LevelUniqueQueue); ok && !luq.IsEmpty() { |
|
||||||
// Just run the level queue - we shut it down once it's flushed
|
|
||||||
go luq.Run(func(_ func()) {}, func(_ func()) {}) |
|
||||||
go func() { |
|
||||||
_ = luq.Flush(0) |
|
||||||
for !luq.IsEmpty() { |
|
||||||
_ = luq.Flush(0) |
|
||||||
select { |
|
||||||
case <-time.After(100 * time.Millisecond): |
|
||||||
case <-luq.shutdownCtx.Done(): |
|
||||||
if luq.byteFIFO.Len(luq.terminateCtx) > 0 { |
|
||||||
log.Warn("LevelUniqueQueue: %s shut down before completely flushed", luq.Name()) |
|
||||||
} |
|
||||||
return |
|
||||||
} |
|
||||||
} |
|
||||||
log.Debug("LevelUniqueQueue: %s flushed so shutting down", luq.Name()) |
|
||||||
luq.Shutdown() |
|
||||||
GetManager().Remove(luq.qid) |
|
||||||
}() |
|
||||||
} else { |
|
||||||
log.Debug("PersistableChannelUniqueQueue: %s Skipping running the empty level queue", q.delayedStarter.name) |
|
||||||
_ = q.internal.Flush(0) |
|
||||||
q.internal.(*LevelUniqueQueue).Shutdown() |
|
||||||
GetManager().Remove(q.internal.(*LevelUniqueQueue).qid) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Flush(timeout time.Duration) error { |
|
||||||
return q.channelQueue.Flush(timeout) |
|
||||||
} |
|
||||||
|
|
||||||
// FlushWithContext flushes the queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) FlushWithContext(ctx context.Context) error { |
|
||||||
return q.channelQueue.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty checks if a queue is empty
|
|
||||||
func (q *PersistableChannelUniqueQueue) IsEmpty() bool { |
|
||||||
return q.channelQueue.IsEmpty() |
|
||||||
} |
|
||||||
|
|
||||||
// IsPaused will return if the pool or queue is paused
|
|
||||||
func (q *PersistableChannelUniqueQueue) IsPaused() bool { |
|
||||||
return q.channelQueue.IsPaused() |
|
||||||
} |
|
||||||
|
|
||||||
// Pause will pause the pool or queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Pause() { |
|
||||||
q.channelQueue.Pause() |
|
||||||
} |
|
||||||
|
|
||||||
// Resume will resume the pool or queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Resume() { |
|
||||||
q.channelQueue.Resume() |
|
||||||
} |
|
||||||
|
|
||||||
// IsPausedIsResumed will return a bool indicating if the pool or queue is paused and a channel that will be closed when it is resumed
|
|
||||||
func (q *PersistableChannelUniqueQueue) IsPausedIsResumed() (paused, resumed <-chan struct{}) { |
|
||||||
return q.channelQueue.IsPausedIsResumed() |
|
||||||
} |
|
||||||
|
|
||||||
// Shutdown processing this queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Shutdown() { |
|
||||||
log.Trace("PersistableChannelUniqueQueue: %s Shutting down", q.delayedStarter.name) |
|
||||||
q.lock.Lock() |
|
||||||
select { |
|
||||||
case <-q.closed: |
|
||||||
q.lock.Unlock() |
|
||||||
return |
|
||||||
default: |
|
||||||
if q.internal != nil { |
|
||||||
q.internal.(*LevelUniqueQueue).Shutdown() |
|
||||||
} |
|
||||||
close(q.closed) |
|
||||||
q.lock.Unlock() |
|
||||||
} |
|
||||||
|
|
||||||
log.Trace("PersistableChannelUniqueQueue: %s Cancelling pools", q.delayedStarter.name) |
|
||||||
q.internal.(*LevelUniqueQueue).baseCtxCancel() |
|
||||||
q.channelQueue.baseCtxCancel() |
|
||||||
log.Trace("PersistableChannelUniqueQueue: %s Waiting til done", q.delayedStarter.name) |
|
||||||
q.channelQueue.Wait() |
|
||||||
q.internal.(*LevelUniqueQueue).Wait() |
|
||||||
// Redirect all remaining data in the chan to the internal channel
|
|
||||||
close(q.channelQueue.dataChan) |
|
||||||
log.Trace("PersistableChannelUniqueQueue: %s Redirecting remaining data", q.delayedStarter.name) |
|
||||||
countOK, countLost := 0, 0 |
|
||||||
for data := range q.channelQueue.dataChan { |
|
||||||
err := q.internal.(*LevelUniqueQueue).Push(data) |
|
||||||
if err != nil { |
|
||||||
log.Error("PersistableChannelUniqueQueue: %s Unable redirect %v due to: %v", q.delayedStarter.name, data, err) |
|
||||||
countLost++ |
|
||||||
} else { |
|
||||||
countOK++ |
|
||||||
} |
|
||||||
} |
|
||||||
if countLost > 0 { |
|
||||||
log.Warn("PersistableChannelUniqueQueue: %s %d will be restored on restart, %d lost", q.delayedStarter.name, countOK, countLost) |
|
||||||
} else if countOK > 0 { |
|
||||||
log.Warn("PersistableChannelUniqueQueue: %s %d will be restored on restart", q.delayedStarter.name, countOK) |
|
||||||
} |
|
||||||
log.Trace("PersistableChannelUniqueQueue: %s Done Redirecting remaining data", q.delayedStarter.name) |
|
||||||
|
|
||||||
log.Debug("PersistableChannelUniqueQueue: %s Shutdown", q.delayedStarter.name) |
|
||||||
} |
|
||||||
|
|
||||||
// Terminate this queue and close the queue
|
|
||||||
func (q *PersistableChannelUniqueQueue) Terminate() { |
|
||||||
log.Trace("PersistableChannelUniqueQueue: %s Terminating", q.delayedStarter.name) |
|
||||||
q.Shutdown() |
|
||||||
q.lock.Lock() |
|
||||||
defer q.lock.Unlock() |
|
||||||
if q.internal != nil { |
|
||||||
q.internal.(*LevelUniqueQueue).Terminate() |
|
||||||
} |
|
||||||
q.channelQueue.baseCtxFinished() |
|
||||||
log.Debug("PersistableChannelUniqueQueue: %s Terminated", q.delayedStarter.name) |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[PersistableChannelUniqueQueueType] = NewPersistableChannelUniqueQueue |
|
||||||
} |
|
@ -1,265 +0,0 @@ |
|||||||
// Copyright 2023 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"strconv" |
|
||||||
"sync" |
|
||||||
"sync/atomic" |
|
||||||
"testing" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert" |
|
||||||
) |
|
||||||
|
|
||||||
func TestPersistableChannelUniqueQueue(t *testing.T) { |
|
||||||
// Create a temporary directory for the queue
|
|
||||||
tmpDir := t.TempDir() |
|
||||||
_ = log.NewLogger(1000, "console", "console", `{"level":"warn","stacktracelevel":"NONE","stderr":true}`) |
|
||||||
|
|
||||||
// Common function to create the Queue
|
|
||||||
newQueue := func(name string, handle func(data ...Data) []Data) Queue { |
|
||||||
q, err := NewPersistableChannelUniqueQueue(handle, |
|
||||||
PersistableChannelUniqueQueueConfiguration{ |
|
||||||
Name: name, |
|
||||||
DataDir: tmpDir, |
|
||||||
QueueLength: 200, |
|
||||||
MaxWorkers: 1, |
|
||||||
BlockTimeout: 1 * time.Second, |
|
||||||
BoostTimeout: 5 * time.Minute, |
|
||||||
BoostWorkers: 1, |
|
||||||
Workers: 0, |
|
||||||
}, "task-0") |
|
||||||
assert.NoError(t, err) |
|
||||||
return q |
|
||||||
} |
|
||||||
|
|
||||||
// runs the provided queue and provides some timer function
|
|
||||||
type channels struct { |
|
||||||
readyForShutdown chan struct{} // closed when shutdown functions have been assigned
|
|
||||||
readyForTerminate chan struct{} // closed when terminate functions have been assigned
|
|
||||||
signalShutdown chan struct{} // Should close to signal shutdown
|
|
||||||
doneShutdown chan struct{} // closed when shutdown function is done
|
|
||||||
queueTerminate []func() // list of atTerminate functions to call atTerminate - need to be accessed with lock
|
|
||||||
} |
|
||||||
runQueue := func(q Queue, lock *sync.Mutex) *channels { |
|
||||||
chans := &channels{ |
|
||||||
readyForShutdown: make(chan struct{}), |
|
||||||
readyForTerminate: make(chan struct{}), |
|
||||||
signalShutdown: make(chan struct{}), |
|
||||||
doneShutdown: make(chan struct{}), |
|
||||||
} |
|
||||||
go q.Run(func(atShutdown func()) { |
|
||||||
go func() { |
|
||||||
lock.Lock() |
|
||||||
select { |
|
||||||
case <-chans.readyForShutdown: |
|
||||||
default: |
|
||||||
close(chans.readyForShutdown) |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
<-chans.signalShutdown |
|
||||||
atShutdown() |
|
||||||
close(chans.doneShutdown) |
|
||||||
}() |
|
||||||
}, func(atTerminate func()) { |
|
||||||
lock.Lock() |
|
||||||
defer lock.Unlock() |
|
||||||
select { |
|
||||||
case <-chans.readyForTerminate: |
|
||||||
default: |
|
||||||
close(chans.readyForTerminate) |
|
||||||
} |
|
||||||
chans.queueTerminate = append(chans.queueTerminate, atTerminate) |
|
||||||
}) |
|
||||||
|
|
||||||
return chans |
|
||||||
} |
|
||||||
|
|
||||||
// call to shutdown and terminate the queue associated with the channels
|
|
||||||
doTerminate := func(chans *channels, lock *sync.Mutex) { |
|
||||||
<-chans.readyForTerminate |
|
||||||
|
|
||||||
lock.Lock() |
|
||||||
callbacks := []func(){} |
|
||||||
callbacks = append(callbacks, chans.queueTerminate...) |
|
||||||
lock.Unlock() |
|
||||||
|
|
||||||
for _, callback := range callbacks { |
|
||||||
callback() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
mapLock := sync.Mutex{} |
|
||||||
executedInitial := map[string][]string{} |
|
||||||
hasInitial := map[string][]string{} |
|
||||||
|
|
||||||
fillQueue := func(name string, done chan int64) { |
|
||||||
t.Run("Initial Filling: "+name, func(t *testing.T) { |
|
||||||
lock := sync.Mutex{} |
|
||||||
|
|
||||||
startAt100Queued := make(chan struct{}) |
|
||||||
stopAt20Shutdown := make(chan struct{}) // stop and shutdown at the 20th item
|
|
||||||
|
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
<-startAt100Queued |
|
||||||
for _, datum := range data { |
|
||||||
s := datum.(string) |
|
||||||
mapLock.Lock() |
|
||||||
executedInitial[name] = append(executedInitial[name], s) |
|
||||||
mapLock.Unlock() |
|
||||||
if s == "task-20" { |
|
||||||
close(stopAt20Shutdown) |
|
||||||
} |
|
||||||
} |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
q := newQueue(name, handle) |
|
||||||
|
|
||||||
// add 100 tasks to the queue
|
|
||||||
for i := 0; i < 100; i++ { |
|
||||||
_ = q.Push("task-" + strconv.Itoa(i)) |
|
||||||
} |
|
||||||
close(startAt100Queued) |
|
||||||
|
|
||||||
chans := runQueue(q, &lock) |
|
||||||
|
|
||||||
<-chans.readyForShutdown |
|
||||||
<-stopAt20Shutdown |
|
||||||
close(chans.signalShutdown) |
|
||||||
<-chans.doneShutdown |
|
||||||
_ = q.Push("final") |
|
||||||
|
|
||||||
// check which tasks are still in the queue
|
|
||||||
for i := 0; i < 100; i++ { |
|
||||||
if has, _ := q.(UniqueQueue).Has("task-" + strconv.Itoa(i)); has { |
|
||||||
mapLock.Lock() |
|
||||||
hasInitial[name] = append(hasInitial[name], "task-"+strconv.Itoa(i)) |
|
||||||
mapLock.Unlock() |
|
||||||
} |
|
||||||
} |
|
||||||
if has, _ := q.(UniqueQueue).Has("final"); has { |
|
||||||
mapLock.Lock() |
|
||||||
hasInitial[name] = append(hasInitial[name], "final") |
|
||||||
mapLock.Unlock() |
|
||||||
} else { |
|
||||||
assert.Fail(t, "UnqueQueue %s should have \"final\"", name) |
|
||||||
} |
|
||||||
doTerminate(chans, &lock) |
|
||||||
mapLock.Lock() |
|
||||||
assert.Equal(t, 101, len(executedInitial[name])+len(hasInitial[name])) |
|
||||||
mapLock.Unlock() |
|
||||||
}) |
|
||||||
mapLock.Lock() |
|
||||||
count := int64(len(hasInitial[name])) |
|
||||||
mapLock.Unlock() |
|
||||||
done <- count |
|
||||||
close(done) |
|
||||||
} |
|
||||||
|
|
||||||
hasQueueAChan := make(chan int64) |
|
||||||
hasQueueBChan := make(chan int64) |
|
||||||
|
|
||||||
go fillQueue("QueueA", hasQueueAChan) |
|
||||||
go fillQueue("QueueB", hasQueueBChan) |
|
||||||
|
|
||||||
hasA := <-hasQueueAChan |
|
||||||
hasB := <-hasQueueBChan |
|
||||||
|
|
||||||
executedEmpty := map[string][]string{} |
|
||||||
hasEmpty := map[string][]string{} |
|
||||||
emptyQueue := func(name string, numInQueue int64, done chan struct{}) { |
|
||||||
t.Run("Empty Queue: "+name, func(t *testing.T) { |
|
||||||
lock := sync.Mutex{} |
|
||||||
stop := make(chan struct{}) |
|
||||||
|
|
||||||
// collect the tasks that have been executed
|
|
||||||
atomicCount := int64(0) |
|
||||||
handle := func(data ...Data) []Data { |
|
||||||
lock.Lock() |
|
||||||
for _, datum := range data { |
|
||||||
mapLock.Lock() |
|
||||||
executedEmpty[name] = append(executedEmpty[name], datum.(string)) |
|
||||||
mapLock.Unlock() |
|
||||||
count := atomic.AddInt64(&atomicCount, 1) |
|
||||||
if count >= numInQueue { |
|
||||||
close(stop) |
|
||||||
} |
|
||||||
} |
|
||||||
lock.Unlock() |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
q := newQueue(name, handle) |
|
||||||
chans := runQueue(q, &lock) |
|
||||||
|
|
||||||
<-chans.readyForShutdown |
|
||||||
<-stop |
|
||||||
close(chans.signalShutdown) |
|
||||||
<-chans.doneShutdown |
|
||||||
|
|
||||||
// check which tasks are still in the queue
|
|
||||||
for i := 0; i < 100; i++ { |
|
||||||
if has, _ := q.(UniqueQueue).Has("task-" + strconv.Itoa(i)); has { |
|
||||||
mapLock.Lock() |
|
||||||
hasEmpty[name] = append(hasEmpty[name], "task-"+strconv.Itoa(i)) |
|
||||||
mapLock.Unlock() |
|
||||||
} |
|
||||||
} |
|
||||||
doTerminate(chans, &lock) |
|
||||||
|
|
||||||
mapLock.Lock() |
|
||||||
assert.Equal(t, 101, len(executedInitial[name])+len(executedEmpty[name])) |
|
||||||
assert.Empty(t, hasEmpty[name]) |
|
||||||
mapLock.Unlock() |
|
||||||
}) |
|
||||||
close(done) |
|
||||||
} |
|
||||||
|
|
||||||
doneA := make(chan struct{}) |
|
||||||
doneB := make(chan struct{}) |
|
||||||
|
|
||||||
go emptyQueue("QueueA", hasA, doneA) |
|
||||||
go emptyQueue("QueueB", hasB, doneB) |
|
||||||
|
|
||||||
<-doneA |
|
||||||
<-doneB |
|
||||||
|
|
||||||
mapLock.Lock() |
|
||||||
t.Logf("TestPersistableChannelUniqueQueue executedInitiallyA=%v, executedInitiallyB=%v, executedToEmptyA=%v, executedToEmptyB=%v", |
|
||||||
len(executedInitial["QueueA"]), len(executedInitial["QueueB"]), len(executedEmpty["QueueA"]), len(executedEmpty["QueueB"])) |
|
||||||
|
|
||||||
// reset and rerun
|
|
||||||
executedInitial = map[string][]string{} |
|
||||||
hasInitial = map[string][]string{} |
|
||||||
executedEmpty = map[string][]string{} |
|
||||||
hasEmpty = map[string][]string{} |
|
||||||
mapLock.Unlock() |
|
||||||
|
|
||||||
hasQueueAChan = make(chan int64) |
|
||||||
hasQueueBChan = make(chan int64) |
|
||||||
|
|
||||||
go fillQueue("QueueA", hasQueueAChan) |
|
||||||
go fillQueue("QueueB", hasQueueBChan) |
|
||||||
|
|
||||||
hasA = <-hasQueueAChan |
|
||||||
hasB = <-hasQueueBChan |
|
||||||
|
|
||||||
doneA = make(chan struct{}) |
|
||||||
doneB = make(chan struct{}) |
|
||||||
|
|
||||||
go emptyQueue("QueueA", hasA, doneA) |
|
||||||
go emptyQueue("QueueB", hasB, doneB) |
|
||||||
|
|
||||||
<-doneA |
|
||||||
<-doneB |
|
||||||
|
|
||||||
mapLock.Lock() |
|
||||||
t.Logf("TestPersistableChannelUniqueQueue executedInitiallyA=%v, executedInitiallyB=%v, executedToEmptyA=%v, executedToEmptyB=%v", |
|
||||||
len(executedInitial["QueueA"]), len(executedInitial["QueueB"]), len(executedEmpty["QueueA"]), len(executedEmpty["QueueB"])) |
|
||||||
mapLock.Unlock() |
|
||||||
} |
|
@ -1,141 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
|
|
||||||
"github.com/redis/go-redis/v9" |
|
||||||
) |
|
||||||
|
|
||||||
// RedisUniqueQueueType is the type for redis queue
|
|
||||||
const RedisUniqueQueueType Type = "unique-redis" |
|
||||||
|
|
||||||
// RedisUniqueQueue redis queue
|
|
||||||
type RedisUniqueQueue struct { |
|
||||||
*ByteFIFOUniqueQueue |
|
||||||
} |
|
||||||
|
|
||||||
// RedisUniqueQueueConfiguration is the configuration for the redis queue
|
|
||||||
type RedisUniqueQueueConfiguration struct { |
|
||||||
ByteFIFOQueueConfiguration |
|
||||||
RedisUniqueByteFIFOConfiguration |
|
||||||
} |
|
||||||
|
|
||||||
// NewRedisUniqueQueue creates single redis or cluster redis queue.
|
|
||||||
//
|
|
||||||
// Please note that this Queue does not guarantee that a particular
|
|
||||||
// task cannot be processed twice or more at the same time. Uniqueness is
|
|
||||||
// only guaranteed whilst the task is waiting in the queue.
|
|
||||||
func NewRedisUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(RedisUniqueQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(RedisUniqueQueueConfiguration) |
|
||||||
|
|
||||||
byteFIFO, err := NewRedisUniqueByteFIFO(config.RedisUniqueByteFIFOConfiguration) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
if len(byteFIFO.setName) == 0 { |
|
||||||
byteFIFO.setName = byteFIFO.queueName + "_unique" |
|
||||||
} |
|
||||||
|
|
||||||
byteFIFOQueue, err := NewByteFIFOUniqueQueue(RedisUniqueQueueType, byteFIFO, handle, config.ByteFIFOQueueConfiguration, exemplar) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
queue := &RedisUniqueQueue{ |
|
||||||
ByteFIFOUniqueQueue: byteFIFOQueue, |
|
||||||
} |
|
||||||
|
|
||||||
queue.qid = GetManager().Add(queue, RedisUniqueQueueType, config, exemplar) |
|
||||||
|
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
|
|
||||||
var _ UniqueByteFIFO = &RedisUniqueByteFIFO{} |
|
||||||
|
|
||||||
// RedisUniqueByteFIFO represents a UniqueByteFIFO formed from a redisClient
|
|
||||||
type RedisUniqueByteFIFO struct { |
|
||||||
RedisByteFIFO |
|
||||||
setName string |
|
||||||
} |
|
||||||
|
|
||||||
// RedisUniqueByteFIFOConfiguration is the configuration for the RedisUniqueByteFIFO
|
|
||||||
type RedisUniqueByteFIFOConfiguration struct { |
|
||||||
RedisByteFIFOConfiguration |
|
||||||
SetName string |
|
||||||
} |
|
||||||
|
|
||||||
// NewRedisUniqueByteFIFO creates a UniqueByteFIFO formed from a redisClient
|
|
||||||
func NewRedisUniqueByteFIFO(config RedisUniqueByteFIFOConfiguration) (*RedisUniqueByteFIFO, error) { |
|
||||||
internal, err := NewRedisByteFIFO(config.RedisByteFIFOConfiguration) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
|
|
||||||
fifo := &RedisUniqueByteFIFO{ |
|
||||||
RedisByteFIFO: *internal, |
|
||||||
setName: config.SetName, |
|
||||||
} |
|
||||||
|
|
||||||
return fifo, nil |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc pushes data to the end of the fifo and calls the callback if it is added
|
|
||||||
func (fifo *RedisUniqueByteFIFO) PushFunc(ctx context.Context, data []byte, fn func() error) error { |
|
||||||
added, err := fifo.client.SAdd(ctx, fifo.setName, data).Result() |
|
||||||
if err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
if added == 0 { |
|
||||||
return ErrAlreadyInQueue |
|
||||||
} |
|
||||||
if fn != nil { |
|
||||||
if err := fn(); err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
} |
|
||||||
return fifo.client.RPush(ctx, fifo.queueName, data).Err() |
|
||||||
} |
|
||||||
|
|
||||||
// PushBack pushes data to the top of the fifo
|
|
||||||
func (fifo *RedisUniqueByteFIFO) PushBack(ctx context.Context, data []byte) error { |
|
||||||
added, err := fifo.client.SAdd(ctx, fifo.setName, data).Result() |
|
||||||
if err != nil { |
|
||||||
return err |
|
||||||
} |
|
||||||
if added == 0 { |
|
||||||
return ErrAlreadyInQueue |
|
||||||
} |
|
||||||
return fifo.client.LPush(ctx, fifo.queueName, data).Err() |
|
||||||
} |
|
||||||
|
|
||||||
// Pop pops data from the start of the fifo
|
|
||||||
func (fifo *RedisUniqueByteFIFO) Pop(ctx context.Context) ([]byte, error) { |
|
||||||
data, err := fifo.client.LPop(ctx, fifo.queueName).Bytes() |
|
||||||
if err != nil && err != redis.Nil { |
|
||||||
return data, err |
|
||||||
} |
|
||||||
|
|
||||||
if len(data) == 0 { |
|
||||||
return data, nil |
|
||||||
} |
|
||||||
|
|
||||||
err = fifo.client.SRem(ctx, fifo.setName, data).Err() |
|
||||||
return data, err |
|
||||||
} |
|
||||||
|
|
||||||
// Has returns whether the fifo contains this data
|
|
||||||
func (fifo *RedisUniqueByteFIFO) Has(ctx context.Context, data []byte) (bool, error) { |
|
||||||
return fifo.client.SIsMember(ctx, fifo.setName, data).Result() |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[RedisUniqueQueueType] = NewRedisUniqueQueue |
|
||||||
} |
|
@ -1,174 +0,0 @@ |
|||||||
// Copyright 2020 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"fmt" |
|
||||||
"sync" |
|
||||||
"time" |
|
||||||
) |
|
||||||
|
|
||||||
// WrappedUniqueQueueType is the type for a wrapped delayed starting queue
|
|
||||||
const WrappedUniqueQueueType Type = "unique-wrapped" |
|
||||||
|
|
||||||
// WrappedUniqueQueueConfiguration is the configuration for a WrappedUniqueQueue
|
|
||||||
type WrappedUniqueQueueConfiguration struct { |
|
||||||
Underlying Type |
|
||||||
Timeout time.Duration |
|
||||||
MaxAttempts int |
|
||||||
Config interface{} |
|
||||||
QueueLength int |
|
||||||
Name string |
|
||||||
} |
|
||||||
|
|
||||||
// WrappedUniqueQueue wraps a delayed starting unique queue
|
|
||||||
type WrappedUniqueQueue struct { |
|
||||||
*WrappedQueue |
|
||||||
table map[Data]bool |
|
||||||
tlock sync.Mutex |
|
||||||
ready bool |
|
||||||
} |
|
||||||
|
|
||||||
// NewWrappedUniqueQueue will attempt to create a unique queue of the provided type,
|
|
||||||
// but if there is a problem creating this queue it will instead create
|
|
||||||
// a WrappedUniqueQueue with delayed startup of the queue instead and a
|
|
||||||
// channel which will be redirected to the queue
|
|
||||||
//
|
|
||||||
// Please note that this Queue does not guarantee that a particular
|
|
||||||
// task cannot be processed twice or more at the same time. Uniqueness is
|
|
||||||
// only guaranteed whilst the task is waiting in the queue.
|
|
||||||
func NewWrappedUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { |
|
||||||
configInterface, err := toConfig(WrappedUniqueQueueConfiguration{}, cfg) |
|
||||||
if err != nil { |
|
||||||
return nil, err |
|
||||||
} |
|
||||||
config := configInterface.(WrappedUniqueQueueConfiguration) |
|
||||||
|
|
||||||
queue, err := NewQueue(config.Underlying, handle, config.Config, exemplar) |
|
||||||
if err == nil { |
|
||||||
// Just return the queue there is no need to wrap
|
|
||||||
return queue, nil |
|
||||||
} |
|
||||||
if IsErrInvalidConfiguration(err) { |
|
||||||
// Retrying ain't gonna make this any better...
|
|
||||||
return nil, ErrInvalidConfiguration{cfg: cfg} |
|
||||||
} |
|
||||||
|
|
||||||
wrapped := &WrappedUniqueQueue{ |
|
||||||
WrappedQueue: &WrappedQueue{ |
|
||||||
channel: make(chan Data, config.QueueLength), |
|
||||||
exemplar: exemplar, |
|
||||||
delayedStarter: delayedStarter{ |
|
||||||
cfg: config.Config, |
|
||||||
underlying: config.Underlying, |
|
||||||
timeout: config.Timeout, |
|
||||||
maxAttempts: config.MaxAttempts, |
|
||||||
name: config.Name, |
|
||||||
}, |
|
||||||
}, |
|
||||||
table: map[Data]bool{}, |
|
||||||
} |
|
||||||
|
|
||||||
// wrapped.handle is passed to the delayedStarting internal queue and is run to handle
|
|
||||||
// data passed to
|
|
||||||
wrapped.handle = func(data ...Data) (unhandled []Data) { |
|
||||||
for _, datum := range data { |
|
||||||
wrapped.tlock.Lock() |
|
||||||
if !wrapped.ready { |
|
||||||
delete(wrapped.table, data) |
|
||||||
// If our table is empty all of the requests we have buffered between the
|
|
||||||
// wrapper queue starting and the internal queue starting have been handled.
|
|
||||||
// We can stop buffering requests in our local table and just pass Push
|
|
||||||
// direct to the internal queue
|
|
||||||
if len(wrapped.table) == 0 { |
|
||||||
wrapped.ready = true |
|
||||||
} |
|
||||||
} |
|
||||||
wrapped.tlock.Unlock() |
|
||||||
if u := handle(datum); u != nil { |
|
||||||
unhandled = append(unhandled, u...) |
|
||||||
} |
|
||||||
} |
|
||||||
return unhandled |
|
||||||
} |
|
||||||
_ = GetManager().Add(queue, WrappedUniqueQueueType, config, exemplar) |
|
||||||
return wrapped, nil |
|
||||||
} |
|
||||||
|
|
||||||
// Push will push the data to the internal channel checking it against the exemplar
|
|
||||||
func (q *WrappedUniqueQueue) Push(data Data) error { |
|
||||||
return q.PushFunc(data, nil) |
|
||||||
} |
|
||||||
|
|
||||||
// PushFunc will push the data to the internal channel checking it against the exemplar
|
|
||||||
func (q *WrappedUniqueQueue) PushFunc(data Data, fn func() error) error { |
|
||||||
if !assignableTo(data, q.exemplar) { |
|
||||||
return fmt.Errorf("unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) |
|
||||||
} |
|
||||||
|
|
||||||
q.tlock.Lock() |
|
||||||
if q.ready { |
|
||||||
// ready means our table is empty and all of the requests we have buffered between the
|
|
||||||
// wrapper queue starting and the internal queue starting have been handled.
|
|
||||||
// We can stop buffering requests in our local table and just pass Push
|
|
||||||
// direct to the internal queue
|
|
||||||
q.tlock.Unlock() |
|
||||||
return q.internal.(UniqueQueue).PushFunc(data, fn) |
|
||||||
} |
|
||||||
|
|
||||||
locked := true |
|
||||||
defer func() { |
|
||||||
if locked { |
|
||||||
q.tlock.Unlock() |
|
||||||
} |
|
||||||
}() |
|
||||||
if _, ok := q.table[data]; ok { |
|
||||||
return ErrAlreadyInQueue |
|
||||||
} |
|
||||||
// FIXME: We probably need to implement some sort of limit here
|
|
||||||
// If the downstream queue blocks this table will grow without limit
|
|
||||||
q.table[data] = true |
|
||||||
if fn != nil { |
|
||||||
err := fn() |
|
||||||
if err != nil { |
|
||||||
delete(q.table, data) |
|
||||||
return err |
|
||||||
} |
|
||||||
} |
|
||||||
locked = false |
|
||||||
q.tlock.Unlock() |
|
||||||
|
|
||||||
q.channel <- data |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Has checks if the data is in the queue
|
|
||||||
func (q *WrappedUniqueQueue) Has(data Data) (bool, error) { |
|
||||||
q.tlock.Lock() |
|
||||||
defer q.tlock.Unlock() |
|
||||||
if q.ready { |
|
||||||
return q.internal.(UniqueQueue).Has(data) |
|
||||||
} |
|
||||||
_, has := q.table[data] |
|
||||||
return has, nil |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty checks whether the queue is empty
|
|
||||||
func (q *WrappedUniqueQueue) IsEmpty() bool { |
|
||||||
q.tlock.Lock() |
|
||||||
if len(q.table) > 0 { |
|
||||||
q.tlock.Unlock() |
|
||||||
return false |
|
||||||
} |
|
||||||
if q.ready { |
|
||||||
q.tlock.Unlock() |
|
||||||
return q.internal.IsEmpty() |
|
||||||
} |
|
||||||
q.tlock.Unlock() |
|
||||||
return false |
|
||||||
} |
|
||||||
|
|
||||||
func init() { |
|
||||||
queuesMap[WrappedUniqueQueueType] = NewWrappedUniqueQueue |
|
||||||
} |
|
@ -0,0 +1,331 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"sync" |
||||||
|
"sync/atomic" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/log" |
||||||
|
) |
||||||
|
|
||||||
|
var ( |
||||||
|
infiniteTimerC = make(chan time.Time) |
||||||
|
batchDebounceDuration = 100 * time.Millisecond |
||||||
|
workerIdleDuration = 1 * time.Second |
||||||
|
|
||||||
|
unhandledItemRequeueDuration atomic.Int64 // to avoid data race during test
|
||||||
|
) |
||||||
|
|
||||||
|
func init() { |
||||||
|
unhandledItemRequeueDuration.Store(int64(5 * time.Second)) |
||||||
|
} |
||||||
|
|
||||||
|
// workerGroup is a group of workers to work with a WorkerPoolQueue
|
||||||
|
type workerGroup[T any] struct { |
||||||
|
q *WorkerPoolQueue[T] |
||||||
|
wg sync.WaitGroup |
||||||
|
|
||||||
|
ctxWorker context.Context |
||||||
|
ctxWorkerCancel context.CancelFunc |
||||||
|
|
||||||
|
batchBuffer []T |
||||||
|
popItemChan chan []byte |
||||||
|
popItemErr chan error |
||||||
|
} |
||||||
|
|
||||||
|
func (wg *workerGroup[T]) doPrepareWorkerContext() { |
||||||
|
wg.ctxWorker, wg.ctxWorkerCancel = context.WithCancel(wg.q.ctxRun) |
||||||
|
} |
||||||
|
|
||||||
|
// doDispatchBatchToWorker dispatches a batch of items to worker's channel.
|
||||||
|
// If the channel is full, it tries to start a new worker if possible.
|
||||||
|
func (q *WorkerPoolQueue[T]) doDispatchBatchToWorker(wg *workerGroup[T], flushChan chan flushType) { |
||||||
|
batch := wg.batchBuffer |
||||||
|
wg.batchBuffer = nil |
||||||
|
|
||||||
|
if len(batch) == 0 { |
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
full := false |
||||||
|
select { |
||||||
|
case q.batchChan <- batch: |
||||||
|
default: |
||||||
|
full = true |
||||||
|
} |
||||||
|
|
||||||
|
q.workerNumMu.Lock() |
||||||
|
noWorker := q.workerNum == 0 |
||||||
|
if full || noWorker { |
||||||
|
if q.workerNum < q.workerMaxNum || noWorker && q.workerMaxNum <= 0 { |
||||||
|
q.workerNum++ |
||||||
|
q.doStartNewWorker(wg) |
||||||
|
} |
||||||
|
} |
||||||
|
q.workerNumMu.Unlock() |
||||||
|
|
||||||
|
if full { |
||||||
|
select { |
||||||
|
case q.batchChan <- batch: |
||||||
|
case flush := <-flushChan: |
||||||
|
q.doWorkerHandle(batch) |
||||||
|
q.doFlush(wg, flush) |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
wg.batchBuffer = batch // return the batch to buffer, the "doRun" function will handle it
|
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// doWorkerHandle calls the safeHandler to handle a batch of items, and it increases/decreases the active worker number.
|
||||||
|
// If the context has been canceled, it should not be caller because the "Push" still needs the context, in such case, call q.safeHandler directly
|
||||||
|
func (q *WorkerPoolQueue[T]) doWorkerHandle(batch []T) { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
q.workerActiveNum++ |
||||||
|
q.workerNumMu.Unlock() |
||||||
|
|
||||||
|
defer func() { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
q.workerActiveNum-- |
||||||
|
q.workerNumMu.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
unhandled := q.safeHandler(batch...) |
||||||
|
// if none of the items were handled, it should back-off for a few seconds
|
||||||
|
// in this case the handler (eg: document indexer) may have encountered some errors/failures
|
||||||
|
if len(unhandled) == len(batch) && unhandledItemRequeueDuration.Load() != 0 { |
||||||
|
log.Error("Queue %q failed to handle batch of %d items, backoff for a few seconds", q.GetName(), len(batch)) |
||||||
|
select { |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
case <-time.After(time.Duration(unhandledItemRequeueDuration.Load())): |
||||||
|
} |
||||||
|
} |
||||||
|
for _, item := range unhandled { |
||||||
|
if err := q.Push(item); err != nil { |
||||||
|
if !q.basePushForShutdown(item) { |
||||||
|
log.Error("Failed to requeue item for queue %q when calling handler: %v", q.GetName(), err) |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// basePushForShutdown tries to requeue items into the base queue when the WorkerPoolQueue is shutting down.
|
||||||
|
// If the queue is shutting down, it returns true and try to push the items
|
||||||
|
// Otherwise it does nothing and returns false
|
||||||
|
func (q *WorkerPoolQueue[T]) basePushForShutdown(items ...T) bool { |
||||||
|
ctxShutdown := q.ctxShutdown.Load() |
||||||
|
if ctxShutdown == nil { |
||||||
|
return false |
||||||
|
} |
||||||
|
for _, item := range items { |
||||||
|
// if there is still any error, the queue can do nothing instead of losing the items
|
||||||
|
if err := q.baseQueue.PushItem(*ctxShutdown, q.marshal(item)); err != nil { |
||||||
|
log.Error("Failed to requeue item for queue %q when shutting down: %v", q.GetName(), err) |
||||||
|
} |
||||||
|
} |
||||||
|
return true |
||||||
|
} |
||||||
|
|
||||||
|
// doStartNewWorker starts a new worker for the queue, the worker reads from worker's channel and handles the items.
|
||||||
|
func (q *WorkerPoolQueue[T]) doStartNewWorker(wp *workerGroup[T]) { |
||||||
|
wp.wg.Add(1) |
||||||
|
|
||||||
|
go func() { |
||||||
|
defer wp.wg.Done() |
||||||
|
|
||||||
|
log.Debug("Queue %q starts new worker", q.GetName()) |
||||||
|
defer log.Debug("Queue %q stops idle worker", q.GetName()) |
||||||
|
|
||||||
|
t := time.NewTicker(workerIdleDuration) |
||||||
|
keepWorking := true |
||||||
|
stopWorking := func() { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
keepWorking = false |
||||||
|
q.workerNum-- |
||||||
|
q.workerNumMu.Unlock() |
||||||
|
} |
||||||
|
for keepWorking { |
||||||
|
select { |
||||||
|
case <-wp.ctxWorker.Done(): |
||||||
|
stopWorking() |
||||||
|
case batch, ok := <-q.batchChan: |
||||||
|
if !ok { |
||||||
|
stopWorking() |
||||||
|
} else { |
||||||
|
q.doWorkerHandle(batch) |
||||||
|
t.Reset(workerIdleDuration) |
||||||
|
} |
||||||
|
case <-t.C: |
||||||
|
q.workerNumMu.Lock() |
||||||
|
keepWorking = q.workerNum <= 1 |
||||||
|
if !keepWorking { |
||||||
|
q.workerNum-- |
||||||
|
} |
||||||
|
q.workerNumMu.Unlock() |
||||||
|
} |
||||||
|
} |
||||||
|
}() |
||||||
|
} |
||||||
|
|
||||||
|
// doFlush flushes the queue: it tries to read all items from the queue and handles them.
|
||||||
|
// It is for testing purpose only. It's not designed to work for a cluster.
|
||||||
|
func (q *WorkerPoolQueue[T]) doFlush(wg *workerGroup[T], flush flushType) { |
||||||
|
log.Debug("Queue %q starts flushing", q.GetName()) |
||||||
|
defer log.Debug("Queue %q finishes flushing", q.GetName()) |
||||||
|
|
||||||
|
// stop all workers, and prepare a new worker context to start new workers
|
||||||
|
|
||||||
|
wg.ctxWorkerCancel() |
||||||
|
wg.wg.Wait() |
||||||
|
|
||||||
|
defer func() { |
||||||
|
close(flush) |
||||||
|
wg.doPrepareWorkerContext() |
||||||
|
}() |
||||||
|
|
||||||
|
// drain the batch channel first
|
||||||
|
loop: |
||||||
|
for { |
||||||
|
select { |
||||||
|
case batch := <-q.batchChan: |
||||||
|
q.doWorkerHandle(batch) |
||||||
|
default: |
||||||
|
break loop |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// drain the popItem channel
|
||||||
|
emptyCounter := 0 |
||||||
|
for { |
||||||
|
select { |
||||||
|
case data, dataOk := <-wg.popItemChan: |
||||||
|
if !dataOk { |
||||||
|
return |
||||||
|
} |
||||||
|
emptyCounter = 0 |
||||||
|
if v, jsonOk := q.unmarshal(data); !jsonOk { |
||||||
|
continue |
||||||
|
} else { |
||||||
|
q.doWorkerHandle([]T{v}) |
||||||
|
} |
||||||
|
case err := <-wg.popItemErr: |
||||||
|
if !q.isCtxRunCanceled() { |
||||||
|
log.Error("Failed to pop item from queue %q (doFlush): %v", q.GetName(), err) |
||||||
|
} |
||||||
|
return |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
log.Debug("Queue %q is shutting down", q.GetName()) |
||||||
|
return |
||||||
|
case <-time.After(20 * time.Millisecond): |
||||||
|
// There is no reliable way to make sure all queue items are consumed by the Flush, there always might be some items stored in some buffers/temp variables.
|
||||||
|
// If we run Gitea in a cluster, we can even not guarantee all items are consumed in a deterministic instance.
|
||||||
|
// Luckily, the "Flush" trick is only used in tests, so far so good.
|
||||||
|
if cnt, _ := q.baseQueue.Len(q.ctxRun); cnt == 0 && len(wg.popItemChan) == 0 { |
||||||
|
emptyCounter++ |
||||||
|
} |
||||||
|
if emptyCounter >= 2 { |
||||||
|
return |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) isCtxRunCanceled() bool { |
||||||
|
select { |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
return true |
||||||
|
default: |
||||||
|
return false |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
var skipFlushChan = make(chan flushType) // an empty flush chan, used to skip reading other flush requests
|
||||||
|
|
||||||
|
// doRun is the main loop of the queue. All related "doXxx" functions are executed in its context.
|
||||||
|
func (q *WorkerPoolQueue[T]) doRun() { |
||||||
|
log.Debug("Queue %q starts running", q.GetName()) |
||||||
|
defer log.Debug("Queue %q stops running", q.GetName()) |
||||||
|
|
||||||
|
wg := &workerGroup[T]{q: q} |
||||||
|
wg.doPrepareWorkerContext() |
||||||
|
wg.popItemChan, wg.popItemErr = popItemByChan(q.ctxRun, q.baseQueue.PopItem) |
||||||
|
|
||||||
|
defer func() { |
||||||
|
q.ctxRunCancel() |
||||||
|
|
||||||
|
// drain all data on the fly
|
||||||
|
// since the queue is shutting down, the items can't be dispatched to workers because the context is canceled
|
||||||
|
// it can't call doWorkerHandle either, because there is no chance to push unhandled items back to the queue
|
||||||
|
var unhandled []T |
||||||
|
close(q.batchChan) |
||||||
|
for batch := range q.batchChan { |
||||||
|
unhandled = append(unhandled, batch...) |
||||||
|
} |
||||||
|
unhandled = append(unhandled, wg.batchBuffer...) |
||||||
|
for data := range wg.popItemChan { |
||||||
|
if v, ok := q.unmarshal(data); ok { |
||||||
|
unhandled = append(unhandled, v) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
ctxShutdownPtr := q.ctxShutdown.Load() |
||||||
|
if ctxShutdownPtr != nil { |
||||||
|
// if there is a shutdown context, try to push the items back to the base queue
|
||||||
|
q.basePushForShutdown(unhandled...) |
||||||
|
workerDone := make(chan struct{}) |
||||||
|
// the only way to wait for the workers, because the handlers do not have context to wait for
|
||||||
|
go func() { wg.wg.Wait(); close(workerDone) }() |
||||||
|
select { |
||||||
|
case <-workerDone: |
||||||
|
case <-(*ctxShutdownPtr).Done(): |
||||||
|
log.Error("Queue %q is shutting down, but workers are still running after timeout", q.GetName()) |
||||||
|
} |
||||||
|
} else { |
||||||
|
// if there is no shutdown context, just call the handler to try to handle the items. if the handler fails again, the items are lost
|
||||||
|
q.safeHandler(unhandled...) |
||||||
|
} |
||||||
|
|
||||||
|
close(q.shutdownDone) |
||||||
|
}() |
||||||
|
|
||||||
|
var batchDispatchC <-chan time.Time = infiniteTimerC |
||||||
|
for { |
||||||
|
select { |
||||||
|
case data, dataOk := <-wg.popItemChan: |
||||||
|
if !dataOk { |
||||||
|
return |
||||||
|
} |
||||||
|
if v, jsonOk := q.unmarshal(data); !jsonOk { |
||||||
|
testRecorder.Record("pop:corrupted:%s", data) // in rare cases the levelqueue(leveldb) might be corrupted
|
||||||
|
continue |
||||||
|
} else { |
||||||
|
wg.batchBuffer = append(wg.batchBuffer, v) |
||||||
|
} |
||||||
|
if len(wg.batchBuffer) >= q.batchLength { |
||||||
|
q.doDispatchBatchToWorker(wg, q.flushChan) |
||||||
|
} else if batchDispatchC == infiniteTimerC { |
||||||
|
batchDispatchC = time.After(batchDebounceDuration) |
||||||
|
} // else: batchDispatchC is already a debounce timer, it will be triggered soon
|
||||||
|
case <-batchDispatchC: |
||||||
|
batchDispatchC = infiniteTimerC |
||||||
|
q.doDispatchBatchToWorker(wg, q.flushChan) |
||||||
|
case flush := <-q.flushChan: |
||||||
|
// before flushing, it needs to try to dispatch the batch to worker first, in case there is no worker running
|
||||||
|
// after the flushing, there is at least one worker running, so "doFlush" could wait for workers to finish
|
||||||
|
// since we are already in a "flush" operation, so the dispatching function shouldn't read the flush chan.
|
||||||
|
q.doDispatchBatchToWorker(wg, skipFlushChan) |
||||||
|
q.doFlush(wg, flush) |
||||||
|
case err := <-wg.popItemErr: |
||||||
|
if !q.isCtxRunCanceled() { |
||||||
|
log.Error("Failed to pop item from queue %q (doRun): %v", q.GetName(), err) |
||||||
|
} |
||||||
|
return |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
log.Debug("Queue %q is shutting down", q.GetName()) |
||||||
|
return |
||||||
|
} |
||||||
|
} |
||||||
|
} |
@ -1,613 +0,0 @@ |
|||||||
// Copyright 2019 The Gitea Authors. All rights reserved.
|
|
||||||
// SPDX-License-Identifier: MIT
|
|
||||||
|
|
||||||
package queue |
|
||||||
|
|
||||||
import ( |
|
||||||
"context" |
|
||||||
"fmt" |
|
||||||
"runtime/pprof" |
|
||||||
"sync" |
|
||||||
"sync/atomic" |
|
||||||
"time" |
|
||||||
|
|
||||||
"code.gitea.io/gitea/modules/log" |
|
||||||
"code.gitea.io/gitea/modules/process" |
|
||||||
"code.gitea.io/gitea/modules/util" |
|
||||||
) |
|
||||||
|
|
||||||
// WorkerPool represent a dynamically growable worker pool for a
|
|
||||||
// provided handler function. They have an internal channel which
|
|
||||||
// they use to detect if there is a block and will grow and shrink in
|
|
||||||
// response to demand as per configuration.
|
|
||||||
type WorkerPool struct { |
|
||||||
// This field requires to be the first one in the struct.
|
|
||||||
// This is to allow 64 bit atomic operations on 32-bit machines.
|
|
||||||
// See: https://pkg.go.dev/sync/atomic#pkg-note-BUG & Gitea issue 19518
|
|
||||||
numInQueue int64 |
|
||||||
lock sync.Mutex |
|
||||||
baseCtx context.Context |
|
||||||
baseCtxCancel context.CancelFunc |
|
||||||
baseCtxFinished process.FinishedFunc |
|
||||||
paused chan struct{} |
|
||||||
resumed chan struct{} |
|
||||||
cond *sync.Cond |
|
||||||
qid int64 |
|
||||||
maxNumberOfWorkers int |
|
||||||
numberOfWorkers int |
|
||||||
batchLength int |
|
||||||
handle HandlerFunc |
|
||||||
dataChan chan Data |
|
||||||
blockTimeout time.Duration |
|
||||||
boostTimeout time.Duration |
|
||||||
boostWorkers int |
|
||||||
} |
|
||||||
|
|
||||||
var ( |
|
||||||
_ Flushable = &WorkerPool{} |
|
||||||
_ ManagedPool = &WorkerPool{} |
|
||||||
) |
|
||||||
|
|
||||||
// WorkerPoolConfiguration is the basic configuration for a WorkerPool
|
|
||||||
type WorkerPoolConfiguration struct { |
|
||||||
Name string |
|
||||||
QueueLength int |
|
||||||
BatchLength int |
|
||||||
BlockTimeout time.Duration |
|
||||||
BoostTimeout time.Duration |
|
||||||
BoostWorkers int |
|
||||||
MaxWorkers int |
|
||||||
} |
|
||||||
|
|
||||||
// NewWorkerPool creates a new worker pool
|
|
||||||
func NewWorkerPool(handle HandlerFunc, config WorkerPoolConfiguration) *WorkerPool { |
|
||||||
ctx, cancel, finished := process.GetManager().AddTypedContext(context.Background(), fmt.Sprintf("Queue: %s", config.Name), process.SystemProcessType, false) |
|
||||||
|
|
||||||
dataChan := make(chan Data, config.QueueLength) |
|
||||||
pool := &WorkerPool{ |
|
||||||
baseCtx: ctx, |
|
||||||
baseCtxCancel: cancel, |
|
||||||
baseCtxFinished: finished, |
|
||||||
batchLength: config.BatchLength, |
|
||||||
dataChan: dataChan, |
|
||||||
resumed: closedChan, |
|
||||||
paused: make(chan struct{}), |
|
||||||
handle: handle, |
|
||||||
blockTimeout: config.BlockTimeout, |
|
||||||
boostTimeout: config.BoostTimeout, |
|
||||||
boostWorkers: config.BoostWorkers, |
|
||||||
maxNumberOfWorkers: config.MaxWorkers, |
|
||||||
} |
|
||||||
|
|
||||||
return pool |
|
||||||
} |
|
||||||
|
|
||||||
// Done returns when this worker pool's base context has been cancelled
|
|
||||||
func (p *WorkerPool) Done() <-chan struct{} { |
|
||||||
return p.baseCtx.Done() |
|
||||||
} |
|
||||||
|
|
||||||
// Push pushes the data to the internal channel
|
|
||||||
func (p *WorkerPool) Push(data Data) { |
|
||||||
atomic.AddInt64(&p.numInQueue, 1) |
|
||||||
p.lock.Lock() |
|
||||||
select { |
|
||||||
case <-p.paused: |
|
||||||
p.lock.Unlock() |
|
||||||
p.dataChan <- data |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
if p.blockTimeout > 0 && p.boostTimeout > 0 && (p.numberOfWorkers <= p.maxNumberOfWorkers || p.maxNumberOfWorkers < 0) { |
|
||||||
if p.numberOfWorkers == 0 { |
|
||||||
p.zeroBoost() |
|
||||||
} else { |
|
||||||
p.lock.Unlock() |
|
||||||
} |
|
||||||
p.pushBoost(data) |
|
||||||
} else { |
|
||||||
p.lock.Unlock() |
|
||||||
p.dataChan <- data |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// HasNoWorkerScaling will return true if the queue has no workers, and has no worker boosting
|
|
||||||
func (p *WorkerPool) HasNoWorkerScaling() bool { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.hasNoWorkerScaling() |
|
||||||
} |
|
||||||
|
|
||||||
func (p *WorkerPool) hasNoWorkerScaling() bool { |
|
||||||
return p.numberOfWorkers == 0 && (p.boostTimeout == 0 || p.boostWorkers == 0 || p.maxNumberOfWorkers == 0) |
|
||||||
} |
|
||||||
|
|
||||||
// zeroBoost will add a temporary boost worker for a no worker queue
|
|
||||||
// p.lock must be locked at the start of this function BUT it will be unlocked by the end of this function
|
|
||||||
// (This is because addWorkers has to be called whilst unlocked)
|
|
||||||
func (p *WorkerPool) zeroBoost() { |
|
||||||
ctx, cancel := context.WithTimeout(p.baseCtx, p.boostTimeout) |
|
||||||
mq := GetManager().GetManagedQueue(p.qid) |
|
||||||
boost := p.boostWorkers |
|
||||||
if (boost+p.numberOfWorkers) > p.maxNumberOfWorkers && p.maxNumberOfWorkers >= 0 { |
|
||||||
boost = p.maxNumberOfWorkers - p.numberOfWorkers |
|
||||||
} |
|
||||||
if mq != nil { |
|
||||||
log.Debug("WorkerPool: %d (for %s) has zero workers - adding %d temporary workers for %s", p.qid, mq.Name, boost, p.boostTimeout) |
|
||||||
|
|
||||||
start := time.Now() |
|
||||||
pid := mq.RegisterWorkers(boost, start, true, start.Add(p.boostTimeout), cancel, false) |
|
||||||
cancel = func() { |
|
||||||
mq.RemoveWorkers(pid) |
|
||||||
} |
|
||||||
} else { |
|
||||||
log.Debug("WorkerPool: %d has zero workers - adding %d temporary workers for %s", p.qid, p.boostWorkers, p.boostTimeout) |
|
||||||
} |
|
||||||
p.lock.Unlock() |
|
||||||
p.addWorkers(ctx, cancel, boost) |
|
||||||
} |
|
||||||
|
|
||||||
func (p *WorkerPool) pushBoost(data Data) { |
|
||||||
select { |
|
||||||
case p.dataChan <- data: |
|
||||||
default: |
|
||||||
p.lock.Lock() |
|
||||||
if p.blockTimeout <= 0 { |
|
||||||
p.lock.Unlock() |
|
||||||
p.dataChan <- data |
|
||||||
return |
|
||||||
} |
|
||||||
ourTimeout := p.blockTimeout |
|
||||||
timer := time.NewTimer(p.blockTimeout) |
|
||||||
p.lock.Unlock() |
|
||||||
select { |
|
||||||
case p.dataChan <- data: |
|
||||||
util.StopTimer(timer) |
|
||||||
case <-timer.C: |
|
||||||
p.lock.Lock() |
|
||||||
if p.blockTimeout > ourTimeout || (p.numberOfWorkers > p.maxNumberOfWorkers && p.maxNumberOfWorkers >= 0) { |
|
||||||
p.lock.Unlock() |
|
||||||
p.dataChan <- data |
|
||||||
return |
|
||||||
} |
|
||||||
p.blockTimeout *= 2 |
|
||||||
boostCtx, boostCtxCancel := context.WithCancel(p.baseCtx) |
|
||||||
mq := GetManager().GetManagedQueue(p.qid) |
|
||||||
boost := p.boostWorkers |
|
||||||
if (boost+p.numberOfWorkers) > p.maxNumberOfWorkers && p.maxNumberOfWorkers >= 0 { |
|
||||||
boost = p.maxNumberOfWorkers - p.numberOfWorkers |
|
||||||
} |
|
||||||
if mq != nil { |
|
||||||
log.Debug("WorkerPool: %d (for %s) Channel blocked for %v - adding %d temporary workers for %s, block timeout now %v", p.qid, mq.Name, ourTimeout, boost, p.boostTimeout, p.blockTimeout) |
|
||||||
|
|
||||||
start := time.Now() |
|
||||||
pid := mq.RegisterWorkers(boost, start, true, start.Add(p.boostTimeout), boostCtxCancel, false) |
|
||||||
go func() { |
|
||||||
<-boostCtx.Done() |
|
||||||
mq.RemoveWorkers(pid) |
|
||||||
boostCtxCancel() |
|
||||||
}() |
|
||||||
} else { |
|
||||||
log.Debug("WorkerPool: %d Channel blocked for %v - adding %d temporary workers for %s, block timeout now %v", p.qid, ourTimeout, p.boostWorkers, p.boostTimeout, p.blockTimeout) |
|
||||||
} |
|
||||||
go func() { |
|
||||||
<-time.After(p.boostTimeout) |
|
||||||
boostCtxCancel() |
|
||||||
p.lock.Lock() |
|
||||||
p.blockTimeout /= 2 |
|
||||||
p.lock.Unlock() |
|
||||||
}() |
|
||||||
p.lock.Unlock() |
|
||||||
p.addWorkers(boostCtx, boostCtxCancel, boost) |
|
||||||
p.dataChan <- data |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// NumberOfWorkers returns the number of current workers in the pool
|
|
||||||
func (p *WorkerPool) NumberOfWorkers() int { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.numberOfWorkers |
|
||||||
} |
|
||||||
|
|
||||||
// NumberInQueue returns the number of items in the queue
|
|
||||||
func (p *WorkerPool) NumberInQueue() int64 { |
|
||||||
return atomic.LoadInt64(&p.numInQueue) |
|
||||||
} |
|
||||||
|
|
||||||
// MaxNumberOfWorkers returns the maximum number of workers automatically added to the pool
|
|
||||||
func (p *WorkerPool) MaxNumberOfWorkers() int { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.maxNumberOfWorkers |
|
||||||
} |
|
||||||
|
|
||||||
// BoostWorkers returns the number of workers for a boost
|
|
||||||
func (p *WorkerPool) BoostWorkers() int { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.boostWorkers |
|
||||||
} |
|
||||||
|
|
||||||
// BoostTimeout returns the timeout of the next boost
|
|
||||||
func (p *WorkerPool) BoostTimeout() time.Duration { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.boostTimeout |
|
||||||
} |
|
||||||
|
|
||||||
// BlockTimeout returns the timeout til the next boost
|
|
||||||
func (p *WorkerPool) BlockTimeout() time.Duration { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.blockTimeout |
|
||||||
} |
|
||||||
|
|
||||||
// SetPoolSettings sets the setable boost values
|
|
||||||
func (p *WorkerPool) SetPoolSettings(maxNumberOfWorkers, boostWorkers int, timeout time.Duration) { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
p.maxNumberOfWorkers = maxNumberOfWorkers |
|
||||||
p.boostWorkers = boostWorkers |
|
||||||
p.boostTimeout = timeout |
|
||||||
} |
|
||||||
|
|
||||||
// SetMaxNumberOfWorkers sets the maximum number of workers automatically added to the pool
|
|
||||||
// Changing this number will not change the number of current workers but will change the limit
|
|
||||||
// for future additions
|
|
||||||
func (p *WorkerPool) SetMaxNumberOfWorkers(newMax int) { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
p.maxNumberOfWorkers = newMax |
|
||||||
} |
|
||||||
|
|
||||||
func (p *WorkerPool) commonRegisterWorkers(number int, timeout time.Duration, isFlusher bool) (context.Context, context.CancelFunc) { |
|
||||||
var ctx context.Context |
|
||||||
var cancel context.CancelFunc |
|
||||||
start := time.Now() |
|
||||||
end := start |
|
||||||
hasTimeout := false |
|
||||||
if timeout > 0 { |
|
||||||
ctx, cancel = context.WithTimeout(p.baseCtx, timeout) |
|
||||||
end = start.Add(timeout) |
|
||||||
hasTimeout = true |
|
||||||
} else { |
|
||||||
ctx, cancel = context.WithCancel(p.baseCtx) |
|
||||||
} |
|
||||||
|
|
||||||
mq := GetManager().GetManagedQueue(p.qid) |
|
||||||
if mq != nil { |
|
||||||
pid := mq.RegisterWorkers(number, start, hasTimeout, end, cancel, isFlusher) |
|
||||||
log.Trace("WorkerPool: %d (for %s) adding %d workers with group id: %d", p.qid, mq.Name, number, pid) |
|
||||||
return ctx, func() { |
|
||||||
mq.RemoveWorkers(pid) |
|
||||||
} |
|
||||||
} |
|
||||||
log.Trace("WorkerPool: %d adding %d workers (no group id)", p.qid, number) |
|
||||||
|
|
||||||
return ctx, cancel |
|
||||||
} |
|
||||||
|
|
||||||
// AddWorkers adds workers to the pool - this allows the number of workers to go above the limit
|
|
||||||
func (p *WorkerPool) AddWorkers(number int, timeout time.Duration) context.CancelFunc { |
|
||||||
ctx, cancel := p.commonRegisterWorkers(number, timeout, false) |
|
||||||
p.addWorkers(ctx, cancel, number) |
|
||||||
return cancel |
|
||||||
} |
|
||||||
|
|
||||||
// addWorkers adds workers to the pool
|
|
||||||
func (p *WorkerPool) addWorkers(ctx context.Context, cancel context.CancelFunc, number int) { |
|
||||||
for i := 0; i < number; i++ { |
|
||||||
p.lock.Lock() |
|
||||||
if p.cond == nil { |
|
||||||
p.cond = sync.NewCond(&p.lock) |
|
||||||
} |
|
||||||
p.numberOfWorkers++ |
|
||||||
p.lock.Unlock() |
|
||||||
go func() { |
|
||||||
pprof.SetGoroutineLabels(ctx) |
|
||||||
p.doWork(ctx) |
|
||||||
|
|
||||||
p.lock.Lock() |
|
||||||
p.numberOfWorkers-- |
|
||||||
if p.numberOfWorkers == 0 { |
|
||||||
p.cond.Broadcast() |
|
||||||
cancel() |
|
||||||
} else if p.numberOfWorkers < 0 { |
|
||||||
// numberOfWorkers can't go negative but...
|
|
||||||
log.Warn("Number of Workers < 0 for QID %d - this shouldn't happen", p.qid) |
|
||||||
p.numberOfWorkers = 0 |
|
||||||
p.cond.Broadcast() |
|
||||||
cancel() |
|
||||||
} |
|
||||||
select { |
|
||||||
case <-p.baseCtx.Done(): |
|
||||||
// Don't warn or check for ongoing work if the baseCtx is shutdown
|
|
||||||
case <-p.paused: |
|
||||||
// Don't warn or check for ongoing work if the pool is paused
|
|
||||||
default: |
|
||||||
if p.hasNoWorkerScaling() { |
|
||||||
log.Warn( |
|
||||||
"Queue: %d is configured to be non-scaling and has no workers - this configuration is likely incorrect.\n"+ |
|
||||||
"The queue will be paused to prevent data-loss with the assumption that you will add workers and unpause as required.", p.qid) |
|
||||||
p.pause() |
|
||||||
} else if p.numberOfWorkers == 0 && atomic.LoadInt64(&p.numInQueue) > 0 { |
|
||||||
// OK there are no workers but... there's still work to be done -> Reboost
|
|
||||||
p.zeroBoost() |
|
||||||
// p.lock will be unlocked by zeroBoost
|
|
||||||
return |
|
||||||
} |
|
||||||
} |
|
||||||
p.lock.Unlock() |
|
||||||
}() |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Wait for WorkerPool to finish
|
|
||||||
func (p *WorkerPool) Wait() { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
if p.cond == nil { |
|
||||||
p.cond = sync.NewCond(&p.lock) |
|
||||||
} |
|
||||||
if p.numberOfWorkers <= 0 { |
|
||||||
return |
|
||||||
} |
|
||||||
p.cond.Wait() |
|
||||||
} |
|
||||||
|
|
||||||
// IsPaused returns if the pool is paused
|
|
||||||
func (p *WorkerPool) IsPaused() bool { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
select { |
|
||||||
case <-p.paused: |
|
||||||
return true |
|
||||||
default: |
|
||||||
return false |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// IsPausedIsResumed returns if the pool is paused and a channel that is closed when it is resumed
|
|
||||||
func (p *WorkerPool) IsPausedIsResumed() (<-chan struct{}, <-chan struct{}) { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
return p.paused, p.resumed |
|
||||||
} |
|
||||||
|
|
||||||
// Pause pauses the WorkerPool
|
|
||||||
func (p *WorkerPool) Pause() { |
|
||||||
p.lock.Lock() |
|
||||||
defer p.lock.Unlock() |
|
||||||
p.pause() |
|
||||||
} |
|
||||||
|
|
||||||
func (p *WorkerPool) pause() { |
|
||||||
select { |
|
||||||
case <-p.paused: |
|
||||||
default: |
|
||||||
p.resumed = make(chan struct{}) |
|
||||||
close(p.paused) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// Resume resumes the WorkerPool
|
|
||||||
func (p *WorkerPool) Resume() { |
|
||||||
p.lock.Lock() // can't defer unlock because of the zeroBoost at the end
|
|
||||||
select { |
|
||||||
case <-p.resumed: |
|
||||||
// already resumed - there's nothing to do
|
|
||||||
p.lock.Unlock() |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
p.paused = make(chan struct{}) |
|
||||||
close(p.resumed) |
|
||||||
|
|
||||||
// OK now we need to check if we need to add some workers...
|
|
||||||
if p.numberOfWorkers > 0 || p.hasNoWorkerScaling() || atomic.LoadInt64(&p.numInQueue) == 0 { |
|
||||||
// We either have workers, can't scale or there's no work to be done -> so just resume
|
|
||||||
p.lock.Unlock() |
|
||||||
return |
|
||||||
} |
|
||||||
|
|
||||||
// OK we got some work but no workers we need to think about boosting
|
|
||||||
select { |
|
||||||
case <-p.baseCtx.Done(): |
|
||||||
// don't bother boosting if the baseCtx is done
|
|
||||||
p.lock.Unlock() |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
// OK we'd better add some boost workers!
|
|
||||||
p.zeroBoost() |
|
||||||
// p.zeroBoost will unlock the lock
|
|
||||||
} |
|
||||||
|
|
||||||
// CleanUp will drain the remaining contents of the channel
|
|
||||||
// This should be called after AddWorkers context is closed
|
|
||||||
func (p *WorkerPool) CleanUp(ctx context.Context) { |
|
||||||
log.Trace("WorkerPool: %d CleanUp", p.qid) |
|
||||||
close(p.dataChan) |
|
||||||
for data := range p.dataChan { |
|
||||||
if unhandled := p.handle(data); unhandled != nil { |
|
||||||
if unhandled != nil { |
|
||||||
log.Error("Unhandled Data in clean-up of queue %d", p.qid) |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
atomic.AddInt64(&p.numInQueue, -1) |
|
||||||
select { |
|
||||||
case <-ctx.Done(): |
|
||||||
log.Warn("WorkerPool: %d Cleanup context closed before finishing clean-up", p.qid) |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
} |
|
||||||
log.Trace("WorkerPool: %d CleanUp Done", p.qid) |
|
||||||
} |
|
||||||
|
|
||||||
// Flush flushes the channel with a timeout - the Flush worker will be registered as a flush worker with the manager
|
|
||||||
func (p *WorkerPool) Flush(timeout time.Duration) error { |
|
||||||
ctx, cancel := p.commonRegisterWorkers(1, timeout, true) |
|
||||||
defer cancel() |
|
||||||
return p.FlushWithContext(ctx) |
|
||||||
} |
|
||||||
|
|
||||||
// IsEmpty returns if true if the worker queue is empty
|
|
||||||
func (p *WorkerPool) IsEmpty() bool { |
|
||||||
return atomic.LoadInt64(&p.numInQueue) == 0 |
|
||||||
} |
|
||||||
|
|
||||||
// contextError returns either ctx.Done(), the base context's error or nil
|
|
||||||
func (p *WorkerPool) contextError(ctx context.Context) error { |
|
||||||
select { |
|
||||||
case <-p.baseCtx.Done(): |
|
||||||
return p.baseCtx.Err() |
|
||||||
case <-ctx.Done(): |
|
||||||
return ctx.Err() |
|
||||||
default: |
|
||||||
return nil |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
// FlushWithContext is very similar to CleanUp but it will return as soon as the dataChan is empty
|
|
||||||
// NB: The worker will not be registered with the manager.
|
|
||||||
func (p *WorkerPool) FlushWithContext(ctx context.Context) error { |
|
||||||
log.Trace("WorkerPool: %d Flush", p.qid) |
|
||||||
paused, _ := p.IsPausedIsResumed() |
|
||||||
for { |
|
||||||
// Because select will return any case that is satisified at random we precheck here before looking at dataChan.
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
// Ensure that even if paused that the cancelled error is still sent
|
|
||||||
return p.contextError(ctx) |
|
||||||
case <-p.baseCtx.Done(): |
|
||||||
return p.baseCtx.Err() |
|
||||||
case <-ctx.Done(): |
|
||||||
return ctx.Err() |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
return p.contextError(ctx) |
|
||||||
case data, ok := <-p.dataChan: |
|
||||||
if !ok { |
|
||||||
return nil |
|
||||||
} |
|
||||||
if unhandled := p.handle(data); unhandled != nil { |
|
||||||
log.Error("Unhandled Data whilst flushing queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1) |
|
||||||
case <-p.baseCtx.Done(): |
|
||||||
return p.baseCtx.Err() |
|
||||||
case <-ctx.Done(): |
|
||||||
return ctx.Err() |
|
||||||
default: |
|
||||||
return nil |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
func (p *WorkerPool) doWork(ctx context.Context) { |
|
||||||
pprof.SetGoroutineLabels(ctx) |
|
||||||
delay := time.Millisecond * 300 |
|
||||||
|
|
||||||
// Create a common timer - we will use this elsewhere
|
|
||||||
timer := time.NewTimer(0) |
|
||||||
util.StopTimer(timer) |
|
||||||
|
|
||||||
paused, _ := p.IsPausedIsResumed() |
|
||||||
data := make([]Data, 0, p.batchLength) |
|
||||||
for { |
|
||||||
// Because select will return any case that is satisified at random we precheck here before looking at dataChan.
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
log.Trace("Worker for Queue %d Pausing", p.qid) |
|
||||||
if len(data) > 0 { |
|
||||||
log.Trace("Handling: %d data, %v", len(data), data) |
|
||||||
if unhandled := p.handle(data...); unhandled != nil { |
|
||||||
log.Error("Unhandled Data in queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1*int64(len(data))) |
|
||||||
} |
|
||||||
_, resumed := p.IsPausedIsResumed() |
|
||||||
select { |
|
||||||
case <-resumed: |
|
||||||
paused, _ = p.IsPausedIsResumed() |
|
||||||
log.Trace("Worker for Queue %d Resuming", p.qid) |
|
||||||
util.StopTimer(timer) |
|
||||||
case <-ctx.Done(): |
|
||||||
log.Trace("Worker shutting down") |
|
||||||
return |
|
||||||
} |
|
||||||
case <-ctx.Done(): |
|
||||||
if len(data) > 0 { |
|
||||||
log.Trace("Handling: %d data, %v", len(data), data) |
|
||||||
if unhandled := p.handle(data...); unhandled != nil { |
|
||||||
log.Error("Unhandled Data in queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1*int64(len(data))) |
|
||||||
} |
|
||||||
log.Trace("Worker shutting down") |
|
||||||
return |
|
||||||
default: |
|
||||||
} |
|
||||||
|
|
||||||
select { |
|
||||||
case <-paused: |
|
||||||
// go back around
|
|
||||||
case <-ctx.Done(): |
|
||||||
if len(data) > 0 { |
|
||||||
log.Trace("Handling: %d data, %v", len(data), data) |
|
||||||
if unhandled := p.handle(data...); unhandled != nil { |
|
||||||
log.Error("Unhandled Data in queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1*int64(len(data))) |
|
||||||
} |
|
||||||
log.Trace("Worker shutting down") |
|
||||||
return |
|
||||||
case datum, ok := <-p.dataChan: |
|
||||||
if !ok { |
|
||||||
// the dataChan has been closed - we should finish up:
|
|
||||||
if len(data) > 0 { |
|
||||||
log.Trace("Handling: %d data, %v", len(data), data) |
|
||||||
if unhandled := p.handle(data...); unhandled != nil { |
|
||||||
log.Error("Unhandled Data in queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1*int64(len(data))) |
|
||||||
} |
|
||||||
log.Trace("Worker shutting down") |
|
||||||
return |
|
||||||
} |
|
||||||
data = append(data, datum) |
|
||||||
util.StopTimer(timer) |
|
||||||
|
|
||||||
if len(data) >= p.batchLength { |
|
||||||
log.Trace("Handling: %d data, %v", len(data), data) |
|
||||||
if unhandled := p.handle(data...); unhandled != nil { |
|
||||||
log.Error("Unhandled Data in queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1*int64(len(data))) |
|
||||||
data = make([]Data, 0, p.batchLength) |
|
||||||
} else { |
|
||||||
timer.Reset(delay) |
|
||||||
} |
|
||||||
case <-timer.C: |
|
||||||
delay = time.Millisecond * 100 |
|
||||||
if len(data) > 0 { |
|
||||||
log.Trace("Handling: %d data, %v", len(data), data) |
|
||||||
if unhandled := p.handle(data...); unhandled != nil { |
|
||||||
log.Error("Unhandled Data in queue %d", p.qid) |
|
||||||
} |
|
||||||
atomic.AddInt64(&p.numInQueue, -1*int64(len(data))) |
|
||||||
data = make([]Data, 0, p.batchLength) |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
@ -0,0 +1,241 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"fmt" |
||||||
|
"sync" |
||||||
|
"sync/atomic" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/graceful" |
||||||
|
"code.gitea.io/gitea/modules/json" |
||||||
|
"code.gitea.io/gitea/modules/log" |
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
) |
||||||
|
|
||||||
|
// WorkerPoolQueue is a queue that uses a pool of workers to process items
|
||||||
|
// It can use different underlying (base) queue types
|
||||||
|
type WorkerPoolQueue[T any] struct { |
||||||
|
ctxRun context.Context |
||||||
|
ctxRunCancel context.CancelFunc |
||||||
|
ctxShutdown atomic.Pointer[context.Context] |
||||||
|
shutdownDone chan struct{} |
||||||
|
|
||||||
|
origHandler HandlerFuncT[T] |
||||||
|
safeHandler HandlerFuncT[T] |
||||||
|
|
||||||
|
baseQueueType string |
||||||
|
baseConfig *BaseConfig |
||||||
|
baseQueue baseQueue |
||||||
|
|
||||||
|
batchChan chan []T |
||||||
|
flushChan chan flushType |
||||||
|
|
||||||
|
batchLength int |
||||||
|
workerNum int |
||||||
|
workerMaxNum int |
||||||
|
workerActiveNum int |
||||||
|
workerNumMu sync.Mutex |
||||||
|
} |
||||||
|
|
||||||
|
type flushType chan struct{} |
||||||
|
|
||||||
|
var _ ManagedWorkerPoolQueue = (*WorkerPoolQueue[any])(nil) |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetName() string { |
||||||
|
return q.baseConfig.ManagedName |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetType() string { |
||||||
|
return q.baseQueueType |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetItemTypeName() string { |
||||||
|
var t T |
||||||
|
return fmt.Sprintf("%T", t) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetWorkerNumber() int { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
defer q.workerNumMu.Unlock() |
||||||
|
return q.workerNum |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetWorkerActiveNumber() int { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
defer q.workerNumMu.Unlock() |
||||||
|
return q.workerActiveNum |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetWorkerMaxNumber() int { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
defer q.workerNumMu.Unlock() |
||||||
|
return q.workerMaxNum |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) SetWorkerMaxNumber(num int) { |
||||||
|
q.workerNumMu.Lock() |
||||||
|
defer q.workerNumMu.Unlock() |
||||||
|
q.workerMaxNum = num |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) GetQueueItemNumber() int { |
||||||
|
cnt, err := q.baseQueue.Len(q.ctxRun) |
||||||
|
if err != nil { |
||||||
|
log.Error("Failed to get number of items in queue %q: %v", q.GetName(), err) |
||||||
|
} |
||||||
|
return cnt |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) FlushWithContext(ctx context.Context, timeout time.Duration) (err error) { |
||||||
|
if q.isBaseQueueDummy() { |
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
log.Debug("Try to flush queue %q with timeout %v", q.GetName(), timeout) |
||||||
|
defer log.Debug("Finish flushing queue %q, err: %v", q.GetName(), err) |
||||||
|
|
||||||
|
var after <-chan time.Time |
||||||
|
after = infiniteTimerC |
||||||
|
if timeout > 0 { |
||||||
|
after = time.After(timeout) |
||||||
|
} |
||||||
|
c := make(flushType) |
||||||
|
|
||||||
|
// send flush request
|
||||||
|
// if it blocks, it means that there is a flush in progress or the queue hasn't been started yet
|
||||||
|
select { |
||||||
|
case q.flushChan <- c: |
||||||
|
case <-ctx.Done(): |
||||||
|
return ctx.Err() |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
return q.ctxRun.Err() |
||||||
|
case <-after: |
||||||
|
return context.DeadlineExceeded |
||||||
|
} |
||||||
|
|
||||||
|
// wait for flush to finish
|
||||||
|
select { |
||||||
|
case <-c: |
||||||
|
return nil |
||||||
|
case <-ctx.Done(): |
||||||
|
return ctx.Err() |
||||||
|
case <-q.ctxRun.Done(): |
||||||
|
return q.ctxRun.Err() |
||||||
|
case <-after: |
||||||
|
return context.DeadlineExceeded |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) marshal(data T) []byte { |
||||||
|
bs, err := json.Marshal(data) |
||||||
|
if err != nil { |
||||||
|
log.Error("Failed to marshal item for queue %q: %v", q.GetName(), err) |
||||||
|
return nil |
||||||
|
} |
||||||
|
return bs |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) unmarshal(data []byte) (t T, ok bool) { |
||||||
|
if err := json.Unmarshal(data, &t); err != nil { |
||||||
|
log.Error("Failed to unmarshal item from queue %q: %v", q.GetName(), err) |
||||||
|
return t, false |
||||||
|
} |
||||||
|
return t, true |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) isBaseQueueDummy() bool { |
||||||
|
_, isDummy := q.baseQueue.(*baseDummy) |
||||||
|
return isDummy |
||||||
|
} |
||||||
|
|
||||||
|
// Push adds an item to the queue, it may block for a while and then returns an error if the queue is full
|
||||||
|
func (q *WorkerPoolQueue[T]) Push(data T) error { |
||||||
|
if q.isBaseQueueDummy() && q.safeHandler != nil { |
||||||
|
// FIXME: the "immediate" queue is only for testing, but it really causes problems because its behavior is different from a real queue.
|
||||||
|
// Even if tests pass, it doesn't mean that there is no bug in code.
|
||||||
|
if data, ok := q.unmarshal(q.marshal(data)); ok { |
||||||
|
q.safeHandler(data) |
||||||
|
} |
||||||
|
} |
||||||
|
return q.baseQueue.PushItem(q.ctxRun, q.marshal(data)) |
||||||
|
} |
||||||
|
|
||||||
|
// Has only works for unique queues. Keep in mind that this check may not be reliable (due to lacking of proper transaction support)
|
||||||
|
// There could be a small chance that duplicate items appear in the queue
|
||||||
|
func (q *WorkerPoolQueue[T]) Has(data T) (bool, error) { |
||||||
|
return q.baseQueue.HasItem(q.ctxRun, q.marshal(data)) |
||||||
|
} |
||||||
|
|
||||||
|
func (q *WorkerPoolQueue[T]) Run(atShutdown, atTerminate func(func())) { |
||||||
|
atShutdown(func() { |
||||||
|
// in case some queue handlers are slow or have hanging bugs, at most wait for a short time
|
||||||
|
q.ShutdownWait(1 * time.Second) |
||||||
|
}) |
||||||
|
q.doRun() |
||||||
|
} |
||||||
|
|
||||||
|
// ShutdownWait shuts down the queue, waits for all workers to finish their jobs, and pushes the unhandled items back to the base queue
|
||||||
|
// It waits for all workers (handlers) to finish their jobs, in case some buggy handlers would hang forever, a reasonable timeout is needed
|
||||||
|
func (q *WorkerPoolQueue[T]) ShutdownWait(timeout time.Duration) { |
||||||
|
shutdownCtx, shutdownCtxCancel := context.WithTimeout(context.Background(), timeout) |
||||||
|
defer shutdownCtxCancel() |
||||||
|
if q.ctxShutdown.CompareAndSwap(nil, &shutdownCtx) { |
||||||
|
q.ctxRunCancel() |
||||||
|
} |
||||||
|
<-q.shutdownDone |
||||||
|
} |
||||||
|
|
||||||
|
func getNewQueueFn(t string) (string, func(cfg *BaseConfig, unique bool) (baseQueue, error)) { |
||||||
|
switch t { |
||||||
|
case "dummy", "immediate": |
||||||
|
return t, newBaseDummy |
||||||
|
case "channel": |
||||||
|
return t, newBaseChannelGeneric |
||||||
|
case "redis": |
||||||
|
return t, newBaseRedisGeneric |
||||||
|
default: // level(leveldb,levelqueue,persistable-channel)
|
||||||
|
return "level", newBaseLevelQueueGeneric |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func NewWorkerPoolQueueBySetting[T any](name string, queueSetting setting.QueueSettings, handler HandlerFuncT[T], unique bool) (*WorkerPoolQueue[T], error) { |
||||||
|
if handler == nil { |
||||||
|
log.Debug("Use dummy queue for %q because handler is nil and caller doesn't want to process the queue items", name) |
||||||
|
queueSetting.Type = "dummy" |
||||||
|
} |
||||||
|
|
||||||
|
var w WorkerPoolQueue[T] |
||||||
|
var err error |
||||||
|
queueType, newQueueFn := getNewQueueFn(queueSetting.Type) |
||||||
|
w.baseQueueType = queueType |
||||||
|
w.baseConfig = toBaseConfig(name, queueSetting) |
||||||
|
w.baseQueue, err = newQueueFn(w.baseConfig, unique) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
log.Trace("Created queue %q of type %q", name, queueType) |
||||||
|
|
||||||
|
w.ctxRun, w.ctxRunCancel = context.WithCancel(graceful.GetManager().ShutdownContext()) |
||||||
|
w.batchChan = make(chan []T) |
||||||
|
w.flushChan = make(chan flushType) |
||||||
|
w.shutdownDone = make(chan struct{}) |
||||||
|
w.workerMaxNum = queueSetting.MaxWorkers |
||||||
|
w.batchLength = queueSetting.BatchLength |
||||||
|
|
||||||
|
w.origHandler = handler |
||||||
|
w.safeHandler = func(t ...T) (unhandled []T) { |
||||||
|
defer func() { |
||||||
|
err := recover() |
||||||
|
if err != nil { |
||||||
|
log.Error("Recovered from panic in queue %q handler: %v\n%s", name, err, log.Stack(2)) |
||||||
|
} |
||||||
|
}() |
||||||
|
return w.origHandler(t...) |
||||||
|
} |
||||||
|
|
||||||
|
return &w, nil |
||||||
|
} |
@ -0,0 +1,260 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package queue |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"strconv" |
||||||
|
"sync" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert" |
||||||
|
) |
||||||
|
|
||||||
|
func runWorkerPoolQueue[T any](q *WorkerPoolQueue[T]) func() { |
||||||
|
var stop func() |
||||||
|
started := make(chan struct{}) |
||||||
|
stopped := make(chan struct{}) |
||||||
|
go func() { |
||||||
|
q.Run(func(f func()) { stop = f; close(started) }, nil) |
||||||
|
close(stopped) |
||||||
|
}() |
||||||
|
<-started |
||||||
|
return func() { |
||||||
|
stop() |
||||||
|
<-stopped |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func TestWorkerPoolQueueUnhandled(t *testing.T) { |
||||||
|
oldUnhandledItemRequeueDuration := unhandledItemRequeueDuration.Load() |
||||||
|
unhandledItemRequeueDuration.Store(0) |
||||||
|
defer unhandledItemRequeueDuration.Store(oldUnhandledItemRequeueDuration) |
||||||
|
|
||||||
|
mu := sync.Mutex{} |
||||||
|
|
||||||
|
test := func(t *testing.T, queueSetting setting.QueueSettings) { |
||||||
|
queueSetting.Length = 100 |
||||||
|
queueSetting.Type = "channel" |
||||||
|
queueSetting.Datadir = t.TempDir() + "/test-queue" |
||||||
|
m := map[int]int{} |
||||||
|
|
||||||
|
// odds are handled once, evens are handled twice
|
||||||
|
handler := func(items ...int) (unhandled []int) { |
||||||
|
testRecorder.Record("handle:%v", items) |
||||||
|
for _, item := range items { |
||||||
|
mu.Lock() |
||||||
|
if item%2 == 0 && m[item] == 0 { |
||||||
|
unhandled = append(unhandled, item) |
||||||
|
} |
||||||
|
m[item]++ |
||||||
|
mu.Unlock() |
||||||
|
} |
||||||
|
return unhandled |
||||||
|
} |
||||||
|
|
||||||
|
q, _ := NewWorkerPoolQueueBySetting("test-workpoolqueue", queueSetting, handler, false) |
||||||
|
stop := runWorkerPoolQueue(q) |
||||||
|
for i := 0; i < queueSetting.Length; i++ { |
||||||
|
testRecorder.Record("push:%v", i) |
||||||
|
assert.NoError(t, q.Push(i)) |
||||||
|
} |
||||||
|
assert.NoError(t, q.FlushWithContext(context.Background(), 0)) |
||||||
|
stop() |
||||||
|
|
||||||
|
ok := true |
||||||
|
for i := 0; i < queueSetting.Length; i++ { |
||||||
|
if i%2 == 0 { |
||||||
|
ok = ok && assert.EqualValues(t, 2, m[i], "test %s: item %d", t.Name(), i) |
||||||
|
} else { |
||||||
|
ok = ok && assert.EqualValues(t, 1, m[i], "test %s: item %d", t.Name(), i) |
||||||
|
} |
||||||
|
} |
||||||
|
if !ok { |
||||||
|
t.Logf("m: %v", m) |
||||||
|
t.Logf("records: %v", testRecorder.Records()) |
||||||
|
} |
||||||
|
testRecorder.Reset() |
||||||
|
} |
||||||
|
|
||||||
|
runCount := 2 // we can run these tests even hundreds times to see its stability
|
||||||
|
t.Run("1/1", func(t *testing.T) { |
||||||
|
for i := 0; i < runCount; i++ { |
||||||
|
test(t, setting.QueueSettings{BatchLength: 1, MaxWorkers: 1}) |
||||||
|
} |
||||||
|
}) |
||||||
|
t.Run("3/1", func(t *testing.T) { |
||||||
|
for i := 0; i < runCount; i++ { |
||||||
|
test(t, setting.QueueSettings{BatchLength: 3, MaxWorkers: 1}) |
||||||
|
} |
||||||
|
}) |
||||||
|
t.Run("4/5", func(t *testing.T) { |
||||||
|
for i := 0; i < runCount; i++ { |
||||||
|
test(t, setting.QueueSettings{BatchLength: 4, MaxWorkers: 5}) |
||||||
|
} |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func TestWorkerPoolQueuePersistence(t *testing.T) { |
||||||
|
runCount := 2 // we can run these tests even hundreds times to see its stability
|
||||||
|
t.Run("1/1", func(t *testing.T) { |
||||||
|
for i := 0; i < runCount; i++ { |
||||||
|
testWorkerPoolQueuePersistence(t, setting.QueueSettings{BatchLength: 1, MaxWorkers: 1, Length: 100}) |
||||||
|
} |
||||||
|
}) |
||||||
|
t.Run("3/1", func(t *testing.T) { |
||||||
|
for i := 0; i < runCount; i++ { |
||||||
|
testWorkerPoolQueuePersistence(t, setting.QueueSettings{BatchLength: 3, MaxWorkers: 1, Length: 100}) |
||||||
|
} |
||||||
|
}) |
||||||
|
t.Run("4/5", func(t *testing.T) { |
||||||
|
for i := 0; i < runCount; i++ { |
||||||
|
testWorkerPoolQueuePersistence(t, setting.QueueSettings{BatchLength: 4, MaxWorkers: 5, Length: 100}) |
||||||
|
} |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func testWorkerPoolQueuePersistence(t *testing.T, queueSetting setting.QueueSettings) { |
||||||
|
testCount := queueSetting.Length |
||||||
|
queueSetting.Type = "level" |
||||||
|
queueSetting.Datadir = t.TempDir() + "/test-queue" |
||||||
|
|
||||||
|
mu := sync.Mutex{} |
||||||
|
|
||||||
|
var tasksQ1, tasksQ2 []string |
||||||
|
q1 := func() { |
||||||
|
startWhenAllReady := make(chan struct{}) // only start data consuming when the "testCount" tasks are all pushed into queue
|
||||||
|
stopAt20Shutdown := make(chan struct{}) // stop and shutdown at the 20th item
|
||||||
|
|
||||||
|
testHandler := func(data ...string) []string { |
||||||
|
<-startWhenAllReady |
||||||
|
time.Sleep(10 * time.Millisecond) |
||||||
|
for _, s := range data { |
||||||
|
mu.Lock() |
||||||
|
tasksQ1 = append(tasksQ1, s) |
||||||
|
mu.Unlock() |
||||||
|
|
||||||
|
if s == "task-20" { |
||||||
|
close(stopAt20Shutdown) |
||||||
|
} |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
q, _ := NewWorkerPoolQueueBySetting("pr_patch_checker_test", queueSetting, testHandler, true) |
||||||
|
stop := runWorkerPoolQueue(q) |
||||||
|
for i := 0; i < testCount; i++ { |
||||||
|
_ = q.Push("task-" + strconv.Itoa(i)) |
||||||
|
} |
||||||
|
close(startWhenAllReady) |
||||||
|
<-stopAt20Shutdown // it's possible to have more than 20 tasks executed
|
||||||
|
stop() |
||||||
|
} |
||||||
|
|
||||||
|
q1() // run some tasks and shutdown at an intermediate point
|
||||||
|
|
||||||
|
time.Sleep(100 * time.Millisecond) // because the handler in q1 has a slight delay, we need to wait for it to finish
|
||||||
|
|
||||||
|
q2 := func() { |
||||||
|
testHandler := func(data ...string) []string { |
||||||
|
for _, s := range data { |
||||||
|
mu.Lock() |
||||||
|
tasksQ2 = append(tasksQ2, s) |
||||||
|
mu.Unlock() |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
q, _ := NewWorkerPoolQueueBySetting("pr_patch_checker_test", queueSetting, testHandler, true) |
||||||
|
stop := runWorkerPoolQueue(q) |
||||||
|
assert.NoError(t, q.FlushWithContext(context.Background(), 0)) |
||||||
|
stop() |
||||||
|
} |
||||||
|
|
||||||
|
q2() // restart the queue to continue to execute the tasks in it
|
||||||
|
|
||||||
|
assert.NotZero(t, len(tasksQ1)) |
||||||
|
assert.NotZero(t, len(tasksQ2)) |
||||||
|
assert.EqualValues(t, testCount, len(tasksQ1)+len(tasksQ2)) |
||||||
|
} |
||||||
|
|
||||||
|
func TestWorkerPoolQueueActiveWorkers(t *testing.T) { |
||||||
|
oldWorkerIdleDuration := workerIdleDuration |
||||||
|
workerIdleDuration = 300 * time.Millisecond |
||||||
|
defer func() { |
||||||
|
workerIdleDuration = oldWorkerIdleDuration |
||||||
|
}() |
||||||
|
|
||||||
|
handler := func(items ...int) (unhandled []int) { |
||||||
|
time.Sleep(100 * time.Millisecond) |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
q, _ := NewWorkerPoolQueueBySetting("test-workpoolqueue", setting.QueueSettings{Type: "channel", BatchLength: 1, MaxWorkers: 1, Length: 100}, handler, false) |
||||||
|
stop := runWorkerPoolQueue(q) |
||||||
|
for i := 0; i < 5; i++ { |
||||||
|
assert.NoError(t, q.Push(i)) |
||||||
|
} |
||||||
|
|
||||||
|
time.Sleep(50 * time.Millisecond) |
||||||
|
assert.EqualValues(t, 1, q.GetWorkerNumber()) |
||||||
|
assert.EqualValues(t, 1, q.GetWorkerActiveNumber()) |
||||||
|
time.Sleep(500 * time.Millisecond) |
||||||
|
assert.EqualValues(t, 1, q.GetWorkerNumber()) |
||||||
|
assert.EqualValues(t, 0, q.GetWorkerActiveNumber()) |
||||||
|
time.Sleep(workerIdleDuration) |
||||||
|
assert.EqualValues(t, 1, q.GetWorkerNumber()) // there is at least one worker after the queue begins working
|
||||||
|
stop() |
||||||
|
|
||||||
|
q, _ = NewWorkerPoolQueueBySetting("test-workpoolqueue", setting.QueueSettings{Type: "channel", BatchLength: 1, MaxWorkers: 3, Length: 100}, handler, false) |
||||||
|
stop = runWorkerPoolQueue(q) |
||||||
|
for i := 0; i < 15; i++ { |
||||||
|
assert.NoError(t, q.Push(i)) |
||||||
|
} |
||||||
|
|
||||||
|
time.Sleep(50 * time.Millisecond) |
||||||
|
assert.EqualValues(t, 3, q.GetWorkerNumber()) |
||||||
|
assert.EqualValues(t, 3, q.GetWorkerActiveNumber()) |
||||||
|
time.Sleep(500 * time.Millisecond) |
||||||
|
assert.EqualValues(t, 3, q.GetWorkerNumber()) |
||||||
|
assert.EqualValues(t, 0, q.GetWorkerActiveNumber()) |
||||||
|
time.Sleep(workerIdleDuration) |
||||||
|
assert.EqualValues(t, 1, q.GetWorkerNumber()) // there is at least one worker after the queue begins working
|
||||||
|
stop() |
||||||
|
} |
||||||
|
|
||||||
|
func TestWorkerPoolQueueShutdown(t *testing.T) { |
||||||
|
oldUnhandledItemRequeueDuration := unhandledItemRequeueDuration.Load() |
||||||
|
unhandledItemRequeueDuration.Store(int64(100 * time.Millisecond)) |
||||||
|
defer unhandledItemRequeueDuration.Store(oldUnhandledItemRequeueDuration) |
||||||
|
|
||||||
|
// simulate a slow handler, it doesn't handle any item (all items will be pushed back to the queue)
|
||||||
|
handlerCalled := make(chan struct{}) |
||||||
|
handler := func(items ...int) (unhandled []int) { |
||||||
|
if items[0] == 0 { |
||||||
|
close(handlerCalled) |
||||||
|
} |
||||||
|
time.Sleep(100 * time.Millisecond) |
||||||
|
return items |
||||||
|
} |
||||||
|
|
||||||
|
qs := setting.QueueSettings{Type: "level", Datadir: t.TempDir() + "/queue", BatchLength: 3, MaxWorkers: 4, Length: 20} |
||||||
|
q, _ := NewWorkerPoolQueueBySetting("test-workpoolqueue", qs, handler, false) |
||||||
|
stop := runWorkerPoolQueue(q) |
||||||
|
for i := 0; i < qs.Length; i++ { |
||||||
|
assert.NoError(t, q.Push(i)) |
||||||
|
} |
||||||
|
<-handlerCalled |
||||||
|
time.Sleep(50 * time.Millisecond) // wait for a while to make sure all workers are active
|
||||||
|
assert.EqualValues(t, 4, q.GetWorkerActiveNumber()) |
||||||
|
stop() // stop triggers shutdown
|
||||||
|
assert.EqualValues(t, 0, q.GetWorkerActiveNumber()) |
||||||
|
|
||||||
|
// no item was ever handled, so we still get all of them again
|
||||||
|
q, _ = NewWorkerPoolQueueBySetting("test-workpoolqueue", qs, handler, false) |
||||||
|
assert.EqualValues(t, 20, q.GetQueueItemNumber()) |
||||||
|
} |
@ -0,0 +1,59 @@ |
|||||||
|
// Copyright 2023 The Gitea Authors. All rights reserved.
|
||||||
|
// SPDX-License-Identifier: MIT
|
||||||
|
|
||||||
|
package admin |
||||||
|
|
||||||
|
import ( |
||||||
|
"net/http" |
||||||
|
"strconv" |
||||||
|
|
||||||
|
"code.gitea.io/gitea/modules/context" |
||||||
|
"code.gitea.io/gitea/modules/queue" |
||||||
|
"code.gitea.io/gitea/modules/setting" |
||||||
|
) |
||||||
|
|
||||||
|
// Queue shows details for a specific queue
|
||||||
|
func Queue(ctx *context.Context) { |
||||||
|
qid := ctx.ParamsInt64("qid") |
||||||
|
mq := queue.GetManager().GetManagedQueue(qid) |
||||||
|
if mq == nil { |
||||||
|
ctx.Status(http.StatusNotFound) |
||||||
|
return |
||||||
|
} |
||||||
|
ctx.Data["Title"] = ctx.Tr("admin.monitor.queue", mq.GetName()) |
||||||
|
ctx.Data["PageIsAdminMonitor"] = true |
||||||
|
ctx.Data["Queue"] = mq |
||||||
|
ctx.HTML(http.StatusOK, tplQueueManage) |
||||||
|
} |
||||||
|
|
||||||
|
// QueueSet sets the maximum number of workers and other settings for this queue
|
||||||
|
func QueueSet(ctx *context.Context) { |
||||||
|
qid := ctx.ParamsInt64("qid") |
||||||
|
mq := queue.GetManager().GetManagedQueue(qid) |
||||||
|
if mq == nil { |
||||||
|
ctx.Status(http.StatusNotFound) |
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
maxNumberStr := ctx.FormString("max-number") |
||||||
|
|
||||||
|
var err error |
||||||
|
var maxNumber int |
||||||
|
if len(maxNumberStr) > 0 { |
||||||
|
maxNumber, err = strconv.Atoi(maxNumberStr) |
||||||
|
if err != nil { |
||||||
|
ctx.Flash.Error(ctx.Tr("admin.monitor.queue.settings.maxnumberworkers.error")) |
||||||
|
ctx.Redirect(setting.AppSubURL + "/admin/monitor/queue/" + strconv.FormatInt(qid, 10)) |
||||||
|
return |
||||||
|
} |
||||||
|
if maxNumber < -1 { |
||||||
|
maxNumber = -1 |
||||||
|
} |
||||||
|
} else { |
||||||
|
maxNumber = mq.GetWorkerMaxNumber() |
||||||
|
} |
||||||
|
|
||||||
|
mq.SetWorkerMaxNumber(maxNumber) |
||||||
|
ctx.Flash.Success(ctx.Tr("admin.monitor.queue.settings.changed")) |
||||||
|
ctx.Redirect(setting.AppSubURL + "/admin/monitor/queue/" + strconv.FormatInt(qid, 10)) |
||||||
|
} |
@ -0,0 +1,48 @@ |
|||||||
|
{{template "admin/layout_head" (dict "ctxData" . "pageClass" "admin monitor")}} |
||||||
|
<div class="admin-setting-content"> |
||||||
|
<h4 class="ui top attached header"> |
||||||
|
{{.locale.Tr "admin.monitor.queue" .Queue.GetName}} |
||||||
|
</h4> |
||||||
|
<div class="ui attached table segment"> |
||||||
|
<table class="ui very basic striped table"> |
||||||
|
<thead> |
||||||
|
<tr> |
||||||
|
<th>{{.locale.Tr "admin.monitor.queue.name"}}</th> |
||||||
|
<th>{{.locale.Tr "admin.monitor.queue.type"}}</th> |
||||||
|
<th>{{.locale.Tr "admin.monitor.queue.exemplar"}}</th> |
||||||
|
<th>{{.locale.Tr "admin.monitor.queue.numberworkers"}}</th> |
||||||
|
<th>{{.locale.Tr "admin.monitor.queue.maxnumberworkers"}}</th> |
||||||
|
<th>{{.locale.Tr "admin.monitor.queue.numberinqueue"}}</th> |
||||||
|
</tr> |
||||||
|
</thead> |
||||||
|
<tbody> |
||||||
|
<tr> |
||||||
|
<td>{{.Queue.GetName}}</td> |
||||||
|
<td>{{.Queue.GetType}}</td> |
||||||
|
<td>{{.Queue.GetItemTypeName}}</td> |
||||||
|
<td>{{$sum := .Queue.GetWorkerNumber}}{{if lt $sum 0}}-{{else}}{{$sum}}{{end}}</td> |
||||||
|
<td>{{if lt $sum 0}}-{{else}}{{.Queue.GetWorkerMaxNumber}}{{end}}</td> |
||||||
|
<td>{{$sum = .Queue.GetQueueItemNumber}}{{if lt $sum 0}}-{{else}}{{$sum}}{{end}}</td> |
||||||
|
</tr> |
||||||
|
</tbody> |
||||||
|
</table> |
||||||
|
</div> |
||||||
|
|
||||||
|
<h4 class="ui top attached header"> |
||||||
|
{{.locale.Tr "admin.monitor.queue.settings.title"}} |
||||||
|
</h4> |
||||||
|
<div class="ui attached segment"> |
||||||
|
<p>{{.locale.Tr "admin.monitor.queue.settings.desc"}}</p> |
||||||
|
<form method="POST" action="{{.Link}}/set"> |
||||||
|
{{$.CsrfTokenHtml}} |
||||||
|
<div class="ui form"> |
||||||
|
<div class="inline field"> |
||||||
|
<label for="max-number">{{.locale.Tr "admin.monitor.queue.settings.maxnumberworkers"}}</label> |
||||||
|
<input name="max-number" type="text" placeholder="{{.locale.Tr "admin.monitor.queue.settings.maxnumberworkers.placeholder" .Queue.GetWorkerMaxNumber}}"> |
||||||
|
</div> |
||||||
|
<button class="ui submit button">{{.locale.Tr "admin.monitor.queue.settings.submit"}}</button> |
||||||
|
</div> |
||||||
|
</form> |
||||||
|
</div> |
||||||
|
</div> |
||||||
|
{{template "admin/layout_footer" .}} |
Loading…
Reference in new issue