mirror of https://github.com/milvus-io/milvus.git
enhance: add disk quota and max collections into db properties (#34368)
issue: #34385 Signed-off-by: jaime <yun.zhang@zilliz.com>pull/34438/head
parent
a1c65dec89
commit
60be454db0
|
@ -335,7 +335,7 @@ func TestVectorCreateCollection(t *testing.T) {
|
|||
expectedBody: PrintErr(ErrDefault),
|
||||
})
|
||||
|
||||
err := merr.WrapErrCollectionNumLimitExceeded(65535)
|
||||
err := merr.WrapErrCollectionNumLimitExceeded("default", 65535)
|
||||
mp2 := mocks.NewMockProxy(t)
|
||||
mp2.EXPECT().CreateCollection(mock.Anything, mock.Anything).Return(merr.Status(err), nil).Once()
|
||||
testCases = append(testCases, testCase{
|
||||
|
|
|
@ -56,6 +56,15 @@ func (c *Database) Equal(other Database) bool {
|
|||
checkParamsEqual(c.Properties, other.Properties)
|
||||
}
|
||||
|
||||
func (c *Database) GetProperty(key string) string {
|
||||
for _, e := range c.Properties {
|
||||
if e.GetKey() == key {
|
||||
return e.GetValue()
|
||||
}
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func MarshalDatabaseModel(db *Database) *pb.DatabaseInfo {
|
||||
if db == nil {
|
||||
return nil
|
||||
|
|
|
@ -113,6 +113,7 @@ type collectionInfo struct {
|
|||
type databaseInfo struct {
|
||||
dbID typeutil.UniqueID
|
||||
createdTimestamp uint64
|
||||
properties map[string]string
|
||||
}
|
||||
|
||||
// schemaInfo is a helper function wraps *schemapb.CollectionSchema
|
||||
|
@ -1150,6 +1151,7 @@ func (m *MetaCache) GetDatabaseInfo(ctx context.Context, database string) (*data
|
|||
dbInfo := &databaseInfo{
|
||||
dbID: resp.GetDbID(),
|
||||
createdTimestamp: resp.GetCreatedTimestamp(),
|
||||
properties: funcutil.KeyValuePair2Map(resp.GetProperties()),
|
||||
}
|
||||
m.dbInfo[database] = dbInfo
|
||||
return dbInfo, nil
|
||||
|
|
|
@ -20,6 +20,7 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"strconv"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -81,17 +82,8 @@ func (t *createCollectionTask) validate() error {
|
|||
|
||||
// 2. check db-collection capacity
|
||||
db2CollIDs := t.core.meta.ListAllAvailCollections(t.ctx)
|
||||
|
||||
collIDs, ok := db2CollIDs[t.dbID]
|
||||
if !ok {
|
||||
log.Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName()))
|
||||
return merr.WrapErrDatabaseNotFound(t.Req.GetDbName(), "failed to create collection")
|
||||
}
|
||||
|
||||
maxColNumPerDB := Params.QuotaConfig.MaxCollectionNumPerDB.GetAsInt()
|
||||
if len(collIDs) >= maxColNumPerDB {
|
||||
log.Warn("unable to create collection because the number of collection has reached the limit in DB", zap.Int("maxCollectionNumPerDB", maxColNumPerDB))
|
||||
return merr.WrapErrCollectionNumLimitExceeded(maxColNumPerDB, "max number of collection has reached the limit in DB")
|
||||
if err := t.checkMaxCollectionsPerDB(db2CollIDs); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// 3. check total collection number
|
||||
|
@ -103,7 +95,7 @@ func (t *createCollectionTask) validate() error {
|
|||
maxCollectionNum := Params.QuotaConfig.MaxCollectionNum.GetAsInt()
|
||||
if totalCollections >= maxCollectionNum {
|
||||
log.Warn("unable to create collection because the number of collection has reached the limit", zap.Int("max_collection_num", maxCollectionNum))
|
||||
return merr.WrapErrCollectionNumLimitExceeded(maxCollectionNum, "max number of collection has reached the limit")
|
||||
return merr.WrapErrCollectionNumLimitExceeded(t.Req.GetDbName(), maxCollectionNum)
|
||||
}
|
||||
|
||||
// 4. check collection * shard * partition
|
||||
|
@ -114,6 +106,43 @@ func (t *createCollectionTask) validate() error {
|
|||
return checkGeneralCapacity(t.ctx, 1, newPartNum, t.Req.GetShardsNum(), t.core, t.ts)
|
||||
}
|
||||
|
||||
// checkMaxCollectionsPerDB DB properties take precedence over quota configurations for max collections.
|
||||
func (t *createCollectionTask) checkMaxCollectionsPerDB(db2CollIDs map[int64][]int64) error {
|
||||
collIDs, ok := db2CollIDs[t.dbID]
|
||||
if !ok {
|
||||
log.Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName()))
|
||||
return merr.WrapErrDatabaseNotFound(t.Req.GetDbName(), "failed to create collection")
|
||||
}
|
||||
|
||||
db, err := t.core.meta.GetDatabaseByName(t.ctx, t.Req.GetDbName(), typeutil.MaxTimestamp)
|
||||
if err != nil {
|
||||
log.Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName()))
|
||||
return merr.WrapErrDatabaseNotFound(t.Req.GetDbName(), "failed to create collection")
|
||||
}
|
||||
|
||||
check := func(maxColNumPerDB int) error {
|
||||
if len(collIDs) >= maxColNumPerDB {
|
||||
log.Warn("unable to create collection because the number of collection has reached the limit in DB", zap.Int("maxCollectionNumPerDB", maxColNumPerDB))
|
||||
return merr.WrapErrCollectionNumLimitExceeded(t.Req.GetDbName(), maxColNumPerDB)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
maxColNumPerDBStr := db.GetProperty(common.DatabaseMaxCollectionsKey)
|
||||
if maxColNumPerDBStr != "" {
|
||||
maxColNumPerDB, err := strconv.Atoi(maxColNumPerDBStr)
|
||||
if err != nil {
|
||||
log.Warn("parse value of property fail", zap.String("key", common.DatabaseMaxCollectionsKey),
|
||||
zap.String("value", maxColNumPerDBStr), zap.Error(err))
|
||||
return fmt.Errorf(fmt.Sprintf("parse value of property fail, key:%s, value:%s", common.DatabaseMaxCollectionsKey, maxColNumPerDBStr))
|
||||
}
|
||||
return check(maxColNumPerDB)
|
||||
}
|
||||
|
||||
maxColNumPerDB := Params.QuotaConfig.MaxCollectionNumPerDB.GetAsInt()
|
||||
return check(maxColNumPerDB)
|
||||
}
|
||||
|
||||
func checkDefaultValue(schema *schemapb.CollectionSchema) error {
|
||||
for _, fieldSchema := range schema.Fields {
|
||||
if fieldSchema.GetDefaultValue() != nil {
|
||||
|
|
|
@ -126,11 +126,13 @@ func Test_createCollectionTask_validate(t *testing.T) {
|
|||
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNum.Key)
|
||||
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.On("ListAllAvailCollections",
|
||||
meta.EXPECT().ListAllAvailCollections(
|
||||
mock.Anything,
|
||||
).Return(map[int64][]int64{
|
||||
1: {1, 2},
|
||||
}, nil)
|
||||
).Return(map[int64][]int64{1: {1, 2}})
|
||||
|
||||
meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&model.Database{Name: "db1"}, nil).Once()
|
||||
|
||||
core := newTestCore(withMeta(meta))
|
||||
task := createCollectionTask{
|
||||
baseTask: newBaseTask(context.TODO(), core),
|
||||
|
@ -152,16 +154,69 @@ func Test_createCollectionTask_validate(t *testing.T) {
|
|||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("collection num per db exceeds limit", func(t *testing.T) {
|
||||
t.Run("collection num per db exceeds limit with db properties", func(t *testing.T) {
|
||||
paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNumPerDB.Key, strconv.Itoa(2))
|
||||
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNumPerDB.Key)
|
||||
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.On("ListAllAvailCollections",
|
||||
mock.Anything,
|
||||
).Return(map[int64][]int64{
|
||||
1: {1, 2},
|
||||
}, nil)
|
||||
meta.EXPECT().ListAllAvailCollections(mock.Anything).Return(map[int64][]int64{util.DefaultDBID: {1, 2}})
|
||||
|
||||
// test reach limit
|
||||
meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&model.Database{
|
||||
Name: "db1",
|
||||
Properties: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DatabaseMaxCollectionsKey,
|
||||
Value: "2",
|
||||
},
|
||||
},
|
||||
}, nil).Once()
|
||||
|
||||
core := newTestCore(withMeta(meta))
|
||||
task := createCollectionTask{
|
||||
baseTask: newBaseTask(context.TODO(), core),
|
||||
Req: &milvuspb.CreateCollectionRequest{
|
||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||
},
|
||||
dbID: util.DefaultDBID,
|
||||
}
|
||||
err := task.validate()
|
||||
assert.Error(t, err)
|
||||
|
||||
// invalid properties
|
||||
meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&model.Database{
|
||||
Name: "db1",
|
||||
Properties: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DatabaseMaxCollectionsKey,
|
||||
Value: "invalid-value",
|
||||
},
|
||||
},
|
||||
}, nil).Once()
|
||||
core = newTestCore(withMeta(meta))
|
||||
task = createCollectionTask{
|
||||
baseTask: newBaseTask(context.TODO(), core),
|
||||
Req: &milvuspb.CreateCollectionRequest{
|
||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||
},
|
||||
dbID: util.DefaultDBID,
|
||||
}
|
||||
|
||||
err = task.validate()
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("collection num per db exceeds limit with global configuration", func(t *testing.T) {
|
||||
paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNumPerDB.Key, strconv.Itoa(2))
|
||||
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNumPerDB.Key)
|
||||
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.EXPECT().ListAllAvailCollections(mock.Anything).Return(map[int64][]int64{1: {1, 2}})
|
||||
meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&model.Database{Name: "db1"}, nil).Once()
|
||||
|
||||
core := newTestCore(withMeta(meta))
|
||||
task := createCollectionTask{
|
||||
baseTask: newBaseTask(context.TODO(), core),
|
||||
|
@ -188,11 +243,10 @@ func Test_createCollectionTask_validate(t *testing.T) {
|
|||
defer paramtable.Get().Reset(Params.RootCoordCfg.MaxGeneralCapacity.Key)
|
||||
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.On("ListAllAvailCollections",
|
||||
mock.Anything,
|
||||
).Return(map[int64][]int64{
|
||||
1: {1, 2},
|
||||
}, nil)
|
||||
meta.EXPECT().ListAllAvailCollections(mock.Anything).Return(map[int64][]int64{1: {1, 2}})
|
||||
meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&model.Database{Name: "db1"}, nil).Once()
|
||||
|
||||
meta.On("GetDatabaseByID",
|
||||
mock.Anything, mock.Anything, mock.Anything,
|
||||
).Return(&model.Database{
|
||||
|
@ -225,15 +279,24 @@ func Test_createCollectionTask_validate(t *testing.T) {
|
|||
assert.ErrorIs(t, err, merr.ErrGeneralCapacityExceeded)
|
||||
})
|
||||
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
t.Run("ok", func(t *testing.T) {
|
||||
paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNumPerDB.Key, "1")
|
||||
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNumPerDB.Key)
|
||||
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.On("ListAllAvailCollections",
|
||||
mock.Anything,
|
||||
).Return(map[int64][]int64{
|
||||
1: {1, 2},
|
||||
}, nil)
|
||||
meta.On("GetDatabaseByID", mock.Anything,
|
||||
mock.Anything, mock.Anything).Return(nil, errors.New("mock"))
|
||||
meta.EXPECT().ListAllAvailCollections(mock.Anything).Return(map[int64][]int64{1: {1, 2}})
|
||||
meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&model.Database{
|
||||
Name: "db1",
|
||||
Properties: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DatabaseMaxCollectionsKey,
|
||||
Value: "3",
|
||||
},
|
||||
},
|
||||
}, nil).Once()
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(nil, errors.New("mock"))
|
||||
|
||||
core := newTestCore(withMeta(meta))
|
||||
task := createCollectionTask{
|
||||
|
@ -249,9 +312,6 @@ func Test_createCollectionTask_validate(t *testing.T) {
|
|||
paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNum.Key, strconv.Itoa(math.MaxInt64))
|
||||
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNum.Key)
|
||||
|
||||
paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNumPerDB.Key, strconv.Itoa(math.MaxInt64))
|
||||
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNumPerDB.Key)
|
||||
|
||||
err := task.validate()
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
|
|
@ -27,6 +27,7 @@ import (
|
|||
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/exp/maps"
|
||||
"golang.org/x/sync/errgroup"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
|
@ -901,14 +902,37 @@ func (q *QuotaCenter) coolOffReading(realTimeSearchRate, realTimeQueryRate, cool
|
|||
}
|
||||
}
|
||||
|
||||
func (q *QuotaCenter) getDenyWritingDBs() map[int64]struct{} {
|
||||
dbIDs := make(map[int64]struct{})
|
||||
for _, dbID := range lo.Uniq(q.collectionIDToDBID.Values()) {
|
||||
if db, err := q.meta.GetDatabaseByID(q.ctx, dbID, typeutil.MaxTimestamp); err == nil {
|
||||
if v := db.GetProperty(common.DatabaseForceDenyWritingKey); v != "" {
|
||||
if dbForceDenyWritingEnabled, _ := strconv.ParseBool(v); dbForceDenyWritingEnabled {
|
||||
dbIDs[dbID] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return dbIDs
|
||||
}
|
||||
|
||||
// calculateWriteRates calculates and sets dml rates.
|
||||
func (q *QuotaCenter) calculateWriteRates() error {
|
||||
log := log.Ctx(context.Background()).WithRateGroup("rootcoord.QuotaCenter", 1.0, 60.0)
|
||||
// check force deny writing of cluster level
|
||||
if Params.QuotaConfig.ForceDenyWriting.GetAsBool() {
|
||||
return q.forceDenyWriting(commonpb.ErrorCode_ForceDeny, true, nil, nil, nil)
|
||||
}
|
||||
|
||||
if err := q.checkDiskQuota(); err != nil {
|
||||
// check force deny writing of db level
|
||||
dbIDs := q.getDenyWritingDBs()
|
||||
if len(dbIDs) != 0 {
|
||||
if err := q.forceDenyWriting(commonpb.ErrorCode_ForceDeny, false, maps.Keys(dbIDs), nil, nil); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
if err := q.checkDiskQuota(dbIDs); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -1288,7 +1312,7 @@ func (q *QuotaCenter) getCollectionLimitProperties(collection int64) map[string]
|
|||
}
|
||||
|
||||
// checkDiskQuota checks if disk quota exceeded.
|
||||
func (q *QuotaCenter) checkDiskQuota() error {
|
||||
func (q *QuotaCenter) checkDiskQuota(denyWritingDBs map[int64]struct{}) error {
|
||||
q.diskMu.Lock()
|
||||
defer q.diskMu.Unlock()
|
||||
if !Params.QuotaConfig.DiskProtectionEnabled.GetAsBool() {
|
||||
|
@ -1298,6 +1322,7 @@ func (q *QuotaCenter) checkDiskQuota() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// check disk quota of cluster level
|
||||
totalDiskQuota := Params.QuotaConfig.DiskQuota.GetAsFloat()
|
||||
total := q.dataCoordMetrics.TotalBinlogSize
|
||||
if float64(total) >= totalDiskQuota {
|
||||
|
@ -1326,19 +1351,14 @@ func (q *QuotaCenter) checkDiskQuota() error {
|
|||
log.Warn("cannot find db id for collection", zap.Int64("collection", collection))
|
||||
continue
|
||||
}
|
||||
dbSizeInfo[dbID] += binlogSize
|
||||
}
|
||||
|
||||
dbs := make([]int64, 0)
|
||||
dbDiskQuota := Params.QuotaConfig.DiskQuotaPerDB.GetAsFloat()
|
||||
for dbID, binlogSize := range dbSizeInfo {
|
||||
if float64(binlogSize) >= dbDiskQuota {
|
||||
log.RatedWarn(10, "db disk quota exceeded",
|
||||
zap.Int64("db", dbID),
|
||||
zap.Int64("db disk usage", binlogSize),
|
||||
zap.Float64("db disk quota", dbDiskQuota))
|
||||
dbs = append(dbs, dbID)
|
||||
// skip db that has already denied writing
|
||||
if denyWritingDBs != nil {
|
||||
if _, ok = denyWritingDBs[dbID]; ok {
|
||||
continue
|
||||
}
|
||||
}
|
||||
dbSizeInfo[dbID] += binlogSize
|
||||
}
|
||||
|
||||
col2partitions := make(map[int64][]int64)
|
||||
|
@ -1356,7 +1376,8 @@ func (q *QuotaCenter) checkDiskQuota() error {
|
|||
}
|
||||
}
|
||||
|
||||
err := q.forceDenyWriting(commonpb.ErrorCode_DiskQuotaExhausted, false, dbs, collections, col2partitions)
|
||||
dbIDs := q.checkDBDiskQuota(dbSizeInfo)
|
||||
err := q.forceDenyWriting(commonpb.ErrorCode_DiskQuotaExhausted, false, dbIDs, collections, col2partitions)
|
||||
if err != nil {
|
||||
log.Warn("fail to force deny writing", zap.Error(err))
|
||||
return err
|
||||
|
@ -1365,6 +1386,35 @@ func (q *QuotaCenter) checkDiskQuota() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (q *QuotaCenter) checkDBDiskQuota(dbSizeInfo map[int64]int64) []int64 {
|
||||
dbIDs := make([]int64, 0)
|
||||
checkDiskQuota := func(dbID, binlogSize int64, quota float64) {
|
||||
if float64(binlogSize) >= quota {
|
||||
log.RatedWarn(10, "db disk quota exceeded",
|
||||
zap.Int64("db", dbID),
|
||||
zap.Int64("db disk usage", binlogSize),
|
||||
zap.Float64("db disk quota", quota))
|
||||
dbIDs = append(dbIDs, dbID)
|
||||
}
|
||||
}
|
||||
|
||||
// DB properties take precedence over quota configuration for disk quota.
|
||||
for dbID, binlogSize := range dbSizeInfo {
|
||||
db, err := q.meta.GetDatabaseByID(q.ctx, dbID, typeutil.MaxTimestamp)
|
||||
if err == nil {
|
||||
if dbDiskQuotaStr := db.GetProperty(common.DatabaseDiskQuotaKey); dbDiskQuotaStr != "" {
|
||||
if dbDiskQuotaBytes, err := strconv.ParseFloat(dbDiskQuotaStr, 64); err == nil {
|
||||
dbDiskQuotaMB := dbDiskQuotaBytes * 1024 * 1024
|
||||
checkDiskQuota(dbID, binlogSize, dbDiskQuotaMB)
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
checkDiskQuota(dbID, binlogSize, Params.QuotaConfig.DiskQuotaPerDB.GetAsFloat())
|
||||
}
|
||||
return dbIDs
|
||||
}
|
||||
|
||||
func (q *QuotaCenter) toRequestLimiter(limiter *rlinternal.RateLimiterNode) *proxypb.Limiter {
|
||||
var rates []*internalpb.Rate
|
||||
switch q.rateAllocateStrategy {
|
||||
|
|
|
@ -21,6 +21,7 @@ import (
|
|||
"encoding/json"
|
||||
"fmt"
|
||||
"math"
|
||||
"strconv"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -74,6 +75,7 @@ func TestQuotaCenter(t *testing.T) {
|
|||
collectionIDToDBID.Insert(1, 0)
|
||||
collectionIDToDBID.Insert(2, 0)
|
||||
collectionIDToDBID.Insert(3, 0)
|
||||
collectionIDToDBID.Insert(4, 1)
|
||||
|
||||
t.Run("test QuotaCenter", func(t *testing.T) {
|
||||
qc := mocks.NewMockQueryCoordClient(t)
|
||||
|
@ -467,6 +469,7 @@ func TestQuotaCenter(t *testing.T) {
|
|||
qc := mocks.NewMockQueryCoordClient(t)
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.EXPECT().GetCollectionByIDWithMaxTs(mock.Anything, mock.Anything).Return(nil, merr.ErrCollectionNotFound).Maybe()
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).Return(nil, merr.ErrDatabaseNotFound).Maybe()
|
||||
quotaCenter := NewQuotaCenter(pcm, qc, dc, core.tsoAllocator, meta)
|
||||
type ttCase struct {
|
||||
delay time.Duration
|
||||
|
@ -663,6 +666,7 @@ func TestQuotaCenter(t *testing.T) {
|
|||
paramtable.Get().Save(Params.QuotaConfig.ForceDenyWriting.Key, "true")
|
||||
quotaCenter.writableCollections = map[int64]map[int64][]int64{
|
||||
0: collectionIDToPartitionIDs,
|
||||
1: {4: {}},
|
||||
}
|
||||
quotaCenter.collectionIDToDBID = collectionIDToDBID
|
||||
quotaCenter.collectionIDToDBID = collectionIDToDBID
|
||||
|
@ -679,6 +683,47 @@ func TestQuotaCenter(t *testing.T) {
|
|||
|
||||
paramtable.Get().Reset(Params.QuotaConfig.ForceDenyWriting.Key)
|
||||
|
||||
// force deny writing for databases
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).
|
||||
RunAndReturn(func(ctx context.Context, i int64, u uint64) (*model.Database, error) {
|
||||
if i == 1 {
|
||||
return &model.Database{
|
||||
ID: 1,
|
||||
Name: "db4",
|
||||
Properties: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DatabaseForceDenyWritingKey,
|
||||
Value: "true",
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
return nil, errors.New("mock error")
|
||||
}).Maybe()
|
||||
quotaCenter.resetAllCurrentRates()
|
||||
err = quotaCenter.calculateWriteRates()
|
||||
assert.NoError(t, err)
|
||||
rln := quotaCenter.rateLimiter.GetDatabaseLimiters(0)
|
||||
limiters = rln.GetLimiters()
|
||||
a, _ = limiters.Get(internalpb.RateType_DMLInsert)
|
||||
assert.NotEqual(t, Limit(0), a.Limit())
|
||||
b, _ = limiters.Get(internalpb.RateType_DMLUpsert)
|
||||
assert.NotEqual(t, Limit(0), b.Limit())
|
||||
c, _ = limiters.Get(internalpb.RateType_DMLDelete)
|
||||
assert.NotEqual(t, Limit(0), c.Limit())
|
||||
|
||||
rln = quotaCenter.rateLimiter.GetDatabaseLimiters(1)
|
||||
limiters = rln.GetLimiters()
|
||||
a, _ = limiters.Get(internalpb.RateType_DMLInsert)
|
||||
assert.Equal(t, Limit(0), a.Limit())
|
||||
b, _ = limiters.Get(internalpb.RateType_DMLUpsert)
|
||||
assert.Equal(t, Limit(0), b.Limit())
|
||||
c, _ = limiters.Get(internalpb.RateType_DMLDelete)
|
||||
assert.Equal(t, Limit(0), c.Limit())
|
||||
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).Unset()
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).Return(nil, merr.ErrDatabaseNotFound).Maybe()
|
||||
|
||||
// disable tt delay protection
|
||||
disableTtBak := Params.QuotaConfig.TtProtectionEnabled.GetValue()
|
||||
paramtable.Get().Save(Params.QuotaConfig.TtProtectionEnabled.Key, "false")
|
||||
|
@ -697,7 +742,11 @@ func TestQuotaCenter(t *testing.T) {
|
|||
for collection := range collections {
|
||||
states := quotaCenter.rateLimiter.GetCollectionLimiters(db, collection).GetQuotaStates()
|
||||
code, _ := states.Get(milvuspb.QuotaState_DenyToWrite)
|
||||
assert.Equal(t, commonpb.ErrorCode_MemoryQuotaExhausted, code)
|
||||
if db == 0 {
|
||||
assert.Equal(t, commonpb.ErrorCode_MemoryQuotaExhausted, code)
|
||||
} else {
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, code)
|
||||
}
|
||||
}
|
||||
}
|
||||
paramtable.Get().Save(Params.QuotaConfig.TtProtectionEnabled.Key, disableTtBak)
|
||||
|
@ -821,8 +870,9 @@ func TestQuotaCenter(t *testing.T) {
|
|||
qc := mocks.NewMockQueryCoordClient(t)
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
meta.EXPECT().GetCollectionByIDWithMaxTs(mock.Anything, mock.Anything).Return(nil, merr.ErrCollectionNotFound).Maybe()
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).Return(nil, merr.ErrDatabaseNotFound).Maybe()
|
||||
quotaCenter := NewQuotaCenter(pcm, qc, dc, core.tsoAllocator, meta)
|
||||
quotaCenter.checkDiskQuota()
|
||||
quotaCenter.checkDiskQuota(nil)
|
||||
|
||||
checkLimiter := func(notEquals ...int64) {
|
||||
for db, collections := range quotaCenter.writableCollections {
|
||||
|
@ -863,7 +913,7 @@ func TestQuotaCenter(t *testing.T) {
|
|||
}
|
||||
quotaCenter.collectionIDToDBID = collectionIDToDBID
|
||||
quotaCenter.resetAllCurrentRates()
|
||||
quotaCenter.checkDiskQuota()
|
||||
quotaCenter.checkDiskQuota(nil)
|
||||
checkLimiter()
|
||||
paramtable.Get().Reset(Params.QuotaConfig.DiskQuota.Key)
|
||||
paramtable.Get().Reset(Params.QuotaConfig.DiskQuotaPerCollection.Key)
|
||||
|
@ -878,7 +928,7 @@ func TestQuotaCenter(t *testing.T) {
|
|||
0: collectionIDToPartitionIDs,
|
||||
}
|
||||
quotaCenter.resetAllCurrentRates()
|
||||
quotaCenter.checkDiskQuota()
|
||||
quotaCenter.checkDiskQuota(nil)
|
||||
checkLimiter(1)
|
||||
paramtable.Get().Save(Params.QuotaConfig.DiskQuotaPerCollection.Key, colQuotaBackup)
|
||||
})
|
||||
|
@ -1686,6 +1736,72 @@ func TestResetAllCurrentRates(t *testing.T) {
|
|||
assert.NotNil(t, collection)
|
||||
}
|
||||
|
||||
func newQuotaCenterForTesting(t *testing.T, ctx context.Context, meta IMetaTable) *QuotaCenter {
|
||||
qc := mocks.NewMockQueryCoordClient(t)
|
||||
pcm := proxyutil.NewMockProxyClientManager(t)
|
||||
dc := mocks.NewMockDataCoordClient(t)
|
||||
core, _ := NewCore(ctx, nil)
|
||||
core.tsoAllocator = newMockTsoAllocator()
|
||||
quotaCenter := NewQuotaCenter(pcm, qc, dc, core.tsoAllocator, meta)
|
||||
quotaCenter.rateLimiter.GetRootLimiters().GetLimiters().Insert(internalpb.RateType_DMLInsert, ratelimitutil.NewLimiter(500, 500))
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(1, 10, 100,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(1, 10, 101,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(2, 20, 200,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(2, 30, 300,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(4, 40, 400,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
|
||||
quotaCenter.dataCoordMetrics = &metricsinfo.DataCoordQuotaMetrics{
|
||||
TotalBinlogSize: 200 * 1024 * 1024,
|
||||
CollectionBinlogSize: map[int64]int64{
|
||||
10: 15 * 1024 * 1024,
|
||||
20: 6 * 1024 * 1024,
|
||||
30: 6 * 1024 * 1024,
|
||||
40: 4 * 1024 * 1024,
|
||||
},
|
||||
PartitionsBinlogSize: map[int64]map[int64]int64{
|
||||
10: {
|
||||
100: 10 * 1024 * 1024,
|
||||
101: 5 * 1024 * 1024,
|
||||
},
|
||||
20: {
|
||||
200: 6 * 1024 * 1024,
|
||||
},
|
||||
30: {
|
||||
300: 6 * 1024 * 1024,
|
||||
},
|
||||
40: {
|
||||
400: 4 * 1024 * 1024,
|
||||
},
|
||||
},
|
||||
}
|
||||
quotaCenter.collectionIDToDBID = typeutil.NewConcurrentMap[int64, int64]()
|
||||
quotaCenter.collectionIDToDBID.Insert(10, 1)
|
||||
quotaCenter.collectionIDToDBID.Insert(20, 2)
|
||||
quotaCenter.collectionIDToDBID.Insert(30, 2)
|
||||
quotaCenter.collectionIDToDBID.Insert(40, 4)
|
||||
return quotaCenter
|
||||
}
|
||||
|
||||
func TestCheckDiskQuota(t *testing.T) {
|
||||
paramtable.Init()
|
||||
ctx := context.Background()
|
||||
|
@ -1701,20 +1817,21 @@ func TestCheckDiskQuota(t *testing.T) {
|
|||
|
||||
Params.Save(Params.QuotaConfig.DiskProtectionEnabled.Key, "false")
|
||||
defer Params.Reset(Params.QuotaConfig.DiskProtectionEnabled.Key)
|
||||
err := quotaCenter.checkDiskQuota()
|
||||
err := quotaCenter.checkDiskQuota(nil)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("disk quota check enable", func(t *testing.T) {
|
||||
diskQuotaStr := "10"
|
||||
Params.Save(Params.QuotaConfig.DiskProtectionEnabled.Key, "true")
|
||||
defer Params.Reset(Params.QuotaConfig.DiskProtectionEnabled.Key)
|
||||
Params.Save(Params.QuotaConfig.DiskQuota.Key, "150")
|
||||
defer Params.Reset(Params.QuotaConfig.DiskQuota.Key)
|
||||
Params.Save(Params.QuotaConfig.DiskQuotaPerDB.Key, "10")
|
||||
Params.Save(Params.QuotaConfig.DiskQuotaPerDB.Key, diskQuotaStr)
|
||||
defer Params.Reset(Params.QuotaConfig.DiskQuotaPerDB.Key)
|
||||
Params.Save(Params.QuotaConfig.DiskQuotaPerCollection.Key, "10")
|
||||
Params.Save(Params.QuotaConfig.DiskQuotaPerCollection.Key, diskQuotaStr)
|
||||
defer Params.Reset(Params.QuotaConfig.DiskQuotaPerCollection.Key)
|
||||
Params.Save(Params.QuotaConfig.DiskQuotaPerPartition.Key, "10")
|
||||
Params.Save(Params.QuotaConfig.DiskQuotaPerPartition.Key, diskQuotaStr)
|
||||
defer Params.Reset(Params.QuotaConfig.DiskQuotaPerPartition.Key)
|
||||
|
||||
Params.Save(Params.QuotaConfig.DMLLimitEnabled.Key, "true")
|
||||
|
@ -1728,61 +1845,25 @@ func TestCheckDiskQuota(t *testing.T) {
|
|||
Params.Save(Params.QuotaConfig.DMLMaxInsertRatePerPartition.Key, "10")
|
||||
defer Params.Reset(Params.QuotaConfig.DMLMaxInsertRatePerPartition.Key)
|
||||
|
||||
qc := mocks.NewMockQueryCoordClient(t)
|
||||
meta := mockrootcoord.NewIMetaTable(t)
|
||||
pcm := proxyutil.NewMockProxyClientManager(t)
|
||||
dc := mocks.NewMockDataCoordClient(t)
|
||||
core, _ := NewCore(ctx, nil)
|
||||
core.tsoAllocator = newMockTsoAllocator()
|
||||
meta.EXPECT().GetCollectionByIDWithMaxTs(mock.Anything, mock.Anything).Return(nil, errors.New("mock error"))
|
||||
|
||||
quotaCenter := NewQuotaCenter(pcm, qc, dc, core.tsoAllocator, meta)
|
||||
quotaCenter.rateLimiter.GetRootLimiters().GetLimiters().Insert(internalpb.RateType_DMLInsert, ratelimitutil.NewLimiter(500, 500))
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(1, 10, 100,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(1, 10, 101,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(2, 20, 200,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
quotaCenter.rateLimiter.GetOrCreatePartitionLimiters(2, 30, 300,
|
||||
newParamLimiterFunc(internalpb.RateScope_Database, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Collection, allOps),
|
||||
newParamLimiterFunc(internalpb.RateScope_Partition, allOps),
|
||||
)
|
||||
|
||||
quotaCenter.dataCoordMetrics = &metricsinfo.DataCoordQuotaMetrics{
|
||||
TotalBinlogSize: 200 * 1024 * 1024,
|
||||
CollectionBinlogSize: map[int64]int64{
|
||||
10: 15 * 1024 * 1024,
|
||||
20: 6 * 1024 * 1024,
|
||||
30: 6 * 1024 * 1024,
|
||||
},
|
||||
PartitionsBinlogSize: map[int64]map[int64]int64{
|
||||
10: {
|
||||
100: 10 * 1024 * 1024,
|
||||
101: 5 * 1024 * 1024,
|
||||
},
|
||||
20: {
|
||||
200: 6 * 1024 * 1024,
|
||||
},
|
||||
30: {
|
||||
300: 6 * 1024 * 1024,
|
||||
},
|
||||
},
|
||||
}
|
||||
quotaCenter.collectionIDToDBID = typeutil.NewConcurrentMap[int64, int64]()
|
||||
quotaCenter.collectionIDToDBID.Insert(10, 1)
|
||||
quotaCenter.collectionIDToDBID.Insert(20, 2)
|
||||
quotaCenter.collectionIDToDBID.Insert(30, 2)
|
||||
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).
|
||||
RunAndReturn(func(ctx context.Context, i int64, u uint64) (*model.Database, error) {
|
||||
if i == 4 {
|
||||
return &model.Database{
|
||||
ID: 1,
|
||||
Name: "db4",
|
||||
Properties: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DatabaseDiskQuotaKey,
|
||||
Value: "2",
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
return nil, errors.New("mock error")
|
||||
}).Maybe()
|
||||
quotaCenter := newQuotaCenterForTesting(t, ctx, meta)
|
||||
|
||||
checkRate := func(rateNode *interalratelimitutil.RateLimiterNode, expectValue float64) {
|
||||
insertRate, ok := rateNode.GetLimiters().Get(internalpb.RateType_DMLInsert)
|
||||
|
@ -1790,27 +1871,32 @@ func TestCheckDiskQuota(t *testing.T) {
|
|||
assert.EqualValues(t, expectValue, insertRate.Limit())
|
||||
}
|
||||
|
||||
configQuotaValue := float64(10 * 1024 * 1024)
|
||||
diskQuota, err := strconv.ParseFloat(diskQuotaStr, 64)
|
||||
assert.NoError(t, err)
|
||||
configQuotaValue := 1024 * 1024 * diskQuota
|
||||
|
||||
{
|
||||
err := quotaCenter.checkDiskQuota()
|
||||
err := quotaCenter.checkDiskQuota(nil)
|
||||
assert.NoError(t, err)
|
||||
checkRate(quotaCenter.rateLimiter.GetRootLimiters(), 0)
|
||||
}
|
||||
|
||||
{
|
||||
Params.Save(Params.QuotaConfig.DiskQuota.Key, "999")
|
||||
err := quotaCenter.checkDiskQuota()
|
||||
err := quotaCenter.checkDiskQuota(nil)
|
||||
assert.NoError(t, err)
|
||||
checkRate(quotaCenter.rateLimiter.GetDatabaseLimiters(1), 0)
|
||||
checkRate(quotaCenter.rateLimiter.GetDatabaseLimiters(2), 0)
|
||||
checkRate(quotaCenter.rateLimiter.GetDatabaseLimiters(4), 0)
|
||||
checkRate(quotaCenter.rateLimiter.GetCollectionLimiters(1, 10), 0)
|
||||
checkRate(quotaCenter.rateLimiter.GetCollectionLimiters(2, 20), configQuotaValue)
|
||||
checkRate(quotaCenter.rateLimiter.GetCollectionLimiters(2, 30), configQuotaValue)
|
||||
checkRate(quotaCenter.rateLimiter.GetCollectionLimiters(4, 40), configQuotaValue)
|
||||
checkRate(quotaCenter.rateLimiter.GetPartitionLimiters(1, 10, 100), 0)
|
||||
checkRate(quotaCenter.rateLimiter.GetPartitionLimiters(1, 10, 101), configQuotaValue)
|
||||
checkRate(quotaCenter.rateLimiter.GetPartitionLimiters(2, 20, 200), configQuotaValue)
|
||||
checkRate(quotaCenter.rateLimiter.GetPartitionLimiters(2, 30, 300), configQuotaValue)
|
||||
checkRate(quotaCenter.rateLimiter.GetPartitionLimiters(4, 40, 400), configQuotaValue)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
|
|
@ -147,8 +147,11 @@ const (
|
|||
PartitionDiskQuotaKey = "partition.diskProtection.diskQuota.mb"
|
||||
|
||||
// database level properties
|
||||
DatabaseReplicaNumber = "database.replica.number"
|
||||
DatabaseResourceGroups = "database.resource_groups"
|
||||
DatabaseReplicaNumber = "database.replica.number"
|
||||
DatabaseResourceGroups = "database.resource_groups"
|
||||
DatabaseDiskQuotaKey = "database.diskQuota.mb"
|
||||
DatabaseMaxCollectionsKey = "database.max.collections"
|
||||
DatabaseForceDenyWritingKey = "database.force.deny.writing"
|
||||
)
|
||||
|
||||
// common properties
|
||||
|
|
|
@ -483,8 +483,8 @@ func WrapErrCollectionNotLoaded(collection any, msg ...string) error {
|
|||
return err
|
||||
}
|
||||
|
||||
func WrapErrCollectionNumLimitExceeded(limit int, msg ...string) error {
|
||||
err := wrapFields(ErrCollectionNumLimitExceeded, value("limit", limit))
|
||||
func WrapErrCollectionNumLimitExceeded(db string, limit int, msg ...string) error {
|
||||
err := wrapFields(ErrCollectionNumLimitExceeded, value("dbName", db), value("limit", limit))
|
||||
if len(msg) > 0 {
|
||||
err = errors.Wrap(err, strings.Join(msg, "->"))
|
||||
}
|
||||
|
|
|
@ -112,3 +112,12 @@ func (m *ConcurrentMap[K, V]) Remove(key K) {
|
|||
func (m *ConcurrentMap[K, V]) Len() int {
|
||||
return int(m.len.Load())
|
||||
}
|
||||
|
||||
func (m *ConcurrentMap[K, V]) Values() []V {
|
||||
ret := make([]V, m.Len())
|
||||
m.inner.Range(func(key, value any) bool {
|
||||
ret = append(ret, value.(V))
|
||||
return true
|
||||
})
|
||||
return ret
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue