mirror of
https://github.com/akvorado/akvorado.git
synced 2025-12-11 22:14:02 +01:00
outlet/kafka: fix scaler hysteresis
Previously, the scaler was scaling up and down independently. Because when scaling up/down, Kafka rebalances the topic, temporarily, we get scale down requests and the rate limiter won't stop them as it is independant from the scale up rate limiter. Instead, the rate limit for increase acts as a gracetime where everything is ignored, then between that and the rate limit for decrease, we only consider increasing the number of workers, past that, we scaling down as long as we have a majority of scale down requests (compared to steady ones). Fix #2080 (hopefully)
This commit is contained in:
@@ -16,6 +16,7 @@ identified with a specific icon:
|
||||
ClickHouse, Kafka and remote data sources (previously, `verify` was set to
|
||||
false by default)
|
||||
- 🩹 *outlet*: provide additional gracetime for a worker to send to ClickHouse
|
||||
- 🩹 *outlet*: enhance scaling up and down workers to avoid hysteresis
|
||||
- 🌱 *common*: enable block and mutex profiling
|
||||
- 🌱 *config*: rename `verify` to `skip-verify` in TLS configurations for
|
||||
ClickHouse, Kafka and remote data sources (with inverted logic)
|
||||
|
||||
1
go.mod
1
go.mod
@@ -58,7 +58,6 @@ require (
|
||||
golang.org/x/sync v0.18.0
|
||||
golang.org/x/sys v0.38.0
|
||||
golang.org/x/text v0.30.0
|
||||
golang.org/x/time v0.14.0
|
||||
google.golang.org/grpc v1.76.0
|
||||
google.golang.org/protobuf v1.36.10
|
||||
gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637
|
||||
|
||||
2
go.sum
2
go.sum
@@ -590,8 +590,6 @@ golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8=
|
||||
golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE=
|
||||
golang.org/x/text v0.30.0 h1:yznKA/E9zq54KzlzBEAWn1NXSQ8DIp/NYMy88xJjl4k=
|
||||
golang.org/x/text v0.30.0/go.mod h1:yDdHFIX9t+tORqspjENWgzaCVXgk0yYnYuSZ8UzzBVM=
|
||||
golang.org/x/time v0.14.0 h1:MRx4UaLrDotUKUdCIqzPC48t1Y9hANFKIRpNx+Te8PI=
|
||||
golang.org/x/time v0.14.0/go.mod h1:eL/Oa2bBBK0TkX57Fyni+NgnyQQN4LitPmob2Hjnqw4=
|
||||
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||
golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
|
||||
|
||||
@@ -95,6 +95,8 @@ func (w *worker) processIncomingFlow(ctx context.Context, data []byte) error {
|
||||
w.scaleRequestChan <- kafka.ScaleIncrease
|
||||
case clickhouse.WorkerStatusUnderloaded:
|
||||
w.scaleRequestChan <- kafka.ScaleDecrease
|
||||
default:
|
||||
w.scaleRequestChan <- kafka.ScaleSteady
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -27,7 +27,7 @@ type Configuration struct {
|
||||
// WorkerIncreaseRateLimit is the duration that should elapse before increasing the number of workers
|
||||
WorkerIncreaseRateLimit time.Duration `validate:"min=10s"`
|
||||
// WorkerDecreaseRateLimit is the duration that should elapse before decreasing the number of workers
|
||||
WorkerDecreaseRateLimit time.Duration `validate:"min=10s"`
|
||||
WorkerDecreaseRateLimit time.Duration `validate:"min=20s,gtfield=WorkerIncreaseRateLimit"`
|
||||
}
|
||||
|
||||
// DefaultConfiguration represents the default configuration for the Kafka exporter.
|
||||
|
||||
@@ -248,7 +248,7 @@ func TestWorkerScaling(t *testing.T) {
|
||||
c.StartWorkers(func(_ int, ch chan<- ScaleRequest) (ReceiveFunc, ShutdownFunc) {
|
||||
return func(context.Context, []byte) error {
|
||||
c := msg.Add(1)
|
||||
if c <= 5 {
|
||||
if c <= 1 {
|
||||
t.Logf("received message %d, request a scale increase", c)
|
||||
ch <- ScaleIncrease
|
||||
} else {
|
||||
@@ -273,16 +273,13 @@ func TestWorkerScaling(t *testing.T) {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
// Send 5 messages in a row
|
||||
t.Log("Send 5 messages")
|
||||
for range 5 {
|
||||
record := &kgo.Record{
|
||||
Topic: expectedTopicName,
|
||||
Value: []byte("hello"),
|
||||
}
|
||||
if results := producer.ProduceSync(context.Background(), record); results.FirstErr() != nil {
|
||||
t.Fatalf("ProduceSync() error:\n%+v", results.FirstErr())
|
||||
}
|
||||
t.Log("Send 1 message (increase)")
|
||||
record := &kgo.Record{
|
||||
Topic: expectedTopicName,
|
||||
Value: []byte("hello"),
|
||||
}
|
||||
if results := producer.ProduceSync(context.Background(), record); results.FirstErr() != nil {
|
||||
t.Fatalf("ProduceSync() error:\n%+v", results.FirstErr())
|
||||
}
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
t.Log("Check if workers increased to 9")
|
||||
@@ -296,9 +293,8 @@ func TestWorkerScaling(t *testing.T) {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
// Send 1 message
|
||||
t.Log("Send 1 message")
|
||||
for range 1 {
|
||||
t.Log("Send 10 messages (decrease)")
|
||||
for range 10 {
|
||||
record := &kgo.Record{
|
||||
Topic: expectedTopicName,
|
||||
Value: []byte("hello"),
|
||||
|
||||
@@ -6,8 +6,6 @@ package kafka
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"golang.org/x/time/rate"
|
||||
)
|
||||
|
||||
// ScaleRequest is a request to scale the workers
|
||||
@@ -18,6 +16,8 @@ const (
|
||||
ScaleIncrease ScaleRequest = iota + 1
|
||||
// ScaleDecrease is a request to decrease the number of workers
|
||||
ScaleDecrease
|
||||
// ScaleSteady is a request to keep the number of workers as is
|
||||
ScaleSteady
|
||||
)
|
||||
|
||||
// scalerConfiguration is the configuration for the scaler subcomponent
|
||||
@@ -68,32 +68,50 @@ func (s *scalerState) nextWorkerCount(request ScaleRequest, currentWorkers, minW
|
||||
// runScaler starts the automatic scaling loop
|
||||
func runScaler(ctx context.Context, config scalerConfiguration) chan<- ScaleRequest {
|
||||
ch := make(chan ScaleRequest, config.maxWorkers)
|
||||
down := rate.Sometimes{Interval: config.decreaseRateLimit}
|
||||
up := rate.Sometimes{Interval: config.increaseRateLimit}
|
||||
go func() {
|
||||
state := new(scalerState)
|
||||
var last time.Time
|
||||
var decreaseCount int
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case request := <-ch:
|
||||
switch request {
|
||||
case ScaleIncrease:
|
||||
up.Do(func() {
|
||||
// During config.increaseRateLimit, we ignore everything.
|
||||
// Between config.increaseRateLimit and
|
||||
// config.decreaseRateLimit, we only use the increase request.
|
||||
// Past this limit, we take whatever we get.
|
||||
now := time.Now()
|
||||
if last.Add(config.increaseRateLimit).After(now) {
|
||||
continue
|
||||
}
|
||||
if request == ScaleIncrease {
|
||||
current := config.getWorkerCount()
|
||||
target := state.nextWorkerCount(ScaleIncrease, current, config.minWorkers, config.maxWorkers)
|
||||
if target > current {
|
||||
config.increaseWorkers(current, target)
|
||||
}
|
||||
last = now
|
||||
decreaseCount = 0
|
||||
continue
|
||||
}
|
||||
if request == ScaleSteady {
|
||||
decreaseCount--
|
||||
}
|
||||
if last.Add(config.decreaseRateLimit).After(now) {
|
||||
continue
|
||||
}
|
||||
if request == ScaleDecrease {
|
||||
decreaseCount++
|
||||
if decreaseCount >= 10 {
|
||||
current := config.getWorkerCount()
|
||||
target := state.nextWorkerCount(request, current, config.minWorkers, config.maxWorkers)
|
||||
if target > current {
|
||||
config.increaseWorkers(current, target)
|
||||
}
|
||||
})
|
||||
case ScaleDecrease:
|
||||
down.Do(func() {
|
||||
current := config.getWorkerCount()
|
||||
target := state.nextWorkerCount(request, current, config.minWorkers, config.maxWorkers)
|
||||
target := state.nextWorkerCount(ScaleDecrease, current, config.minWorkers, config.maxWorkers)
|
||||
if target < current {
|
||||
config.decreaseWorkers(current, target)
|
||||
}
|
||||
})
|
||||
last = now
|
||||
decreaseCount = 0
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -43,11 +43,43 @@ func TestScalerWithoutRateLimiter(t *testing.T) {
|
||||
},
|
||||
expected: []int{9, 13, 15, 16},
|
||||
}, {
|
||||
name: "scale up twice, then down",
|
||||
name: "scale up twice, then down a lot",
|
||||
minWorkers: 1,
|
||||
maxWorkers: 16,
|
||||
requests: []ScaleRequest{ScaleIncrease, ScaleIncrease, ScaleDecrease},
|
||||
expected: []int{9, 13, 12},
|
||||
requests: []ScaleRequest{
|
||||
ScaleIncrease, ScaleIncrease,
|
||||
// We need 10 decrease to decrease
|
||||
ScaleDecrease, ScaleDecrease, ScaleDecrease, ScaleDecrease, ScaleDecrease,
|
||||
ScaleDecrease, ScaleDecrease, ScaleDecrease, ScaleDecrease, ScaleDecrease,
|
||||
},
|
||||
expected: []int{9, 13, 12},
|
||||
}, {
|
||||
name: "scale up twice, then down, steady, and repeat",
|
||||
minWorkers: 1,
|
||||
maxWorkers: 16,
|
||||
requests: []ScaleRequest{
|
||||
ScaleIncrease, ScaleIncrease,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady,
|
||||
},
|
||||
expected: []int{9, 13},
|
||||
}, {
|
||||
name: "scale up twice, then down, steady, down, steady, down, down, repeat",
|
||||
minWorkers: 1,
|
||||
maxWorkers: 16,
|
||||
requests: []ScaleRequest{
|
||||
ScaleIncrease, ScaleIncrease,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleDecrease,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleDecrease,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleDecrease,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleDecrease,
|
||||
ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleSteady, ScaleDecrease, ScaleDecrease,
|
||||
},
|
||||
expected: []int{9, 13, 12},
|
||||
},
|
||||
// No more tests, the state logic is tested in TestScalerState
|
||||
} {
|
||||
@@ -110,8 +142,8 @@ func TestScalerRateLimiter(t *testing.T) {
|
||||
config := scalerConfiguration{
|
||||
minWorkers: 1,
|
||||
maxWorkers: 15,
|
||||
increaseRateLimit: time.Second,
|
||||
decreaseRateLimit: time.Second,
|
||||
increaseRateLimit: time.Minute,
|
||||
decreaseRateLimit: 5 * time.Minute,
|
||||
getWorkerCount: func() int {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
@@ -133,51 +165,112 @@ func TestScalerRateLimiter(t *testing.T) {
|
||||
},
|
||||
}
|
||||
ch := runScaler(ctx, config)
|
||||
// Collapsing increases
|
||||
for range 10 {
|
||||
ch <- ScaleIncrease
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
func() {
|
||||
check := func(expected []int) {
|
||||
t.Helper()
|
||||
time.Sleep(time.Millisecond)
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
if diff := helpers.Diff(got, []int{8}); diff != "" {
|
||||
if diff := helpers.Diff(got, expected); diff != "" {
|
||||
t.Fatalf("runScaler() (-got, +want):\n%s", diff)
|
||||
}
|
||||
}()
|
||||
// Collapsing decreases
|
||||
}
|
||||
// Increase on first scale request
|
||||
ch <- ScaleIncrease
|
||||
check([]int{8})
|
||||
|
||||
// Collapsing further increases
|
||||
for range 10 {
|
||||
time.Sleep(5 * time.Second)
|
||||
ch <- ScaleIncrease
|
||||
}
|
||||
// time == 50 seconds
|
||||
check([]int{8})
|
||||
|
||||
// Then increase again
|
||||
time.Sleep(10 * time.Second)
|
||||
ch <- ScaleIncrease
|
||||
// time = 1 minute
|
||||
check([]int{8, 12})
|
||||
|
||||
// Do not decrease (too soon)
|
||||
for range 10 {
|
||||
time.Sleep(6 * time.Second)
|
||||
ch <- ScaleDecrease
|
||||
}
|
||||
// time = 1 minute
|
||||
check([]int{8, 12})
|
||||
|
||||
// Do not decrease even after 4 minutes
|
||||
for range 40 {
|
||||
time.Sleep(6 * time.Second)
|
||||
ch <- ScaleDecrease
|
||||
}
|
||||
// time = 5 minutes
|
||||
check([]int{8, 12})
|
||||
|
||||
// Decrease (5-second timeout done)
|
||||
for range 10 {
|
||||
time.Sleep(6 * time.Second)
|
||||
ch <- ScaleDecrease
|
||||
}
|
||||
// time = 6 minutes
|
||||
check([]int{8, 12, 11})
|
||||
|
||||
// Do not increase
|
||||
for range 10 {
|
||||
time.Sleep(5 * time.Second)
|
||||
ch <- ScaleIncrease
|
||||
}
|
||||
// time = 50 seconds
|
||||
check([]int{8, 12, 11})
|
||||
|
||||
// Increase after 10 more seconds
|
||||
time.Sleep(10 * time.Second)
|
||||
ch <- ScaleIncrease
|
||||
// time = 1 minute
|
||||
check([]int{8, 12, 11, 12})
|
||||
|
||||
// When mixing increase and decrease, increase
|
||||
for range 60 {
|
||||
time.Sleep(time.Second)
|
||||
ch <- ScaleIncrease
|
||||
ch <- ScaleDecrease
|
||||
}
|
||||
// time = 1 minute
|
||||
check([]int{8, 12, 11, 12, 13})
|
||||
|
||||
// When we only have a few increase at the beginning, but mostly decrease after that, decrease
|
||||
time.Sleep(55 * time.Second)
|
||||
ch <- ScaleIncrease
|
||||
ch <- ScaleIncrease
|
||||
ch <- ScaleIncrease
|
||||
ch <- ScaleIncrease
|
||||
for range 295 {
|
||||
time.Sleep(time.Second)
|
||||
ch <- ScaleDecrease
|
||||
}
|
||||
check([]int{8, 12, 11, 12, 13, 12})
|
||||
|
||||
// If we have many decrease requests at once, we decrease
|
||||
time.Sleep(300 * time.Second)
|
||||
for range 10 {
|
||||
ch <- ScaleDecrease
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
func() {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
if diff := helpers.Diff(got, []int{8, 7}); diff != "" {
|
||||
t.Fatalf("runScaler() (-got, +want):\n%s", diff)
|
||||
}
|
||||
}()
|
||||
// Still no increase
|
||||
ch <- ScaleIncrease
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
func() {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
if diff := helpers.Diff(got, []int{8, 7}); diff != "" {
|
||||
t.Fatalf("runScaler() (-got, +want):\n%s", diff)
|
||||
}
|
||||
}()
|
||||
// Rearm increase rate limiter
|
||||
time.Sleep(900 * time.Millisecond)
|
||||
ch <- ScaleIncrease
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
func() {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
if diff := helpers.Diff(got, []int{8, 7, 8}); diff != "" {
|
||||
t.Fatalf("runScaler() (-got, +want):\n%s", diff)
|
||||
}
|
||||
}()
|
||||
check([]int{8, 12, 11, 12, 13, 12, 11})
|
||||
|
||||
// But if they are mixed with steady requests, we shouldn't decrease
|
||||
time.Sleep(300 * time.Second)
|
||||
for range 10 {
|
||||
ch <- ScaleDecrease
|
||||
ch <- ScaleSteady
|
||||
}
|
||||
check([]int{8, 12, 11, 12, 13, 12, 11})
|
||||
|
||||
// But if we have less Steady than decrease, we should scale down
|
||||
for range 10 {
|
||||
ch <- ScaleDecrease
|
||||
}
|
||||
check([]int{8, 12, 11, 12, 13, 12, 11, 10})
|
||||
})
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user