Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/storage/chunk/client/congestion/controller_test.go

311 lines
7.8 KiB

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 available
2 years ago
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/v3/pkg/logqlmodel/stats"
"github.com/grafana/loki/v3/pkg/storage/chunk/client"
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 available
2 years ago
)
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))
metrics.Unregister()
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 available
2 years ago
}
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))
metrics.Unregister()
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 available
2 years ago
}
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))
metrics.Unregister()
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 available
2 years ago
}
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))
metrics.Unregister()
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 available
2 years ago
}
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)
statsCtx, ctx := stats.NewContext(context.Background())
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 available
2 years ago
// run for 1 second, measure the per-second rate of requests & successful responses
count, success := runAndMeasureRate(ctx, ctrl, time.Second)
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 available
2 years ago
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(ctx, ctrl, time.Second)
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 available
2 years ago
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 may be failing some
require.LessOrEqual(t, success, count)
// should have registered some congestion latency in stats
require.NotZero(t, statsCtx.Store().CongestionControlLatency)
metrics.Unregister()
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 available
2 years ago
}
func runAndMeasureRate(ctx context.Context, ctrl Controller, duration time.Duration) (float64, float64) {
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 available
2 years ago
var count, success float64
tick := time.NewTimer(duration)
defer tick.Stop()
for {
select {
case <-tick.C:
goto result
default:
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.Reader) error {
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 available
2 years ago
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) GetObjectRange(context.Context, string, int64, int64) (io.ReadCloser, error) {
panic("not implemented")
}
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 available
2 years ago
func (m *mockObjectClient) ObjectExists(context.Context, string) (bool, error) {
panic("not implemented")
}
func (m *mockObjectClient) GetAttributes(context.Context, string) (client.ObjectAttributes, error) {
panic("not implemented")
}
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 available
2 years ago
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(start requestFailer) *mockObjectClient {
return &mockObjectClient{strategy: start}
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 available
2 years ago
}
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
}