Skip to content

Commit

Permalink
resource_control: allow configuration of the maximum retry time for t…
Browse files Browse the repository at this point in the history
…he local bucket (#8352)  (#8365)

* client/controller: record context error and add slowlog about token bucket (#8344) (#8355)

close #8343, ref #8349

client/controller: record context error and add slowlog about token bucket
- record low process start time, and log it if it's too slow
- record the context error

Signed-off-by: Shuning Chen <[email protected]>

* This is an automated cherry-pick of #8352

close #8349

Signed-off-by: nolouch <[email protected]>
Signed-off-by: Shuning Chen <[email protected]>

---------

Signed-off-by: Shuning Chen <[email protected]>
Signed-off-by: nolouch <[email protected]>
Co-authored-by: Ti Chi Robot <[email protected]>
  • Loading branch information
nolouch and ti-chi-bot committed Jul 4, 2024
1 parent 358de10 commit 173c2e1
Show file tree
Hide file tree
Showing 9 changed files with 207 additions and 77 deletions.
72 changes: 65 additions & 7 deletions client/resource_group/controller/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,12 @@ const (
defaultTargetPeriod = 5 * time.Second
// defaultMaxWaitDuration is the max duration to wait for the token before throwing error.
defaultMaxWaitDuration = 30 * time.Second
// defaultLTBTokenRPCMaxDelay is the upper bound of backoff delay for local token bucket RPC.
defaultLTBTokenRPCMaxDelay = 1 * time.Second
// defaultWaitRetryTimes is the times to retry when waiting for the token.
defaultWaitRetryTimes = 20
// defaultWaitRetryInterval is the interval to retry when waiting for the token.
defaultWaitRetryInterval = 50 * time.Millisecond
)

const (
Expand All @@ -73,18 +79,36 @@ const (

// Because the resource manager has not been deployed in microservice mode,
// do not enable this function.
defaultDegradedModeWaitDuration = 0
defaultDegradedModeWaitDuration = time.Duration(0)
defaultAvgBatchProportion = 0.7
)

// Config is the configuration of the resource manager controller which includes some option for client needed.
type Config struct {
// TokenRPCParams is the parameters for local bucket RPC.
type TokenRPCParams struct {
// WaitRetryInterval is the interval to retry when waiting for the token.
WaitRetryInterval Duration `toml:"wait-retry-interval" json:"wait-retry-interval"`

// WaitRetryTimes is the times to retry when waiting for the token.
WaitRetryTimes int `toml:"wait-retry-times" json:"wait-retry-times"`
}

// LocalBucketConfig is the configuration for local bucket. not export to server side.
type LocalBucketConfig struct {
TokenRPCParams `toml:"token-rpc-params" json:"token-rpc-params"`
}

// BaseConfig is the configuration of the resource manager controller which includes some option for client needed.
// TODO: unified the configuration for client and server, server side in pkg/mcs/resourcemanger/config.go.
type BaseConfig struct {
// EnableDegradedMode is to control whether resource control client enable degraded mode when server is disconnect.
DegradedModeWaitDuration Duration `toml:"degraded-mode-wait-duration" json:"degraded-mode-wait-duration"`

// LTBMaxWaitDuration is the max wait time duration for local token bucket.
LTBMaxWaitDuration Duration `toml:"ltb-max-wait-duration" json:"ltb-max-wait-duration"`

// LTBTokenRPCMaxDelay is the upper bound of backoff delay for local token bucket RPC.
LTBTokenRPCMaxDelay Duration `toml:"ltb-token-rpc-max-delay" json:"ltb-token-rpc-max-delay"`

// RequestUnit is the configuration determines the coefficients of the RRU and WRU cost.
// This configuration should be modified carefully.
RequestUnit RequestUnitConfig `toml:"request-unit" json:"request-unit"`
Expand All @@ -93,13 +117,43 @@ type Config struct {
EnableControllerTraceLog bool `toml:"enable-controller-trace-log" json:"enable-controller-trace-log,string"`
}

// Config is the configuration of the resource manager controller.
type Config struct {
BaseConfig
LocalBucketConfig
}

// Adjust adjusts the configuration.
func (c *Config) Adjust() {
// valid the configuration, TODO: separately add the valid function.
if c.BaseConfig.LTBMaxWaitDuration.Duration == 0 {
c.BaseConfig.LTBMaxWaitDuration = NewDuration(defaultMaxWaitDuration)
}
if c.LocalBucketConfig.WaitRetryInterval.Duration == 0 {
c.LocalBucketConfig.WaitRetryInterval = NewDuration(defaultWaitRetryInterval)
}
// adjust the client settings. calculate the retry times.
if int(c.BaseConfig.LTBTokenRPCMaxDelay.Duration) != int(c.LocalBucketConfig.WaitRetryInterval.Duration)*c.LocalBucketConfig.WaitRetryTimes {
c.LocalBucketConfig.WaitRetryTimes = int(c.BaseConfig.LTBTokenRPCMaxDelay.Duration / c.LocalBucketConfig.WaitRetryInterval.Duration)
}
}

// DefaultConfig returns the default resource manager controller configuration.
func DefaultConfig() *Config {
return &Config{
DegradedModeWaitDuration: NewDuration(defaultDegradedModeWaitDuration),
LTBMaxWaitDuration: NewDuration(defaultMaxWaitDuration),
RequestUnit: DefaultRequestUnitConfig(),
EnableControllerTraceLog: false,
BaseConfig: BaseConfig{
DegradedModeWaitDuration: NewDuration(defaultDegradedModeWaitDuration),
RequestUnit: DefaultRequestUnitConfig(),
EnableControllerTraceLog: false,
LTBMaxWaitDuration: NewDuration(defaultMaxWaitDuration),
LTBTokenRPCMaxDelay: NewDuration(defaultLTBTokenRPCMaxDelay),
},
LocalBucketConfig: LocalBucketConfig{
TokenRPCParams: TokenRPCParams{
WaitRetryInterval: NewDuration(defaultWaitRetryInterval),
WaitRetryTimes: defaultWaitRetryTimes,
},
},
}
}

Expand Down Expand Up @@ -155,6 +209,8 @@ type RUConfig struct {

// some config for client
LTBMaxWaitDuration time.Duration
WaitRetryInterval time.Duration
WaitRetryTimes int
DegradedModeWaitDuration time.Duration
}

Expand All @@ -176,6 +232,8 @@ func GenerateRUConfig(config *Config) *RUConfig {
WriteBytesCost: RequestUnit(config.RequestUnit.WriteCostPerByte),
CPUMsCost: RequestUnit(config.RequestUnit.CPUMsCost),
LTBMaxWaitDuration: config.LTBMaxWaitDuration.Duration,
WaitRetryInterval: config.WaitRetryInterval.Duration,
WaitRetryTimes: config.WaitRetryTimes,
DegradedModeWaitDuration: config.DegradedModeWaitDuration.Duration,
}
}
79 changes: 49 additions & 30 deletions client/resource_group/controller/controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,14 +38,12 @@ import (
)

const (
controllerConfigPath = "resource_group/controller"
maxRetry = 10
retryInterval = 50 * time.Millisecond
maxNotificationChanLen = 200
needTokensAmplification = 1.1
trickleReserveDuration = 1250 * time.Millisecond

watchRetryInterval = 30 * time.Second
controllerConfigPath = "resource_group/controller"
maxNotificationChanLen = 200
needTokensAmplification = 1.1
trickleReserveDuration = 1250 * time.Millisecond
slowNotifyFilterDuration = 10 * time.Millisecond
watchRetryInterval = 30 * time.Second
)

type selectType int
Expand Down Expand Up @@ -104,6 +102,20 @@ func WithMaxWaitDuration(d time.Duration) ResourceControlCreateOption {
}
}

// WithWaitRetryInterval is the option to set the retry interval when waiting for the token.
func WithWaitRetryInterval(d time.Duration) ResourceControlCreateOption {
return func(controller *ResourceGroupsController) {
controller.ruConfig.WaitRetryInterval = d
}
}

// WithWaitRetryTimes is the option to set the times to retry when waiting for the token.
func WithWaitRetryTimes(times int) ResourceControlCreateOption {
return func(controller *ResourceGroupsController) {
controller.ruConfig.WaitRetryTimes = times
}
}

var _ ResourceGroupKVInterceptor = (*ResourceGroupsController)(nil)

// ResourceGroupsController implements ResourceGroupKVInterceptor.
Expand All @@ -119,7 +131,7 @@ type ResourceGroupsController struct {
calculators []ResourceCalculator

// When a signal is received, it means the number of available token is low.
lowTokenNotifyChan chan struct{}
lowTokenNotifyChan chan notifyMsg
// When a token bucket response received from server, it will be sent to the channel.
tokenResponseChan chan []*rmpb.TokenBucketResponse
// When the token bucket of a resource group is updated, it will be sent to the channel.
Expand Down Expand Up @@ -161,7 +173,7 @@ func NewResourceGroupController(
clientUniqueID: clientUniqueID,
provider: provider,
ruConfig: ruConfig,
lowTokenNotifyChan: make(chan struct{}, 1),
lowTokenNotifyChan: make(chan notifyMsg, 1),
tokenResponseChan: make(chan []*rmpb.TokenBucketResponse, 1),
tokenBucketUpdateChan: make(chan *groupCostController, maxNotificationChanLen),
opts: opts,
Expand All @@ -172,6 +184,7 @@ func NewResourceGroupController(
log.Info("load resource controller config", zap.Reflect("config", config), zap.Reflect("ru-config", controller.ruConfig))
controller.calculators = []ResourceCalculator{newKVCalculator(controller.ruConfig), newSQLCalculator(controller.ruConfig)}
controller.safeRuConfig.Store(controller.ruConfig)
enableControllerTraceLog.Store(config.EnableControllerTraceLog)
return controller, nil
}

Expand All @@ -180,12 +193,13 @@ func loadServerConfig(ctx context.Context, provider ResourceGroupProvider) (*Con
if err != nil {
return nil, err
}
config := DefaultConfig()
defer config.Adjust()
kvs := resp.GetKvs()
if len(kvs) == 0 {
log.Warn("[resource group controller] server does not save config, load config failed")
return DefaultConfig(), nil
return config, nil
}
config := &Config{}
err = json.Unmarshal(kvs[0].GetValue(), config)
if err != nil {
return nil, err
Expand Down Expand Up @@ -267,7 +281,7 @@ func (c *ResourceGroupsController) Start(ctx context.Context) {
c.executeOnAllGroups((*groupCostController).updateRunState)
c.executeOnAllGroups((*groupCostController).updateAvgRequestResourcePerSec)
if len(c.run.currentRequests) == 0 {
c.collectTokenBucketRequests(c.loopCtx, FromPeriodReport, periodicReport /* select resource groups which should be reported periodically */)
c.collectTokenBucketRequests(c.loopCtx, FromPeriodReport, periodicReport /* select resource groups which should be reported periodically */, notifyMsg{})
}
case <-watchRetryTimer.C:
if !c.ruConfig.isSingleGroupByKeyspace && watchMetaChannel == nil {
Expand All @@ -288,7 +302,6 @@ func (c *ResourceGroupsController) Start(ctx context.Context) {
watchRetryTimer.Reset(watchRetryInterval)
}
}

case <-emergencyTokenAcquisitionTicker.C:
c.executeOnAllGroups((*groupCostController).resetEmergencyTokenAcquisition)
/* channels */
Expand All @@ -305,11 +318,11 @@ func (c *ResourceGroupsController) Start(ctx context.Context) {
c.handleTokenBucketResponse(resp)
}
c.run.currentRequests = nil
case <-c.lowTokenNotifyChan:
case notifyMsg := <-c.lowTokenNotifyChan:
c.executeOnAllGroups((*groupCostController).updateRunState)
c.executeOnAllGroups((*groupCostController).updateAvgRequestResourcePerSec)
if len(c.run.currentRequests) == 0 {
c.collectTokenBucketRequests(c.loopCtx, FromLowRU, lowToken /* select low tokens resource group */)
c.collectTokenBucketRequests(c.loopCtx, FromLowRU, lowToken /* select low tokens resource group */, notifyMsg)
}
if c.run.inDegradedMode {
c.executeOnAllGroups((*groupCostController).applyDegradedMode)
Expand Down Expand Up @@ -366,10 +379,11 @@ func (c *ResourceGroupsController) Start(ctx context.Context) {
}
for _, item := range resp {
cfgRevision = item.Kv.ModRevision
config := &Config{}
config := DefaultConfig()
if err := json.Unmarshal(item.Kv.Value, config); err != nil {
continue
}
config.Adjust()
c.ruConfig = GenerateRUConfig(config)

// Stay compatible with serverless
Expand All @@ -383,7 +397,6 @@ func (c *ResourceGroupsController) Start(ctx context.Context) {
}
log.Info("load resource controller config after config changed", zap.Reflect("config", config), zap.Reflect("ruConfig", c.ruConfig))
}

case gc := <-c.tokenBucketUpdateChan:
now := gc.run.now
go gc.handleTokenBucketUpdateEvent(c.loopCtx, now)
Expand Down Expand Up @@ -489,7 +502,7 @@ func (c *ResourceGroupsController) handleTokenBucketResponse(resp []*rmpb.TokenB
}
}

func (c *ResourceGroupsController) collectTokenBucketRequests(ctx context.Context, source string, typ selectType) {
func (c *ResourceGroupsController) collectTokenBucketRequests(ctx context.Context, source string, typ selectType, notifyMsg notifyMsg) {
c.run.currentRequests = make([]*rmpb.TokenBucketRequest, 0)
c.groupsController.Range(func(name, value any) bool {
gc := value.(*groupCostController)
Expand All @@ -501,11 +514,11 @@ func (c *ResourceGroupsController) collectTokenBucketRequests(ctx context.Contex
return true
})
if len(c.run.currentRequests) > 0 {
c.sendTokenBucketRequests(ctx, c.run.currentRequests, source)
c.sendTokenBucketRequests(ctx, c.run.currentRequests, source, notifyMsg)
}
}

func (c *ResourceGroupsController) sendTokenBucketRequests(ctx context.Context, requests []*rmpb.TokenBucketRequest, source string) {
func (c *ResourceGroupsController) sendTokenBucketRequests(ctx context.Context, requests []*rmpb.TokenBucketRequest, source string, notifyMsg notifyMsg) {
now := time.Now()
req := &rmpb.TokenBucketsRequest{
Requests: requests,
Expand All @@ -523,13 +536,16 @@ func (c *ResourceGroupsController) sendTokenBucketRequests(ctx context.Context,
if err != nil {
// Don't log any errors caused by the stopper canceling the context.
if !errors.ErrorEqual(err, context.Canceled) {
log.L().Sugar().Infof("[resource group controller] token bucket rpc error: %v", err)
log.Error("[resource group controller] token bucket rpc error", zap.Error(err))
}
resp = nil
failedTokenRequestDuration.Observe(latency.Seconds())
} else {
successfulTokenRequestDuration.Observe(latency.Seconds())
}
if !notifyMsg.startTime.IsZero() && time.Since(notifyMsg.startTime) > slowNotifyFilterDuration {
log.Warn("[resource group controller] slow token bucket request", zap.String("source", source), zap.Duration("cost", time.Since(notifyMsg.startTime)))
}
logControllerTrace("[resource group controller] token bucket response", zap.Time("now", time.Now()), zap.Any("resp", resp), zap.String("source", source), zap.Duration("latency", latency))
c.tokenResponseChan <- resp
}()
Expand Down Expand Up @@ -625,7 +641,7 @@ type groupCostController struct {
// fast path to make once token limit with un-limit burst.
burstable *atomic.Bool

lowRUNotifyChan chan<- struct{}
lowRUNotifyChan chan<- notifyMsg
tokenBucketUpdateChan chan<- *groupCostController

// run contains the state that is updated by the main loop.
Expand Down Expand Up @@ -715,7 +731,7 @@ type tokenCounter struct {
func newGroupCostController(
group *rmpb.ResourceGroup,
mainCfg *RUConfig,
lowRUNotifyChan chan struct{},
lowRUNotifyChan chan notifyMsg,
tokenBucketUpdateChan chan *groupCostController,
) (*groupCostController, error) {
switch group.Mode {
Expand Down Expand Up @@ -834,7 +850,7 @@ func (gc *groupCostController) updateRunState() {
}
*gc.run.consumption = *gc.mu.consumption
gc.mu.Unlock()
logControllerTrace("[resource group controller] update run state", zap.Any("request-unit-consumption", gc.run.consumption))
logControllerTrace("[resource group controller] update run state", zap.String("name", gc.name), zap.Any("request-unit-consumption", gc.run.consumption))
gc.run.now = newTime
}

Expand Down Expand Up @@ -1034,7 +1050,7 @@ func (gc *groupCostController) applyBasicConfigForRUTokenCounters() {
cfg.NewRate = 99999999
})
counter.limiter.Reconfigure(gc.run.now, cfg, resetLowProcess())
log.Info("[resource group controller] resource token bucket enter degraded mode", zap.String("resource-group", gc.name), zap.String("type", rmpb.RequestUnitType_name[int32(typ)]))
log.Info("[resource group controller] resource token bucket enter degraded mode", zap.String("name", gc.name), zap.String("type", rmpb.RequestUnitType_name[int32(typ)]))
}
}

Expand Down Expand Up @@ -1088,6 +1104,9 @@ func (gc *groupCostController) modifyTokenCounter(counter *tokenCounter, bucket
timerDuration = (trickleDuration + trickleReserveDuration) / 2
}
counter.notify.mu.Lock()
if counter.notify.setupNotificationTimer != nil {
counter.notify.setupNotificationTimer.Stop()
}
counter.notify.setupNotificationTimer = time.NewTimer(timerDuration)
counter.notify.setupNotificationCh = counter.notify.setupNotificationTimer.C
counter.notify.setupNotificationThreshold = 1
Expand Down Expand Up @@ -1222,7 +1241,7 @@ func (gc *groupCostController) onRequestWait(
var i int
var d time.Duration
retryLoop:
for i = 0; i < maxRetry; i++ {
for i = 0; i < gc.mainCfg.WaitRetryTimes; i++ {
switch gc.mode {
case rmpb.GroupMode_RawMode:
res := make([]*Reservation, 0, len(requestResourceLimitTypeList))
Expand All @@ -1246,8 +1265,8 @@ func (gc *groupCostController) onRequestWait(
}
}
gc.metrics.requestRetryCounter.Inc()
time.Sleep(retryInterval)
waitDuration += retryInterval
time.Sleep(gc.mainCfg.WaitRetryInterval)
waitDuration += gc.mainCfg.WaitRetryInterval
}
if err != nil {
if errs.ErrClientResourceGroupThrottled.Equal(err) {
Expand All @@ -1260,7 +1279,7 @@ func (gc *groupCostController) onRequestWait(
sub(gc.mu.consumption, delta)
gc.mu.Unlock()
failpoint.Inject("triggerUpdate", func() {
gc.lowRUNotifyChan <- struct{}{}
gc.lowRUNotifyChan <- notifyMsg{}
})
return nil, nil, waitDuration, 0, err
}
Expand Down
2 changes: 1 addition & 1 deletion client/resource_group/controller/controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ func createTestGroupCostController(re *require.Assertions) *groupCostController
JobTypes: []string{"lightning", "br"},
},
}
ch1 := make(chan struct{})
ch1 := make(chan notifyMsg)
ch2 := make(chan *groupCostController)
gc, err := newGroupCostController(group, DefaultRUConfig(), ch1, ch2)
re.NoError(err)
Expand Down
Loading

0 comments on commit 173c2e1

Please sign in to comment.