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-08-18 14:36:10 +08:00
|
|
|
"strconv"
|
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"
|
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
"github.com/milvus-io/milvus/internal/metastore/db/rootcoord"
|
|
|
|
|
|
|
|
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
|
|
|
|
|
|
|
|
"github.com/milvus-io/milvus/internal/kv"
|
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-05 13:29:11 +08:00
|
|
|
kvmetestore "github.com/milvus-io/milvus/internal/metastore/kv/rootcoord"
|
|
|
|
|
|
|
|
"github.com/milvus-io/milvus/internal/allocator"
|
|
|
|
"github.com/milvus-io/milvus/internal/common"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
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"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
|
|
|
"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"
|
|
|
|
"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)
|
|
|
|
|
2021-01-19 14:44:03 +08:00
|
|
|
// ------------------ struct -----------------------
|
|
|
|
|
2022-04-03 11:37:29 +08:00
|
|
|
var Params paramtable.ComponentParam
|
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
|
|
|
|
meta IMetaTable
|
|
|
|
scheduler IScheduler
|
|
|
|
broker Broker
|
|
|
|
garbageCollector GarbageCollector
|
|
|
|
|
|
|
|
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-09-05 13:29:11 +08:00
|
|
|
idAllocator allocator.GIDAllocator
|
|
|
|
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-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
|
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-04-07 22:05:32 +08:00
|
|
|
ctx: ctx,
|
|
|
|
cancel: cancel,
|
|
|
|
factory: factory,
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-03-12 14:22:09 +08:00
|
|
|
core.UpdateStateCode(internalpb.StateCode_Abnormal)
|
2021-01-19 14:44:03 +08:00
|
|
|
return core, nil
|
|
|
|
}
|
|
|
|
|
2021-09-23 15:10:00 +08:00
|
|
|
// UpdateStateCode update state code
|
2021-03-12 14:22:09 +08:00
|
|
|
func (c *Core) UpdateStateCode(code internalpb.StateCode) {
|
2021-02-23 11:40:30 +08:00
|
|
|
c.stateCode.Store(code)
|
|
|
|
}
|
|
|
|
|
2021-11-19 12:11:12 +08:00
|
|
|
func (c *Core) checkHealthy() (internalpb.StateCode, bool) {
|
2021-08-31 11:45:59 +08:00
|
|
|
code := c.stateCode.Load().(internalpb.StateCode)
|
2021-11-19 12:11:12 +08:00
|
|
|
ok := code == internalpb.StateCode_Healthy
|
|
|
|
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{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_TimeTick,
|
|
|
|
Timestamp: t,
|
|
|
|
SourceID: c.session.ServerID,
|
|
|
|
},
|
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
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-05 13:29:11 +08:00
|
|
|
if ts, err := c.tsoAllocator.GenerateTSO(1); err == nil {
|
|
|
|
err := c.sendTimeTick(ts, "timetick loop")
|
2022-01-14 23:55:34 +08:00
|
|
|
if err != nil {
|
2022-09-05 13:29:11 +08:00
|
|
|
log.Warn("failed to send timetick", zap.Error(err))
|
2022-01-14 23:55:34 +08:00
|
|
|
}
|
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-22 19:08:03 +08:00
|
|
|
func (c *Core) SetNewProxyClient(f func(sess *sessionutil.Session) (types.Proxy, error)) {
|
2022-09-05 13:29:11 +08:00
|
|
|
c.proxyCreator = f
|
2021-05-26 20:14:30 +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-11-23 23:01:15 +08:00
|
|
|
// ExpireMetaCache will call invalidate collection meta cache
|
2022-05-19 10:13:56 +08:00
|
|
|
func (c *Core) ExpireMetaCache(ctx context.Context, collNames []string, collectionID UniqueID, ts typeutil.Timestamp) error {
|
|
|
|
// if collectionID is specified, invalidate all the collection meta cache with the specified collectionID and return
|
|
|
|
if collectionID != InvalidCollectionID {
|
|
|
|
req := proxypb.InvalidateCollMetaCacheRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
Timestamp: ts,
|
|
|
|
SourceID: c.session.ServerID,
|
|
|
|
},
|
|
|
|
CollectionID: collectionID,
|
|
|
|
}
|
|
|
|
return c.proxyClientManager.InvalidateCollectionMetaCache(ctx, &req)
|
|
|
|
}
|
|
|
|
|
|
|
|
// if only collNames are specified, invalidate the collection meta cache with the specified collectionName
|
2021-11-23 23:01:15 +08:00
|
|
|
for _, collName := range collNames {
|
|
|
|
req := proxypb.InvalidateCollMetaCacheRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: 0, //TODO, msg type
|
|
|
|
MsgID: 0, //TODO, msg id
|
|
|
|
Timestamp: ts,
|
|
|
|
SourceID: c.session.ServerID,
|
|
|
|
},
|
|
|
|
CollectionName: collName,
|
|
|
|
}
|
2022-05-19 10:13:56 +08:00
|
|
|
err := c.proxyClientManager.InvalidateCollectionMetaCache(ctx, &req)
|
|
|
|
if err != nil {
|
|
|
|
// TODO: try to expire all or directly return err?
|
|
|
|
return err
|
|
|
|
}
|
2021-11-23 23:01:15 +08:00
|
|
|
}
|
2022-05-19 10:13:56 +08:00
|
|
|
return nil
|
2021-11-23 23:01:15 +08:00
|
|
|
}
|
|
|
|
|
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()
|
|
|
|
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
|
|
|
|
|
|
|
c.UpdateStateCode(internalpb.StateCode_Healthy)
|
2021-12-15 11:47:10 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
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
|
|
|
}
|
2021-12-29 14:35:21 +08:00
|
|
|
c.session.Init(typeutil.RootCoordRole, Params.RootCoordCfg.Address, true, true)
|
2022-02-07 10:09:45 +08:00
|
|
|
Params.SetLogger(c.session.ServerID)
|
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-09-05 13:29:11 +08:00
|
|
|
if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, snapshotsSep, Params.EtcdCfg.MetaRootPath, snapshotPrefix); err != nil {
|
|
|
|
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(),
|
|
|
|
f.NewGetCollectionNameFunc(),
|
|
|
|
)
|
|
|
|
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)
|
|
|
|
|
|
|
|
c.factory.Init(&Params)
|
|
|
|
|
|
|
|
chanMap := c.meta.ListCollectionPhysicalChannels()
|
|
|
|
c.chanTimeTick = newTimeTickSync(c.ctx, c.session.ServerID, c.factory, chanMap)
|
|
|
|
c.chanTimeTick.addSession(c.session)
|
|
|
|
c.proxyClientManager = newProxyClientManager(c.proxyCreator)
|
|
|
|
|
|
|
|
c.broker = newServerBroker(c)
|
|
|
|
c.garbageCollector = newGarbageCollectorCtx(c)
|
|
|
|
|
|
|
|
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()
|
|
|
|
|
|
|
|
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-05 13:29:11 +08:00
|
|
|
c.wg.Add(5)
|
|
|
|
go c.tsLoop()
|
|
|
|
go c.startTimeTickLoop()
|
|
|
|
go c.chanTimeTick.startWatch(&c.wg)
|
|
|
|
go c.importManager.expireOldTasksLoop(&c.wg, c.broker.ReleaseSegRefLock)
|
|
|
|
go c.importManager.sendOutTasksLoop(&c.wg)
|
|
|
|
|
|
|
|
c.scheduler.Start()
|
|
|
|
|
|
|
|
Params.RootCoordCfg.CreatedTime = time.Now()
|
|
|
|
Params.RootCoordCfg.UpdatedTime = time.Now()
|
|
|
|
|
|
|
|
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-04-03 11:37:29 +08:00
|
|
|
// Stop stops rootCoord.
|
2021-01-19 14:44:03 +08:00
|
|
|
func (c *Core) Stop() error {
|
2021-12-07 18:39:03 +08:00
|
|
|
c.UpdateStateCode(internalpb.StateCode_Abnormal)
|
|
|
|
|
2021-01-19 14:44:03 +08:00
|
|
|
c.cancel()
|
2021-09-17 12:37:50 +08:00
|
|
|
c.wg.Wait()
|
2021-11-16 22:31:14 +08:00
|
|
|
// wait at most one second to revoke
|
|
|
|
c.session.Revoke(time.Second)
|
2021-01-19 14:44:03 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-09-23 15:10:00 +08:00
|
|
|
// GetComponentStates get states of components
|
2021-03-12 14:22:09 +08:00
|
|
|
func (c *Core) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
|
|
|
code := c.stateCode.Load().(internalpb.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
|
|
|
|
}
|
|
|
|
|
2021-03-12 14:22:09 +08:00
|
|
|
return &internalpb.ComponentStates{
|
|
|
|
State: &internalpb.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: "",
|
|
|
|
},
|
2021-03-12 14:22:09 +08:00
|
|
|
SubcomponentStates: []*internalpb.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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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-09-05 13:29:11 +08:00
|
|
|
log.Info("received request to create collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &createCollectionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to create collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to create collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("name", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to create collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("name", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to drop collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &dropCollectionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to drop collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to drop collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("name", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to drop collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()),
|
|
|
|
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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
|
|
|
|
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-05 13:29:11 +08:00
|
|
|
log.Info("received request to has collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &hasCollectionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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
|
|
|
},
|
2022-09-05 13:29:11 +08:00
|
|
|
Req: in,
|
|
|
|
Rsp: &milvuspb.BoolResponse{},
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
if err := c.scheduler.AddTask(t); err != nil {
|
|
|
|
log.Error("failed to enqueue request to has collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
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 enqueue request to has collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection name", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
2022-04-27 23:03:47 +08:00
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.FailLabel).Inc()
|
2021-01-19 14:44:03 +08:00
|
|
|
return &milvuspb.BoolResponse{
|
2021-11-22 16:01:14 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasCollection failed: "+err.Error()),
|
2021-11-19 12:11:12 +08:00
|
|
|
Value: false,
|
2021-01-19 14:44:03 +08:00
|
|
|
}, nil
|
|
|
|
}
|
2021-11-19 12:11:12 +08:00
|
|
|
|
2022-04-27 23:03:47 +08:00
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.SuccessLabel).Inc()
|
|
|
|
metrics.RootCoordDDLReqLatency.WithLabelValues("HasCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
log.Info("done to has collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection name", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()),
|
|
|
|
zap.Bool("exist", t.Rsp.GetValue()))
|
|
|
|
return t.Rsp, nil
|
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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode"+internalpb.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-05 13:29:11 +08:00
|
|
|
log.Info("received request to describe collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection name", in.GetCollectionName()), zap.Int64("id", in.GetCollectionID()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &describeCollectionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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,
|
|
|
|
Rsp: &milvuspb.DescribeCollectionResponse{},
|
|
|
|
}
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
if err := c.scheduler.AddTask(t); err != nil {
|
|
|
|
log.Error("failed to enqueue request to describe collection",
|
|
|
|
zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection name", in.GetCollectionName()), zap.Int64("id", in.GetCollectionID()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.FailLabel).Inc()
|
|
|
|
return &milvuspb.DescribeCollectionResponse{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "DescribeCollection failed: "+err.Error()),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to describe collection",
|
|
|
|
zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection name", in.GetCollectionName()), zap.Int64("id", in.GetCollectionID()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
2022-04-27 23:03:47 +08:00
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.FailLabel).Inc()
|
2021-01-19 14:44:03 +08:00
|
|
|
return &milvuspb.DescribeCollectionResponse{
|
2021-11-22 16:01:14 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "DescribeCollection failed: "+err.Error()),
|
2021-01-19 14:44:03 +08:00
|
|
|
}, nil
|
|
|
|
}
|
2021-11-19 12:11:12 +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
|
|
|
|
|
|
|
log.Info("done to describe collection", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection name", in.GetCollectionName()), zap.Int64("id", in.GetCollectionID()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
2021-01-19 14:44:03 +08:00
|
|
|
return t.Rsp, nil
|
|
|
|
}
|
|
|
|
|
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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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-05 13:29:11 +08:00
|
|
|
log.Info("received request to show collections", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("dbname", in.GetDbName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &showCollectionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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,
|
2021-11-19 12:11:12 +08:00
|
|
|
Rsp: &milvuspb.ShowCollectionsResponse{},
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
if err := c.scheduler.AddTask(t); err != nil {
|
|
|
|
log.Error("failed to enqueue request to show collections", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("dbname", in.GetDbName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.FailLabel).Inc()
|
|
|
|
return &milvuspb.ShowCollectionsResponse{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowCollections failed: "+err.Error()),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to show collections", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("dbname", in.GetDbName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
2022-04-27 23:03:47 +08:00
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.FailLabel).Inc()
|
2021-03-12 14:22:09 +08:00
|
|
|
return &milvuspb.ShowCollectionsResponse{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowCollections failed: "+err.Error()),
|
2021-01-19 14:44:03 +08:00
|
|
|
}, nil
|
|
|
|
}
|
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
|
|
|
|
|
|
|
log.Info("done to show collections", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("dbname", in.GetDbName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()),
|
|
|
|
zap.Int("num of collections", len(t.Rsp.GetCollectionNames()))) // maybe very large, print number instead.
|
2021-01-19 14:44:03 +08:00
|
|
|
return t.Rsp, 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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to create partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &createPartitionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to create partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to create partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to create partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to drop partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &dropPartitionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to drop partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to drop partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to drop partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
|
|
|
|
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-05 13:29:11 +08:00
|
|
|
log.Info("received request to has partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &hasPartitionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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
|
|
|
},
|
2022-09-05 13:29:11 +08:00
|
|
|
Req: in,
|
|
|
|
Rsp: &milvuspb.BoolResponse{},
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
if err := c.scheduler.AddTask(t); err != nil {
|
|
|
|
log.Error("failed to enqueue request to has partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.FailLabel).Inc()
|
|
|
|
return &milvuspb.BoolResponse{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasPartition failed: "+err.Error()),
|
|
|
|
Value: false,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to has partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
2022-04-27 23:03:47 +08:00
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.FailLabel).Inc()
|
2021-01-19 14:44:03 +08:00
|
|
|
return &milvuspb.BoolResponse{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasPartition failed: "+err.Error()),
|
|
|
|
Value: false,
|
2021-01-19 14:44:03 +08:00
|
|
|
}, nil
|
|
|
|
}
|
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
|
|
|
|
|
|
|
log.Info("done to has partition", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()),
|
|
|
|
zap.Bool("exist", t.Rsp.GetValue()))
|
|
|
|
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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to show partitions", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &showPartitionTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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,
|
2021-11-19 12:11:12 +08:00
|
|
|
Rsp: &milvuspb.ShowPartitionsResponse{},
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2022-09-05 13:29:11 +08:00
|
|
|
|
|
|
|
if err := c.scheduler.AddTask(t); err != nil {
|
|
|
|
log.Error("failed to enqueue request to show partitions", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
2022-04-27 23:03:47 +08:00
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.FailLabel).Inc()
|
2021-03-12 14:22:09 +08:00
|
|
|
return &milvuspb.ShowPartitionsResponse{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowPartitions failed: "+err.Error()),
|
2021-01-19 14:44:03 +08:00
|
|
|
}, nil
|
|
|
|
}
|
2021-11-19 12:11:12 +08:00
|
|
|
|
2022-09-05 13:29:11 +08:00
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to show partitions", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.FailLabel).Inc()
|
|
|
|
return &milvuspb.ShowPartitionsResponse{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowPartitions failed: "+err.Error()),
|
|
|
|
}, nil
|
|
|
|
}
|
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
|
|
|
|
|
|
|
log.Info("done to show partitions", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()),
|
|
|
|
zap.Strings("partitions", t.Rsp.GetPartitionNames()))
|
2021-01-19 14:44:03 +08:00
|
|
|
return t.Rsp, nil
|
|
|
|
}
|
|
|
|
|
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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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-05 13:29:11 +08:00
|
|
|
log.Error("failed to allocate timestamp", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
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{
|
2021-11-19 12:11:12 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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-09-05 13:29:11 +08:00
|
|
|
log.Error("failed to allocate id", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
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) {
|
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))
|
2021-11-19 12:11:12 +08:00
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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-01-15 18:53:34 +08:00
|
|
|
log.Warn("failed to updateTimeTick", zap.String("role", typeutil.RootCoordRole),
|
2021-11-22 16:01:14 +08:00
|
|
|
zap.Int64("msgID", in.Base.MsgID), 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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
|
|
|
|
}
|
|
|
|
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-08-12 13:20:39 +08:00
|
|
|
//ShowConfigurations returns the configurations of RootCoord matching req.Pattern
|
|
|
|
func (c *Core) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
|
|
|
|
if code, ok := c.checkHealthy(); !ok {
|
|
|
|
return &internalpb.ShowConfigurationsResponse{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
|
|
|
|
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{
|
2021-11-22 16:01:14 +08:00
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
}
|
|
|
|
|
2021-12-13 09:59:27 +08:00
|
|
|
log.Debug("GetMetrics success", zap.String("role", typeutil.RootCoordRole),
|
2022-07-21 20:52:27 +08:00
|
|
|
zap.String("metric_type", metricType), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
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
|
|
|
|
}
|
|
|
|
|
2021-12-13 09:59:27 +08:00
|
|
|
log.Warn("GetSystemInfoMetrics from cache failed", zap.String("role", typeutil.RootCoordRole),
|
2022-07-21 20:52:27 +08:00
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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-07-21 20:52:27 +08:00
|
|
|
log.Warn("GetSystemInfoMetrics failed", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("metric_type", metricType), zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Error(err))
|
|
|
|
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),
|
|
|
|
zap.String("metric_type", metricType), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to create alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &createAliasTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to create alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to create alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to create alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to drop alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &dropAliasTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to drop alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to drop alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("alias", in.GetAlias()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to drop alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("alias", in.GetAlias()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.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
|
|
|
|
|
|
|
log.Info("received request to alter alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
t := &alterAliasTask{
|
|
|
|
baseTaskV2: baseTaskV2{
|
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 {
|
|
|
|
log.Error("failed to enqueue request to alter alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()))
|
|
|
|
|
|
|
|
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterAlias", metrics.FailLabel).Inc()
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
|
|
log.Error("failed to alter alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
|
|
|
|
|
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
|
|
|
|
|
|
|
log.Info("done to alter alias", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
|
|
|
|
zap.Int64("msgID", in.GetBase().GetMsgID()), 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{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
|
|
|
|
}, 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-05-09 20:47:52 +08:00
|
|
|
var cID UniqueID
|
2022-04-20 14:03:40 +08:00
|
|
|
var err error
|
2022-09-05 13:29:11 +08:00
|
|
|
if cID, err = c.meta.GetCollectionIDByName(req.GetCollectionName()); 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
|
|
|
|
}
|
|
|
|
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-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-04-20 14:03:40 +08:00
|
|
|
zap.Bool("row-based", req.GetRowBased()),
|
2022-03-31 13:51:28 +08:00
|
|
|
)
|
2022-04-20 14:03:40 +08:00
|
|
|
resp := c.importManager.importJob(ctx, req, cID, pID)
|
2022-03-11 17:13:59 +08:00
|
|
|
return resp, nil
|
|
|
|
}
|
|
|
|
|
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{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
|
|
|
|
}, 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{
|
|
|
|
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
resp := &milvuspb.ListImportTasksResponse{
|
|
|
|
Status: &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_Success,
|
|
|
|
},
|
|
|
|
Tasks: c.importManager.listAllTasks(),
|
|
|
|
}
|
|
|
|
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 {
|
|
|
|
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
|
|
|
|
}
|
2022-06-15 12:20:10 +08:00
|
|
|
// Special case for ImportState_ImportAllocSegment state, where we shall only add segment ref lock and do no other
|
|
|
|
// operations.
|
|
|
|
// TODO: This is inelegant and must get re-structured.
|
|
|
|
if ir.GetState() == commonpb.ImportState_ImportAllocSegment {
|
|
|
|
// Lock the segments, so we don't lose track of them when compaction happens.
|
|
|
|
// Note that these locks will be unlocked in c.postImportPersistLoop() -> checkSegmentLoadedLoop().
|
2022-09-05 13:29:11 +08:00
|
|
|
if err := c.broker.AddSegRefLock(ctx, ir.GetTaskId(), ir.GetSegments()); err != nil {
|
2022-06-15 12:20:10 +08:00
|
|
|
log.Error("failed to acquire segment ref lock", zap.Error(err))
|
|
|
|
return &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
|
|
|
Reason: fmt.Sprintf("failed to acquire segment ref lock %s", err.Error()),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
// Update task store with new segments.
|
|
|
|
c.importManager.appendTaskSegments(ir.GetTaskId(), ir.GetSegments())
|
|
|
|
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-04-01 11:33:28 +08:00
|
|
|
ti, err := c.importManager.updateTaskState(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())
|
|
|
|
log.Info("DataNode is no longer busy",
|
|
|
|
zap.Int64("dataNode ID", ir.GetDatanodeId()),
|
|
|
|
zap.Int64("task ID", ir.GetTaskId()))
|
|
|
|
|
|
|
|
}()
|
|
|
|
c.importManager.sendOutTasks(c.importManager.ctx)
|
|
|
|
}
|
|
|
|
|
|
|
|
// If task failed, send task to idle datanode
|
|
|
|
if ir.GetState() == commonpb.ImportState_ImportFailed {
|
2022-06-15 12:20:10 +08:00
|
|
|
// Release segments when task fails.
|
|
|
|
log.Info("task failed, release segment ref locks")
|
|
|
|
err := retry.Do(ctx, func() error {
|
2022-09-05 13:29:11 +08:00
|
|
|
return c.broker.ReleaseSegRefLock(ctx, ir.GetTaskId(), ir.GetSegments())
|
2022-06-15 12:20:10 +08:00
|
|
|
}, retry.Attempts(100))
|
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to release lock, about to panic!")
|
|
|
|
panic(err)
|
|
|
|
}
|
2022-05-31 15:40:04 +08:00
|
|
|
resendTaskFunc()
|
|
|
|
}
|
|
|
|
|
2022-05-05 21:17:50 +08:00
|
|
|
// So much for reporting, unless the task just reached `ImportPersisted` state.
|
|
|
|
if ir.GetState() != commonpb.ImportState_ImportPersisted {
|
|
|
|
log.Debug("non import-persisted state received, return immediately",
|
|
|
|
zap.Any("task ID", ir.GetTaskId()),
|
|
|
|
zap.Any("import state", ir.GetState()))
|
2022-04-03 11:37:29 +08:00
|
|
|
return &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_Success,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
2022-05-09 20:47:52 +08:00
|
|
|
// Look up collection name on collection ID.
|
2022-03-31 13:51:28 +08:00
|
|
|
var colName string
|
2022-07-22 10:20:29 +08:00
|
|
|
var colMeta *model.Collection
|
2022-09-05 13:29:11 +08:00
|
|
|
if colMeta, err = c.meta.GetCollectionByID(ctx, ti.GetCollectionId(), typeutil.MaxTimestamp); err != nil {
|
2022-05-09 20:47:52 +08:00
|
|
|
log.Error("failed to get collection name",
|
|
|
|
zap.Int64("collection ID", ti.GetCollectionId()),
|
|
|
|
zap.Error(err))
|
2022-06-17 21:14:11 +08:00
|
|
|
// In some unexpected cases, user drop collection when bulkload task still in pending list, the datanode become idle.
|
|
|
|
// If we directly return, the pending tasks will remain in pending list. So we call resendTaskFunc() to push next pending task to idle datanode.
|
|
|
|
resendTaskFunc()
|
2022-03-31 13:51:28 +08:00
|
|
|
return &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_CollectionNameNotFound,
|
2022-05-09 20:47:52 +08:00
|
|
|
Reason: "failed to get collection name for collection ID" + strconv.FormatInt(ti.GetCollectionId(), 10),
|
2022-03-31 13:51:28 +08:00
|
|
|
}, nil
|
|
|
|
}
|
2022-07-22 10:20:29 +08:00
|
|
|
colName = colMeta.Name
|
2022-03-31 13:51:28 +08:00
|
|
|
|
2022-05-31 15:40:04 +08:00
|
|
|
// When DataNode has done its thing, remove it from the busy node list. And send import task again
|
|
|
|
resendTaskFunc()
|
2022-04-01 11:33:28 +08:00
|
|
|
|
2022-05-05 21:17:50 +08:00
|
|
|
// Flush all import data segments.
|
2022-09-05 13:29:11 +08:00
|
|
|
c.broker.Flush(ctx, ti.GetCollectionId(), ir.GetSegments())
|
2022-05-05 21:17:50 +08:00
|
|
|
// Check if data are "queryable" and if indices are built on all segments.
|
|
|
|
go c.postImportPersistLoop(c.ctx, ir.GetTaskId(), ti.GetCollectionId(), colName, ir.GetSegments())
|
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-06-15 12:20:10 +08:00
|
|
|
// CountCompleteIndex checks indexing status of the given segments.
|
|
|
|
// It returns an error if error occurs. It also returns a boolean indicating whether indexing is done (or if no index
|
|
|
|
// is needed).
|
2022-03-28 16:41:28 +08:00
|
|
|
func (c *Core) CountCompleteIndex(ctx context.Context, collectionName string, collectionID UniqueID,
|
2022-06-15 12:20:10 +08:00
|
|
|
allSegmentIDs []UniqueID) (bool, error) {
|
|
|
|
// Note: Index name is always Params.CommonCfg.DefaultIndexName in current Milvus designs as of today.
|
2022-03-28 16:41:28 +08:00
|
|
|
indexName := Params.CommonCfg.DefaultIndexName
|
|
|
|
|
2022-09-05 13:29:11 +08:00
|
|
|
states, err := c.broker.GetSegmentIndexState(ctx, collectionID, indexName, allSegmentIDs)
|
2022-03-28 16:41:28 +08:00
|
|
|
if err != nil {
|
2022-03-31 13:51:28 +08:00
|
|
|
log.Error("failed to get index state in checkSegmentIndexStates", zap.Error(err))
|
2022-06-15 12:20:10 +08:00
|
|
|
return false, err
|
2022-03-28 16:41:28 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Count the # of segments with finished index.
|
|
|
|
ct := 0
|
|
|
|
for _, s := range states {
|
|
|
|
if s.State == commonpb.IndexState_Finished {
|
|
|
|
ct++
|
|
|
|
}
|
|
|
|
}
|
2022-03-31 13:51:28 +08:00
|
|
|
log.Info("segment indexing state checked",
|
2022-08-25 15:48:54 +08:00
|
|
|
//zap.Int64s("segments checked", seg2Check),
|
|
|
|
//zap.Int("# of checked segment", len(seg2Check)),
|
2022-03-28 16:41:28 +08:00
|
|
|
zap.Int("# of segments with complete index", ct),
|
|
|
|
zap.String("collection name", collectionName),
|
|
|
|
zap.Int64("collection ID", collectionID),
|
|
|
|
)
|
2022-08-25 15:48:54 +08:00
|
|
|
return len(allSegmentIDs) == ct, nil
|
2022-03-28 16:41:28 +08:00
|
|
|
}
|
2022-03-31 13:51:28 +08:00
|
|
|
|
2022-05-05 21:17:50 +08:00
|
|
|
func (c *Core) postImportPersistLoop(ctx context.Context, taskID int64, colID int64, colName string, segIDs []UniqueID) {
|
|
|
|
// Loop and check if segments are loaded in queryNodes.
|
|
|
|
c.wg.Add(1)
|
2022-06-15 12:20:10 +08:00
|
|
|
go c.checkSegmentLoadedLoop(ctx, taskID, colID, segIDs)
|
2022-05-05 21:17:50 +08:00
|
|
|
// Check if collection has any indexed fields. If so, start a loop to check segments' index states.
|
2022-09-05 13:29:11 +08:00
|
|
|
if _, err := c.meta.GetCollectionByID(ctx, colID, typeutil.MaxTimestamp); err != nil {
|
2022-05-09 20:47:52 +08:00
|
|
|
log.Error("failed to find meta for collection",
|
2022-06-15 12:20:10 +08:00
|
|
|
zap.Int64("collection ID", colID),
|
|
|
|
zap.Error(err))
|
|
|
|
} else {
|
|
|
|
log.Info("start checking index state", zap.Int64("collection ID", colID))
|
2022-05-05 21:17:50 +08:00
|
|
|
c.wg.Add(1)
|
2022-06-15 12:20:10 +08:00
|
|
|
go c.checkCompleteIndexLoop(ctx, taskID, colID, colName, segIDs)
|
2022-05-05 21:17:50 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// checkSegmentLoadedLoop loops and checks if all segments in `segIDs` are loaded in queryNodes.
|
|
|
|
func (c *Core) checkSegmentLoadedLoop(ctx context.Context, taskID int64, colID int64, segIDs []UniqueID) {
|
2022-03-31 13:51:28 +08:00
|
|
|
defer c.wg.Done()
|
2022-05-05 21:17:50 +08:00
|
|
|
ticker := time.NewTicker(time.Duration(Params.RootCoordCfg.ImportSegmentStateCheckInterval*1000) * time.Millisecond)
|
2022-04-03 11:37:29 +08:00
|
|
|
defer ticker.Stop()
|
2022-05-05 21:17:50 +08:00
|
|
|
expireTicker := time.NewTicker(time.Duration(Params.RootCoordCfg.ImportSegmentStateWaitLimit*1000) * time.Millisecond)
|
2022-04-03 11:37:29 +08:00
|
|
|
defer expireTicker.Stop()
|
2022-06-15 12:20:10 +08:00
|
|
|
defer func() {
|
|
|
|
log.Info("we are done checking segment loading state, release segment ref locks")
|
|
|
|
err := retry.Do(ctx, func() error {
|
2022-09-05 13:29:11 +08:00
|
|
|
return c.broker.ReleaseSegRefLock(ctx, taskID, segIDs)
|
2022-06-15 12:20:10 +08:00
|
|
|
}, retry.Attempts(100))
|
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to release lock, about to panic!")
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}()
|
2022-03-31 13:51:28 +08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-c.ctx.Done():
|
2022-05-05 21:17:50 +08:00
|
|
|
log.Info("(in check segment loaded loop) context done, exiting checkSegmentLoadedLoop")
|
2022-03-31 13:51:28 +08:00
|
|
|
return
|
|
|
|
case <-ticker.C:
|
2022-09-05 13:29:11 +08:00
|
|
|
resp, err := c.broker.GetQuerySegmentInfo(ctx, colID, segIDs)
|
2022-06-15 12:20:10 +08:00
|
|
|
log.Debug("(in check segment loaded loop)",
|
|
|
|
zap.Int64("task ID", taskID),
|
|
|
|
zap.Int64("collection ID", colID),
|
|
|
|
zap.Int64s("segment IDs expected", segIDs),
|
|
|
|
zap.Int("# of segments found", len(resp.GetInfos())))
|
2022-05-05 21:17:50 +08:00
|
|
|
if err != nil {
|
2022-05-06 21:35:51 +08:00
|
|
|
log.Warn("(in check segment loaded loop) failed to call get segment info on queryCoord",
|
|
|
|
zap.Int64("task ID", taskID),
|
2022-05-05 21:17:50 +08:00
|
|
|
zap.Int64("collection ID", colID),
|
|
|
|
zap.Int64s("segment IDs", segIDs))
|
2022-06-15 12:20:10 +08:00
|
|
|
} else if len(resp.GetInfos()) == len(segIDs) {
|
2022-05-05 21:17:50 +08:00
|
|
|
// Check if all segment info are loaded in queryNodes.
|
2022-05-06 21:35:51 +08:00
|
|
|
log.Info("(in check segment loaded loop) all import data segments loaded in queryNodes",
|
|
|
|
zap.Int64("task ID", taskID),
|
2022-05-05 21:17:50 +08:00
|
|
|
zap.Int64("collection ID", colID),
|
|
|
|
zap.Int64s("segment IDs", segIDs))
|
2022-05-12 19:23:53 +08:00
|
|
|
c.importManager.setTaskDataQueryable(taskID)
|
2022-03-31 13:51:28 +08:00
|
|
|
return
|
|
|
|
}
|
|
|
|
case <-expireTicker.C:
|
2022-05-05 21:17:50 +08:00
|
|
|
log.Warn("(in check segment loaded loop) segments still not loaded after max wait time",
|
|
|
|
zap.Int64("task ID", taskID),
|
|
|
|
zap.Int64("collection ID", colID),
|
|
|
|
zap.Int64s("segment IDs", segIDs))
|
2022-03-31 13:51:28 +08:00
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-05 21:17:50 +08:00
|
|
|
// checkCompleteIndexLoop loops and checks if all indices are built for an import task's segments.
|
|
|
|
func (c *Core) checkCompleteIndexLoop(ctx context.Context, taskID int64, colID int64, colName string, segIDs []UniqueID) {
|
|
|
|
defer c.wg.Done()
|
|
|
|
ticker := time.NewTicker(time.Duration(Params.RootCoordCfg.ImportIndexCheckInterval*1000) * time.Millisecond)
|
|
|
|
defer ticker.Stop()
|
|
|
|
expireTicker := time.NewTicker(time.Duration(Params.RootCoordCfg.ImportIndexWaitLimit*1000) * time.Millisecond)
|
|
|
|
defer expireTicker.Stop()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-c.ctx.Done():
|
|
|
|
log.Info("(in check complete index loop) context done, exiting checkCompleteIndexLoop")
|
|
|
|
return
|
|
|
|
case <-ticker.C:
|
2022-06-15 12:20:10 +08:00
|
|
|
if done, err := c.CountCompleteIndex(ctx, colName, colID, segIDs); err == nil && done {
|
|
|
|
log.Info("(in check complete index loop) indices are built or no index needed",
|
2022-05-06 21:35:51 +08:00
|
|
|
zap.Int64("task ID", taskID))
|
2022-05-12 19:23:53 +08:00
|
|
|
c.importManager.setTaskDataIndexed(taskID)
|
2022-05-05 21:17:50 +08:00
|
|
|
return
|
2022-06-15 12:20:10 +08:00
|
|
|
} else if err != nil {
|
|
|
|
log.Error("(in check complete index loop) an error occurs",
|
|
|
|
zap.Error(err))
|
2022-05-05 21:17:50 +08:00
|
|
|
}
|
|
|
|
case <-expireTicker.C:
|
|
|
|
log.Warn("(in check complete index loop) indexing is taken too long",
|
|
|
|
zap.Int64("task ID", taskID),
|
|
|
|
zap.Int64("collection ID", colID),
|
|
|
|
zap.Int64s("segment IDs", segIDs))
|
|
|
|
return
|
|
|
|
}
|
2022-03-31 13:51: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{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: 0, //TODO, msg type
|
|
|
|
MsgID: 0, //TODO, msg id
|
|
|
|
SourceID: c.session.ServerID,
|
|
|
|
},
|
|
|
|
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{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: 0, //TODO, msg type
|
|
|
|
MsgID: 0, //TODO, msg id
|
|
|
|
SourceID: c.session.ServerID,
|
|
|
|
},
|
|
|
|
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
|
|
|
|
// 1. decode ciphertext password to raw password
|
|
|
|
// 2. encrypt raw password
|
|
|
|
// 3. save in to etcd
|
|
|
|
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 {
|
|
|
|
log.Error("ListCredUsers query usernames failed", zap.String("role", typeutil.RootCoordRole),
|
|
|
|
zap.Int64("msgID", in.Base.MsgID), 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
|
|
|
|
}
|