2021-12-20 17:45:37 +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-20 17:45:37 +08:00
// http://www.apache.org/licenses/LICENSE-2.0
2021-04-19 11:12:56 +08:00
//
2021-12-20 17:45:37 +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 (
"context"
2021-01-25 18:33:10 +08:00
"fmt"
2021-01-19 14:44:03 +08:00
"math/rand"
2022-03-17 17:17:22 +08:00
"os"
2021-01-19 14:44:03 +08:00
"sync"
"time"
2023-02-26 11:31:49 +08:00
"github.com/cockroachdb/errors"
2022-11-29 20:05:14 +08:00
"github.com/samber/lo"
2023-09-07 07:25:14 +08:00
"github.com/tikv/client-go/v2/txnkv"
2023-04-06 19:14:32 +08:00
clientv3 "go.etcd.io/etcd/client/v3"
2023-10-11 21:01:35 +08:00
"go.uber.org/atomic"
2023-04-06 19:14:32 +08:00
"go.uber.org/zap"
2022-10-18 13:39:26 +08:00
"golang.org/x/sync/errgroup"
2023-06-09 01:28:37 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
2022-09-26 18:06:54 +08:00
"github.com/milvus-io/milvus/internal/allocator"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
2023-09-26 09:57:25 +08:00
"github.com/milvus-io/milvus/internal/kv/tikv"
2022-08-11 12:12:38 +08:00
"github.com/milvus-io/milvus/internal/metastore"
2022-09-05 13:29:11 +08:00
kvmetestore "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord"
2022-07-22 10:20:29 +08:00
"github.com/milvus-io/milvus/internal/metastore/model"
2022-09-26 18:06:54 +08:00
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
2021-06-22 16:14:09 +08:00
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
2023-04-06 19:14:32 +08:00
tso2 "github.com/milvus-io/milvus/internal/tso"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/types"
2022-04-21 19:57:42 +08:00
"github.com/milvus-io/milvus/internal/util/dependency"
2023-12-20 19:22:42 +08:00
"github.com/milvus-io/milvus/internal/util/proxyutil"
2021-05-21 19:28:52 +08:00
"github.com/milvus-io/milvus/internal/util/sessionutil"
2023-04-06 19:14:32 +08:00
tsoutil2 "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/pkg/common"
2024-06-26 13:36:05 +08:00
"github.com/milvus-io/milvus/pkg/kv"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/crypto"
2023-12-22 18:36:44 +08:00
"github.com/milvus-io/milvus/pkg/util/expr"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
2021-01-19 14:44:03 +08:00
)
2022-03-28 16:41:28 +08:00
// UniqueID is an alias of typeutil.UniqueID.
type UniqueID = typeutil . UniqueID
2022-09-05 13:29:11 +08:00
// Timestamp is an alias of typeutil.Timestamp
type Timestamp = typeutil . Timestamp
2022-05-19 10:13:56 +08:00
const InvalidCollectionID = UniqueID ( 0 )
2022-11-04 14:25:38 +08:00
var Params * paramtable . ComponentParam = paramtable . Get ( )
2021-12-23 18:39:11 +08:00
2022-09-05 13:29:11 +08:00
type Opt func ( * Core )
2021-09-15 22:05:49 +08:00
2023-09-07 07:25:14 +08:00
type metaKVCreator func ( ) ( kv . MetaKv , error )
2021-05-26 20:14:30 +08:00
2022-09-05 13:29:11 +08:00
// Core root coordinator core
type Core struct {
ctx context . Context
cancel context . CancelFunc
wg sync . WaitGroup
etcdCli * clientv3 . Client
2023-09-07 07:25:14 +08:00
tikvCli * txnkv . Client
2022-11-04 14:25:38 +08:00
address string
2022-09-05 13:29:11 +08:00
meta IMetaTable
scheduler IScheduler
broker Broker
2022-09-24 12:42:51 +08:00
ddlTsLockManager DdlTsLockManager
2022-09-05 13:29:11 +08:00
garbageCollector GarbageCollector
2022-09-21 15:46:51 +08:00
stepExecutor StepExecutor
2022-09-05 13:29:11 +08:00
metaKVCreator metaKVCreator
2023-12-20 19:22:42 +08:00
proxyCreator proxyutil . ProxyCreator
proxyWatcher * proxyutil . ProxyWatcher
proxyClientManager proxyutil . ProxyClientManagerInterface
2021-05-26 20:14:30 +08:00
2021-09-03 17:15:26 +08:00
metricsCacheManager * metricsinfo . MetricsCacheManager
2021-05-21 16:08:12 +08:00
chanTimeTick * timetickSync
2022-10-09 10:06:58 +08:00
idAllocator allocator . Interface
2023-04-06 19:14:32 +08:00
tsoAllocator tso2 . Allocator
2021-01-19 14:44:03 +08:00
2023-09-26 09:57:25 +08:00
dataCoord types . DataCoordClient
queryCoord types . QueryCoordClient
2021-01-19 14:44:03 +08:00
2022-09-16 09:56:47 +08:00
quotaCenter * QuotaCenter
2023-10-11 21:01:35 +08:00
stateCode atomic . Int32
2021-01-19 14:44:03 +08:00
initOnce sync . Once
startOnce sync . Once
2022-09-05 13:29:11 +08:00
session * sessionutil . Session
2021-05-21 19:28:52 +08:00
2022-04-07 22:05:32 +08:00
factory dependency . Factory
2022-03-21 15:47:23 +08:00
2022-09-29 18:35:02 +08:00
enableActiveStandBy bool
2023-02-09 15:24:31 +08:00
activateFunc func ( ) error
2021-01-19 14:44:03 +08:00
}
// --------------------- function --------------------------
2021-12-28 19:47:21 +08:00
// NewCore creates a new rootcoord core
2022-04-07 22:05:32 +08:00
func NewCore ( c context . Context , factory dependency . Factory ) ( * Core , error ) {
2021-01-19 14:44:03 +08:00
ctx , cancel := context . WithCancel ( c )
rand . Seed ( time . Now ( ) . UnixNano ( ) )
core := & Core {
2022-09-29 18:35:02 +08:00
ctx : ctx ,
cancel : cancel ,
factory : factory ,
2022-12-07 18:01:19 +08:00
enableActiveStandBy : Params . RootCoordCfg . EnableActiveStandby . GetAsBool ( ) ,
2021-01-19 14:44:03 +08:00
}
2022-10-10 15:55:22 +08:00
core . UpdateStateCode ( commonpb . StateCode_Abnormal )
2023-12-20 19:22:42 +08:00
core . SetProxyCreator ( proxyutil . DefaultProxyCreator )
2022-10-08 15:38:58 +08:00
2023-12-22 18:36:44 +08:00
expr . Register ( "rootcoord" , core )
2021-01-19 14:44:03 +08:00
return core , nil
}
2021-09-23 15:10:00 +08:00
// UpdateStateCode update state code
2022-10-10 15:55:22 +08:00
func ( c * Core ) UpdateStateCode ( code commonpb . StateCode ) {
2023-10-11 21:01:35 +08:00
c . stateCode . Store ( int32 ( code ) )
2022-10-25 09:51:30 +08:00
log . Info ( "update rootcoord state" , zap . String ( "state" , code . String ( ) ) )
2021-02-23 11:40:30 +08:00
}
2023-10-11 21:01:35 +08:00
func ( c * Core ) GetStateCode ( ) commonpb . StateCode {
return commonpb . StateCode ( c . stateCode . Load ( ) )
2021-11-19 12:11:12 +08:00
}
2022-09-05 13:29:11 +08:00
func ( c * Core ) sendTimeTick ( t Timestamp , reason string ) error {
pc := c . chanTimeTick . listDmlChannels ( )
pt := make ( [ ] uint64 , len ( pc ) )
for i := 0 ; i < len ( pt ) ; i ++ {
pt [ i ] = t
2021-02-20 15:38:44 +08:00
}
2022-09-05 13:29:11 +08:00
ttMsg := internalpb . ChannelTimeTickMsg {
2022-10-21 15:57:28 +08:00
Base : commonpbutil . NewMsgBase (
commonpbutil . WithMsgType ( commonpb . MsgType_TimeTick ) ,
commonpbutil . WithTimeStamp ( t ) ,
2022-11-14 14:41:11 +08:00
commonpbutil . WithSourceID ( ddlSourceID ) ,
2022-10-21 15:57:28 +08:00
) ,
2022-09-05 13:29:11 +08:00
ChannelNames : pc ,
Timestamps : pt ,
DefaultTimestamp : t ,
2022-06-15 12:20:10 +08:00
}
2022-09-05 13:29:11 +08:00
return c . chanTimeTick . updateTimeTick ( & ttMsg , reason )
2021-01-19 14:44:03 +08:00
}
2022-09-21 15:46:51 +08:00
func ( c * Core ) sendMinDdlTsAsTt ( ) {
2023-10-20 14:26:09 +08:00
if ! paramtable . Get ( ) . CommonCfg . TTMsgEnabled . GetAsBool ( ) {
return
}
2023-10-11 21:01:35 +08:00
code := c . GetStateCode ( )
2022-11-14 14:41:11 +08:00
if code != commonpb . StateCode_Healthy {
log . Warn ( "rootCoord is not healthy, skip send timetick" )
return
}
2022-10-09 12:32:57 +08:00
minBgDdlTs := c . ddlTsLockManager . GetMinDdlTs ( )
minNormalDdlTs := c . scheduler . GetMinDdlTs ( )
minDdlTs := funcutil . Min ( minBgDdlTs , minNormalDdlTs )
// zero -> ddlTsLockManager and scheduler not started.
if minDdlTs == typeutil . ZeroTimestamp {
log . Warn ( "zero ts was met, this should be only occurred in starting state" , zap . Uint64 ( "minBgDdlTs" , minBgDdlTs ) , zap . Uint64 ( "minNormalDdlTs" , minNormalDdlTs ) )
return
}
// max -> abnormal case, impossible.
if minDdlTs == typeutil . MaxTimestamp {
log . Warn ( "ddl ts is abnormal, max ts was met" , zap . Uint64 ( "minBgDdlTs" , minBgDdlTs ) , zap . Uint64 ( "minNormalDdlTs" , minNormalDdlTs ) )
return
}
if err := c . sendTimeTick ( minDdlTs , "timetick loop" ) ; err != nil {
2022-09-21 15:46:51 +08:00
log . Warn ( "failed to send timetick" , zap . Error ( err ) )
}
}
2021-01-19 14:44:03 +08:00
func ( c * Core ) startTimeTickLoop ( ) {
2021-09-17 12:37:50 +08:00
defer c . wg . Done ( )
2022-12-07 18:01:19 +08:00
ticker := time . NewTicker ( Params . ProxyCfg . TimeTickInterval . GetAsDuration ( time . Millisecond ) )
2023-02-23 18:59:45 +08:00
defer ticker . Stop ( )
2021-05-31 16:48:31 +08:00
for {
select {
case <- c . ctx . Done ( ) :
2023-04-29 16:06:42 +08:00
log . Info ( "rootcoord's timetick loop quit!" )
2021-05-31 16:48:31 +08:00
return
case <- ticker . C :
2022-09-21 15:46:51 +08:00
c . sendMinDdlTsAsTt ( )
2021-01-19 14:44:03 +08:00
}
}
}
2021-01-27 16:38:18 +08:00
func ( c * Core ) tsLoop ( ) {
2021-09-17 12:37:50 +08:00
defer c . wg . Done ( )
2023-04-06 19:14:32 +08:00
tsoTicker := time . NewTicker ( tso2 . UpdateTimestampStep )
2021-01-27 16:38:18 +08:00
defer tsoTicker . Stop ( )
ctx , cancel := context . WithCancel ( c . ctx )
defer cancel ( )
for {
select {
case <- tsoTicker . C :
2022-09-05 13:29:11 +08:00
if err := c . tsoAllocator . UpdateTSO ( ) ; err != nil {
2023-03-13 18:01:53 +08:00
log . Warn ( "failed to update tso" , zap . Error ( err ) )
2021-03-15 15:45:17 +08:00
continue
2021-01-27 16:38:18 +08:00
}
2022-09-05 13:29:11 +08:00
ts := c . tsoAllocator . GetLastSavedTime ( )
2022-04-27 23:03:47 +08:00
metrics . RootCoordTimestampSaved . Set ( float64 ( ts . Unix ( ) ) )
2023-03-13 18:01:53 +08:00
2021-01-27 16:38:18 +08:00
case <- ctx . Done ( ) :
2023-04-29 16:06:42 +08:00
log . Info ( "rootcoord's ts loop quit!" )
2021-01-27 16:38:18 +08:00
return
}
}
}
2021-05-14 21:26:06 +08:00
2023-09-26 09:57:25 +08:00
func ( c * Core ) SetProxyCreator ( f func ( ctx context . Context , addr string , nodeID int64 ) ( types . ProxyClient , error ) ) {
2023-01-12 19:49:40 +08:00
c . proxyCreator = f
}
2023-09-26 09:57:25 +08:00
func ( c * Core ) SetDataCoordClient ( s types . DataCoordClient ) error {
2023-01-12 19:49:40 +08:00
if s == nil {
return errors . New ( "null DataCoord interface" )
2022-06-15 12:20:10 +08:00
}
2022-09-05 13:29:11 +08:00
c . dataCoord = s
2021-01-24 20:26:35 +08:00
return nil
}
2023-09-26 09:57:25 +08:00
func ( c * Core ) SetQueryCoordClient ( s types . QueryCoordClient ) error {
2023-01-12 19:49:40 +08:00
if s == nil {
return errors . New ( "null QueryCoord interface" )
2022-05-05 21:17:50 +08:00
}
2022-09-05 13:29:11 +08:00
c . queryCoord = s
2021-02-05 14:09:55 +08:00
return nil
}
2021-06-17 16:47:57 +08:00
// Register register rootcoord at etcd
2021-05-25 15:06:05 +08:00
func ( c * Core ) Register ( ) error {
2021-12-15 11:47:10 +08:00
c . session . Register ( )
2024-04-10 18:49:18 +08:00
afterRegister := func ( ) {
metrics . NumNodes . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , typeutil . RootCoordRole ) . Inc ( )
log . Info ( "RootCoord Register Finished" )
c . session . LivenessCheck ( c . ctx , func ( ) {
log . Error ( "Root Coord disconnected from etcd, process will exit" , zap . Int64 ( "Server Id" , c . session . ServerID ) )
os . Exit ( 1 )
} )
}
2022-09-29 18:35:02 +08:00
if c . enableActiveStandBy {
2024-04-10 18:49:18 +08:00
go func ( ) {
if err := c . session . ProcessActiveStandBy ( c . activateFunc ) ; err != nil {
log . Warn ( "failed to activate standby rootcoord server" , zap . Error ( err ) )
2024-05-27 10:13:59 +08:00
panic ( err )
2024-04-10 18:49:18 +08:00
}
afterRegister ( )
} ( )
} else {
afterRegister ( )
2022-09-29 18:35:02 +08:00
}
2022-01-18 12:09:37 +08:00
2021-12-15 11:47:10 +08:00
return nil
}
2022-11-04 14:25:38 +08:00
func ( c * Core ) SetAddress ( address string ) {
c . address = address
}
2021-12-31 14:23:55 +08:00
// SetEtcdClient sets the etcdCli of Core
2021-12-29 14:35:21 +08:00
func ( c * Core ) SetEtcdClient ( etcdClient * clientv3 . Client ) {
c . etcdCli = etcdClient
}
2023-09-07 07:25:14 +08:00
// SetTiKVClient sets the tikvCli of Core
func ( c * Core ) SetTiKVClient ( client * txnkv . Client ) {
c . tikvCli = client
}
2021-12-15 11:47:10 +08:00
func ( c * Core ) initSession ( ) error {
2023-10-27 07:36:12 +08:00
c . session = sessionutil . NewSession ( c . ctx )
2021-06-03 19:01:33 +08:00
if c . session == nil {
2021-10-13 10:50:41 +08:00
return fmt . Errorf ( "session is nil, the etcd client connection may have failed" )
2021-06-03 19:01:33 +08:00
}
2022-11-04 14:25:38 +08:00
c . session . Init ( typeutil . RootCoordRole , c . address , true , true )
2022-09-29 18:35:02 +08:00
c . session . SetEnableActiveStandBy ( c . enableActiveStandBy )
2021-05-25 15:06:05 +08:00
return nil
}
2022-09-05 13:29:11 +08:00
func ( c * Core ) initKVCreator ( ) {
if c . metaKVCreator == nil {
2023-09-07 07:25:14 +08:00
if Params . MetaStoreCfg . MetaStoreType . GetValue ( ) == util . MetaStoreTypeTiKV {
c . metaKVCreator = func ( ) ( kv . MetaKv , error ) {
2023-11-23 19:34:23 +08:00
return tikv . NewTiKV ( c . tikvCli , Params . TiKVCfg . MetaRootPath . GetValue ( ) ,
tikv . WithRequestTimeout ( paramtable . Get ( ) . ServiceParam . TiKVCfg . RequestTimeout . GetAsDuration ( time . Millisecond ) ) ) , nil
2023-09-07 07:25:14 +08:00
}
} else {
c . metaKVCreator = func ( ) ( kv . MetaKv , error ) {
2023-11-23 19:34:23 +08:00
return etcdkv . NewEtcdKV ( c . etcdCli , Params . EtcdCfg . MetaRootPath . GetValue ( ) ,
etcdkv . WithRequestTimeout ( paramtable . Get ( ) . ServiceParam . EtcdCfg . RequestTimeout . GetAsDuration ( time . Millisecond ) ) ) , nil
2023-09-07 07:25:14 +08:00
}
}
2022-03-25 11:03:25 +08:00
}
2022-09-05 13:29:11 +08:00
}
2022-07-22 10:20:29 +08:00
2022-09-05 13:29:11 +08:00
func ( c * Core ) initMetaTable ( ) error {
fn := func ( ) error {
var catalog metastore . RootCoordCatalog
var err error
2022-08-11 12:12:38 +08:00
2022-12-07 18:01:19 +08:00
switch Params . MetaStoreCfg . MetaStoreType . GetValue ( ) {
2022-09-05 13:29:11 +08:00
case util . MetaStoreTypeEtcd :
2023-09-07 07:25:14 +08:00
log . Info ( "Using etcd as meta storage." )
2022-09-05 13:29:11 +08:00
var metaKV kv . MetaKv
var ss * kvmetestore . SuffixSnapshot
var err error
2022-08-11 12:12:38 +08:00
2023-09-07 07:25:14 +08:00
if metaKV , err = c . metaKVCreator ( ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
}
2022-08-11 12:12:38 +08:00
2022-11-17 18:59:09 +08:00
if ss , err = kvmetestore . NewSuffixSnapshot ( metaKV , kvmetestore . SnapshotsSep , Params . EtcdCfg . MetaRootPath . GetValue ( ) , kvmetestore . SnapshotPrefix ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
2021-02-26 15:17:47 +08:00
}
2023-09-07 07:25:14 +08:00
catalog = & kvmetestore . Catalog { Txn : metaKV , Snapshot : ss }
case util . MetaStoreTypeTiKV :
log . Info ( "Using tikv as meta storage." )
var metaKV kv . MetaKv
var ss * kvmetestore . SuffixSnapshot
var err error
if metaKV , err = c . metaKVCreator ( ) ; err != nil {
return err
}
2022-08-11 12:12:38 +08:00
2023-09-07 07:25:14 +08:00
if ss , err = kvmetestore . NewSuffixSnapshot ( metaKV , kvmetestore . SnapshotsSep , Params . TiKVCfg . MetaRootPath . GetValue ( ) , kvmetestore . SnapshotPrefix ) ; err != nil {
return err
}
2022-09-05 13:29:11 +08:00
catalog = & kvmetestore . Catalog { Txn : metaKV , Snapshot : ss }
default :
2022-12-07 18:01:19 +08:00
return retry . Unrecoverable ( fmt . Errorf ( "not supported meta store: %s" , Params . MetaStoreCfg . MetaStoreType . GetValue ( ) ) )
2021-01-19 14:44:03 +08:00
}
2023-06-25 17:20:43 +08:00
if c . meta , err = NewMetaTable ( c . ctx , catalog , c . tsoAllocator ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
2021-04-08 17:31:39 +08:00
}
2022-09-05 13:29:11 +08:00
return nil
}
2021-04-08 17:31:39 +08:00
2022-09-05 13:29:11 +08:00
return retry . Do ( c . ctx , fn , retry . Attempts ( 10 ) )
}
2021-06-04 15:00:34 +08:00
2022-09-05 13:29:11 +08:00
func ( c * Core ) initIDAllocator ( ) error {
2023-09-07 07:25:14 +08:00
var tsoKV kv . TxnKV
var kvPath string
if Params . MetaStoreCfg . MetaStoreType . GetValue ( ) == util . MetaStoreTypeTiKV {
kvPath = Params . TiKVCfg . KvRootPath . GetValue ( )
tsoKV = tsoutil2 . NewTSOTiKVBase ( c . tikvCli , kvPath , globalIDAllocatorSubPath )
} else {
kvPath = Params . EtcdCfg . KvRootPath . GetValue ( )
tsoKV = tsoutil2 . NewTSOKVBase ( c . etcdCli , kvPath , globalIDAllocatorSubPath )
}
2022-09-05 13:29:11 +08:00
idAllocator := allocator . NewGlobalIDAllocator ( globalIDAllocatorKey , tsoKV )
if err := idAllocator . Initialize ( ) ; err != nil {
return err
}
c . idAllocator = idAllocator
2023-06-25 17:20:43 +08:00
log . Info ( "id allocator initialized" ,
2023-09-07 07:25:14 +08:00
zap . String ( "root_path" , kvPath ) ,
2023-06-25 17:20:43 +08:00
zap . String ( "sub_path" , globalIDAllocatorSubPath ) ,
zap . String ( "key" , globalIDAllocatorKey ) )
2022-09-05 13:29:11 +08:00
return nil
}
2021-05-26 20:14:30 +08:00
2022-09-05 13:29:11 +08:00
func ( c * Core ) initTSOAllocator ( ) error {
2023-09-07 07:25:14 +08:00
var tsoKV kv . TxnKV
var kvPath string
if Params . MetaStoreCfg . MetaStoreType . GetValue ( ) == util . MetaStoreTypeTiKV {
kvPath = Params . TiKVCfg . KvRootPath . GetValue ( )
tsoKV = tsoutil2 . NewTSOTiKVBase ( c . tikvCli , Params . TiKVCfg . KvRootPath . GetValue ( ) , globalIDAllocatorSubPath )
} else {
kvPath = Params . EtcdCfg . KvRootPath . GetValue ( )
tsoKV = tsoutil2 . NewTSOKVBase ( c . etcdCli , Params . EtcdCfg . KvRootPath . GetValue ( ) , globalIDAllocatorSubPath )
}
2023-04-06 19:14:32 +08:00
tsoAllocator := tso2 . NewGlobalTSOAllocator ( globalTSOAllocatorKey , tsoKV )
2022-09-05 13:29:11 +08:00
if err := tsoAllocator . Initialize ( ) ; err != nil {
return err
}
c . tsoAllocator = tsoAllocator
2021-05-21 16:08:12 +08:00
2023-06-25 17:20:43 +08:00
log . Info ( "tso allocator initialized" ,
2023-09-07 07:25:14 +08:00
zap . String ( "root_path" , kvPath ) ,
2023-06-25 17:20:43 +08:00
zap . String ( "sub_path" , globalIDAllocatorSubPath ) ,
zap . String ( "key" , globalIDAllocatorKey ) )
2022-09-05 13:29:11 +08:00
return nil
}
2021-09-03 17:15:26 +08:00
2022-09-05 13:29:11 +08:00
func ( c * Core ) initInternal ( ) error {
2023-02-10 17:40:32 +08:00
c . UpdateStateCode ( commonpb . StateCode_Initializing )
2022-09-05 13:29:11 +08:00
c . initKVCreator ( )
2023-06-25 17:20:43 +08:00
if err := c . initIDAllocator ( ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
}
2023-06-25 17:20:43 +08:00
if err := c . initTSOAllocator ( ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
}
2023-06-25 17:20:43 +08:00
if err := c . initMetaTable ( ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
2021-01-26 19:24:09 +08:00
}
2022-09-05 13:29:11 +08:00
c . scheduler = newScheduler ( c . ctx , c . idAllocator , c . tsoAllocator )
2022-11-04 14:25:38 +08:00
c . factory . Init ( Params )
2022-09-05 13:29:11 +08:00
chanMap := c . meta . ListCollectionPhysicalChannels ( )
c . chanTimeTick = newTimeTickSync ( c . ctx , c . session . ServerID , c . factory , chanMap )
2023-11-27 16:30:26 +08:00
log . Info ( "create TimeTick sync done" )
2023-12-20 19:22:42 +08:00
c . proxyClientManager = proxyutil . NewProxyClientManager ( c . proxyCreator )
2022-09-05 13:29:11 +08:00
c . broker = newServerBroker ( c )
2022-09-24 12:42:51 +08:00
c . ddlTsLockManager = newDdlTsLockManager ( c . tsoAllocator )
2022-09-21 15:46:51 +08:00
c . garbageCollector = newBgGarbageCollector ( c )
c . stepExecutor = newBgStepExecutor ( c . ctx )
2022-09-05 13:29:11 +08:00
2023-12-20 19:22:42 +08:00
c . proxyWatcher = proxyutil . NewProxyWatcher (
2022-09-05 13:29:11 +08:00
c . etcdCli ,
c . chanTimeTick . initSessions ,
2023-12-12 10:18:39 +08:00
c . proxyClientManager . AddProxyClients ,
2022-09-05 13:29:11 +08:00
)
2023-12-20 19:22:42 +08:00
c . proxyWatcher . AddSessionFunc ( c . chanTimeTick . addSession , c . proxyClientManager . AddProxyClient )
c . proxyWatcher . DelSessionFunc ( c . chanTimeTick . delSession , c . proxyClientManager . DelProxyClient )
2023-11-27 16:30:26 +08:00
log . Info ( "init proxy manager done" )
2022-09-05 13:29:11 +08:00
c . metricsCacheManager = metricsinfo . NewMetricsCacheManager ( )
2023-06-21 15:02:42 +08:00
c . quotaCenter = NewQuotaCenter ( c . proxyClientManager , c . queryCoord , c . dataCoord , c . tsoAllocator , c . meta )
2022-09-16 09:56:47 +08:00
log . Debug ( "RootCoord init QuotaCenter done" )
2022-09-05 13:29:11 +08:00
if err := c . initCredentials ( ) ; err != nil {
return err
}
2023-11-27 16:30:26 +08:00
log . Info ( "init credentials done" )
2022-09-05 13:29:11 +08:00
if err := c . initRbac ( ) ; err != nil {
return err
}
2023-11-27 16:30:26 +08:00
log . Info ( "init rootcoord done" , zap . Int64 ( "nodeID" , paramtable . GetNodeID ( ) ) , zap . String ( "Address" , c . address ) )
2022-09-05 13:29:11 +08:00
return nil
}
// Init initialize routine
func ( c * Core ) Init ( ) error {
var initError error
2023-01-30 11:11:50 +08:00
c . factory . Init ( Params )
if err := c . initSession ( ) ; err != nil {
return err
}
if c . enableActiveStandBy {
2023-02-09 15:24:31 +08:00
c . activateFunc = func ( ) error {
2023-01-30 11:11:50 +08:00
log . Info ( "RootCoord switch from standby to active, activating" )
2023-02-09 15:24:31 +08:00
var err error
2023-01-30 11:11:50 +08:00
c . initOnce . Do ( func ( ) {
2023-02-09 15:24:31 +08:00
if err = c . initInternal ( ) ; err != nil {
log . Error ( "RootCoord init failed" , zap . Error ( err ) )
2023-01-30 11:11:50 +08:00
}
} )
2023-02-09 15:24:31 +08:00
if err != nil {
return err
}
2023-01-30 11:11:50 +08:00
c . startOnce . Do ( func ( ) {
2023-02-09 15:24:31 +08:00
if err = c . startInternal ( ) ; err != nil {
log . Error ( "RootCoord start failed" , zap . Error ( err ) )
2023-01-30 11:11:50 +08:00
}
} )
2023-09-06 17:43:14 +08:00
log . Info ( "RootCoord startup success" , zap . String ( "address" , c . session . Address ) )
2023-02-09 15:24:31 +08:00
return err
2023-01-30 11:11:50 +08:00
}
c . UpdateStateCode ( commonpb . StateCode_StandBy )
log . Info ( "RootCoord enter standby mode successfully" )
} else {
c . initOnce . Do ( func ( ) {
initError = c . initInternal ( )
} )
}
2021-01-19 14:44:03 +08:00
return initError
}
2022-09-05 13:29:11 +08:00
func ( c * Core ) initCredentials ( ) error {
credInfo , _ := c . meta . GetCredential ( util . UserRoot )
2022-04-21 19:57:42 +08:00
if credInfo == nil {
log . Debug ( "RootCoord init user root" )
encryptedRootPassword , _ := crypto . PasswordEncrypt ( util . DefaultRootPassword )
2022-09-05 13:29:11 +08:00
err := c . meta . AddCredential ( & internalpb . CredentialInfo { Username : util . UserRoot , EncryptedPassword : encryptedRootPassword } )
2022-04-21 19:57:42 +08:00
return err
}
return nil
}
2023-03-13 10:59:53 +08:00
func ( c * Core ) initRbac ( ) error {
var err error
2022-08-04 11:04:34 +08:00
// create default roles, including admin, public
2022-08-23 10:26:53 +08:00
for _ , role := range util . DefaultRoles {
2023-03-13 10:59:53 +08:00
err = c . meta . CreateRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : role } )
if err != nil && ! common . IsIgnorableError ( err ) {
return errors . Wrap ( err , "failed to create role" )
2022-08-23 10:26:53 +08:00
}
2022-08-04 11:04:34 +08:00
}
2024-05-21 22:39:46 +08:00
if Params . ProxyCfg . EnablePublicPrivilege . GetAsBool ( ) {
err = c . initPublicRolePrivilege ( )
if err != nil {
return err
}
}
if Params . RoleCfg . Enabled . GetAsBool ( ) {
return c . initBuiltinRoles ( )
}
return nil
}
func ( c * Core ) initPublicRolePrivilege ( ) error {
2022-08-04 11:04:34 +08:00
// grant privileges for the public role
globalPrivileges := [ ] string {
commonpb . ObjectPrivilege_PrivilegeDescribeCollection . String ( ) ,
}
2022-08-05 16:28:35 +08:00
collectionPrivileges := [ ] string {
commonpb . ObjectPrivilege_PrivilegeIndexDetail . String ( ) ,
}
2022-08-04 11:04:34 +08:00
2024-05-21 22:39:46 +08:00
var err error
2022-08-04 11:04:34 +08:00
for _ , globalPrivilege := range globalPrivileges {
2023-03-13 10:59:53 +08:00
err = c . meta . OperatePrivilege ( util . DefaultTenant , & milvuspb . GrantEntity {
2022-08-04 11:04:34 +08:00
Role : & milvuspb . RoleEntity { Name : util . RolePublic } ,
Object : & milvuspb . ObjectEntity { Name : commonpb . ObjectType_Global . String ( ) } ,
2022-08-15 16:40:48 +08:00
ObjectName : util . AnyWord ,
2023-06-25 17:20:43 +08:00
DbName : util . AnyWord ,
2022-08-04 11:04:34 +08:00
Grantor : & milvuspb . GrantorEntity {
2022-08-26 19:22:56 +08:00
User : & milvuspb . UserEntity { Name : util . UserRoot } ,
2022-08-04 11:04:34 +08:00
Privilege : & milvuspb . PrivilegeEntity { Name : globalPrivilege } ,
} ,
2023-03-13 10:59:53 +08:00
} , milvuspb . OperatePrivilegeType_Grant )
if err != nil && ! common . IsIgnorableError ( err ) {
return errors . Wrap ( err , "failed to grant global privilege" )
2022-08-04 11:04:34 +08:00
}
}
2022-08-05 16:28:35 +08:00
for _ , collectionPrivilege := range collectionPrivileges {
2023-03-13 10:59:53 +08:00
err = c . meta . OperatePrivilege ( util . DefaultTenant , & milvuspb . GrantEntity {
2022-08-05 16:28:35 +08:00
Role : & milvuspb . RoleEntity { Name : util . RolePublic } ,
Object : & milvuspb . ObjectEntity { Name : commonpb . ObjectType_Collection . String ( ) } ,
2022-08-15 16:40:48 +08:00
ObjectName : util . AnyWord ,
2023-06-25 17:20:43 +08:00
DbName : util . AnyWord ,
2022-08-05 16:28:35 +08:00
Grantor : & milvuspb . GrantorEntity {
2022-08-26 19:22:56 +08:00
User : & milvuspb . UserEntity { Name : util . UserRoot } ,
2022-08-05 16:28:35 +08:00
Privilege : & milvuspb . PrivilegeEntity { Name : collectionPrivilege } ,
} ,
2023-03-13 10:59:53 +08:00
} , milvuspb . OperatePrivilegeType_Grant )
if err != nil && ! common . IsIgnorableError ( err ) {
return errors . Wrap ( err , "failed to grant collection privilege" )
2022-08-05 16:28:35 +08:00
}
}
2023-12-18 14:28:41 +08:00
return nil
}
func ( c * Core ) initBuiltinRoles ( ) error {
rolePrivilegesMap := Params . RoleCfg . Roles . GetAsRoleDetails ( )
for role , privilegesJSON := range rolePrivilegesMap {
err := c . meta . CreateRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : role } )
if err != nil && ! common . IsIgnorableError ( err ) {
2024-01-05 16:12:48 +08:00
log . Error ( "create a builtin role fail" , zap . String ( "roleName" , role ) , zap . Error ( err ) )
2023-12-18 14:28:41 +08:00
return errors . Wrapf ( err , "failed to create a builtin role: %s" , role )
}
for _ , privilege := range privilegesJSON [ util . RoleConfigPrivileges ] {
privilegeName := privilege [ util . RoleConfigPrivilege ]
if ! util . IsAnyWord ( privilege [ util . RoleConfigPrivilege ] ) {
privilegeName = util . PrivilegeNameForMetastore ( privilege [ util . RoleConfigPrivilege ] )
}
err := c . meta . OperatePrivilege ( util . DefaultTenant , & milvuspb . GrantEntity {
Role : & milvuspb . RoleEntity { Name : role } ,
Object : & milvuspb . ObjectEntity { Name : privilege [ util . RoleConfigObjectType ] } ,
ObjectName : privilege [ util . RoleConfigObjectName ] ,
DbName : privilege [ util . RoleConfigDBName ] ,
Grantor : & milvuspb . GrantorEntity {
User : & milvuspb . UserEntity { Name : util . UserRoot } ,
Privilege : & milvuspb . PrivilegeEntity { Name : privilegeName } ,
} ,
} , milvuspb . OperatePrivilegeType_Grant )
if err != nil && ! common . IsIgnorableError ( err ) {
2024-01-05 16:12:48 +08:00
log . Error ( "grant privilege to builtin role fail" , zap . String ( "roleName" , role ) , zap . Any ( "privilege" , privilege ) , zap . Error ( err ) )
2023-12-18 14:28:41 +08:00
return errors . Wrapf ( err , "failed to grant privilege: <%s, %s, %s> of db: %s to role: %s" , privilege [ util . RoleConfigObjectType ] , privilege [ util . RoleConfigObjectName ] , privilege [ util . RoleConfigPrivilege ] , privilege [ util . RoleConfigDBName ] , role )
}
}
util . BuiltinRoles = append ( util . BuiltinRoles , role )
log . Info ( "init a builtin role successfully" , zap . String ( "roleName" , role ) )
}
2022-08-04 11:04:34 +08:00
return nil
}
2022-09-05 13:29:11 +08:00
func ( c * Core ) restore ( ctx context . Context ) error {
2023-06-25 17:20:43 +08:00
dbs , err := c . meta . ListDatabases ( ctx , typeutil . MaxTimestamp )
2022-09-05 13:29:11 +08:00
if err != nil {
return err
}
2023-06-25 17:20:43 +08:00
for _ , db := range dbs {
colls , err := c . meta . ListCollections ( ctx , db . Name , typeutil . MaxTimestamp , false )
2022-09-05 13:29:11 +08:00
if err != nil {
return err
}
2023-06-25 17:20:43 +08:00
for _ , coll := range colls {
2023-08-03 10:37:06 +08:00
ts , err := c . tsoAllocator . GenerateTSO ( 1 )
if err != nil {
return err
}
if coll . Available ( ) {
for _ , part := range coll . Partitions {
2023-06-25 17:20:43 +08:00
switch part . State {
case pb . PartitionState_PartitionDropping :
go c . garbageCollector . ReDropPartition ( coll . DBID , coll . PhysicalChannelNames , part . Clone ( ) , ts )
case pb . PartitionState_PartitionCreating :
go c . garbageCollector . RemoveCreatingPartition ( coll . DBID , part . Clone ( ) , ts )
default :
}
2023-08-03 10:37:06 +08:00
}
} else {
switch coll . State {
case pb . CollectionState_CollectionDropping :
go c . garbageCollector . ReDropCollection ( coll . Clone ( ) , ts )
case pb . CollectionState_CollectionCreating :
go c . garbageCollector . RemoveCreatingCollection ( coll . Clone ( ) )
default :
2023-06-25 17:20:43 +08:00
}
2022-09-05 13:29:11 +08:00
}
}
}
return nil
}
func ( c * Core ) startInternal ( ) error {
2023-12-20 19:22:42 +08:00
if err := c . proxyWatcher . WatchProxy ( c . ctx ) ; err != nil {
2022-09-05 13:29:11 +08:00
log . Fatal ( "rootcoord failed to watch proxy" , zap . Error ( err ) )
// you can not just stuck here,
panic ( err )
2022-06-14 16:18:09 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . restore ( c . ctx ) ; err != nil {
panic ( err )
2022-06-14 16:18:09 +08:00
}
2022-12-07 18:01:19 +08:00
if Params . QuotaConfig . QuotaAndLimitsEnabled . GetAsBool ( ) {
2024-04-01 21:09:18 +08:00
c . quotaCenter . Start ( )
2022-09-16 09:56:47 +08:00
}
2022-09-05 13:29:11 +08:00
c . scheduler . Start ( )
2022-09-21 15:46:51 +08:00
c . stepExecutor . Start ( )
2023-08-07 11:59:07 +08:00
go func ( ) {
// refresh rbac cache
if err := retry . Do ( c . ctx , func ( ) error {
if err := c . proxyClientManager . RefreshPolicyInfoCache ( c . ctx , & proxypb . RefreshPolicyInfoCacheRequest {
OpType : int32 ( typeutil . CacheRefresh ) ,
} ) ; err != nil {
2023-11-27 16:30:26 +08:00
log . RatedWarn ( 60 , "fail to refresh policy info cache" , zap . Error ( err ) )
2023-08-07 11:59:07 +08:00
return err
}
return nil
} , retry . Attempts ( 100 ) , retry . Sleep ( time . Second ) ) ; err != nil {
2023-08-22 09:04:21 +08:00
log . Warn ( "fail to refresh policy info cache" , zap . Error ( err ) )
2023-08-07 11:59:07 +08:00
}
} ( )
2022-09-05 13:29:11 +08:00
2023-01-30 11:11:50 +08:00
c . startServerLoop ( )
c . UpdateStateCode ( commonpb . StateCode_Healthy )
2023-10-16 10:24:10 +08:00
sessionutil . SaveServerInfo ( typeutil . RootCoordRole , c . session . ServerID )
2023-01-30 11:11:50 +08:00
logutil . Logger ( c . ctx ) . Info ( "rootcoord startup successfully" )
2022-09-05 13:29:11 +08:00
return nil
2022-06-14 16:18:09 +08:00
}
2022-12-27 12:47:34 +08:00
func ( c * Core ) startServerLoop ( ) {
2024-03-22 15:29:09 +08:00
c . wg . Add ( 3 )
2022-12-27 12:47:34 +08:00
go c . startTimeTickLoop ( )
go c . tsLoop ( )
go c . chanTimeTick . startWatch ( & c . wg )
}
2022-04-03 11:37:29 +08:00
// Start starts RootCoord.
2021-01-19 14:44:03 +08:00
func ( c * Core ) Start ( ) error {
2022-09-05 13:29:11 +08:00
var err error
2023-01-30 11:11:50 +08:00
if ! c . enableActiveStandBy {
c . startOnce . Do ( func ( ) {
err = c . startInternal ( )
} )
}
2022-09-05 13:29:11 +08:00
return err
2021-01-19 14:44:03 +08:00
}
2022-10-25 09:51:30 +08:00
func ( c * Core ) stopExecutor ( ) {
if c . stepExecutor != nil {
c . stepExecutor . Stop ( )
log . Info ( "stop rootcoord executor" )
}
}
func ( c * Core ) stopScheduler ( ) {
if c . scheduler != nil {
c . scheduler . Stop ( )
log . Info ( "stop rootcoord scheduler" )
}
}
func ( c * Core ) cancelIfNotNil ( ) {
if c . cancel != nil {
c . cancel ( )
log . Info ( "cancel rootcoord goroutines" )
}
}
func ( c * Core ) revokeSession ( ) {
if c . session != nil {
// wait at most one second to revoke
2023-04-12 20:12:28 +08:00
c . session . Stop ( )
2024-03-15 10:33:05 +08:00
log . Info ( "rootcoord session stop" )
2022-10-25 09:51:30 +08:00
}
}
2022-04-03 11:37:29 +08:00
// Stop stops rootCoord.
2021-01-19 14:44:03 +08:00
func ( c * Core ) Stop ( ) error {
2022-10-10 15:55:22 +08:00
c . UpdateStateCode ( commonpb . StateCode_Abnormal )
2022-10-25 09:51:30 +08:00
c . stopExecutor ( )
c . stopScheduler ( )
2023-12-20 19:22:42 +08:00
if c . proxyWatcher != nil {
c . proxyWatcher . Stop ( )
2023-08-23 19:08:23 +08:00
}
2023-03-10 11:03:52 +08:00
if c . quotaCenter != nil {
c . quotaCenter . stop ( )
}
2024-03-15 10:33:05 +08:00
2022-10-25 09:51:30 +08:00
c . revokeSession ( )
2024-03-15 10:33:05 +08:00
c . cancelIfNotNil ( )
c . wg . Wait ( )
2021-01-19 14:44:03 +08:00
return nil
}
2021-09-23 15:10:00 +08:00
// GetComponentStates get states of components
2023-09-26 09:57:25 +08:00
func ( c * Core ) GetComponentStates ( ctx context . Context , req * milvuspb . GetComponentStatesRequest ) ( * milvuspb . ComponentStates , error ) {
2023-10-11 21:01:35 +08:00
code := c . GetStateCode ( )
2023-11-22 09:12:22 +08:00
log . Debug ( "RootCoord current state" , zap . String ( "StateCode" , code . String ( ) ) )
2021-01-26 19:24:09 +08:00
2021-11-19 13:57:12 +08:00
nodeID := common . NotRegisteredID
if c . session != nil && c . session . Registered ( ) {
nodeID = c . session . ServerID
}
2022-10-10 15:55:22 +08:00
return & milvuspb . ComponentStates {
State : & milvuspb . ComponentInfo {
2021-11-19 13:57:12 +08:00
// NodeID: c.session.ServerID, // will race with Core.Register()
NodeID : nodeID ,
2021-06-17 16:47:57 +08:00
Role : typeutil . RootCoordRole ,
2021-01-20 11:02:29 +08:00
StateCode : code ,
ExtraInfo : nil ,
2021-01-19 14:44:03 +08:00
} ,
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-10-10 15:55:22 +08:00
SubcomponentStates : [ ] * milvuspb . ComponentInfo {
2021-01-26 17:47:38 +08:00
{
2021-11-19 13:57:12 +08:00
NodeID : nodeID ,
2021-06-17 16:47:57 +08:00
Role : typeutil . RootCoordRole ,
2021-01-26 17:47:38 +08:00
StateCode : code ,
ExtraInfo : nil ,
} ,
} ,
2021-01-19 14:44:03 +08:00
} , nil
}
2021-09-23 15:10:00 +08:00
// GetTimeTickChannel get timetick channel name
2023-09-26 09:57:25 +08:00
func ( c * Core ) GetTimeTickChannel ( ctx context . Context , req * internalpb . GetTimeTickChannelRequest ) ( * milvuspb . StringResponse , error ) {
2021-02-26 17:44:24 +08:00
return & milvuspb . StringResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2023-03-13 14:29:53 +08:00
Value : Params . CommonCfg . RootCoordTimeTick . GetValue ( ) ,
2021-02-26 17:44:24 +08:00
} , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// GetStatisticsChannel get statistics channel name
2023-09-26 09:57:25 +08:00
func ( c * Core ) GetStatisticsChannel ( ctx context . Context , req * internalpb . GetStatisticsChannelRequest ) ( * milvuspb . StringResponse , error ) {
2021-02-26 17:44:24 +08:00
return & milvuspb . StringResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2023-03-13 14:29:53 +08:00
Value : Params . CommonCfg . RootCoordStatistics . GetValue ( ) ,
2021-02-26 17:44:24 +08:00
} , nil
2021-01-19 14:44:03 +08:00
}
2023-06-25 17:20:43 +08:00
func ( c * Core ) CreateDatabase ( ctx context . Context , in * milvuspb . CreateDatabaseRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2023-06-25 17:20:43 +08:00
}
method := "CreateDatabase"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "CreateDatabase" )
log . Ctx ( ctx ) . Info ( "received request to create database" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) , zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) )
t := & createDatabaseTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Ctx ( ctx ) . Info ( "failed to enqueue request to create database" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) , zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-10-11 21:01:35 +08:00
return merr . Status ( err ) , nil
2023-06-25 17:20:43 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Ctx ( ctx ) . Info ( "failed to create database" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) ,
zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) , zap . Uint64 ( "ts" , t . GetTs ( ) ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-10-11 21:01:35 +08:00
return merr . Status ( err ) , nil
2023-06-25 17:20:43 +08:00
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
log . Ctx ( ctx ) . Info ( "done to create database" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) ,
zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) , zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2023-06-25 17:20:43 +08:00
}
func ( c * Core ) DropDatabase ( ctx context . Context , in * milvuspb . DropDatabaseRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2023-06-25 17:20:43 +08:00
}
method := "DropDatabase"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "DropDatabase" )
log . Ctx ( ctx ) . Info ( "received request to drop database" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) , zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) )
t := & dropDatabaseTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Ctx ( ctx ) . Info ( "failed to enqueue request to drop database" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) , zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-10-11 21:01:35 +08:00
return merr . Status ( err ) , nil
2023-06-25 17:20:43 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Ctx ( ctx ) . Info ( "failed to drop database" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) ,
zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) , zap . Uint64 ( "ts" , t . GetTs ( ) ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-10-11 21:01:35 +08:00
return merr . Status ( err ) , nil
2023-06-25 17:20:43 +08:00
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2024-04-02 14:27:13 +08:00
metrics . CleanupRootCoordDBMetrics ( in . GetDbName ( ) )
2023-06-25 17:20:43 +08:00
log . Ctx ( ctx ) . Info ( "done to drop database" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) , zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2023-06-25 17:20:43 +08:00
}
func ( c * Core ) ListDatabases ( ctx context . Context , in * milvuspb . ListDatabasesRequest ) ( * milvuspb . ListDatabasesResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
ret := & milvuspb . ListDatabasesResponse { Status : merr . Status ( err ) }
2023-06-25 17:20:43 +08:00
return ret , nil
}
method := "ListDatabases"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "ListDatabases" )
log := log . Ctx ( ctx ) . With ( zap . Int64 ( "msgID" , in . GetBase ( ) . GetMsgID ( ) ) )
log . Info ( "received request to list databases" )
t := & listDatabaseTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Resp : & milvuspb . ListDatabasesResponse { } ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Info ( "failed to enqueue request to list databases" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
return & milvuspb . ListDatabasesResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2023-06-25 17:20:43 +08:00
} , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Info ( "failed to list databases" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
return & milvuspb . ListDatabasesResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2023-06-25 17:20:43 +08:00
} , nil
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
log . Info ( "done to list databases" , zap . Int ( "num of databases" , len ( t . Resp . GetDbNames ( ) ) ) )
return t . Resp , nil
}
2021-09-23 15:10:00 +08:00
// CreateCollection create collection
2021-02-26 17:44:24 +08:00
func ( c * Core ) CreateCollection ( ctx context . Context , in * milvuspb . CreateCollectionRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-01-25 18:33:10 +08:00
}
2021-11-19 12:11:12 +08:00
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateCollection" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "CreateCollection" )
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to create collection" ,
2023-06-25 17:20:43 +08:00
zap . String ( "dbName" , in . GetDbName ( ) ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) ,
zap . String ( "role" , typeutil . RootCoordRole ) )
2022-09-05 13:29:11 +08:00
t := & createCollectionTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to create collection" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to create collection" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "CreateCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "CreateCollection" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "done to create collection" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// DropCollection drop collection
2021-02-26 17:44:24 +08:00
func ( c * Core ) DropCollection ( ctx context . Context , in * milvuspb . DropCollectionRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-01-25 18:33:10 +08:00
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropCollection" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "DropCollection" )
2022-09-05 13:29:11 +08:00
2023-06-25 17:20:43 +08:00
log . Ctx ( ctx ) . Info ( "received request to drop collection" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "dbName" , in . GetDbName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
t := & dropCollectionTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to drop collection" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to drop collection" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "DropCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "DropCollection" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
log . Ctx ( ctx ) . Info ( "done to drop collection" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) ,
2022-09-05 13:29:11 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// HasCollection check collection existence
2021-02-26 17:44:24 +08:00
func ( c * Core ) HasCollection ( ctx context . Context , in * milvuspb . HasCollectionRequest ) ( * milvuspb . BoolResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-01-25 18:33:10 +08:00
return & milvuspb . BoolResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-01-25 18:33:10 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasCollection" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "HasCollection" )
2021-11-19 12:11:12 +08:00
2022-09-24 12:42:51 +08:00
ts := getTravelTs ( in )
2023-07-14 15:56:31 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collectionName" , in . GetCollectionName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , ts ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & hasCollectionTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Rsp : & milvuspb . BoolResponse { } ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to enqueue request to has collection" , zap . Error ( err ) )
2022-10-28 13:25:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasCollection" , metrics . FailLabel ) . Inc ( )
return & milvuspb . BoolResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to has collection" , zap . Error ( err ) )
2022-10-28 13:25:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasCollection" , metrics . FailLabel ) . Inc ( )
return & milvuspb . BoolResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "HasCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "HasCollection" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-24 12:42:51 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2022-09-24 12:42:51 +08:00
}
2023-01-04 16:37:35 +08:00
func ( c * Core ) describeCollection ( ctx context . Context , in * milvuspb . DescribeCollectionRequest , allowUnavailable bool ) ( * model . Collection , error ) {
2022-09-24 12:42:51 +08:00
ts := getTravelTs ( in )
if in . GetCollectionName ( ) != "" {
2023-06-25 17:20:43 +08:00
return c . meta . GetCollectionByName ( ctx , in . GetDbName ( ) , in . GetCollectionName ( ) , ts )
2022-09-05 13:29:11 +08:00
}
2023-06-25 17:20:43 +08:00
return c . meta . GetCollectionByID ( ctx , in . GetDbName ( ) , in . GetCollectionID ( ) , ts , allowUnavailable )
2022-09-24 12:42:51 +08:00
}
2022-09-05 13:29:11 +08:00
2024-03-11 17:19:02 +08:00
func convertModelToDesc ( collInfo * model . Collection , aliases [ ] string , dbName string ) * milvuspb . DescribeCollectionResponse {
resp := & milvuspb . DescribeCollectionResponse {
Status : merr . Success ( ) ,
DbName : dbName ,
}
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
resp . Schema = & schemapb . CollectionSchema {
2023-05-18 09:33:24 +08:00
Name : collInfo . Name ,
Description : collInfo . Description ,
AutoID : collInfo . AutoID ,
Fields : model . MarshalFieldModels ( collInfo . Fields ) ,
EnableDynamicField : collInfo . EnableDynamicField ,
2022-09-24 12:42:51 +08:00
}
resp . CollectionID = collInfo . CollectionID
resp . VirtualChannelNames = collInfo . VirtualChannelNames
resp . PhysicalChannelNames = collInfo . PhysicalChannelNames
if collInfo . ShardsNum == 0 {
collInfo . ShardsNum = int32 ( len ( collInfo . VirtualChannelNames ) )
2021-01-19 14:44:03 +08:00
}
2022-09-24 12:42:51 +08:00
resp . ShardsNum = collInfo . ShardsNum
resp . ConsistencyLevel = collInfo . ConsistencyLevel
2021-11-19 12:11:12 +08:00
2022-09-24 12:42:51 +08:00
resp . CreatedTimestamp = collInfo . CreateTime
createdPhysicalTime , _ := tsoutil . ParseHybridTs ( collInfo . CreateTime )
resp . CreatedUtcTimestamp = uint64 ( createdPhysicalTime )
resp . Aliases = aliases
resp . StartPositions = collInfo . StartPositions
resp . CollectionName = resp . Schema . Name
2022-10-10 20:31:22 +08:00
resp . Properties = collInfo . Properties
2023-06-06 10:24:34 +08:00
resp . NumPartitions = int64 ( len ( collInfo . Partitions ) )
2024-04-10 21:23:20 +08:00
resp . DbId = collInfo . DBID
2022-09-24 12:42:51 +08:00
return resp
2021-01-19 14:44:03 +08:00
}
2023-01-04 16:37:35 +08:00
func ( c * Core ) describeCollectionImpl ( ctx context . Context , in * milvuspb . DescribeCollectionRequest , allowUnavailable bool ) ( * milvuspb . DescribeCollectionResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-01-25 18:33:10 +08:00
return & milvuspb . DescribeCollectionResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-01-25 18:33:10 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeCollection" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "DescribeCollection" )
2021-11-19 12:11:12 +08:00
2022-09-24 12:42:51 +08:00
ts := getTravelTs ( in )
2023-07-14 15:56:31 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collectionName" , in . GetCollectionName ( ) ) ,
2023-06-25 17:20:43 +08:00
zap . String ( "dbName" , in . GetDbName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Int64 ( "id" , in . GetCollectionID ( ) ) ,
2023-01-04 16:37:35 +08:00
zap . Uint64 ( "ts" , ts ) ,
zap . Bool ( "allowUnavailable" , allowUnavailable ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & describeCollectionTask {
2023-01-04 16:37:35 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2023-10-11 21:01:35 +08:00
Rsp : & milvuspb . DescribeCollectionResponse { Status : merr . Success ( ) } ,
2023-01-04 16:37:35 +08:00
allowUnavailable : allowUnavailable ,
2022-10-28 13:25:34 +08:00
}
2022-09-27 19:18:54 +08:00
2022-10-28 13:25:34 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to enqueue request to describe collection" , zap . Error ( err ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeCollection" , metrics . FailLabel ) . Inc ( )
2021-01-19 14:44:03 +08:00
return & milvuspb . DescribeCollectionResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2021-01-19 14:44:03 +08:00
} , nil
}
2021-11-19 12:11:12 +08:00
2022-10-28 13:25:34 +08:00
if err := t . WaitToFinish ( ) ; err != nil {
2023-09-04 09:57:09 +08:00
log . Warn ( "failed to describe collection" , zap . Error ( err ) )
2022-10-28 13:25:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeCollection" , metrics . FailLabel ) . Inc ( )
return & milvuspb . DescribeCollectionResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
}
2022-09-24 12:42:51 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "DescribeCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "DescribeCollection" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
2023-01-04 16:37:35 +08:00
// DescribeCollection return collection info
func ( c * Core ) DescribeCollection ( ctx context . Context , in * milvuspb . DescribeCollectionRequest ) ( * milvuspb . DescribeCollectionResponse , error ) {
return c . describeCollectionImpl ( ctx , in , false )
}
// DescribeCollectionInternal same to DescribeCollection, but will return unavailable collections and
// only used in internal RPC.
// When query cluster tried to do recovery, it'll be healthy until all collections' targets were recovered,
// so during this time, releasing request generated by rootcoord's recovery won't succeed. So in theory, rootcoord goes
// to be healthy, querycoord recovers all collections' targets, and then querycoord serves the releasing request sent
// by rootcoord, eventually, the dropping collections will be released.
func ( c * Core ) DescribeCollectionInternal ( ctx context . Context , in * milvuspb . DescribeCollectionRequest ) ( * milvuspb . DescribeCollectionResponse , error ) {
return c . describeCollectionImpl ( ctx , in , true )
}
2021-09-23 15:10:00 +08:00
// ShowCollections list all collection names
2021-03-12 14:22:09 +08:00
func ( c * Core ) ShowCollections ( ctx context . Context , in * milvuspb . ShowCollectionsRequest ) ( * milvuspb . ShowCollectionsResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-03-12 14:22:09 +08:00
return & milvuspb . ShowCollectionsResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-01-25 18:33:10 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowCollections" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "ShowCollections" )
2021-11-19 12:11:12 +08:00
2022-09-24 12:42:51 +08:00
ts := getTravelTs ( in )
2022-11-14 15:29:06 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "dbname" , in . GetDbName ( ) ) ,
zap . Uint64 ( "ts" , ts ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & showCollectionTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Rsp : & milvuspb . ShowCollectionsResponse { } ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to enqueue request to show collections" , zap . Error ( err ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowCollections" , metrics . FailLabel ) . Inc ( )
2022-10-28 13:25:34 +08:00
return & milvuspb . ShowCollectionsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
2022-09-05 13:29:11 +08:00
}
2022-10-28 13:25:34 +08:00
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to show collections" , zap . Error ( err ) )
2022-10-28 13:25:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowCollections" , metrics . FailLabel ) . Inc ( )
return & milvuspb . ShowCollectionsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowCollections" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "ShowCollections" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "ShowCollections" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
2022-10-10 20:31:22 +08:00
func ( c * Core ) AlterCollection ( ctx context . Context , in * milvuspb . AlterCollectionRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2022-10-10 20:31:22 +08:00
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterCollection" , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "AlterCollection" )
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to alter collection" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
2024-03-18 15:19:06 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) ,
zap . Any ( "props" , in . Properties ) )
2022-10-10 20:31:22 +08:00
t := & alterCollectionTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2022-10-10 20:31:22 +08:00
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-30 14:11:15 +08:00
log . Warn ( "failed to enqueue request to alter collection" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-10-10 20:31:22 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "name" , in . GetCollectionName ( ) ) )
2022-10-10 20:31:22 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-10-10 20:31:22 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-30 14:11:15 +08:00
log . Warn ( "failed to alter collection" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-10-10 20:31:22 +08:00
zap . Error ( err ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-10-10 20:31:22 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-10-10 20:31:22 +08:00
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "AlterCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "AlterCollection" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-10-10 20:31:22 +08:00
2022-11-14 15:29:06 +08:00
log . Info ( "done to alter collection" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) ,
2022-10-10 20:31:22 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-10-10 20:31:22 +08:00
}
2024-04-25 10:53:25 +08:00
func ( c * Core ) AlterDatabase ( ctx context . Context , in * rootcoordpb . AlterDatabaseRequest ) ( * commonpb . Status , error ) {
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
}
method := "AlterDatabase"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
log . Ctx ( ctx ) . Info ( "received request to alter database" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "name" , in . GetDbName ( ) ) ,
zap . Any ( "props" , in . Properties ) )
t := & alterDatabaseTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Warn ( "failed to enqueue request to alter database" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "name" , in . GetDbName ( ) ) ,
zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
return merr . Status ( err ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Warn ( "failed to alter database" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) ,
zap . String ( "name" , in . GetDbName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
return merr . Status ( err ) , nil
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( method ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
log . Ctx ( ctx ) . Info ( "done to alter database" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "name" , in . GetDbName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
return merr . Success ( ) , nil
}
2021-09-23 15:10:00 +08:00
// CreatePartition create partition
2021-02-26 17:44:24 +08:00
func ( c * Core ) CreatePartition ( ctx context . Context , in * milvuspb . CreatePartitionRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-01-25 18:33:10 +08:00
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreatePartition" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "CreatePartition" )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to create partition" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) )
2022-09-05 13:29:11 +08:00
t := & createPartitionTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to create partition" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreatePartition" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to create partition" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreatePartition" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreatePartition" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "CreatePartition" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "CreatePartition" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "done to create partition" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// DropPartition drop partition
2021-02-26 17:44:24 +08:00
func ( c * Core ) DropPartition ( ctx context . Context , in * milvuspb . DropPartitionRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-01-25 18:33:10 +08:00
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropPartition" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "DropPartition" )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to drop partition" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) )
2022-09-05 13:29:11 +08:00
t := & dropPartitionTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to drop partition" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropPartition" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to drop partition" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropPartition" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropPartition" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "DropPartition" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "DropPartition" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "done to drop partition" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// HasPartition check partition existence
2021-02-26 17:44:24 +08:00
func ( c * Core ) HasPartition ( ctx context . Context , in * milvuspb . HasPartitionRequest ) ( * milvuspb . BoolResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-01-25 18:33:10 +08:00
return & milvuspb . BoolResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-01-25 18:33:10 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasPartition" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "HasPartition" )
2021-11-19 12:11:12 +08:00
2022-09-24 12:42:51 +08:00
// TODO(longjiquan): why HasPartitionRequest doesn't contain Timestamp but other requests do.
ts := typeutil . MaxTimestamp
2022-11-14 15:29:06 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . String ( "partition" , in . GetPartitionName ( ) ) ,
zap . Uint64 ( "ts" , ts ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & hasPartitionTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Rsp : & milvuspb . BoolResponse { } ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to enqueue request to has partition" , zap . Error ( err ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasPartition" , metrics . FailLabel ) . Inc ( )
2022-10-28 13:25:34 +08:00
return & milvuspb . BoolResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
2022-09-05 13:29:11 +08:00
}
2022-10-28 13:25:34 +08:00
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to has partition" , zap . Error ( err ) )
2022-10-28 13:25:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasPartition" , metrics . FailLabel ) . Inc ( )
return & milvuspb . BoolResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-10-28 13:25:34 +08:00
} , nil
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasPartition" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "HasPartition" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "HasPartition" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
2023-01-04 16:37:35 +08:00
func ( c * Core ) showPartitionsImpl ( ctx context . Context , in * milvuspb . ShowPartitionsRequest , allowUnavailable bool ) ( * milvuspb . ShowPartitionsResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-03-12 14:22:09 +08:00
return & milvuspb . ShowPartitionsResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-01-25 18:33:10 +08:00
} , nil
}
2021-11-19 12:11:12 +08:00
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowPartitions" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "ShowPartitions" )
2022-09-05 13:29:11 +08:00
2023-04-17 11:00:30 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . Int64 ( "collection_id" , in . GetCollectionID ( ) ) ,
zap . Strings ( "partitions" , in . GetPartitionNames ( ) ) ,
zap . Bool ( "allowUnavailable" , allowUnavailable ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & showPartitionTask {
2023-01-04 16:37:35 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Rsp : & milvuspb . ShowPartitionsResponse { } ,
allowUnavailable : allowUnavailable ,
2021-01-19 14:44:03 +08:00
}
2021-11-19 12:11:12 +08:00
2022-10-28 13:25:34 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to enqueue request to show partitions" , zap . Error ( err ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowPartitions" , metrics . FailLabel ) . Inc ( )
2022-10-28 13:25:34 +08:00
return & milvuspb . ShowPartitionsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2023-01-04 16:37:35 +08:00
// Status: common.StatusFromError(err),
2022-10-28 13:25:34 +08:00
} , nil
2022-09-05 13:29:11 +08:00
}
2022-09-24 12:42:51 +08:00
2022-10-28 13:25:34 +08:00
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Info ( "failed to show partitions" , zap . Error ( err ) )
2022-10-28 13:25:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowPartitions" , metrics . FailLabel ) . Inc ( )
return & milvuspb . ShowPartitionsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2023-01-04 16:37:35 +08:00
// Status: common.StatusFromError(err),
2022-10-28 13:25:34 +08:00
} , nil
2022-09-24 12:42:51 +08:00
}
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowPartitions" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "ShowPartitions" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "ShowPartitions" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
2023-01-04 16:37:35 +08:00
// ShowPartitions list all partition names
func ( c * Core ) ShowPartitions ( ctx context . Context , in * milvuspb . ShowPartitionsRequest ) ( * milvuspb . ShowPartitionsResponse , error ) {
return c . showPartitionsImpl ( ctx , in , false )
}
// ShowPartitionsInternal same to ShowPartitions, only used in internal RPC.
func ( c * Core ) ShowPartitionsInternal ( ctx context . Context , in * milvuspb . ShowPartitionsRequest ) ( * milvuspb . ShowPartitionsResponse , error ) {
return c . showPartitionsImpl ( ctx , in , true )
}
2021-09-23 15:10:00 +08:00
// ShowSegments list all segments
2021-03-12 14:22:09 +08:00
func ( c * Core ) ShowSegments ( ctx context . Context , in * milvuspb . ShowSegmentsRequest ) ( * milvuspb . ShowSegmentsResponse , error ) {
2022-09-05 13:29:11 +08:00
// ShowSegments Only used in GetPersistentSegmentInfo, it's already deprecated for a long time.
// Though we continue to keep current logic, it's not right enough since RootCoord only contains indexed segments.
2023-10-11 21:01:35 +08:00
return & milvuspb . ShowSegmentsResponse { Status : merr . Success ( ) } , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// AllocTimestamp alloc timestamp
2021-06-22 16:14:09 +08:00
func ( c * Core ) AllocTimestamp ( ctx context . Context , in * rootcoordpb . AllocTimestampRequest ) ( * rootcoordpb . AllocTimestampResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocTimestampResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-05-26 20:14:30 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
ts , err := c . tsoAllocator . GenerateTSO ( in . GetCount ( ) )
2021-01-19 14:44:03 +08:00
if err != nil {
2022-09-24 12:42:51 +08:00
log . Ctx ( ctx ) . Error ( "failed to allocate timestamp" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-11-14 15:29:06 +08:00
zap . Error ( err ) )
2022-09-05 13:29:11 +08:00
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocTimestampResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2021-01-19 14:44:03 +08:00
} , nil
}
2021-07-14 17:11:54 +08:00
2022-09-05 13:29:11 +08:00
// return first available timestamp
ts = ts - uint64 ( in . GetCount ( ) ) + 1
2022-04-27 23:03:47 +08:00
metrics . RootCoordTimestamp . Set ( float64 ( ts ) )
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocTimestampResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2021-01-19 14:44:03 +08:00
Timestamp : ts ,
2022-09-05 13:29:11 +08:00
Count : in . GetCount ( ) ,
2021-01-19 14:44:03 +08:00
} , nil
}
2021-09-23 15:10:00 +08:00
// AllocID alloc ids
2021-06-22 16:14:09 +08:00
func ( c * Core ) AllocID ( ctx context . Context , in * rootcoordpb . AllocIDRequest ) ( * rootcoordpb . AllocIDResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocIDResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-05-26 20:14:30 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
start , _ , err := c . idAllocator . Alloc ( in . Count )
2021-01-19 14:44:03 +08:00
if err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to allocate id" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) )
2022-09-05 13:29:11 +08:00
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocIDResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2021-11-19 12:11:12 +08:00
Count : in . Count ,
2021-01-19 14:44:03 +08:00
} , nil
}
2022-09-05 13:29:11 +08:00
2022-03-02 21:11:57 +08:00
metrics . RootCoordIDAllocCounter . Add ( float64 ( in . Count ) )
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocIDResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2021-11-19 12:11:12 +08:00
ID : start ,
Count : in . Count ,
2021-01-19 14:44:03 +08:00
} , nil
}
2021-05-21 16:08:12 +08:00
// UpdateChannelTimeTick used to handle ChannelTimeTickMsg
func ( c * Core ) UpdateChannelTimeTick ( ctx context . Context , in * internalpb . ChannelTimeTickMsg ) ( * commonpb . Status , error ) {
2022-11-14 15:29:06 +08:00
log := log . Ctx ( ctx )
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
log . Warn ( "failed to updateTimeTick because rootcoord is not healthy" , zap . Error ( err ) )
return merr . Status ( err ) , nil
2021-05-21 16:08:12 +08:00
}
if in . Base . MsgType != commonpb . MsgType_TimeTick {
2022-01-15 18:53:34 +08:00
log . Warn ( "failed to updateTimeTick because base messasge is not timetick, state" , zap . Any ( "base message type" , in . Base . MsgType ) )
2023-03-13 14:29:53 +08:00
return merr . Status ( merr . WrapErrParameterInvalid ( commonpb . MsgType_TimeTick . String ( ) , in . Base . MsgType . String ( ) , "invalid message type" ) ) , nil
2021-05-21 16:08:12 +08:00
}
2021-11-25 10:07:15 +08:00
err := c . chanTimeTick . updateTimeTick ( in , "gRPC" )
2021-05-21 16:08:12 +08:00
if err != nil {
2022-11-14 15:29:06 +08:00
log . Warn ( "failed to updateTimeTick" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-05-21 16:08:12 +08:00
}
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-05-21 16:08:12 +08:00
}
2021-06-17 17:45:56 +08:00
2022-05-19 10:13:56 +08:00
// InvalidateCollectionMetaCache notifies RootCoord to release the collection cache in Proxies.
func ( c * Core ) InvalidateCollectionMetaCache ( ctx context . Context , in * proxypb . InvalidateCollMetaCacheRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2022-05-19 10:13:56 +08:00
}
err := c . proxyClientManager . InvalidateCollectionMetaCache ( ctx , in )
if err != nil {
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-05-19 10:13:56 +08:00
}
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-06-17 17:45:56 +08:00
}
2021-07-01 14:58:17 +08:00
2022-10-21 15:57:28 +08:00
// ShowConfigurations returns the configurations of RootCoord matching req.Pattern
2022-08-12 13:20:39 +08:00
func ( c * Core ) ShowConfigurations ( ctx context . Context , req * internalpb . ShowConfigurationsRequest ) ( * internalpb . ShowConfigurationsResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2022-08-12 13:20:39 +08:00
return & internalpb . ShowConfigurationsResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2022-08-12 13:20:39 +08:00
Configuations : nil ,
} , nil
}
2022-12-09 14:31:21 +08:00
configList := make ( [ ] * commonpb . KeyValuePair , 0 )
2023-01-13 15:31:41 +08:00
for key , value := range Params . GetComponentConfigurations ( "rootcoord" , req . Pattern ) {
2022-12-09 14:31:21 +08:00
configList = append ( configList ,
& commonpb . KeyValuePair {
Key : key ,
Value : value ,
} )
}
return & internalpb . ShowConfigurationsResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-12-09 14:31:21 +08:00
Configuations : configList ,
} , nil
2022-08-12 13:20:39 +08:00
}
2021-09-23 15:10:00 +08:00
// GetMetrics get metrics
2021-11-22 16:01:14 +08:00
func ( c * Core ) GetMetrics ( ctx context . Context , in * milvuspb . GetMetricsRequest ) ( * milvuspb . GetMetricsResponse , error ) {
2023-10-16 15:04:09 +08:00
if err := merr . CheckHealthyStandby ( c . GetStateCode ( ) ) ; err != nil {
2021-08-31 11:45:59 +08:00
return & milvuspb . GetMetricsResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2021-08-31 11:45:59 +08:00
Response : "" ,
} , nil
}
2021-11-22 16:01:14 +08:00
metricType , err := metricsinfo . ParseMetricType ( in . Request )
2021-08-31 11:45:59 +08:00
if err != nil {
2022-07-21 20:52:27 +08:00
log . Warn ( "ParseMetricType failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2023-01-04 17:39:35 +08:00
zap . Int64 ( "nodeID" , c . session . ServerID ) , zap . String ( "req" , in . Request ) , zap . Error ( err ) )
2021-08-31 11:45:59 +08:00
return & milvuspb . GetMetricsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2021-08-31 11:45:59 +08:00
Response : "" ,
} , nil
}
if metricType == metricsinfo . SystemInfoMetrics {
2023-01-04 17:39:35 +08:00
metrics , err := c . metricsCacheManager . GetSystemInfoMetrics ( )
if err != nil {
metrics , err = c . getSystemInfoMetrics ( ctx , in )
2021-09-03 17:15:26 +08:00
}
2021-11-22 16:01:14 +08:00
if err != nil {
2022-11-14 15:29:06 +08:00
log . Warn ( "GetSystemInfoMetrics failed" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
2023-01-04 17:39:35 +08:00
zap . String ( "metricType" , metricType ) ,
2022-11-14 15:29:06 +08:00
zap . Error ( err ) )
2022-07-21 20:52:27 +08:00
return & milvuspb . GetMetricsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( err ) ,
2022-07-21 20:52:27 +08:00
Response : "" ,
} , nil
2021-11-22 16:01:14 +08:00
}
2021-08-31 11:45:59 +08:00
2023-01-04 17:39:35 +08:00
c . metricsCacheManager . UpdateSystemInfoMetrics ( metrics )
return metrics , err
2021-08-31 11:45:59 +08:00
}
2023-01-04 17:39:35 +08:00
log . RatedWarn ( 60 , "GetMetrics failed, metric type not implemented" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "metricType" , metricType ) )
2021-08-31 11:45:59 +08:00
return & milvuspb . GetMetricsResponse {
2023-03-13 14:29:53 +08:00
Status : merr . Status ( merr . WrapErrMetricNotFound ( metricType ) ) ,
2021-08-31 11:45:59 +08:00
Response : "" ,
} , nil
}
2021-09-18 11:13:51 +08:00
2021-09-23 15:10:00 +08:00
// CreateAlias create collection alias
2021-09-18 11:13:51 +08:00
func ( c * Core ) CreateAlias ( ctx context . Context , in * milvuspb . CreateAliasRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateAlias" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "CreateAlias" )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to create alias" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
t := & createAliasTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to create alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateAlias" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to create alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateAlias" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-09-18 11:13:51 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "CreateAlias" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "CreateAlias" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "CreateAlias" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "done to create alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-09-18 11:13:51 +08:00
}
2021-09-23 15:10:00 +08:00
// DropAlias drop collection alias
2021-09-18 11:13:51 +08:00
func ( c * Core ) DropAlias ( ctx context . Context , in * milvuspb . DropAliasRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropAlias" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "DropAlias" )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to drop alias" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "alias" , in . GetAlias ( ) ) )
2022-09-05 13:29:11 +08:00
t := & dropAliasTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to drop alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "alias" , in . GetAlias ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropAlias" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to drop alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
zap . String ( "alias" , in . GetAlias ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropAlias" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-09-18 11:13:51 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropAlias" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "DropAlias" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "DropAlias" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "done to drop alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . String ( "alias" , in . GetAlias ( ) ) ,
2022-11-14 15:29:06 +08:00
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-09-18 11:13:51 +08:00
}
2021-09-23 15:10:00 +08:00
// AlterAlias alter collection alias
2021-09-18 11:13:51 +08:00
func ( c * Core ) AlterAlias ( ctx context . Context , in * milvuspb . AlterAliasRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DropAlias" , metrics . TotalLabel ) . Inc ( )
2022-03-02 21:11:57 +08:00
tr := timerecord . NewTimeRecorder ( "AlterAlias" )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Info ( "received request to alter alias" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
t := & alterAliasTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
2021-09-18 11:13:51 +08:00
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to enqueue request to alter alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterAlias" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2022-09-05 13:29:11 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
2023-04-17 11:00:30 +08:00
log . Ctx ( ctx ) . Info ( "failed to alter alias" ,
2022-11-14 15:29:06 +08:00
zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-09-05 13:29:11 +08:00
zap . Error ( err ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2022-09-05 13:29:11 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterAlias" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2021-09-18 11:13:51 +08:00
}
2021-11-19 12:11:12 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterAlias" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "AlterAlias" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-06-26 17:52:44 +08:00
metrics . RootCoordDDLReqLatencyInQueue . WithLabelValues ( "AlterAlias" ) . Observe ( float64 ( t . queueDur . Milliseconds ( ) ) )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Info ( "done to alter alias" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2021-09-18 11:13:51 +08:00
}
2022-03-11 17:13:59 +08:00
2024-01-11 19:12:51 +08:00
// DescribeAlias describe collection alias
func ( c * Core ) DescribeAlias ( ctx context . Context , in * milvuspb . DescribeAliasRequest ) ( * milvuspb . DescribeAliasResponse , error ) {
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & milvuspb . DescribeAliasResponse {
Status : merr . Status ( err ) ,
} , nil
}
log := log . Ctx ( ctx ) . With (
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "db" , in . GetDbName ( ) ) ,
zap . String ( "alias" , in . GetAlias ( ) ) )
method := "DescribeAlias"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "DescribeAlias" )
log . Info ( "received request to describe alias" )
if in . GetAlias ( ) == "" {
return & milvuspb . DescribeAliasResponse {
Status : merr . Status ( merr . WrapErrParameterMissing ( "alias" , "no input alias" ) ) ,
} , nil
}
collectionName , err := c . meta . DescribeAlias ( ctx , in . GetDbName ( ) , in . GetAlias ( ) , 0 )
if err != nil {
log . Warn ( "fail to DescribeAlias" , zap . Error ( err ) )
return & milvuspb . DescribeAliasResponse {
Status : merr . Status ( err ) ,
} , nil
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
log . Info ( "done to describe alias" )
return & milvuspb . DescribeAliasResponse {
Status : merr . Status ( nil ) ,
DbName : in . GetDbName ( ) ,
Alias : in . GetAlias ( ) ,
Collection : collectionName ,
} , nil
}
// ListAliases list aliases
func ( c * Core ) ListAliases ( ctx context . Context , in * milvuspb . ListAliasesRequest ) ( * milvuspb . ListAliasesResponse , error ) {
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & milvuspb . ListAliasesResponse {
Status : merr . Status ( err ) ,
} , nil
}
method := "ListAliases"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
log := log . Ctx ( ctx ) . With (
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "db" , in . GetDbName ( ) ) ,
zap . String ( "collectionName" , in . GetCollectionName ( ) ) )
log . Info ( "received request to list aliases" )
aliases , err := c . meta . ListAliases ( ctx , in . GetDbName ( ) , in . GetCollectionName ( ) , 0 )
if err != nil {
log . Warn ( "fail to ListAliases" , zap . Error ( err ) )
return & milvuspb . ListAliasesResponse {
Status : merr . Status ( err ) ,
} , nil
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
log . Info ( "done to list aliases" )
return & milvuspb . ListAliasesResponse {
2024-02-12 08:30:55 +08:00
Status : merr . Status ( nil ) ,
DbName : in . GetDbName ( ) ,
CollectionName : in . GetCollectionName ( ) ,
Aliases : aliases ,
2024-01-11 19:12:51 +08:00
} , nil
}
2022-04-11 19:49:34 +08:00
// ExpireCredCache will call invalidate credential cache
func ( c * Core ) ExpireCredCache ( ctx context . Context , username string ) error {
req := proxypb . InvalidateCredCacheRequest {
2022-10-21 15:57:28 +08:00
Base : commonpbutil . NewMsgBase (
commonpbutil . WithSourceID ( c . session . ServerID ) ,
) ,
2022-04-11 19:49:34 +08:00
Username : username ,
}
return c . proxyClientManager . InvalidateCredentialCache ( ctx , & req )
}
// UpdateCredCache will call update credential cache
func ( c * Core ) UpdateCredCache ( ctx context . Context , credInfo * internalpb . CredentialInfo ) error {
req := proxypb . UpdateCredCacheRequest {
2022-10-21 15:57:28 +08:00
Base : commonpbutil . NewMsgBase (
commonpbutil . WithSourceID ( c . session . ServerID ) ,
) ,
2022-04-11 19:49:34 +08:00
Username : credInfo . Username ,
2022-06-29 20:02:18 +08:00
Password : credInfo . Sha256Password ,
2022-04-11 19:49:34 +08:00
}
return c . proxyClientManager . UpdateCredentialCache ( ctx , & req )
}
// CreateCredential create new user and password
2022-10-21 15:57:28 +08:00
// 1. decode ciphertext password to raw password
// 2. encrypt raw password
// 3. save in to etcd
2022-04-11 19:49:34 +08:00
func ( c * Core ) CreateCredential ( ctx context . Context , credInfo * internalpb . CredentialInfo ) ( * commonpb . Status , error ) {
2022-04-27 23:03:47 +08:00
method := "CreateCredential"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . String ( "username" , credInfo . Username ) )
ctxLog . Debug ( method )
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2023-08-04 18:37:08 +08:00
}
2022-04-11 19:49:34 +08:00
2022-06-29 20:02:18 +08:00
// insert to db
2022-09-05 13:29:11 +08:00
err := c . meta . AddCredential ( credInfo )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "CreateCredential save credential failed" , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-08-04 18:37:08 +08:00
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_CreateCredentialFailure ) , nil
2022-04-11 19:49:34 +08:00
}
2022-06-29 20:02:18 +08:00
// update proxy's local cache
err = c . UpdateCredCache ( ctx , credInfo )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "CreateCredential add cache failed" , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2022-04-11 19:49:34 +08:00
}
log . Debug ( "CreateCredential success" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , credInfo . Username ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2022-04-11 19:49:34 +08:00
metrics . RootCoordNumOfCredentials . Inc ( )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-04-11 19:49:34 +08:00
}
// GetCredential get credential by username
func ( c * Core ) GetCredential ( ctx context . Context , in * rootcoordpb . GetCredentialRequest ) ( * rootcoordpb . GetCredentialResponse , error ) {
2022-04-27 23:03:47 +08:00
method := "GetCredential"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . String ( "username" , in . Username ) )
ctxLog . Debug ( method )
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & rootcoordpb . GetCredentialResponse { Status : merr . Status ( err ) } , nil
2023-08-04 18:37:08 +08:00
}
2022-04-11 19:49:34 +08:00
2022-09-05 13:29:11 +08:00
credInfo , err := c . meta . GetCredential ( in . Username )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "GetCredential query credential failed" , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2022-04-11 19:49:34 +08:00
return & rootcoordpb . GetCredentialResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( err , commonpb . ErrorCode_GetCredentialFailure ) ,
} , nil
2022-04-11 19:49:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( "GetCredential success" )
2022-04-11 19:49:34 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2022-04-11 19:49:34 +08:00
return & rootcoordpb . GetCredentialResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-04-11 19:49:34 +08:00
Username : credInfo . Username ,
Password : credInfo . EncryptedPassword ,
} , nil
}
// UpdateCredential update password for a user
func ( c * Core ) UpdateCredential ( ctx context . Context , credInfo * internalpb . CredentialInfo ) ( * commonpb . Status , error ) {
2022-04-27 23:03:47 +08:00
method := "UpdateCredential"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . String ( "username" , credInfo . Username ) )
ctxLog . Debug ( method )
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2023-08-04 18:37:08 +08:00
}
2022-06-29 20:02:18 +08:00
// update data on storage
2022-09-05 13:29:11 +08:00
err := c . meta . AlterCredential ( credInfo )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "UpdateCredential save credential failed" , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-08-04 18:37:08 +08:00
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_UpdateCredentialFailure ) , nil
2022-04-11 19:49:34 +08:00
}
2022-06-29 20:02:18 +08:00
// update proxy's local cache
err = c . UpdateCredCache ( ctx , credInfo )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "UpdateCredential update cache failed" , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-08-04 18:37:08 +08:00
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_UpdateCredentialFailure ) , nil
2022-04-11 19:49:34 +08:00
}
2023-08-04 18:37:08 +08:00
log . Debug ( "UpdateCredential success" )
2022-04-11 19:49:34 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-04-11 19:49:34 +08:00
}
// DeleteCredential delete a user
func ( c * Core ) DeleteCredential ( ctx context . Context , in * milvuspb . DeleteCredentialRequest ) ( * commonpb . Status , error ) {
2022-04-27 23:03:47 +08:00
method := "DeleteCredential"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . String ( "username" , in . Username ) )
ctxLog . Debug ( method )
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2023-08-04 18:37:08 +08:00
}
var status * commonpb . Status
defer func ( ) {
if status . Code != 0 {
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
}
} ( )
2022-04-11 19:49:34 +08:00
2023-08-04 18:37:08 +08:00
redoTask := newBaseRedoTask ( c . stepExecutor )
redoTask . AddSyncStep ( NewSimpleStep ( "delete credential meta data" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . meta . DeleteCredential ( in . Username )
if err != nil {
ctxLog . Warn ( "delete credential meta data failed" , zap . Error ( err ) )
}
return nil , err
} ) )
redoTask . AddAsyncStep ( NewSimpleStep ( "delete credential cache" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . ExpireCredCache ( ctx , in . Username )
if err != nil {
ctxLog . Warn ( "delete credential cache failed" , zap . Error ( err ) )
}
return nil , err
} ) )
redoTask . AddAsyncStep ( NewSimpleStep ( "delete user role cache for the user" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . proxyClientManager . RefreshPolicyInfoCache ( ctx , & proxypb . RefreshPolicyInfoCacheRequest {
OpType : int32 ( typeutil . CacheDeleteUser ) ,
OpKey : in . Username ,
} )
if err != nil {
ctxLog . Warn ( "delete user role cache failed for the user" , zap . Error ( err ) )
}
return nil , err
} ) )
2023-03-13 14:29:53 +08:00
2023-08-04 18:37:08 +08:00
err := redoTask . Execute ( ctx )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
errMsg := "fail to execute task when deleting the user"
ctxLog . Warn ( errMsg , zap . Error ( err ) )
status = merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_DeleteCredentialFailure )
2023-03-13 14:29:53 +08:00
return status , nil
2022-04-11 19:49:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( "DeleteCredential success" )
2022-04-11 19:49:34 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2022-04-11 19:49:34 +08:00
metrics . RootCoordNumOfCredentials . Dec ( )
2023-10-11 21:01:35 +08:00
status = merr . Success ( )
2023-08-04 18:37:08 +08:00
return status , nil
2022-04-11 19:49:34 +08:00
}
// ListCredUsers list all usernames
func ( c * Core ) ListCredUsers ( ctx context . Context , in * milvuspb . ListCredUsersRequest ) ( * milvuspb . ListCredUsersResponse , error ) {
2022-04-27 23:03:47 +08:00
method := "ListCredUsers"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) )
ctxLog . Debug ( method )
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & milvuspb . ListCredUsersResponse { Status : merr . Status ( err ) } , nil
2023-08-04 18:37:08 +08:00
}
2022-04-11 19:49:34 +08:00
2022-09-05 13:29:11 +08:00
credInfo , err := c . meta . ListCredentialUsernames ( )
2022-04-11 19:49:34 +08:00
if err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "ListCredUsers query usernames failed" , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
status := merr . Status ( err )
return & milvuspb . ListCredUsersResponse { Status : status } , nil
2022-04-11 19:49:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( "ListCredUsers success" )
2022-04-11 19:49:34 +08:00
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2022-04-11 19:49:34 +08:00
return & milvuspb . ListCredUsersResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-04-11 19:49:34 +08:00
Usernames : credInfo . Usernames ,
} , nil
}
2022-08-04 11:04:34 +08:00
// CreateRole create role
// - check the node health
// - check if the role is existed
// - check if the role num has reached the limit
2022-09-05 13:29:11 +08:00
// - create the role by the meta api
2022-08-04 11:04:34 +08:00
func ( c * Core ) CreateRole ( ctx context . Context , in * milvuspb . CreateRoleRequest ) ( * commonpb . Status , error ) {
method := "CreateRole"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method + " begin" )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2022-08-04 11:04:34 +08:00
}
entity := in . Entity
2022-09-05 13:29:11 +08:00
err := c . meta . CreateRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : entity . Name } )
2022-08-04 11:04:34 +08:00
if err != nil {
2022-09-02 21:12:59 +08:00
errMsg := "fail to create role"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_CreateRoleFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
metrics . RootCoordNumOfRoles . Inc ( )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-08-04 11:04:34 +08:00
}
// DropRole drop role
// - check the node health
// - check if the role name is existed
// - check if the role has some grant info
// - get all role mapping of this role
// - drop these role mappings
2022-09-05 13:29:11 +08:00
// - drop the role by the meta api
2022-08-04 11:04:34 +08:00
func ( c * Core ) DropRole ( ctx context . Context , in * milvuspb . DropRoleRequest ) ( * commonpb . Status , error ) {
method := "DropRole"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . String ( "role_name" , in . RoleName ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2022-08-04 11:04:34 +08:00
}
2023-12-18 14:28:41 +08:00
for util . IsBuiltinRole ( in . GetRoleName ( ) ) {
err := merr . WrapErrPrivilegeNotPermitted ( "the role[%s] is a builtin role, which can't be dropped" , in . GetRoleName ( ) )
return merr . Status ( err ) , nil
}
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . RoleName } , false ) ; err != nil {
2023-01-12 13:55:42 +08:00
errMsg := "not found the role, maybe the role isn't existed or internal system error"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_DropRoleFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
grantEntities , err := c . meta . SelectGrant ( util . DefaultTenant , & milvuspb . GrantEntity {
2022-08-04 11:04:34 +08:00
Role : & milvuspb . RoleEntity { Name : in . RoleName } ,
} )
if len ( grantEntities ) != 0 {
errMsg := "fail to drop the role that it has privileges. Use REVOKE API to revoke privileges"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_DropRoleFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
redoTask := newBaseRedoTask ( c . stepExecutor )
redoTask . AddSyncStep ( NewSimpleStep ( "drop role meta data" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . meta . DropRole ( util . DefaultTenant , in . RoleName )
if err != nil {
ctxLog . Warn ( "drop role mata data failed" , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
return nil , err
} ) )
redoTask . AddAsyncStep ( NewSimpleStep ( "drop role cache" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . proxyClientManager . RefreshPolicyInfoCache ( ctx , & proxypb . RefreshPolicyInfoCacheRequest {
OpType : int32 ( typeutil . CacheDropRole ) ,
OpKey : in . RoleName ,
} )
if err != nil {
ctxLog . Warn ( "delete user role cache failed for the role" , zap . Error ( err ) )
}
return nil , err
} ) )
err = redoTask . Execute ( ctx )
if err != nil {
errMsg := "fail to execute task when dropping the role"
ctxLog . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_DropRoleFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" , zap . String ( "role_name" , in . RoleName ) )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
metrics . RootCoordNumOfRoles . Dec ( )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-08-04 11:04:34 +08:00
}
// OperateUserRole operate the relationship between a user and a role
// - check the node health
// - check if the role is valid
// - check if the user is valid
2022-09-05 13:29:11 +08:00
// - operate the user-role by the meta api
2022-08-04 11:04:34 +08:00
// - update the policy cache
func ( c * Core ) OperateUserRole ( ctx context . Context , in * milvuspb . OperateUserRoleRequest ) ( * commonpb . Status , error ) {
method := "OperateUserRole-" + in . Type . String ( )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . RoleName } , false ) ; err != nil {
2023-01-12 13:55:42 +08:00
errMsg := "not found the role, maybe the role isn't existed or internal system error"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_OperateUserRoleFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-05-16 10:15:22 +08:00
if in . Type != milvuspb . OperateUserRoleType_RemoveUserFromRole {
if _ , err := c . meta . SelectUser ( util . DefaultTenant , & milvuspb . UserEntity { Name : in . Username } , false ) ; err != nil {
errMsg := "not found the user, maybe the user isn't existed or internal system error"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_OperateUserRoleFailure ) , nil
2023-05-16 10:15:22 +08:00
}
2022-08-04 11:04:34 +08:00
}
2023-05-16 10:15:22 +08:00
2023-08-04 18:37:08 +08:00
redoTask := newBaseRedoTask ( c . stepExecutor )
redoTask . AddSyncStep ( NewSimpleStep ( "operate user role meta data" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . meta . OperateUserRole ( util . DefaultTenant , & milvuspb . UserEntity { Name : in . Username } , & milvuspb . RoleEntity { Name : in . RoleName } , in . Type )
if err != nil && ! common . IsIgnorableError ( err ) {
log . Warn ( "operate user role mata data failed" , zap . Error ( err ) )
return nil , err
2022-08-23 10:26:53 +08:00
}
2023-08-04 18:37:08 +08:00
return nil , nil
} ) )
redoTask . AddAsyncStep ( NewSimpleStep ( "operate user role cache" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
2022-08-23 10:26:53 +08:00
var opType int32
switch in . Type {
case milvuspb . OperateUserRoleType_AddUserToRole :
opType = int32 ( typeutil . CacheAddUserToRole )
case milvuspb . OperateUserRoleType_RemoveUserFromRole :
opType = int32 ( typeutil . CacheRemoveUserFromRole )
default :
errMsg := "invalid operate type for the OperateUserRole api"
2023-08-04 18:37:08 +08:00
log . Warn ( errMsg , zap . Any ( "in" , in ) )
return nil , nil
2022-08-23 10:26:53 +08:00
}
if err := c . proxyClientManager . RefreshPolicyInfoCache ( ctx , & proxypb . RefreshPolicyInfoCacheRequest {
OpType : opType ,
OpKey : funcutil . EncodeUserRoleCache ( in . Username , in . RoleName ) ,
} ) ; err != nil {
2023-08-04 18:37:08 +08:00
log . Warn ( "fail to refresh policy info cache" , zap . Any ( "in" , in ) , zap . Error ( err ) )
return nil , err
2022-08-23 10:26:53 +08:00
}
2023-08-04 18:37:08 +08:00
return nil , nil
} ) )
err := redoTask . Execute ( ctx )
if err != nil {
errMsg := "fail to execute task when operate the user and role"
log . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_OperateUserRoleFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-08-04 11:04:34 +08:00
}
// SelectRole select role
// - check the node health
// - check if the role is valid when this param is provided
2022-09-05 13:29:11 +08:00
// - select role by the meta api
2022-08-04 11:04:34 +08:00
func ( c * Core ) SelectRole ( ctx context . Context , in * milvuspb . SelectRoleRequest ) ( * milvuspb . SelectRoleResponse , error ) {
method := "SelectRole"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & milvuspb . SelectRoleResponse { Status : merr . Status ( err ) } , nil
2022-08-04 11:04:34 +08:00
}
if in . Role != nil {
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . Role . Name } , false ) ; err != nil {
2023-10-13 17:15:34 +08:00
if errors . Is ( err , merr . ErrIoKeyNotFound ) {
2022-08-10 17:20:41 +08:00
return & milvuspb . SelectRoleResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-08-10 17:20:41 +08:00
} , nil
}
2022-09-02 21:12:59 +08:00
errMsg := "fail to select the role to check the role name"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectRoleResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_SelectRoleFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
}
2022-09-05 13:29:11 +08:00
roleResults , err := c . meta . SelectRole ( util . DefaultTenant , in . Role , in . IncludeUserInfo )
2022-08-04 11:04:34 +08:00
if err != nil {
errMsg := "fail to select the role"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectRoleResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_SelectRoleFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & milvuspb . SelectRoleResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-08-04 11:04:34 +08:00
Results : roleResults ,
} , nil
}
// SelectUser select user
// - check the node health
// - check if the user is valid when this param is provided
2022-09-05 13:29:11 +08:00
// - select user by the meta api
2022-08-04 11:04:34 +08:00
func ( c * Core ) SelectUser ( ctx context . Context , in * milvuspb . SelectUserRequest ) ( * milvuspb . SelectUserResponse , error ) {
method := "SelectUser"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & milvuspb . SelectUserResponse { Status : merr . Status ( err ) } , nil
2022-08-04 11:04:34 +08:00
}
if in . User != nil {
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectUser ( util . DefaultTenant , & milvuspb . UserEntity { Name : in . User . Name } , false ) ; err != nil {
2023-10-13 17:15:34 +08:00
if errors . Is ( err , merr . ErrIoKeyNotFound ) {
2022-08-10 17:20:41 +08:00
return & milvuspb . SelectUserResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-08-10 17:20:41 +08:00
} , nil
}
2022-09-02 21:12:59 +08:00
errMsg := "fail to select the user to check the username"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectUserResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_SelectUserFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
}
2022-09-05 13:29:11 +08:00
userResults , err := c . meta . SelectUser ( util . DefaultTenant , in . User , in . IncludeRoleInfo )
2022-08-04 11:04:34 +08:00
if err != nil {
errMsg := "fail to select the user"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectUserResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_SelectUserFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & milvuspb . SelectUserResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-08-04 11:04:34 +08:00
Results : userResults ,
} , nil
}
func ( c * Core ) isValidRole ( entity * milvuspb . RoleEntity ) error {
if entity == nil {
2022-09-02 21:12:59 +08:00
return errors . New ( "the role entity is nil" )
2022-08-04 11:04:34 +08:00
}
if entity . Name == "" {
2022-09-02 21:12:59 +08:00
return errors . New ( "the name in the role entity is empty" )
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : entity . Name } , false ) ; err != nil {
2023-01-12 13:55:42 +08:00
log . Warn ( "fail to select the role" , zap . String ( "role_name" , entity . Name ) , zap . Error ( err ) )
return errors . New ( "not found the role, maybe the role isn't existed or internal system error" )
2022-08-04 11:04:34 +08:00
}
return nil
}
func ( c * Core ) isValidObject ( entity * milvuspb . ObjectEntity ) error {
if entity == nil {
2022-09-02 21:12:59 +08:00
return errors . New ( "the object entity is nil" )
2022-08-04 11:04:34 +08:00
}
if _ , ok := commonpb . ObjectType_value [ entity . Name ] ; ! ok {
2022-11-29 20:05:14 +08:00
return fmt . Errorf ( "not found the object type[name: %s], supported the object types: %v" , entity . Name , lo . Keys ( commonpb . ObjectType_value ) )
2022-08-04 11:04:34 +08:00
}
return nil
}
func ( c * Core ) isValidGrantor ( entity * milvuspb . GrantorEntity , object string ) error {
if entity == nil {
2022-09-02 21:12:59 +08:00
return errors . New ( "the grantor entity is nil" )
2022-08-04 11:04:34 +08:00
}
if entity . User == nil {
2022-09-02 21:12:59 +08:00
return errors . New ( "the user entity in the grantor entity is nil" )
2022-08-04 11:04:34 +08:00
}
if entity . User . Name == "" {
2022-09-02 21:12:59 +08:00
return errors . New ( "the name in the user entity of the grantor entity is empty" )
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectUser ( util . DefaultTenant , & milvuspb . UserEntity { Name : entity . User . Name } , false ) ; err != nil {
2023-01-12 13:55:42 +08:00
log . Warn ( "fail to select the user" , zap . String ( "username" , entity . User . Name ) , zap . Error ( err ) )
return errors . New ( "not found the user, maybe the user isn't existed or internal system error" )
2022-08-04 11:04:34 +08:00
}
if entity . Privilege == nil {
2022-09-02 21:12:59 +08:00
return errors . New ( "the privilege entity in the grantor entity is nil" )
2022-08-04 11:04:34 +08:00
}
2022-08-15 16:40:48 +08:00
if util . IsAnyWord ( entity . Privilege . Name ) {
return nil
}
2022-08-05 16:28:35 +08:00
if privilegeName := util . PrivilegeNameForMetastore ( entity . Privilege . Name ) ; privilegeName == "" {
2022-11-29 20:05:14 +08:00
return fmt . Errorf ( "not found the privilege name[%s]" , entity . Privilege . Name )
2022-08-04 11:04:34 +08:00
}
2022-08-15 16:40:48 +08:00
privileges , ok := util . ObjectPrivileges [ object ]
2022-08-04 11:04:34 +08:00
if ! ok {
2022-11-29 20:05:14 +08:00
return fmt . Errorf ( "not found the object type[name: %s], supported the object types: %v" , object , lo . Keys ( commonpb . ObjectType_value ) )
2022-08-04 11:04:34 +08:00
}
for _ , privilege := range privileges {
if privilege == entity . Privilege . Name {
return nil
}
}
2022-11-29 20:05:14 +08:00
return fmt . Errorf ( "not found the privilege name[%s]" , entity . Privilege . Name )
2022-08-04 11:04:34 +08:00
}
// OperatePrivilege operate the privilege, including grant and revoke
// - check the node health
// - check if the operating type is valid
// - check if the entity is nil
// - check if the params, including the resource entity, the principal entity, the grantor entity, is valid
2022-09-05 13:29:11 +08:00
// - operate the privilege by the meta api
2022-08-04 11:04:34 +08:00
// - update the policy cache
func ( c * Core ) OperatePrivilege ( ctx context . Context , in * milvuspb . OperatePrivilegeRequest ) ( * commonpb . Status , error ) {
method := "OperatePrivilege"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2022-08-04 11:04:34 +08:00
}
if in . Type != milvuspb . OperatePrivilegeType_Grant && in . Type != milvuspb . OperatePrivilegeType_Revoke {
errMsg := fmt . Sprintf ( "invalid operate privilege type, current type: %s, valid value: [%s, %s]" , in . Type , milvuspb . OperatePrivilegeType_Grant , milvuspb . OperatePrivilegeType_Revoke )
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_OperatePrivilegeFailure ) , nil
2022-08-04 11:04:34 +08:00
}
if in . Entity == nil {
errMsg := "the grant entity in the request is nil"
2023-08-04 18:37:08 +08:00
ctxLog . Error ( errMsg )
return merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_OperatePrivilegeFailure ) , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidObject ( in . Entity . Object ) ; err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "" , zap . Error ( err ) )
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_OperatePrivilegeFailure ) , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidRole ( in . Entity . Role ) ; err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "" , zap . Error ( err ) )
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_OperatePrivilegeFailure ) , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidGrantor ( in . Entity . Grantor , in . Entity . Object . Name ) ; err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Error ( "" , zap . Error ( err ) )
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_OperatePrivilegeFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( "before PrivilegeNameForMetastore" , zap . String ( "privilege" , in . Entity . Grantor . Privilege . Name ) )
2022-08-15 16:40:48 +08:00
if ! util . IsAnyWord ( in . Entity . Grantor . Privilege . Name ) {
in . Entity . Grantor . Privilege . Name = util . PrivilegeNameForMetastore ( in . Entity . Grantor . Privilege . Name )
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( "after PrivilegeNameForMetastore" , zap . String ( "privilege" , in . Entity . Grantor . Privilege . Name ) )
2022-08-04 11:04:34 +08:00
if in . Entity . Object . Name == commonpb . ObjectType_Global . String ( ) {
2022-08-15 16:40:48 +08:00
in . Entity . ObjectName = util . AnyWord
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
redoTask := newBaseRedoTask ( c . stepExecutor )
redoTask . AddSyncStep ( NewSimpleStep ( "operate privilege meta data" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
err := c . meta . OperatePrivilege ( util . DefaultTenant , in . Entity , in . Type )
if err != nil && ! common . IsIgnorableError ( err ) {
log . Warn ( "fail to operate the privilege" , zap . Any ( "in" , in ) , zap . Error ( err ) )
return nil , err
}
return nil , nil
} ) )
redoTask . AddAsyncStep ( NewSimpleStep ( "operate privilege cache" , func ( ctx context . Context ) ( [ ] nestedStep , error ) {
2022-08-23 10:26:53 +08:00
var opType int32
switch in . Type {
case milvuspb . OperatePrivilegeType_Grant :
opType = int32 ( typeutil . CacheGrantPrivilege )
case milvuspb . OperatePrivilegeType_Revoke :
opType = int32 ( typeutil . CacheRevokePrivilege )
default :
2023-08-04 18:37:08 +08:00
log . Warn ( "invalid operate type for the OperatePrivilege api" , zap . Any ( "in" , in ) )
return nil , nil
2022-08-23 10:26:53 +08:00
}
if err := c . proxyClientManager . RefreshPolicyInfoCache ( ctx , & proxypb . RefreshPolicyInfoCacheRequest {
OpType : opType ,
2023-06-25 17:20:43 +08:00
OpKey : funcutil . PolicyForPrivilege ( in . Entity . Role . Name , in . Entity . Object . Name , in . Entity . ObjectName , in . Entity . Grantor . Privilege . Name , in . Entity . DbName ) ,
2022-08-23 10:26:53 +08:00
} ) ; err != nil {
2023-08-04 18:37:08 +08:00
log . Warn ( "fail to refresh policy info cache" , zap . Any ( "in" , in ) , zap . Error ( err ) )
return nil , err
2022-08-23 10:26:53 +08:00
}
2023-08-04 18:37:08 +08:00
return nil , nil
} ) )
err := redoTask . Execute ( ctx )
if err != nil {
errMsg := "fail to execute task when operating the privilege"
log . Warn ( errMsg , zap . Error ( err ) )
return merr . StatusWithErrorCode ( err , commonpb . ErrorCode_OperatePrivilegeFailure ) , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2022-08-04 11:04:34 +08:00
}
// SelectGrant select grant
// - check the node health
// - check if the principal entity is valid
// - check if the resource entity which is provided by the user is valid
2022-09-05 13:29:11 +08:00
// - select grant by the meta api
2022-08-04 11:04:34 +08:00
func ( c * Core ) SelectGrant ( ctx context . Context , in * milvuspb . SelectGrantRequest ) ( * milvuspb . SelectGrantResponse , error ) {
method := "SelectGrant"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2023-08-04 18:37:08 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
if in . Entity == nil {
errMsg := "the grant entity in the request is nil"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_SelectGrantFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidRole ( in . Entity . Role ) ; err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "" , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( err , commonpb . ErrorCode_SelectGrantFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
if in . Entity . Object != nil {
if err := c . isValidObject ( in . Entity . Object ) ; err != nil {
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( "" , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( err , commonpb . ErrorCode_SelectGrantFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
}
2022-09-05 13:29:11 +08:00
grantEntities , err := c . meta . SelectGrant ( util . DefaultTenant , in . Entity )
2023-10-13 17:15:34 +08:00
if errors . Is ( err , merr . ErrIoKeyNotFound ) {
2022-08-23 10:26:53 +08:00
return & milvuspb . SelectGrantResponse {
2023-12-14 16:20:43 +08:00
Status : merr . Success ( ) ,
Entities : grantEntities ,
2022-08-23 10:26:53 +08:00
} , nil
}
2022-08-04 11:04:34 +08:00
if err != nil {
errMsg := "fail to select the grant"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_SelectGrantFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & milvuspb . SelectGrantResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-08-04 11:04:34 +08:00
Entities : grantEntities ,
} , nil
}
func ( c * Core ) ListPolicy ( ctx context . Context , in * internalpb . ListPolicyRequest ) ( * internalpb . ListPolicyResponse , error ) {
method := "PolicyList"
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( method )
2023-08-04 18:37:08 +08:00
ctxLog := log . Ctx ( ctx ) . With ( zap . String ( "role" , typeutil . RootCoordRole ) , zap . Any ( "in" , in ) )
ctxLog . Debug ( method )
2022-08-04 11:04:34 +08:00
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2022-08-04 11:04:34 +08:00
return & internalpb . ListPolicyResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2023-08-04 18:37:08 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
policies , err := c . meta . ListPolicy ( util . DefaultTenant )
2022-08-04 11:04:34 +08:00
if err != nil {
2022-09-02 21:12:59 +08:00
errMsg := "fail to list policy"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & internalpb . ListPolicyResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_ListPolicyFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
userRoles , err := c . meta . ListUserRole ( util . DefaultTenant )
2022-08-04 11:04:34 +08:00
if err != nil {
2022-09-02 21:12:59 +08:00
errMsg := "fail to list user-role"
2023-08-04 18:37:08 +08:00
ctxLog . Warn ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & internalpb . ListPolicyResponse {
2023-08-04 18:37:08 +08:00
Status : merr . StatusWithErrorCode ( errors . New ( errMsg ) , commonpb . ErrorCode_ListPolicyFailure ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
2023-08-04 18:37:08 +08:00
ctxLog . Debug ( method + " success" )
2022-08-04 11:04:34 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & internalpb . ListPolicyResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2022-08-04 11:04:34 +08:00
PolicyInfos : policies ,
UserRoles : userRoles ,
} , nil
}
2022-10-18 13:39:26 +08:00
2023-01-19 14:13:43 +08:00
func ( c * Core ) RenameCollection ( ctx context . Context , req * milvuspb . RenameCollectionRequest ) ( * commonpb . Status , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return merr . Status ( err ) , nil
2023-01-19 14:13:43 +08:00
}
log := log . Ctx ( ctx ) . With ( zap . String ( "oldCollectionName" , req . GetOldName ( ) ) , zap . String ( "newCollectionName" , req . GetNewName ( ) ) )
log . Info ( "received request to rename collection" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "RenameCollection" , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "RenameCollection" )
t := & renameCollectionTask {
2023-06-26 17:52:44 +08:00
baseTask : newBaseTask ( ctx , c ) ,
Req : req ,
2023-01-19 14:13:43 +08:00
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Warn ( "failed to enqueue request to rename collection" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "RenameCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2023-01-19 14:13:43 +08:00
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Warn ( "failed to rename collection" , zap . Uint64 ( "ts" , t . GetTs ( ) ) , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "RenameCollection" , metrics . FailLabel ) . Inc ( )
2023-03-13 14:29:53 +08:00
return merr . Status ( err ) , nil
2023-01-19 14:13:43 +08:00
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( "RenameCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "RenameCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
log . Info ( "done to rename collection" , zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2023-10-11 21:01:35 +08:00
return merr . Success ( ) , nil
2023-01-19 14:13:43 +08:00
}
2024-04-12 16:01:19 +08:00
func ( c * Core ) DescribeDatabase ( ctx context . Context , req * rootcoordpb . DescribeDatabaseRequest ) ( * rootcoordpb . DescribeDatabaseResponse , error ) {
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
return & rootcoordpb . DescribeDatabaseResponse { Status : merr . Status ( err ) } , nil
}
log := log . Ctx ( ctx ) . With ( zap . String ( "dbName" , req . GetDbName ( ) ) )
log . Info ( "received request to describe database " )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeDatabase" , metrics . TotalLabel ) . Inc ( )
tr := timerecord . NewTimeRecorder ( "DescribeDatabase" )
t := & describeDBTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : req ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Warn ( "failed to enqueue request to describe database" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeDatabase" , metrics . FailLabel ) . Inc ( )
return & rootcoordpb . DescribeDatabaseResponse { Status : merr . Status ( err ) } , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Warn ( "failed to describe database" , zap . Uint64 ( "ts" , t . GetTs ( ) ) , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeDatabase" , metrics . FailLabel ) . Inc ( )
return & rootcoordpb . DescribeDatabaseResponse { Status : merr . Status ( err ) } , nil
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeDatabase" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "DescribeDatabase" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
log . Info ( "done to describe database" , zap . Uint64 ( "ts" , t . GetTs ( ) ) )
return t . Rsp , nil
}
2022-10-18 13:39:26 +08:00
func ( c * Core ) CheckHealth ( ctx context . Context , in * milvuspb . CheckHealthRequest ) ( * milvuspb . CheckHealthResponse , error ) {
2023-10-11 21:01:35 +08:00
if err := merr . CheckHealthy ( c . GetStateCode ( ) ) ; err != nil {
2023-09-07 19:23:15 +08:00
return & milvuspb . CheckHealthResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Status ( err ) ,
2023-09-07 19:23:15 +08:00
IsHealthy : false ,
2023-10-11 21:01:35 +08:00
Reasons : [ ] string { fmt . Sprintf ( "serverID=%d: %v" , c . session . ServerID , err ) } ,
2023-09-07 19:23:15 +08:00
} , nil
2022-10-18 13:39:26 +08:00
}
group , ctx := errgroup . WithContext ( ctx )
2024-07-01 10:18:07 +08:00
errs := typeutil . NewConcurrentSet [ error ] ( )
2022-10-18 13:39:26 +08:00
2023-12-14 18:22:39 +08:00
proxyClients := c . proxyClientManager . GetProxyClients ( )
proxyClients . Range ( func ( key int64 , value types . ProxyClient ) bool {
nodeID := key
proxyClient := value
2022-10-18 13:39:26 +08:00
group . Go ( func ( ) error {
2023-09-26 09:57:25 +08:00
sta , err := proxyClient . GetComponentStates ( ctx , & milvuspb . GetComponentStatesRequest { } )
2023-10-11 21:01:35 +08:00
if err != nil {
2024-07-01 10:18:07 +08:00
errs . Insert ( err )
2023-10-11 21:01:35 +08:00
return err
}
err = merr . AnalyzeState ( "Proxy" , nodeID , sta )
if err != nil {
2024-07-01 10:18:07 +08:00
errs . Insert ( err )
2022-10-18 13:39:26 +08:00
}
2024-07-01 10:18:07 +08:00
return err
2022-10-18 13:39:26 +08:00
} )
2023-12-14 18:22:39 +08:00
return true
} )
2022-10-18 13:39:26 +08:00
2024-07-01 10:18:07 +08:00
maxDelay := Params . QuotaConfig . MaxTimeTickDelay . GetAsDuration ( time . Second )
if maxDelay > 0 {
group . Go ( func ( ) error {
err := CheckTimeTickLagExceeded ( ctx , c . queryCoord , c . dataCoord , maxDelay )
if err != nil {
errs . Insert ( err )
}
return err
} )
}
2022-10-18 13:39:26 +08:00
err := group . Wait ( )
2024-07-01 10:18:07 +08:00
if err != nil {
2023-09-07 19:23:15 +08:00
return & milvuspb . CheckHealthResponse {
2023-10-11 21:01:35 +08:00
Status : merr . Success ( ) ,
2023-09-07 19:23:15 +08:00
IsHealthy : false ,
2024-07-01 10:18:07 +08:00
Reasons : lo . Map ( errs . Collect ( ) , func ( e error , i int ) string {
return err . Error ( )
} ) ,
2023-09-07 19:23:15 +08:00
} , nil
2022-10-18 13:39:26 +08:00
}
2024-07-01 10:18:07 +08:00
return & milvuspb . CheckHealthResponse { Status : merr . Success ( ) , IsHealthy : true , Reasons : [ ] string { } } , nil
2022-10-18 13:39:26 +08:00
}