mirror of https://github.com/grafana/loki
Dynamic client-side throttling to avoid object storage rate-limits (GCS only) (#10140)
**What this PR does / why we need it**: Across the various cloud providers' object storage services, there are different rate-limits implemented. Rate-limits can be imposed under multiple conditions, such as server-side scale up (ramping up from low volume to high, "local" limit), reaching some defined upper limit ("absolute" limit), etc. We cannot know apriori when these rate-limits will be imposed, so we can't set up a client-side limiter to only allow a certain number of requests through per second. Additionally, that would require global coordination between queriers - which is difficult. With the above constraints, I have instead taken inspiration from TCP's [congestion control algorithms](https://en.wikipedia.org/wiki/TCP_congestion_control). This PR implements [AIMD](https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease) (Additive Increase, Multiplicative Decrease), which is used in the congestion _avoidance_ phase of congestion control. The default window size (object store requests per second) is 2000; in other words, we skip the "slow start" phase. The controller uses the Go [`rate.Limiter`](https://pkg.go.dev/golang.org/x/time/rate), which implements the token-bucket algorithm. To put it simply: - every successful request widens the window (per second client rate-limit) - every rate-limited response reduces the window size by a backoff factor (0.5 by default, so it will halve) - when the limit has been reached, the querier will be delayed from making further requests until tokens are availablepull/10264/head
parent
176e9b7292
commit
beed298549
@ -0,0 +1,73 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"flag" |
||||
|
||||
"github.com/grafana/loki/pkg/storage/chunk/client/hedging" |
||||
) |
||||
|
||||
type Config struct { |
||||
Enabled bool `yaml:"enabled"` |
||||
Controller ControllerConfig `yaml:"controller"` |
||||
Retry RetrierConfig `yaml:"retry"` |
||||
Hedge HedgerConfig `yaml:"hedging"` |
||||
} |
||||
|
||||
func (c *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { |
||||
f.BoolVar(&c.Enabled, prefix+"enabled", false, "Use storage congestion control (default: disabled).") |
||||
|
||||
c.Controller.RegisterFlagsWithPrefix(prefix+"congestion-control.", f) |
||||
c.Retry.RegisterFlagsWithPrefix(prefix+"retry.", f) |
||||
c.Hedge.RegisterFlagsWithPrefix(prefix+"hedge.", f) |
||||
} |
||||
|
||||
type AIMD struct { |
||||
Start uint `yaml:"start"` |
||||
UpperBound uint `yaml:"upper_bound"` |
||||
BackoffFactor float64 `yaml:"backoff_factor"` |
||||
} |
||||
|
||||
type ControllerConfig struct { |
||||
Strategy string `yaml:"strategy"` |
||||
AIMD AIMD `yaml:"aimd"` |
||||
} |
||||
|
||||
func (c *ControllerConfig) RegisterFlags(f *flag.FlagSet) { |
||||
c.RegisterFlagsWithPrefix("", f) |
||||
} |
||||
|
||||
func (c *ControllerConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { |
||||
f.StringVar(&c.Strategy, prefix+"strategy", "", "Congestion control strategy to use (default: none, options: 'aimd').") |
||||
f.UintVar(&c.AIMD.Start, prefix+"strategy.aimd.start", 2000, "AIMD starting throughput window size: how many requests can be sent per second (default: 2000).") |
||||
f.UintVar(&c.AIMD.UpperBound, prefix+"strategy.aimd.upper-bound", 10000, "AIMD maximum throughput window size: upper limit of requests sent per second (default: 10000).") |
||||
f.Float64Var(&c.AIMD.BackoffFactor, prefix+"strategy.aimd.backoff-factor", 0.5, "AIMD backoff factor when upstream service is throttled to decrease number of requests sent per second (default: 0.5).") |
||||
} |
||||
|
||||
type RetrierConfig struct { |
||||
Strategy string `yaml:"strategy"` |
||||
Limit int `yaml:"limit"` |
||||
} |
||||
|
||||
func (c *RetrierConfig) RegisterFlags(f *flag.FlagSet) { |
||||
c.RegisterFlagsWithPrefix("", f) |
||||
} |
||||
|
||||
func (c *RetrierConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { |
||||
f.StringVar(&c.Strategy, prefix+"strategy", "", "Congestion control retry strategy to use (default: none, options: 'limited').") |
||||
f.IntVar(&c.Limit, prefix+"strategy.limited.limit", 2, "Maximum number of retries allowed.") |
||||
} |
||||
|
||||
type HedgerConfig struct { |
||||
hedging.Config |
||||
|
||||
Strategy string `yaml:"strategy"` |
||||
} |
||||
|
||||
func (c *HedgerConfig) RegisterFlags(f *flag.FlagSet) { |
||||
c.RegisterFlagsWithPrefix("", f) |
||||
} |
||||
|
||||
func (c *HedgerConfig) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { |
||||
f.StringVar(&c.Strategy, prefix+"strategy", "", "Congestion control hedge strategy to use (default: none, options: 'limited').") |
||||
// TODO hedge configs
|
||||
} |
@ -0,0 +1,48 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"strings" |
||||
|
||||
"github.com/go-kit/log" |
||||
"github.com/go-kit/log/level" |
||||
) |
||||
|
||||
func NewController(cfg Config, logger log.Logger, metrics *Metrics) Controller { |
||||
logger = log.With(logger, "component", "congestion_control") |
||||
|
||||
return newController(cfg, logger). |
||||
withRetrier(newRetrier(cfg, logger)). |
||||
withHedger(newHedger(cfg, logger)). |
||||
withMetrics(metrics) |
||||
} |
||||
|
||||
func newController(cfg Config, logger log.Logger) Controller { |
||||
strat := strings.ToLower(cfg.Controller.Strategy) |
||||
switch strat { |
||||
case "aimd": |
||||
return NewAIMDController(cfg) |
||||
default: |
||||
level.Warn(logger).Log("msg", "unrecognized congestion control strategy in config, using noop", "strategy", strat) |
||||
return NewNoopController(cfg) |
||||
} |
||||
} |
||||
|
||||
func newRetrier(cfg Config, logger log.Logger) Retrier { |
||||
strat := strings.ToLower(cfg.Retry.Strategy) |
||||
switch strat { |
||||
case "limited": |
||||
return NewLimitedRetrier(cfg) |
||||
default: |
||||
level.Warn(logger).Log("msg", "unrecognized retried strategy in config, using noop", "strategy", strat) |
||||
return NewNoopRetrier(cfg) |
||||
} |
||||
} |
||||
|
||||
func newHedger(cfg Config, logger log.Logger) Hedger { |
||||
strat := strings.ToLower(cfg.Hedge.Strategy) |
||||
switch strat { |
||||
default: |
||||
level.Warn(logger).Log("msg", "unrecognized hedging strategy in config, using noop", "strategy", strat) |
||||
return NewNoopHedger(cfg) |
||||
} |
||||
} |
@ -0,0 +1,69 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"testing" |
||||
|
||||
"github.com/go-kit/log" |
||||
"github.com/stretchr/testify/require" |
||||
) |
||||
|
||||
func TestZeroValueConstruction(t *testing.T) { |
||||
cfg := Config{} |
||||
ctrl := NewController(cfg, log.NewNopLogger(), NewMetrics(t.Name(), cfg)) |
||||
|
||||
require.IsType(t, &NoopController{}, ctrl) |
||||
require.IsType(t, &NoopRetrier{}, ctrl.getRetrier()) |
||||
require.IsType(t, &NoopHedger{}, ctrl.getHedger()) |
||||
} |
||||
|
||||
func TestAIMDConstruction(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
}, |
||||
} |
||||
ctrl := NewController(cfg, log.NewNopLogger(), NewMetrics(t.Name(), cfg)) |
||||
|
||||
require.IsType(t, &AIMDController{}, ctrl) |
||||
require.IsType(t, &NoopRetrier{}, ctrl.getRetrier()) |
||||
require.IsType(t, &NoopHedger{}, ctrl.getHedger()) |
||||
} |
||||
|
||||
func TestRetrierConstruction(t *testing.T) { |
||||
cfg := Config{ |
||||
Retry: RetrierConfig{ |
||||
Strategy: "limited", |
||||
}, |
||||
} |
||||
ctrl := NewController(cfg, log.NewNopLogger(), NewMetrics(t.Name(), cfg)) |
||||
|
||||
require.IsType(t, &NoopController{}, ctrl) |
||||
require.IsType(t, &LimitedRetrier{}, ctrl.getRetrier()) |
||||
require.IsType(t, &NoopHedger{}, ctrl.getHedger()) |
||||
} |
||||
|
||||
func TestCombinedConstruction(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
}, |
||||
Retry: RetrierConfig{ |
||||
Strategy: "limited", |
||||
}, |
||||
} |
||||
ctrl := NewController(cfg, log.NewNopLogger(), NewMetrics(t.Name(), cfg)) |
||||
|
||||
require.IsType(t, &AIMDController{}, ctrl) |
||||
require.IsType(t, &LimitedRetrier{}, ctrl.getRetrier()) |
||||
require.IsType(t, &NoopHedger{}, ctrl.getHedger()) |
||||
} |
||||
|
||||
func TestHedgerConstruction(t *testing.T) { |
||||
//cfg := Config{
|
||||
// Hedge: HedgerConfig{
|
||||
// Strategy: "dont-hedge-retries",
|
||||
// },
|
||||
//}
|
||||
// TODO(dannyk): implement hedging
|
||||
t.Skip("hedging not yet implemented") |
||||
} |
@ -0,0 +1,216 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"context" |
||||
"errors" |
||||
"io" |
||||
"math" |
||||
"time" |
||||
|
||||
"golang.org/x/time/rate" |
||||
|
||||
"github.com/grafana/loki/pkg/storage/chunk/client" |
||||
) |
||||
|
||||
// AIMDController implements the Additive-Increase/Multiplicative-Decrease algorithm which is used in TCP congestion avoidance.
|
||||
// https://en.wikipedia.org/wiki/Additive_increase/multiplicative_decrease
|
||||
type AIMDController struct { |
||||
inner client.ObjectClient |
||||
|
||||
retrier Retrier |
||||
hedger Hedger |
||||
metrics *Metrics |
||||
|
||||
limiter *rate.Limiter |
||||
backoffFactor float64 |
||||
upperBound rate.Limit |
||||
} |
||||
|
||||
func NewAIMDController(cfg Config) *AIMDController { |
||||
lowerBound := rate.Limit(cfg.Controller.AIMD.Start) |
||||
upperBound := rate.Limit(cfg.Controller.AIMD.UpperBound) |
||||
|
||||
if lowerBound <= 0 { |
||||
lowerBound = 1 |
||||
} |
||||
|
||||
if upperBound <= 0 { |
||||
// set to infinity if not defined
|
||||
upperBound = rate.Limit(math.Inf(1)) |
||||
} |
||||
|
||||
backoffFactor := cfg.Controller.AIMD.BackoffFactor |
||||
if backoffFactor <= 0 { |
||||
// AIMD algorithm calls for halving rate
|
||||
backoffFactor = 0.5 |
||||
} |
||||
|
||||
return &AIMDController{ |
||||
limiter: rate.NewLimiter(lowerBound, int(lowerBound)), |
||||
backoffFactor: backoffFactor, |
||||
upperBound: upperBound, |
||||
} |
||||
} |
||||
|
||||
func (a *AIMDController) Wrap(client client.ObjectClient) client.ObjectClient { |
||||
a.inner = client |
||||
return a |
||||
} |
||||
|
||||
func (a *AIMDController) withRetrier(r Retrier) Controller { |
||||
a.retrier = r |
||||
return a |
||||
} |
||||
|
||||
func (a *AIMDController) withHedger(h Hedger) Controller { |
||||
a.hedger = h |
||||
return a |
||||
} |
||||
|
||||
func (a *AIMDController) withMetrics(m *Metrics) Controller { |
||||
a.metrics = m |
||||
|
||||
a.updateLimitMetric() |
||||
return a |
||||
} |
||||
|
||||
func (a *AIMDController) PutObject(ctx context.Context, objectKey string, object io.ReadSeeker) error { |
||||
return a.inner.PutObject(ctx, objectKey, object) |
||||
} |
||||
|
||||
func (a *AIMDController) GetObject(ctx context.Context, objectKey string) (io.ReadCloser, int64, error) { |
||||
// Only GetObject implements congestion avoidance; the other methods are either non-idempotent which means they
|
||||
// cannot be retried, or are too low volume to care about
|
||||
|
||||
// TODO(dannyk): use hedging client to handle requests, do NOT hedge retries
|
||||
|
||||
rc, sz, err := a.retrier.Do( |
||||
func(attempt int) (io.ReadCloser, int64, error) { |
||||
a.metrics.requests.Add(1) |
||||
|
||||
// in retry
|
||||
if attempt > 0 { |
||||
a.metrics.retries.Add(1) |
||||
} |
||||
|
||||
// apply back-pressure while rate-limit has been exceeded
|
||||
//
|
||||
// using Reserve() is slower because it assumes a constant wait time as tokens are replenished, but in experimentation
|
||||
// it's faster to sit in a hot loop and probe every so often if there are tokens available
|
||||
for !a.limiter.Allow() { |
||||
delay := time.Millisecond * 10 |
||||
time.Sleep(delay) |
||||
a.metrics.backoffSec.Add(delay.Seconds()) |
||||
} |
||||
|
||||
// It is vitally important that retries are DISABLED in the inner implementation.
|
||||
// Some object storage clients implement retries internally, and this will interfere here.
|
||||
return a.inner.GetObject(ctx, objectKey) |
||||
}, |
||||
a.IsRetryableErr, |
||||
a.additiveIncrease, |
||||
a.multiplicativeDecrease, |
||||
) |
||||
|
||||
if errors.Is(err, RetriesExceeded) { |
||||
a.metrics.retriesExceeded.Add(1) |
||||
} |
||||
|
||||
return rc, sz, err |
||||
} |
||||
|
||||
func (a *AIMDController) List(ctx context.Context, prefix string, delimiter string) ([]client.StorageObject, []client.StorageCommonPrefix, error) { |
||||
return a.inner.List(ctx, prefix, delimiter) |
||||
} |
||||
|
||||
func (a *AIMDController) DeleteObject(ctx context.Context, objectKey string) error { |
||||
return a.inner.DeleteObject(ctx, objectKey) |
||||
} |
||||
|
||||
func (a *AIMDController) IsObjectNotFoundErr(err error) bool { |
||||
return a.inner.IsObjectNotFoundErr(err) |
||||
} |
||||
|
||||
func (a *AIMDController) IsRetryableErr(err error) bool { |
||||
retryable := a.inner.IsRetryableErr(err) |
||||
if !retryable { |
||||
a.metrics.nonRetryableErrors.Inc() |
||||
} |
||||
|
||||
return retryable |
||||
} |
||||
|
||||
func (a *AIMDController) Stop() { |
||||
a.inner.Stop() |
||||
} |
||||
|
||||
// additiveIncrease increases the number of requests per second that can be sent linearly.
|
||||
// it should never exceed the defined upper bound.
|
||||
func (a *AIMDController) additiveIncrease() { |
||||
newLimit := a.limiter.Limit() + 1 |
||||
|
||||
if newLimit > a.upperBound { |
||||
newLimit = a.upperBound |
||||
} |
||||
|
||||
a.limiter.SetLimit(newLimit) |
||||
a.limiter.SetBurst(int(newLimit)) |
||||
|
||||
a.updateLimitMetric() |
||||
} |
||||
|
||||
// multiplicativeDecrease reduces the number of requests per second that can be sent exponentially.
|
||||
// it should never be set lower than 1.
|
||||
func (a *AIMDController) multiplicativeDecrease() { |
||||
newLimit := math.Ceil(math.Max(1, float64(a.limiter.Limit())*a.backoffFactor)) |
||||
|
||||
a.limiter.SetLimit(rate.Limit(newLimit)) |
||||
a.limiter.SetBurst(int(newLimit)) |
||||
|
||||
a.updateLimitMetric() |
||||
} |
||||
|
||||
func (a *AIMDController) updateLimitMetric() { |
||||
a.metrics.currentLimit.Set(float64(a.limiter.Limit())) |
||||
} |
||||
func (a *AIMDController) getRetrier() Retrier { return a.retrier } |
||||
func (a *AIMDController) getHedger() Hedger { return a.hedger } |
||||
func (a *AIMDController) getMetrics() *Metrics { return a.metrics } |
||||
|
||||
type NoopController struct { |
||||
retrier Retrier |
||||
hedger Hedger |
||||
metrics *Metrics |
||||
} |
||||
|
||||
func NewNoopController(Config) *NoopController { |
||||
return &NoopController{} |
||||
} |
||||
|
||||
func (n *NoopController) PutObject(context.Context, string, io.ReadSeeker) error { return nil } |
||||
func (n *NoopController) GetObject(context.Context, string) (io.ReadCloser, int64, error) { |
||||
return nil, 0, nil |
||||
} |
||||
func (n *NoopController) List(context.Context, string, string) ([]client.StorageObject, []client.StorageCommonPrefix, error) { |
||||
return nil, nil, nil |
||||
} |
||||
func (n *NoopController) DeleteObject(context.Context, string) error { return nil } |
||||
func (n *NoopController) IsObjectNotFoundErr(error) bool { return false } |
||||
func (n *NoopController) IsRetryableErr(error) bool { return false } |
||||
func (n *NoopController) Stop() {} |
||||
func (n *NoopController) Wrap(c client.ObjectClient) client.ObjectClient { return c } |
||||
func (n *NoopController) withRetrier(r Retrier) Controller { |
||||
n.retrier = r |
||||
return n |
||||
} |
||||
func (n *NoopController) withHedger(h Hedger) Controller { |
||||
n.hedger = h |
||||
return n |
||||
} |
||||
func (n *NoopController) withMetrics(m *Metrics) Controller { |
||||
n.metrics = m |
||||
return n |
||||
} |
||||
func (n *NoopController) getRetrier() Retrier { return n.retrier } |
||||
func (n *NoopController) getHedger() Hedger { return n.hedger } |
||||
func (n *NoopController) getMetrics() *Metrics { return n.metrics } |
@ -0,0 +1,290 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"context" |
||||
"io" |
||||
"strings" |
||||
"sync" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/go-kit/log" |
||||
"github.com/pkg/errors" |
||||
"github.com/prometheus/client_golang/prometheus/testutil" |
||||
"github.com/stretchr/testify/require" |
||||
"go.uber.org/atomic" |
||||
|
||||
"github.com/grafana/loki/pkg/storage/chunk/client" |
||||
) |
||||
|
||||
var errFakeFailure = errors.New("fake failure") |
||||
|
||||
func TestRequestNoopRetry(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
}, |
||||
} |
||||
|
||||
metrics := NewMetrics(t.Name(), cfg) |
||||
ctrl := NewController(cfg, log.NewNopLogger(), metrics) |
||||
|
||||
// allow 1 request through, fail the rest
|
||||
cli := newMockObjectClient(maxFailer{max: 1}) |
||||
ctrl.Wrap(cli) |
||||
|
||||
ctx := context.Background() |
||||
|
||||
// first request succeeds
|
||||
_, _, err := ctrl.GetObject(ctx, "foo") |
||||
require.NoError(t, err) |
||||
|
||||
// nothing is done for failed requests
|
||||
_, _, err = ctrl.GetObject(ctx, "foo") |
||||
require.ErrorIs(t, err, errFakeFailure) |
||||
|
||||
require.EqualValues(t, 2, testutil.ToFloat64(metrics.requests)) |
||||
require.EqualValues(t, 0, testutil.ToFloat64(metrics.retries)) |
||||
} |
||||
|
||||
func TestRequestZeroLimitedRetry(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
}, |
||||
Retry: RetrierConfig{ |
||||
Strategy: "limited", |
||||
Limit: 0, |
||||
}, |
||||
} |
||||
|
||||
metrics := NewMetrics(t.Name(), cfg) |
||||
ctrl := NewController(cfg, log.NewNopLogger(), metrics) |
||||
|
||||
// fail all requests
|
||||
cli := newMockObjectClient(maxFailer{max: 0}) |
||||
ctrl.Wrap(cli) |
||||
|
||||
ctx := context.Background() |
||||
|
||||
// first request fails, no retry is executed because limit = 0
|
||||
_, _, err := ctrl.GetObject(ctx, "foo") |
||||
require.ErrorIs(t, err, RetriesExceeded) |
||||
|
||||
require.EqualValues(t, 1, testutil.ToFloat64(metrics.requests)) |
||||
require.EqualValues(t, 0, testutil.ToFloat64(metrics.retries)) |
||||
} |
||||
|
||||
func TestRequestLimitedRetry(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
}, |
||||
Retry: RetrierConfig{ |
||||
Strategy: "limited", |
||||
Limit: 2, |
||||
}, |
||||
} |
||||
|
||||
metrics := NewMetrics(t.Name(), cfg) |
||||
ctrl := NewController(cfg, log.NewNopLogger(), metrics) |
||||
|
||||
// allow 1 request through, fail the rest
|
||||
cli := newMockObjectClient(maxFailer{max: 1}) |
||||
ctrl.Wrap(cli) |
||||
|
||||
ctx := context.Background() |
||||
|
||||
// first request succeeds, no retries
|
||||
_, _, err := ctrl.GetObject(ctx, "foo") |
||||
require.NoError(t, err) |
||||
require.EqualValues(t, 0, testutil.ToFloat64(metrics.retriesExceeded)) |
||||
require.EqualValues(t, 0, testutil.ToFloat64(metrics.retries)) |
||||
require.EqualValues(t, 1, testutil.ToFloat64(metrics.requests)) |
||||
|
||||
// all requests will now fail, which should incur 1 request & 2 retries
|
||||
_, _, err = ctrl.GetObject(ctx, "foo") |
||||
require.ErrorIs(t, err, RetriesExceeded) |
||||
require.EqualValues(t, 1, testutil.ToFloat64(metrics.retriesExceeded)) |
||||
require.EqualValues(t, 2, testutil.ToFloat64(metrics.retries)) |
||||
require.EqualValues(t, 4, testutil.ToFloat64(metrics.requests)) |
||||
} |
||||
|
||||
func TestRequestLimitedRetryNonRetryableErr(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
}, |
||||
Retry: RetrierConfig{ |
||||
Strategy: "limited", |
||||
Limit: 2, |
||||
}, |
||||
} |
||||
|
||||
metrics := NewMetrics(t.Name(), cfg) |
||||
ctrl := NewController(cfg, log.NewNopLogger(), metrics) |
||||
|
||||
// fail all requests
|
||||
cli := newMockObjectClient(maxFailer{max: 0}) |
||||
// mark errors as non-retryable
|
||||
cli.nonRetryableErrs = true |
||||
ctrl.Wrap(cli) |
||||
|
||||
ctx := context.Background() |
||||
|
||||
// request fails, retries not done since error is non-retryable
|
||||
_, _, err := ctrl.GetObject(ctx, "foo") |
||||
require.ErrorIs(t, err, errFakeFailure) |
||||
require.EqualValues(t, 0, testutil.ToFloat64(metrics.retries)) |
||||
require.EqualValues(t, 1, testutil.ToFloat64(metrics.nonRetryableErrors)) |
||||
require.EqualValues(t, 1, testutil.ToFloat64(metrics.requests)) |
||||
} |
||||
|
||||
func TestAIMDReducedThroughput(t *testing.T) { |
||||
cfg := Config{ |
||||
Controller: ControllerConfig{ |
||||
Strategy: "aimd", |
||||
AIMD: AIMD{ |
||||
Start: 1000, |
||||
UpperBound: 5000, |
||||
BackoffFactor: 0.5, |
||||
}, |
||||
}, |
||||
Retry: RetrierConfig{ |
||||
Strategy: "limited", |
||||
Limit: 1, |
||||
}, |
||||
} |
||||
|
||||
var trigger atomic.Bool |
||||
|
||||
metrics := NewMetrics(t.Name(), cfg) |
||||
ctrl := NewController(cfg, log.NewNopLogger(), metrics) |
||||
|
||||
// fail requests only when triggered
|
||||
cli := newMockObjectClient(triggeredFailer{trigger: &trigger}) |
||||
ctrl.Wrap(cli) |
||||
|
||||
// run for 1 second, measure the per-second rate of requests & successful responses
|
||||
count, success := runAndMeasureRate(ctrl, time.Second) |
||||
require.Greater(t, count, 1.0) |
||||
require.Greater(t, success, 1.0) |
||||
// no time spent backing off because the per-second limit will not be hit
|
||||
require.EqualValues(t, 0, testutil.ToFloat64(metrics.backoffSec)) |
||||
|
||||
previousCount, previousSuccess := count, success |
||||
|
||||
var wg sync.WaitGroup |
||||
done := make(chan bool, 1) |
||||
|
||||
// every 100ms trigger a failure
|
||||
wg.Add(1) |
||||
go func(trigger *atomic.Bool) { |
||||
defer wg.Done() |
||||
|
||||
tick := time.NewTicker(time.Millisecond * 100) |
||||
defer tick.Stop() |
||||
for { |
||||
select { |
||||
case <-tick.C: |
||||
trigger.Store(true) |
||||
case <-done: |
||||
return |
||||
} |
||||
} |
||||
}(&trigger) |
||||
|
||||
// now, run the requests again but there will now be a failure rate & some throttling involved
|
||||
count, success = runAndMeasureRate(ctrl, time.Second) |
||||
done <- true |
||||
|
||||
wg.Wait() |
||||
|
||||
// should have processed fewer requests than the last period
|
||||
require.Less(t, count, previousCount) |
||||
require.Less(t, success, previousSuccess) |
||||
|
||||
// should have fewer successful requests than total since we are failing some
|
||||
require.Less(t, success, count) |
||||
} |
||||
|
||||
func runAndMeasureRate(ctrl Controller, duration time.Duration) (float64, float64) { |
||||
var count, success float64 |
||||
|
||||
tick := time.NewTimer(duration) |
||||
defer tick.Stop() |
||||
for { |
||||
select { |
||||
case <-tick.C: |
||||
goto result |
||||
default: |
||||
ctx := context.Background() |
||||
|
||||
count++ |
||||
_, _, err := ctrl.GetObject(ctx, "foo") |
||||
if err == nil { |
||||
success++ |
||||
} |
||||
} |
||||
} |
||||
|
||||
result: |
||||
return count / duration.Seconds(), success / duration.Seconds() |
||||
} |
||||
|
||||
type mockObjectClient struct { |
||||
reqCounter atomic.Uint64 |
||||
strategy requestFailer |
||||
nonRetryableErrs bool |
||||
} |
||||
|
||||
func (m *mockObjectClient) PutObject(context.Context, string, io.ReadSeeker) error { |
||||
panic("not implemented") |
||||
} |
||||
|
||||
func (m *mockObjectClient) GetObject(context.Context, string) (io.ReadCloser, int64, error) { |
||||
time.Sleep(time.Millisecond * 10) |
||||
if m.strategy.fail(m.reqCounter.Inc()) { |
||||
return nil, 0, errFakeFailure |
||||
} |
||||
|
||||
return io.NopCloser(strings.NewReader("bar")), 3, nil |
||||
} |
||||
|
||||
func (m *mockObjectClient) List(context.Context, string, string) ([]client.StorageObject, []client.StorageCommonPrefix, error) { |
||||
panic("not implemented") |
||||
} |
||||
|
||||
func (m *mockObjectClient) DeleteObject(context.Context, string) error { |
||||
panic("not implemented") |
||||
} |
||||
func (m *mockObjectClient) IsObjectNotFoundErr(error) bool { return false } |
||||
func (m *mockObjectClient) IsRetryableErr(error) bool { return !m.nonRetryableErrs } |
||||
func (m *mockObjectClient) Stop() {} |
||||
|
||||
func newMockObjectClient(strat requestFailer) *mockObjectClient { |
||||
return &mockObjectClient{strategy: strat} |
||||
} |
||||
|
||||
type requestFailer interface { |
||||
fail(i uint64) bool |
||||
} |
||||
|
||||
type maxFailer struct { |
||||
max uint64 |
||||
} |
||||
|
||||
func (m maxFailer) fail(i uint64) bool { return i > m.max } |
||||
|
||||
type triggeredFailer struct { |
||||
trigger *atomic.Bool |
||||
} |
||||
|
||||
func (t triggeredFailer) fail(_ uint64) bool { |
||||
if t.trigger.Load() { |
||||
t.trigger.Store(false) |
||||
return true |
||||
} |
||||
|
||||
return false |
||||
} |
@ -0,0 +1,17 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"net/http" |
||||
|
||||
"github.com/grafana/loki/pkg/storage/chunk/client/hedging" |
||||
) |
||||
|
||||
type NoopHedger struct{} |
||||
|
||||
func NewNoopHedger(Config) *NoopHedger { |
||||
return &NoopHedger{} |
||||
} |
||||
|
||||
func (n NoopHedger) HTTPClient(hedging.Config) (*http.Client, error) { |
||||
return http.DefaultClient, nil |
||||
} |
@ -0,0 +1,50 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"io" |
||||
"net/http" |
||||
|
||||
"github.com/grafana/loki/pkg/storage/chunk/client" |
||||
"github.com/grafana/loki/pkg/storage/chunk/client/hedging" |
||||
) |
||||
|
||||
// Controller handles congestion by:
|
||||
// - determining if calls to object storage can be retried
|
||||
// - defining and enforcing a back-pressure mechanism
|
||||
// - centralising retries & hedging
|
||||
type Controller interface { |
||||
client.ObjectClient |
||||
|
||||
// Wrap wraps a given object store client and handles congestion against its backend service
|
||||
Wrap(client client.ObjectClient) client.ObjectClient |
||||
|
||||
withRetrier(Retrier) Controller |
||||
withHedger(Hedger) Controller |
||||
withMetrics(*Metrics) Controller |
||||
|
||||
getRetrier() Retrier |
||||
getHedger() Hedger |
||||
getMetrics() *Metrics |
||||
} |
||||
|
||||
type DoRequestFunc func(attempt int) (io.ReadCloser, int64, error) |
||||
type IsRetryableErrFunc func(err error) bool |
||||
|
||||
// Retrier orchestrates requests & subsequent retries (if configured).
|
||||
// NOTE: this only supports ObjectClient.GetObject calls right now.
|
||||
type Retrier interface { |
||||
// Do executes a given function which is expected to be a GetObject call, and its return signature matches that.
|
||||
// Any failed requests will be retried.
|
||||
//
|
||||
// count is the current request count; any positive number indicates retries, 0 indicates first attempt.
|
||||
Do(fn DoRequestFunc, isRetryable IsRetryableErrFunc, onSuccess func(), onError func()) (io.ReadCloser, int64, error) |
||||
} |
||||
|
||||
// Hedger orchestrates request "hedging", which is the process of sending a new request when the old request is
|
||||
// taking too long, and returning the response that is received first
|
||||
type Hedger interface { |
||||
// HTTPClient returns an HTTP client which is responsible for handling both the initial and all hedged requests.
|
||||
// It is recommended that retries are not hedged.
|
||||
// Bear in mind this function can be called several times, and should return the same client each time.
|
||||
HTTPClient(cfg hedging.Config) (*http.Client, error) |
||||
} |
@ -0,0 +1,82 @@ |
||||
package congestion |
||||
|
||||
import "github.com/prometheus/client_golang/prometheus" |
||||
|
||||
type Metrics struct { |
||||
currentLimit prometheus.Gauge |
||||
backoffSec prometheus.Counter |
||||
requests prometheus.Counter |
||||
retries prometheus.Counter |
||||
nonRetryableErrors prometheus.Counter |
||||
retriesExceeded prometheus.Counter |
||||
} |
||||
|
||||
func (m Metrics) Unregister() { |
||||
prometheus.Unregister(m.currentLimit) |
||||
} |
||||
|
||||
// NewMetrics creates metrics to be used for monitoring congestion control.
|
||||
// It needs to accept a "name" because congestion control is used in object clients, and there can be many object clients
|
||||
// creates for the same store (multiple period configs, etc). It is the responsibility of the caller to ensure uniqueness,
|
||||
// otherwise a duplicate registration panic will occur.
|
||||
func NewMetrics(name string, cfg Config) *Metrics { |
||||
labels := map[string]string{ |
||||
"strategy": cfg.Controller.Strategy, |
||||
"name": name, |
||||
} |
||||
|
||||
const namespace = "loki" |
||||
const subsystem = "store_congestion_control" |
||||
m := Metrics{ |
||||
currentLimit: prometheus.NewGauge(prometheus.GaugeOpts{ |
||||
Namespace: namespace, |
||||
Subsystem: subsystem, |
||||
Name: "limit", |
||||
Help: "Current per-second request limit to control congestion", |
||||
ConstLabels: labels, |
||||
}), |
||||
backoffSec: prometheus.NewCounter(prometheus.CounterOpts{ |
||||
Namespace: namespace, |
||||
Subsystem: subsystem, |
||||
Name: "backoff_seconds_total", |
||||
Help: "How much time is spent backing off once throughput limit is encountered", |
||||
ConstLabels: labels, |
||||
}), |
||||
requests: prometheus.NewCounter(prometheus.CounterOpts{ |
||||
Namespace: namespace, |
||||
Subsystem: subsystem, |
||||
Name: "requests_total", |
||||
Help: "How many requests were issued to the store", |
||||
ConstLabels: labels, |
||||
}), |
||||
retries: prometheus.NewCounter(prometheus.CounterOpts{ |
||||
Namespace: namespace, |
||||
Subsystem: subsystem, |
||||
Name: "retries_total", |
||||
Help: "How many retries occurred", |
||||
ConstLabels: labels, |
||||
}), |
||||
nonRetryableErrors: prometheus.NewCounter(prometheus.CounterOpts{ |
||||
Namespace: namespace, |
||||
Subsystem: subsystem, |
||||
Name: "non_retryable_errors_total", |
||||
Help: "How many request errors occurred which could not be retried", |
||||
ConstLabels: labels, |
||||
}), |
||||
retriesExceeded: prometheus.NewCounter(prometheus.CounterOpts{ |
||||
Namespace: namespace, |
||||
Subsystem: subsystem, |
||||
Name: "retries_exceeded_total", |
||||
Help: "How many times the number of retries exceeded the configured limit.", |
||||
ConstLabels: labels, |
||||
}), |
||||
} |
||||
|
||||
prometheus.MustRegister(m.currentLimit) |
||||
prometheus.MustRegister(m.backoffSec) |
||||
prometheus.MustRegister(m.requests) |
||||
prometheus.MustRegister(m.retries) |
||||
prometheus.MustRegister(m.nonRetryableErrors) |
||||
prometheus.MustRegister(m.retriesExceeded) |
||||
return &m |
||||
} |
@ -0,0 +1,54 @@ |
||||
package congestion |
||||
|
||||
import ( |
||||
"errors" |
||||
"io" |
||||
) |
||||
|
||||
var RetriesExceeded = errors.New("retries exceeded") |
||||
|
||||
type NoopRetrier struct{} |
||||
|
||||
func NewNoopRetrier(Config) *NoopRetrier { |
||||
return &NoopRetrier{} |
||||
} |
||||
|
||||
func (n NoopRetrier) Do(fn DoRequestFunc, _ IsRetryableErrFunc, _ func(), _ func()) (io.ReadCloser, int64, error) { |
||||
// don't retry, just execute the given function once
|
||||
return fn(0) |
||||
} |
||||
|
||||
// LimitedRetrier executes the initial request plus a configurable limit of subsequent retries.
|
||||
// limit=0 is equivalent to NoopRetrier
|
||||
type LimitedRetrier struct { |
||||
limit int |
||||
} |
||||
|
||||
func NewLimitedRetrier(cfg Config) *LimitedRetrier { |
||||
return &LimitedRetrier{limit: cfg.Retry.Limit} |
||||
} |
||||
|
||||
func (l *LimitedRetrier) Do(fn DoRequestFunc, isRetryable IsRetryableErrFunc, onSuccess func(), onError func()) (io.ReadCloser, int64, error) { |
||||
// i = 0 is initial request
|
||||
// i > 0 is retry
|
||||
for i := 0; i <= l.limit; i++ { |
||||
rc, sz, err := fn(i) |
||||
|
||||
if err != nil { |
||||
if !isRetryable(err) { |
||||
return rc, sz, err |
||||
} |
||||
|
||||
// TODO(dannyk): consider this more carefully
|
||||
// only decrease rate-limit if error is retryable, otherwise all errors (context cancelled, dial errors, timeouts, etc)
|
||||
// which may be mostly client-side would inappropriately reduce throughput
|
||||
onError() |
||||
continue |
||||
} |
||||
|
||||
onSuccess() |
||||
return rc, sz, err |
||||
} |
||||
|
||||
return nil, 0, RetriesExceeded |
||||
} |
Loading…
Reference in new issue