Skip to content

Commit

Permalink
fix: [2.4] Fix checkGeneralCapacity slowly (#38074)
Browse files Browse the repository at this point in the history
Cache the general count to speed up checkGeneralCapacity.

issue: #37630

pr: #37976

---------

Signed-off-by: bigsheeper <[email protected]>
  • Loading branch information
bigsheeper authored Dec 5, 2024
1 parent 8646bba commit 1f8299f
Show file tree
Hide file tree
Showing 7 changed files with 72 additions and 83 deletions.
25 changes: 4 additions & 21 deletions internal/rootcoord/constrant.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"context"

"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

const (
Expand All @@ -35,7 +34,6 @@ func checkGeneralCapacity(ctx context.Context, newColNum int,
newParNum int64,
newShardNum int32,
core *Core,
ts typeutil.Timestamp,
) error {
var addedNum int64 = 0
if newColNum > 0 && newParNum > 0 && newShardNum > 0 {
Expand All @@ -46,25 +44,10 @@ func checkGeneralCapacity(ctx context.Context, newColNum int,
addedNum += newParNum
}

var generalNum int64 = 0
collectionsMap := core.meta.ListAllAvailCollections(ctx)
for dbId, collectionIDs := range collectionsMap {
db, err := core.meta.GetDatabaseByID(ctx, dbId, ts)
if err == nil {
for _, collectionId := range collectionIDs {
collection, err := core.meta.GetCollectionByID(ctx, db.Name, collectionId, ts, true)
if err == nil {
partNum := int64(collection.GetPartitionNum(false))
shardNum := int64(collection.ShardsNum)
generalNum += partNum * shardNum
}
}
}
}

generalNum += addedNum
if generalNum > Params.RootCoordCfg.MaxGeneralCapacity.GetAsInt64() {
return merr.WrapGeneralCapacityExceed(generalNum, Params.RootCoordCfg.MaxGeneralCapacity.GetAsInt64(),
generalCount := core.meta.GetGeneralCount(ctx)
generalCount += int(addedNum)
if generalCount > Params.RootCoordCfg.MaxGeneralCapacity.GetAsInt() {
return merr.WrapGeneralCapacityExceed(generalCount, Params.RootCoordCfg.MaxGeneralCapacity.GetAsInt64(),
"failed checking constraint: sum_collections(parition*shard) exceeding the max general capacity:")
}
return nil
Expand Down
2 changes: 1 addition & 1 deletion internal/rootcoord/create_collection_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ func (t *createCollectionTask) validate() error {
if t.Req.GetNumPartitions() > 0 {
newPartNum = t.Req.GetNumPartitions()
}
return checkGeneralCapacity(t.ctx, 1, newPartNum, t.Req.GetShardsNum(), t.core, t.ts)
return checkGeneralCapacity(t.ctx, 1, newPartNum, t.Req.GetShardsNum(), t.core)
}

// checkMaxCollectionsPerDB DB properties take precedence over quota configurations for max collections.
Expand Down
31 changes: 4 additions & 27 deletions internal/rootcoord/create_collection_task_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,23 +246,7 @@ func Test_createCollectionTask_validate(t *testing.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()

meta.On("GetDatabaseByID",
mock.Anything, mock.Anything, mock.Anything,
).Return(&model.Database{
Name: "default",
}, nil)
meta.On("GetCollectionByID",
mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything,
).Return(&model.Collection{
Name: "default",
ShardsNum: 2,
Partitions: []*model.Partition{
{
PartitionID: 1,
},
},
}, nil)
meta.EXPECT().GetGeneralCount(mock.Anything).Return(1)

core := newTestCore(withMeta(meta))

Expand Down Expand Up @@ -295,8 +279,7 @@ func Test_createCollectionTask_validate(t *testing.T) {
},
},
}, nil).Once()
meta.EXPECT().GetDatabaseByID(mock.Anything, mock.Anything, mock.Anything).
Return(nil, errors.New("mock"))
meta.EXPECT().GetGeneralCount(mock.Anything).Return(0)

core := newTestCore(withMeta(meta))
task := createCollectionTask{
Expand Down Expand Up @@ -642,6 +625,7 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
).Return(map[int64][]int64{
util.DefaultDBID: {1, 2},
}, nil)
meta.EXPECT().GetGeneralCount(mock.Anything).Return(0)

paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNum.Key, strconv.Itoa(math.MaxInt64))
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNum.Key)
Expand All @@ -662,8 +646,6 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
})

t.Run("invalid schema", func(t *testing.T) {
meta.On("GetDatabaseByID", mock.Anything,
mock.Anything, mock.Anything).Return(nil, errors.New("mock"))
core := newTestCore(withMeta(meta))
collectionName := funcutil.GenRandomStr()
task := &createCollectionTask{
Expand Down Expand Up @@ -692,8 +674,6 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
}
marshaledSchema, err := proto.Marshal(schema)
assert.NoError(t, err)
meta.On("GetDatabaseByID", mock.Anything,
mock.Anything, mock.Anything).Return(nil, errors.New("mock"))
core := newTestCore(withInvalidIDAllocator(), withMeta(meta))

task := createCollectionTask{
Expand All @@ -716,8 +696,6 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
field1 := funcutil.GenRandomStr()

ticker := newRocksMqTtSynchronizer()
meta.On("GetDatabaseByID", mock.Anything,
mock.Anything, mock.Anything).Return(nil, errors.New("mock"))

core := newTestCore(withValidIDAllocator(), withTtSynchronizer(ticker), withMeta(meta))

Expand Down Expand Up @@ -1056,8 +1034,7 @@ func Test_createCollectionTask_PartitionKey(t *testing.T) {
).Return(map[int64][]int64{
util.DefaultDBID: {1, 2},
}, nil)
meta.On("GetDatabaseByID", mock.Anything,
mock.Anything, mock.Anything).Return(nil, errors.New("mock"))
meta.EXPECT().GetGeneralCount(mock.Anything).Return(0)

paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNum.Key, strconv.Itoa(math.MaxInt64))
defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNum.Key)
Expand Down
2 changes: 1 addition & 1 deletion internal/rootcoord/create_partition_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func (t *createPartitionTask) Prepare(ctx context.Context) error {
return err
}
t.collMeta = collMeta
return checkGeneralCapacity(ctx, 0, 1, 0, t.core, t.ts)
return checkGeneralCapacity(ctx, 0, 1, 0, t.core)
}

func (t *createPartitionTask) Execute(ctx context.Context) error {
Expand Down
10 changes: 1 addition & 9 deletions internal/rootcoord/create_partition_task_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"context"
"testing"

"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"

Expand Down Expand Up @@ -62,14 +61,7 @@ func Test_createPartitionTask_Prepare(t *testing.T) {
mock.Anything,
mock.Anything,
).Return(coll.Clone(), nil)
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().GetGeneralCount(mock.Anything).Return(0)

core := newTestCore(withMeta(meta))
task := &createPartitionTask{
Expand Down
19 changes: 19 additions & 0 deletions internal/rootcoord/meta_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ type IMetaTable interface {
ListAliases(ctx context.Context, dbName string, collectionName string, ts Timestamp) ([]string, error)
AlterCollection(ctx context.Context, oldColl *model.Collection, newColl *model.Collection, ts Timestamp) error
RenameCollection(ctx context.Context, dbName string, oldName string, newDBName string, newName string, ts Timestamp) error
GetGeneralCount(ctx context.Context) int

// TODO: it'll be a big cost if we handle the time travel logic, since we should always list all aliases in catalog.
IsAlias(db, name string) bool
Expand Down Expand Up @@ -114,6 +115,8 @@ type MetaTable struct {
dbName2Meta map[string]*model.Database // database name -> db meta
collID2Meta map[typeutil.UniqueID]*model.Collection // collection id -> collection meta

generalCnt int // sum of product of partition number and shard number

// collections *collectionDb
names *nameDb
aliases *nameDb
Expand Down Expand Up @@ -187,6 +190,7 @@ func (mt *MetaTable) reload() error {
}
for _, collection := range collections {
mt.collID2Meta[collection.CollectionID] = collection
mt.generalCnt += len(collection.Partitions) * int(collection.ShardsNum)
if collection.Available() {
mt.names.insert(dbName, collection.Name, collection.CollectionID)
collectionNum++
Expand Down Expand Up @@ -409,6 +413,8 @@ func (mt *MetaTable) AddCollection(ctx context.Context, coll *model.Collection)
mt.collID2Meta[coll.CollectionID] = coll.Clone()
mt.names.insert(db.Name, coll.Name, coll.CollectionID)

mt.generalCnt += len(coll.Partitions) * int(coll.ShardsNum)

log.Ctx(ctx).Info("add collection to meta table",
zap.Int64("dbID", coll.DBID),
zap.String("collection", coll.Name),
Expand Down Expand Up @@ -513,6 +519,8 @@ func (mt *MetaTable) RemoveCollection(ctx context.Context, collectionID UniqueID
mt.removeAllNamesIfMatchedInternal(collectionID, allNames)
mt.removeCollectionByIDInternal(collectionID)

mt.generalCnt -= len(coll.Partitions) * int(coll.ShardsNum)

log.Ctx(ctx).Info("remove collection",
zap.Int64("dbID", coll.DBID),
zap.String("name", coll.Name),
Expand Down Expand Up @@ -861,6 +869,8 @@ func (mt *MetaTable) AddPartition(ctx context.Context, partition *model.Partitio
}
mt.collID2Meta[partition.CollectionID].Partitions = append(mt.collID2Meta[partition.CollectionID].Partitions, partition.Clone())

mt.generalCnt += int(coll.ShardsNum) // 1 partition * shardNum

log.Ctx(ctx).Info("add partition to meta table",
zap.Int64("collection", partition.CollectionID), zap.String("partition", partition.PartitionName),
zap.Int64("partitionid", partition.PartitionID), zap.Uint64("ts", partition.PartitionCreatedTimestamp))
Expand Down Expand Up @@ -925,6 +935,7 @@ func (mt *MetaTable) RemovePartition(ctx context.Context, dbID int64, collection
}
if loc != -1 {
coll.Partitions = append(coll.Partitions[:loc], coll.Partitions[loc+1:]...)
mt.generalCnt -= int(coll.ShardsNum) // 1 partition * shardNum
}
log.Info("remove partition", zap.Int64("collection", collectionID), zap.Int64("partition", partitionID), zap.Uint64("ts", ts))
return nil
Expand Down Expand Up @@ -1193,6 +1204,14 @@ func (mt *MetaTable) ListAliasesByID(collID UniqueID) []string {
return mt.listAliasesByID(collID)
}

// GetGeneralCount gets the general count(sum of product of partition number and shard number).
func (mt *MetaTable) GetGeneralCount(ctx context.Context) int {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()

return mt.generalCnt
}

// AddCredential add credential
func (mt *MetaTable) AddCredential(credInfo *internalpb.CredentialInfo) error {
if credInfo.Username == "" {
Expand Down
66 changes: 42 additions & 24 deletions internal/rootcoord/mocks/meta_table.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

0 comments on commit 1f8299f

Please sign in to comment.