Disable deny writing when the growing segment size exceeds the watermark (#26163) (#26208)

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/26233/head
yihao.dai 2023-08-09 13:19:16 +08:00 committed by GitHub
parent 76ed3070ff
commit f31d4fb176
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 28 additions and 19 deletions

View File

@ -576,10 +576,11 @@ quotaAndLimits:
queryNodeMemoryLowWaterLevel: 0.85 # (0, 1], memoryLowWaterLevel in QueryNodes
queryNodeMemoryHighWaterLevel: 0.95 # (0, 1], memoryHighWaterLevel in QueryNodes
growingSegmentsSizeProtection:
# 1. No action will be taken if the ratio of growing segments size is less than the low water level.
# 2. The DML rate will be reduced if the ratio of growing segments size is greater than the low water level and less than the high water level.
# 3. All DML requests will be rejected if the ratio of growing segments size is greater than the high water level.
# No action will be taken if the growing segments size is less than the low watermark.
# When the growing segments size exceeds the low watermark, the dml rate will be reduced,
# but the rate will not be lower than `minRateRatio * dmlRate`.
enabled: false
minRateRatio: 0.5
lowWaterLevel: 0.2
highWaterLevel: 0.4
diskProtection:

View File

@ -640,24 +640,19 @@ func (q *QuotaCenter) getGrowingSegmentsSizeFactor() map[int64]float64 {
if cur <= low {
continue
}
if cur >= high {
log.RatedWarn(10, "QuotaCenter: QueryNode growing segments size to high water level",
zap.String("Node", fmt.Sprintf("%s-%d", typeutil.QueryNodeRole, nodeID)),
zap.Int64s("collections", metric.Effect.CollectionIDs),
zap.Int64("segmentsSize", metric.GrowingSegmentsSize),
zap.Uint64("TotalMem", metric.Hms.Memory),
zap.Float64("highWaterLevel", high))
updateCollectionFactor(0, metric.Effect.CollectionIDs)
continue
}
factor := (high - cur) / (high - low)
if factor < Params.QuotaConfig.GrowingSegmentsSizeMinRateRatio {
factor = Params.QuotaConfig.GrowingSegmentsSizeMinRateRatio
}
updateCollectionFactor(factor, metric.Effect.CollectionIDs)
log.RatedWarn(10, "QuotaCenter: QueryNode growing segments size to low water level, limit writing rate",
log.RatedWarn(10, "QuotaCenter: QueryNode growing segments size exceeds watermark, limit writing rate",
zap.String("Node", fmt.Sprintf("%s-%d", typeutil.QueryNodeRole, nodeID)),
zap.Int64s("collections", metric.Effect.CollectionIDs),
zap.Int64("segmentsSize", metric.GrowingSegmentsSize),
zap.Uint64("TotalMem", metric.Hms.Memory),
zap.Float64("lowWaterLevel", low))
zap.Float64("highWatermark", high),
zap.Float64("lowWatermark", low),
zap.Float64("factor", factor))
}
return collectionFactor
}

View File

@ -410,6 +410,7 @@ func TestQuotaCenter(t *testing.T) {
t.Run("test GrowingSegmentsSize factors", func(t *testing.T) {
qc := types.NewMockQueryCoord(t)
quotaCenter := NewQuotaCenter(pcm, qc, &dataCoordMockForQuota{}, core.tsoAllocator)
defaultRatio := Params.QuotaConfig.GrowingSegmentsSizeMinRateRatio
tests := []struct {
low float64
high float64
@ -421,15 +422,15 @@ func TestQuotaCenter(t *testing.T) {
{0.8, 0.9, 80, 100, 1},
{0.8, 0.9, 82, 100, 0.8},
{0.8, 0.9, 85, 100, 0.5},
{0.8, 0.9, 88, 100, 0.2},
{0.8, 0.9, 90, 100, 0},
{0.8, 0.9, 88, 100, defaultRatio},
{0.8, 0.9, 90, 100, defaultRatio},
{0.85, 0.95, 25, 100, 1},
{0.85, 0.95, 85, 100, 1},
{0.85, 0.95, 87, 100, 0.8},
{0.85, 0.95, 90, 100, 0.5},
{0.85, 0.95, 93, 100, 0.2},
{0.85, 0.95, 95, 100, 0},
{0.85, 0.95, 93, 100, defaultRatio},
{0.85, 0.95, 95, 100, defaultRatio},
}
quotaCenter.writableCollections = append(quotaCenter.writableCollections, 1, 2, 3)

View File

@ -111,6 +111,7 @@ type quotaConfig struct {
QueryNodeMemoryLowWaterLevel float64
QueryNodeMemoryHighWaterLevel float64
GrowingSegmentsSizeProtectionEnabled bool
GrowingSegmentsSizeMinRateRatio float64
GrowingSegmentsSizeLowWaterLevel float64
GrowingSegmentsSizeHighWaterLevel float64
DiskProtectionEnabled bool
@ -188,6 +189,7 @@ func (p *quotaConfig) init(base *BaseTable) {
p.initQueryNodeMemoryLowWaterLevel()
p.initQueryNodeMemoryHighWaterLevel()
p.initGrowingSegmentsSizeProtectionEnabled()
p.initGrowingSegmentsSizeMinRateRatio()
p.initGrowingSegmentsSizeLowWaterLevel()
p.initGrowingSegmentsSizeHighWaterLevel()
p.initDiskProtectionEnabled()
@ -726,6 +728,15 @@ func (p *quotaConfig) initGrowingSegmentsSizeProtectionEnabled() {
p.GrowingSegmentsSizeProtectionEnabled = p.Base.ParseBool("quotaAndLimits.limitWriting.growingSegmentsSizeProtection.enabled", false)
}
func (p *quotaConfig) initGrowingSegmentsSizeMinRateRatio() {
defaultGrowingSegmentsSizeMinRateRatio := 0.5
p.GrowingSegmentsSizeMinRateRatio = p.Base.ParseFloatWithDefault("quotaAndLimits.limitWriting.growingSegmentsSizeProtection.minRateRatio", defaultGrowingSegmentsSizeMinRateRatio)
if p.GrowingSegmentsSizeMinRateRatio <= 0 || p.GrowingSegmentsSizeMinRateRatio > 1 {
log.Warn("GrowingSegmentsSizeMinRateRatio must in the range of `(0, 1]`, use default value", zap.Float64("default", defaultGrowingSegmentsSizeMinRateRatio))
p.GrowingSegmentsSizeMinRateRatio = defaultGrowingSegmentsSizeMinRateRatio
}
}
func (p *quotaConfig) initGrowingSegmentsSizeLowWaterLevel() {
defaultGrowingSegSizeLowWaterLevel := 0.2
p.GrowingSegmentsSizeLowWaterLevel = p.Base.ParseFloatWithDefault("quotaAndLimits.limitWriting.growingSegmentsSizeProtection.lowWaterLevel", defaultGrowingSegSizeLowWaterLevel)

View File

@ -173,6 +173,7 @@ func TestQuotaParam(t *testing.T) {
assert.Equal(t, defaultLowWaterLevel, qc.QueryNodeMemoryLowWaterLevel)
assert.Equal(t, defaultHighWaterLevel, qc.QueryNodeMemoryHighWaterLevel)
assert.Equal(t, false, qc.GrowingSegmentsSizeProtectionEnabled)
assert.Equal(t, 0.5, qc.GrowingSegmentsSizeMinRateRatio)
assert.Equal(t, 0.2, qc.GrowingSegmentsSizeLowWaterLevel)
assert.Equal(t, 0.4, qc.GrowingSegmentsSizeHighWaterLevel)
assert.Equal(t, true, qc.DiskProtectionEnabled)