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"
2022-04-25 11:07:47 +08:00
"errors"
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"
"sync/atomic"
2021-11-22 16:23:17 +08:00
"syscall"
2021-01-19 14:44:03 +08:00
"time"
2022-10-18 13:39:26 +08:00
"golang.org/x/sync/errgroup"
2022-10-16 20:49:27 +08:00
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
2022-09-26 18:06:54 +08:00
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/common"
2022-10-08 15:38:58 +08:00
pnc "github.com/milvus-io/milvus/internal/distributed/proxy/client"
2022-09-05 13:29:11 +08:00
"github.com/milvus-io/milvus/internal/kv"
2022-09-26 18:06:54 +08:00
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log"
2022-08-11 12:12:38 +08:00
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/metastore/db/dao"
"github.com/milvus-io/milvus/internal/metastore/db/dbcore"
2022-09-26 18:06:54 +08:00
"github.com/milvus-io/milvus/internal/metastore/db/rootcoord"
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"
2021-06-01 11:04:31 +08:00
"github.com/milvus-io/milvus/internal/metrics"
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"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/tso"
"github.com/milvus-io/milvus/internal/types"
2022-04-21 19:57:42 +08:00
"github.com/milvus-io/milvus/internal/util"
2022-10-21 15:57:28 +08:00
"github.com/milvus-io/milvus/internal/util/commonpbutil"
2022-04-21 19:57:42 +08:00
"github.com/milvus-io/milvus/internal/util/crypto"
"github.com/milvus-io/milvus/internal/util/dependency"
2022-08-11 12:12:38 +08:00
"github.com/milvus-io/milvus/internal/util/errorutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
2021-10-01 11:08:24 +08:00
"github.com/milvus-io/milvus/internal/util/metricsinfo"
2021-12-23 18:39:11 +08:00
"github.com/milvus-io/milvus/internal/util/paramtable"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/util/retry"
2021-05-21 19:28:52 +08:00
"github.com/milvus-io/milvus/internal/util/sessionutil"
2022-03-28 16:41:28 +08:00
"github.com/milvus-io/milvus/internal/util/timerecord"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
2022-07-22 10:20:29 +08:00
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
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
2022-09-05 13:29:11 +08:00
type metaKVCreator func ( root string ) ( kv . MetaKv , error )
2022-03-25 11:03:25 +08:00
2022-09-05 13:29:11 +08:00
func defaultMetaKVCreator ( etcdCli * clientv3 . Client ) metaKVCreator {
return func ( root string ) ( kv . MetaKv , error ) {
return etcdkv . NewEtcdKV ( etcdCli , root ) , nil
}
}
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
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
proxyCreator proxyCreator
proxyManager * proxyManager
2021-05-26 20:14:30 +08:00
proxyClientManager * proxyClientManager
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
2022-09-05 13:29:11 +08:00
tsoAllocator tso . Allocator
2021-01-19 14:44:03 +08:00
2022-09-05 13:29:11 +08:00
dataCoord types . DataCoord
queryCoord types . QueryCoord
indexCoord types . IndexCoord
2021-01-19 14:44:03 +08:00
2022-09-16 09:56:47 +08:00
quotaCenter * QuotaCenter
2022-09-05 13:29:11 +08:00
stateCode atomic . Value
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
importManager * importManager
2022-09-29 18:35:02 +08:00
enableActiveStandBy bool
activateFunc func ( )
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 ,
enableActiveStandBy : Params . RootCoordCfg . EnableActiveStandby ,
2021-01-19 14:44:03 +08:00
}
2022-10-10 15:55:22 +08:00
core . UpdateStateCode ( commonpb . StateCode_Abnormal )
2022-10-08 15:38:58 +08:00
core . proxyCreator = func ( se * sessionutil . Session ) ( types . Proxy , error ) {
cli , err := pnc . NewClient ( c , se . Address )
if err != nil {
return nil , err
}
if err := cli . Init ( ) ; err != nil {
return nil , err
}
if err := cli . Start ( ) ; err != nil {
return nil , err
}
return cli , nil
}
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 ) {
2021-02-23 11:40:30 +08:00
c . stateCode . Store ( 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
}
2022-10-10 15:55:22 +08:00
func ( c * Core ) checkHealthy ( ) ( commonpb . StateCode , bool ) {
code := c . stateCode . Load ( ) . ( commonpb . StateCode )
ok := code == commonpb . StateCode_Healthy
2021-11-19 12:11:12 +08:00
return code , ok
}
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 ( ) {
2022-11-14 14:41:11 +08:00
code := c . stateCode . Load ( ) . ( commonpb . StateCode )
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 ( )
2021-12-27 21:38:45 +08:00
ticker := time . NewTicker ( Params . ProxyCfg . TimeTickInterval )
2021-05-31 16:48:31 +08:00
for {
select {
case <- c . ctx . Done ( ) :
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 ( )
2021-02-24 17:12:06 +08:00
tsoTicker := time . NewTicker ( tso . 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 {
2021-03-15 15:45:17 +08:00
log . Warn ( "failed to update timestamp: " , zap . Error ( err ) )
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 ( ) ) )
2022-09-05 13:29:11 +08:00
if err := c . tsoAllocator . UpdateTSO ( ) ; err != nil {
2021-03-15 15:45:17 +08:00
log . Warn ( "failed to update id: " , zap . Error ( err ) )
continue
2021-01-27 16:38:18 +08:00
}
case <- ctx . Done ( ) :
return
}
}
}
2021-05-14 21:26:06 +08:00
2021-06-21 18:22:13 +08:00
func ( c * Core ) SetDataCoord ( ctx context . Context , s types . DataCoord ) error {
2022-09-05 13:29:11 +08:00
if err := s . Init ( ) ; err != nil {
return err
2022-06-15 12:20:10 +08:00
}
2022-09-05 13:29:11 +08:00
if err := s . Start ( ) ; err != nil {
return err
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
}
2021-06-21 17:28:03 +08:00
func ( c * Core ) SetIndexCoord ( s types . IndexCoord ) error {
2022-09-05 13:29:11 +08:00
if err := s . Init ( ) ; err != nil {
return err
2021-01-24 20:26:35 +08:00
}
2022-09-05 13:29:11 +08:00
if err := s . Start ( ) ; err != nil {
return err
2021-02-20 15:38:44 +08:00
}
2022-09-05 13:29:11 +08:00
c . indexCoord = s
2021-01-24 20:26:35 +08:00
return nil
}
2021-06-22 16:44:09 +08:00
func ( c * Core ) SetQueryCoord ( s types . QueryCoord ) error {
2022-09-05 13:29:11 +08:00
if err := s . Init ( ) ; err != nil {
return err
2021-06-22 16:08:08 +08:00
}
2022-09-05 13:29:11 +08:00
if err := s . Start ( ) ; err != nil {
return err
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 ( )
2022-09-29 18:35:02 +08:00
if c . enableActiveStandBy {
c . session . ProcessActiveStandBy ( c . activateFunc )
} else {
2022-10-10 15:55:22 +08:00
c . UpdateStateCode ( commonpb . StateCode_Healthy )
log . Debug ( "RootCoord start successfully " , zap . String ( "State Code" , commonpb . StateCode_Healthy . String ( ) ) )
2022-09-29 18:35:02 +08:00
}
log . Info ( "RootCoord Register Finished" )
2021-12-15 11:47:10 +08:00
go c . session . LivenessCheck ( c . ctx , func ( ) {
log . Error ( "Root Coord disconnected from etcd, process will exit" , zap . Int64 ( "Server Id" , c . session . ServerID ) )
if err := c . Stop ( ) ; err != nil {
log . Fatal ( "failed to stop server" , zap . Error ( err ) )
}
// manually send signal to starter goroutine
2021-12-29 14:35:21 +08:00
if c . session . TriggerKill {
2022-03-17 17:17:22 +08:00
if p , err := os . FindProcess ( os . Getpid ( ) ) ; err == nil {
p . Signal ( syscall . SIGINT )
}
2021-12-29 14:35:21 +08:00
}
2021-12-15 11:47:10 +08:00
} )
2022-01-18 12:09:37 +08:00
2022-10-10 15:55:22 +08:00
c . UpdateStateCode ( commonpb . StateCode_Healthy )
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
}
2021-12-15 11:47:10 +08:00
func ( c * Core ) initSession ( ) error {
2022-02-07 10:09:45 +08:00
c . session = sessionutil . NewSession ( c . ctx , Params . EtcdCfg . MetaRootPath , c . etcdCli )
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 {
c . metaKVCreator = defaultMetaKVCreator ( c . etcdCli )
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-09-05 13:29:11 +08:00
switch Params . MetaStoreCfg . MetaStoreType {
case util . MetaStoreTypeEtcd :
var metaKV kv . MetaKv
var ss * kvmetestore . SuffixSnapshot
var err error
2022-08-11 12:12:38 +08:00
2022-09-05 13:29:11 +08:00
if metaKV , err = c . metaKVCreator ( Params . EtcdCfg . MetaRootPath ) ; err != nil {
return err
}
2022-08-11 12:12:38 +08:00
2022-10-12 11:37:23 +08:00
if ss , err = kvmetestore . NewSuffixSnapshot ( metaKV , kvmetestore . SnapshotsSep , Params . EtcdCfg . MetaRootPath , kvmetestore . SnapshotPrefix ) ; err != nil {
2022-09-05 13:29:11 +08:00
return err
2021-02-26 15:17:47 +08:00
}
2022-08-11 12:12:38 +08:00
2022-09-05 13:29:11 +08:00
catalog = & kvmetestore . Catalog { Txn : metaKV , Snapshot : ss }
case util . MetaStoreTypeMysql :
// connect to database
err := dbcore . Connect ( & Params . DBCfg )
if err != nil {
return err
2021-08-13 11:04:09 +08:00
}
2022-09-05 13:29:11 +08:00
catalog = rootcoord . NewTableCatalog ( dbcore . NewTxImpl ( ) , dao . NewMetaDomain ( ) )
default :
return retry . Unrecoverable ( fmt . Errorf ( "not supported meta store: %s" , Params . MetaStoreCfg . MetaStoreType ) )
2021-01-19 14:44:03 +08:00
}
2022-09-05 13:29:11 +08:00
if c . meta , err = NewMetaTable ( c . ctx , catalog ) ; err != nil {
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 {
tsoKV := tsoutil . NewTSOKVBase ( c . etcdCli , Params . EtcdCfg . KvRootPath , globalIDAllocatorSubPath )
idAllocator := allocator . NewGlobalIDAllocator ( globalIDAllocatorKey , tsoKV )
if err := idAllocator . Initialize ( ) ; err != nil {
return err
}
c . idAllocator = idAllocator
return nil
}
2021-05-26 20:14:30 +08:00
2022-09-05 13:29:11 +08:00
func ( c * Core ) initTSOAllocator ( ) error {
tsoKV := tsoutil . NewTSOKVBase ( c . etcdCli , Params . EtcdCfg . KvRootPath , globalTSOAllocatorSubPath )
tsoAllocator := tso . NewGlobalTSOAllocator ( globalTSOAllocatorKey , tsoKV )
if err := tsoAllocator . Initialize ( ) ; err != nil {
return err
}
c . tsoAllocator = tsoAllocator
2021-05-21 16:08:12 +08:00
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 ) initImportManager ( ) error {
impTaskKv , err := c . metaKVCreator ( Params . EtcdCfg . KvRootPath )
if err != nil {
return err
}
2022-03-21 15:47:23 +08:00
2022-09-05 13:29:11 +08:00
f := NewImportFactory ( c )
c . importManager = newImportManager (
c . ctx ,
impTaskKv ,
f . NewIDAllocator ( ) ,
f . NewImportFunc ( ) ,
2022-09-26 18:06:54 +08:00
f . NewMarkSegmentsDroppedFunc ( ) ,
2022-09-05 13:29:11 +08:00
f . NewGetCollectionNameFunc ( ) ,
2022-09-26 18:06:54 +08:00
f . NewDescribeIndexFunc ( ) ,
f . NewGetSegmentIndexStateFunc ( ) ,
f . NewUnsetIsImportingStateFunc ( ) ,
2022-09-05 13:29:11 +08:00
)
c . importManager . init ( c . ctx )
2022-08-04 11:04:34 +08:00
2022-09-05 13:29:11 +08:00
return nil
}
func ( c * Core ) initInternal ( ) error {
if err := c . initSession ( ) ; err != nil {
return err
}
c . initKVCreator ( )
if err := c . initMetaTable ( ) ; err != nil {
return err
}
if err := c . initIDAllocator ( ) ; err != nil {
return err
}
if err := c . initTSOAllocator ( ) ; err != nil {
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 )
c . proxyClientManager = newProxyClientManager ( c . proxyCreator )
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
c . proxyManager = newProxyManager (
c . ctx ,
c . etcdCli ,
c . chanTimeTick . initSessions ,
c . proxyClientManager . GetProxyClients ,
)
c . proxyManager . AddSessionFunc ( c . chanTimeTick . addSession , c . proxyClientManager . AddProxyClient )
c . proxyManager . DelSessionFunc ( c . chanTimeTick . delSession , c . proxyClientManager . DelProxyClient )
c . metricsCacheManager = metricsinfo . NewMetricsCacheManager ( )
2022-09-16 09:56:47 +08:00
c . quotaCenter = NewQuotaCenter ( c . proxyClientManager , c . queryCoord , c . dataCoord , c . tsoAllocator )
log . Debug ( "RootCoord init QuotaCenter done" )
2022-09-05 13:29:11 +08:00
if err := c . initImportManager ( ) ; err != nil {
return err
}
if err := c . initCredentials ( ) ; err != nil {
return err
}
if err := c . initRbac ( ) ; err != nil {
return err
}
return nil
}
// Init initialize routine
func ( c * Core ) Init ( ) error {
var initError error
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
}
2022-08-04 11:04:34 +08:00
func ( c * Core ) initRbac ( ) ( initError error ) {
// create default roles, including admin, public
2022-08-23 10:26:53 +08:00
for _ , role := range util . DefaultRoles {
2022-09-05 13:29:11 +08:00
if initError = c . meta . CreateRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : role } ) ; initError != nil {
2022-08-23 10:26:53 +08:00
if common . IsIgnorableError ( initError ) {
initError = nil
continue
}
return
}
2022-08-04 11:04:34 +08:00
}
// grant privileges for the public role
globalPrivileges := [ ] string {
commonpb . ObjectPrivilege_PrivilegeDescribeCollection . String ( ) ,
commonpb . ObjectPrivilege_PrivilegeShowCollections . String ( ) ,
}
2022-08-05 16:28:35 +08:00
collectionPrivileges := [ ] string {
commonpb . ObjectPrivilege_PrivilegeIndexDetail . String ( ) ,
}
2022-08-04 11:04:34 +08:00
for _ , globalPrivilege := range globalPrivileges {
2022-09-05 13:29:11 +08:00
if initError = 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 ,
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 } ,
} ,
} , milvuspb . OperatePrivilegeType_Grant ) ; initError != nil {
2022-08-23 10:26:53 +08:00
if common . IsIgnorableError ( initError ) {
initError = nil
continue
}
2022-08-04 11:04:34 +08:00
return
}
}
2022-08-05 16:28:35 +08:00
for _ , collectionPrivilege := range collectionPrivileges {
2022-09-05 13:29:11 +08:00
if initError = 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 ,
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 } ,
} ,
} , milvuspb . OperatePrivilegeType_Grant ) ; initError != nil {
2022-08-23 10:26:53 +08:00
if common . IsIgnorableError ( initError ) {
initError = nil
continue
}
2022-08-05 16:28:35 +08:00
return
}
}
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 {
colls , err := c . meta . ListAbnormalCollections ( ctx , typeutil . MaxTimestamp )
if err != nil {
return err
}
for _ , coll := range colls {
ts , err := c . tsoAllocator . GenerateTSO ( 1 )
if err != nil {
return err
}
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 :
}
}
colls , err = c . meta . ListCollections ( ctx , typeutil . MaxTimestamp )
if err != nil {
return err
}
for _ , coll := range colls {
for _ , part := range coll . Partitions {
ts , err := c . tsoAllocator . GenerateTSO ( 1 )
if err != nil {
return err
}
switch part . State {
case pb . PartitionState_PartitionDropping :
go c . garbageCollector . ReDropPartition ( coll . PhysicalChannelNames , part . Clone ( ) , ts )
default :
}
}
}
return nil
}
func ( c * Core ) startInternal ( ) error {
if err := c . proxyManager . WatchProxy ( ) ; err != nil {
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-09-26 18:06:54 +08:00
c . wg . Add ( 6 )
2022-09-05 13:29:11 +08:00
go c . startTimeTickLoop ( )
2022-09-26 18:06:54 +08:00
go c . tsLoop ( )
2022-09-05 13:29:11 +08:00
go c . chanTimeTick . startWatch ( & c . wg )
2022-09-26 18:06:54 +08:00
go c . importManager . cleanupLoop ( & c . wg )
2022-09-05 13:29:11 +08:00
go c . importManager . sendOutTasksLoop ( & c . wg )
2022-09-26 18:06:54 +08:00
go c . importManager . flipTaskStateLoop ( & c . wg )
Params . RootCoordCfg . CreatedTime = time . Now ( )
Params . RootCoordCfg . UpdatedTime = time . Now ( )
2022-09-05 13:29:11 +08:00
2022-09-26 16:48:53 +08:00
if Params . QuotaConfig . QuotaAndLimitsEnabled {
2022-09-16 09:56:47 +08:00
go c . quotaCenter . run ( )
}
2022-09-05 13:29:11 +08:00
c . scheduler . Start ( )
2022-09-21 15:46:51 +08:00
c . stepExecutor . Start ( )
2022-09-05 13:29:11 +08:00
Params . RootCoordCfg . CreatedTime = time . Now ( )
Params . RootCoordCfg . UpdatedTime = time . Now ( )
2022-09-29 18:35:02 +08:00
if c . enableActiveStandBy {
c . activateFunc = func ( ) {
// todo to complete
log . Info ( "rootcoord switch from standby to active, activating" )
2022-10-10 15:55:22 +08:00
c . UpdateStateCode ( commonpb . StateCode_Healthy )
2022-09-29 18:35:02 +08:00
}
2022-10-10 15:55:22 +08:00
c . UpdateStateCode ( commonpb . StateCode_StandBy )
2022-09-29 18:35:02 +08:00
}
2022-09-05 13:29:11 +08:00
return nil
2022-06-14 16:18:09 +08:00
}
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
2021-01-19 14:44:03 +08:00
c . startOnce . Do ( func ( ) {
2022-09-05 13:29:11 +08:00
err = c . startInternal ( )
2021-01-19 14:44:03 +08:00
} )
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
c . session . Revoke ( time . Second )
log . Info ( "revoke rootcoord session" )
}
}
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 ( )
c . cancelIfNotNil ( )
2021-09-17 12:37:50 +08:00
c . wg . Wait ( )
2022-10-25 09:51:30 +08:00
c . revokeSession ( )
2021-01-19 14:44:03 +08:00
return nil
}
2021-09-23 15:10:00 +08:00
// GetComponentStates get states of components
2022-10-10 15:55:22 +08:00
func ( c * Core ) GetComponentStates ( ctx context . Context ) ( * milvuspb . ComponentStates , error ) {
code := c . stateCode . Load ( ) . ( commonpb . StateCode )
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
} ,
2021-01-26 17:47:38 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-01-26 17:47:38 +08:00
Reason : "" ,
} ,
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
2021-02-26 17:44:24 +08:00
func ( c * Core ) GetTimeTickChannel ( ctx context . Context ) ( * milvuspb . StringResponse , error ) {
return & milvuspb . StringResponse {
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-02-26 17:44:24 +08:00
Reason : "" ,
} ,
2022-03-04 11:17:56 +08:00
Value : Params . CommonCfg . RootCoordTimeTick ,
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
2021-02-26 17:44:24 +08:00
func ( c * Core ) GetStatisticsChannel ( ctx context . Context ) ( * milvuspb . StringResponse , error ) {
return & milvuspb . StringResponse {
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-02-26 17:44:24 +08:00
Reason : "" ,
} ,
2022-03-04 11:17:56 +08:00
Value : Params . CommonCfg . RootCoordStatistics ,
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
// CreateCollection create collection
2021-02-26 17:44:24 +08:00
func ( c * Core ) CreateCollection ( ctx context . Context , in * milvuspb . CreateCollectionRequest ) ( * commonpb . Status , error ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
t := & createCollectionTask {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-03-13 14:42:53 +08:00
ctx : ctx ,
2021-01-19 14:44:03 +08:00
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-01-19 14:44:03 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to enqueue request to create 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 ( "CreateCollection" , metrics . FailLabel ) . Inc ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to create 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 ( "CreateCollection" , metrics . FailLabel ) . Inc ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
2022-03-02 21:11:57 +08:00
metrics . RootCoordNumOfCollections . Inc ( )
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 ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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
2022-09-24 12:42:51 +08:00
log . Ctx ( ctx ) . Info ( "received request 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
t := & dropCollectionTask {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-03-13 14:42:53 +08:00
ctx : ctx ,
2021-01-19 14:44:03 +08:00
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-01-19 14:44:03 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-09-24 12:42:51 +08:00
log . Ctx ( ctx ) . Error ( "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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-09-24 12:42:51 +08:00
log . Ctx ( ctx ) . Error ( "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 ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
2022-03-02 21:11:57 +08:00
metrics . RootCoordNumOfCollections . Dec ( )
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 ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-01-25 18:33:10 +08:00
return & milvuspb . BoolResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
2021-11-19 12:11:12 +08:00
Value : false ,
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 )
2022-11-14 15:29:06 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collection name" , in . GetCollectionName ( ) ) ,
zap . Uint64 ( "ts" , ts ) )
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
log . Info ( "received request to has collection" )
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 {
log . Error ( "failed to enqueue request to has collection" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasCollection" , metrics . FailLabel ) . Inc ( )
return & milvuspb . BoolResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "HasCollection failed: " + err . Error ( ) ) ,
Value : false ,
} , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
log . Error ( "failed to has collection" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasCollection" , metrics . FailLabel ) . Inc ( )
return & milvuspb . BoolResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "HasCollection failed: " + err . Error ( ) ) ,
Value : false ,
} , 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 ( ) ) )
2022-10-28 13:25:34 +08:00
log . Info ( "done to has collection" , zap . Bool ( "exist" , t . Rsp . GetValue ( ) ) )
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
}
func ( c * Core ) describeCollection ( ctx context . Context , in * milvuspb . DescribeCollectionRequest ) ( * model . Collection , error ) {
ts := getTravelTs ( in )
if in . GetCollectionName ( ) != "" {
return c . meta . GetCollectionByName ( ctx , in . GetCollectionName ( ) , ts )
2022-09-05 13:29:11 +08:00
}
2022-09-28 19:26:54 +08:00
return c . meta . GetCollectionByID ( ctx , in . GetCollectionID ( ) , ts )
2022-09-24 12:42:51 +08:00
}
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
func convertModelToDesc ( collInfo * model . Collection , aliases [ ] string ) * milvuspb . DescribeCollectionResponse {
resp := & milvuspb . DescribeCollectionResponse { Status : succStatus ( ) }
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
resp . Schema = & schemapb . CollectionSchema {
Name : collInfo . Name ,
Description : collInfo . Description ,
AutoID : collInfo . AutoID ,
Fields : model . MarshalFieldModels ( collInfo . Fields ) ,
}
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
2022-09-24 12:42:51 +08:00
return resp
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// DescribeCollection return collection info
2021-02-26 17:44:24 +08:00
func ( c * Core ) DescribeCollection ( ctx context . Context , in * milvuspb . DescribeCollectionRequest ) ( * milvuspb . DescribeCollectionResponse , error ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-01-25 18:33:10 +08:00
return & milvuspb . DescribeCollectionResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
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 )
2022-11-14 15:29:06 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collection name" , in . GetCollectionName ( ) ) ,
zap . Int64 ( "id" , in . GetCollectionID ( ) ) ,
zap . Uint64 ( "ts" , ts ) )
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
// TODO(longjiquan): log may be very frequent here.
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
log . Info ( "received request to describe collection" )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & describeCollectionTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Rsp : & milvuspb . DescribeCollectionResponse { } ,
}
2022-09-27 19:18:54 +08:00
2022-10-28 13:25:34 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
log . Error ( "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 {
2022-09-24 12:42:51 +08:00
// TODO: use commonpb.ErrorCode_CollectionNotExists. SDK use commonpb.ErrorCode_UnexpectedError now.
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) ,
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 {
log . Error ( "failed to describe collection" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "DescribeCollection" , metrics . FailLabel ) . Inc ( )
return & milvuspb . DescribeCollectionResponse {
// TODO: use commonpb.ErrorCode_CollectionNotExists. SDK use commonpb.ErrorCode_UnexpectedError now.
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) ,
} , 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 ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
log . Info ( "done to describe collection" , zap . Int64 ( "collection_id" , t . Rsp . GetCollectionID ( ) ) )
2022-09-24 12:42:51 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-03-12 14:22:09 +08:00
return & milvuspb . ShowCollectionsResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
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-09-24 12:42:51 +08:00
log . Info ( "received request to show collections" )
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 {
log . Error ( "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 {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "ShowCollections failed: " + err . Error ( ) ) ,
} , nil
2022-09-05 13:29:11 +08:00
}
2022-10-28 13:25:34 +08:00
if err := t . WaitToFinish ( ) ; err != nil {
log . Error ( "failed to show collections" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowCollections" , metrics . FailLabel ) . Inc ( )
return & milvuspb . ShowCollectionsResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "ShowCollections failed: " + err . Error ( ) ) ,
} , 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 ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
log . Info ( "done to show collections" , zap . Int ( "num of collections" , len ( t . Rsp . GetCollectionNames ( ) ) ) ) // maybe very large, print number instead.
2022-09-24 12:42:51 +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 ) {
if code , ok := c . checkHealthy ( ) ; ! ok {
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , nil
}
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 ) ,
zap . String ( "name" , in . GetCollectionName ( ) ) )
2022-10-10 20:31:22 +08:00
t := & alterCollectionTask {
baseTask : baseTask {
ctx : ctx ,
core : c ,
done : make ( chan error , 1 ) ,
} ,
Req : in ,
}
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to enqueue request to alter collection" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to alter collection" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
metrics . RootCoordDDLReqCounter . WithLabelValues ( "AlterCollection" , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( "AlterCollection" ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
metrics . RootCoordNumOfCollections . Dec ( )
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 ( ) ) )
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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 {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-03-13 14:42:53 +08:00
ctx : ctx ,
2021-01-19 14:44:03 +08:00
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-01-19 14:44:03 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to enqueue request to create partition" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to create partition" ,
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 ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
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 ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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 {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-03-13 14:42:53 +08:00
ctx : ctx ,
2021-01-19 14:44:03 +08:00
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-01-19 14:44:03 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to enqueue request to drop partition" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "failed to drop partition" ,
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 ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
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 ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-01-25 18:33:10 +08:00
return & milvuspb . BoolResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
2021-11-19 12:11:12 +08:00
Value : false ,
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-09-24 12:42:51 +08:00
log . Info ( "received request to has partition" )
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 {
log . Error ( "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 {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "HasPartition failed: " + err . Error ( ) ) ,
Value : false ,
} , nil
2022-09-05 13:29:11 +08:00
}
2022-10-28 13:25:34 +08:00
if err := t . WaitToFinish ( ) ; err != nil {
log . Error ( "failed to has partition" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "HasPartition" , metrics . FailLabel ) . Inc ( )
return & milvuspb . BoolResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "HasPartition failed: " + err . Error ( ) ) ,
Value : false ,
} , 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 ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
log . Info ( "done to has partition" , zap . Bool ( "exist" , t . Rsp . GetValue ( ) ) )
2022-09-24 12:42:51 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
2021-09-23 15:10:00 +08:00
// ShowPartitions list all partition names
2021-03-12 14:22:09 +08:00
func ( c * Core ) ShowPartitions ( ctx context . Context , in * milvuspb . ShowPartitionsRequest ) ( * milvuspb . ShowPartitionsResponse , error ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-03-12 14:22:09 +08:00
return & milvuspb . ShowPartitionsResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
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
2022-11-14 15:29:06 +08:00
log := log . Ctx ( ctx ) . With ( zap . String ( "collection" , in . GetCollectionName ( ) ) )
2022-09-05 13:29:11 +08:00
2022-09-24 12:42:51 +08:00
log . Info ( "received request to show partitions" )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
t := & showPartitionTask {
baseTask : newBaseTask ( ctx , c ) ,
Req : in ,
Rsp : & milvuspb . ShowPartitionsResponse { } ,
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 {
log . Error ( "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 {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "ShowPartitions failed: " + err . Error ( ) ) ,
} , 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 {
log . Error ( "failed to show partitions" , zap . Error ( err ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( "ShowPartitions" , metrics . FailLabel ) . Inc ( )
return & milvuspb . ShowPartitionsResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "ShowPartitions failed: " + err . Error ( ) ) ,
} , 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 ( ) ) )
2022-09-05 13:29:11 +08:00
2022-10-28 13:25:34 +08:00
log . Info ( "done to show partitions" , zap . Strings ( "partitions" , t . Rsp . GetPartitionNames ( ) ) )
2022-09-24 12:42:51 +08:00
2022-10-28 13:25:34 +08:00
return t . Rsp , nil
2021-01-19 14:44:03 +08:00
}
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.
return & milvuspb . ShowSegmentsResponse { Status : succStatus ( ) } , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocTimestampResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
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 {
2021-11-19 12:11:12 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "AllocTimestamp failed: " + err . Error ( ) ) ,
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 {
2021-11-19 12:11:12 +08:00
Status : succStatus ( ) ,
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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-06-22 16:14:09 +08:00
return & rootcoordpb . AllocIDResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
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 {
2021-11-19 12:11:12 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "AllocID failed: " + err . Error ( ) ) ,
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 {
2021-11-19 12:11:12 +08:00
Status : succStatus ( ) ,
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 )
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-01-15 18:53:34 +08:00
log . Warn ( "failed to updateTimeTick because rootcoord is not healthy" , zap . Any ( "state" , code ) )
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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 ) )
2021-11-19 12:11:12 +08:00
msgTypeName := commonpb . MsgType_name [ int32 ( in . Base . GetMsgType ( ) ) ]
return failStatus ( commonpb . ErrorCode_UnexpectedError , "invalid message type " + msgTypeName ) , 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 ) )
2021-11-19 12:11:12 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "UpdateTimeTick failed: " + err . Error ( ) ) , nil
2021-05-21 16:08:12 +08:00
}
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , nil
2022-05-19 10:13:56 +08:00
}
err := c . proxyClientManager . InvalidateCollectionMetaCache ( ctx , in )
if err != nil {
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
return succStatus ( ) , 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 ) {
if code , ok := c . checkHealthy ( ) ; ! ok {
return & internalpb . ShowConfigurationsResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
2022-08-12 13:20:39 +08:00
Configuations : nil ,
} , nil
}
return getComponentConfigurations ( ctx , req ) , nil
}
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 ) {
if code , ok := c . checkHealthy ( ) ; ! ok {
2021-08-31 11:45:59 +08:00
return & milvuspb . GetMetricsResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
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 ) ,
2021-11-22 16:01:14 +08:00
zap . Int64 ( "node_id" , c . session . ServerID ) , zap . String ( "req" , in . Request ) , zap . Error ( err ) )
2021-08-31 11:45:59 +08:00
return & milvuspb . GetMetricsResponse {
2021-11-19 12:11:12 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "ParseMetricType failed: " + err . Error ( ) ) ,
2021-08-31 11:45:59 +08:00
Response : "" ,
} , nil
}
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Debug ( "GetMetrics success" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "metric_type" , metricType ) )
2021-08-31 11:45:59 +08:00
if metricType == metricsinfo . SystemInfoMetrics {
2021-09-03 17:15:26 +08:00
ret , err := c . metricsCacheManager . GetSystemInfoMetrics ( )
if err == nil && ret != nil {
return ret , nil
}
2022-11-14 15:29:06 +08:00
log . Warn ( "GetSystemInfoMetrics from cache failed" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . Error ( err ) )
2021-08-31 11:45:59 +08:00
2021-11-22 16:01:14 +08:00
systemInfoMetrics , err := c . getSystemInfoMetrics ( ctx , in )
if err != nil {
2022-11-14 15:29:06 +08:00
log . Warn ( "GetSystemInfoMetrics failed" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "metric_type" , metricType ) ,
zap . Error ( err ) )
2022-07-21 20:52:27 +08:00
return & milvuspb . GetMetricsResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , fmt . Sprintf ( "getSystemInfoMetrics failed: %s" , err . Error ( ) ) ) ,
Response : "" ,
} , nil
2021-11-22 16:01:14 +08:00
}
2021-08-31 11:45:59 +08:00
2021-09-03 17:15:26 +08:00
c . metricsCacheManager . UpdateSystemInfoMetrics ( systemInfoMetrics )
2021-08-31 11:45:59 +08:00
return systemInfoMetrics , err
}
2022-07-21 20:52:27 +08:00
log . Warn ( "GetMetrics failed, metric type not implemented" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-11-14 15:29:06 +08:00
zap . String ( "metric_type" , metricType ) )
2021-08-31 11:45:59 +08:00
return & milvuspb . GetMetricsResponse {
2021-11-19 12:11:12 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , metricsinfo . MsgUnimplementedMetric ) ,
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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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 {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-09-18 11:13:51 +08:00
ctx : ctx ,
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-09-18 11:13:51 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to enqueue request to create alias" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to create alias" ,
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 ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Info ( "done to create alias" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "alias" , in . GetAlias ( ) ) ,
zap . String ( "collection" , in . GetCollectionName ( ) ) ,
zap . Uint64 ( "ts" , t . GetTs ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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 {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-09-18 11:13:51 +08:00
ctx : ctx ,
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-09-18 11:13:51 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to enqueue request to drop alias" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to drop alias" ,
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 ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
2022-09-05 13:29:11 +08:00
2022-11-14 15:29:06 +08:00
log . Info ( "done to drop alias" ,
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 ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , 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 ) {
2021-11-19 12:11:12 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , 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 {
2022-09-24 12:42:51 +08:00
baseTask : baseTask {
2021-09-18 11:13:51 +08:00
ctx : ctx ,
core : c ,
2022-09-05 13:29:11 +08:00
done : make ( chan error , 1 ) ,
2021-09-18 11:13:51 +08:00
} ,
Req : in ,
}
2022-09-05 13:29:11 +08:00
if err := c . scheduler . AddTask ( t ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to enqueue request to alter alias" ,
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 ( )
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , nil
}
if err := t . WaitToFinish ( ) ; err != nil {
2022-11-14 15:29:06 +08:00
log . Error ( "failed to alter alias" ,
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 ( )
2022-09-05 13:29:11 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) , 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 ( ) ) )
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 ( ) ) )
2021-11-19 12:11:12 +08:00
return succStatus ( ) , nil
2021-09-18 11:13:51 +08:00
}
2022-03-11 17:13:59 +08:00
2022-03-31 13:51:28 +08:00
// Import imports large files (json, numpy, etc.) on MinIO/S3 storage into Milvus storage.
2022-03-11 17:13:59 +08:00
func ( c * Core ) Import ( ctx context . Context , req * milvuspb . ImportRequest ) ( * milvuspb . ImportResponse , error ) {
2022-03-21 15:47:23 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
return & milvuspb . ImportResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
2022-03-21 15:47:23 +08:00
} , nil
2022-03-11 17:13:59 +08:00
}
2022-03-31 13:51:28 +08:00
// Get collection/partition ID from collection/partition name.
2022-09-27 10:44:53 +08:00
var colInfo * model . Collection
2022-04-20 14:03:40 +08:00
var err error
2022-09-27 10:44:53 +08:00
if colInfo , err = c . meta . GetCollectionByName ( ctx , req . GetCollectionName ( ) , typeutil . MaxTimestamp ) ; err != nil {
2022-05-09 20:47:52 +08:00
log . Error ( "failed to find collection ID from its name" ,
zap . String ( "collection name" , req . GetCollectionName ( ) ) ,
zap . Error ( err ) )
return nil , err
}
2022-09-27 10:44:53 +08:00
cID := colInfo . CollectionID
2022-09-26 18:06:54 +08:00
req . ChannelNames = c . meta . GetCollectionVirtualChannels ( cID )
if req . GetPartitionName ( ) == "" {
req . PartitionName = Params . CommonCfg . DefaultPartitionName
}
2022-05-09 20:47:52 +08:00
var pID UniqueID
2022-09-05 13:29:11 +08:00
if pID , err = c . meta . GetPartitionByName ( cID , req . GetPartitionName ( ) , typeutil . MaxTimestamp ) ; err != nil {
2022-05-09 20:47:52 +08:00
log . Error ( "failed to get partition ID from its name" ,
zap . String ( "partition name" , req . GetPartitionName ( ) ) ,
zap . Error ( err ) )
2022-04-20 14:03:40 +08:00
return nil , err
}
2022-05-31 15:40:04 +08:00
log . Info ( "RootCoord receive import request" ,
2022-03-31 13:51:28 +08:00
zap . String ( "collection name" , req . GetCollectionName ( ) ) ,
zap . Int64 ( "collection ID" , cID ) ,
zap . String ( "partition name" , req . GetPartitionName ( ) ) ,
2022-09-26 18:06:54 +08:00
zap . Strings ( "virtual channel names" , req . GetChannelNames ( ) ) ,
2022-04-20 14:03:40 +08:00
zap . Int64 ( "partition ID" , pID ) ,
2022-03-31 13:51:28 +08:00
zap . Int ( "# of files = " , len ( req . GetFiles ( ) ) ) ,
)
2022-09-26 18:06:54 +08:00
importJobResp := c . importManager . importJob ( ctx , req , cID , pID )
return importJobResp , nil
2022-03-11 17:13:59 +08:00
}
2022-04-03 11:37:29 +08:00
// GetImportState returns the current state of an import task.
2022-03-11 17:13:59 +08:00
func ( c * Core ) GetImportState ( ctx context . Context , req * milvuspb . GetImportStateRequest ) ( * milvuspb . GetImportStateResponse , error ) {
2022-03-21 15:47:23 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
return & milvuspb . GetImportStateResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
2022-03-21 15:47:23 +08:00
} , nil
}
2022-04-06 15:33:32 +08:00
return c . importManager . getTaskState ( req . GetTask ( ) ) , nil
2022-03-11 17:13:59 +08:00
}
2022-04-25 17:37:46 +08:00
// ListImportTasks returns id array of all import tasks.
func ( c * Core ) ListImportTasks ( ctx context . Context , req * milvuspb . ListImportTasksRequest ) ( * milvuspb . ListImportTasksResponse , error ) {
if code , ok := c . checkHealthy ( ) ; ! ok {
return & milvuspb . ListImportTasksResponse {
2022-10-10 15:55:22 +08:00
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) ,
2022-04-25 17:37:46 +08:00
} , nil
}
2022-11-07 17:11:02 +08:00
colID := int64 ( - 1 )
collectionName := req . GetCollectionName ( )
if len ( collectionName ) != 0 {
// if the collection name is specified but not found, user may input a wrong name, the collection doesn't exist or has been dropped.
// we will return error to notify user the name is incorrect.
colInfo , err := c . meta . GetCollectionByName ( ctx , req . GetCollectionName ( ) , typeutil . MaxTimestamp )
if err != nil {
err = fmt . Errorf ( "failed to find collection ID from its name: '%s', error: %w" , req . GetCollectionName ( ) , err )
log . Error ( "ListImportTasks failed" , zap . Error ( err ) )
return & milvuspb . ListImportTasksResponse {
Status : failStatus ( commonpb . ErrorCode_IllegalCollectionName , err . Error ( ) ) ,
} , nil
}
colID = colInfo . CollectionID
}
// if the collection name is not specified, the colID is -1, listAllTasks will return all tasks
tasks , err := c . importManager . listAllTasks ( colID , req . GetLimit ( ) )
if err != nil {
err = fmt . Errorf ( "failed to list import tasks, collection name: '%s', error: %w" , req . GetCollectionName ( ) , err )
log . Error ( "ListImportTasks failed" , zap . Error ( err ) )
return & milvuspb . ListImportTasksResponse {
Status : failStatus ( commonpb . ErrorCode_UnexpectedError , err . Error ( ) ) ,
} , nil
}
2022-04-25 17:37:46 +08:00
resp := & milvuspb . ListImportTasksResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} ,
2022-11-07 17:11:02 +08:00
Tasks : tasks ,
2022-04-25 17:37:46 +08:00
}
return resp , nil
}
2022-03-31 13:51:28 +08:00
// ReportImport reports import task state to RootCoord.
2022-04-01 11:33:28 +08:00
func ( c * Core ) ReportImport ( ctx context . Context , ir * rootcoordpb . ImportResult ) ( * commonpb . Status , error ) {
2022-05-31 15:40:04 +08:00
log . Info ( "RootCoord receive import state report" ,
2022-04-06 15:33:32 +08:00
zap . Int64 ( "task ID" , ir . GetTaskId ( ) ) ,
zap . Any ( "import state" , ir . GetState ( ) ) )
2022-03-21 15:47:23 +08:00
if code , ok := c . checkHealthy ( ) ; ! ok {
2022-10-10 15:55:22 +08:00
return failStatus ( commonpb . ErrorCode_UnexpectedError , "StateCode=" + commonpb . StateCode_name [ int32 ( code ) ] ) , nil
2022-03-21 15:47:23 +08:00
}
2022-09-26 18:06:54 +08:00
// If setting ImportState_ImportCompleted, simply update the state and return directly.
if ir . GetState ( ) == commonpb . ImportState_ImportCompleted {
if err := c . importManager . setImportTaskState ( ir . GetTaskId ( ) , commonpb . ImportState_ImportCompleted ) ; err != nil {
errMsg := "failed to set import task as ImportState_ImportCompleted"
log . Error ( errMsg , zap . Error ( err ) )
2022-06-15 12:20:10 +08:00
return & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
2022-09-26 18:06:54 +08:00
Reason : fmt . Sprintf ( "%s %s" , errMsg , err . Error ( ) ) ,
2022-06-15 12:20:10 +08:00
} , nil
}
return & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} , nil
}
2022-03-31 13:51:28 +08:00
// Upon receiving ReportImport request, update the related task's state in task store.
2022-09-26 18:06:54 +08:00
ti , err := c . importManager . updateTaskInfo ( ir )
2022-03-21 15:47:23 +08:00
if err != nil {
return & commonpb . Status {
2022-03-31 13:51:28 +08:00
ErrorCode : commonpb . ErrorCode_UpdateImportTaskFailure ,
2022-03-21 15:47:23 +08:00
Reason : err . Error ( ) ,
} , nil
2022-03-11 17:13:59 +08:00
}
2022-04-03 11:37:29 +08:00
2022-05-31 15:40:04 +08:00
// This method update a busy node to idle node, and send import task to idle node
resendTaskFunc := func ( ) {
func ( ) {
c . importManager . busyNodesLock . Lock ( )
defer c . importManager . busyNodesLock . Unlock ( )
delete ( c . importManager . busyNodes , ir . GetDatanodeId ( ) )
2022-09-26 18:06:54 +08:00
log . Info ( "a DataNode is no longer busy after processing task" ,
2022-05-31 15:40:04 +08:00
zap . Int64 ( "dataNode ID" , ir . GetDatanodeId ( ) ) ,
zap . Int64 ( "task ID" , ir . GetTaskId ( ) ) )
} ( )
2022-09-26 18:06:54 +08:00
err := c . importManager . sendOutTasks ( c . importManager . ctx )
if err != nil {
log . Error ( "fail to send out import task to datanodes" )
}
2022-05-31 15:40:04 +08:00
}
// If task failed, send task to idle datanode
if ir . GetState ( ) == commonpb . ImportState_ImportFailed {
2022-09-26 18:06:54 +08:00
// When a DataNode failed importing, remove this DataNode from the busy node list and send out import tasks again.
log . Info ( "an import task has failed, marking DataNode available and resending import task" ,
zap . Int64 ( "task ID" , ir . GetTaskId ( ) ) )
2022-05-31 15:40:04 +08:00
resendTaskFunc ( )
2022-09-26 18:06:54 +08:00
} else if ir . GetState ( ) != commonpb . ImportState_ImportPersisted {
log . Debug ( "unexpected import task state reported, return immediately (this should not happen)" ,
2022-05-05 21:17:50 +08:00
zap . Any ( "task ID" , ir . GetTaskId ( ) ) ,
zap . Any ( "import state" , ir . GetState ( ) ) )
2022-06-17 21:14:11 +08:00
resendTaskFunc ( )
2022-09-26 18:06:54 +08:00
} else {
// Here ir.GetState() == commonpb.ImportState_ImportPersisted
// When a DataNode finishes importing, remove this DataNode from the busy node list and send out import tasks again.
resendTaskFunc ( )
// Flush all import data segments.
if err := c . broker . Flush ( ctx , ti . GetCollectionId ( ) , ir . GetSegments ( ) ) ; err != nil {
2022-10-27 16:21:34 +08:00
log . Error ( "failed to call Flush on bulk insert segments" ,
2022-09-26 18:06:54 +08:00
zap . Int64 ( "task ID" , ir . GetTaskId ( ) ) )
return & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : err . Error ( ) ,
} , nil
}
2022-03-31 13:51:28 +08:00
}
2022-03-11 17:13:59 +08:00
2022-03-21 15:47:23 +08:00
return & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} , nil
2022-03-11 17:13:59 +08:00
}
2022-03-28 16:41:28 +08:00
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 . WithMsgType ( 0 ) , //TODO, msg type
commonpbutil . WithMsgID ( 0 ) , //TODO, msg id
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 . WithMsgType ( 0 ) , //TODO, msg type
commonpbutil . WithMsgID ( 0 ) , //TODO, msg id
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 )
2022-04-11 19:49:34 +08:00
log . Debug ( "CreateCredential" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , credInfo . Username ) )
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 {
2022-06-29 20:02:18 +08:00
log . Error ( "CreateCredential save credential failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-04-11 19:49:34 +08:00
zap . String ( "username" , credInfo . Username ) , 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 failStatus ( commonpb . ErrorCode_CreateCredentialFailure , "CreateCredential failed: " + err . Error ( ) ) , nil
}
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 {
2022-06-29 20:02:18 +08:00
log . Warn ( "CreateCredential add cache failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-04-11 19:49:34 +08:00
zap . String ( "username" , credInfo . Username ) , 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 ( )
return succStatus ( ) , nil
}
// 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 )
2022-04-11 19:49:34 +08:00
log . Debug ( "GetCredential" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , in . Username ) )
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 {
log . Error ( "GetCredential query credential failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , in . Username ) , 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 {
Status : failStatus ( commonpb . ErrorCode_GetCredentialFailure , "GetCredential failed: " + err . Error ( ) ) ,
} , err
}
log . Debug ( "GetCredential success" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , in . 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
return & rootcoordpb . GetCredentialResponse {
Status : succStatus ( ) ,
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 )
2022-04-11 19:49:34 +08:00
log . Debug ( "UpdateCredential" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , credInfo . Username ) )
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 {
2022-06-29 20:02:18 +08:00
log . Error ( "UpdateCredential save credential failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-04-11 19:49:34 +08:00
zap . String ( "username" , credInfo . Username ) , 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 failStatus ( commonpb . ErrorCode_UpdateCredentialFailure , "UpdateCredential failed: " + err . Error ( ) ) , nil
}
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 {
2022-06-29 20:02:18 +08:00
log . Error ( "UpdateCredential update cache failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-04-11 19:49:34 +08:00
zap . String ( "username" , credInfo . Username ) , 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 failStatus ( commonpb . ErrorCode_UpdateCredentialFailure , "UpdateCredential failed: " + err . Error ( ) ) , nil
}
log . Debug ( "UpdateCredential 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
return succStatus ( ) , nil
}
// 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 )
2022-04-11 19:49:34 +08:00
2022-06-29 20:02:18 +08:00
// delete data on storage
2022-09-05 13:29:11 +08:00
err := c . meta . DeleteCredential ( in . Username )
2022-04-11 19:49:34 +08:00
if err != nil {
2022-06-29 20:02:18 +08:00
log . Error ( "DeleteCredential remove credential failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-04-11 19:49:34 +08:00
zap . String ( "username" , in . Username ) , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2022-06-29 20:02:18 +08:00
return failStatus ( commonpb . ErrorCode_DeleteCredentialFailure , "DeleteCredential failed: " + err . Error ( ) ) , err
2022-04-11 19:49:34 +08:00
}
2022-06-29 20:02:18 +08:00
// invalidate proxy's local cache
err = c . ExpireCredCache ( ctx , in . Username )
2022-04-11 19:49:34 +08:00
if err != nil {
2022-06-29 20:02:18 +08:00
log . Error ( "DeleteCredential expire credential cache failed" , zap . String ( "role" , typeutil . RootCoordRole ) ,
2022-04-11 19:49:34 +08:00
zap . String ( "username" , in . Username ) , zap . Error ( err ) )
2022-04-27 23:03:47 +08:00
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . FailLabel ) . Inc ( )
2022-06-29 20:02:18 +08:00
return failStatus ( commonpb . ErrorCode_DeleteCredentialFailure , "DeleteCredential failed: " + err . Error ( ) ) , nil
2022-04-11 19:49:34 +08:00
}
log . Debug ( "DeleteCredential success" , zap . String ( "role" , typeutil . RootCoordRole ) ,
zap . String ( "username" , in . 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 . Dec ( )
return succStatus ( ) , nil
}
// 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 )
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 {
2022-11-14 15:29:06 +08:00
log . Ctx ( ctx ) . Error ( "ListCredUsers query usernames failed" ,
zap . String ( "role" , typeutil . RootCoordRole ) ,
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 & milvuspb . ListCredUsersResponse {
Status : failStatus ( commonpb . ErrorCode_ListCredUsersFailure , "ListCredUsers failed: " + err . Error ( ) ) ,
} , err
}
log . Debug ( "ListCredUsers success" , zap . String ( "role" , typeutil . RootCoordRole ) )
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 {
Status : succStatus ( ) ,
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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return errorutil . UnhealthyStatus ( code ) , errorutil . UnhealthyError ( )
}
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"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_CreateRoleFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" , zap . String ( "role_name" , entity . Name ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
metrics . RootCoordNumOfRoles . Inc ( )
return succStatus ( ) , nil
}
// 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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return errorutil . UnhealthyStatus ( code ) , errorutil . UnhealthyError ( )
}
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . RoleName } , false ) ; err != nil {
2022-09-02 21:12:59 +08:00
errMsg := "the role isn't existed"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_DropRoleFailure , errMsg ) , 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"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_DropRoleFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
2022-09-05 13:29:11 +08:00
roleResults , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . RoleName } , true )
2022-08-04 11:04:34 +08:00
if err != nil {
errMsg := "fail to select a role by role name"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_DropRoleFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( "role to user info" , zap . Int ( "counter" , len ( roleResults ) ) )
for _ , roleResult := range roleResults {
for index , userEntity := range roleResult . Users {
2022-09-05 13:29:11 +08:00
if err = c . meta . OperateUserRole ( util . DefaultTenant ,
2022-09-02 21:12:59 +08:00
& milvuspb . UserEntity { Name : userEntity . Name } ,
& milvuspb . RoleEntity { Name : roleResult . Role . Name } , milvuspb . OperateUserRoleType_RemoveUserFromRole ) ; err != nil {
2022-08-23 10:26:53 +08:00
if common . IsIgnorableError ( err ) {
continue
}
2022-08-04 11:04:34 +08:00
errMsg := "fail to remove user from role"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . String ( "role_name" , roleResult . Role . Name ) , zap . String ( "username" , userEntity . Name ) , zap . Int ( "current_index" , index ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperateUserRoleFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
}
}
2022-09-05 13:29:11 +08:00
if err = c . meta . DropGrant ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . RoleName } ) ; err != nil {
2022-08-26 19:22:56 +08:00
errMsg := "fail to drop the grant"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_DropRoleFailure , errMsg ) , nil
2022-08-26 19:22:56 +08:00
}
2022-09-05 13:29:11 +08:00
if err = c . meta . DropRole ( util . DefaultTenant , in . RoleName ) ; err != nil {
2022-08-04 11:04:34 +08:00
errMsg := "fail to drop the role"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_DropRoleFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" , zap . String ( "role_name" , in . RoleName ) )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
metrics . RootCoordNumOfRoles . Dec ( )
return succStatus ( ) , nil
}
// 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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return errorutil . UnhealthyStatus ( code ) , errorutil . UnhealthyError ( )
}
2022-09-05 13:29:11 +08:00
if _ , err := c . meta . SelectRole ( util . DefaultTenant , & milvuspb . RoleEntity { Name : in . RoleName } , false ) ; err != nil {
2022-08-04 11:04:34 +08:00
errMsg := "fail to check the role name"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperateUserRoleFailure , errMsg ) , nil
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 : in . Username } , false ) ; err != nil {
2022-08-04 11:04:34 +08:00
errMsg := "fail to check the username"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperateUserRoleFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
2022-08-23 10:26:53 +08:00
updateCache := true
2022-09-05 13:29:11 +08:00
if err := c . meta . OperateUserRole ( util . DefaultTenant , & milvuspb . UserEntity { Name : in . Username } , & milvuspb . RoleEntity { Name : in . RoleName } , in . Type ) ; err != nil {
2022-08-23 10:26:53 +08:00
if ! common . IsIgnorableError ( err ) {
errMsg := "fail to operate user to role"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperateUserRoleFailure , errMsg ) , nil
2022-08-23 10:26:53 +08:00
}
updateCache = false
2022-08-04 11:04:34 +08:00
}
2022-08-23 10:26:53 +08:00
if updateCache {
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"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) )
return failStatus ( commonpb . ErrorCode_OperateUserRoleFailure , errMsg ) , 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 {
2022-09-02 21:12:59 +08:00
errMsg := "fail to refresh policy info cache"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperateUserRoleFailure , errMsg ) , nil
2022-08-23 10:26:53 +08:00
}
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return succStatus ( ) , nil
}
// 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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return & milvuspb . SelectRoleResponse { Status : errorutil . UnhealthyStatus ( code ) } , errorutil . UnhealthyError ( )
}
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 {
2022-08-10 17:20:41 +08:00
if common . IsKeyNotExistError ( err ) {
return & milvuspb . SelectRoleResponse {
Status : succStatus ( ) ,
} , nil
}
2022-09-02 21:12:59 +08:00
errMsg := "fail to select the role to check the role name"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectRoleResponse {
Status : failStatus ( commonpb . ErrorCode_SelectRoleFailure , errMsg ) ,
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"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectRoleResponse {
Status : failStatus ( commonpb . ErrorCode_SelectRoleFailure , errMsg ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & milvuspb . SelectRoleResponse {
Status : succStatus ( ) ,
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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return & milvuspb . SelectUserResponse { Status : errorutil . UnhealthyStatus ( code ) } , errorutil . UnhealthyError ( )
}
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 {
2022-08-10 17:20:41 +08:00
if common . IsKeyNotExistError ( err ) {
return & milvuspb . SelectUserResponse {
Status : succStatus ( ) ,
} , nil
}
2022-09-02 21:12:59 +08:00
errMsg := "fail to select the user to check the username"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectUserResponse {
Status : failStatus ( commonpb . ErrorCode_SelectUserFailure , errMsg ) ,
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"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectUserResponse {
Status : failStatus ( commonpb . ErrorCode_SelectUserFailure , errMsg ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & milvuspb . SelectUserResponse {
Status : succStatus ( ) ,
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 {
2022-08-04 11:04:34 +08:00
return err
}
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-09-02 21:12:59 +08:00
return fmt . Errorf ( "the object type in the object entity[name: %s] is invalid" , entity . Name )
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 {
2022-08-04 11:04:34 +08:00
return err
}
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-09-02 21:12:59 +08:00
return fmt . Errorf ( "the privilege name[%s] in the privilege entity is invalid" , 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-09-02 21:12:59 +08:00
return fmt . Errorf ( "the object type[%s] is invalid" , object )
2022-08-04 11:04:34 +08:00
}
for _ , privilege := range privileges {
if privilege == entity . Privilege . Name {
return nil
}
}
2022-09-02 21:12:59 +08:00
return fmt . Errorf ( "the privilege name[%s] is invalid" , 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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return errorutil . UnhealthyStatus ( code ) , errorutil . UnhealthyError ( )
}
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 )
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
if in . Entity == nil {
errMsg := "the grant entity in the request is nil"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , errMsg ) , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidObject ( in . Entity . Object ) ; err != nil {
2022-09-02 21:12:59 +08:00
log . Error ( "" , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , err . Error ( ) ) , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidRole ( in . Entity . Role ) ; err != nil {
2022-09-02 21:12:59 +08:00
log . Error ( "" , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , err . Error ( ) ) , nil
2022-08-04 11:04:34 +08:00
}
if err := c . isValidGrantor ( in . Entity . Grantor , in . Entity . Object . Name ) ; err != nil {
2022-09-02 21:12:59 +08:00
log . Error ( "" , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , err . Error ( ) ) , nil
2022-08-04 11:04:34 +08:00
}
2022-08-05 16:28:35 +08:00
logger . 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 )
}
2022-08-05 16:28:35 +08:00
logger . 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
}
2022-08-23 10:26:53 +08:00
updateCache := true
2022-09-05 13:29:11 +08:00
if err := c . meta . OperatePrivilege ( util . DefaultTenant , in . Entity , in . Type ) ; err != nil {
2022-08-23 10:26:53 +08:00
if ! common . IsIgnorableError ( err ) {
errMsg := "fail to operate the privilege"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , errMsg ) , nil
2022-08-23 10:26:53 +08:00
}
updateCache = false
2022-08-04 11:04:34 +08:00
}
2022-08-23 10:26:53 +08:00
if updateCache {
var opType int32
switch in . Type {
case milvuspb . OperatePrivilegeType_Grant :
opType = int32 ( typeutil . CacheGrantPrivilege )
case milvuspb . OperatePrivilegeType_Revoke :
opType = int32 ( typeutil . CacheRevokePrivilege )
default :
errMsg := "invalid operate type for the OperatePrivilege api"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , errMsg ) , nil
2022-08-23 10:26:53 +08:00
}
if err := c . proxyClientManager . RefreshPolicyInfoCache ( ctx , & proxypb . RefreshPolicyInfoCacheRequest {
OpType : opType ,
OpKey : funcutil . PolicyForPrivilege ( in . Entity . Role . Name , in . Entity . Object . Name , in . Entity . ObjectName , in . Entity . Grantor . Privilege . Name ) ,
} ) ; err != nil {
2022-09-02 21:12:59 +08:00
errMsg := "fail to refresh policy info cache"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
return failStatus ( commonpb . ErrorCode_OperatePrivilegeFailure , errMsg ) , nil
2022-08-23 10:26:53 +08:00
}
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return succStatus ( ) , nil
}
// 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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return & milvuspb . SelectGrantResponse {
Status : errorutil . UnhealthyStatus ( code ) ,
} , errorutil . UnhealthyError ( )
}
if in . Entity == nil {
errMsg := "the grant entity in the request is nil"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
Status : failStatus ( commonpb . ErrorCode_SelectGrantFailure , errMsg ) ,
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 {
2022-09-02 21:12:59 +08:00
log . Error ( "" , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
Status : failStatus ( commonpb . ErrorCode_SelectGrantFailure , err . Error ( ) ) ,
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 {
2022-09-02 21:12:59 +08:00
log . Error ( "" , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
Status : failStatus ( commonpb . ErrorCode_SelectGrantFailure , err . Error ( ) ) ,
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 )
2022-08-23 10:26:53 +08:00
if common . IsKeyNotExistError ( err ) {
return & milvuspb . SelectGrantResponse {
Status : succStatus ( ) ,
} , nil
}
2022-08-04 11:04:34 +08:00
if err != nil {
errMsg := "fail to select the grant"
2022-09-02 21:12:59 +08:00
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & milvuspb . SelectGrantResponse {
Status : failStatus ( commonpb . ErrorCode_SelectGrantFailure , errMsg ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & milvuspb . SelectGrantResponse {
Status : succStatus ( ) ,
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 )
logger . Debug ( method , zap . Any ( "in" , in ) )
if code , ok := c . checkHealthy ( ) ; ! ok {
return & internalpb . ListPolicyResponse {
Status : errorutil . UnhealthyStatus ( code ) ,
} , errorutil . UnhealthyError ( )
}
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"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & internalpb . ListPolicyResponse {
2022-09-02 21:12:59 +08:00
Status : failStatus ( commonpb . ErrorCode_ListPolicyFailure , errMsg ) ,
} , 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"
log . Error ( errMsg , zap . Any ( "in" , in ) , zap . Error ( err ) )
2022-08-04 11:04:34 +08:00
return & internalpb . ListPolicyResponse {
Status : failStatus ( commonpb . ErrorCode_ListPolicyFailure , "fail to list user-role" ) ,
2022-09-02 21:12:59 +08:00
} , nil
2022-08-04 11:04:34 +08:00
}
logger . Debug ( method + " success" )
metrics . RootCoordDDLReqCounter . WithLabelValues ( method , metrics . SuccessLabel ) . Inc ( )
metrics . RootCoordDDLReqLatency . WithLabelValues ( method ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
return & internalpb . ListPolicyResponse {
Status : succStatus ( ) ,
PolicyInfos : policies ,
UserRoles : userRoles ,
} , nil
}
2022-10-18 13:39:26 +08:00
func ( c * Core ) CheckHealth ( ctx context . Context , in * milvuspb . CheckHealthRequest ) ( * milvuspb . CheckHealthResponse , error ) {
if _ , ok := c . checkHealthy ( ) ; ! ok {
reason := errorutil . UnHealthReason ( "rootcoord" , c . session . ServerID , "rootcoord is unhealthy" )
return & milvuspb . CheckHealthResponse { IsHealthy : false , Reasons : [ ] string { reason } } , nil
}
mu := & sync . Mutex { }
group , ctx := errgroup . WithContext ( ctx )
errReasons := make ( [ ] string , 0 , len ( c . proxyClientManager . proxyClient ) )
for nodeID , proxyClient := range c . proxyClientManager . proxyClient {
nodeID := nodeID
proxyClient := proxyClient
group . Go ( func ( ) error {
sta , err := proxyClient . GetComponentStates ( ctx )
isHealthy , reason := errorutil . UnHealthReasonWithComponentStatesOrErr ( "proxy" , nodeID , sta , err )
if ! isHealthy {
mu . Lock ( )
defer mu . Unlock ( )
errReasons = append ( errReasons , reason )
}
return err
} )
}
err := group . Wait ( )
if err != nil || len ( errReasons ) != 0 {
return & milvuspb . CheckHealthResponse { IsHealthy : false , Reasons : errReasons } , nil
}
return & milvuspb . CheckHealthResponse { IsHealthy : true , Reasons : errReasons } , nil
}