Support embedded etcd (#14333)

Signed-off-by: xiaofan-luan <xiaofan.luan@zilliz.com>
This commit is contained in:
Xiaofan 2021-12-29 14:35:21 +08:00 committed by GitHub
parent 673806b1c1
commit 69087ff8dd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
90 changed files with 1371 additions and 621 deletions

View File

@ -34,10 +34,7 @@ type DataCoord struct {
// NewDataCoord creates a new DataCoord // NewDataCoord creates a new DataCoord
func NewDataCoord(ctx context.Context, factory msgstream.Factory) (*DataCoord, error) { func NewDataCoord(ctx context.Context, factory msgstream.Factory) (*DataCoord, error) {
s, err := grpcdatacoordclient.NewServer(ctx, factory) s := grpcdatacoordclient.NewServer(ctx, factory)
if err != nil {
return nil, err
}
return &DataCoord{ return &DataCoord{
ctx: ctx, ctx: ctx,

View File

@ -232,7 +232,7 @@ func main() {
os.Exit(-1) os.Exit(-1)
} }
var localMsg = false var local = false
role := roles.MilvusRoles{} role := roles.MilvusRoles{}
switch serverType { switch serverType {
case typeutil.RootCoordRole: case typeutil.RootCoordRole:
@ -260,7 +260,7 @@ func main() {
role.EnableDataNode = true role.EnableDataNode = true
role.EnableIndexCoord = true role.EnableIndexCoord = true
role.EnableIndexNode = true role.EnableIndexNode = true
localMsg = true local = true
case roleMixture: case roleMixture:
role.EnableRootCoord = enableRootCoord role.EnableRootCoord = enableRootCoord
role.EnableQueryCoord = enableQueryCoord role.EnableQueryCoord = enableQueryCoord
@ -291,7 +291,7 @@ func main() {
panic(err) panic(err)
} }
defer removePidFile(fd) defer removePidFile(fd)
role.Run(localMsg, svrAlias) role.Run(local, svrAlias)
case "stop": case "stop":
if err := stopPid(filename, runtimeDir); err != nil { if err := stopPid(filename, runtimeDir); err != nil {
fmt.Fprintf(os.Stderr, "%s\n\n", err.Error()) fmt.Fprintf(os.Stderr, "%s\n\n", err.Error())

View File

@ -43,13 +43,17 @@ import (
"github.com/milvus-io/milvus/internal/querycoord" "github.com/milvus-io/milvus/internal/querycoord"
"github.com/milvus-io/milvus/internal/querynode" "github.com/milvus-io/milvus/internal/querynode"
"github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/rootcoord"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/healthz" "github.com/milvus-io/milvus/internal/util/healthz"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/rocksmq/server/rocksmq" "github.com/milvus-io/milvus/internal/util/rocksmq/server/rocksmq"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
var Params paramtable.GlobalParamTable
func newMsgFactory(localMsg bool) msgstream.Factory { func newMsgFactory(localMsg bool) msgstream.Factory {
if localMsg { if localMsg {
return msgstream.NewRmsFactory() return msgstream.NewRmsFactory()
@ -338,7 +342,7 @@ func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, alias st
} }
// Run Milvus components. // Run Milvus components.
func (mr *MilvusRoles) Run(localMsg bool, alias string) { func (mr *MilvusRoles) Run(local bool, alias string) {
if os.Getenv(metricsinfo.DeployModeEnvKey) == metricsinfo.StandaloneDeployMode { if os.Getenv(metricsinfo.DeployModeEnvKey) == metricsinfo.StandaloneDeployMode {
closer := trace.InitTracing("standalone") closer := trace.InitTracing("standalone")
if closer != nil { if closer != nil {
@ -349,7 +353,8 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
// only standalone enable localMsg // only standalone enable localMsg
if localMsg { if local {
Params.Init()
if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode); err != nil { if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode); err != nil {
log.Error("Failed to set deploy mode: ", zap.Error(err)) log.Error("Failed to set deploy mode: ", zap.Error(err))
} }
@ -358,6 +363,12 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
panic(err) panic(err)
} }
defer stopRocksmq() defer stopRocksmq()
if Params.BaseParams.UseEmbedEtcd {
// start etcd server
etcd.InitEtcdServer(&Params.BaseParams)
defer etcd.StopEtcdServer()
}
} else { } else {
if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode); err != nil { if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode); err != nil {
log.Error("Failed to set deploy mode: ", zap.Error(err)) log.Error("Failed to set deploy mode: ", zap.Error(err))
@ -366,7 +377,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var rc *components.RootCoord var rc *components.RootCoord
if mr.EnableRootCoord { if mr.EnableRootCoord {
rc = mr.runRootCoord(ctx, localMsg) rc = mr.runRootCoord(ctx, local)
if rc != nil { if rc != nil {
defer rc.Stop() defer rc.Stop()
} }
@ -375,7 +386,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var pn *components.Proxy var pn *components.Proxy
if mr.EnableProxy { if mr.EnableProxy {
pctx := logutil.WithModule(ctx, "Proxy") pctx := logutil.WithModule(ctx, "Proxy")
pn = mr.runProxy(pctx, localMsg, alias) pn = mr.runProxy(pctx, local, alias)
if pn != nil { if pn != nil {
defer pn.Stop() defer pn.Stop()
} }
@ -383,7 +394,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var qs *components.QueryCoord var qs *components.QueryCoord
if mr.EnableQueryCoord { if mr.EnableQueryCoord {
qs = mr.runQueryCoord(ctx, localMsg) qs = mr.runQueryCoord(ctx, local)
if qs != nil { if qs != nil {
defer qs.Stop() defer qs.Stop()
} }
@ -391,7 +402,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var qn *components.QueryNode var qn *components.QueryNode
if mr.EnableQueryNode { if mr.EnableQueryNode {
qn = mr.runQueryNode(ctx, localMsg, alias) qn = mr.runQueryNode(ctx, local, alias)
if qn != nil { if qn != nil {
defer qn.Stop() defer qn.Stop()
} }
@ -399,7 +410,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var ds *components.DataCoord var ds *components.DataCoord
if mr.EnableDataCoord { if mr.EnableDataCoord {
ds = mr.runDataCoord(ctx, localMsg) ds = mr.runDataCoord(ctx, local)
if ds != nil { if ds != nil {
defer ds.Stop() defer ds.Stop()
} }
@ -407,7 +418,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var dn *components.DataNode var dn *components.DataNode
if mr.EnableDataNode { if mr.EnableDataNode {
dn = mr.runDataNode(ctx, localMsg, alias) dn = mr.runDataNode(ctx, local, alias)
if dn != nil { if dn != nil {
defer dn.Stop() defer dn.Stop()
} }
@ -415,7 +426,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var is *components.IndexCoord var is *components.IndexCoord
if mr.EnableIndexCoord { if mr.EnableIndexCoord {
is = mr.runIndexCoord(ctx, localMsg) is = mr.runIndexCoord(ctx, local)
if is != nil { if is != nil {
defer is.Stop() defer is.Stop()
} }
@ -423,13 +434,13 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
var in *components.IndexNode var in *components.IndexNode
if mr.EnableIndexNode { if mr.EnableIndexNode {
in = mr.runIndexNode(ctx, localMsg, alias) in = mr.runIndexNode(ctx, local, alias)
if in != nil { if in != nil {
defer in.Stop() defer in.Stop()
} }
} }
if localMsg { if local {
standaloneHealthzHandler := func(w http.ResponseWriter, r *http.Request) { standaloneHealthzHandler := func(w http.ResponseWriter, r *http.Request) {
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel() defer cancel()

View File

@ -10,6 +10,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -27,11 +28,14 @@ var (
func main() { func main() {
flag.Parse() flag.Parse()
etcdkv, err := etcdkv.NewEtcdKV([]string{*etcdAddr}, *rootPath)
etcdCli, err := etcd.GetRemoteEtcdClient([]string{*etcdAddr})
if err != nil { if err != nil {
log.Fatal("failed to connect to etcd", zap.Error(err)) log.Fatal("failed to connect to etcd", zap.Error(err))
} }
etcdkv := etcdkv.NewEtcdKV(etcdCli, *rootPath)
keys, values, err := etcdkv.LoadWithPrefix("/") keys, values, err := etcdkv.LoadWithPrefix("/")
if err != nil { if err != nil {
log.Fatal("failed to list ", zap.Error(err)) log.Fatal("failed to list ", zap.Error(err))

View File

@ -17,24 +17,24 @@
package allocator package allocator
import ( import (
"os"
"strings"
"testing" "testing"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
var gTestIDAllocator *GlobalIDAllocator var gTestIDAllocator *GlobalIDAllocator
var Params paramtable.GlobalParamTable
func TestGlobalTSOAllocator_All(t *testing.T) { func TestGlobalTSOAllocator_All(t *testing.T) {
endpoints := os.Getenv("ETCD_ENDPOINTS") Params.Init()
if endpoints == "" { etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
endpoints = "localhost:2379"
}
etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "gidTest")
assert.NoError(t, err) assert.NoError(t, err)
defer etcdCli.Close()
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "gidTest")
gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", etcdKV) gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", etcdKV)

View File

@ -45,6 +45,7 @@ import (
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/minio/minio-go/v7" "github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials" "github.com/minio/minio-go/v7/pkg/credentials"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -82,7 +83,7 @@ const (
) )
type dataNodeCreatorFunc func(ctx context.Context, addr string) (types.DataNode, error) type dataNodeCreatorFunc func(ctx context.Context, addr string) (types.DataNode, error)
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdClient *clientv3.Client) (types.RootCoord, error)
// makes sure Server implements `DataCoord` // makes sure Server implements `DataCoord`
var _ types.DataCoord = (*Server)(nil) var _ types.DataCoord = (*Server)(nil)
@ -100,6 +101,7 @@ type Server struct {
isServing ServerState isServing ServerState
helper ServerHelper helper ServerHelper
etcdCli *clientv3.Client
kvClient *etcdkv.EtcdKV kvClient *etcdkv.EtcdKV
meta *meta meta *meta
segmentManager Manager segmentManager Manager
@ -177,7 +179,7 @@ func SetSegmentManager(manager Manager) Option {
} }
// CreateServer creates a `Server` instance // CreateServer creates a `Server` instance
func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option) (*Server, error) { func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option) *Server {
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
s := &Server{ s := &Server{
ctx: ctx, ctx: ctx,
@ -194,15 +196,15 @@ func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option
for _, opt := range opts { for _, opt := range opts {
opt(s) opt(s)
} }
return s, nil return s
} }
func defaultDataNodeCreatorFunc(ctx context.Context, addr string) (types.DataNode, error) { func defaultDataNodeCreatorFunc(ctx context.Context, addr string) (types.DataNode, error) {
return datanodeclient.NewClient(ctx, addr) return datanodeclient.NewClient(ctx, addr)
} }
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, client *clientv3.Client) (types.RootCoord, error) {
return rootcoordclient.NewClient(ctx, metaRootPath, etcdEndpoints) return rootcoordclient.NewClient(ctx, metaRootPath, client)
} }
// QuitSignal returns signal when server quits // QuitSignal returns signal when server quits
@ -219,17 +221,19 @@ func (s *Server) Register() error {
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err)) logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if s.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
return nil return nil
} }
func (s *Server) initSession() error { func (s *Server) initSession() error {
s.session = sessionutil.NewSession(s.ctx, Params.DataCoordCfg.MetaRootPath, Params.DataCoordCfg.EtcdEndpoints) s.session = sessionutil.NewSession(s.ctx, Params.DataCoordCfg.MetaRootPath, s.etcdCli)
if s.session == nil { if s.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
s.session.Init(typeutil.DataCoordRole, Params.DataCoordCfg.Address, true) s.session.Init(typeutil.DataCoordRole, Params.DataCoordCfg.Address, true, true)
Params.DataCoordCfg.NodeID = s.session.ServerID Params.DataCoordCfg.NodeID = s.session.ServerID
Params.BaseParams.SetLogger(Params.DataCoordCfg.NodeID) Params.BaseParams.SetLogger(Params.DataCoordCfg.NodeID)
return nil return nil
@ -311,6 +315,11 @@ func (s *Server) initCluster() error {
return nil return nil
} }
// SetEtcdClient sets etcd client for datacoord.
func (s *Server) SetEtcdClient(client *clientv3.Client) {
s.etcdCli = client
}
func (s *Server) createCompactionHandler() { func (s *Server) createCompactionHandler() {
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh) s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh)
s.compactionHandler.start() s.compactionHandler.start()
@ -407,20 +416,17 @@ func (s *Server) startSegmentManager() {
} }
func (s *Server) initMeta() error { func (s *Server) initMeta() error {
connectEtcdFn := func() error { etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.DataCoordCfg.MetaRootPath)
etcdKV, err := etcdkv.NewEtcdKV(Params.DataCoordCfg.EtcdEndpoints, Params.DataCoordCfg.MetaRootPath) s.kvClient = etcdKV
if err != nil { reloadEtcdFn := func() error {
return err var err error
}
s.kvClient = etcdKV
s.meta, err = newMeta(s.kvClient) s.meta, err = newMeta(s.kvClient)
if err != nil { if err != nil {
return err return err
} }
return nil return nil
} }
return retry.Do(s.ctx, connectEtcdFn, retry.Attempts(connEtcdMaxRetryTime)) return retry.Do(s.ctx, reloadEtcdFn, retry.Attempts(connEtcdMaxRetryTime))
} }
func (s *Server) startServerLoop() { func (s *Server) startServerLoop() {
@ -724,7 +730,7 @@ func (s *Server) handleFlushingSegments(ctx context.Context) {
func (s *Server) initRootCoordClient() error { func (s *Server) initRootCoordClient() error {
var err error var err error
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.DataCoordCfg.MetaRootPath, Params.DataCoordCfg.EtcdEndpoints); err != nil { if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.DataCoordCfg.MetaRootPath, s.etcdCli); err != nil {
return err return err
} }
if err = s.rootCoordClient.Init(); err != nil { if err = s.rootCoordClient.Init(); err != nil {

View File

@ -37,8 +37,8 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -613,8 +613,7 @@ func TestGetFlushedSegments(t *testing.T) {
func TestService_WatchServices(t *testing.T) { func TestService_WatchServices(t *testing.T) {
factory := msgstream.NewPmsFactory() factory := msgstream.NewPmsFactory()
svr, err := CreateServer(context.TODO(), factory) svr := CreateServer(context.TODO(), factory)
assert.Nil(t, err)
svr.serverLoopWg.Add(1) svr.serverLoopWg.Add(1)
ech := make(chan *sessionutil.SessionEvent) ech := make(chan *sessionutil.SessionEvent)
@ -1522,7 +1521,7 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil return newMockRootCoordService(), nil
} }
@ -1565,7 +1564,7 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil return newMockRootCoordService(), nil
} }
@ -1593,7 +1592,7 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil return newMockRootCoordService(), nil
} }
@ -1621,7 +1620,7 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil return newMockRootCoordService(), nil
} }
@ -1699,7 +1698,7 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil return newMockRootCoordService(), nil
} }
@ -1954,7 +1953,7 @@ func TestOptions(t *testing.T) {
t.Run("SetRootCoordCreator", func(t *testing.T) { t.Run("SetRootCoordCreator", func(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, endpoints []string) (types.RootCoord, error) { var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, etcdClient *clientv3.Client) (types.RootCoord, error) {
return nil, errors.New("dummy") return nil, errors.New("dummy")
} }
opt := SetRootCoordCreator(crt) opt := SetRootCoordCreator(crt)
@ -1992,8 +1991,7 @@ func TestOptions(t *testing.T) {
factory := msgstream.NewPmsFactory() factory := msgstream.NewPmsFactory()
svr, err := CreateServer(context.TODO(), factory, opt) svr := CreateServer(context.TODO(), factory, opt)
assert.Nil(t, err)
dn, err := svr.dataNodeCreator(context.Background(), "") dn, err := svr.dataNodeCreator(context.Background(), "")
assert.Nil(t, dn) assert.Nil(t, dn)
assert.Nil(t, err) assert.Nil(t, err)
@ -2246,18 +2244,18 @@ func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Se
err = factory.SetParams(m) err = factory.SetParams(m)
assert.Nil(t, err) assert.Nil(t, err)
etcdCli, err := initEtcd(Params.DataCoordCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(Params.DataCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.DataCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
svr, err := CreateServer(context.TODO(), factory, opts...) svr := CreateServer(context.TODO(), factory, opts...)
assert.Nil(t, err) svr.SetEtcdClient(etcdCli)
svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) { svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) {
return newMockDataNodeClient(0, receiveCh) return newMockDataNodeClient(0, receiveCh)
} }
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil return newMockRootCoordService(), nil
} }
assert.Nil(t, err) assert.Nil(t, err)
@ -2276,20 +2274,3 @@ func closeTestServer(t *testing.T, svr *Server) {
err = svr.CleanMeta() err = svr.CleanMeta()
assert.Nil(t, err) assert.Nil(t, err)
} }
func initEtcd(etcdEndpoints []string) (*clientv3.Client, error) {
var etcdCli *clientv3.Client
connectEtcdFn := func() error {
etcd, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints, DialTimeout: 5 * time.Second})
if err != nil {
return err
}
etcdCli = etcd
return nil
}
err := retry.Do(context.TODO(), connectEtcdFn, retry.Attempts(300))
if err != nil {
return nil, err
}
return etcdCli, nil
}

View File

@ -83,6 +83,7 @@ var Params paramtable.GlobalParamTable
// services in datanode package. // services in datanode package.
// //
// DataNode implements `types.Component`, `types.DataNode` interfaces. // DataNode implements `types.Component`, `types.DataNode` interfaces.
// `etcdCli` is a connection of etcd
// `rootCoord` is a grpc client of root coordinator. // `rootCoord` is a grpc client of root coordinator.
// `dataCoord` is a grpc client of data service. // `dataCoord` is a grpc client of data service.
// `NodeID` is unique to each datanode. // `NodeID` is unique to each datanode.
@ -109,6 +110,7 @@ type DataNode struct {
segmentCache *Cache segmentCache *Cache
compactionExecutor *compactionExecutor compactionExecutor *compactionExecutor
etcdCli *clientv3.Client
rootCoord types.RootCoord rootCoord types.RootCoord
dataCoord types.DataCoord dataCoord types.DataCoord
@ -144,6 +146,11 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
return node return node
} }
// Set etcd client
func (node *DataNode) SetEtcdClient(etcdCli *clientv3.Client) {
node.etcdCli = etcdCli
}
// SetRootCoord sets RootCoord's grpc client, error is returned if repeatedly set. // SetRootCoord sets RootCoord's grpc client, error is returned if repeatedly set.
func (node *DataNode) SetRootCoord(rc types.RootCoord) error { func (node *DataNode) SetRootCoord(rc types.RootCoord) error {
switch { switch {
@ -182,18 +189,20 @@ func (node *DataNode) Register() error {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if node.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
return nil return nil
} }
func (node *DataNode) initSession() error { func (node *DataNode) initSession() error {
node.session = sessionutil.NewSession(node.ctx, Params.DataNodeCfg.MetaRootPath, Params.DataNodeCfg.EtcdEndpoints) node.session = sessionutil.NewSession(node.ctx, Params.DataNodeCfg.MetaRootPath, node.etcdCli)
if node.session == nil { if node.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
node.session.Init(typeutil.DataNodeRole, Params.DataNodeCfg.IP+":"+strconv.Itoa(Params.DataNodeCfg.Port), false) node.session.Init(typeutil.DataNodeRole, Params.DataNodeCfg.IP+":"+strconv.Itoa(Params.DataNodeCfg.Port), false, true)
Params.DataNodeCfg.NodeID = node.session.ServerID Params.DataNodeCfg.NodeID = node.session.ServerID
node.NodeID = node.session.ServerID node.NodeID = node.session.ServerID
Params.BaseParams.SetLogger(Params.DataNodeCfg.NodeID) Params.BaseParams.SetLogger(Params.DataNodeCfg.NodeID)
@ -411,10 +420,7 @@ func (node *DataNode) Start() error {
} }
connectEtcdFn := func() error { connectEtcdFn := func() error {
etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.DataNodeCfg.MetaRootPath)
if err != nil {
return err
}
node.watchKv = etcdKV node.watchKv = etcdKV
return nil return nil
} }

View File

@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
@ -62,7 +63,11 @@ func TestMain(t *testing.M) {
func TestDataNode(t *testing.T) { func TestDataNode(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
node := newIDLEDataNodeMock(ctx) node := newIDLEDataNodeMock(ctx)
err := node.Init() etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
defer etcdCli.Close()
node.SetEtcdClient(etcdCli)
err = node.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = node.Start() err = node.Start()
assert.Nil(t, err) assert.Nil(t, err)
@ -164,6 +169,7 @@ func TestDataNode(t *testing.T) {
dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments" dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments"
node1 := newIDLEDataNodeMock(context.TODO()) node1 := newIDLEDataNodeMock(context.TODO())
node1.SetEtcdClient(etcdCli)
err = node1.Init() err = node1.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = node1.Start() err = node1.Start()
@ -475,7 +481,11 @@ func TestDataNode(t *testing.T) {
func TestWatchChannel(t *testing.T) { func TestWatchChannel(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
node := newIDLEDataNodeMock(ctx) node := newIDLEDataNodeMock(ctx)
err := node.Init() etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
defer etcdCli.Close()
node.SetEtcdClient(etcdCli)
err = node.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = node.Start() err = node.Start()
assert.Nil(t, err) assert.Nil(t, err)
@ -485,8 +495,7 @@ func TestWatchChannel(t *testing.T) {
defer cancel() defer cancel()
t.Run("test watch channel", func(t *testing.T) { t.Run("test watch channel", func(t *testing.T) {
kv, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath) kv := etcdkv.NewEtcdKV(etcdCli, Params.DataNodeCfg.MetaRootPath)
require.NoError(t, err)
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid" oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, oldInvalidCh) path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, oldInvalidCh)
err = kv.Save(path, string([]byte{23})) err = kv.Save(path, string([]byte{23}))

View File

@ -42,6 +42,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/util/etcd"
) )
const ctxTimeInMillisecond = 5000 const ctxTimeInMillisecond = 5000
@ -106,10 +107,11 @@ func makeNewChannelNames(names []string, suffix string) []string {
} }
func clearEtcd(rootPath string) error { func clearEtcd(rootPath string) error {
etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, rootPath) client, err := etcd.GetEtcdClient(&Params.BaseParams)
if err != nil { if err != nil {
return err return err
} }
etcdKV := etcdkv.NewEtcdKV(client, rootPath)
err = etcdKV.RemoveWithPrefix("writer/segment") err = etcdKV.RemoveWithPrefix("writer/segment")
if err != nil { if err != nil {

View File

@ -28,6 +28,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
@ -62,7 +63,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
rootcoordpb.RegisterRootCoordServer(grpcServer, rootCoord) rootcoordpb.RegisterRootCoordServer(grpcServer, rootCoord)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.RootCoordRole, "127.0.0.1:9999", true) session.Init(typeutil.RootCoordRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
rootCoord, ok := cm.GetRootCoordClient() rootCoord, ok := cm.GetRootCoordClient()
@ -79,7 +80,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
querypb.RegisterQueryCoordServer(grpcServer, queryCoord) querypb.RegisterQueryCoordServer(grpcServer, queryCoord)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.QueryCoordRole, "127.0.0.1:9999", true) session.Init(typeutil.QueryCoordRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
queryCoord, ok := cm.GetQueryCoordClient() queryCoord, ok := cm.GetQueryCoordClient()
@ -96,7 +97,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
datapb.RegisterDataCoordServer(grpcServer, dataCoord) datapb.RegisterDataCoordServer(grpcServer, dataCoord)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.DataCoordRole, "127.0.0.1:9999", true) session.Init(typeutil.DataCoordRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
dataCoord, ok := cm.GetDataCoordClient() dataCoord, ok := cm.GetDataCoordClient()
@ -113,7 +114,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
indexpb.RegisterIndexCoordServer(grpcServer, indexCoord) indexpb.RegisterIndexCoordServer(grpcServer, indexCoord)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.IndexCoordRole, "127.0.0.1:9999", true) session.Init(typeutil.IndexCoordRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
indexCoord, ok := cm.GetIndexCoordClient() indexCoord, ok := cm.GetIndexCoordClient()
@ -130,7 +131,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
querypb.RegisterQueryNodeServer(grpcServer, queryNode) querypb.RegisterQueryNodeServer(grpcServer, queryNode)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.QueryNodeRole, "127.0.0.1:9999", true) session.Init(typeutil.QueryNodeRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
queryNodes, ok := cm.GetQueryNodeClients() queryNodes, ok := cm.GetQueryNodeClients()
@ -147,7 +148,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
datapb.RegisterDataNodeServer(grpcServer, dataNode) datapb.RegisterDataNodeServer(grpcServer, dataNode)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.DataNodeRole, "127.0.0.1:9999", true) session.Init(typeutil.DataNodeRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
dataNodes, ok := cm.GetDataNodeClients() dataNodes, ok := cm.GetDataNodeClients()
@ -164,7 +165,7 @@ func TestConnectionManager(t *testing.T) {
defer grpcServer.Stop() defer grpcServer.Stop()
indexpb.RegisterIndexNodeServer(grpcServer, indexNode) indexpb.RegisterIndexNodeServer(grpcServer, indexNode)
go grpcServer.Serve(lis) go grpcServer.Serve(lis)
session.Init(typeutil.IndexNodeRole, "127.0.0.1:9999", true) session.Init(typeutil.IndexNodeRole, "127.0.0.1:9999", true, false)
session.Register() session.Register()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
indexNodes, ok := cm.GetIndexNodeClients() indexNodes, ok := cm.GetIndexNodeClients()
@ -250,6 +251,10 @@ func initSession(ctx context.Context) *sessionutil.Session {
log.Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath)) log.Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath))
log.Debug("etcdPoints", zap.Any("etcdPoints", etcdEndpoints)) log.Debug("etcdPoints", zap.Any("etcdPoints", etcdEndpoints))
session := sessionutil.NewSession(ctx, metaRootPath, etcdEndpoints) etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
if err != nil {
panic(err)
}
session := sessionutil.NewSession(ctx, metaRootPath, etcdCli)
return session return session
} }

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
@ -43,8 +44,8 @@ type Client struct {
} }
// NewClient creates a new client instance // NewClient creates a new client instance
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
if sess == nil { if sess == nil {
err := fmt.Errorf("new session error, maybe can not connect to etcd") err := fmt.Errorf("new session error, maybe can not connect to etcd")
log.Debug("DataCoordClient NewClient failed", zap.Error(err)) log.Debug("DataCoordClient NewClient failed", zap.Error(err))

View File

@ -21,9 +21,9 @@ import (
"errors" "errors"
"testing" "testing"
"github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/mock"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
@ -32,7 +32,9 @@ func Test_NewClient(t *testing.T) {
proxy.Params.InitOnce() proxy.Params.InitOnce()
ctx := context.Background() ctx := context.Background()
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
assert.Nil(t, err)
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, client) assert.NotNil(t, client)

View File

@ -36,10 +36,12 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -54,7 +56,9 @@ type Server struct {
cancel context.CancelFunc cancel context.CancelFunc
wg sync.WaitGroup wg sync.WaitGroup
dataCoord types.DataCoord dataCoord types.DataCoordComponent
etcdCli *clientv3.Client
grpcErrChan chan error grpcErrChan chan error
grpcServer *grpc.Server grpcServer *grpc.Server
@ -62,8 +66,7 @@ type Server struct {
} }
// NewServer new data service grpc server // NewServer new data service grpc server
func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) (*Server, error) { func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) *Server {
var err error
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
s := &Server{ s := &Server{
@ -71,11 +74,8 @@ func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord
cancel: cancel, cancel: cancel,
grpcErrChan: make(chan error), grpcErrChan: make(chan error),
} }
s.dataCoord, err = datacoord.CreateServer(s.ctx, factory, opts...) s.dataCoord = datacoord.CreateServer(s.ctx, factory, opts...)
if err != nil { return s
return nil, err
}
return s, nil
} }
func (s *Server) init() error { func (s *Server) init() error {
@ -89,7 +89,15 @@ func (s *Server) init() error {
datacoord.Params.DataCoordCfg.Port = Params.Port datacoord.Params.DataCoordCfg.Port = Params.Port
datacoord.Params.DataCoordCfg.Address = Params.GetAddress() datacoord.Params.DataCoordCfg.Address = Params.GetAddress()
err := s.startGrpc() etcdCli, err := etcd.GetEtcdClient(&datacoord.Params.BaseParams)
if err != nil {
log.Debug("DataCoord connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.dataCoord.SetEtcdClient(etcdCli)
err = s.startGrpc()
if err != nil { if err != nil {
log.Debug("DataCoord startGrpc failed", zap.Error(err)) log.Debug("DataCoord startGrpc failed", zap.Error(err))
return err return err
@ -178,6 +186,9 @@ func (s *Server) Stop() error {
} }
s.cancel() s.cancel()
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.grpcServer != nil { if s.grpcServer != nil {
log.Debug("Graceful stop grpc server...") log.Debug("Graceful stop grpc server...")
s.grpcServer.GracefulStop() s.grpcServer.GracefulStop()

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
@ -72,6 +73,9 @@ func (m *MockDataCoord) Register() error {
return m.regErr return m.regErr
} }
func (m *MockDataCoord) SetEtcdClient(etcdClient *clientv3.Client) {
}
func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
return m.states, m.err return m.states, m.err
} }
@ -163,13 +167,12 @@ func (m *MockDataCoord) DropVirtualChannel(ctx context.Context, req *datapb.Drop
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func Test_NewServer(t *testing.T) { func Test_NewServer(t *testing.T) {
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server := NewServer(ctx, nil)
assert.Nil(t, err)
assert.NotNil(t, server) assert.NotNil(t, server)
t.Run("Run", func(t *testing.T) { t.Run("Run", func(t *testing.T) {
server.dataCoord = &MockDataCoord{} server.dataCoord = &MockDataCoord{}
err = server.Run() err := server.Run()
assert.Nil(t, err) assert.Nil(t, err)
}) })
@ -335,21 +338,20 @@ func Test_NewServer(t *testing.T) {
assert.NotNil(t, resp) assert.NotNil(t, resp)
}) })
err = server.Stop() err := server.Stop()
assert.Nil(t, err) assert.Nil(t, err)
} }
func Test_Run(t *testing.T) { func Test_Run(t *testing.T) {
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server := NewServer(ctx, nil)
assert.Nil(t, err)
assert.NotNil(t, server) assert.NotNil(t, server)
server.dataCoord = &MockDataCoord{ server.dataCoord = &MockDataCoord{
regErr: errors.New("error"), regErr: errors.New("error"),
} }
err = server.Run() err := server.Run()
assert.Error(t, err) assert.Error(t, err)
server.dataCoord = &MockDataCoord{ server.dataCoord = &MockDataCoord{

View File

@ -37,10 +37,12 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -56,14 +58,14 @@ type Server struct {
grpcServer *grpc.Server grpcServer *grpc.Server
ctx context.Context ctx context.Context
cancel context.CancelFunc cancel context.CancelFunc
etcdCli *clientv3.Client
msFactory msgstream.Factory msFactory msgstream.Factory
rootCoord types.RootCoord rootCoord types.RootCoord
dataCoord types.DataCoord dataCoord types.DataCoord
newRootCoordClient func(string, []string) (types.RootCoord, error) newRootCoordClient func(string, *clientv3.Client) (types.RootCoord, error)
newDataCoordClient func(string, []string) (types.DataCoord, error) newDataCoordClient func(string, *clientv3.Client) (types.DataCoord, error)
closer io.Closer closer io.Closer
} }
@ -76,11 +78,11 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
cancel: cancel, cancel: cancel,
msFactory: factory, msFactory: factory,
grpcErrChan: make(chan error), grpcErrChan: make(chan error),
newRootCoordClient: func(etcdMetaRoot string, etcdEndpoints []string) (types.RootCoord, error) { newRootCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.RootCoord, error) {
return rcc.NewClient(ctx1, etcdMetaRoot, etcdEndpoints) return rcc.NewClient(ctx1, etcdMetaRoot, client)
}, },
newDataCoordClient: func(etcdMetaRoot string, etcdEndpoints []string) (types.DataCoord, error) { newDataCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.DataCoord, error) {
return dcc.NewClient(ctx1, etcdMetaRoot, etcdEndpoints) return dcc.NewClient(ctx1, etcdMetaRoot, client)
}, },
} }
@ -131,6 +133,10 @@ func (s *Server) startGrpcLoop(listener net.Listener) {
} }
func (s *Server) SetEtcdClient(client *clientv3.Client) {
s.datanode.SetEtcdClient(client)
}
func (s *Server) SetRootCoordInterface(ms types.RootCoord) error { func (s *Server) SetRootCoordInterface(ms types.RootCoord) error {
return s.datanode.SetRootCoord(ms) return s.datanode.SetRootCoord(ms)
} }
@ -162,6 +168,9 @@ func (s *Server) Stop() error {
} }
} }
s.cancel() s.cancel()
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.grpcServer != nil { if s.grpcServer != nil {
log.Debug("Graceful stop grpc server...") log.Debug("Graceful stop grpc server...")
// make graceful stop has a timeout // make graceful stop has a timeout
@ -198,12 +207,19 @@ func (s *Server) init() error {
dn.Params.DataNodeCfg.Port = Params.Port dn.Params.DataNodeCfg.Port = Params.Port
dn.Params.DataNodeCfg.IP = Params.IP dn.Params.DataNodeCfg.IP = Params.IP
etcdCli, err := etcd.GetEtcdClient(&dn.Params.BaseParams)
if err != nil {
log.Debug("DataNode connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.SetEtcdClient(s.etcdCli)
closer := trace.InitTracing(fmt.Sprintf("data_node ip: %s, port: %d", Params.IP, Params.Port)) closer := trace.InitTracing(fmt.Sprintf("data_node ip: %s, port: %d", Params.IP, Params.Port))
s.closer = closer s.closer = closer
addr := Params.IP + ":" + strconv.Itoa(Params.Port) addr := Params.IP + ":" + strconv.Itoa(Params.Port)
log.Debug("DataNode address", zap.String("address", addr)) log.Debug("DataNode address", zap.String("address", addr))
err := s.startGrpc() err = s.startGrpc()
if err != nil { if err != nil {
return err return err
} }
@ -211,7 +227,7 @@ func (s *Server) init() error {
// --- RootCoord Client --- // --- RootCoord Client ---
if s.newRootCoordClient != nil { if s.newRootCoordClient != nil {
log.Debug("Init root coord client ...") log.Debug("Init root coord client ...")
rootCoordClient, err := s.newRootCoordClient(dn.Params.DataNodeCfg.MetaRootPath, dn.Params.DataNodeCfg.EtcdEndpoints) rootCoordClient, err := s.newRootCoordClient(dn.Params.DataNodeCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("DataNode newRootCoordClient failed", zap.Error(err)) log.Debug("DataNode newRootCoordClient failed", zap.Error(err))
panic(err) panic(err)
@ -238,7 +254,7 @@ func (s *Server) init() error {
// --- Data Server Client --- // --- Data Server Client ---
if s.newDataCoordClient != nil { if s.newDataCoordClient != nil {
log.Debug("DataNode Init data service client ...") log.Debug("DataNode Init data service client ...")
dataCoordClient, err := s.newDataCoordClient(dn.Params.DataNodeCfg.MetaRootPath, dn.Params.DataNodeCfg.EtcdEndpoints) dataCoordClient, err := s.newDataCoordClient(dn.Params.DataNodeCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("DataNode newDataCoordClient failed", zap.Error(err)) log.Debug("DataNode newDataCoordClient failed", zap.Error(err))
panic(err) panic(err)

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
@ -107,6 +108,9 @@ func (m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPla
return m.status, m.err return m.status, m.err
} }
func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) {
}
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type mockDataCoord struct { type mockDataCoord struct {
types.DataCoord types.DataCoord
@ -174,11 +178,11 @@ func Test_NewServer(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, server) assert.NotNil(t, server)
server.newRootCoordClient = func(string, []string) (types.RootCoord, error) { server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) {
return &mockRootCoord{}, nil return &mockRootCoord{}, nil
} }
server.newDataCoordClient = func(string, []string) (types.DataCoord, error) { server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) {
return &mockDataCoord{}, nil return &mockDataCoord{}, nil
} }
@ -247,11 +251,11 @@ func Test_Run(t *testing.T) {
regErr: errors.New("error"), regErr: errors.New("error"),
} }
server.newRootCoordClient = func(string, []string) (types.RootCoord, error) { server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) {
return &mockRootCoord{}, nil return &mockRootCoord{}, nil
} }
server.newDataCoordClient = func(string, []string) (types.DataCoord, error) { server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) {
return &mockDataCoord{}, nil return &mockDataCoord{}, nil
} }

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
@ -43,8 +44,8 @@ type Client struct {
} }
// NewClient creates a new IndexCoord client. // NewClient creates a new IndexCoord client.
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
if sess == nil { if sess == nil {
err := fmt.Errorf("new session error, maybe can not connect to etcd") err := fmt.Errorf("new session error, maybe can not connect to etcd")
log.Debug("IndexCoordClient NewClient failed", zap.Error(err)) log.Debug("IndexCoordClient NewClient failed", zap.Error(err))

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -42,7 +43,9 @@ func TestIndexCoordClient(t *testing.T) {
err = server.Run() err = server.Run()
assert.Nil(t, err) assert.Nil(t, err)
icc, err := NewClient(ctx, indexcoord.Params.IndexCoordCfg.MetaRootPath, indexcoord.Params.IndexCoordCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.BaseParams)
assert.Nil(t, err)
icc, err := NewClient(ctx, indexcoord.Params.IndexCoordCfg.MetaRootPath, etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, icc) assert.NotNil(t, icc)

View File

@ -32,10 +32,12 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -49,7 +51,7 @@ type UniqueID = typeutil.UniqueID
// Server is the grpc wrapper of IndexCoord. // Server is the grpc wrapper of IndexCoord.
type Server struct { type Server struct {
indexcoord types.IndexCoord indexcoord types.IndexCoordComponent
grpcServer *grpc.Server grpcServer *grpc.Server
grpcErrChan chan error grpcErrChan chan error
@ -58,6 +60,8 @@ type Server struct {
loopCancel func() loopCancel func()
loopWg sync.WaitGroup loopWg sync.WaitGroup
etcdCli *clientv3.Client
closer io.Closer closer io.Closer
} }
@ -86,6 +90,14 @@ func (s *Server) init() error {
closer := trace.InitTracing("IndexCoord") closer := trace.InitTracing("IndexCoord")
s.closer = closer s.closer = closer
etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.BaseParams)
if err != nil {
log.Debug("IndexCoord connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.indexcoord.SetEtcdClient(s.etcdCli)
s.loopWg.Add(1) s.loopWg.Add(1)
go s.startGrpcLoop(indexcoord.Params.IndexCoordCfg.Port) go s.startGrpcLoop(indexcoord.Params.IndexCoordCfg.Port)
// wait for grpc IndexCoord loop start // wait for grpc IndexCoord loop start
@ -126,7 +138,9 @@ func (s *Server) Stop() error {
if s.indexcoord != nil { if s.indexcoord != nil {
s.indexcoord.Stop() s.indexcoord.Stop()
} }
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
s.loopCancel() s.loopCancel()
if s.grpcServer != nil { if s.grpcServer != nil {
log.Debug("Graceful stop grpc server...") log.Debug("Graceful stop grpc server...")
@ -138,7 +152,7 @@ func (s *Server) Stop() error {
} }
// SetClient sets the IndexCoord's instance. // SetClient sets the IndexCoord's instance.
func (s *Server) SetClient(indexCoordClient types.IndexCoord) error { func (s *Server) SetClient(indexCoordClient types.IndexCoordComponent) error {
s.indexcoord = indexCoordClient s.indexcoord = indexCoordClient
return nil return nil
} }

View File

@ -25,6 +25,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
etcd "github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -33,7 +34,11 @@ func TestIndexCoordinateServer(t *testing.T) {
server, err := NewServer(ctx) server, err := NewServer(ctx)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, server) assert.NotNil(t, server)
Params.Init()
etcd, err := etcd.GetEtcdClient(&Params.BaseParamTable)
assert.NoError(t, err)
indexCoordClient := &indexcoord.Mock{} indexCoordClient := &indexcoord.Mock{}
indexCoordClient.SetEtcdClient(etcd)
err = server.SetClient(indexCoordClient) err = server.SetClient(indexCoordClient)
assert.Nil(t, err) assert.Nil(t, err)
err = server.Run() err = server.Run()

View File

@ -26,13 +26,17 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/mock" "github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
var ParamsGlobal paramtable.GlobalParamTable
func Test_NewClient(t *testing.T) { func Test_NewClient(t *testing.T) {
ClientParams.InitOnce(typeutil.IndexNodeRole) ClientParams.InitOnce(typeutil.IndexNodeRole)
ctx := context.Background() ctx := context.Background()
@ -123,6 +127,10 @@ func TestIndexNodeClient(t *testing.T) {
assert.NotNil(t, ins) assert.NotNil(t, ins)
inm := &indexnode.Mock{} inm := &indexnode.Mock{}
ParamsGlobal.InitOnce()
etcdCli, err := etcd.GetEtcdClient(&ParamsGlobal.BaseParams)
assert.NoError(t, err)
inm.SetEtcdClient(etcdCli)
err = ins.SetClient(inm) err = ins.SetClient(inm)
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -33,10 +33,12 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -46,7 +48,7 @@ var Params paramtable.GrpcServerConfig
// Server is the grpc wrapper of IndexNode. // Server is the grpc wrapper of IndexNode.
type Server struct { type Server struct {
indexnode types.IndexNode indexnode types.IndexNodeComponent
grpcServer *grpc.Server grpcServer *grpc.Server
grpcErrChan chan error grpcErrChan chan error
@ -55,7 +57,8 @@ type Server struct {
loopCancel func() loopCancel func()
loopWg sync.WaitGroup loopWg sync.WaitGroup
closer io.Closer etcdCli *clientv3.Client
closer io.Closer
} }
// Run initializes and starts IndexNode's grpc service. // Run initializes and starts IndexNode's grpc service.
@ -142,6 +145,13 @@ func (s *Server) init() error {
return err return err
} }
etcdCli, err := etcd.GetEtcdClient(&indexnode.Params.BaseParams)
if err != nil {
log.Debug("IndexNode connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.indexnode.SetEtcdClient(etcdCli)
err = s.indexnode.Init() err = s.indexnode.Init()
if err != nil { if err != nil {
log.Error("IndexNode Init failed", zap.Error(err)) log.Error("IndexNode Init failed", zap.Error(err))
@ -178,6 +188,9 @@ func (s *Server) Stop() error {
if s.indexnode != nil { if s.indexnode != nil {
s.indexnode.Stop() s.indexnode.Stop()
} }
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.grpcServer != nil { if s.grpcServer != nil {
log.Debug("Graceful stop grpc server...") log.Debug("Graceful stop grpc server...")
s.grpcServer.GracefulStop() s.grpcServer.GracefulStop()
@ -188,11 +201,16 @@ func (s *Server) Stop() error {
} }
// SetClient sets the IndexNode's instance. // SetClient sets the IndexNode's instance.
func (s *Server) SetClient(indexNodeClient types.IndexNode) error { func (s *Server) SetClient(indexNodeClient types.IndexNodeComponent) error {
s.indexnode = indexNodeClient s.indexnode = indexNodeClient
return nil return nil
} }
// SetEtcdClient sets the etcd client for QueryNode component.
func (s *Server) SetEtcdClient(etcdCli *clientv3.Client) {
s.indexnode.SetEtcdClient(etcdCli)
}
// GetComponentStates gets the component states of IndexNode. // GetComponentStates gets the component states of IndexNode.
func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) { func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
return s.indexnode.GetComponentStates(ctx) return s.indexnode.GetComponentStates(ctx)

View File

@ -27,9 +27,13 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
var ParamsGlobal paramtable.GlobalParamTable
func TestIndexNodeServer(t *testing.T) { func TestIndexNodeServer(t *testing.T) {
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx) server, err := NewServer(ctx)
@ -37,6 +41,10 @@ func TestIndexNodeServer(t *testing.T) {
assert.NotNil(t, server) assert.NotNil(t, server)
inm := &indexnode.Mock{} inm := &indexnode.Mock{}
ParamsGlobal.InitOnce()
etcdCli, err := etcd.GetEtcdClient(&ParamsGlobal.BaseParams)
assert.NoError(t, err)
inm.SetEtcdClient(etcdCli)
err = server.SetClient(inm) err = server.SetClient(inm)
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -38,11 +38,13 @@ import (
"github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -59,6 +61,7 @@ type Server struct {
grpcErrChan chan error grpcErrChan chan error
etcdCli *clientv3.Client
rootCoordClient types.RootCoord rootCoordClient types.RootCoord
dataCoordClient types.DataCoord dataCoordClient types.DataCoord
queryCoordClient types.QueryCoord queryCoordClient types.QueryCoord
@ -172,6 +175,13 @@ func (s *Server) init() error {
s.closer = closer s.closer = closer
log.Debug("init Proxy's tracer done", zap.String("service name", serviceName)) log.Debug("init Proxy's tracer done", zap.String("service name", serviceName))
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
if err != nil {
log.Debug("Proxy connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.proxy.SetEtcdClient(s.etcdCli)
s.wg.Add(1) s.wg.Add(1)
go s.startGrpcLoop(Params.Port) go s.startGrpcLoop(Params.Port)
log.Debug("waiting for grpc server of Proxy to be started") log.Debug("waiting for grpc server of Proxy to be started")
@ -184,7 +194,7 @@ func (s *Server) init() error {
if s.rootCoordClient == nil { if s.rootCoordClient == nil {
var err error var err error
log.Debug("create RootCoord client for Proxy") log.Debug("create RootCoord client for Proxy")
s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints) s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create RootCoord client for Proxy", zap.Error(err)) log.Warn("failed to create RootCoord client for Proxy", zap.Error(err))
return err return err
@ -213,7 +223,7 @@ func (s *Server) init() error {
if s.dataCoordClient == nil { if s.dataCoordClient == nil {
var err error var err error
log.Debug("create DataCoord client for Proxy") log.Debug("create DataCoord client for Proxy")
s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints) s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create DataCoord client for Proxy", zap.Error(err)) log.Warn("failed to create DataCoord client for Proxy", zap.Error(err))
return err return err
@ -242,7 +252,7 @@ func (s *Server) init() error {
if s.indexCoordClient == nil { if s.indexCoordClient == nil {
var err error var err error
log.Debug("create IndexCoord client for Proxy") log.Debug("create IndexCoord client for Proxy")
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints) s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err)) log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err))
return err return err
@ -271,7 +281,7 @@ func (s *Server) init() error {
if s.queryCoordClient == nil { if s.queryCoordClient == nil {
var err error var err error
log.Debug("create QueryCoord client for Proxy") log.Debug("create QueryCoord client for Proxy")
s.queryCoordClient, err = qcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints) s.queryCoordClient, err = qcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err)) log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err))
return err return err
@ -334,6 +344,10 @@ func (s *Server) Stop() error {
} }
} }
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.grpcServer != nil { if s.grpcServer != nil {
log.Debug("Graceful stop grpc server...") log.Debug("Graceful stop grpc server...")
s.grpcServer.GracefulStop() s.grpcServer.GracefulStop()

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
@ -616,6 +617,9 @@ func (m *MockProxy) UpdateStateCode(stateCode internalpb.StateCode) {
} }
func (m *MockProxy) SetEtcdClient(etcdClient *clientv3.Client) {
}
func (m *MockProxy) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) { func (m *MockProxy) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
return nil, nil return nil, nil
} }

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
@ -43,8 +44,8 @@ type Client struct {
} }
// NewClient creates a client for QueryCoord grpc call. // NewClient creates a client for QueryCoord grpc call.
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
if sess == nil { if sess == nil {
err := fmt.Errorf("new session error, maybe can not connect to etcd") err := fmt.Errorf("new session error, maybe can not connect to etcd")
log.Debug("QueryCoordClient NewClient failed", zap.Error(err)) log.Debug("QueryCoordClient NewClient failed", zap.Error(err))

View File

@ -25,6 +25,7 @@ import (
"google.golang.org/grpc" "google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -32,7 +33,10 @@ func Test_NewClient(t *testing.T) {
proxy.Params.InitOnce() proxy.Params.InitOnce()
ctx := context.Background() ctx := context.Background()
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
assert.NoError(t, err)
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, client) assert.NotNil(t, client)

View File

@ -36,10 +36,12 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
qc "github.com/milvus-io/milvus/internal/querycoord" qc "github.com/milvus-io/milvus/internal/querycoord"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -60,6 +62,8 @@ type Server struct {
msFactory msgstream.Factory msFactory msgstream.Factory
etcdCli *clientv3.Client
dataCoord types.DataCoord dataCoord types.DataCoord
rootCoord types.RootCoord rootCoord types.RootCoord
indexCoord types.IndexCoord indexCoord types.IndexCoord
@ -111,17 +115,25 @@ func (s *Server) init() error {
closer := trace.InitTracing("querycoord") closer := trace.InitTracing("querycoord")
s.closer = closer s.closer = closer
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
if err != nil {
log.Debug("QueryCoord connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.SetEtcdClient(etcdCli)
s.wg.Add(1) s.wg.Add(1)
go s.startGrpcLoop(Params.Port) go s.startGrpcLoop(Params.Port)
// wait for grpc server loop start // wait for grpc server loop start
err := <-s.grpcErrChan err = <-s.grpcErrChan
if err != nil { if err != nil {
return err return err
} }
// --- Master Server Client --- // --- Master Server Client ---
if s.rootCoord == nil { if s.rootCoord == nil {
s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints) s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err)) log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err))
panic(err) panic(err)
@ -152,7 +164,7 @@ func (s *Server) init() error {
// --- Data service client --- // --- Data service client ---
if s.dataCoord == nil { if s.dataCoord == nil {
s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints) s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err)) log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err))
panic(err) panic(err)
@ -180,7 +192,7 @@ func (s *Server) init() error {
// --- IndexCoord --- // --- IndexCoord ---
if s.indexCoord == nil { if s.indexCoord == nil {
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints) s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err)) log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
panic(err) panic(err)
@ -273,6 +285,9 @@ func (s *Server) Stop() error {
return err return err
} }
} }
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
err := s.queryCoord.Stop() err := s.queryCoord.Stop()
s.loopCancel() s.loopCancel()
if s.grpcServer != nil { if s.grpcServer != nil {
@ -282,6 +297,11 @@ func (s *Server) Stop() error {
return err return err
} }
// SetRootCoord sets root coordinator's client
func (s *Server) SetEtcdClient(etcdClient *clientv3.Client) {
s.queryCoord.SetEtcdClient(etcdClient)
}
// SetRootCoord sets the RootCoord's client for QueryCoord component. // SetRootCoord sets the RootCoord's client for QueryCoord component.
func (s *Server) SetRootCoord(m types.RootCoord) error { func (s *Server) SetRootCoord(m types.RootCoord) error {
s.queryCoord.SetRootCoord(m) s.queryCoord.SetRootCoord(m)

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
@ -69,6 +70,9 @@ func (m *MockQueryCoord) Register() error {
func (m *MockQueryCoord) UpdateStateCode(code internalpb.StateCode) { func (m *MockQueryCoord) UpdateStateCode(code internalpb.StateCode) {
} }
func (m *MockQueryCoord) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockQueryCoord) SetRootCoord(types.RootCoord) error { func (m *MockQueryCoord) SetRootCoord(types.RootCoord) error {
return nil return nil
} }

View File

@ -36,11 +36,13 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
qn "github.com/milvus-io/milvus/internal/querynode" qn "github.com/milvus-io/milvus/internal/querynode"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
@ -61,6 +63,7 @@ type Server struct {
grpcServer *grpc.Server grpcServer *grpc.Server
etcdCli *clientv3.Client
rootCoord types.RootCoord rootCoord types.RootCoord
indexCoord types.IndexCoord indexCoord types.IndexCoord
@ -93,17 +96,26 @@ func (s *Server) init() error {
s.closer = closer s.closer = closer
log.Debug("QueryNode", zap.Int("port", Params.Port)) log.Debug("QueryNode", zap.Int("port", Params.Port))
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
if err != nil {
log.Debug("QueryNode connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.SetEtcdClient(etcdCli)
log.Debug("QueryNode connect to etcd successfully")
s.wg.Add(1) s.wg.Add(1)
go s.startGrpcLoop(Params.Port) go s.startGrpcLoop(Params.Port)
// wait for grpc server loop start // wait for grpc server loop start
err := <-s.grpcErrChan err = <-s.grpcErrChan
if err != nil { if err != nil {
return err return err
} }
// --- RootCoord Client --- // --- RootCoord Client ---
if s.rootCoord == nil { if s.rootCoord == nil {
s.rootCoord, err = rcc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, qn.Params.QueryNodeCfg.EtcdEndpoints) s.rootCoord, err = rcc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryNode new RootCoordClient failed", zap.Error(err)) log.Debug("QueryNode new RootCoordClient failed", zap.Error(err))
panic(err) panic(err)
@ -133,7 +145,7 @@ func (s *Server) init() error {
// --- IndexCoord --- // --- IndexCoord ---
if s.indexCoord == nil { if s.indexCoord == nil {
s.indexCoord, err = icc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, qn.Params.QueryNodeCfg.EtcdEndpoints) s.indexCoord, err = icc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryNode new IndexCoordClient failed", zap.Error(err)) log.Debug("QueryNode new IndexCoordClient failed", zap.Error(err))
panic(err) panic(err)
@ -260,6 +272,9 @@ func (s *Server) Stop() error {
return err return err
} }
} }
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
s.cancel() s.cancel()
if s.grpcServer != nil { if s.grpcServer != nil {
@ -275,6 +290,11 @@ func (s *Server) Stop() error {
return nil return nil
} }
// SetEtcdClient sets the etcd client for QueryNode component.
func (s *Server) SetEtcdClient(etcdCli *clientv3.Client) {
s.querynode.SetEtcdClient(etcdCli)
}
// SetRootCoord sets the RootCoord's client for QueryNode component. // SetRootCoord sets the RootCoord's client for QueryNode component.
func (s *Server) SetRootCoord(rootCoord types.RootCoord) error { func (s *Server) SetRootCoord(rootCoord types.RootCoord) error {
return s.querynode.SetRootCoord(rootCoord) return s.querynode.SetRootCoord(rootCoord)

View File

@ -28,6 +28,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
@ -112,6 +113,9 @@ func (m *MockQueryNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetrics
return m.metricResp, m.err return m.metricResp, m.err
} }
func (m *MockQueryNode) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockQueryNode) UpdateStateCode(code internalpb.StateCode) { func (m *MockQueryNode) UpdateStateCode(code internalpb.StateCode) {
} }
@ -149,6 +153,9 @@ func (m *MockRootCoord) Register() error {
return m.regErr return m.regErr
} }
func (m *MockRootCoord) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
return &internalpb.ComponentStates{ return &internalpb.ComponentStates{
State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy}, State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy},
@ -182,6 +189,9 @@ func (m *MockIndexCoord) Register() error {
return m.regErr return m.regErr
} }
func (m *MockIndexCoord) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
return &internalpb.ComponentStates{ return &internalpb.ComponentStates{
State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy}, State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy},

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
@ -49,8 +50,8 @@ type Client struct {
// metaRoot is the path in etcd for root coordinator registration // metaRoot is the path in etcd for root coordinator registration
// etcdEndpoints are the address list for etcd end points // etcdEndpoints are the address list for etcd end points
// timeout is default setting for each grpc call // timeout is default setting for each grpc call
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
if sess == nil { if sess == nil {
err := fmt.Errorf("new session error, maybe can not connect to etcd") err := fmt.Errorf("new session error, maybe can not connect to etcd")
log.Debug("QueryCoordClient NewClient failed", zap.Error(err)) log.Debug("QueryCoordClient NewClient failed", zap.Error(err))

View File

@ -25,6 +25,7 @@ import (
"google.golang.org/grpc" "google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proxy" "github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -32,7 +33,9 @@ func Test_NewClient(t *testing.T) {
proxy.Params.InitOnce() proxy.Params.InitOnce()
ctx := context.Background() ctx := context.Background()
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
assert.NoError(t, err)
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, client) assert.NotNil(t, client)

View File

@ -25,10 +25,7 @@ import (
"time" "time"
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing" ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
pnc "github.com/milvus-io/milvus/internal/distributed/proxy/client" pnc "github.com/milvus-io/milvus/internal/distributed/proxy/client"
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream" "github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
@ -39,14 +36,20 @@ import (
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/rootcoord"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
) )
var Params paramtable.GrpcServerConfig var Params paramtable.GrpcServerConfig
@ -62,13 +65,14 @@ type Server struct {
ctx context.Context ctx context.Context
cancel context.CancelFunc cancel context.CancelFunc
etcdCli *clientv3.Client
dataCoord types.DataCoord dataCoord types.DataCoord
indexCoord types.IndexCoord indexCoord types.IndexCoord
queryCoord types.QueryCoord queryCoord types.QueryCoord
newIndexCoordClient func(string, []string) types.IndexCoord newIndexCoordClient func(string, *clientv3.Client) types.IndexCoord
newDataCoordClient func(string, []string) types.DataCoord newDataCoordClient func(string, *clientv3.Client) types.DataCoord
newQueryCoordClient func(string, []string) types.QueryCoord newQueryCoordClient func(string, *clientv3.Client) types.QueryCoord
closer io.Closer closer io.Closer
} }
@ -106,22 +110,22 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
} }
func (s *Server) setClient() { func (s *Server) setClient() {
s.newDataCoordClient = func(etcdMetaRoot string, etcdEndpoints []string) types.DataCoord { s.newDataCoordClient = func(etcdMetaRoot string, etcdCli *clientv3.Client) types.DataCoord {
dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdEndpoints) dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdCli)
if err != nil { if err != nil {
panic(err) panic(err)
} }
return dsClient return dsClient
} }
s.newIndexCoordClient = func(metaRootPath string, etcdEndpoints []string) types.IndexCoord { s.newIndexCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.IndexCoord {
isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdEndpoints) isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdCli)
if err != nil { if err != nil {
panic(err) panic(err)
} }
return isClient return isClient
} }
s.newQueryCoordClient = func(metaRootPath string, etcdEndpoints []string) types.QueryCoord { s.newQueryCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.QueryCoord {
qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdEndpoints) qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdCli)
if err != nil { if err != nil {
panic(err) panic(err)
} }
@ -149,17 +153,25 @@ func (s *Server) init() error {
rootcoord.Params.InitOnce() rootcoord.Params.InitOnce()
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress() rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
rootcoord.Params.RootCoordCfg.Port = Params.Port rootcoord.Params.RootCoordCfg.Port = Params.Port
log.Debug("grpc init done ...") log.Debug("init params done..")
closer := trace.InitTracing("root_coord") closer := trace.InitTracing("root_coord")
s.closer = closer s.closer = closer
log.Debug("init params done") etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
if err != nil {
log.Debug("RootCoord connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.rootCoord.SetEtcdClient(s.etcdCli)
log.Debug("etcd connect done ...")
err := s.startGrpc(Params.Port) err = s.startGrpc(Params.Port)
if err != nil { if err != nil {
return err return err
} }
log.Debug("grpc init done ...")
s.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing) s.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
log.Debug("RootCoord", zap.Any("State", internalpb.StateCode_Initializing)) log.Debug("RootCoord", zap.Any("State", internalpb.StateCode_Initializing))
@ -181,7 +193,7 @@ func (s *Server) init() error {
if s.newDataCoordClient != nil { if s.newDataCoordClient != nil {
log.Debug("RootCoord start to create DataCoord client") log.Debug("RootCoord start to create DataCoord client")
dataCoord := s.newDataCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints) dataCoord := s.newDataCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, s.etcdCli)
if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil { if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil {
panic(err) panic(err)
} }
@ -189,7 +201,7 @@ func (s *Server) init() error {
} }
if s.newIndexCoordClient != nil { if s.newIndexCoordClient != nil {
log.Debug("RootCoord start to create IndexCoord client") log.Debug("RootCoord start to create IndexCoord client")
indexCoord := s.newIndexCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints) indexCoord := s.newIndexCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, s.etcdCli)
if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil { if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil {
panic(err) panic(err)
} }
@ -197,7 +209,7 @@ func (s *Server) init() error {
} }
if s.newQueryCoordClient != nil { if s.newQueryCoordClient != nil {
log.Debug("RootCoord start to create QueryCoord client") log.Debug("RootCoord start to create QueryCoord client")
queryCoord := s.newQueryCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints) queryCoord := s.newQueryCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, s.etcdCli)
if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil { if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil {
panic(err) panic(err)
} }
@ -273,6 +285,9 @@ func (s *Server) Stop() error {
log.Error("Failed to close opentracing", zap.Error(err)) log.Error("Failed to close opentracing", zap.Error(err))
} }
} }
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.indexCoord != nil { if s.indexCoord != nil {
if err := s.indexCoord.Stop(); err != nil { if err := s.indexCoord.Stop(); err != nil {
log.Error("Failed to close indexCoord client", zap.Error(err)) log.Error("Failed to close indexCoord client", zap.Error(err))

View File

@ -41,6 +41,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/rootcoord"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
@ -98,7 +99,7 @@ func TestGrpcService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing) svr.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
@ -115,6 +116,7 @@ func TestGrpcService(t *testing.T) {
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress() rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
core.SetEtcdClient(etcdCli)
err = core.Init() err = core.Init()
assert.Nil(t, err) assert.Nil(t, err)
@ -214,7 +216,7 @@ func TestGrpcService(t *testing.T) {
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Healthy) svr.rootCoord.UpdateStateCode(internalpb.StateCode_Healthy)
cli, err := rcc.NewClient(context.Background(), rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints) cli, err := rcc.NewClient(context.Background(), rootcoord.Params.RootCoordCfg.MetaRootPath, etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
err = cli.Init() err = cli.Init()
@ -794,6 +796,9 @@ type mockCore struct {
func (m *mockCore) UpdateStateCode(internalpb.StateCode) { func (m *mockCore) UpdateStateCode(internalpb.StateCode) {
} }
func (m *mockCore) SetEtcdClient(etcdClient *clientv3.Client) {
}
func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error { func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error {
return nil return nil
} }
@ -895,13 +900,13 @@ func TestRun(t *testing.T) {
assert.NotNil(t, err) assert.NotNil(t, err)
assert.EqualError(t, err, "listen tcp: address 1000000: invalid port") assert.EqualError(t, err, "listen tcp: address 1000000: invalid port")
svr.newDataCoordClient = func(string, []string) types.DataCoord { svr.newDataCoordClient = func(string, *clientv3.Client) types.DataCoord {
return &mockDataCoord{} return &mockDataCoord{}
} }
svr.newIndexCoordClient = func(string, []string) types.IndexCoord { svr.newIndexCoordClient = func(string, *clientv3.Client) types.IndexCoord {
return &mockIndex{} return &mockIndex{}
} }
svr.newQueryCoordClient = func(string, []string) types.QueryCoord { svr.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord {
return &mockQuery{} return &mockQuery{}
} }
@ -912,7 +917,7 @@ func TestRun(t *testing.T) {
rootcoord.Params.Init() rootcoord.Params.Init()
rootcoord.Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal) rootcoord.Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())

View File

@ -50,6 +50,7 @@ import (
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
) )
// make sure IndexCoord implements types.IndexCoord // make sure IndexCoord implements types.IndexCoord
@ -79,7 +80,8 @@ type IndexCoord struct {
idAllocator *allocator.GlobalIDAllocator idAllocator *allocator.GlobalIDAllocator
kv kv.BaseKV etcdCli *clientv3.Client
kv kv.BaseKV
metaTable *metaTable metaTable *metaTable
nodeManager *NodeManager nodeManager *NodeManager
@ -129,17 +131,19 @@ func (i *IndexCoord) Register() error {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if i.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
return nil return nil
} }
func (i *IndexCoord) initSession() error { func (i *IndexCoord) initSession() error {
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints) i.session = sessionutil.NewSession(i.loopCtx, Params.IndexCoordCfg.MetaRootPath, i.etcdCli)
if i.session == nil { if i.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
i.session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true) i.session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true, true)
Params.BaseParams.SetLogger(i.session.ServerID) Params.BaseParams.SetLogger(i.session.ServerID)
return nil return nil
} }
@ -160,10 +164,7 @@ func (i *IndexCoord) Init() error {
} }
connectEtcdFn := func() error { connectEtcdFn := func() error {
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.IndexCoordCfg.MetaRootPath)
if err != nil {
return err
}
metakv, err := NewMetaTable(etcdKV) metakv, err := NewMetaTable(etcdKV)
if err != nil { if err != nil {
return err return err
@ -207,12 +208,7 @@ func (i *IndexCoord) Init() error {
//init idAllocator //init idAllocator
kvRootPath := Params.IndexCoordCfg.KvRootPath kvRootPath := Params.IndexCoordCfg.KvRootPath
etcdKV, err := tsoutil.NewTSOKVBase(Params.IndexCoordCfg.EtcdEndpoints, kvRootPath, "index_gid") etcdKV := tsoutil.NewTSOKVBase(i.etcdCli, kvRootPath, "index_gid")
if err != nil {
log.Error("IndexCoord TSOKVBase initialize failed", zap.Error(err))
initErr = err
return
}
i.idAllocator = allocator.NewGlobalIDAllocator("idTimestamp", etcdKV) i.idAllocator = allocator.NewGlobalIDAllocator("idTimestamp", etcdKV)
if err := i.idAllocator.Initialize(); err != nil { if err := i.idAllocator.Initialize(); err != nil {
@ -306,6 +302,10 @@ func (i *IndexCoord) Stop() error {
return nil return nil
} }
func (i *IndexCoord) SetEtcdClient(etcdClient *clientv3.Client) {
i.etcdCli = etcdClient
}
// UpdateStateCode updates the component state of IndexCoord. // UpdateStateCode updates the component state of IndexCoord.
func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) { func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) {
i.stateCode.Store(code) i.stateCode.Store(code)

View File

@ -28,12 +28,13 @@ import (
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
) )
// Mock is an alternative to IndexCoord, it will return specific results based on specific parameters. // Mock is an alternative to IndexCoord, it will return specific results based on specific parameters.
type Mock struct { type Mock struct {
etcdKV *etcdkv.EtcdKV etcdKV *etcdkv.EtcdKV
etcdCli *clientv3.Client
Failure bool Failure bool
} }
@ -67,14 +68,24 @@ func (icm *Mock) Register() error {
if icm.Failure { if icm.Failure {
return errors.New("IndexCoordinate register failed") return errors.New("IndexCoordinate register failed")
} }
icm.etcdKV, _ = etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath) icm.etcdKV = etcdkv.NewEtcdKV(icm.etcdCli, Params.IndexCoordCfg.MetaRootPath)
err := icm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole) err := icm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole)
session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints) if err != nil {
session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true) return err
}
session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, icm.etcdCli)
session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true, false)
session.Register() session.Register()
return err return err
} }
func (icm *Mock) SetEtcdClient(client *clientv3.Client) {
icm.etcdCli = client
}
func (icm *Mock) UpdateStateCode(stateCode internalpb.StateCode) {
}
// GetComponentStates gets the component states of the mocked IndexCoord, if Param `Failure` is true, it will return an error, // GetComponentStates gets the component states of the mocked IndexCoord, if Param `Failure` is true, it will return an error,
// and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`. // and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`.
func (icm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { func (icm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {

View File

@ -24,13 +24,19 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
func TestIndexCoordMock(t *testing.T) { func TestIndexCoordMock(t *testing.T) {
Params.Init() Params.Init()
icm := Mock{} icm := Mock{}
err := icm.Register()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
icm.SetEtcdClient(etcdCli)
defer etcdCli.Close()
err = icm.Register()
assert.Nil(t, err) assert.Nil(t, err)
err = icm.Init() err = icm.Init()
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -41,7 +42,11 @@ import (
func TestIndexCoord(t *testing.T) { func TestIndexCoord(t *testing.T) {
ctx := context.Background() ctx := context.Background()
inm0 := &indexnode.Mock{} inm0 := &indexnode.Mock{}
err := inm0.Init() Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
inm0.SetEtcdClient(etcdCli)
err = inm0.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = inm0.Register() err = inm0.Register()
assert.Nil(t, err) assert.Nil(t, err)
@ -53,8 +58,8 @@ func TestIndexCoord(t *testing.T) {
ic.durationInterval = time.Second ic.durationInterval = time.Second
ic.assignTaskInterval = 200 * time.Millisecond ic.assignTaskInterval = 200 * time.Millisecond
ic.taskLimit = 20 ic.taskLimit = 20
Params.Init()
ic.SetEtcdClient(etcdCli)
err = ic.Init() err = ic.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = ic.Register() err = ic.Register()
@ -72,6 +77,8 @@ func TestIndexCoord(t *testing.T) {
Build: true, Build: true,
Failure: false, Failure: false,
} }
inm.SetEtcdClient(etcdCli)
err = in.SetClient(inm) err = in.SetClient(inm)
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -25,13 +25,17 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
func TestMetaTable(t *testing.T) { func TestMetaTable(t *testing.T) {
Params.Init() Params.Init()
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.NoError(t, err)
assert.Nil(t, err) assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.IndexCoordCfg.MetaRootPath)
req := &indexpb.BuildIndexRequest{ req := &indexpb.BuildIndexRequest{
IndexBuildID: 1, IndexBuildID: 1,
@ -310,8 +314,11 @@ func TestMetaTable(t *testing.T) {
func TestMetaTable_Error(t *testing.T) { func TestMetaTable_Error(t *testing.T) {
Params.Init() Params.Init()
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) defer etcdCli.Close()
assert.NoError(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.IndexCoordCfg.MetaRootPath)
t.Run("reloadFromKV error", func(t *testing.T) { t.Run("reloadFromKV error", func(t *testing.T) {
value := "indexMeta-1" value := "indexMeta-1"

View File

@ -21,6 +21,7 @@ import (
"testing" "testing"
"github.com/milvus-io/milvus/internal/indexnode" "github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -31,6 +32,11 @@ func TestGetSystemInfoMetrics(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
Params.Init() Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.NoError(t, err)
ic.SetEtcdClient(etcdCli)
err = ic.Init() err = ic.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = ic.Start() err = ic.Start()

View File

@ -51,10 +51,10 @@ import (
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -64,6 +64,9 @@ type UniqueID = typeutil.UniqueID
// make sure IndexNode implements types.IndexNode // make sure IndexNode implements types.IndexNode
var _ types.IndexNode = (*IndexNode)(nil) var _ types.IndexNode = (*IndexNode)(nil)
// make sure IndexNode implements types.IndexNodeComponent
var _ types.IndexNodeComponent = (*IndexNode)(nil)
var Params paramtable.GlobalParamTable var Params paramtable.GlobalParamTable
// IndexNode is a component that executes the task of building indexes. // IndexNode is a component that executes the task of building indexes.
@ -84,6 +87,7 @@ type IndexNode struct {
startCallbacks []func() startCallbacks []func()
closeCallbacks []func() closeCallbacks []func()
etcdCli *clientv3.Client
etcdKV *etcdkv.EtcdKV etcdKV *etcdkv.EtcdKV
finishedTasks map[UniqueID]commonpb.IndexState finishedTasks map[UniqueID]commonpb.IndexState
@ -122,7 +126,9 @@ func (i *IndexNode) Register() error {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if i.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
return nil return nil
} }
@ -139,11 +145,11 @@ func (i *IndexNode) initKnowhere() {
} }
func (i *IndexNode) initSession() error { func (i *IndexNode) initSession() error {
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints) i.session = sessionutil.NewSession(i.loopCtx, Params.IndexNodeCfg.MetaRootPath, i.etcdCli)
if i.session == nil { if i.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
i.session.Init(typeutil.IndexNodeRole, Params.IndexNodeCfg.IP+":"+strconv.Itoa(Params.IndexNodeCfg.Port), false) i.session.Init(typeutil.IndexNodeRole, Params.IndexNodeCfg.IP+":"+strconv.Itoa(Params.IndexNodeCfg.Port), false, true)
Params.IndexNodeCfg.NodeID = i.session.ServerID Params.IndexNodeCfg.NodeID = i.session.ServerID
Params.BaseParams.SetLogger(Params.IndexNodeCfg.NodeID) Params.BaseParams.SetLogger(Params.IndexNodeCfg.NodeID)
return nil return nil
@ -165,18 +171,8 @@ func (i *IndexNode) Init() error {
} }
log.Debug("IndexNode init session successful", zap.Int64("serverID", i.session.ServerID)) log.Debug("IndexNode init session successful", zap.Int64("serverID", i.session.ServerID))
connectEtcdFn := func() error { etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.IndexNodeCfg.MetaRootPath)
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath) i.etcdKV = etcdKV
i.etcdKV = etcdKV
return err
}
err = retry.Do(i.loopCtx, connectEtcdFn, retry.Attempts(300))
if err != nil {
log.Error("IndexNode failed to connect to etcd", zap.Error(err))
initErr = err
return
}
log.Debug("IndexNode connected to etcd successfully")
option := &miniokv.Option{ option := &miniokv.Option{
Address: Params.IndexNodeCfg.MinIOAddress, Address: Params.IndexNodeCfg.MinIOAddress,
@ -250,6 +246,11 @@ func (i *IndexNode) UpdateStateCode(code internalpb.StateCode) {
i.stateCode.Store(code) i.stateCode.Store(code)
} }
// SetEtcdClient assigns parameter client to its member etcdCli
func (node *IndexNode) SetEtcdClient(client *clientv3.Client) {
node.etcdCli = client
}
func (i *IndexNode) isHealthy() bool { func (i *IndexNode) isHealthy() bool {
code := i.stateCode.Load().(internalpb.StateCode) code := i.stateCode.Load().(internalpb.StateCode)
return code == internalpb.StateCode_Healthy return code == internalpb.StateCode_Healthy

View File

@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
) )
// Mock is an alternative to IndexNode, it will return specific results based on specific parameters. // Mock is an alternative to IndexNode, it will return specific results based on specific parameters.
@ -46,7 +47,8 @@ type Mock struct {
cancel context.CancelFunc cancel context.CancelFunc
wg sync.WaitGroup wg sync.WaitGroup
etcdKV *etcdkv.EtcdKV etcdCli *clientv3.Client
etcdKV *etcdkv.EtcdKV
buildIndex chan *indexpb.CreateIndexRequest buildIndex chan *indexpb.CreateIndexRequest
} }
@ -182,16 +184,25 @@ func (inm *Mock) Register() error {
return errors.New("IndexNode register failed") return errors.New("IndexNode register failed")
} }
Params.Init() Params.Init()
inm.etcdKV, _ = etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath) inm.etcdKV = etcdkv.NewEtcdKV(inm.etcdCli, Params.IndexNodeCfg.MetaRootPath)
if err := inm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexNodeRole); err != nil { if err := inm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexNodeRole); err != nil {
return err return err
} }
session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints) session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, inm.etcdCli)
session.Init(typeutil.IndexNodeRole, "localhost:21121", false) session.Init(typeutil.IndexNodeRole, "localhost:21121", false, false)
session.Register() session.Register()
return nil return nil
} }
// SetClient sets the IndexNode's instance.
func (inm *Mock) UpdateStateCode(stateCode internalpb.StateCode) {
}
// SetEtcdClient assigns parameter client to its member etcdCli
func (inm *Mock) SetEtcdClient(client *clientv3.Client) {
inm.etcdCli = client
}
// GetComponentStates gets the component states of the mocked IndexNode, if the internal member `Err` is true, it will return an error, // GetComponentStates gets the component states of the mocked IndexNode, if the internal member `Err` is true, it will return an error,
// and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`. // and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`.
func (inm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { func (inm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -37,7 +38,11 @@ func TestIndexNodeMock(t *testing.T) {
inm := Mock{ inm := Mock{
Build: true, Build: true,
} }
err := inm.Register() etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
inm.SetEtcdClient(etcdCli)
defer etcdCli.Close()
err = inm.Register()
assert.Nil(t, err) assert.Nil(t, err)
err = inm.Init() err = inm.Init()
assert.Nil(t, err) assert.Nil(t, err)
@ -151,8 +156,11 @@ func TestIndexNodeMockFiled(t *testing.T) {
Build: true, Build: true,
Err: false, Err: false,
} }
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
err := inm.Register() assert.NoError(t, err)
inm.SetEtcdClient(etcdCli)
defer etcdCli.Close()
err = inm.Register()
assert.Nil(t, err) assert.Nil(t, err)
err = inm.Init() err = inm.Init()
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"go.uber.org/zap" "go.uber.org/zap"
@ -71,6 +72,11 @@ func TestIndexNode(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
Params.Init() Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
in.SetEtcdClient(etcdCli)
defer etcdCli.Close()
err = in.Init() err = in.Init()
assert.Nil(t, err) assert.Nil(t, err)
@ -477,6 +483,11 @@ func TestCreateIndexFailed(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
Params.Init() Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
in.SetEtcdClient(etcdCli)
defer etcdCli.Close()
err = in.Init() err = in.Init()
assert.Nil(t, err) assert.Nil(t, err)
@ -744,6 +755,11 @@ func TestIndexNode_Error(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
Params.Init() Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
in.SetEtcdClient(etcdCli)
defer etcdCli.Close()
err = in.Init() err = in.Init()
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -40,34 +40,17 @@ type EtcdKV struct {
} }
// NewEtcdKV creates a new etcd kv. // NewEtcdKV creates a new etcd kv.
func NewEtcdKV(etcdEndpoints []string, rootPath string) (*EtcdKV, error) { func NewEtcdKV(client *clientv3.Client, rootPath string) *EtcdKV {
client, err := clientv3.New(clientv3.Config{
Endpoints: etcdEndpoints,
DialTimeout: 5 * time.Second,
})
if err != nil {
return nil, err
}
kv := &EtcdKV{ kv := &EtcdKV{
client: client, client: client,
rootPath: rootPath, rootPath: rootPath,
} }
return kv
return kv, nil
}
// NewEtcdKVWithClient creates a new etcd kv with a client.
func NewEtcdKVWithClient(cli *clientv3.Client, rootPath string) *EtcdKV {
return &EtcdKV{
client: cli,
rootPath: rootPath,
}
} }
// Close closes the connection to etcd. // Close closes the connection to etcd.
func (kv *EtcdKV) Close() { func (kv *EtcdKV) Close() {
kv.client.Close() log.Debug("etcd kv closed", zap.String("path", kv.rootPath))
} }
// GetPath returns the path of the key. // GetPath returns the path of the key.

View File

@ -18,18 +18,18 @@ package etcdkv_test
import ( import (
"os" "os"
"strings"
"testing" "testing"
"time" "time"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
) )
var Params paramtable.BaseTable var Params paramtable.GlobalParamTable
func TestMain(m *testing.M) { func TestMain(m *testing.M) {
Params.Init() Params.Init()
@ -38,17 +38,12 @@ func TestMain(m *testing.M) {
} }
func TestEtcdKV_Load(te *testing.T) { func TestEtcdKV_Load(te *testing.T) {
endpoints, err := Params.Load("_EtcdEndpoints") etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
if err != nil { defer etcdCli.Close()
panic(err) assert.NoError(te, err)
}
etcdEndPoints := strings.Split(endpoints, ",")
te.Run("EtcdKV SaveAndLoad", func(t *testing.T) { te.Run("EtcdKV SaveAndLoad", func(t *testing.T) {
rootPath := "/etcd/test/root/saveandload" rootPath := "/etcd/test/root/saveandload"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
require.NoError(t, err)
err = etcdKV.RemoveWithPrefix("") err = etcdKV.RemoveWithPrefix("")
require.NoError(t, err) require.NoError(t, err)
@ -157,8 +152,7 @@ func TestEtcdKV_Load(te *testing.T) {
te.Run("EtcdKV LoadWithRevision", func(t *testing.T) { te.Run("EtcdKV LoadWithRevision", func(t *testing.T) {
rootPath := "/etcd/test/root/LoadWithRevision" rootPath := "/etcd/test/root/LoadWithRevision"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
@ -202,8 +196,7 @@ func TestEtcdKV_Load(te *testing.T) {
te.Run("EtcdKV MultiSaveAndMultiLoad", func(t *testing.T) { te.Run("EtcdKV MultiSaveAndMultiLoad", func(t *testing.T) {
rootPath := "/etcd/test/root/multi_save_and_multi_load" rootPath := "/etcd/test/root/multi_save_and_multi_load"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
@ -311,9 +304,7 @@ func TestEtcdKV_Load(te *testing.T) {
te.Run("EtcdKV MultiRemoveWithPrefix", func(t *testing.T) { te.Run("EtcdKV MultiRemoveWithPrefix", func(t *testing.T) {
rootPath := "/etcd/test/root/multi_remove_with_prefix" rootPath := "/etcd/test/root/multi_remove_with_prefix"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
require.NoError(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
@ -399,8 +390,7 @@ func TestEtcdKV_Load(te *testing.T) {
te.Run("EtcdKV Watch", func(t *testing.T) { te.Run("EtcdKV Watch", func(t *testing.T) {
rootPath := "/etcd/test/root/watch" rootPath := "/etcd/test/root/watch"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
@ -416,9 +406,7 @@ func TestEtcdKV_Load(te *testing.T) {
te.Run("Etcd Revision", func(t *testing.T) { te.Run("Etcd Revision", func(t *testing.T) {
rootPath := "/etcd/test/root/watch" rootPath := "/etcd/test/root/watch"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
@ -467,8 +455,7 @@ func TestEtcdKV_Load(te *testing.T) {
te.Run("Etcd Lease", func(t *testing.T) { te.Run("Etcd Lease", func(t *testing.T) {
rootPath := "/etcd/test/root/lease" rootPath := "/etcd/test/root/lease"
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")

View File

@ -19,6 +19,7 @@ package etcdkv
import ( import (
"github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"go.etcd.io/etcd/server/v3/embed" "go.etcd.io/etcd/server/v3/embed"
"go.uber.org/zap" "go.uber.org/zap"
@ -49,9 +50,10 @@ func NewMetaKvFactory(rootPath string, param *paramtable.BaseParamTable) (kv.Met
} }
return metaKv, err return metaKv, err
} }
metaKv, err := NewEtcdKV(param.EtcdEndpoints, rootPath) client, err := etcd.GetEtcdClient(param)
if err != nil { if err != nil {
return nil, err return nil, err
} }
metaKv := NewEtcdKV(client, rootPath)
return metaKv, err return metaKv, err
} }

View File

@ -38,6 +38,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/mqclient" "github.com/milvus-io/milvus/internal/util/mqclient"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
@ -73,10 +74,12 @@ func (f *fixture) setup() []parameters {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
f.etcdKV, err = etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
defer etcdCli.Close()
if err != nil { if err != nil {
log.Fatalf("New clientv3 error = %v", err) log.Fatalf("New clientv3 error = %v", err)
} }
f.etcdKV = etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
idAllocator := allocator.NewGlobalIDAllocator("dummy", f.etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", f.etcdKV)
_ = idAllocator.Initialize() _ = idAllocator.Initialize()
err = rocksmq.InitRmq(rocksdbName, idAllocator) err = rocksmq.InitRmq(rocksdbName, idAllocator)
@ -1342,10 +1345,11 @@ func initRmq(name string) *etcdkv.EtcdKV {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
if err != nil { if err != nil {
log.Fatalf("New clientv3 error = %v", err) log.Fatalf("New clientv3 error = %v", err)
} }
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
_ = idAllocator.Initialize() _ = idAllocator.Initialize()

View File

@ -38,6 +38,7 @@ import (
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -67,6 +68,7 @@ type Proxy struct {
stateCode atomic.Value stateCode atomic.Value
etcdCli *clientv3.Client
rootCoord types.RootCoord rootCoord types.RootCoord
indexCoord types.IndexCoord indexCoord types.IndexCoord
dataCoord types.DataCoord dataCoord types.DataCoord
@ -116,7 +118,9 @@ func (node *Proxy) Register() error {
if err := node.Stop(); err != nil { if err := node.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if node.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
// TODO Reset the logger // TODO Reset the logger
//Params.initLogCfg() //Params.initLogCfg()
@ -125,11 +129,11 @@ func (node *Proxy) Register() error {
// initSession initialize the session of Proxy. // initSession initialize the session of Proxy.
func (node *Proxy) initSession() error { func (node *Proxy) initSession() error {
node.session = sessionutil.NewSession(node.ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints) node.session = sessionutil.NewSession(node.ctx, Params.ProxyCfg.MetaRootPath, node.etcdCli)
if node.session == nil { if node.session == nil {
return errors.New("new session failed, maybe etcd cannot be connected") return errors.New("new session failed, maybe etcd cannot be connected")
} }
node.session.Init(typeutil.ProxyRole, Params.ProxyCfg.NetworkAddress, false) node.session.Init(typeutil.ProxyRole, Params.ProxyCfg.NetworkAddress, false, true)
Params.ProxyCfg.ProxyID = node.session.ServerID Params.ProxyCfg.ProxyID = node.session.ServerID
Params.BaseParams.SetLogger(Params.ProxyCfg.ProxyID) Params.BaseParams.SetLogger(Params.ProxyCfg.ProxyID)
return nil return nil
@ -425,6 +429,11 @@ func (node *Proxy) AddCloseCallback(callbacks ...func()) {
node.closeCallbacks = append(node.closeCallbacks, callbacks...) node.closeCallbacks = append(node.closeCallbacks, callbacks...)
} }
// SetEtcdClient sets etcd client for proxy.
func (node *Proxy) SetEtcdClient(client *clientv3.Client) {
node.etcdCli = client
}
// SetRootCoordClient sets RootCoord client for proxy. // SetRootCoordClient sets RootCoord client for proxy.
func (node *Proxy) SetRootCoordClient(cli types.RootCoord) { func (node *Proxy) SetRootCoordClient(cli types.RootCoord) {
node.rootCoord = cli node.rootCoord = cli

View File

@ -29,6 +29,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"go.uber.org/zap" "go.uber.org/zap"
@ -211,13 +212,9 @@ func runDataCoord(ctx context.Context, localMsg bool) *grpcdatacoordclient.Serve
} }
factory := newMsgFactory(localMsg) factory := newMsgFactory(localMsg)
var err error ds = grpcdatacoordclient.NewServer(ctx, factory)
ds, err = grpcdatacoordclient.NewServer(ctx, factory)
if err != nil {
panic(err)
}
wg.Done() wg.Done()
err = ds.Run() err := ds.Run()
if err != nil { if err != nil {
panic(err) panic(err)
} }
@ -310,6 +307,11 @@ func runIndexNode(ctx context.Context, localMsg bool, alias string) *grpcindexno
panic(err) panic(err)
} }
wg.Done() wg.Done()
etcd, err := etcd.GetEtcdClient(&indexnode.Params.BaseParams)
if err != nil {
panic(err)
}
in.SetEtcdClient(etcd)
err = in.Run() err = in.Run()
if err != nil { if err != nil {
panic(err) panic(err)
@ -419,7 +421,11 @@ func TestProxy(t *testing.T) {
Params.Init() Params.Init()
log.Info("Initialize parameter table of Proxy") log.Info("Initialize parameter table of Proxy")
rootCoordClient, err := rcc.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints) etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdcli.Close()
assert.NoError(t, err)
proxy.SetEtcdClient(etcdcli)
rootCoordClient, err := rcc.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = rootCoordClient.Init() err = rootCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -428,7 +434,7 @@ func TestProxy(t *testing.T) {
proxy.SetRootCoordClient(rootCoordClient) proxy.SetRootCoordClient(rootCoordClient)
log.Info("Proxy set root coordinator client") log.Info("Proxy set root coordinator client")
dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints) dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = dataCoordClient.Init() err = dataCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -437,7 +443,7 @@ func TestProxy(t *testing.T) {
proxy.SetDataCoordClient(dataCoordClient) proxy.SetDataCoordClient(dataCoordClient)
log.Info("Proxy set data coordinator client") log.Info("Proxy set data coordinator client")
queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints) queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = queryCoordClient.Init() err = queryCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -446,7 +452,7 @@ func TestProxy(t *testing.T) {
proxy.SetQueryCoordClient(queryCoordClient) proxy.SetQueryCoordClient(queryCoordClient)
log.Info("Proxy set query coordinator client") log.Info("Proxy set query coordinator client")
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints) indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = indexCoordClient.Init() err = indexCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -456,7 +462,6 @@ func TestProxy(t *testing.T) {
log.Info("Proxy set index coordinator client") log.Info("Proxy set index coordinator client")
proxy.UpdateStateCode(internalpb.StateCode_Initializing) proxy.UpdateStateCode(internalpb.StateCode_Initializing)
err = proxy.Init() err = proxy.Init()
assert.NoError(t, err) assert.NoError(t, err)

View File

@ -25,6 +25,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
@ -32,10 +33,13 @@ import (
func TestShuffleChannelsToQueryNode(t *testing.T) { func TestShuffleChannelsToQueryNode(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.Nil(t, err) assert.Nil(t, err)
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true) clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
clusterSession.Register() clusterSession.Register()
meta, err := newMeta(baseCtx, kv, nil, nil) meta, err := newMeta(baseCtx, kv, nil, nil)
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -41,6 +41,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
@ -390,13 +391,15 @@ func TestQueryNodeCluster_getMetrics(t *testing.T) {
} }
func TestReloadClusterFromKV(t *testing.T) { func TestReloadClusterFromKV(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.Nil(t, err)
t.Run("Test LoadOnlineNodes", func(t *testing.T) { t.Run("Test LoadOnlineNodes", func(t *testing.T) {
refreshParams() refreshParams()
baseCtx := context.Background() baseCtx := context.Background()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
assert.Nil(t, err) clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
clusterSession.Register() clusterSession.Register()
cluster := &queryNodeCluster{ cluster := &queryNodeCluster{
ctx: baseCtx, ctx: baseCtx,
@ -422,10 +425,9 @@ func TestReloadClusterFromKV(t *testing.T) {
t.Run("Test LoadOfflineNodes", func(t *testing.T) { t.Run("Test LoadOfflineNodes", func(t *testing.T) {
refreshParams() refreshParams()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
assert.Nil(t, err) clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
clusterSession.Register() clusterSession.Register()
cluster := &queryNodeCluster{ cluster := &queryNodeCluster{
client: kv, client: kv,
@ -459,10 +461,12 @@ func TestReloadClusterFromKV(t *testing.T) {
func TestGrpcRequest(t *testing.T) { func TestGrpcRequest(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) defer etcdCli.Close()
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true) kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
clusterSession.Register() clusterSession.Register()
factory := msgstream.NewPmsFactory() factory := msgstream.NewPmsFactory()
m := map[string]interface{}{ m := map[string]interface{}{

View File

@ -28,14 +28,17 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
) )
func TestReloadFromKV(t *testing.T) { func TestReloadFromKV(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.Nil(t, err) assert.Nil(t, err)
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
meta, err := newMeta(baseCtx, kv, nil, nil) meta, err := newMeta(baseCtx, kv, nil, nil)
assert.Nil(t, err) assert.Nil(t, err)
@ -88,8 +91,10 @@ func TestReloadFromKV(t *testing.T) {
func TestCheckIndexLoop(t *testing.T) { func TestCheckIndexLoop(t *testing.T) {
refreshParams() refreshParams()
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.Nil(t, err) assert.Nil(t, err)
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
meta, err := newMeta(ctx, kv, nil, nil) meta, err := newMeta(ctx, kv, nil, nil)
assert.Nil(t, err) assert.Nil(t, err)
@ -152,8 +157,11 @@ func TestCheckIndexLoop(t *testing.T) {
func TestProcessHandoffAfterIndexDone(t *testing.T) { func TestProcessHandoffAfterIndexDone(t *testing.T) {
refreshParams() refreshParams()
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
meta, err := newMeta(ctx, kv, nil, nil) meta, err := newMeta(ctx, kv, nil, nil)
assert.Nil(t, err) assert.Nil(t, err)
taskScheduler := &TaskScheduler{ taskScheduler := &TaskScheduler{
@ -162,8 +170,7 @@ func TestProcessHandoffAfterIndexDone(t *testing.T) {
client: kv, client: kv,
triggerTaskQueue: NewTaskQueue(), triggerTaskQueue: NewTaskQueue(),
} }
idAllocatorKV, err := tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID") idAllocatorKV := tsoutil.NewTSOKVBase(etcdCli, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
assert.Nil(t, err)
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV) idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
err = idAllocator.Initialize() err = idAllocator.Initialize()
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -31,6 +31,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util" "github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/etcd"
) )
func successResult() error { return nil } func successResult() error { return nil }
@ -63,8 +64,10 @@ func (tk *testKv) Load(key string) (string, error) {
func TestReplica_Release(t *testing.T) { func TestReplica_Release(t *testing.T) {
refreshParams() refreshParams()
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
meta, err := newMeta(context.Background(), etcdKV, nil, nil) meta, err := newMeta(context.Background(), etcdKV, nil, nil)
assert.Nil(t, err) assert.Nil(t, err)
err = meta.addCollection(1, querypb.LoadType_loadCollection, nil) err = meta.addCollection(1, querypb.LoadType_loadCollection, nil)
@ -93,8 +96,10 @@ func TestReplica_Release(t *testing.T) {
func TestMetaFunc(t *testing.T) { func TestMetaFunc(t *testing.T) {
refreshParams() refreshParams()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
nodeID := defaultQueryNodeID nodeID := defaultQueryNodeID
segmentInfos := make(map[UniqueID]*querypb.SegmentInfo) segmentInfos := make(map[UniqueID]*querypb.SegmentInfo)
@ -285,8 +290,10 @@ func TestMetaFunc(t *testing.T) {
func TestReloadMetaFromKV(t *testing.T) { func TestReloadMetaFromKV(t *testing.T) {
refreshParams() refreshParams()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
meta := &MetaReplica{ meta := &MetaReplica{
client: kv, client: kv,
collectionInfos: map[UniqueID]*querypb.CollectionInfo{}, collectionInfos: map[UniqueID]*querypb.CollectionInfo{},

View File

@ -31,6 +31,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
@ -100,9 +101,13 @@ func newQueryNodeServerMock(ctx context.Context) *queryNodeServerMock {
} }
func (qs *queryNodeServerMock) Register() error { func (qs *queryNodeServerMock) Register() error {
log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints)) log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath))
qs.session = sessionutil.NewSession(qs.ctx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
qs.session.Init(typeutil.QueryNodeRole, qs.queryNodeIP+":"+strconv.FormatInt(qs.queryNodePort, 10), false) if err != nil {
return err
}
qs.session = sessionutil.NewSession(qs.ctx, Params.QueryCoordCfg.MetaRootPath, etcdCli)
qs.session.Init(typeutil.QueryNodeRole, qs.queryNodeIP+":"+strconv.FormatInt(qs.queryNodePort, 10), false, false)
qs.queryNodeID = qs.session.ServerID qs.queryNodeID = qs.session.ServerID
log.Debug("query nodeID", zap.Int64("nodeID", qs.queryNodeID)) log.Debug("query nodeID", zap.Int64("nodeID", qs.queryNodeID))
log.Debug("query node address", zap.String("address", qs.session.Address)) log.Debug("query node address", zap.String("address", qs.session.Address))

View File

@ -40,11 +40,11 @@ import (
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -84,6 +84,7 @@ type QueryCoord struct {
metricsCacheManager *metricsinfo.MetricsCacheManager metricsCacheManager *metricsinfo.MetricsCacheManager
etcdCli *clientv3.Client
dataCoordClient types.DataCoord dataCoordClient types.DataCoord
rootCoordClient types.RootCoord rootCoordClient types.RootCoord
indexCoordClient types.IndexCoord indexCoordClient types.IndexCoord
@ -106,17 +107,19 @@ func (qc *QueryCoord) Register() error {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if qc.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
return nil return nil
} }
func (qc *QueryCoord) initSession() error { func (qc *QueryCoord) initSession() error {
qc.session = sessionutil.NewSession(qc.loopCtx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) qc.session = sessionutil.NewSession(qc.loopCtx, Params.QueryCoordCfg.MetaRootPath, qc.etcdCli)
if qc.session == nil { if qc.session == nil {
return fmt.Errorf("session is nil, the etcd client connection may have failed") return fmt.Errorf("session is nil, the etcd client connection may have failed")
} }
qc.session.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true) qc.session.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, true)
Params.QueryCoordCfg.NodeID = uint64(qc.session.ServerID) Params.QueryCoordCfg.NodeID = uint64(qc.session.ServerID)
Params.BaseParams.SetLogger(qc.session.ServerID) Params.BaseParams.SetLogger(qc.session.ServerID)
return nil return nil
@ -124,17 +127,7 @@ func (qc *QueryCoord) initSession() error {
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler // Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
func (qc *QueryCoord) Init() error { func (qc *QueryCoord) Init() error {
log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.String("address", Params.QueryCoordCfg.Address))
zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints), zap.String("address", Params.QueryCoordCfg.Address))
//connect etcd
connectEtcdFn := func() error {
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
if err != nil {
return err
}
qc.kvClient = etcdKV
return nil
}
var initError error var initError error
qc.initOnce.Do(func() { qc.initOnce.Do(func() {
err := qc.initSession() err := qc.initSession()
@ -144,20 +137,12 @@ func (qc *QueryCoord) Init() error {
return return
} }
log.Debug("queryCoord try to connect etcd") log.Debug("queryCoord try to connect etcd")
initError = retry.Do(qc.loopCtx, connectEtcdFn, retry.Attempts(300)) etcdKV := etcdkv.NewEtcdKV(qc.etcdCli, Params.QueryCoordCfg.MetaRootPath)
if initError != nil { qc.kvClient = etcdKV
log.Debug("query coordinator try to connect etcd failed", zap.Error(initError))
return
}
log.Debug("query coordinator try to connect etcd success") log.Debug("query coordinator try to connect etcd success")
// init id allocator // init id allocator
var idAllocatorKV *etcdkv.EtcdKV idAllocatorKV := tsoutil.NewTSOKVBase(qc.etcdCli, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
if initError != nil {
log.Debug("query coordinator idAllocatorKV initialize failed", zap.Error(initError))
return
}
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV) idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
initError = idAllocator.Initialize() initError = idAllocator.Initialize()
if initError != nil { if initError != nil {
@ -294,6 +279,11 @@ func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord,
return service, nil return service, nil
} }
// SetRootCoord sets root coordinator's client
func (qc *QueryCoord) SetEtcdClient(etcdClient *clientv3.Client) {
qc.etcdCli = etcdClient
}
// SetRootCoord sets root coordinator's client // SetRootCoord sets root coordinator's client
func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error { func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error {
if rootCoord == nil { if rootCoord == nil {

View File

@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/msgstream" "github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -91,7 +92,11 @@ func startQueryCoord(ctx context.Context) (*QueryCoord, error) {
coord.SetRootCoord(rootCoord) coord.SetRootCoord(rootCoord)
coord.SetDataCoord(dataCoord) coord.SetDataCoord(dataCoord)
coord.SetIndexCoord(indexCoord) coord.SetIndexCoord(indexCoord)
etcd, err := etcd.GetEtcdClient(&Params.BaseParams)
if err != nil {
return nil, err
}
coord.SetEtcdClient(etcd)
err = coord.Init() err = coord.Init()
if err != nil { if err != nil {
return nil, err return nil, err
@ -132,7 +137,11 @@ func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) {
coord.SetRootCoord(rootCoord) coord.SetRootCoord(rootCoord)
coord.SetDataCoord(dataCoord) coord.SetDataCoord(dataCoord)
etcd, err := etcd.GetEtcdClient(&Params.BaseParams)
if err != nil {
return nil, err
}
coord.SetEtcdClient(etcd)
err = coord.Init() err = coord.Init()
if err != nil { if err != nil {
return nil, err return nil, err
@ -147,11 +156,12 @@ func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) {
func TestWatchNodeLoop(t *testing.T) { func TestWatchNodeLoop(t *testing.T) {
baseCtx := context.Background() baseCtx := context.Background()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
t.Run("Test OfflineNodes", func(t *testing.T) { t.Run("Test OfflineNodes", func(t *testing.T) {
refreshParams() refreshParams()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
assert.Nil(t, err) kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
kvs := make(map[string]string) kvs := make(map[string]string)
session := &sessionutil.Session{ session := &sessionutil.Session{

View File

@ -30,16 +30,20 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
//func waitQueryNodeOnline(cluster *queryNodeCluster, nodeID int64) //func waitQueryNodeOnline(cluster *queryNodeCluster, nodeID int64)
func removeNodeSession(id int64) error { func removeNodeSession(id int64) error {
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
if err != nil { if err != nil {
return err return err
} }
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
err = kv.Remove(fmt.Sprintf("session/"+typeutil.QueryNodeRole+"-%d", id)) err = kv.Remove(fmt.Sprintf("session/"+typeutil.QueryNodeRole+"-%d", id))
if err != nil { if err != nil {
return err return err
@ -48,10 +52,12 @@ func removeNodeSession(id int64) error {
} }
func removeAllSession() error { func removeAllSession() error {
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
if err != nil { if err != nil {
return err return err
} }
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
err = kv.RemoveWithPrefix("session") err = kv.RemoveWithPrefix("session")
if err != nil { if err != nil {
return err return err
@ -187,8 +193,10 @@ func TestQueryNode_getMetrics(t *testing.T) {
func TestNewQueryNode(t *testing.T) { func TestNewQueryNode(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
queryNode1, err := startQueryNodeServer(baseCtx) queryNode1, err := startQueryNodeServer(baseCtx)
assert.Nil(t, err) assert.Nil(t, err)
@ -211,8 +219,10 @@ func TestNewQueryNode(t *testing.T) {
func TestReleaseCollectionOnOfflineNode(t *testing.T) { func TestReleaseCollectionOnOfflineNode(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
node, err := newQueryNode(baseCtx, "test", 100, kv) node, err := newQueryNode(baseCtx, "test", 100, kv)
assert.Nil(t, err) assert.Nil(t, err)
@ -280,8 +290,10 @@ func TestSealedSegmentChangeAfterQueryNodeStop(t *testing.T) {
func TestGrpcRequestWithNodeOffline(t *testing.T) { func TestGrpcRequestWithNodeOffline(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
nodeServer, err := startQueryNodeServer(baseCtx) nodeServer, err := startQueryNodeServer(baseCtx)
assert.Nil(t, err) assert.Nil(t, err)
address := nodeServer.queryNodeIP address := nodeServer.queryNodeIP

View File

@ -25,6 +25,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
minioKV "github.com/milvus-io/milvus/internal/kv/minio" minioKV "github.com/milvus-io/milvus/internal/kv/minio"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
@ -32,10 +33,12 @@ import (
func TestShuffleSegmentsToQueryNode(t *testing.T) { func TestShuffleSegmentsToQueryNode(t *testing.T) {
refreshParams() refreshParams()
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer etcdCli.Close()
assert.Nil(t, err) assert.Nil(t, err)
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true) clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
meta, err := newMeta(baseCtx, kv, nil, nil) meta, err := newMeta(baseCtx, kv, nil, nil)
assert.Nil(t, err) assert.Nil(t, err)
cluster := &queryNodeCluster{ cluster := &queryNodeCluster{

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -208,8 +209,10 @@ func TestWatchQueryChannel_ClearEtcdInfoAfterAssignedNodeDown(t *testing.T) {
func TestUnMarshalTask(t *testing.T) { func TestUnMarshalTask(t *testing.T) {
refreshParams() refreshParams()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
taskScheduler := &TaskScheduler{ taskScheduler := &TaskScheduler{
ctx: baseCtx, ctx: baseCtx,
@ -454,7 +457,10 @@ func TestUnMarshalTask(t *testing.T) {
func TestReloadTaskFromKV(t *testing.T) { func TestReloadTaskFromKV(t *testing.T) {
refreshParams() refreshParams()
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
defer etcdCli.Close()
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
assert.Nil(t, err) assert.Nil(t, err)
baseCtx, cancel := context.WithCancel(context.Background()) baseCtx, cancel := context.WithCancel(context.Background())
taskScheduler := &TaskScheduler{ taskScheduler := &TaskScheduler{

View File

@ -20,6 +20,7 @@ import (
"context" "context"
"encoding/json" "encoding/json"
"math/rand" "math/rand"
"sync"
"sync/atomic" "sync/atomic"
"testing" "testing"
@ -29,6 +30,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
queryPb "github.com/milvus-io/milvus/internal/proto/querypb" queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
) )
@ -228,7 +230,10 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
var wg sync.WaitGroup
wg.Add(1)
t.Run("test GetSegmentInfo", func(t *testing.T) { t.Run("test GetSegmentInfo", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -251,7 +256,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test no collection in historical", func(t *testing.T) { t.Run("test no collection in historical", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -272,7 +279,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test no collection in streaming", func(t *testing.T) { t.Run("test no collection in streaming", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -293,7 +302,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test different segment type", func(t *testing.T) { t.Run("test different segment type", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -335,7 +346,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test GetSegmentInfo with indexed segment", func(t *testing.T) { t.Run("test GetSegmentInfo with indexed segment", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -367,7 +380,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test GetSegmentInfo without streaming partition", func(t *testing.T) { t.Run("test GetSegmentInfo without streaming partition", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -386,7 +401,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test GetSegmentInfo without streaming segment", func(t *testing.T) { t.Run("test GetSegmentInfo without streaming segment", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -405,7 +422,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test GetSegmentInfo without historical partition", func(t *testing.T) { t.Run("test GetSegmentInfo without historical partition", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -424,7 +443,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("test GetSegmentInfo without historical segment", func(t *testing.T) { t.Run("test GetSegmentInfo without historical segment", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -442,6 +463,7 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
}) })
wg.Wait()
} }
func TestImpl_isHealthy(t *testing.T) { func TestImpl_isHealthy(t *testing.T) {
@ -458,11 +480,17 @@ func TestImpl_GetMetrics(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
defer etcdCli.Close()
var wg sync.WaitGroup
wg.Add(1)
t.Run("test GetMetrics", func(t *testing.T) { t.Run("test GetMetrics", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, etcdCli)
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints)
metricReq := make(map[string]string) metricReq := make(map[string]string)
metricReq[metricsinfo.MetricTypeKey] = "system_info" metricReq[metricsinfo.MetricTypeKey] = "system_info"
@ -481,7 +509,9 @@ func TestImpl_GetMetrics(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
}) })
wg.Add(1)
t.Run("test ParseMetricType failed", func(t *testing.T) { t.Run("test ParseMetricType failed", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -499,13 +529,17 @@ func TestImpl_GetMetrics(t *testing.T) {
_, err = node.GetMetrics(ctx, req) _, err = node.GetMetrics(ctx, req)
assert.NoError(t, err) assert.NoError(t, err)
}) })
wg.Wait()
} }
func TestImpl_ReleaseSegments(t *testing.T) { func TestImpl_ReleaseSegments(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
var wg sync.WaitGroup
wg.Add(1)
t.Run("test valid", func(t *testing.T) { t.Run("test valid", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -520,7 +554,9 @@ func TestImpl_ReleaseSegments(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
}) })
wg.Add(1)
t.Run("test invalid query node", func(t *testing.T) { t.Run("test invalid query node", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -536,7 +572,9 @@ func TestImpl_ReleaseSegments(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
}) })
wg.Add(1)
t.Run("test segment not exists", func(t *testing.T) { t.Run("test segment not exists", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -557,4 +595,5 @@ func TestImpl_ReleaseSegments(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
assert.NotEqual(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.NotEqual(t, commonpb.ErrorCode_Success, status.ErrorCode)
}) })
wg.Wait()
} }

View File

@ -24,6 +24,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
) )
@ -34,7 +35,10 @@ func TestGetSystemInfoMetrics(t *testing.T) {
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
defer etcdCli.Close()
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, etcdCli)
req := &milvuspb.GetMetricsRequest{ req := &milvuspb.GetMetricsRequest{
Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels), Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels),

View File

@ -42,6 +42,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util" "github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
) )
@ -370,8 +371,12 @@ func genMinioKV(ctx context.Context) (*minioKV.MinIOKV, error) {
} }
func genEtcdKV() (*etcdkv.EtcdKV, error) { func genEtcdKV() (*etcdkv.EtcdKV, error) {
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
return etcdKV, err if err != nil {
return nil, err
}
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
return etcdKV, nil
} }
func genFactory() (msgstream.Factory, error) { func genFactory() (msgstream.Factory, error) {
@ -1275,14 +1280,12 @@ func genSimpleChangeInfo() *querypb.SealedSegmentsChangeInfo {
func saveChangeInfo(key string, value string) error { func saveChangeInfo(key string, value string) error {
log.Debug(".. [query node unittest] Saving change info") log.Debug(".. [query node unittest] Saving change info")
kv, err := genEtcdKV() kv, err := genEtcdKV()
if err != nil { if err != nil {
return err return err
} }
key = util.ChangeInfoMetaPrefix + "/" + key key = util.ChangeInfoMetaPrefix + "/" + key
return kv.Save(key, value) return kv.Save(key, value)
} }
@ -1293,16 +1296,17 @@ func genSimpleQueryNode(ctx context.Context) (*QueryNode, error) {
return nil, err return nil, err
} }
node := NewQueryNode(ctx, fac) node := NewQueryNode(ctx, fac)
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
if err != nil {
return nil, err
}
node.etcdCli = etcdCli
session := &sessionutil.Session{ session := &sessionutil.Session{
ServerID: 1, ServerID: 1,
} }
node.session = session node.session = session
etcdKV, err := genEtcdKV() etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
if err != nil {
return nil, err
}
node.etcdKV = etcdKV node.etcdKV = etcdKV
node.tSafeReplica = newTSafeReplica() node.tSafeReplica = newTSafeReplica()

View File

@ -40,6 +40,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/proto/segcorepb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
@ -131,8 +132,10 @@ func TestQueryCollection_withoutVChannel(t *testing.T) {
factory := msgstream.NewPmsFactory() factory := msgstream.NewPmsFactory()
err := factory.SetParams(m) err := factory.SetParams(m)
assert.Nil(t, err) assert.Nil(t, err)
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
schema := genTestCollectionSchema(0, false, 2) schema := genTestCollectionSchema(0, false, 2)
historicalReplica := newCollectionReplica(etcdKV) historicalReplica := newCollectionReplica(etcdKV)
@ -689,8 +692,10 @@ func TestQueryCollection_AddPopUnsolvedMsg(t *testing.T) {
func TestQueryCollection_adjustByChangeInfo(t *testing.T) { func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO()) ctx, cancel := context.WithCancel(context.TODO())
var wg sync.WaitGroup
wg.Add(1)
t.Run("test adjustByChangeInfo", func(t *testing.T) { t.Run("test adjustByChangeInfo", func(t *testing.T) {
defer wg.Done()
qc, err := genSimpleQueryCollection(ctx, cancel) qc, err := genSimpleQueryCollection(ctx, cancel)
assert.Nil(t, err) assert.Nil(t, err)
@ -710,7 +715,9 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
assert.Len(t, ids, 0) assert.Len(t, ids, 0)
}) })
wg.Add(1)
t.Run("test mismatch collectionID when adjustByChangeInfo", func(t *testing.T) { t.Run("test mismatch collectionID when adjustByChangeInfo", func(t *testing.T) {
defer wg.Done()
qc, err := genSimpleQueryCollection(ctx, cancel) qc, err := genSimpleQueryCollection(ctx, cancel)
assert.Nil(t, err) assert.Nil(t, err)
@ -723,7 +730,9 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
qc.adjustByChangeInfo(segmentChangeInfos) qc.adjustByChangeInfo(segmentChangeInfos)
}) })
wg.Add(1)
t.Run("test no segment when adjustByChangeInfo", func(t *testing.T) { t.Run("test no segment when adjustByChangeInfo", func(t *testing.T) {
defer wg.Done()
qc, err := genSimpleQueryCollection(ctx, cancel) qc, err := genSimpleQueryCollection(ctx, cancel)
assert.Nil(t, err) assert.Nil(t, err)
@ -735,13 +744,17 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
qc.adjustByChangeInfo(segmentChangeInfos) qc.adjustByChangeInfo(segmentChangeInfos)
}) })
wg.Wait()
} }
func TestQueryCollection_search_while_release(t *testing.T) { func TestQueryCollection_search_while_release(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
var wgAll sync.WaitGroup
wgAll.Add(1)
t.Run("test search while release collection", func(t *testing.T) { t.Run("test search while release collection", func(t *testing.T) {
defer wgAll.Done()
queryCollection, err := genSimpleQueryCollection(ctx, cancel) queryCollection, err := genSimpleQueryCollection(ctx, cancel)
assert.NoError(t, err) assert.NoError(t, err)
@ -778,7 +791,9 @@ func TestQueryCollection_search_while_release(t *testing.T) {
wg.Wait() wg.Wait()
}) })
wgAll.Add(1)
t.Run("test search while release partition", func(t *testing.T) { t.Run("test search while release partition", func(t *testing.T) {
defer wgAll.Done()
queryCollection, err := genSimpleQueryCollection(ctx, cancel) queryCollection, err := genSimpleQueryCollection(ctx, cancel)
assert.NoError(t, err) assert.NoError(t, err)
@ -814,4 +829,5 @@ func TestQueryCollection_search_while_release(t *testing.T) {
} }
wg.Wait() wg.Wait()
}) })
wgAll.Wait()
} }

View File

@ -55,6 +55,7 @@ import (
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -99,6 +100,9 @@ type QueryNode struct {
// segment loader // segment loader
loader *segmentLoader loader *segmentLoader
// etcd client
etcdCli *clientv3.Client
// clients // clients
rootCoord types.RootCoord rootCoord types.RootCoord
indexCoord types.IndexCoord indexCoord types.IndexCoord
@ -129,11 +133,11 @@ func NewQueryNode(ctx context.Context, factory msgstream.Factory) *QueryNode {
} }
func (node *QueryNode) initSession() error { func (node *QueryNode) initSession() error {
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, node.etcdCli)
if node.session == nil { if node.session == nil {
return fmt.Errorf("session is nil, the etcd client connection may have failed") return fmt.Errorf("session is nil, the etcd client connection may have failed")
} }
node.session.Init(typeutil.QueryNodeRole, Params.QueryNodeCfg.QueryNodeIP+":"+strconv.FormatInt(Params.QueryNodeCfg.QueryNodePort, 10), false) node.session.Init(typeutil.QueryNodeRole, Params.QueryNodeCfg.QueryNodeIP+":"+strconv.FormatInt(Params.QueryNodeCfg.QueryNodePort, 10), false, true)
Params.QueryNodeCfg.QueryNodeID = node.session.ServerID Params.QueryNodeCfg.QueryNodeID = node.session.ServerID
Params.BaseParams.SetLogger(Params.QueryNodeCfg.QueryNodeID) Params.BaseParams.SetLogger(Params.QueryNodeCfg.QueryNodeID)
log.Debug("QueryNode", zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID), zap.String("node address", node.session.Address)) log.Debug("QueryNode", zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID), zap.String("node address", node.session.Address))
@ -150,7 +154,9 @@ func (node *QueryNode) Register() error {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if node.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
//TODO Reset the logger //TODO Reset the logger
@ -179,7 +185,7 @@ func (node *QueryNode) Init() error {
var initError error = nil var initError error = nil
node.initOnce.Do(func() { node.initOnce.Do(func() {
//ctx := context.Background() //ctx := context.Background()
log.Debug("QueryNode session info", zap.String("metaPath", Params.QueryNodeCfg.MetaRootPath), zap.Strings("etcdEndPoints", Params.QueryNodeCfg.EtcdEndpoints)) log.Debug("QueryNode session info", zap.String("metaPath", Params.QueryNodeCfg.MetaRootPath))
err := node.initSession() err := node.initSession()
if err != nil { if err != nil {
log.Error("QueryNode init session failed", zap.Error(err)) log.Error("QueryNode init session failed", zap.Error(err))
@ -187,28 +193,9 @@ func (node *QueryNode) Init() error {
return return
} }
Params.QueryNodeCfg.Refresh() Params.QueryNodeCfg.Refresh()
connectEtcdFn := func() error {
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) node.etcdKV = etcdkv.NewEtcdKV(node.etcdCli, Params.QueryNodeCfg.MetaRootPath)
if err != nil { log.Debug("queryNode try to connect etcd success", zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath))
return err
}
node.etcdKV = etcdKV
return err
}
log.Debug("queryNode try to connect etcd",
zap.Any("EtcdEndpoints", Params.QueryNodeCfg.EtcdEndpoints),
zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath),
)
err = retry.Do(node.queryNodeLoopCtx, connectEtcdFn, retry.Attempts(300))
if err != nil {
log.Debug("queryNode try to connect etcd failed", zap.Error(err))
initError = err
return
}
log.Debug("queryNode try to connect etcd success",
zap.Any("EtcdEndpoints", Params.QueryNodeCfg.EtcdEndpoints),
zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath),
)
node.tSafeReplica = newTSafeReplica() node.tSafeReplica = newTSafeReplica()
streamingReplica := newCollectionReplica(node.etcdKV) streamingReplica := newCollectionReplica(node.etcdKV)
@ -328,6 +315,11 @@ func (node *QueryNode) UpdateStateCode(code internalpb.StateCode) {
node.stateCode.Store(code) node.stateCode.Store(code)
} }
// SetEtcdClient assigns parameter client to its member etcdCli
func (node *QueryNode) SetEtcdClient(client *clientv3.Client) {
node.etcdCli = client
}
// SetRootCoord assigns parameter rc to its member rootCoord. // SetRootCoord assigns parameter rc to its member rootCoord.
func (node *QueryNode) SetRootCoord(rc types.RootCoord) error { func (node *QueryNode) SetRootCoord(rc types.RootCoord) error {
if rc == nil { if rc == nil {

View File

@ -21,6 +21,7 @@ import (
"math/rand" "math/rand"
"os" "os"
"strconv" "strconv"
"sync"
"testing" "testing"
"time" "time"
@ -34,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
) )
// mock of query coordinator client // mock of query coordinator client
@ -186,11 +188,11 @@ func newQueryNodeMock() *QueryNode {
cancel() cancel()
}() }()
} }
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath)
if err != nil { if err != nil {
panic(err) panic(err)
} }
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
msFactory, err := newMessageStreamFactory() msFactory, err := newMessageStreamFactory()
if err != nil { if err != nil {
@ -270,9 +272,14 @@ func TestQueryNode_register(t *testing.T) {
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
defer etcdcli.Close()
node.SetEtcdClient(etcdcli)
err = node.initSession() err = node.initSession()
assert.NoError(t, err) assert.NoError(t, err)
node.session.TriggerKill = false
err = node.Register() err = node.Register()
assert.NoError(t, err) assert.NoError(t, err)
} }
@ -283,7 +290,10 @@ func TestQueryNode_init(t *testing.T) {
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
defer etcdcli.Close()
node.SetEtcdClient(etcdcli)
err = node.Init() err = node.Init()
assert.Error(t, err) assert.Error(t, err)
} }
@ -322,7 +332,10 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
var wg sync.WaitGroup
wg.Add(1)
t.Run("test cleanup segments", func(t *testing.T) { t.Run("test cleanup segments", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -330,7 +343,9 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
}) })
wg.Add(1)
t.Run("test cleanup segments no segment", func(t *testing.T) { t.Run("test cleanup segments no segment", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -348,13 +363,16 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) {
err = node.removeSegments(segmentChangeInfos) err = node.removeSegments(segmentChangeInfos)
assert.Error(t, err) assert.Error(t, err)
}) })
wg.Wait()
} }
func TestQueryNode_watchChangeInfo(t *testing.T) { func TestQueryNode_watchChangeInfo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
var wg sync.WaitGroup
wg.Add(1)
t.Run("test watchChangeInfo", func(t *testing.T) { t.Run("test watchChangeInfo", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -369,7 +387,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
}) })
wg.Add(1)
t.Run("test watchChangeInfo key error", func(t *testing.T) { t.Run("test watchChangeInfo key error", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -381,7 +401,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
}) })
wg.Add(1)
t.Run("test watchChangeInfo unmarshal error", func(t *testing.T) { t.Run("test watchChangeInfo unmarshal error", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -393,7 +415,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
}) })
wg.Add(1)
t.Run("test watchChangeInfo adjustByChangeInfo error", func(t *testing.T) { t.Run("test watchChangeInfo adjustByChangeInfo error", func(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -417,4 +441,5 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
}) })
wg.Wait()
} }

View File

@ -19,6 +19,7 @@ package rootcoord
import ( import (
"context" "context"
"errors" "errors"
"sync"
"testing" "testing"
"github.com/milvus-io/milvus/internal/msgstream" "github.com/milvus-io/milvus/internal/msgstream"
@ -82,12 +83,17 @@ func TestDmlChannels(t *testing.T) {
} }
func TestDmChannelsFailure(t *testing.T) { func TestDmChannelsFailure(t *testing.T) {
var wg sync.WaitGroup
wg.Add(1)
t.Run("Test newDmlChannels", func(t *testing.T) { t.Run("Test newDmlChannels", func(t *testing.T) {
defer wg.Done()
mockFactory := &FailMessageStreamFactory{} mockFactory := &FailMessageStreamFactory{}
assert.Panics(t, func() { newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) }) assert.Panics(t, func() { newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) })
}) })
wg.Add(1)
t.Run("Test broadcast", func(t *testing.T) { t.Run("Test broadcast", func(t *testing.T) {
defer wg.Done()
mockFactory := &FailMessageStreamFactory{errBroadcast: true} mockFactory := &FailMessageStreamFactory{errBroadcast: true}
dml := newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) dml := newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1)
chanName0 := dml.getChannelName() chanName0 := dml.getChannelName()
@ -101,6 +107,7 @@ func TestDmChannelsFailure(t *testing.T) {
assert.Empty(t, v) assert.Empty(t, v)
assert.Error(t, err) assert.Error(t, err)
}) })
wg.Wait()
} }
// FailMessageStreamFactory mock MessageStreamFactory failure // FailMessageStreamFactory mock MessageStreamFactory failure

View File

@ -24,10 +24,9 @@ import (
"testing" "testing"
"time" "time"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
func TestMetaSnapshot(t *testing.T) { func TestMetaSnapshot(t *testing.T) {
@ -38,7 +37,7 @@ func TestMetaSnapshot(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
tsKey := "timestamp" tsKey := "timestamp"
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
@ -178,7 +177,7 @@ func TestGetRevOnEtcd(t *testing.T) {
tsKey := "timestamp" tsKey := "timestamp"
key := path.Join(rootPath, tsKey) key := path.Join(rootPath, tsKey)
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
@ -222,7 +221,7 @@ func TestLoad(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
tsKey := "timestamp" tsKey := "timestamp"
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
@ -270,7 +269,7 @@ func TestMultiSave(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
tsKey := "timestamp" tsKey := "timestamp"
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
@ -334,8 +333,9 @@ func TestMultiSaveAndRemoveWithPrefix(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
tsKey := "timestamp" tsKey := "timestamp"
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
var vtso typeutil.Timestamp var vtso typeutil.Timestamp
ftso := func() typeutil.Timestamp { ftso := func() typeutil.Timestamp {
@ -411,7 +411,7 @@ func TestTsBackward(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
tsKey := "timestamp" tsKey := "timestamp"
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
@ -438,7 +438,7 @@ func TestFix7150(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
tsKey := "timestamp" tsKey := "timestamp"
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()

View File

@ -21,6 +21,7 @@ import (
"fmt" "fmt"
"math/rand" "math/rand"
"path" "path"
"sync"
"testing" "testing"
"time" "time"
@ -31,9 +32,10 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
pb "github.com/milvus-io/milvus/internal/proto/etcdpb" pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3" "github.com/stretchr/testify/require"
) )
type mockTestKV struct { type mockTestKV struct {
@ -220,13 +222,14 @@ func TestMetaTable(t *testing.T) {
return vtso return vtso
} }
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) require.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7) skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, skv) assert.NotNil(t, skv)
txnKV := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath) txnKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
mt, err := NewMetaTable(txnKV, skv) mt, err := NewMetaTable(txnKV, skv)
assert.Nil(t, err) assert.Nil(t, err)
@ -293,7 +296,10 @@ func TestMetaTable(t *testing.T) {
}, },
} }
var wg sync.WaitGroup
wg.Add(1)
t.Run("add collection", func(t *testing.T) { t.Run("add collection", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
err = mt.AddCollection(collInfo, ts, nil, "") err = mt.AddCollection(collInfo, ts, nil, "")
assert.NotNil(t, err) assert.NotNil(t, err)
@ -321,7 +327,9 @@ func TestMetaTable(t *testing.T) {
assert.Equal(t, "false", flag) assert.Equal(t, "false", flag)
}) })
wg.Add(1)
t.Run("add alias", func(t *testing.T) { t.Run("add alias", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
exists := mt.IsAlias(aliasName1) exists := mt.IsAlias(aliasName1)
assert.False(t, exists) assert.False(t, exists)
@ -332,8 +340,9 @@ func TestMetaTable(t *testing.T) {
exists = mt.IsAlias(aliasName1) exists = mt.IsAlias(aliasName1)
assert.True(t, exists) assert.True(t, exists)
}) })
wg.Add(1)
t.Run("alter alias", func(t *testing.T) { t.Run("alter alias", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
err = mt.AlterAlias(aliasName1, collName, ts) err = mt.AlterAlias(aliasName1, collName, ts)
assert.Nil(t, err) assert.Nil(t, err)
@ -341,13 +350,17 @@ func TestMetaTable(t *testing.T) {
assert.NotNil(t, err) assert.NotNil(t, err)
}) })
wg.Add(1)
t.Run("delete alias", func(t *testing.T) { t.Run("delete alias", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
err = mt.DropAlias(aliasName1, ts) err = mt.DropAlias(aliasName1, ts)
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("add partition", func(t *testing.T) { t.Run("add partition", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
err = mt.AddPartition(collID, partName, partID, ts, "") err = mt.AddPartition(collID, partName, partID, ts, "")
assert.Nil(t, err) assert.Nil(t, err)
@ -365,7 +378,9 @@ func TestMetaTable(t *testing.T) {
assert.Equal(t, "false", flag) assert.Equal(t, "false", flag)
}) })
wg.Add(1)
t.Run("add segment index", func(t *testing.T) { t.Run("add segment index", func(t *testing.T) {
defer wg.Done()
segIdxInfo := pb.SegmentIndexInfo{ segIdxInfo := pb.SegmentIndexInfo{
CollectionID: collID, CollectionID: collID,
PartitionID: partID, PartitionID: partID,
@ -387,7 +402,9 @@ func TestMetaTable(t *testing.T) {
assert.EqualError(t, err, fmt.Sprintf("index id = %d exist", segIdxInfo.IndexID)) assert.EqualError(t, err, fmt.Sprintf("index id = %d exist", segIdxInfo.IndexID))
}) })
wg.Add(1)
t.Run("get not indexed segments", func(t *testing.T) { t.Run("get not indexed segments", func(t *testing.T) {
defer wg.Done()
params := []*commonpb.KeyValuePair{ params := []*commonpb.KeyValuePair{
{ {
Key: "field110-i1", Key: "field110-i1",
@ -442,7 +459,9 @@ func TestMetaTable(t *testing.T) {
}) })
wg.Add(1)
t.Run("get index by name", func(t *testing.T) { t.Run("get index by name", func(t *testing.T) {
defer wg.Done()
_, idx, err := mt.GetIndexByName(collName, "field110") _, idx, err := mt.GetIndexByName(collName, "field110")
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, 1, len(idx)) assert.Equal(t, 1, len(idx))
@ -464,7 +483,9 @@ func TestMetaTable(t *testing.T) {
assert.Zero(t, len(idx)) assert.Zero(t, len(idx))
}) })
wg.Add(1)
t.Run("reload meta", func(t *testing.T) { t.Run("reload meta", func(t *testing.T) {
defer wg.Done()
te := pb.TenantMeta{ te := pb.TenantMeta{
ID: 100, ID: 100,
} }
@ -480,7 +501,9 @@ func TestMetaTable(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("drop index", func(t *testing.T) { t.Run("drop index", func(t *testing.T) {
defer wg.Done()
idx, ok, err := mt.DropIndex(collName, "field110", "field110") idx, ok, err := mt.DropIndex(collName, "field110", "field110")
assert.Nil(t, err) assert.Nil(t, err)
assert.True(t, ok) assert.True(t, ok)
@ -503,7 +526,9 @@ func TestMetaTable(t *testing.T) {
assert.NotNil(t, err) assert.NotNil(t, err)
}) })
wg.Add(1)
t.Run("drop partition", func(t *testing.T) { t.Run("drop partition", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
id, err := mt.DeletePartition(collID, partName, ts, "") id, err := mt.DeletePartition(collID, partName, ts, "")
assert.Nil(t, err) assert.Nil(t, err)
@ -515,7 +540,9 @@ func TestMetaTable(t *testing.T) {
assert.Equal(t, "false", flag) assert.Equal(t, "false", flag)
}) })
wg.Add(1)
t.Run("drop collection", func(t *testing.T) { t.Run("drop collection", func(t *testing.T) {
defer wg.Done()
ts := ftso() ts := ftso()
err = mt.DeleteCollection(collIDInvalid, ts, "") err = mt.DeleteCollection(collIDInvalid, ts, "")
assert.NotNil(t, err) assert.NotNil(t, err)
@ -535,7 +562,7 @@ func TestMetaTable(t *testing.T) {
}) })
/////////////////////////// these tests should run at last, it only used to hit the error lines //////////////////////// /////////////////////////// these tests should run at last, it only used to hit the error lines ////////////////////////
txnkv := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath) txnkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
mockKV := &mockTestKV{} mockKV := &mockTestKV{}
mt.snapshot = mockKV mt.snapshot = mockKV
mockTxnKV := &mockTestTxnKV{ mockTxnKV := &mockTestTxnKV{
@ -549,7 +576,9 @@ func TestMetaTable(t *testing.T) {
} }
mt.txn = mockTxnKV mt.txn = mockTxnKV
wg.Add(1)
t.Run("add collection failed", func(t *testing.T) { t.Run("add collection failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -562,7 +591,9 @@ func TestMetaTable(t *testing.T) {
assert.Panics(t, func() { mt.AddCollection(collInfo, 0, idxInfo, "") }) assert.Panics(t, func() { mt.AddCollection(collInfo, 0, idxInfo, "") })
}) })
wg.Add(1)
t.Run("delete collection failed", func(t *testing.T) { t.Run("delete collection failed", func(t *testing.T) {
defer wg.Done()
mockKV.multiSave = func(kvs map[string]string, ts typeutil.Timestamp) error { mockKV.multiSave = func(kvs map[string]string, ts typeutil.Timestamp) error {
return nil return nil
} }
@ -573,7 +604,9 @@ func TestMetaTable(t *testing.T) {
assert.Panics(t, func() { mt.DeleteCollection(collInfo.ID, ts, "") }) assert.Panics(t, func() { mt.DeleteCollection(collInfo.ID, ts, "") })
}) })
wg.Add(1)
t.Run("get collection failed", func(t *testing.T) { t.Run("get collection failed", func(t *testing.T) {
defer wg.Done()
mockKV.save = func(key string, value string, ts typeutil.Timestamp) error { mockKV.save = func(key string, value string, ts typeutil.Timestamp) error {
return nil return nil
} }
@ -592,7 +625,9 @@ func TestMetaTable(t *testing.T) {
}) })
wg.Add(1)
t.Run("add partition failed", func(t *testing.T) { t.Run("add partition failed", func(t *testing.T) {
defer wg.Done()
mockKV.save = func(key string, value string, ts typeutil.Timestamp) error { mockKV.save = func(key string, value string, ts typeutil.Timestamp) error {
return nil return nil
} }
@ -653,7 +688,9 @@ func TestMetaTable(t *testing.T) {
assert.EqualError(t, err, fmt.Sprintf("partition id = %d already exists", partID)) assert.EqualError(t, err, fmt.Sprintf("partition id = %d already exists", partID))
}) })
wg.Add(1)
t.Run("has partition failed", func(t *testing.T) { t.Run("has partition failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -676,7 +713,9 @@ func TestMetaTable(t *testing.T) {
assert.False(t, mt.HasPartition(collInfo.ID, partName, 0)) assert.False(t, mt.HasPartition(collInfo.ID, partName, 0))
}) })
wg.Add(1)
t.Run("delete partition failed", func(t *testing.T) { t.Run("delete partition failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -714,7 +753,9 @@ func TestMetaTable(t *testing.T) {
assert.EqualError(t, err, fmt.Sprintf("can't find collection id = %d", collInfo.ID)) assert.EqualError(t, err, fmt.Sprintf("can't find collection id = %d", collInfo.ID))
}) })
wg.Add(1)
t.Run("add index failed", func(t *testing.T) { t.Run("add index failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -768,7 +809,9 @@ func TestMetaTable(t *testing.T) {
assert.Panics(t, func() { mt.AddIndex(&segIdxInfo) }) assert.Panics(t, func() { mt.AddIndex(&segIdxInfo) })
}) })
wg.Add(1)
t.Run("drop index failed", func(t *testing.T) { t.Run("drop index failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -833,7 +876,9 @@ func TestMetaTable(t *testing.T) {
assert.Panics(t, func() { mt.DropIndex(collInfo.Schema.Name, collInfo.Schema.Fields[0].Name, idxInfo[0].IndexName) }) assert.Panics(t, func() { mt.DropIndex(collInfo.Schema.Name, collInfo.Schema.Fields[0].Name, idxInfo[0].IndexName) })
}) })
wg.Add(1)
t.Run("get segment index info by id", func(t *testing.T) { t.Run("get segment index info by id", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -882,7 +927,9 @@ func TestMetaTable(t *testing.T) {
assert.EqualError(t, err, fmt.Sprintf("can't find index name = %s on segment = %d, with filed id = 11", idxInfo[0].IndexName, segIdxInfo.SegmentID)) assert.EqualError(t, err, fmt.Sprintf("can't find index name = %s on segment = %d, with filed id = 11", idxInfo[0].IndexName, segIdxInfo.SegmentID))
}) })
wg.Add(1)
t.Run("get field schema failed", func(t *testing.T) { t.Run("get field schema failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -913,7 +960,9 @@ func TestMetaTable(t *testing.T) {
assert.EqualError(t, err, fmt.Sprintf("collection %s not found", collInfo.Schema.Name)) assert.EqualError(t, err, fmt.Sprintf("collection %s not found", collInfo.Schema.Name))
}) })
wg.Add(1)
t.Run("is segment indexed", func(t *testing.T) { t.Run("is segment indexed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -935,7 +984,9 @@ func TestMetaTable(t *testing.T) {
assert.False(t, mt.IsSegmentIndexed(idx.SegmentID, &field, nil)) assert.False(t, mt.IsSegmentIndexed(idx.SegmentID, &field, nil))
}) })
wg.Add(1)
t.Run("get not indexed segments", func(t *testing.T) { t.Run("get not indexed segments", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -1028,7 +1079,9 @@ func TestMetaTable(t *testing.T) {
//assert.EqualError(t, err, "multi save error") //assert.EqualError(t, err, "multi save error")
}) })
wg.Add(1)
t.Run("get index by name failed", func(t *testing.T) { t.Run("get index by name failed", func(t *testing.T) {
defer wg.Done()
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
return nil, nil, nil return nil, nil, nil
} }
@ -1064,6 +1117,7 @@ func TestMetaTable(t *testing.T) {
assert.NotNil(t, err) assert.NotNil(t, err)
assert.EqualError(t, err, fmt.Sprintf("cannot find index, id = %d", idxInfo[0].IndexID)) assert.EqualError(t, err, fmt.Sprintf("cannot find index, id = %d", idxInfo[0].IndexID))
}) })
wg.Wait()
} }
func TestMetaWithTimestamp(t *testing.T) { func TestMetaWithTimestamp(t *testing.T) {
@ -1088,15 +1142,14 @@ func TestMetaWithTimestamp(t *testing.T) {
vtso++ vtso++
return vtso return vtso
} }
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()
skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7) skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, skv) assert.NotNil(t, skv)
txnKV := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath) txnKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
mt, err := NewMetaTable(txnKV, skv) mt, err := NewMetaTable(txnKV, skv)
assert.Nil(t, err) assert.Nil(t, err)
@ -1246,7 +1299,7 @@ func TestFixIssue10540(t *testing.T) {
Params.Init() Params.Init()
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close() defer etcdCli.Close()

View File

@ -22,9 +22,9 @@ import (
"testing" "testing"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
) )
func TestProxyClientManager_GetProxyClients(t *testing.T) { func TestProxyClientManager_GetProxyClients(t *testing.T) {
@ -32,7 +32,8 @@ func TestProxyClientManager_GetProxyClients(t *testing.T) {
core, err := NewCore(context.Background(), nil) core, err := NewCore(context.Background(), nil)
assert.Nil(t, err) assert.Nil(t, err)
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) cli, err := etcd.GetEtcdClient(&Params.BaseParams)
defer cli.Close()
assert.Nil(t, err) assert.Nil(t, err)
core.etcdCli = cli core.etcdCli = cli
@ -58,8 +59,9 @@ func TestProxyClientManager_AddProxyClient(t *testing.T) {
core, err := NewCore(context.Background(), nil) core, err := NewCore(context.Background(), nil)
assert.Nil(t, err) assert.Nil(t, err)
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) cli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer cli.Close()
core.etcdCli = cli core.etcdCli = cli
core.SetNewProxyClient( core.SetNewProxyClient(
@ -84,8 +86,9 @@ func TestProxyClientManager_InvalidateCollectionMetaCache(t *testing.T) {
core, err := NewCore(ctx, nil) core, err := NewCore(ctx, nil)
assert.Nil(t, err) assert.Nil(t, err)
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) cli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer cli.Close()
core.etcdCli = cli core.etcdCli = cli
pcm := newProxyClientManager(core) pcm := newProxyClientManager(core)
@ -114,8 +117,9 @@ func TestProxyClientManager_ReleaseDQLMessageStream(t *testing.T) {
core, err := NewCore(ctx, nil) core, err := NewCore(ctx, nil)
assert.Nil(t, err) assert.Nil(t, err)
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) cli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.Nil(t, err)
defer cli.Close()
core.etcdCli = cli core.etcdCli = cli
pcm := newProxyClientManager(core) pcm := newProxyClientManager(core)

View File

@ -46,20 +46,16 @@ type proxyManager struct {
// newProxyManager helper function to create a proxyManager // newProxyManager helper function to create a proxyManager
// etcdEndpoints is the address list of etcd // etcdEndpoints is the address list of etcd
// fns are the custom getSessions function list // fns are the custom getSessions function list
func newProxyManager(ctx context.Context, etcdEndpoints []string, fns ...func([]*sessionutil.Session)) (*proxyManager, error) { func newProxyManager(ctx context.Context, client *clientv3.Client, fns ...func([]*sessionutil.Session)) *proxyManager {
cli, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints})
if err != nil {
return nil, err
}
ctx2, cancel2 := context.WithCancel(ctx) ctx2, cancel2 := context.WithCancel(ctx)
p := &proxyManager{ p := &proxyManager{
ctx: ctx2, ctx: ctx2,
cancel: cancel2, cancel: cancel2,
lock: sync.Mutex{}, lock: sync.Mutex{},
etcdCli: cli, etcdCli: client,
} }
p.getSessions = append(p.getSessions, fns...) p.getSessions = append(p.getSessions, fns...)
return p, nil return p
} }
// AddSession adds functions to addSessions function list // AddSession adds functions to addSessions function list

View File

@ -23,6 +23,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -31,22 +32,23 @@ import (
func TestProxyManager(t *testing.T) { func TestProxyManager(t *testing.T) {
Params.Init() Params.Init()
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
assert.Nil(t, err)
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
defer etcdCli.Close()
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel() defer cancel()
sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
cli.Delete(ctx, sessKey, clientv3.WithPrefix()) etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
defer cli.Delete(ctx, sessKey, clientv3.WithPrefix()) defer etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
s1 := sessionutil.Session{ s1 := sessionutil.Session{
ServerID: 100, ServerID: 100,
} }
b1, err := json.Marshal(&s1) b1, err := json.Marshal(&s1)
assert.Nil(t, err) assert.Nil(t, err)
k1 := path.Join(sessKey, typeutil.ProxyRole+"-100") k1 := path.Join(sessKey, typeutil.ProxyRole+"-100")
_, err = cli.Put(ctx, k1, string(b1)) _, err = etcdCli.Put(ctx, k1, string(b1))
assert.Nil(t, err) assert.Nil(t, err)
s0 := sessionutil.Session{ s0 := sessionutil.Session{
@ -55,7 +57,7 @@ func TestProxyManager(t *testing.T) {
b0, err := json.Marshal(&s0) b0, err := json.Marshal(&s0)
assert.Nil(t, err) assert.Nil(t, err)
k0 := path.Join(sessKey, typeutil.ProxyRole+"-99") k0 := path.Join(sessKey, typeutil.ProxyRole+"-99")
_, err = cli.Put(ctx, k0, string(b0)) _, err = etcdCli.Put(ctx, k0, string(b0))
assert.Nil(t, err) assert.Nil(t, err)
f1 := func(sess []*sessionutil.Session) { f1 := func(sess []*sessionutil.Session) {
@ -64,8 +66,7 @@ func TestProxyManager(t *testing.T) {
assert.Equal(t, int64(99), sess[1].ServerID) assert.Equal(t, int64(99), sess[1].ServerID)
t.Log("get sessions", sess[0], sess[1]) t.Log("get sessions", sess[0], sess[1])
} }
pm := newProxyManager(ctx, etcdCli, f1)
pm, err := newProxyManager(ctx, Params.RootCoordCfg.EtcdEndpoints, f1)
assert.Nil(t, err) assert.Nil(t, err)
fa := func(sess *sessionutil.Session) { fa := func(sess *sessionutil.Session) {
assert.Equal(t, int64(101), sess.ServerID) assert.Equal(t, int64(101), sess.ServerID)
@ -88,10 +89,10 @@ func TestProxyManager(t *testing.T) {
b2, err := json.Marshal(&s2) b2, err := json.Marshal(&s2)
assert.Nil(t, err) assert.Nil(t, err)
k2 := path.Join(sessKey, typeutil.ProxyRole+"-101") k2 := path.Join(sessKey, typeutil.ProxyRole+"-101")
_, err = cli.Put(ctx, k2, string(b2)) _, err = etcdCli.Put(ctx, k2, string(b2))
assert.Nil(t, err) assert.Nil(t, err)
_, err = cli.Delete(ctx, k1) _, err = etcdCli.Delete(ctx, k1)
assert.Nil(t, err) assert.Nil(t, err)
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
pm.Stop() pm.Stop()

View File

@ -926,17 +926,23 @@ func (c *Core) Register() error {
log.Fatal("failed to stop server", zap.Error(err)) log.Fatal("failed to stop server", zap.Error(err))
} }
// manually send signal to starter goroutine // manually send signal to starter goroutine
syscall.Kill(syscall.Getpid(), syscall.SIGINT) if c.session.TriggerKill {
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
}
}) })
return nil return nil
} }
func (c *Core) SetEtcdClient(etcdClient *clientv3.Client) {
c.etcdCli = etcdClient
}
func (c *Core) initSession() error { func (c *Core) initSession() error {
c.session = sessionutil.NewSession(c.ctx, Params.RootCoordCfg.MetaRootPath, Params.RootCoordCfg.EtcdEndpoints) c.session = sessionutil.NewSession(c.ctx, Params.RootCoordCfg.MetaRootPath, c.etcdCli)
if c.session == nil { if c.session == nil {
return fmt.Errorf("session is nil, the etcd client connection may have failed") return fmt.Errorf("session is nil, the etcd client connection may have failed")
} }
c.session.Init(typeutil.RootCoordRole, Params.RootCoordCfg.Address, true) c.session.Init(typeutil.RootCoordRole, Params.RootCoordCfg.Address, true, true)
Params.BaseParams.SetLogger(c.session.ServerID) Params.BaseParams.SetLogger(c.session.ServerID)
return nil return nil
} }
@ -946,7 +952,7 @@ func (c *Core) Init() error {
var initError error var initError error
if c.kvBaseCreate == nil { if c.kvBaseCreate == nil {
c.kvBaseCreate = func(root string) (kv.TxnKV, error) { c.kvBaseCreate = func(root string) (kv.TxnKV, error) {
return etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, root) return etcdkv.NewEtcdKV(c.etcdCli, root), nil
} }
} }
c.initOnce.Do(func() { c.initOnce.Do(func() {
@ -956,10 +962,6 @@ func (c *Core) Init() error {
return return
} }
connectEtcdFn := func() error { connectEtcdFn := func() error {
if c.etcdCli, initError = clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints, DialTimeout: 5 * time.Second}); initError != nil {
log.Error("RootCoord failed to new Etcd client", zap.Any("reason", initError))
return initError
}
if c.kvBase, initError = c.kvBaseCreate(Params.RootCoordCfg.KvRootPath); initError != nil { if c.kvBase, initError = c.kvBaseCreate(Params.RootCoordCfg.KvRootPath); initError != nil {
log.Error("RootCoord failed to new EtcdKV", zap.Any("reason", initError)) log.Error("RootCoord failed to new EtcdKV", zap.Any("reason", initError))
return initError return initError
@ -989,10 +991,7 @@ func (c *Core) Init() error {
} }
log.Debug("RootCoord, Setting TSO and ID Allocator") log.Debug("RootCoord, Setting TSO and ID Allocator")
kv, initError := tsoutil.NewTSOKVBase(Params.RootCoordCfg.EtcdEndpoints, Params.RootCoordCfg.KvRootPath, "gid") kv := tsoutil.NewTSOKVBase(c.etcdCli, Params.RootCoordCfg.KvRootPath, "gid")
if initError != nil {
return
}
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", kv) idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", kv)
if initError = idAllocator.Initialize(); initError != nil { if initError = idAllocator.Initialize(); initError != nil {
return return
@ -1004,10 +1003,7 @@ func (c *Core) Init() error {
return idAllocator.UpdateID() return idAllocator.UpdateID()
} }
kv, initError = tsoutil.NewTSOKVBase(Params.RootCoordCfg.EtcdEndpoints, Params.RootCoordCfg.KvRootPath, "tso") kv = tsoutil.NewTSOKVBase(c.etcdCli, Params.RootCoordCfg.KvRootPath, "tso")
if initError != nil {
return
}
tsoAllocator := tso.NewGlobalTSOAllocator("timestamp", kv) tsoAllocator := tso.NewGlobalTSOAllocator("timestamp", kv)
if initError = tsoAllocator.Initialize(); initError != nil { if initError = tsoAllocator.Initialize(); initError != nil {
return return
@ -1033,15 +1029,12 @@ func (c *Core) Init() error {
c.proxyClientManager = newProxyClientManager(c) c.proxyClientManager = newProxyClientManager(c)
log.Debug("RootCoord, set proxy manager") log.Debug("RootCoord, set proxy manager")
c.proxyManager, initError = newProxyManager( c.proxyManager = newProxyManager(
c.ctx, c.ctx,
Params.RootCoordCfg.EtcdEndpoints, c.etcdCli,
c.chanTimeTick.getProxy, c.chanTimeTick.getProxy,
c.proxyClientManager.GetProxyClients, c.proxyClientManager.GetProxyClients,
) )
if initError != nil {
return
}
c.proxyManager.AddSession(c.chanTimeTick.addProxy, c.proxyClientManager.AddProxyClient) c.proxyManager.AddSession(c.chanTimeTick.addProxy, c.proxyClientManager.AddProxyClient)
c.proxyManager.DelSession(c.chanTimeTick.delProxy, c.proxyClientManager.DelProxyClient) c.proxyManager.DelSession(c.chanTimeTick.delProxy, c.proxyClientManager.DelProxyClient)

View File

@ -37,6 +37,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
@ -48,10 +49,10 @@ import (
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
) )
@ -433,9 +434,15 @@ func TestRootCoordInit(t *testing.T) {
coreFactory := msgstream.NewPmsFactory() coreFactory := msgstream.NewPmsFactory()
Params.Init() Params.Init()
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
defer etcdCli.Close()
core, err := NewCore(ctx, coreFactory) core, err := NewCore(ctx, coreFactory)
require.Nil(t, err) require.Nil(t, err)
assert.Nil(t, err) assert.Nil(t, err)
core.SetEtcdClient(etcdCli)
randVal := rand.Int() randVal := rand.Int()
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath) Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
@ -443,12 +450,13 @@ func TestRootCoordInit(t *testing.T) {
err = core.Init() err = core.Init()
assert.Nil(t, err) assert.Nil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
// inject kvBaseCreate fail // inject kvBaseCreate fail
core, err = NewCore(ctx, coreFactory) core, err = NewCore(ctx, coreFactory)
core.SetEtcdClient(etcdCli)
require.Nil(t, err) require.Nil(t, err)
assert.Nil(t, err) assert.Nil(t, err)
randVal = rand.Int() randVal = rand.Int()
@ -462,11 +470,13 @@ func TestRootCoordInit(t *testing.T) {
err = core.Init() err = core.Init()
assert.NotNil(t, err) assert.NotNil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
// inject metaKV create fail // inject metaKV create fail
core, err = NewCore(ctx, coreFactory) core, err = NewCore(ctx, coreFactory)
core.SetEtcdClient(etcdCli)
require.Nil(t, err) require.Nil(t, err)
assert.Nil(t, err) assert.Nil(t, err)
randVal = rand.Int() randVal = rand.Int()
@ -483,11 +493,13 @@ func TestRootCoordInit(t *testing.T) {
err = core.Init() err = core.Init()
assert.NotNil(t, err) assert.NotNil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
// inject newSuffixSnapshot failure // inject newSuffixSnapshot failure
core, err = NewCore(ctx, coreFactory) core, err = NewCore(ctx, coreFactory)
core.SetEtcdClient(etcdCli)
require.Nil(t, err) require.Nil(t, err)
assert.Nil(t, err) assert.Nil(t, err)
randVal = rand.Int() randVal = rand.Int()
@ -501,11 +513,13 @@ func TestRootCoordInit(t *testing.T) {
err = core.Init() err = core.Init()
assert.NotNil(t, err) assert.NotNil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
// inject newMetaTable failure // inject newMetaTable failure
core, err = NewCore(ctx, coreFactory) core, err = NewCore(ctx, coreFactory)
core.SetEtcdClient(etcdCli)
require.Nil(t, err) require.Nil(t, err)
assert.Nil(t, err) assert.Nil(t, err)
randVal = rand.Int() randVal = rand.Int()
@ -520,6 +534,7 @@ func TestRootCoordInit(t *testing.T) {
err = core.Init() err = core.Init()
assert.NotNil(t, err) assert.NotNil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
@ -544,7 +559,6 @@ func TestRootCoord(t *testing.T) {
core, err := NewCore(ctx, coreFactory) core, err := NewCore(ctx, coreFactory)
assert.Nil(t, err) assert.Nil(t, err)
randVal := rand.Int() randVal := rand.Int()
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal) Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal) Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal)
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath) Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
@ -553,8 +567,10 @@ func TestRootCoord(t *testing.T) {
Params.RootCoordCfg.DmlChannelName = fmt.Sprintf("rootcoord-dml-test-%d", randVal) Params.RootCoordCfg.DmlChannelName = fmt.Sprintf("rootcoord-dml-test-%d", randVal)
Params.RootCoordCfg.DeltaChannelName = fmt.Sprintf("rootcoord-delta-test-%d", randVal) Params.RootCoordCfg.DeltaChannelName = fmt.Sprintf("rootcoord-delta-test-%d", randVal)
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints, DialTimeout: 5 * time.Second}) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err) assert.NoError(t, err)
defer etcdCli.Close()
sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
@ -616,6 +632,8 @@ func TestRootCoord(t *testing.T) {
dmlStream, _ := tmpFactory.NewMsgStream(ctx) dmlStream, _ := tmpFactory.NewMsgStream(ctx)
clearMsgChan(1500*time.Millisecond, dmlStream.Chan()) clearMsgChan(1500*time.Millisecond, dmlStream.Chan())
core.SetEtcdClient(etcdCli)
err = core.Init() err = core.Init()
assert.Nil(t, err) assert.Nil(t, err)
@ -631,13 +649,18 @@ func TestRootCoord(t *testing.T) {
err = core.Start() err = core.Start()
assert.Nil(t, err) assert.Nil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
shardsNum := int32(8) shardsNum := int32(8)
fmt.Printf("hello world2")
var wg sync.WaitGroup
wg.Add(1)
t.Run("time tick", func(t *testing.T) { t.Run("time tick", func(t *testing.T) {
defer wg.Done()
ttmsg, ok := <-timeTickStream.Chan() ttmsg, ok := <-timeTickStream.Chan()
assert.True(t, ok) assert.True(t, ok)
assert.Equal(t, 1, len(ttmsg.Msgs)) assert.Equal(t, 1, len(ttmsg.Msgs))
@ -655,7 +678,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, ttm2.Base.Timestamp, ttm.Base.Timestamp+1) assert.Equal(t, ttm2.Base.Timestamp, ttm.Base.Timestamp+1)
}) })
wg.Add(1)
t.Run("create collection", func(t *testing.T) { t.Run("create collection", func(t *testing.T) {
defer wg.Done()
schema := schemapb.CollectionSchema{ schema := schemapb.CollectionSchema{
Name: collName, Name: collName,
AutoID: true, AutoID: true,
@ -789,7 +814,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("has collection", func(t *testing.T) { t.Run("has collection", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.HasCollectionRequest{ req := &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasCollection, MsgType: commonpb.MsgType_HasCollection,
@ -837,7 +864,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, true, rsp.Value) assert.Equal(t, true, rsp.Value)
}) })
wg.Add(1)
t.Run("describe collection", func(t *testing.T) { t.Run("describe collection", func(t *testing.T) {
defer wg.Done()
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0) collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
req := &milvuspb.DescribeCollectionRequest{ req := &milvuspb.DescribeCollectionRequest{
@ -860,7 +889,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, shardsNum, rsp.ShardsNum) assert.Equal(t, shardsNum, rsp.ShardsNum)
}) })
wg.Add(1)
t.Run("show collection", func(t *testing.T) { t.Run("show collection", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.ShowCollectionsRequest{ req := &milvuspb.ShowCollectionsRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowCollections, MsgType: commonpb.MsgType_ShowCollections,
@ -877,7 +908,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, len(rsp.CollectionNames), 2) assert.Equal(t, len(rsp.CollectionNames), 2)
}) })
wg.Add(1)
t.Run("create partition", func(t *testing.T) { t.Run("create partition", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.CreatePartitionRequest{ req := &milvuspb.CreatePartitionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_CreatePartition, MsgType: commonpb.MsgType_CreatePartition,
@ -932,7 +965,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("has partition", func(t *testing.T) { t.Run("has partition", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.HasPartitionRequest{ req := &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasPartition, MsgType: commonpb.MsgType_HasPartition,
@ -950,7 +985,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, true, rsp.Value) assert.Equal(t, true, rsp.Value)
}) })
wg.Add(1)
t.Run("show partition", func(t *testing.T) { t.Run("show partition", func(t *testing.T) {
defer wg.Done()
coll, err := core.MetaTable.GetCollectionByName(collName, 0) coll, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
req := &milvuspb.ShowPartitionsRequest{ req := &milvuspb.ShowPartitionsRequest{
@ -971,7 +1008,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, 2, len(rsp.PartitionIDs)) assert.Equal(t, 2, len(rsp.PartitionIDs))
}) })
wg.Add(1)
t.Run("show segment", func(t *testing.T) { t.Run("show segment", func(t *testing.T) {
defer wg.Done()
coll, err := core.MetaTable.GetCollectionByName(collName, 0) coll, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
partID := coll.PartitionIDs[1] partID := coll.PartitionIDs[1]
@ -996,7 +1035,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, 1, len(rsp.SegmentIDs)) assert.Equal(t, 1, len(rsp.SegmentIDs))
}) })
wg.Add(1)
t.Run("create index", func(t *testing.T) { t.Run("create index", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.CreateIndexRequest{ req := &milvuspb.CreateIndexRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_CreateIndex, MsgType: commonpb.MsgType_CreateIndex,
@ -1038,7 +1079,9 @@ func TestRootCoord(t *testing.T) {
assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode) assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
}) })
wg.Add(1)
t.Run("describe segment", func(t *testing.T) { t.Run("describe segment", func(t *testing.T) {
defer wg.Done()
coll, err := core.MetaTable.GetCollectionByName(collName, 0) coll, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
@ -1058,7 +1101,9 @@ func TestRootCoord(t *testing.T) {
t.Logf("index id = %d", rsp.IndexID) t.Logf("index id = %d", rsp.IndexID)
}) })
wg.Add(1)
t.Run("describe index", func(t *testing.T) { t.Run("describe index", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DescribeIndexRequest{ req := &milvuspb.DescribeIndexRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeIndex, MsgType: commonpb.MsgType_DescribeIndex,
@ -1079,7 +1124,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, "vector", rsp.IndexDescriptions[0].FieldName) assert.Equal(t, "vector", rsp.IndexDescriptions[0].FieldName)
}) })
wg.Add(1)
t.Run("describe index not exist", func(t *testing.T) { t.Run("describe index not exist", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DescribeIndexRequest{ req := &milvuspb.DescribeIndexRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeIndex, MsgType: commonpb.MsgType_DescribeIndex,
@ -1098,7 +1145,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, 0, len(rsp.IndexDescriptions)) assert.Equal(t, 0, len(rsp.IndexDescriptions))
}) })
wg.Add(1)
t.Run("flush segment", func(t *testing.T) { t.Run("flush segment", func(t *testing.T) {
defer wg.Done()
coll, err := core.MetaTable.GetCollectionByName(collName, 0) coll, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
partID := coll.PartitionIDs[1] partID := coll.PartitionIDs[1]
@ -1136,7 +1185,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName) assert.Equal(t, Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
}) })
wg.Add(1)
t.Run("over ride index", func(t *testing.T) { t.Run("over ride index", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.CreateIndexRequest{ req := &milvuspb.CreateIndexRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_CreateIndex, MsgType: commonpb.MsgType_CreateIndex,
@ -1180,7 +1231,9 @@ func TestRootCoord(t *testing.T) {
}) })
wg.Add(1)
t.Run("drop index", func(t *testing.T) { t.Run("drop index", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DropIndexRequest{ req := &milvuspb.DropIndexRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropIndex, MsgType: commonpb.MsgType_DropIndex,
@ -1211,7 +1264,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, 0, len(idx)) assert.Equal(t, 0, len(idx))
}) })
wg.Add(1)
t.Run("drop partition", func(t *testing.T) { t.Run("drop partition", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DropPartitionRequest{ req := &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropPartition, MsgType: commonpb.MsgType_DropPartition,
@ -1267,7 +1322,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("remove DQL msgstream", func(t *testing.T) { t.Run("remove DQL msgstream", func(t *testing.T) {
defer wg.Done()
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0) collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
@ -1283,7 +1340,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
}) })
wg.Add(1)
t.Run("drop collection", func(t *testing.T) { t.Run("drop collection", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DropCollectionRequest{ req := &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropCollection, MsgType: commonpb.MsgType_DropCollection,
@ -1356,7 +1415,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("context_cancel", func(t *testing.T) { t.Run("context_cancel", func(t *testing.T) {
defer wg.Done()
ctx2, cancel2 := context.WithTimeout(ctx, time.Millisecond*100) ctx2, cancel2 := context.WithTimeout(ctx, time.Millisecond*100)
defer cancel2() defer cancel2()
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
@ -1516,7 +1577,9 @@ func TestRootCoord(t *testing.T) {
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
}) })
wg.Add(1)
t.Run("undefined req type", func(t *testing.T) { t.Run("undefined req type", func(t *testing.T) {
defer wg.Done()
st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{ st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined, MsgType: commonpb.MsgType_Undefined,
@ -1673,7 +1736,9 @@ func TestRootCoord(t *testing.T) {
}) })
wg.Add(1)
t.Run("alloc time tick", func(t *testing.T) { t.Run("alloc time tick", func(t *testing.T) {
defer wg.Done()
req := &rootcoordpb.AllocTimestampRequest{ req := &rootcoordpb.AllocTimestampRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined, MsgType: commonpb.MsgType_Undefined,
@ -1689,7 +1754,9 @@ func TestRootCoord(t *testing.T) {
assert.NotZero(t, rsp.Timestamp) assert.NotZero(t, rsp.Timestamp)
}) })
wg.Add(1)
t.Run("alloc id", func(t *testing.T) { t.Run("alloc id", func(t *testing.T) {
defer wg.Done()
req := &rootcoordpb.AllocIDRequest{ req := &rootcoordpb.AllocIDRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined, MsgType: commonpb.MsgType_Undefined,
@ -1705,14 +1772,18 @@ func TestRootCoord(t *testing.T) {
assert.NotZero(t, rsp.ID) assert.NotZero(t, rsp.ID)
}) })
wg.Add(1)
t.Run("get_channels", func(t *testing.T) { t.Run("get_channels", func(t *testing.T) {
defer wg.Done()
_, err := core.GetTimeTickChannel(ctx) _, err := core.GetTimeTickChannel(ctx)
assert.Nil(t, err) assert.Nil(t, err)
_, err = core.GetStatisticsChannel(ctx) _, err = core.GetStatisticsChannel(ctx)
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("channel timetick", func(t *testing.T) { t.Run("channel timetick", func(t *testing.T) {
defer wg.Done()
const ( const (
proxyIDInvalid = 102 proxyIDInvalid = 102
proxyName0 = "proxy_0" proxyName0 = "proxy_0"
@ -1825,7 +1896,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
wg.Add(1)
t.Run("create alias", func(t *testing.T) { t.Run("create alias", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.CreateAliasRequest{ req := &milvuspb.CreateAliasRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_CreateAlias, MsgType: commonpb.MsgType_CreateAlias,
@ -1841,7 +1914,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
}) })
wg.Add(1)
t.Run("describe collection2", func(t *testing.T) { t.Run("describe collection2", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DescribeCollectionRequest{ req := &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection, MsgType: commonpb.MsgType_DescribeCollection,
@ -1879,7 +1954,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
wg.Add(1)
t.Run("alter alias", func(t *testing.T) { t.Run("alter alias", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.AlterAliasRequest{ req := &milvuspb.AlterAliasRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_AlterAlias, MsgType: commonpb.MsgType_AlterAlias,
@ -1895,7 +1972,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
}) })
wg.Add(1)
t.Run("drop collection with alias", func(t *testing.T) { t.Run("drop collection with alias", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DropCollectionRequest{ req := &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropAlias, MsgType: commonpb.MsgType_DropAlias,
@ -1910,7 +1989,9 @@ func TestRootCoord(t *testing.T) {
assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode) assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
}) })
wg.Add(1)
t.Run("drop alias", func(t *testing.T) { t.Run("drop alias", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.DropAliasRequest{ req := &milvuspb.DropAliasRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropAlias, MsgType: commonpb.MsgType_DropAlias,
@ -1951,7 +2032,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
wg.Add(1)
t.Run("get metrics", func(t *testing.T) { t.Run("get metrics", func(t *testing.T) {
defer wg.Done()
// not healthy // not healthy
stateSave := core.stateCode.Load().(internalpb.StateCode) stateSave := core.stateCode.Load().(internalpb.StateCode)
core.UpdateStateCode(internalpb.StateCode_Abnormal) core.UpdateStateCode(internalpb.StateCode_Abnormal)
@ -1985,7 +2068,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
}) })
wg.Add(1)
t.Run("get system info", func(t *testing.T) { t.Run("get system info", func(t *testing.T) {
defer wg.Done()
// normal case // normal case
systemInfoMetricType := metricsinfo.SystemInfoMetrics systemInfoMetricType := metricsinfo.SystemInfoMetrics
req, err := metricsinfo.ConstructRequestByMetricType(systemInfoMetricType) req, err := metricsinfo.ConstructRequestByMetricType(systemInfoMetricType)
@ -2002,7 +2087,9 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, st.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, st.Status.ErrorCode)
assert.NotEqual(t, internalpb.StateCode_Healthy, st.State.StateCode) assert.NotEqual(t, internalpb.StateCode_Healthy, st.State.StateCode)
wg.Add(1)
t.Run("state_not_healthy", func(t *testing.T) { t.Run("state_not_healthy", func(t *testing.T) {
defer wg.Done()
st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{ st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_CreateCollection, MsgType: commonpb.MsgType_CreateCollection,
@ -2159,7 +2246,9 @@ func TestRootCoord(t *testing.T) {
}) })
wg.Add(1)
t.Run("alloc_error", func(t *testing.T) { t.Run("alloc_error", func(t *testing.T) {
defer wg.Done()
core.Stop() core.Stop()
core.IDAllocator = func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) { core.IDAllocator = func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) {
return 0, 0, fmt.Errorf("id allocator error test") return 0, 0, fmt.Errorf("id allocator error test")
@ -2195,9 +2284,9 @@ func TestRootCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.NotEqual(t, commonpb.ErrorCode_Success, p2.Status.ErrorCode) assert.NotEqual(t, commonpb.ErrorCode_Success, p2.Status.ErrorCode)
}) })
wg.Wait()
err = core.Stop() err = core.Stop()
assert.Nil(t, err) assert.Nil(t, err)
} }
func TestRootCoord2(t *testing.T) { func TestRootCoord2(t *testing.T) {
@ -2215,6 +2304,11 @@ func TestRootCoord2(t *testing.T) {
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
core, err := NewCore(ctx, msFactory) core, err := NewCore(ctx, msFactory)
assert.Nil(t, err) assert.Nil(t, err)
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
defer etcdCli.Close()
randVal := rand.Int() randVal := rand.Int()
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal) Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
@ -2248,12 +2342,14 @@ func TestRootCoord2(t *testing.T) {
return nil, nil return nil, nil
} }
core.SetEtcdClient(etcdCli)
err = core.Init() err = core.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = core.Start() err = core.Start()
assert.Nil(t, err) assert.Nil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
@ -2270,7 +2366,10 @@ func TestRootCoord2(t *testing.T) {
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
var wg sync.WaitGroup
wg.Add(1)
t.Run("time tick", func(t *testing.T) { t.Run("time tick", func(t *testing.T) {
defer wg.Done()
ttmsg, ok := <-timeTickStream.Chan() ttmsg, ok := <-timeTickStream.Chan()
assert.True(t, ok) assert.True(t, ok)
assert.Equal(t, 1, len(ttmsg.Msgs)) assert.Equal(t, 1, len(ttmsg.Msgs))
@ -2279,7 +2378,9 @@ func TestRootCoord2(t *testing.T) {
assert.Greater(t, ttm.Base.Timestamp, typeutil.Timestamp(0)) assert.Greater(t, ttm.Base.Timestamp, typeutil.Timestamp(0))
}) })
wg.Add(1)
t.Run("create collection", func(t *testing.T) { t.Run("create collection", func(t *testing.T) {
defer wg.Done()
schema := schemapb.CollectionSchema{ schema := schemapb.CollectionSchema{
Name: collName, Name: collName,
} }
@ -2314,7 +2415,9 @@ func TestRootCoord2(t *testing.T) {
t.Log("time tick", m1.Base.Timestamp) t.Log("time tick", m1.Base.Timestamp)
}) })
wg.Add(1)
t.Run("describe collection", func(t *testing.T) { t.Run("describe collection", func(t *testing.T) {
defer wg.Done()
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0) collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
req := &milvuspb.DescribeCollectionRequest{ req := &milvuspb.DescribeCollectionRequest{
@ -2336,6 +2439,7 @@ func TestRootCoord2(t *testing.T) {
assert.Equal(t, common.DefaultShardsNum, int32(len(rsp.PhysicalChannelNames))) assert.Equal(t, common.DefaultShardsNum, int32(len(rsp.PhysicalChannelNames)))
assert.Equal(t, common.DefaultShardsNum, rsp.ShardsNum) assert.Equal(t, common.DefaultShardsNum, rsp.ShardsNum)
}) })
wg.Wait()
err = core.Stop() err = core.Stop()
assert.Nil(t, err) assert.Nil(t, err)
} }
@ -2516,12 +2620,17 @@ func TestCheckFlushedSegments(t *testing.T) {
return nil, nil return nil, nil
} }
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.Nil(t, err)
defer etcdCli.Close()
core.SetEtcdClient(etcdCli)
err = core.Init() err = core.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = core.Start() err = core.Start()
assert.Nil(t, err) assert.Nil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)
@ -2537,7 +2646,11 @@ func TestCheckFlushedSegments(t *testing.T) {
timeTickStream.Start() timeTickStream.Start()
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
var wg sync.WaitGroup
wg.Add(1)
t.Run("check flushed segments", func(t *testing.T) { t.Run("check flushed segments", func(t *testing.T) {
defer wg.Done()
ctx := context.Background() ctx := context.Background()
var collID int64 = 1 var collID int64 = 1
var partID int64 = 2 var partID int64 = 2
@ -2620,6 +2733,7 @@ func TestCheckFlushedSegments(t *testing.T) {
core.checkFlushedSegments(core.ctx) core.checkFlushedSegments(core.ctx)
}) })
wg.Wait()
err = core.Stop() err = core.Stop()
assert.Nil(t, err) assert.Nil(t, err)
} }
@ -2641,7 +2755,6 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) {
core, err := NewCore(ctx, msFactory) core, err := NewCore(ctx, msFactory)
assert.Nil(t, err) assert.Nil(t, err)
randVal := rand.Int() randVal := rand.Int()
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal) Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal) Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal)
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath) Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
@ -2673,12 +2786,18 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) {
return nil, nil return nil, nil
} }
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
defer etcdCli.Close()
core.SetEtcdClient(etcdCli)
err = core.Init() err = core.Init()
assert.Nil(t, err) assert.Nil(t, err)
err = core.Start() err = core.Start()
assert.Nil(t, err) assert.Nil(t, err)
core.session.TriggerKill = false
err = core.Register() err = core.Register()
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -23,6 +23,7 @@ import (
"time" "time"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -262,8 +263,10 @@ func Test_SuffixSnapshotLoad(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
sep := "_ts" sep := "_ts"
etcdkv, err := etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, rootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
require.Nil(t, err) require.Nil(t, err)
defer etcdCli.Close()
etcdkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
defer etcdkv.Close() defer etcdkv.Close()
var vtso typeutil.Timestamp var vtso typeutil.Timestamp
@ -312,8 +315,10 @@ func Test_SuffixSnapshotMultiSave(t *testing.T) {
Params.Init() Params.Init()
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
sep := "_ts" sep := "_ts"
etcdkv, err := etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, rootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
require.Nil(t, err) require.Nil(t, err)
defer etcdCli.Close()
etcdkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
defer etcdkv.Close() defer etcdkv.Close()
var vtso typeutil.Timestamp var vtso typeutil.Timestamp
@ -386,7 +391,10 @@ func Test_SuffixSnapshotMultiSaveAndRemoveWithPrefix(t *testing.T) {
rootPath := fmt.Sprintf("/test/meta/%d", randVal) rootPath := fmt.Sprintf("/test/meta/%d", randVal)
sep := "_ts" sep := "_ts"
etcdkv, err := etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, rootPath) etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
require.Nil(t, err)
defer etcdCli.Close()
etcdkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
require.Nil(t, err) require.Nil(t, err)
defer etcdkv.Close() defer etcdkv.Close()

View File

@ -18,6 +18,7 @@ package rootcoord
import ( import (
"context" "context"
"sync"
"testing" "testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -52,7 +53,10 @@ func TestTimetickSync(t *testing.T) {
Params.RootCoordCfg.DeltaChannelName = "rootcoord-delta" Params.RootCoordCfg.DeltaChannelName = "rootcoord-delta"
ttSync := newTimeTickSync(ctx, session, factory, nil) ttSync := newTimeTickSync(ctx, session, factory, nil)
var wg sync.WaitGroup
wg.Add(1)
t.Run("sendToChannel", func(t *testing.T) { t.Run("sendToChannel", func(t *testing.T) {
defer wg.Done()
ttSync.sendToChannel() ttSync.sendToChannel()
ttSync.proxyTimeTick[1] = nil ttSync.proxyTimeTick[1] = nil
@ -67,14 +71,18 @@ func TestTimetickSync(t *testing.T) {
ttSync.sendToChannel() ttSync.sendToChannel()
}) })
wg.Add(1)
t.Run("RemoveDdlTimeTick", func(t *testing.T) { t.Run("RemoveDdlTimeTick", func(t *testing.T) {
defer wg.Done()
ttSync.addDdlTimeTick(uint64(1), "1") ttSync.addDdlTimeTick(uint64(1), "1")
ttSync.addDdlTimeTick(uint64(2), "2") ttSync.addDdlTimeTick(uint64(2), "2")
ttSync.removeDdlTimeTick(uint64(1), "1") ttSync.removeDdlTimeTick(uint64(1), "1")
assert.Equal(t, ttSync.ddlMinTs, uint64(2)) assert.Equal(t, ttSync.ddlMinTs, uint64(2))
}) })
wg.Add(1)
t.Run("UpdateTimeTick", func(t *testing.T) { t.Run("UpdateTimeTick", func(t *testing.T) {
defer wg.Done()
msg := &internalpb.ChannelTimeTickMsg{ msg := &internalpb.ChannelTimeTickMsg{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick, MsgType: commonpb.MsgType_TimeTick,
@ -105,7 +113,9 @@ func TestTimetickSync(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
}) })
wg.Add(1)
t.Run("minTimeTick", func(t *testing.T) { t.Run("minTimeTick", func(t *testing.T) {
defer wg.Done()
tts := make([]uint64, 2) tts := make([]uint64, 2)
tts[0] = uint64(5) tts[0] = uint64(5)
tts[1] = uint64(3) tts[1] = uint64(3)
@ -113,4 +123,5 @@ func TestTimetickSync(t *testing.T) {
ret := minTimeTick(tts...) ret := minTimeTick(tts...)
assert.Equal(t, ret, tts[1]) assert.Equal(t, ret, tts[1])
}) })
wg.Wait()
} }

View File

@ -18,6 +18,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -30,8 +31,11 @@ func TestGlobalTSOAllocator_Initialize(t *testing.T) {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err) assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
err = gTestTsoAllocator.Initialize() err = gTestTsoAllocator.Initialize()
assert.Nil(t, err) assert.Nil(t, err)
@ -67,8 +71,11 @@ func TestGlobalTSOAllocator_All(t *testing.T) {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err) assert.NoError(t, err)
defer etcdCli.Close()
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
t.Run("Initialize", func(t *testing.T) { t.Run("Initialize", func(t *testing.T) {
err := gTestTsoAllocator.Initialize() err := gTestTsoAllocator.Initialize()
@ -153,7 +160,10 @@ func TestGlobalTSOAllocator_Fail(t *testing.T) {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err)
defer etcdCli.Close()
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
assert.NoError(t, err) assert.NoError(t, err)
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
t.Run("Initialize", func(t *testing.T) { t.Run("Initialize", func(t *testing.T) {
@ -194,7 +204,10 @@ func TestGlobalTSOAllocator_Update(t *testing.T) {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err)
defer etcdCli.Close()
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
assert.NoError(t, err) assert.NoError(t, err)
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
err = gTestTsoAllocator.Initialize() err = gTestTsoAllocator.Initialize()
@ -216,7 +229,10 @@ func TestGlobalTSOAllocator_load(t *testing.T) {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err)
defer etcdCli.Close()
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
assert.NoError(t, err) assert.NoError(t, err)
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
err = gTestTsoAllocator.Initialize() err = gTestTsoAllocator.Initialize()

View File

@ -23,6 +23,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
) )
// TimeTickProvider is the interface all services implement // TimeTickProvider is the interface all services implement
@ -74,6 +75,9 @@ type DataNodeComponent interface {
// GetStateCode return state code of this data node // GetStateCode return state code of this data node
GetStateCode() internalpb.StateCode GetStateCode() internalpb.StateCode
// SetEtcdClient set etcd client for DataNode
SetEtcdClient(etcdClient *clientv3.Client)
// SetRootCoord set RootCoord for DataNode // SetRootCoord set RootCoord for DataNode
// `rootCoord` is a client of root coordinator. // `rootCoord` is a client of root coordinator.
// //
@ -248,6 +252,15 @@ type DataCoord interface {
DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error)
} }
// DataCoordComponent defines the interface of DataCoord component.
type DataCoordComponent interface {
DataCoord
// SetEtcdClient set EtcdClient for DataCoord
// `etcdClient` is a client of etcd
SetEtcdClient(etcdClient *clientv3.Client)
}
// IndexNode is the interface `indexnode` package implements // IndexNode is the interface `indexnode` package implements
type IndexNode interface { type IndexNode interface {
Component Component
@ -261,6 +274,18 @@ type IndexNode interface {
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
} }
// IndexNodeComponent is used by grpc server of IndexNode
type IndexNodeComponent interface {
IndexNode
// SetEtcdClient set etcd client for QueryCoord
SetEtcdClient(etcdClient *clientv3.Client)
// UpdateStateCode updates state code for QueryCoord
// `stateCode` is current statement of this query coord, indicating whether it's healthy.
UpdateStateCode(stateCode internalpb.StateCode)
}
// IndexCoord is the interface `indexcoord` package implements // IndexCoord is the interface `indexcoord` package implements
type IndexCoord interface { type IndexCoord interface {
Component Component
@ -287,6 +312,18 @@ type IndexCoord interface {
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
} }
// IndexCoordComponent is used by grpc server of IndexCoord
type IndexCoordComponent interface {
IndexCoord
// SetEtcdClient set etcd client for QueryCoord
SetEtcdClient(etcdClient *clientv3.Client)
// UpdateStateCode updates state code for QueryCoord
// `stateCode` is current statement of this query coord, indicating whether it's healthy.
UpdateStateCode(stateCode internalpb.StateCode)
}
// RootCoord is the interface `rootcoord` package implements // RootCoord is the interface `rootcoord` package implements
type RootCoord interface { type RootCoord interface {
Component Component
@ -540,6 +577,10 @@ type RootCoord interface {
type RootCoordComponent interface { type RootCoordComponent interface {
RootCoord RootCoord
// SetEtcdClient set EtcdClient for RootCoord
// `etcdClient` is a client of etcd
SetEtcdClient(etcdClient *clientv3.Client)
// UpdateStateCode updates state code for RootCoord // UpdateStateCode updates state code for RootCoord
// State includes: Initializing, Healthy and Abnormal // State includes: Initializing, Healthy and Abnormal
UpdateStateCode(internalpb.StateCode) UpdateStateCode(internalpb.StateCode)
@ -606,6 +647,10 @@ type Proxy interface {
type ProxyComponent interface { type ProxyComponent interface {
Proxy Proxy
// SetEtcdClient set EtcdClient for Proxy
// `etcdClient` is a client of etcd
SetEtcdClient(etcdClient *clientv3.Client)
// SetRootCoord set RootCoord for Proxy // SetRootCoord set RootCoord for Proxy
// `rootCoord` is a client of root coordinator. // `rootCoord` is a client of root coordinator.
SetRootCoordClient(rootCoord RootCoord) SetRootCoordClient(rootCoord RootCoord)
@ -1021,6 +1066,9 @@ type QueryNodeComponent interface {
// `stateCode` is current statement of this query node, indicating whether it's healthy. // `stateCode` is current statement of this query node, indicating whether it's healthy.
UpdateStateCode(stateCode internalpb.StateCode) UpdateStateCode(stateCode internalpb.StateCode)
// SetEtcdClient set etcd client for QueryNode
SetEtcdClient(etcdClient *clientv3.Client)
// SetRootCoord set RootCoord for QueryNode // SetRootCoord set RootCoord for QueryNode
// `rootCoord` is a client of root coordinator. Pass to segmentLoader. // `rootCoord` is a client of root coordinator. Pass to segmentLoader.
// //
@ -1063,6 +1111,9 @@ type QueryCoord interface {
type QueryCoordComponent interface { type QueryCoordComponent interface {
QueryCoord QueryCoord
// SetEtcdClient set etcd client for QueryCoord
SetEtcdClient(etcdClient *clientv3.Client)
// UpdateStateCode updates state code for QueryCoord // UpdateStateCode updates state code for QueryCoord
// `stateCode` is current statement of this query coord, indicating whether it's healthy. // `stateCode` is current statement of this query coord, indicating whether it's healthy.
UpdateStateCode(stateCode internalpb.StateCode) UpdateStateCode(stateCode internalpb.StateCode)

View File

@ -0,0 +1,80 @@
// 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
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// 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.
package etcd
import (
"fmt"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/util/paramtable"
clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/server/v3/embed"
"go.etcd.io/etcd/server/v3/etcdserver/api/v3client"
)
var EtcdServer *embed.Etcd
func InitEtcdServer(pt *paramtable.BaseParamTable) error {
if pt.UseEmbedEtcd {
path := pt.EtcdConfigPath
fmt.Println("path", path, "data", pt.EtcdDataDir)
var cfg *embed.Config
if len(path) > 0 {
cfgFromFile, err := embed.ConfigFromFile(path)
if err != nil {
return err
}
cfg = cfgFromFile
} else {
cfg = embed.NewConfig()
}
cfg.Dir = pt.EtcdDataDir
e, err := embed.StartEtcd(cfg)
if err != nil {
return err
}
EtcdServer = e
log.Info("finish init embedded etcd")
}
return nil
}
func StopEtcdServer() {
if EtcdServer != nil {
EtcdServer.Close()
}
}
func GetEtcdClient(pt *paramtable.BaseParamTable) (*clientv3.Client, error) {
if pt.UseEmbedEtcd {
return GetEmbedEtcdClient()
}
return GetRemoteEtcdClient(pt.EtcdEndpoints)
}
func GetEmbedEtcdClient() (*clientv3.Client, error) {
client := v3client.New(EtcdServer.Server)
return client, nil
}
func GetRemoteEtcdClient(endpoints []string) (*clientv3.Client, error) {
return clientv3.New(clientv3.Config{
Endpoints: endpoints,
DialTimeout: 5 * time.Second,
})
}

View File

@ -0,0 +1,55 @@
// 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
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// 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.
package etcd
import (
"context"
"os"
"path"
"testing"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
)
var Params paramtable.GlobalParamTable
func TestEtcd(t *testing.T) {
Params.Init()
Params.BaseParams.UseEmbedEtcd = true
Params.BaseParams.EtcdDataDir = "/tmp/data"
err := InitEtcdServer(&Params.BaseParams)
assert.NoError(t, err)
defer os.RemoveAll(Params.BaseParams.EtcdDataDir)
defer StopEtcdServer()
// port is binded
err = InitEtcdServer(&Params.BaseParams)
assert.Error(t, err)
etcdCli, err := GetEtcdClient(&Params.BaseParams)
assert.NoError(t, err)
key := path.Join("test", "test")
_, err = etcdCli.Put(context.TODO(), key, "value")
assert.NoError(t, err)
resp, err := etcdCli.Get(context.TODO(), key)
assert.NoError(t, err)
assert.False(t, resp.Count < 1)
assert.Equal(t, string(resp.Kvs[0].Value), "value")
}

View File

@ -112,7 +112,7 @@ func (c *ClientBase) resetConnection(client interface{}) {
func (c *ClientBase) connect(ctx context.Context) error { func (c *ClientBase) connect(ctx context.Context) error {
addr, err := c.getAddrFunc() addr, err := c.getAddrFunc()
if err != nil { if err != nil {
log.Error("failed to get cclient address", zap.Error(err)) log.Error("failed to get client address", zap.Error(err))
return err return err
} }

View File

@ -289,7 +289,6 @@ type rootCoordConfig struct {
Port int Port int
PulsarAddress string PulsarAddress string
EtcdEndpoints []string
MetaRootPath string MetaRootPath string
KvRootPath string KvRootPath string
@ -316,7 +315,6 @@ func (p *rootCoordConfig) init(bp *BaseParamTable) {
p.BaseParams = bp p.BaseParams = bp
p.initPulsarAddress() p.initPulsarAddress()
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initKvRootPath() p.initKvRootPath()
@ -345,14 +343,6 @@ func (p *rootCoordConfig) initPulsarAddress() {
p.PulsarAddress = addr p.PulsarAddress = addr
} }
func (p *rootCoordConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *rootCoordConfig) initMetaRootPath() { func (p *rootCoordConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {
@ -469,7 +459,6 @@ type proxyConfig struct {
Alias string Alias string
EtcdEndpoints []string
MetaRootPath string MetaRootPath string
PulsarAddress string PulsarAddress string
@ -507,7 +496,6 @@ type proxyConfig struct {
func (p *proxyConfig) init(bp *BaseParamTable) { func (p *proxyConfig) init(bp *BaseParamTable) {
p.BaseParams = bp p.BaseParams = bp
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initPulsarAddress() p.initPulsarAddress()
p.initRocksmqPath() p.initRocksmqPath()
@ -654,14 +642,6 @@ func (p *proxyConfig) initPulsarMaxMessageSize() {
} }
} }
func (p *proxyConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *proxyConfig) initMetaRootPath() { func (p *proxyConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {
@ -711,9 +691,8 @@ type queryCoordConfig struct {
SearchResultChannelPrefix string SearchResultChannelPrefix string
// --- etcd --- // --- etcd ---
EtcdEndpoints []string MetaRootPath string
MetaRootPath string KvRootPath string
KvRootPath string
//--- Minio --- //--- Minio ---
MinioEndPoint string MinioEndPoint string
@ -752,7 +731,6 @@ func (p *queryCoordConfig) init(bp *BaseParamTable) {
p.initTimeTickChannelName() p.initTimeTickChannelName()
// --- etcd --- // --- etcd ---
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initKvRootPath() p.initKvRootPath()
@ -824,14 +802,6 @@ func (p *queryCoordConfig) initTimeTickChannelName() {
p.TimeTickChannelName = strings.Join(s, "-") p.TimeTickChannelName = strings.Join(s, "-")
} }
func (p *queryCoordConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *queryCoordConfig) initMetaRootPath() { func (p *queryCoordConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {
@ -980,7 +950,6 @@ type queryNodeConfig struct {
PulsarAddress string PulsarAddress string
RocksmqPath string RocksmqPath string
EtcdEndpoints []string
MetaRootPath string MetaRootPath string
Alias string Alias string
@ -1053,7 +1022,6 @@ func (p *queryNodeConfig) init(bp *BaseParamTable) {
p.initPulsarAddress() p.initPulsarAddress()
p.initRocksmqPath() p.initRocksmqPath()
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initGracefulTime() p.initGracefulTime()
@ -1237,15 +1205,6 @@ func (p *queryNodeConfig) initStatsChannelName() {
p.StatsChannelName = strings.Join(s, "-") p.StatsChannelName = strings.Join(s, "-")
} }
// ETCD configs
func (p *queryNodeConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *queryNodeConfig) initMetaRootPath() { func (p *queryNodeConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {
@ -1298,7 +1257,6 @@ type dataCoordConfig struct {
Address string Address string
// --- ETCD --- // --- ETCD ---
EtcdEndpoints []string
MetaRootPath string MetaRootPath string
KvRootPath string KvRootPath string
SegmentBinlogSubPath string SegmentBinlogSubPath string
@ -1352,7 +1310,6 @@ type dataCoordConfig struct {
func (p *dataCoordConfig) init(bp *BaseParamTable) { func (p *dataCoordConfig) init(bp *BaseParamTable) {
p.BaseParams = bp p.BaseParams = bp
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initKvRootPath() p.initKvRootPath()
p.initSegmentBinlogSubPath() p.initSegmentBinlogSubPath()
@ -1394,14 +1351,6 @@ func (p *dataCoordConfig) init(bp *BaseParamTable) {
p.initGCDropTolerance() p.initGCDropTolerance()
} }
func (p *dataCoordConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *dataCoordConfig) initPulsarAddress() { func (p *dataCoordConfig) initPulsarAddress() {
addr, err := p.BaseParams.Load("_PulsarAddress") addr, err := p.BaseParams.Load("_PulsarAddress")
if err != nil { if err != nil {
@ -1656,7 +1605,6 @@ type dataNodeConfig struct {
MsgChannelSubName string MsgChannelSubName string
// etcd // etcd
EtcdEndpoints []string
MetaRootPath string MetaRootPath string
ChannelWatchSubPath string ChannelWatchSubPath string
@ -1689,7 +1637,6 @@ func (p *dataNodeConfig) init(bp *BaseParamTable) {
p.initTimeTickChannelName() p.initTimeTickChannelName()
p.initMsgChannelSubName() p.initMsgChannelSubName()
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initChannelWatchPath() p.initChannelWatchPath()
@ -1792,14 +1739,6 @@ func (p *dataNodeConfig) initMsgChannelSubName() {
p.MsgChannelSubName = strings.Join(s, "-") p.MsgChannelSubName = strings.Join(s, "-")
} }
func (p *dataNodeConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *dataNodeConfig) initMetaRootPath() { func (p *dataNodeConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {
@ -1883,7 +1822,6 @@ type indexCoordConfig struct {
Address string Address string
Port int Port int
EtcdEndpoints []string
KvRootPath string KvRootPath string
MetaRootPath string MetaRootPath string
IndexStorageRootPath string IndexStorageRootPath string
@ -1901,7 +1839,6 @@ type indexCoordConfig struct {
func (p *indexCoordConfig) init(bp *BaseParamTable) { func (p *indexCoordConfig) init(bp *BaseParamTable) {
p.BaseParams = bp p.BaseParams = bp
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initKvRootPath() p.initKvRootPath()
p.initMinIOAddress() p.initMinIOAddress()
@ -1912,14 +1849,6 @@ func (p *indexCoordConfig) init(bp *BaseParamTable) {
p.initIndexStorageRootPath() p.initIndexStorageRootPath()
} }
func (p *indexCoordConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *indexCoordConfig) initMetaRootPath() { func (p *indexCoordConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {
@ -2013,7 +1942,6 @@ type indexNodeConfig struct {
NodeID int64 NodeID int64
Alias string Alias string
EtcdEndpoints []string
MetaRootPath string MetaRootPath string
IndexStorageRootPath string IndexStorageRootPath string
@ -2037,7 +1965,6 @@ func (p *indexNodeConfig) init(bp *BaseParamTable) {
p.initMinIOSecretAccessKey() p.initMinIOSecretAccessKey()
p.initMinIOUseSSL() p.initMinIOUseSSL()
p.initMinioBucketName() p.initMinioBucketName()
p.initEtcdEndpoints()
p.initMetaRootPath() p.initMetaRootPath()
p.initIndexStorageRootPath() p.initIndexStorageRootPath()
p.initKnowhereSimdType() p.initKnowhereSimdType()
@ -2083,14 +2010,6 @@ func (p *indexNodeConfig) initMinIOUseSSL() {
} }
} }
func (p *indexNodeConfig) initEtcdEndpoints() {
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
if err != nil {
panic(err)
}
p.EtcdEndpoints = strings.Split(endpoints, ",")
}
func (p *indexNodeConfig) initMetaRootPath() { func (p *indexNodeConfig) initMetaRootPath() {
rootPath, err := p.BaseParams.Load("etcd.rootPath") rootPath, err := p.BaseParams.Load("etcd.rootPath")
if err != nil { if err != nil {

View File

@ -40,9 +40,6 @@ func TestGlobalParamTable(t *testing.T) {
assert.NotEqual(t, Params.PulsarAddress, "") assert.NotEqual(t, Params.PulsarAddress, "")
t.Logf("pulsar address = %s", Params.PulsarAddress) t.Logf("pulsar address = %s", Params.PulsarAddress)
assert.NotZero(t, len(Params.EtcdEndpoints))
t.Logf("etcd endpoints = %s", Params.EtcdEndpoints)
assert.NotEqual(t, Params.MetaRootPath, "") assert.NotEqual(t, Params.MetaRootPath, "")
t.Logf("meta root path = %s", Params.MetaRootPath) t.Logf("meta root path = %s", Params.MetaRootPath)
@ -88,8 +85,6 @@ func TestGlobalParamTable(t *testing.T) {
t.Run("test proxyConfig", func(t *testing.T) { t.Run("test proxyConfig", func(t *testing.T) {
Params := GlobalParams.ProxyCfg Params := GlobalParams.ProxyCfg
t.Logf("EtcdEndPoints: %v", Params.EtcdEndpoints)
t.Logf("MetaRootPath: %s", Params.MetaRootPath) t.Logf("MetaRootPath: %s", Params.MetaRootPath)
t.Logf("PulsarAddress: %s", Params.PulsarAddress) t.Logf("PulsarAddress: %s", Params.PulsarAddress)
@ -300,9 +295,6 @@ func TestGlobalParamTable(t *testing.T) {
assert.Equal(t, name, "by-dev-dataNode-2") assert.Equal(t, name, "by-dev-dataNode-2")
log.Println("MsgChannelSubName:", name) log.Println("MsgChannelSubName:", name)
endpoints := Params.EtcdEndpoints
log.Println("EtcdEndpoints:", endpoints)
path1 = Params.MetaRootPath path1 = Params.MetaRootPath
log.Println("MetaRootPath:", path1) log.Println("MetaRootPath:", path1)
@ -336,8 +328,6 @@ func TestGlobalParamTable(t *testing.T) {
t.Logf("Port: %v", Params.Port) t.Logf("Port: %v", Params.Port)
t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints)
t.Logf("KvRootPath: %v", Params.KvRootPath) t.Logf("KvRootPath: %v", Params.KvRootPath)
t.Logf("MetaRootPath: %v", Params.MetaRootPath) t.Logf("MetaRootPath: %v", Params.MetaRootPath)
@ -374,8 +364,6 @@ func TestGlobalParamTable(t *testing.T) {
t.Logf("Alias: %v", Params.Alias) t.Logf("Alias: %v", Params.Alias)
t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints)
t.Logf("MetaRootPath: %v", Params.MetaRootPath) t.Logf("MetaRootPath: %v", Params.MetaRootPath)
t.Logf("MinIOAddress: %v", Params.MinIOAddress) t.Logf("MinIOAddress: %v", Params.MinIOAddress)

View File

@ -21,6 +21,7 @@ import (
"github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/allocator"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -32,10 +33,12 @@ func Test_InitRmq(t *testing.T) {
endpoints = "localhost:2379" endpoints = "localhost:2379"
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
defer etcdCli.Close()
if err != nil { if err != nil {
log.Fatalf("New clientv3 error = %v", err) log.Fatalf("New clientv3 error = %v", err)
} }
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
_ = idAllocator.Initialize() _ = idAllocator.Initialize()

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/allocator"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
rocksdbkv "github.com/milvus-io/milvus/internal/kv/rocksdb" rocksdbkv "github.com/milvus-io/milvus/internal/kv/rocksdb"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -314,7 +315,10 @@ func TestRocksmq_Seek(t *testing.T) {
func TestRocksmq_Loop(t *testing.T) { func TestRocksmq_Loop(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -382,7 +386,10 @@ func TestRocksmq_Loop(t *testing.T) {
func TestRocksmq_Goroutines(t *testing.T) { func TestRocksmq_Goroutines(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -454,7 +461,10 @@ func TestRocksmq_Goroutines(t *testing.T) {
*/ */
func TestRocksmq_Throughout(t *testing.T) { func TestRocksmq_Throughout(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -509,7 +519,10 @@ func TestRocksmq_Throughout(t *testing.T) {
func TestRocksmq_MultiChan(t *testing.T) { func TestRocksmq_MultiChan(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -558,7 +571,10 @@ func TestRocksmq_MultiChan(t *testing.T) {
func TestRocksmq_CopyData(t *testing.T) { func TestRocksmq_CopyData(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -621,7 +637,10 @@ func TestRocksmq_CopyData(t *testing.T) {
func TestRocksmq_SeekToLatest(t *testing.T) { func TestRocksmq_SeekToLatest(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -709,7 +728,10 @@ func TestRocksmq_SeekToLatest(t *testing.T) {
func TestRocksmq_Reader(t *testing.T) { func TestRocksmq_Reader(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -769,8 +791,10 @@ func TestRocksmq_Reader(t *testing.T) {
func TestReader_CornerCase(t *testing.T) { func TestReader_CornerCase(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err) assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
_ = idAllocator.Initialize() _ = idAllocator.Initialize()
@ -826,7 +850,10 @@ func TestReader_CornerCase(t *testing.T) {
func TestRocksmq_Close(t *testing.T) { func TestRocksmq_Close(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
assert.Nil(t, err) assert.Nil(t, err)
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
@ -863,8 +890,9 @@ func TestRocksmq_Close(t *testing.T) {
func TestRocksmq_SeekWithNoConsumerError(t *testing.T) { func TestRocksmq_SeekWithNoConsumerError(t *testing.T) {
ep := etcdEndpoints() ep := etcdEndpoints()
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") etcdCli, err := etcd.GetRemoteEtcdClient(ep)
assert.Nil(t, err) assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
defer etcdKV.Close() defer etcdKV.Close()
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
_ = idAllocator.Initialize() _ = idAllocator.Initialize()

View File

@ -55,10 +55,11 @@ type Session struct {
// keepAliveCancel to cancel the etcd KeepAlive // keepAliveCancel to cancel the etcd KeepAlive
keepAliveCancel context.CancelFunc keepAliveCancel context.CancelFunc
ServerID int64 `json:"ServerID,omitempty"` ServerID int64 `json:"ServerID,omitempty"`
ServerName string `json:"ServerName,omitempty"` ServerName string `json:"ServerName,omitempty"`
Address string `json:"Address,omitempty"` Address string `json:"Address,omitempty"`
Exclusive bool `json:"Exclusive,omitempty"` Exclusive bool `json:"Exclusive,omitempty"`
TriggerKill bool
liveCh <-chan bool liveCh <-chan bool
etcdCli *clientv3.Client etcdCli *clientv3.Client
@ -73,7 +74,7 @@ type Session struct {
// ServerID, ServerName, Address, Exclusive will be assigned after Init(). // ServerID, ServerName, Address, Exclusive will be assigned after Init().
// metaRoot is a path in etcd to save session information. // metaRoot is a path in etcd to save session information.
// etcdEndpoints is to init etcdCli when NewSession // etcdEndpoints is to init etcdCli when NewSession
func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *Session { func NewSession(ctx context.Context, metaRoot string, client *clientv3.Client) *Session {
session := &Session{ session := &Session{
ctx: ctx, ctx: ctx,
metaRoot: metaRoot, metaRoot: metaRoot,
@ -83,16 +84,12 @@ func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *S
connectEtcdFn := func() error { connectEtcdFn := func() error {
log.Debug("Session try to connect to etcd") log.Debug("Session try to connect to etcd")
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints, DialTimeout: 5 * time.Second})
if err != nil {
return err
}
ctx2, cancel2 := context.WithTimeout(session.ctx, 5*time.Second) ctx2, cancel2 := context.WithTimeout(session.ctx, 5*time.Second)
defer cancel2() defer cancel2()
if _, err = etcdCli.Get(ctx2, "health"); err != nil { if _, err := client.Get(ctx2, "health"); err != nil {
return err return err
} }
session.etcdCli = etcdCli session.etcdCli = client
return nil return nil
} }
err := retry.Do(ctx, connectEtcdFn, retry.Attempts(300)) err := retry.Do(ctx, connectEtcdFn, retry.Attempts(300))
@ -107,10 +104,11 @@ func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *S
// Init will initialize base struct of the Session, including ServerName, ServerID, // Init will initialize base struct of the Session, including ServerName, ServerID,
// Address, Exclusive. ServerID is obtained in getServerID. // Address, Exclusive. ServerID is obtained in getServerID.
func (s *Session) Init(serverName, address string, exclusive bool) { func (s *Session) Init(serverName, address string, exclusive bool, triggerKill bool) {
s.ServerName = serverName s.ServerName = serverName
s.Address = address s.Address = address
s.Exclusive = exclusive s.Exclusive = exclusive
s.TriggerKill = triggerKill
s.checkIDExist() s.checkIDExist()
serverID, err := s.getServerID() serverID, err := s.getServerID()
if err != nil { if err != nil {

View File

@ -12,6 +12,7 @@ import (
"time" "time"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -34,8 +35,10 @@ func TestGetServerIDConcurrently(t *testing.T) {
} }
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot) etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err) require.NoError(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot)
err = etcdKV.RemoveWithPrefix("") err = etcdKV.RemoveWithPrefix("")
assert.NoError(t, err) assert.NoError(t, err)
@ -45,7 +48,7 @@ func TestGetServerIDConcurrently(t *testing.T) {
var wg sync.WaitGroup var wg sync.WaitGroup
var muList = sync.Mutex{} var muList = sync.Mutex{}
s := NewSession(ctx, metaRoot, etcdEndpoints) s := NewSession(ctx, metaRoot, etcdCli)
res := make([]int64, 0) res := make([]int64, 0)
getIDFunc := func() { getIDFunc := func() {
@ -79,16 +82,17 @@ func TestInit(t *testing.T) {
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot) etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err) require.NoError(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot)
err = etcdKV.RemoveWithPrefix("") err = etcdKV.RemoveWithPrefix("")
assert.NoError(t, err) assert.NoError(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
s := NewSession(ctx, metaRoot, etcdEndpoints) s := NewSession(ctx, metaRoot, etcdCli)
s.Init("inittest", "testAddr", false) s.Init("inittest", "testAddr", false, false)
assert.NotEqual(t, int64(0), s.leaseID) assert.NotEqual(t, int64(0), s.leaseID)
assert.NotEqual(t, int64(0), s.ServerID) assert.NotEqual(t, int64(0), s.ServerID)
s.Register() s.Register()
@ -108,8 +112,10 @@ func TestUpdateSessions(t *testing.T) {
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "") etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err) require.NoError(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "")
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
@ -117,7 +123,7 @@ func TestUpdateSessions(t *testing.T) {
var wg sync.WaitGroup var wg sync.WaitGroup
var muList = sync.Mutex{} var muList = sync.Mutex{}
s := NewSession(ctx, metaRoot, etcdEndpoints) s := NewSession(ctx, metaRoot, etcdCli)
sessions, rev, err := s.GetSessions("test") sessions, rev, err := s.GetSessions("test")
assert.Nil(t, err) assert.Nil(t, err)
@ -127,8 +133,10 @@ func TestUpdateSessions(t *testing.T) {
sList := []*Session{} sList := []*Session{}
getIDFunc := func() { getIDFunc := func() {
singleS := NewSession(ctx, metaRoot, etcdEndpoints) etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
singleS.Init("test", "testAddr", false) require.NoError(t, err)
singleS := NewSession(ctx, metaRoot, etcdCli)
singleS.Init("test", "testAddr", false, false)
singleS.Register() singleS.Register()
muList.Lock() muList.Lock()
sList = append(sList, singleS) sList = append(sList, singleS)
@ -220,12 +228,15 @@ func TestWatcherHandleWatchResp(t *testing.T) {
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/by-dev/session-ut")
require.NoError(t, err)
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
require.NoError(t, err)
defer etcdCli.Close()
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/by-dev/session-ut")
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("/by-dev/session-ut") defer etcdKV.RemoveWithPrefix("/by-dev/session-ut")
s := NewSession(ctx, metaRoot, etcdEndpoints) s := NewSession(ctx, metaRoot, etcdCli)
defer s.Revoke(time.Second) defer s.Revoke(time.Second)
getWatcher := func(s *Session, rewatch Rewatch) *sessionWatcher { getWatcher := func(s *Session, rewatch Rewatch) *sessionWatcher {
@ -317,7 +328,7 @@ func TestWatcherHandleWatchResp(t *testing.T) {
}) })
t.Run("err handled but list failed", func(t *testing.T) { t.Run("err handled but list failed", func(t *testing.T) {
s := NewSession(ctx, "/by-dev/session-ut", etcdEndpoints) s := NewSession(ctx, "/by-dev/session-ut", etcdCli)
s.etcdCli.Close() s.etcdCli.Close()
w := getWatcher(s, func(sessions map[string]*Session) error { w := getWatcher(s, func(sessions map[string]*Session) error {
return nil return nil
@ -364,16 +375,18 @@ func TestSessionRevoke(t *testing.T) {
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
etcdEndpoints := strings.Split(endpoints, ",") etcdEndpoints := strings.Split(endpoints, ",")
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot) etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
assert.NoError(t, err) defer etcdCli.Close()
require.NoError(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot)
err = etcdKV.RemoveWithPrefix("") err = etcdKV.RemoveWithPrefix("")
assert.NoError(t, err) assert.NoError(t, err)
defer etcdKV.Close() defer etcdKV.Close()
defer etcdKV.RemoveWithPrefix("") defer etcdKV.RemoveWithPrefix("")
s = NewSession(ctx, metaRoot, etcdEndpoints) s = NewSession(ctx, metaRoot, etcdCli)
s.Init("revoketest", "testAddr", false) s.Init("revoketest", "testAddr", false, false)
assert.NotPanics(t, func() { assert.NotPanics(t, func() {
s.Revoke(time.Second) s.Revoke(time.Second)
}) })

View File

@ -17,6 +17,7 @@ import (
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
) )
const ( const (
@ -70,6 +71,6 @@ func AddPhysicalTimeOnTs(timeInMs int64, ts uint64) uint64 {
} }
// NewTSOKVBase returns a etcdkv.EtcdKV object // NewTSOKVBase returns a etcdkv.EtcdKV object
func NewTSOKVBase(etcdEndpoints []string, tsoRoot, subPath string) (*etcdkv.EtcdKV, error) { func NewTSOKVBase(client *clientv3.Client, tsoRoot, subPath string) *etcdkv.EtcdKV {
return etcdkv.NewEtcdKV(etcdEndpoints, path.Join(tsoRoot, subPath)) return etcdkv.NewEtcdKV(client, path.Join(tsoRoot, subPath))
} }