2021-12-16 10:07:10 +08:00
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
2021-04-19 11:12:56 +08:00
// with the License. You may obtain a copy of the License at
//
2021-12-16 10:07:10 +08:00
// http://www.apache.org/licenses/LICENSE-2.0
2021-04-19 11:12:56 +08:00
//
2021-12-16 10:07:10 +08:00
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
2021-04-19 11:12:56 +08:00
2021-06-18 21:30:08 +08:00
package rootcoord
2021-01-19 14:44:03 +08:00
import (
2022-07-22 10:20:29 +08:00
"context"
2021-03-05 10:15:27 +08:00
"fmt"
2021-01-19 14:44:03 +08:00
"sync"
2023-02-26 11:31:49 +08:00
"github.com/cockroachdb/errors"
2022-08-25 15:48:54 +08:00
"go.uber.org/zap"
2023-06-25 17:20:43 +08:00
"golang.org/x/exp/maps"
2022-08-25 15:48:54 +08:00
2023-06-09 01:28:37 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
2022-07-22 10:20:29 +08:00
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/metastore/model"
2023-01-19 14:13:43 +08:00
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
2022-04-11 19:49:34 +08:00
"github.com/milvus-io/milvus/internal/proto/internalpb"
2023-06-25 17:20:43 +08:00
"github.com/milvus-io/milvus/internal/tso"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
2023-06-25 17:20:43 +08:00
"github.com/milvus-io/milvus/pkg/util"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/util/contextutil"
"github.com/milvus-io/milvus/pkg/util/funcutil"
2023-09-04 09:57:09 +08:00
"github.com/milvus-io/milvus/pkg/util/merr"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/typeutil"
2021-01-19 14:44:03 +08:00
)
2023-01-19 14:13:43 +08:00
//go:generate mockery --name=IMetaTable --outpkg=mockrootcoord --filename=meta_table.go --with-expecter
2022-09-05 13:29:11 +08:00
type IMetaTable interface {
2023-06-25 17:20:43 +08:00
GetDatabaseByID ( ctx context . Context , dbID int64 , ts Timestamp ) ( * model . Database , error )
GetDatabaseByName ( ctx context . Context , dbName string , ts Timestamp ) ( * model . Database , error )
CreateDatabase ( ctx context . Context , db * model . Database , ts typeutil . Timestamp ) error
DropDatabase ( ctx context . Context , dbName string , ts typeutil . Timestamp ) error
ListDatabases ( ctx context . Context , ts typeutil . Timestamp ) ( [ ] * model . Database , error )
2024-04-25 10:53:25 +08:00
AlterDatabase ( ctx context . Context , oldDB * model . Database , newDB * model . Database , ts typeutil . Timestamp ) error
2023-06-25 17:20:43 +08:00
2022-09-05 13:29:11 +08:00
AddCollection ( ctx context . Context , coll * model . Collection ) error
ChangeCollectionState ( ctx context . Context , collectionID UniqueID , state pb . CollectionState , ts Timestamp ) error
RemoveCollection ( ctx context . Context , collectionID UniqueID , ts Timestamp ) error
2023-06-25 17:20:43 +08:00
GetCollectionByName ( ctx context . Context , dbName string , collectionName string , ts Timestamp ) ( * model . Collection , error )
GetCollectionByID ( ctx context . Context , dbName string , collectionID UniqueID , ts Timestamp , allowUnavailable bool ) ( * model . Collection , error )
2024-04-12 16:01:19 +08:00
GetCollectionByIDWithMaxTs ( ctx context . Context , collectionID UniqueID ) ( * model . Collection , error )
2023-06-25 17:20:43 +08:00
ListCollections ( ctx context . Context , dbName string , ts Timestamp , onlyAvail bool ) ( [ ] * model . Collection , error )
ListAllAvailCollections ( ctx context . Context ) map [ int64 ] [ ] int64
2022-09-05 13:29:11 +08:00
ListCollectionPhysicalChannels ( ) map [ typeutil . UniqueID ] [ ] string
2022-09-26 18:06:54 +08:00
GetCollectionVirtualChannels ( colID int64 ) [ ] string
2022-09-05 13:29:11 +08:00
AddPartition ( ctx context . Context , partition * model . Partition ) error
ChangePartitionState ( ctx context . Context , collectionID UniqueID , partitionID UniqueID , state pb . PartitionState , ts Timestamp ) error
2023-06-25 17:20:43 +08:00
RemovePartition ( ctx context . Context , dbID int64 , collectionID UniqueID , partitionID UniqueID , ts Timestamp ) error
CreateAlias ( ctx context . Context , dbName string , alias string , collectionName string , ts Timestamp ) error
DropAlias ( ctx context . Context , dbName string , alias string , ts Timestamp ) error
AlterAlias ( ctx context . Context , dbName string , alias string , collectionName string , ts Timestamp ) error
2024-01-11 19:12:51 +08:00
DescribeAlias ( ctx context . Context , dbName string , alias string , ts Timestamp ) ( string , error )
ListAliases ( ctx context . Context , dbName string , collectionName string , ts Timestamp ) ( [ ] string , error )
2022-10-10 20:31:22 +08:00
AlterCollection ( ctx context . Context , oldColl * model . Collection , newColl * model . Collection , ts Timestamp ) error
2023-08-29 14:54:26 +08:00
RenameCollection ( ctx context . Context , dbName string , oldName string , newDBName string , newName string , ts Timestamp ) error
2022-09-05 13:29:11 +08:00
// TODO: it'll be a big cost if we handle the time travel logic, since we should always list all aliases in catalog.
2023-06-25 17:20:43 +08:00
IsAlias ( db , name string ) bool
2022-09-05 13:29:11 +08:00
ListAliasesByID ( collID UniqueID ) [ ] string
// TODO: better to accept ctx.
AddCredential ( credInfo * internalpb . CredentialInfo ) error
GetCredential ( username string ) ( * internalpb . CredentialInfo , error )
DeleteCredential ( username string ) error
AlterCredential ( credInfo * internalpb . CredentialInfo ) error
ListCredentialUsernames ( ) ( * milvuspb . ListCredUsersResponse , error )
// TODO: better to accept ctx.
CreateRole ( tenant string , entity * milvuspb . RoleEntity ) error
DropRole ( tenant string , roleName string ) error
OperateUserRole ( tenant string , userEntity * milvuspb . UserEntity , roleEntity * milvuspb . RoleEntity , operateType milvuspb . OperateUserRoleType ) error
SelectRole ( tenant string , entity * milvuspb . RoleEntity , includeUserInfo bool ) ( [ ] * milvuspb . RoleResult , error )
SelectUser ( tenant string , entity * milvuspb . UserEntity , includeRoleInfo bool ) ( [ ] * milvuspb . UserResult , error )
OperatePrivilege ( tenant string , entity * milvuspb . GrantEntity , operateType milvuspb . OperatePrivilegeType ) error
SelectGrant ( tenant string , entity * milvuspb . GrantEntity ) ( [ ] * milvuspb . GrantEntity , error )
DropGrant ( tenant string , role * milvuspb . RoleEntity ) error
ListPolicy ( tenant string ) ( [ ] string , error )
ListUserRole ( tenant string ) ( [ ] string , error )
}
2024-06-13 17:54:09 +08:00
// MetaTable is a persistent meta set of all databases, collections and partitions.
2021-09-23 15:10:00 +08:00
type MetaTable struct {
2022-08-11 12:12:38 +08:00
ctx context . Context
2022-08-20 10:24:51 +08:00
catalog metastore . RootCoordCatalog
2022-07-22 10:20:29 +08:00
2023-06-25 17:20:43 +08:00
tsoAllocator tso . Allocator
dbName2Meta map [ string ] * model . Database // database name -> db meta
collID2Meta map [ typeutil . UniqueID ] * model . Collection // collection id -> collection meta
// collections *collectionDb
names * nameDb
aliases * nameDb
2021-01-19 14:44:03 +08:00
2022-08-24 10:02:52 +08:00
ddLock sync . RWMutex
permissionLock sync . RWMutex
2021-01-19 14:44:03 +08:00
}
2024-06-13 17:54:09 +08:00
// NewMetaTable creates a new MetaTable with specified catalog and allocator.
2023-06-25 17:20:43 +08:00
func NewMetaTable ( ctx context . Context , catalog metastore . RootCoordCatalog , tsoAllocator tso . Allocator ) ( * MetaTable , error ) {
2021-09-23 15:10:00 +08:00
mt := & MetaTable {
2023-06-25 17:20:43 +08:00
ctx : contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) ) ,
catalog : catalog ,
tsoAllocator : tsoAllocator ,
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
if err := mt . reload ( ) ; err != nil {
2021-01-19 14:44:03 +08:00
return nil , err
}
return mt , nil
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) reload ( ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2022-11-08 20:13:03 +08:00
record := timerecord . NewTimeRecorder ( "rootcoord" )
2023-06-25 17:20:43 +08:00
mt . dbName2Meta = make ( map [ string ] * model . Database )
2022-09-05 13:29:11 +08:00
mt . collID2Meta = make ( map [ UniqueID ] * model . Collection )
2023-06-25 17:20:43 +08:00
mt . names = newNameDb ( )
mt . aliases = newNameDb ( )
2021-01-19 14:44:03 +08:00
2022-11-09 17:49:04 +08:00
partitionNum := int64 ( 0 )
2024-04-02 14:27:13 +08:00
metrics . RootCoordNumOfCollections . Reset ( )
metrics . RootCoordNumOfPartitions . Reset ( )
2024-06-21 11:36:00 +08:00
metrics . RootCoordNumOfDatabases . Set ( 0 )
2023-07-16 18:48:33 +08:00
2023-06-25 17:20:43 +08:00
// recover databases.
dbs , err := mt . catalog . ListDatabases ( mt . ctx , typeutil . MaxTimestamp )
2022-06-15 19:14:10 +08:00
if err != nil {
return err
}
2022-11-09 17:49:04 +08:00
2023-06-25 17:20:43 +08:00
log . Info ( "recover databases" , zap . Int ( "num of dbs" , len ( dbs ) ) )
for _ , db := range dbs {
mt . dbName2Meta [ db . Name ] = db
}
dbNames := maps . Keys ( mt . dbName2Meta )
// create default database.
if ! funcutil . SliceContain ( dbNames , util . DefaultDBName ) {
if err := mt . createDefaultDb ( ) ; err != nil {
return err
}
} else {
mt . names . createDbIfNotExist ( util . DefaultDBName )
mt . aliases . createDbIfNotExist ( util . DefaultDBName )
}
// in order to support backward compatibility with meta of the old version, it also
// needs to reload collections that have no database
if err := mt . reloadWithNonDatabase ( ) ; err != nil {
return err
}
// recover collections from db namespace
for dbName , db := range mt . dbName2Meta {
mt . names . createDbIfNotExist ( dbName )
collections , err := mt . catalog . ListCollections ( mt . ctx , db . ID , typeutil . MaxTimestamp )
if err != nil {
return err
}
2024-04-02 14:27:13 +08:00
collectionNum := int64 ( 0 )
2023-06-25 17:20:43 +08:00
for _ , collection := range collections {
mt . collID2Meta [ collection . CollectionID ] = collection
if collection . Available ( ) {
mt . names . insert ( dbName , collection . Name , collection . CollectionID )
collectionNum ++
partitionNum += int64 ( collection . GetPartitionNum ( true ) )
}
}
2024-06-21 11:36:00 +08:00
metrics . RootCoordNumOfDatabases . Inc ( )
2024-04-02 14:27:13 +08:00
metrics . RootCoordNumOfCollections . WithLabelValues ( dbName ) . Add ( float64 ( collectionNum ) )
log . Info ( "collections recovered from db" , zap . String ( "db_name" , dbName ) ,
zap . Int64 ( "collection_num" , collectionNum ) ,
zap . Int64 ( "partition_num" , partitionNum ) )
}
2023-06-25 17:20:43 +08:00
// recover aliases from db namespace
for dbName , db := range mt . dbName2Meta {
mt . aliases . createDbIfNotExist ( dbName )
aliases , err := mt . catalog . ListAliases ( mt . ctx , db . ID , typeutil . MaxTimestamp )
if err != nil {
return err
}
for _ , alias := range aliases {
mt . aliases . insert ( dbName , alias . Name , alias . CollectionID )
}
}
2023-07-16 18:48:33 +08:00
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Add ( float64 ( partitionNum ) )
2023-06-25 17:20:43 +08:00
log . Info ( "RootCoord meta table reload done" , zap . Duration ( "duration" , record . ElapseSpan ( ) ) )
return nil
}
// insert into default database if the collections doesn't inside some database
func ( mt * MetaTable ) reloadWithNonDatabase ( ) error {
collectionNum := int64 ( 0 )
partitionNum := int64 ( 0 )
oldCollections , err := mt . catalog . ListCollections ( mt . ctx , util . NonDBID , typeutil . MaxTimestamp )
if err != nil {
return err
}
for _ , collection := range oldCollections {
mt . collID2Meta [ collection . CollectionID ] = collection
2022-11-09 17:49:04 +08:00
if collection . Available ( ) {
2023-06-25 17:20:43 +08:00
mt . names . insert ( util . DefaultDBName , collection . Name , collection . CollectionID )
2022-11-09 17:49:04 +08:00
collectionNum ++
partitionNum += int64 ( collection . GetPartitionNum ( true ) )
}
2022-06-15 19:14:10 +08:00
}
2023-06-25 17:20:43 +08:00
if collectionNum > 0 {
log . Info ( "recover collections without db" , zap . Int64 ( "collection_num" , collectionNum ) , zap . Int64 ( "partition_num" , partitionNum ) )
}
aliases , err := mt . catalog . ListAliases ( mt . ctx , util . NonDBID , typeutil . MaxTimestamp )
2021-01-19 14:44:03 +08:00
if err != nil {
return err
}
2022-09-05 13:29:11 +08:00
for _ , alias := range aliases {
2023-06-25 17:20:43 +08:00
mt . aliases . insert ( util . DefaultDBName , alias . Name , alias . CollectionID )
2022-06-10 13:10:08 +08:00
}
2023-07-16 18:48:33 +08:00
2024-04-02 14:27:13 +08:00
metrics . RootCoordNumOfCollections . WithLabelValues ( util . DefaultDBName ) . Add ( float64 ( collectionNum ) )
2023-07-16 18:48:33 +08:00
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Add ( float64 ( partitionNum ) )
return nil
2023-06-25 17:20:43 +08:00
}
func ( mt * MetaTable ) createDefaultDb ( ) error {
ts , err := mt . tsoAllocator . GenerateTSO ( 1 )
if err != nil {
return err
}
return mt . createDatabasePrivate ( mt . ctx , model . NewDefaultDatabase ( ) , ts )
}
func ( mt * MetaTable ) CreateDatabase ( ctx context . Context , db * model . Database , ts typeutil . Timestamp ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2024-06-21 11:36:00 +08:00
if err := mt . createDatabasePrivate ( ctx , db , ts ) ; err != nil {
return err
}
metrics . RootCoordNumOfDatabases . Inc ( )
return nil
2023-06-25 17:20:43 +08:00
}
func ( mt * MetaTable ) createDatabasePrivate ( ctx context . Context , db * model . Database , ts typeutil . Timestamp ) error {
dbName := db . Name
if mt . names . exist ( dbName ) || mt . aliases . exist ( dbName ) {
return fmt . Errorf ( "database already exist: %s" , dbName )
}
if err := mt . catalog . CreateDatabase ( ctx , db , ts ) ; err != nil {
return err
}
mt . names . createDbIfNotExist ( dbName )
mt . aliases . createDbIfNotExist ( dbName )
mt . dbName2Meta [ dbName ] = db
2024-06-21 11:36:00 +08:00
log . Ctx ( ctx ) . Info ( "create database" , zap . String ( "db" , dbName ) , zap . Uint64 ( "ts" , ts ) )
2023-06-25 17:20:43 +08:00
return nil
}
2024-03-19 19:21:06 +08:00
func ( mt * MetaTable ) AlterDatabase ( ctx context . Context , oldDB * model . Database , newDB * model . Database , ts typeutil . Timestamp ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
if oldDB . Name != newDB . Name || oldDB . ID != newDB . ID || oldDB . State != newDB . State {
return fmt . Errorf ( "alter database name/id is not supported!" )
}
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
if err := mt . catalog . AlterDatabase ( ctx1 , newDB , ts ) ; err != nil {
return err
}
mt . dbName2Meta [ oldDB . Name ] = newDB
log . Info ( "alter database finished" , zap . String ( "dbName" , oldDB . Name ) , zap . Uint64 ( "ts" , ts ) )
return nil
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) DropDatabase ( ctx context . Context , dbName string , ts typeutil . Timestamp ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
if dbName == util . DefaultDBName {
return fmt . Errorf ( "can not drop default database" )
}
db , err := mt . getDatabaseByNameInternal ( ctx , dbName , typeutil . MaxTimestamp )
if err != nil {
log . Warn ( "not found database" , zap . String ( "db" , dbName ) )
return nil
}
colls , err := mt . listCollectionFromCache ( dbName , true )
if err != nil {
return err
}
if len ( colls ) > 0 {
return fmt . Errorf ( "database:%s not empty, must drop all collections before drop database" , dbName )
}
if err := mt . catalog . DropDatabase ( ctx , db . ID , ts ) ; err != nil {
return err
}
mt . names . dropDb ( dbName )
mt . aliases . dropDb ( dbName )
delete ( mt . dbName2Meta , dbName )
2022-06-10 13:10:08 +08:00
2024-06-21 11:36:00 +08:00
metrics . RootCoordNumOfDatabases . Dec ( )
log . Ctx ( ctx ) . Info ( "drop database" , zap . String ( "db" , dbName ) , zap . Uint64 ( "ts" , ts ) )
2021-01-20 09:36:50 +08:00
return nil
2021-01-19 14:44:03 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) ListDatabases ( ctx context . Context , ts typeutil . Timestamp ) ( [ ] * model . Database , error ) {
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
return maps . Values ( mt . dbName2Meta ) , nil
}
func ( mt * MetaTable ) GetDatabaseByID ( ctx context . Context , dbID int64 , ts Timestamp ) ( * model . Database , error ) {
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
return mt . getDatabaseByIDInternal ( ctx , dbID , ts )
}
func ( mt * MetaTable ) getDatabaseByIDInternal ( ctx context . Context , dbID int64 , ts Timestamp ) ( * model . Database , error ) {
for _ , db := range maps . Values ( mt . dbName2Meta ) {
if db . ID == dbID {
return db , nil
}
}
return nil , fmt . Errorf ( "database dbID:%d not found" , dbID )
}
func ( mt * MetaTable ) GetDatabaseByName ( ctx context . Context , dbName string , ts Timestamp ) ( * model . Database , error ) {
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
return mt . getDatabaseByNameInternal ( ctx , dbName , ts )
}
2024-03-11 17:19:02 +08:00
func ( mt * MetaTable ) getDatabaseByNameInternal ( _ context . Context , dbName string , _ Timestamp ) ( * model . Database , error ) {
2023-06-25 17:20:43 +08:00
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" )
dbName = util . DefaultDBName
}
db , ok := mt . dbName2Meta [ dbName ]
if ! ok {
2024-04-12 16:01:19 +08:00
return nil , merr . WrapErrDatabaseNotFound ( dbName )
2023-06-25 17:20:43 +08:00
}
return db , nil
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) AddCollection ( ctx context . Context , coll * model . Collection ) error {
2021-01-19 14:44:03 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2021-01-20 09:36:50 +08:00
2022-09-05 13:29:11 +08:00
// Note:
// 1, idempotency check was already done outside;
// 2, no need to check time travel logic, since ts should always be the latest;
2021-01-20 09:36:50 +08:00
2023-06-25 17:20:43 +08:00
db , err := mt . getDatabaseByIDInternal ( ctx , coll . DBID , typeutil . MaxTimestamp )
if err != nil {
return err
}
2022-09-05 13:29:11 +08:00
if coll . State != pb . CollectionState_CollectionCreating {
return fmt . Errorf ( "collection state should be creating, collection name: %s, collection id: %d, state: %s" , coll . Name , coll . CollectionID , coll . State )
2022-06-10 13:10:08 +08:00
}
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2022-09-05 13:29:11 +08:00
if err := mt . catalog . CreateCollection ( ctx1 , coll , coll . CreateTime ) ; err != nil {
2022-07-22 10:20:29 +08:00
return err
2022-06-10 13:10:08 +08:00
}
2023-06-25 17:20:43 +08:00
2022-09-05 13:29:11 +08:00
mt . collID2Meta [ coll . CollectionID ] = coll . Clone ( )
2023-06-25 17:20:43 +08:00
mt . names . insert ( db . Name , coll . Name , coll . CollectionID )
log . Ctx ( ctx ) . Info ( "add collection to meta table" ,
zap . Int64 ( "dbID" , coll . DBID ) ,
zap . String ( "collection" , coll . Name ) ,
zap . Int64 ( "id" , coll . CollectionID ) ,
zap . Uint64 ( "ts" , coll . CreateTime ) ,
)
2022-06-10 13:10:08 +08:00
return nil
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) ChangeCollectionState ( ctx context . Context , collectionID UniqueID , state pb . CollectionState , ts Timestamp ) error {
2021-01-19 14:44:03 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2022-09-05 13:29:11 +08:00
coll , ok := mt . collID2Meta [ collectionID ]
2021-01-19 14:44:03 +08:00
if ! ok {
2022-09-05 13:29:11 +08:00
return nil
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
clone := coll . Clone ( )
clone . State = state
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2022-09-05 13:29:11 +08:00
if err := mt . catalog . AlterCollection ( ctx1 , coll , clone , metastore . MODIFY , ts ) ; err != nil {
return err
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
mt . collID2Meta [ collectionID ] = clone
2022-11-09 17:49:04 +08:00
2024-04-02 14:27:13 +08:00
db , err := mt . getDatabaseByIDInternal ( ctx , coll . DBID , typeutil . MaxTimestamp )
if err != nil {
return fmt . Errorf ( "dbID not found for collection:%d" , collectionID )
}
2022-11-09 17:49:04 +08:00
switch state {
case pb . CollectionState_CollectionCreated :
2024-04-02 14:27:13 +08:00
metrics . RootCoordNumOfCollections . WithLabelValues ( db . Name ) . Inc ( )
2022-11-09 17:49:04 +08:00
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Add ( float64 ( coll . GetPartitionNum ( true ) ) )
default :
2024-04-02 14:27:13 +08:00
metrics . RootCoordNumOfCollections . WithLabelValues ( db . Name ) . Dec ( )
2022-11-09 17:49:04 +08:00
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Sub ( float64 ( coll . GetPartitionNum ( true ) ) )
}
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "change collection state" , zap . Int64 ( "collection" , collectionID ) ,
2022-09-05 13:29:11 +08:00
zap . String ( "state" , state . String ( ) ) , zap . Uint64 ( "ts" , ts ) )
2021-05-17 19:15:01 +08:00
2022-09-05 13:29:11 +08:00
return nil
}
2022-06-10 13:10:08 +08:00
2022-10-24 15:57:29 +08:00
func ( mt * MetaTable ) removeIfNameMatchedInternal ( collectionID UniqueID , name string ) {
2023-06-25 17:20:43 +08:00
mt . names . removeIf ( func ( db string , collection string , id UniqueID ) bool {
return collectionID == id
} )
2022-10-24 15:57:29 +08:00
}
func ( mt * MetaTable ) removeIfAliasMatchedInternal ( collectionID UniqueID , alias string ) {
2023-06-25 17:20:43 +08:00
mt . aliases . removeIf ( func ( db string , collection string , id UniqueID ) bool {
return collectionID == id
} )
2022-10-24 15:57:29 +08:00
}
func ( mt * MetaTable ) removeIfMatchedInternal ( collectionID UniqueID , name string ) {
mt . removeIfNameMatchedInternal ( collectionID , name )
mt . removeIfAliasMatchedInternal ( collectionID , name )
}
func ( mt * MetaTable ) removeAllNamesIfMatchedInternal ( collectionID UniqueID , names [ ] string ) {
for _ , name := range names {
mt . removeIfMatchedInternal ( collectionID , name )
}
}
func ( mt * MetaTable ) removeCollectionByIDInternal ( collectionID UniqueID ) {
delete ( mt . collID2Meta , collectionID )
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) RemoveCollection ( ctx context . Context , collectionID UniqueID , ts Timestamp ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
// Note: we cannot handle case that dropping collection with `ts1` but a collection exists in catalog with newer ts
// which is bigger than `ts1`. So we assume that ts should always be the latest.
2023-06-25 17:20:43 +08:00
coll , ok := mt . collID2Meta [ collectionID ]
if ! ok {
log . Warn ( "not found collection, skip remove" , zap . Int64 ( "collectionID" , collectionID ) )
return nil
}
2022-09-05 13:29:11 +08:00
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2022-09-05 13:29:11 +08:00
aliases := mt . listAliasesByID ( collectionID )
2023-12-05 16:04:36 +08:00
newColl := & model . Collection {
CollectionID : collectionID ,
Partitions : model . ClonePartitions ( coll . Partitions ) ,
Fields : model . CloneFields ( coll . Fields ) ,
Aliases : aliases ,
DBID : coll . DBID ,
}
2023-06-25 17:20:43 +08:00
if err := mt . catalog . DropCollection ( ctx1 , newColl , ts ) ; err != nil {
2022-07-22 10:20:29 +08:00
return err
2022-06-10 13:10:08 +08:00
}
2022-09-05 13:29:11 +08:00
2022-10-24 15:57:29 +08:00
allNames := common . CloneStringList ( aliases )
2023-06-25 17:20:43 +08:00
allNames = append ( allNames , coll . Name )
2022-06-10 13:10:08 +08:00
2022-10-24 15:57:29 +08:00
// We cannot delete the name directly, since newly collection with same name may be created.
mt . removeAllNamesIfMatchedInternal ( collectionID , allNames )
mt . removeCollectionByIDInternal ( collectionID )
2022-09-22 17:36:52 +08:00
2023-06-25 17:20:43 +08:00
log . Ctx ( ctx ) . Info ( "remove collection" ,
zap . Int64 ( "dbID" , coll . DBID ) ,
zap . String ( "name" , coll . Name ) ,
zap . Int64 ( "id" , collectionID ) ,
zap . Strings ( "aliases" , aliases ) ,
)
2022-06-10 13:10:08 +08:00
return nil
2021-01-19 14:44:03 +08:00
}
2022-10-21 16:37:29 +08:00
func filterUnavailable ( coll * model . Collection ) * model . Collection {
clone := coll . Clone ( )
// pick available partitions.
clone . Partitions = nil
for _ , partition := range coll . Partitions {
if partition . Available ( ) {
clone . Partitions = append ( clone . Partitions , partition . Clone ( ) )
}
}
return clone
}
// getLatestCollectionByIDInternal should be called with ts = typeutil.MaxTimestamp
2024-04-12 16:01:19 +08:00
func ( mt * MetaTable ) getLatestCollectionByIDInternal ( ctx context . Context , collectionID UniqueID , allowUnavailable bool ) ( * model . Collection , error ) {
2022-10-21 16:37:29 +08:00
coll , ok := mt . collID2Meta [ collectionID ]
2023-01-04 16:37:35 +08:00
if ! ok || coll == nil {
2023-09-04 09:57:09 +08:00
return nil , merr . WrapErrCollectionNotFound ( collectionID )
2023-01-04 16:37:35 +08:00
}
2024-04-12 16:01:19 +08:00
if allowUnavailable {
2023-01-04 16:37:35 +08:00
return coll . Clone ( ) , nil
}
if ! coll . Available ( ) {
2023-09-04 09:57:09 +08:00
return nil , merr . WrapErrCollectionNotFound ( collectionID )
2022-10-21 16:37:29 +08:00
}
return filterUnavailable ( coll ) , nil
}
2022-09-05 13:29:11 +08:00
// getCollectionByIDInternal get collection by collection id without lock.
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) getCollectionByIDInternal ( ctx context . Context , dbName string , collectionID UniqueID , ts Timestamp , allowUnavailable bool ) ( * model . Collection , error ) {
2022-10-21 16:37:29 +08:00
if isMaxTs ( ts ) {
2023-01-04 16:37:35 +08:00
return mt . getLatestCollectionByIDInternal ( ctx , collectionID , allowUnavailable )
2022-10-21 16:37:29 +08:00
}
2022-09-05 13:29:11 +08:00
var coll * model . Collection
coll , ok := mt . collID2Meta [ collectionID ]
2022-09-27 19:18:54 +08:00
if ! ok || coll == nil || ! coll . Available ( ) || coll . CreateTime > ts {
2022-09-05 13:29:11 +08:00
// travel meta information from catalog.
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
db , err := mt . getDatabaseByNameInternal ( ctx , dbName , typeutil . MaxTimestamp )
if err != nil {
return nil , err
}
coll , err = mt . catalog . GetCollectionByID ( ctx1 , db . ID , ts , collectionID )
2022-09-05 13:29:11 +08:00
if err != nil {
return nil , err
}
2021-05-18 14:18:02 +08:00
}
2022-07-22 10:20:29 +08:00
2023-01-04 16:37:35 +08:00
if coll == nil {
// use coll.Name to match error message of regression. TODO: remove this after error code is ready.
2023-09-04 09:57:09 +08:00
return nil , merr . WrapErrCollectionNotFound ( collectionID )
2023-01-04 16:37:35 +08:00
}
if allowUnavailable {
return coll . Clone ( ) , nil
}
if ! coll . Available ( ) {
2022-09-05 13:29:11 +08:00
// use coll.Name to match error message of regression. TODO: remove this after error code is ready.
2023-09-04 09:57:09 +08:00
return nil , merr . WrapErrCollectionNotFound ( dbName , coll . Name )
2022-05-09 20:47:52 +08:00
}
2022-10-21 16:37:29 +08:00
return filterUnavailable ( coll ) , nil
2021-01-21 10:01:29 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) GetCollectionByName ( ctx context . Context , dbName string , collectionName string , ts Timestamp ) ( * model . Collection , error ) {
2021-01-19 14:44:03 +08:00
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
2023-06-25 17:20:43 +08:00
return mt . getCollectionByNameInternal ( ctx , dbName , collectionName , ts )
2023-01-24 10:01:46 +08:00
}
2021-01-19 14:44:03 +08:00
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) getCollectionByNameInternal ( ctx context . Context , dbName string , collectionName string , ts Timestamp ) ( * model . Collection , error ) {
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" , zap . String ( "collectionName" , collectionName ) , zap . Uint64 ( "ts" , ts ) )
dbName = util . DefaultDBName
}
collectionID , ok := mt . aliases . get ( dbName , collectionName )
2022-09-05 13:29:11 +08:00
if ok {
2023-06-25 17:20:43 +08:00
return mt . getCollectionByIDInternal ( ctx , dbName , collectionID , ts , false )
2022-09-05 13:29:11 +08:00
}
2022-07-22 10:20:29 +08:00
2023-06-25 17:20:43 +08:00
collectionID , ok = mt . names . get ( dbName , collectionName )
2022-09-05 13:29:11 +08:00
if ok {
2023-06-25 17:20:43 +08:00
return mt . getCollectionByIDInternal ( ctx , dbName , collectionID , ts , false )
2022-06-10 13:10:08 +08:00
}
2022-07-22 10:20:29 +08:00
2022-10-21 16:37:29 +08:00
if isMaxTs ( ts ) {
2023-09-04 09:57:09 +08:00
return nil , merr . WrapErrCollectionNotFoundWithDB ( dbName , collectionName )
2022-10-21 16:37:29 +08:00
}
2023-06-25 17:20:43 +08:00
db , err := mt . getDatabaseByNameInternal ( ctx , dbName , typeutil . MaxTimestamp )
if err != nil {
return nil , err
}
2022-09-05 13:29:11 +08:00
// travel meta information from catalog. No need to check time travel logic again, since catalog already did.
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
coll , err := mt . catalog . GetCollectionByName ( ctx1 , db . ID , collectionName , ts )
2022-09-05 13:29:11 +08:00
if err != nil {
return nil , err
}
2023-01-19 14:13:43 +08:00
2022-10-21 16:37:29 +08:00
if coll == nil || ! coll . Available ( ) {
2023-09-04 09:57:09 +08:00
return nil , merr . WrapErrCollectionNotFoundWithDB ( dbName , collectionName )
2022-09-27 19:18:54 +08:00
}
2022-10-21 16:37:29 +08:00
return filterUnavailable ( coll ) , nil
2021-02-02 10:09:10 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) GetCollectionByID ( ctx context . Context , dbName string , collectionID UniqueID , ts Timestamp , allowUnavailable bool ) ( * model . Collection , error ) {
2021-01-19 14:44:03 +08:00
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
2022-07-22 10:20:29 +08:00
2023-06-25 17:20:43 +08:00
return mt . getCollectionByIDInternal ( ctx , dbName , collectionID , ts , allowUnavailable )
2021-01-19 14:44:03 +08:00
}
2024-04-12 16:01:19 +08:00
// GetCollectionByIDWithMaxTs get collection, dbName can be ignored if ts is max timestamps
func ( mt * MetaTable ) GetCollectionByIDWithMaxTs ( ctx context . Context , collectionID UniqueID ) ( * model . Collection , error ) {
return mt . GetCollectionByID ( ctx , "" , collectionID , typeutil . MaxTimestamp , false )
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) ListAllAvailCollections ( ctx context . Context ) map [ int64 ] [ ] int64 {
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
ret := make ( map [ int64 ] [ ] int64 , len ( mt . dbName2Meta ) )
for _ , dbMeta := range mt . dbName2Meta {
ret [ dbMeta . ID ] = make ( [ ] int64 , 0 )
}
for collID , collMeta := range mt . collID2Meta {
if ! collMeta . Available ( ) {
continue
}
dbID := collMeta . DBID
if dbID == util . NonDBID {
ret [ util . DefaultDBID ] = append ( ret [ util . DefaultDBID ] , collID )
continue
}
ret [ dbID ] = append ( ret [ dbID ] , collID )
}
return ret
}
func ( mt * MetaTable ) ListCollections ( ctx context . Context , dbName string , ts Timestamp , onlyAvail bool ) ( [ ] * model . Collection , error ) {
2021-09-22 16:20:48 +08:00
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
2022-09-05 13:29:11 +08:00
2022-10-31 13:25:35 +08:00
if isMaxTs ( ts ) {
2023-06-25 17:20:43 +08:00
return mt . listCollectionFromCache ( dbName , onlyAvail )
}
db , err := mt . getDatabaseByNameInternal ( ctx , dbName , typeutil . MaxTimestamp )
if err != nil {
return nil , err
2022-10-31 13:25:35 +08:00
}
2022-09-05 13:29:11 +08:00
// list collections should always be loaded from catalog.
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
colls , err := mt . catalog . ListCollections ( ctx1 , db . ID , ts )
2022-09-05 13:29:11 +08:00
if err != nil {
return nil , err
}
onlineCollections := make ( [ ] * model . Collection , 0 , len ( colls ) )
for _ , coll := range colls {
2023-06-25 17:20:43 +08:00
if onlyAvail && ! coll . Available ( ) {
continue
2021-09-22 16:20:48 +08:00
}
2023-06-25 17:20:43 +08:00
onlineCollections = append ( onlineCollections , coll )
2021-09-22 16:20:48 +08:00
}
2022-09-05 13:29:11 +08:00
return onlineCollections , nil
2021-09-22 16:20:48 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) listCollectionFromCache ( dbName string , onlyAvail bool ) ( [ ] * model . Collection , error ) {
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" )
dbName = util . DefaultDBName
2022-10-31 13:25:35 +08:00
}
2021-06-04 15:00:34 +08:00
2023-06-25 17:20:43 +08:00
db , ok := mt . dbName2Meta [ dbName ]
if ! ok {
2023-10-19 17:24:07 +08:00
return nil , merr . WrapErrDatabaseNotFound ( dbName )
2021-06-04 15:00:34 +08:00
}
2023-06-25 17:20:43 +08:00
collectionFromCache := make ( [ ] * model . Collection , 0 , len ( mt . collID2Meta ) )
for _ , collMeta := range mt . collID2Meta {
if ( collMeta . DBID != util . NonDBID && db . ID == collMeta . DBID ) ||
( collMeta . DBID == util . NonDBID && dbName == util . DefaultDBName ) {
if onlyAvail && ! collMeta . Available ( ) {
continue
}
collectionFromCache = append ( collectionFromCache , collMeta )
2022-09-05 13:29:11 +08:00
}
}
2023-06-25 17:20:43 +08:00
return collectionFromCache , nil
2021-06-04 15:00:34 +08:00
}
2022-09-05 13:29:11 +08:00
// ListCollectionPhysicalChannels list physical channels of all collections.
2021-11-02 15:50:30 +08:00
func ( mt * MetaTable ) ListCollectionPhysicalChannels ( ) map [ typeutil . UniqueID ] [ ] string {
2021-06-04 15:00:34 +08:00
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
2022-09-05 13:29:11 +08:00
chanMap := make ( map [ UniqueID ] [ ] string )
2021-06-04 15:00:34 +08:00
2021-11-02 15:50:30 +08:00
for id , collInfo := range mt . collID2Meta {
2022-09-05 13:29:11 +08:00
chanMap [ id ] = common . CloneStringList ( collInfo . PhysicalChannelNames )
2021-06-04 15:00:34 +08:00
}
2022-09-05 13:29:11 +08:00
2021-11-02 15:50:30 +08:00
return chanMap
2021-06-04 15:00:34 +08:00
}
2022-10-10 20:31:22 +08:00
func ( mt * MetaTable ) AlterCollection ( ctx context . Context , oldColl * model . Collection , newColl * model . Collection , ts Timestamp ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2022-10-10 20:31:22 +08:00
if err := mt . catalog . AlterCollection ( ctx1 , oldColl , newColl , metastore . MODIFY , ts ) ; err != nil {
return err
}
mt . collID2Meta [ oldColl . CollectionID ] = newColl
log . Info ( "alter collection finished" , zap . Int64 ( "collectionID" , oldColl . CollectionID ) , zap . Uint64 ( "ts" , ts ) )
return nil
}
2023-08-29 14:54:26 +08:00
func ( mt * MetaTable ) RenameCollection ( ctx context . Context , dbName string , oldName string , newDBName string , newName string , ts Timestamp ) error {
2023-01-24 10:01:46 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2023-01-19 14:13:43 +08:00
2023-01-24 10:01:46 +08:00
ctx = contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
log := log . Ctx ( ctx ) . With (
2023-08-29 14:54:26 +08:00
zap . String ( "oldDBName" , dbName ) ,
2023-06-25 17:20:43 +08:00
zap . String ( "oldName" , oldName ) ,
2023-08-29 14:54:26 +08:00
zap . String ( "newDBName" , newDBName ) ,
2023-06-25 17:20:43 +08:00
zap . String ( "newName" , newName ) ,
)
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" )
dbName = util . DefaultDBName
}
2023-01-19 14:13:43 +08:00
2023-08-29 14:54:26 +08:00
if newDBName == "" {
log . Warn ( "target db name is empty" )
newDBName = dbName
}
// check target db
targetDB , ok := mt . dbName2Meta [ newDBName ]
if ! ok {
return fmt . Errorf ( "target database:%s not found" , newDBName )
}
2023-09-21 09:45:27 +08:00
// old collection should not be an alias
2023-08-29 14:54:26 +08:00
_ , ok = mt . aliases . get ( dbName , oldName )
2023-01-19 14:13:43 +08:00
if ok {
log . Warn ( "unsupported use a alias to rename collection" )
return fmt . Errorf ( "unsupported use an alias to rename collection, alias:%s" , oldName )
}
// check new collection already exists
2023-08-29 14:54:26 +08:00
newColl , err := mt . getCollectionByNameInternal ( ctx , newDBName , newName , ts )
2023-01-19 14:13:43 +08:00
if newColl != nil {
2023-08-29 14:54:26 +08:00
log . Warn ( "check new collection fail" )
return fmt . Errorf ( "duplicated new collection name %s:%s with other collection name or alias" , newDBName , newName )
2023-01-19 14:13:43 +08:00
}
2023-09-04 09:57:09 +08:00
if err != nil && ! errors . Is ( err , merr . ErrCollectionNotFound ) {
2023-01-19 14:13:43 +08:00
log . Warn ( "check new collection name fail" )
return err
}
// get old collection meta
2023-06-25 17:20:43 +08:00
oldColl , err := mt . getCollectionByNameInternal ( ctx , dbName , oldName , ts )
2023-01-19 14:13:43 +08:00
if err != nil {
2023-08-29 14:54:26 +08:00
log . Warn ( "get old collection fail" )
2023-01-19 14:13:43 +08:00
return err
}
2023-08-29 14:54:26 +08:00
// unsupported rename collection while the collection has aliases
aliases := mt . listAliasesByID ( oldColl . CollectionID )
if len ( aliases ) > 0 && oldColl . DBID != targetDB . ID {
return fmt . Errorf ( "fail to rename db name, must drop all aliases of this collection before rename" )
}
2023-01-19 14:13:43 +08:00
newColl = oldColl . Clone ( )
newColl . Name = newName
2023-08-29 14:54:26 +08:00
newColl . DBID = targetDB . ID
2023-01-19 14:13:43 +08:00
if err := mt . catalog . AlterCollection ( ctx , oldColl , newColl , metastore . MODIFY , ts ) ; err != nil {
return err
}
2023-08-29 14:54:26 +08:00
mt . names . insert ( newDBName , newName , oldColl . CollectionID )
2023-06-25 17:20:43 +08:00
mt . names . remove ( dbName , oldName )
2023-01-19 14:13:43 +08:00
mt . collID2Meta [ oldColl . CollectionID ] = newColl
log . Info ( "rename collection finished" )
return nil
}
2022-09-26 18:06:54 +08:00
// GetCollectionVirtualChannels returns virtual channels of a given collection.
func ( mt * MetaTable ) GetCollectionVirtualChannels ( colID int64 ) [ ] string {
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
for id , collInfo := range mt . collID2Meta {
if id == colID {
return common . CloneStringList ( collInfo . VirtualChannelNames )
}
}
return nil
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) AddPartition ( ctx context . Context , partition * model . Partition ) error {
2021-01-19 14:44:03 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2021-07-03 14:36:18 +08:00
2022-09-05 13:29:11 +08:00
coll , ok := mt . collID2Meta [ partition . CollectionID ]
if ! ok || ! coll . Available ( ) {
return fmt . Errorf ( "collection not exists: %d" , partition . CollectionID )
2021-01-19 14:44:03 +08:00
}
2023-03-20 14:55:57 +08:00
if partition . State != pb . PartitionState_PartitionCreating {
2022-09-05 13:29:11 +08:00
return fmt . Errorf ( "partition state is not created, collection: %d, partition: %d, state: %s" , partition . CollectionID , partition . PartitionID , partition . State )
2022-08-10 10:22:38 +08:00
}
2023-06-25 17:20:43 +08:00
if err := mt . catalog . CreatePartition ( ctx , coll . DBID , partition , partition . PartitionCreatedTimestamp ) ; err != nil {
2022-07-22 10:20:29 +08:00
return err
2022-06-10 13:10:08 +08:00
}
2022-09-05 13:29:11 +08:00
mt . collID2Meta [ partition . CollectionID ] . Partitions = append ( mt . collID2Meta [ partition . CollectionID ] . Partitions , partition . Clone ( ) )
2022-11-09 17:49:04 +08:00
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Inc ( )
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "add partition to meta table" ,
2022-09-05 13:29:11 +08:00
zap . Int64 ( "collection" , partition . CollectionID ) , zap . String ( "partition" , partition . PartitionName ) ,
zap . Int64 ( "partitionid" , partition . PartitionID ) , zap . Uint64 ( "ts" , partition . PartitionCreatedTimestamp ) )
2022-11-09 17:49:04 +08:00
2022-06-10 13:10:08 +08:00
return nil
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) ChangePartitionState ( ctx context . Context , collectionID UniqueID , partitionID UniqueID , state pb . PartitionState , ts Timestamp ) error {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
coll , ok := mt . collID2Meta [ collectionID ]
if ! ok {
return nil
}
for idx , part := range coll . Partitions {
if part . PartitionID == partitionID {
clone := part . Clone ( )
clone . State = state
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
if err := mt . catalog . AlterPartition ( ctx1 , coll . DBID , part , clone , metastore . MODIFY , ts ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
2021-05-18 14:18:02 +08:00
}
2022-09-05 13:29:11 +08:00
mt . collID2Meta [ collectionID ] . Partitions [ idx ] = clone
2022-11-09 17:49:04 +08:00
switch state {
case pb . PartitionState_PartitionCreated :
2023-04-28 15:20:36 +08:00
// support Dynamic load/release partitions
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Inc ( )
2022-11-09 17:49:04 +08:00
default :
metrics . RootCoordNumOfPartitions . WithLabelValues ( ) . Dec ( )
}
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "change partition state" , zap . Int64 ( "collection" , collectionID ) ,
2022-09-05 13:29:11 +08:00
zap . Int64 ( "partition" , partitionID ) , zap . String ( "state" , state . String ( ) ) ,
zap . Uint64 ( "ts" , ts ) )
2022-11-09 17:49:04 +08:00
2022-09-05 13:29:11 +08:00
return nil
2021-05-18 14:18:02 +08:00
}
}
2022-09-05 13:29:11 +08:00
return fmt . Errorf ( "partition not exist, collection: %d, partition: %d" , collectionID , partitionID )
}
2022-07-22 10:20:29 +08:00
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) RemovePartition ( ctx context . Context , dbID int64 , collectionID UniqueID , partitionID UniqueID , ts Timestamp ) error {
2022-09-05 13:29:11 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
if err := mt . catalog . DropPartition ( ctx1 , dbID , collectionID , partitionID , ts ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
2022-06-10 13:10:08 +08:00
}
2022-09-05 13:29:11 +08:00
coll , ok := mt . collID2Meta [ collectionID ]
if ! ok {
return nil
}
2023-09-21 09:45:27 +08:00
loc := - 1
2022-09-05 13:29:11 +08:00
for idx , part := range coll . Partitions {
if part . PartitionID == partitionID {
loc = idx
break
2021-05-18 14:18:02 +08:00
}
2021-07-03 14:36:18 +08:00
}
2022-09-05 13:29:11 +08:00
if loc != - 1 {
coll . Partitions = append ( coll . Partitions [ : loc ] , coll . Partitions [ loc + 1 : ] ... )
}
log . Info ( "remove partition" , zap . Int64 ( "collection" , collectionID ) , zap . Int64 ( "partition" , partitionID ) , zap . Uint64 ( "ts" , ts ) )
return nil
2021-07-03 14:36:18 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) CreateAlias ( ctx context . Context , dbName string , alias string , collectionName string , ts Timestamp ) error {
2022-09-05 13:29:11 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2023-06-25 17:20:43 +08:00
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" , zap . String ( "alias" , alias ) , zap . String ( "collection" , collectionName ) )
dbName = util . DefaultDBName
}
2022-09-05 13:29:11 +08:00
// It's ok that we don't read from catalog when cache missed.
// Since cache always keep the latest version, and the ts should always be the latest.
2023-06-25 17:20:43 +08:00
if ! mt . names . exist ( dbName ) {
2023-10-19 17:24:07 +08:00
return merr . WrapErrDatabaseNotFound ( dbName )
2023-06-25 17:20:43 +08:00
}
2023-07-14 19:46:31 +08:00
if collID , ok := mt . names . get ( dbName , alias ) ; ok {
coll , ok := mt . collID2Meta [ collID ]
if ! ok {
return fmt . Errorf ( "meta error, name mapped non-exist collection id" )
}
// allow alias with dropping&dropped
if coll . State != pb . CollectionState_CollectionDropping && coll . State != pb . CollectionState_CollectionDropped {
2023-10-19 17:24:07 +08:00
return merr . WrapErrAliasCollectionNameConflict ( dbName , alias )
2023-07-14 19:46:31 +08:00
}
2021-07-03 14:36:18 +08:00
}
2022-07-22 10:20:29 +08:00
2023-06-25 17:20:43 +08:00
collectionID , ok := mt . names . get ( dbName , collectionName )
2022-09-05 13:29:11 +08:00
if ! ok {
// you cannot alias to a non-existent collection.
2023-10-19 17:24:07 +08:00
return merr . WrapErrCollectionNotFoundWithDB ( dbName , collectionName )
2022-06-10 13:10:08 +08:00
}
2022-09-05 13:29:11 +08:00
// check if alias exists.
2023-06-25 17:20:43 +08:00
aliasedCollectionID , ok := mt . aliases . get ( dbName , alias )
2022-09-05 13:29:11 +08:00
if ok && aliasedCollectionID == collectionID {
log . Warn ( "add duplicate alias" , zap . String ( "alias" , alias ) , zap . String ( "collection" , collectionName ) , zap . Uint64 ( "ts" , ts ) )
return nil
} else if ok {
// TODO: better to check if aliasedCollectionID exist or is available, though not very possible.
aliasedColl := mt . collID2Meta [ aliasedCollectionID ]
2023-10-19 17:24:07 +08:00
msg := fmt . Sprintf ( "%s is alias to another collection: %s" , alias , aliasedColl . Name )
return merr . WrapErrAliasAlreadyExist ( dbName , alias , msg )
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
// alias didn't exist.
2021-05-14 21:26:06 +08:00
2022-09-05 13:29:11 +08:00
coll , ok := mt . collID2Meta [ collectionID ]
if ! ok || ! coll . Available ( ) {
// you cannot alias to a non-existent collection.
2023-10-19 17:24:07 +08:00
return merr . WrapErrCollectionNotFoundWithDB ( dbName , collectionName )
2022-09-05 13:29:11 +08:00
}
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2022-09-05 13:29:11 +08:00
if err := mt . catalog . CreateAlias ( ctx1 , & model . Alias {
Name : alias ,
CollectionID : collectionID ,
CreatedTime : ts ,
State : pb . AliasState_AliasCreated ,
2023-06-25 17:20:43 +08:00
DbID : coll . DBID ,
2022-09-05 13:29:11 +08:00
} , ts ) ; err != nil {
return err
}
2023-06-25 17:20:43 +08:00
mt . aliases . insert ( dbName , alias , collectionID )
log . Ctx ( ctx ) . Info ( "create alias" ,
zap . String ( "db" , dbName ) ,
zap . String ( "alias" , alias ) ,
zap . String ( "collection" , collectionName ) ,
zap . Int64 ( "id" , coll . CollectionID ) ,
zap . Uint64 ( "ts" , ts ) ,
)
2022-09-05 13:29:11 +08:00
return nil
2021-01-19 14:44:03 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) DropAlias ( ctx context . Context , dbName string , alias string , ts Timestamp ) error {
2022-09-05 13:29:11 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2023-06-25 17:20:43 +08:00
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" , zap . String ( "alias" , alias ) , zap . Uint64 ( "ts" , ts ) )
dbName = util . DefaultDBName
}
2022-09-05 13:29:11 +08:00
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2023-06-25 17:20:43 +08:00
db , err := mt . getDatabaseByNameInternal ( ctx , dbName , typeutil . MaxTimestamp )
if err != nil {
return err
}
if err := mt . catalog . DropAlias ( ctx1 , db . ID , alias , ts ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
}
2023-06-25 17:20:43 +08:00
mt . aliases . remove ( dbName , alias )
log . Ctx ( ctx ) . Info ( "drop alias" ,
zap . String ( "db" , dbName ) ,
zap . String ( "alias" , alias ) ,
zap . Uint64 ( "ts" , ts ) ,
)
2022-09-05 13:29:11 +08:00
return nil
2021-05-14 21:26:06 +08:00
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) AlterAlias ( ctx context . Context , dbName string , alias string , collectionName string , ts Timestamp ) error {
2021-01-19 14:44:03 +08:00
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
2023-06-25 17:20:43 +08:00
// backward compatibility for rolling upgrade
if dbName == "" {
log . Warn ( "db name is empty" , zap . String ( "alias" , alias ) , zap . String ( "collection" , collectionName ) )
dbName = util . DefaultDBName
}
2021-01-19 14:44:03 +08:00
2022-09-05 13:29:11 +08:00
// It's ok that we don't read from catalog when cache missed.
// Since cache always keep the latest version, and the ts should always be the latest.
2023-06-25 17:20:43 +08:00
if ! mt . names . exist ( dbName ) {
2023-10-19 17:24:07 +08:00
return merr . WrapErrDatabaseNotFound ( dbName )
2023-06-25 17:20:43 +08:00
}
2023-07-14 19:46:31 +08:00
if collID , ok := mt . names . get ( dbName , alias ) ; ok {
coll := mt . collID2Meta [ collID ]
// allow alias with dropping&dropped
if coll . State != pb . CollectionState_CollectionDropping && coll . State != pb . CollectionState_CollectionDropped {
2023-10-19 17:24:07 +08:00
return merr . WrapErrAliasCollectionNameConflict ( dbName , alias )
2023-07-14 19:46:31 +08:00
}
2021-01-19 14:44:03 +08:00
}
2023-06-25 17:20:43 +08:00
collectionID , ok := mt . names . get ( dbName , collectionName )
2021-01-19 14:44:03 +08:00
if ! ok {
2022-09-05 13:29:11 +08:00
// you cannot alias to a non-existent collection.
2023-10-19 17:24:07 +08:00
return merr . WrapErrCollectionNotFound ( collectionName )
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
coll , ok := mt . collID2Meta [ collectionID ]
if ! ok || ! coll . Available ( ) {
// you cannot alias to a non-existent collection.
2023-10-19 17:24:07 +08:00
return merr . WrapErrCollectionNotFound ( collectionName )
2022-09-05 13:29:11 +08:00
}
2021-01-19 14:44:03 +08:00
2022-09-05 13:29:11 +08:00
// check if alias exists.
2023-06-25 17:20:43 +08:00
_ , ok = mt . aliases . get ( dbName , alias )
2022-09-05 13:29:11 +08:00
if ! ok {
//
2023-10-19 17:24:07 +08:00
return merr . WrapErrAliasNotFound ( dbName , alias )
2022-09-05 13:29:11 +08:00
}
2022-07-22 10:20:29 +08:00
2022-12-07 18:01:19 +08:00
ctx1 := contextutil . WithTenantID ( ctx , Params . CommonCfg . ClusterName . GetValue ( ) )
2022-09-05 13:29:11 +08:00
if err := mt . catalog . AlterAlias ( ctx1 , & model . Alias {
Name : alias ,
CollectionID : collectionID ,
CreatedTime : ts ,
State : pb . AliasState_AliasCreated ,
2023-06-25 17:20:43 +08:00
DbID : coll . DBID ,
2022-09-05 13:29:11 +08:00
} , ts ) ; err != nil {
return err
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
// alias switch to another collection anyway.
2023-06-25 17:20:43 +08:00
mt . aliases . insert ( dbName , alias , collectionID )
log . Ctx ( ctx ) . Info ( "alter alias" ,
zap . String ( "db" , dbName ) ,
zap . String ( "alias" , alias ) ,
zap . String ( "collection" , collectionName ) ,
zap . Uint64 ( "ts" , ts ) ,
)
2022-09-05 13:29:11 +08:00
return nil
2022-07-22 10:20:29 +08:00
}
2022-06-10 13:10:08 +08:00
2024-01-11 19:12:51 +08:00
func ( mt * MetaTable ) DescribeAlias ( ctx context . Context , dbName string , alias string , ts Timestamp ) ( string , error ) {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
if dbName == "" {
log . Warn ( "db name is empty" , zap . String ( "alias" , alias ) )
dbName = util . DefaultDBName
}
// check if database exists.
dbExist := mt . aliases . exist ( dbName )
if ! dbExist {
return "" , merr . WrapErrDatabaseNotFound ( dbName )
}
// check if alias exists.
collectionID , ok := mt . aliases . get ( dbName , alias )
if ! ok {
return "" , merr . WrapErrAliasNotFound ( dbName , alias )
}
collectionMeta , ok := mt . collID2Meta [ collectionID ]
if ! ok {
return "" , merr . WrapErrCollectionIDOfAliasNotFound ( collectionID )
}
if collectionMeta . State == pb . CollectionState_CollectionCreated {
return collectionMeta . Name , nil
}
return "" , merr . WrapErrAliasNotFound ( dbName , alias )
}
func ( mt * MetaTable ) ListAliases ( ctx context . Context , dbName string , collectionName string , ts Timestamp ) ( [ ] string , error ) {
mt . ddLock . Lock ( )
defer mt . ddLock . Unlock ( )
if dbName == "" {
log . Warn ( "db name is empty" , zap . String ( "collection" , collectionName ) )
dbName = util . DefaultDBName
}
// check if database exists.
dbExist := mt . aliases . exist ( dbName )
if ! dbExist {
return nil , merr . WrapErrDatabaseNotFound ( dbName )
}
var aliases [ ] string
if collectionName == "" {
collections := mt . aliases . listCollections ( dbName )
for name , collectionID := range collections {
if collectionMeta , ok := mt . collID2Meta [ collectionID ] ; ok &&
collectionMeta . State == pb . CollectionState_CollectionCreated {
aliases = append ( aliases , name )
}
}
} else {
collectionID , exist := mt . names . get ( dbName , collectionName )
collectionMeta , exist2 := mt . collID2Meta [ collectionID ]
if exist && exist2 && collectionMeta . State == pb . CollectionState_CollectionCreated {
aliases = mt . listAliasesByID ( collectionID )
} else {
return nil , merr . WrapErrCollectionNotFound ( collectionName )
}
}
return aliases , nil
}
2023-06-25 17:20:43 +08:00
func ( mt * MetaTable ) IsAlias ( db , name string ) bool {
2021-01-21 10:01:29 +08:00
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
2023-06-25 17:20:43 +08:00
_ , ok := mt . aliases . get ( db , name )
2022-09-05 13:29:11 +08:00
return ok
2021-02-02 10:09:10 +08:00
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) listAliasesByID ( collID UniqueID ) [ ] string {
ret := make ( [ ] string , 0 )
2023-06-25 17:20:43 +08:00
mt . aliases . iterate ( func ( db string , collection string , id UniqueID ) bool {
if collID == id {
ret = append ( ret , collection )
2021-09-28 21:52:20 +08:00
}
2023-06-25 17:20:43 +08:00
return true
} )
2022-09-05 13:29:11 +08:00
return ret
2021-01-21 10:01:29 +08:00
}
2022-09-05 13:29:11 +08:00
func ( mt * MetaTable ) ListAliasesByID ( collID UniqueID ) [ ] string {
mt . ddLock . RLock ( )
defer mt . ddLock . RUnlock ( )
2021-01-21 10:01:29 +08:00
2022-09-05 13:29:11 +08:00
return mt . listAliasesByID ( collID )
2022-05-10 21:07:53 +08:00
}
2022-04-11 19:49:34 +08:00
// AddCredential add credential
func ( mt * MetaTable ) AddCredential ( credInfo * internalpb . CredentialInfo ) error {
if credInfo . Username == "" {
return fmt . Errorf ( "username is empty" )
}
2022-08-24 10:02:52 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
usernames , err := mt . catalog . ListCredentials ( mt . ctx )
if err != nil {
return err
}
2022-12-07 18:01:19 +08:00
if len ( usernames ) >= Params . ProxyCfg . MaxUserNum . GetAsInt ( ) {
2022-08-26 19:22:56 +08:00
errMsg := "unable to add user because the number of users has reached the limit"
2022-12-07 18:01:19 +08:00
log . Error ( errMsg , zap . Int ( "max_user_num" , Params . ProxyCfg . MaxUserNum . GetAsInt ( ) ) )
2022-08-26 19:22:56 +08:00
return errors . New ( errMsg )
2022-08-24 10:02:52 +08:00
}
if origin , _ := mt . catalog . GetCredential ( mt . ctx , credInfo . Username ) ; origin != nil {
return fmt . Errorf ( "user already exists: %s" , credInfo . Username )
}
credential := & model . Credential {
Username : credInfo . Username ,
EncryptedPassword : credInfo . EncryptedPassword ,
}
return mt . catalog . CreateCredential ( mt . ctx , credential )
}
2022-08-24 14:32:56 +08:00
// AlterCredential update credential
func ( mt * MetaTable ) AlterCredential ( credInfo * internalpb . CredentialInfo ) error {
if credInfo . Username == "" {
return fmt . Errorf ( "username is empty" )
}
2022-07-22 10:20:29 +08:00
2022-08-26 19:22:56 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
2022-07-22 10:20:29 +08:00
credential := & model . Credential {
Username : credInfo . Username ,
EncryptedPassword : credInfo . EncryptedPassword ,
2022-06-10 13:10:08 +08:00
}
2022-08-24 14:32:56 +08:00
return mt . catalog . AlterCredential ( mt . ctx , credential )
2022-04-11 19:49:34 +08:00
}
// GetCredential get credential by username
2022-08-24 14:32:56 +08:00
func ( mt * MetaTable ) GetCredential ( username string ) ( * internalpb . CredentialInfo , error ) {
2022-08-26 19:22:56 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-07-22 10:20:29 +08:00
credential , err := mt . catalog . GetCredential ( mt . ctx , username )
return model . MarshalCredentialModel ( credential ) , err
2022-04-11 19:49:34 +08:00
}
// DeleteCredential delete credential
func ( mt * MetaTable ) DeleteCredential ( username string ) error {
2022-08-24 10:02:52 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
2022-07-22 10:20:29 +08:00
return mt . catalog . DropCredential ( mt . ctx , username )
}
2022-06-10 13:10:08 +08:00
2022-07-22 10:20:29 +08:00
// ListCredentialUsernames list credential usernames
func ( mt * MetaTable ) ListCredentialUsernames ( ) ( * milvuspb . ListCredUsersResponse , error ) {
2022-08-24 10:02:52 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-07-22 10:20:29 +08:00
usernames , err := mt . catalog . ListCredentials ( mt . ctx )
2022-06-10 13:10:08 +08:00
if err != nil {
2022-07-22 10:20:29 +08:00
return nil , fmt . Errorf ( "list credential usernames err:%w" , err )
2022-06-10 13:10:08 +08:00
}
2022-07-22 10:20:29 +08:00
return & milvuspb . ListCredUsersResponse { Usernames : usernames } , nil
2022-04-11 19:49:34 +08:00
}
2022-08-04 11:04:34 +08:00
// CreateRole create role
func ( mt * MetaTable ) CreateRole ( tenant string , entity * milvuspb . RoleEntity ) error {
if funcutil . IsEmptyString ( entity . Name ) {
return fmt . Errorf ( "the role name in the role info is empty" )
}
2022-08-24 10:02:52 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
results , err := mt . catalog . ListRole ( mt . ctx , tenant , nil , false )
if err != nil {
2023-09-27 11:03:26 +08:00
log . Warn ( "fail to list roles" , zap . Error ( err ) )
2022-08-24 10:02:52 +08:00
return err
}
2023-09-27 11:03:26 +08:00
for _ , result := range results {
if result . GetRole ( ) . GetName ( ) == entity . Name {
log . Info ( "role already exists" , zap . String ( "role" , entity . Name ) )
return common . NewIgnorableError ( errors . Newf ( "role [%s] already exists" , entity ) )
}
}
2022-12-07 18:01:19 +08:00
if len ( results ) >= Params . ProxyCfg . MaxRoleNum . GetAsInt ( ) {
2023-01-12 13:55:42 +08:00
errMsg := "unable to create role because the number of roles has reached the limit"
2023-09-27 11:03:26 +08:00
log . Warn ( errMsg , zap . Int ( "max_role_num" , Params . ProxyCfg . MaxRoleNum . GetAsInt ( ) ) )
2022-08-26 19:22:56 +08:00
return errors . New ( errMsg )
2022-08-24 10:02:52 +08:00
}
2022-08-04 11:04:34 +08:00
return mt . catalog . CreateRole ( mt . ctx , tenant , entity )
}
// DropRole drop role info
func ( mt * MetaTable ) DropRole ( tenant string , roleName string ) error {
2022-08-24 10:02:52 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
2022-08-04 11:04:34 +08:00
return mt . catalog . DropRole ( mt . ctx , tenant , roleName )
}
// OperateUserRole operate the relationship between a user and a role, including adding a user to a role and removing a user from a role
func ( mt * MetaTable ) OperateUserRole ( tenant string , userEntity * milvuspb . UserEntity , roleEntity * milvuspb . RoleEntity , operateType milvuspb . OperateUserRoleType ) error {
if funcutil . IsEmptyString ( userEntity . Name ) {
return fmt . Errorf ( "username in the user entity is empty" )
}
if funcutil . IsEmptyString ( roleEntity . Name ) {
return fmt . Errorf ( "role name in the role entity is empty" )
}
2022-08-24 10:02:52 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
2022-08-23 10:26:53 +08:00
return mt . catalog . AlterUserRole ( mt . ctx , tenant , userEntity , roleEntity , operateType )
2022-08-04 11:04:34 +08:00
}
// SelectRole select role.
// Enter the role condition by the entity param. And this param is nil, which means selecting all roles.
// Get all users that are added to the role by setting the includeUserInfo param to true.
func ( mt * MetaTable ) SelectRole ( tenant string , entity * milvuspb . RoleEntity , includeUserInfo bool ) ( [ ] * milvuspb . RoleResult , error ) {
2022-08-24 10:02:52 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-08-23 10:26:53 +08:00
return mt . catalog . ListRole ( mt . ctx , tenant , entity , includeUserInfo )
2022-08-04 11:04:34 +08:00
}
// SelectUser select user.
// Enter the user condition by the entity param. And this param is nil, which means selecting all users.
// Get all roles that are added the user to by setting the includeRoleInfo param to true.
func ( mt * MetaTable ) SelectUser ( tenant string , entity * milvuspb . UserEntity , includeRoleInfo bool ) ( [ ] * milvuspb . UserResult , error ) {
2022-08-24 10:02:52 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-08-23 10:26:53 +08:00
return mt . catalog . ListUser ( mt . ctx , tenant , entity , includeRoleInfo )
2022-08-04 11:04:34 +08:00
}
// OperatePrivilege grant or revoke privilege by setting the operateType param
func ( mt * MetaTable ) OperatePrivilege ( tenant string , entity * milvuspb . GrantEntity , operateType milvuspb . OperatePrivilegeType ) error {
if funcutil . IsEmptyString ( entity . ObjectName ) {
return fmt . Errorf ( "the object name in the grant entity is empty" )
}
if entity . Object == nil || funcutil . IsEmptyString ( entity . Object . Name ) {
return fmt . Errorf ( "the object entity in the grant entity is invalid" )
}
if entity . Role == nil || funcutil . IsEmptyString ( entity . Role . Name ) {
return fmt . Errorf ( "the role entity in the grant entity is invalid" )
}
if entity . Grantor == nil {
return fmt . Errorf ( "the grantor in the grant entity is empty" )
}
if entity . Grantor . Privilege == nil || funcutil . IsEmptyString ( entity . Grantor . Privilege . Name ) {
return fmt . Errorf ( "the privilege name in the grant entity is empty" )
}
if entity . Grantor . User == nil || funcutil . IsEmptyString ( entity . Grantor . User . Name ) {
return fmt . Errorf ( "the grantor name in the grant entity is empty" )
}
if ! funcutil . IsRevoke ( operateType ) && ! funcutil . IsGrant ( operateType ) {
return fmt . Errorf ( "the operate type in the grant entity is invalid" )
}
2023-06-25 17:20:43 +08:00
if entity . DbName == "" {
entity . DbName = util . DefaultDBName
}
2022-08-04 11:04:34 +08:00
2022-08-24 10:02:52 +08:00
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
2022-08-23 10:26:53 +08:00
return mt . catalog . AlterGrant ( mt . ctx , tenant , entity , operateType )
2022-08-04 11:04:34 +08:00
}
// SelectGrant select grant
// The principal entity MUST be not empty in the grant entity
// The resource entity and the resource name are optional, and the two params should be not empty together when you select some grants about the resource kind.
func ( mt * MetaTable ) SelectGrant ( tenant string , entity * milvuspb . GrantEntity ) ( [ ] * milvuspb . GrantEntity , error ) {
var entities [ ] * milvuspb . GrantEntity
2022-11-25 11:07:12 +08:00
if entity == nil {
return entities , fmt . Errorf ( "the grant entity is nil" )
}
2022-08-04 11:04:34 +08:00
if entity . Role == nil || funcutil . IsEmptyString ( entity . Role . Name ) {
return entities , fmt . Errorf ( "the role entity in the grant entity is invalid" )
}
2023-06-25 17:20:43 +08:00
if entity . DbName == "" {
entity . DbName = util . DefaultDBName
}
2022-08-24 10:02:52 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-08-23 10:26:53 +08:00
return mt . catalog . ListGrant ( mt . ctx , tenant , entity )
2022-08-04 11:04:34 +08:00
}
2022-08-26 19:22:56 +08:00
func ( mt * MetaTable ) DropGrant ( tenant string , role * milvuspb . RoleEntity ) error {
if role == nil || funcutil . IsEmptyString ( role . Name ) {
return fmt . Errorf ( "the role entity is invalid when dropping the grant" )
}
mt . permissionLock . Lock ( )
defer mt . permissionLock . Unlock ( )
return mt . catalog . DeleteGrant ( mt . ctx , tenant , role )
}
2022-08-04 11:04:34 +08:00
func ( mt * MetaTable ) ListPolicy ( tenant string ) ( [ ] string , error ) {
2022-08-24 10:02:52 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-08-04 11:04:34 +08:00
return mt . catalog . ListPolicy ( mt . ctx , tenant )
}
func ( mt * MetaTable ) ListUserRole ( tenant string ) ( [ ] string , error ) {
2022-08-24 10:02:52 +08:00
mt . permissionLock . RLock ( )
defer mt . permissionLock . RUnlock ( )
2022-08-04 11:04:34 +08:00
return mt . catalog . ListUserRole ( mt . ctx , tenant )
}