diff --git a/cmd/components/data_coord.go b/cmd/components/data_coord.go index e574ebcb41..f223f39bc5 100644 --- a/cmd/components/data_coord.go +++ b/cmd/components/data_coord.go @@ -34,10 +34,7 @@ type DataCoord struct { // NewDataCoord creates a new DataCoord func NewDataCoord(ctx context.Context, factory msgstream.Factory) (*DataCoord, error) { - s, err := grpcdatacoordclient.NewServer(ctx, factory) - if err != nil { - return nil, err - } + s := grpcdatacoordclient.NewServer(ctx, factory) return &DataCoord{ ctx: ctx, diff --git a/cmd/main.go b/cmd/main.go index f95ac37cae..1409b10cd4 100644 --- a/cmd/main.go +++ b/cmd/main.go @@ -232,7 +232,7 @@ func main() { os.Exit(-1) } - var localMsg = false + var local = false role := roles.MilvusRoles{} switch serverType { case typeutil.RootCoordRole: @@ -260,7 +260,7 @@ func main() { role.EnableDataNode = true role.EnableIndexCoord = true role.EnableIndexNode = true - localMsg = true + local = true case roleMixture: role.EnableRootCoord = enableRootCoord role.EnableQueryCoord = enableQueryCoord @@ -291,7 +291,7 @@ func main() { panic(err) } defer removePidFile(fd) - role.Run(localMsg, svrAlias) + role.Run(local, svrAlias) case "stop": if err := stopPid(filename, runtimeDir); err != nil { fmt.Fprintf(os.Stderr, "%s\n\n", err.Error()) diff --git a/cmd/roles/roles.go b/cmd/roles/roles.go index db9632afa4..a2b16e7d6c 100644 --- a/cmd/roles/roles.go +++ b/cmd/roles/roles.go @@ -43,13 +43,17 @@ import ( "github.com/milvus-io/milvus/internal/querycoord" "github.com/milvus-io/milvus/internal/querynode" "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/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/trace" "github.com/milvus-io/milvus/internal/util/typeutil" ) +var Params paramtable.GlobalParamTable + func newMsgFactory(localMsg bool) msgstream.Factory { if localMsg { return msgstream.NewRmsFactory() @@ -338,7 +342,7 @@ func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, alias st } // 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 { closer := trace.InitTracing("standalone") if closer != nil { @@ -349,7 +353,8 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { ctx, cancel := context.WithCancel(context.Background()) // only standalone enable localMsg - if localMsg { + if local { + Params.Init() if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode); err != nil { log.Error("Failed to set deploy mode: ", zap.Error(err)) } @@ -358,6 +363,12 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { panic(err) } defer stopRocksmq() + + if Params.BaseParams.UseEmbedEtcd { + // start etcd server + etcd.InitEtcdServer(&Params.BaseParams) + defer etcd.StopEtcdServer() + } } else { if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode); err != nil { 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 if mr.EnableRootCoord { - rc = mr.runRootCoord(ctx, localMsg) + rc = mr.runRootCoord(ctx, local) if rc != nil { defer rc.Stop() } @@ -375,7 +386,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var pn *components.Proxy if mr.EnableProxy { pctx := logutil.WithModule(ctx, "Proxy") - pn = mr.runProxy(pctx, localMsg, alias) + pn = mr.runProxy(pctx, local, alias) if pn != nil { defer pn.Stop() } @@ -383,7 +394,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var qs *components.QueryCoord if mr.EnableQueryCoord { - qs = mr.runQueryCoord(ctx, localMsg) + qs = mr.runQueryCoord(ctx, local) if qs != nil { defer qs.Stop() } @@ -391,7 +402,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var qn *components.QueryNode if mr.EnableQueryNode { - qn = mr.runQueryNode(ctx, localMsg, alias) + qn = mr.runQueryNode(ctx, local, alias) if qn != nil { defer qn.Stop() } @@ -399,7 +410,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var ds *components.DataCoord if mr.EnableDataCoord { - ds = mr.runDataCoord(ctx, localMsg) + ds = mr.runDataCoord(ctx, local) if ds != nil { defer ds.Stop() } @@ -407,7 +418,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var dn *components.DataNode if mr.EnableDataNode { - dn = mr.runDataNode(ctx, localMsg, alias) + dn = mr.runDataNode(ctx, local, alias) if dn != nil { defer dn.Stop() } @@ -415,7 +426,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var is *components.IndexCoord if mr.EnableIndexCoord { - is = mr.runIndexCoord(ctx, localMsg) + is = mr.runIndexCoord(ctx, local) if is != nil { defer is.Stop() } @@ -423,13 +434,13 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) { var in *components.IndexNode if mr.EnableIndexNode { - in = mr.runIndexNode(ctx, localMsg, alias) + in = mr.runIndexNode(ctx, local, alias) if in != nil { defer in.Stop() } } - if localMsg { + if local { standaloneHealthzHandler := func(w http.ResponseWriter, r *http.Request) { ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() diff --git a/cmd/tools/datameta/main.go b/cmd/tools/datameta/main.go index 8a15997f3a..a68b574108 100644 --- a/cmd/tools/datameta/main.go +++ b/cmd/tools/datameta/main.go @@ -10,6 +10,7 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/tsoutil" "go.uber.org/zap" ) @@ -27,11 +28,14 @@ var ( func main() { flag.Parse() - etcdkv, err := etcdkv.NewEtcdKV([]string{*etcdAddr}, *rootPath) + + etcdCli, err := etcd.GetRemoteEtcdClient([]string{*etcdAddr}) if err != nil { log.Fatal("failed to connect to etcd", zap.Error(err)) } + etcdkv := etcdkv.NewEtcdKV(etcdCli, *rootPath) + keys, values, err := etcdkv.LoadWithPrefix("/") if err != nil { log.Fatal("failed to list ", zap.Error(err)) diff --git a/internal/allocator/global_id_test.go b/internal/allocator/global_id_test.go index f47ef1a298..602142b7dc 100644 --- a/internal/allocator/global_id_test.go +++ b/internal/allocator/global_id_test.go @@ -17,24 +17,24 @@ package allocator import ( - "os" - "strings" "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/stretchr/testify/assert" ) var gTestIDAllocator *GlobalIDAllocator +var Params paramtable.GlobalParamTable + func TestGlobalTSOAllocator_All(t *testing.T) { - endpoints := os.Getenv("ETCD_ENDPOINTS") - if endpoints == "" { - endpoints = "localhost:2379" - } - etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "gidTest") + Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.NoError(t, err) + defer etcdCli.Close() + etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "gidTest") gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", etcdKV) diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index affe0e11e5..f777e62c5f 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -45,6 +45,7 @@ import ( "github.com/milvus-io/milvus/internal/util/typeutil" "github.com/minio/minio-go/v7" "github.com/minio/minio-go/v7/pkg/credentials" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -82,7 +83,7 @@ const ( ) 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` var _ types.DataCoord = (*Server)(nil) @@ -100,6 +101,7 @@ type Server struct { isServing ServerState helper ServerHelper + etcdCli *clientv3.Client kvClient *etcdkv.EtcdKV meta *meta segmentManager Manager @@ -177,7 +179,7 @@ func SetSegmentManager(manager Manager) Option { } // 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()) s := &Server{ ctx: ctx, @@ -194,15 +196,15 @@ func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option for _, opt := range opts { opt(s) } - return s, nil + return s } func defaultDataNodeCreatorFunc(ctx context.Context, addr string) (types.DataNode, error) { return datanodeclient.NewClient(ctx, addr) } -func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) { - return rootcoordclient.NewClient(ctx, metaRootPath, etcdEndpoints) +func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, client *clientv3.Client) (types.RootCoord, error) { + return rootcoordclient.NewClient(ctx, metaRootPath, client) } // 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)) } // manually send signal to starter goroutine - syscall.Kill(syscall.Getpid(), syscall.SIGINT) + if s.session.TriggerKill { + syscall.Kill(syscall.Getpid(), syscall.SIGINT) + } }) return nil } 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 { 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.BaseParams.SetLogger(Params.DataCoordCfg.NodeID) return nil @@ -311,6 +315,11 @@ func (s *Server) initCluster() error { return nil } +// SetEtcdClient sets etcd client for datacoord. +func (s *Server) SetEtcdClient(client *clientv3.Client) { + s.etcdCli = client +} + func (s *Server) createCompactionHandler() { s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh) s.compactionHandler.start() @@ -407,20 +416,17 @@ func (s *Server) startSegmentManager() { } func (s *Server) initMeta() error { - connectEtcdFn := func() error { - etcdKV, err := etcdkv.NewEtcdKV(Params.DataCoordCfg.EtcdEndpoints, Params.DataCoordCfg.MetaRootPath) - if err != nil { - return err - } - - s.kvClient = etcdKV + etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.DataCoordCfg.MetaRootPath) + s.kvClient = etcdKV + reloadEtcdFn := func() error { + var err error s.meta, err = newMeta(s.kvClient) if err != nil { return err } return nil } - return retry.Do(s.ctx, connectEtcdFn, retry.Attempts(connEtcdMaxRetryTime)) + return retry.Do(s.ctx, reloadEtcdFn, retry.Attempts(connEtcdMaxRetryTime)) } func (s *Server) startServerLoop() { @@ -724,7 +730,7 @@ func (s *Server) handleFlushingSegments(ctx context.Context) { func (s *Server) initRootCoordClient() 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 } if err = s.rootCoordClient.Init(); err != nil { diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index d929e3b374..f4a59ca291 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -37,8 +37,8 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/retry" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -613,8 +613,7 @@ func TestGetFlushedSegments(t *testing.T) { func TestService_WatchServices(t *testing.T) { factory := msgstream.NewPmsFactory() - svr, err := CreateServer(context.TODO(), factory) - assert.Nil(t, err) + svr := CreateServer(context.TODO(), factory) svr.serverLoopWg.Add(1) ech := make(chan *sessionutil.SessionEvent) @@ -1522,7 +1521,7 @@ func TestGetRecoveryInfo(t *testing.T) { svr := newTestServer(t, nil) 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 } @@ -1565,7 +1564,7 @@ func TestGetRecoveryInfo(t *testing.T) { svr := newTestServer(t, nil) 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 } @@ -1593,7 +1592,7 @@ func TestGetRecoveryInfo(t *testing.T) { svr := newTestServer(t, nil) 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 } @@ -1621,7 +1620,7 @@ func TestGetRecoveryInfo(t *testing.T) { svr := newTestServer(t, nil) 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 } @@ -1699,7 +1698,7 @@ func TestGetRecoveryInfo(t *testing.T) { svr := newTestServer(t, nil) 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 } @@ -1954,7 +1953,7 @@ func TestOptions(t *testing.T) { t.Run("SetRootCoordCreator", func(t *testing.T) { svr := newTestServer(t, nil) 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") } opt := SetRootCoordCreator(crt) @@ -1992,8 +1991,7 @@ func TestOptions(t *testing.T) { factory := msgstream.NewPmsFactory() - svr, err := CreateServer(context.TODO(), factory, opt) - assert.Nil(t, err) + svr := CreateServer(context.TODO(), factory, opt) dn, err := svr.dataNodeCreator(context.Background(), "") assert.Nil(t, dn) assert.Nil(t, err) @@ -2246,18 +2244,18 @@ func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Se err = factory.SetParams(m) assert.Nil(t, err) - etcdCli, err := initEtcd(Params.DataCoordCfg.EtcdEndpoints) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) sessKey := path.Join(Params.DataCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) assert.Nil(t, err) - svr, err := CreateServer(context.TODO(), factory, opts...) - assert.Nil(t, err) + svr := CreateServer(context.TODO(), factory, opts...) + svr.SetEtcdClient(etcdCli) svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) { 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 } assert.Nil(t, err) @@ -2276,20 +2274,3 @@ func closeTestServer(t *testing.T, svr *Server) { err = svr.CleanMeta() 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 -} diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 8e724ebe3e..d8927d8cd2 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -83,6 +83,7 @@ var Params paramtable.GlobalParamTable // services in datanode package. // // DataNode implements `types.Component`, `types.DataNode` interfaces. +// `etcdCli` is a connection of etcd // `rootCoord` is a grpc client of root coordinator. // `dataCoord` is a grpc client of data service. // `NodeID` is unique to each datanode. @@ -109,6 +110,7 @@ type DataNode struct { segmentCache *Cache compactionExecutor *compactionExecutor + etcdCli *clientv3.Client rootCoord types.RootCoord dataCoord types.DataCoord @@ -144,6 +146,11 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode { 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. func (node *DataNode) SetRootCoord(rc types.RootCoord) error { switch { @@ -182,18 +189,20 @@ func (node *DataNode) Register() error { log.Fatal("failed to stop server", zap.Error(err)) } // manually send signal to starter goroutine - syscall.Kill(syscall.Getpid(), syscall.SIGINT) + if node.session.TriggerKill { + syscall.Kill(syscall.Getpid(), syscall.SIGINT) + } }) return nil } 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 { 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 node.NodeID = node.session.ServerID Params.BaseParams.SetLogger(Params.DataNodeCfg.NodeID) @@ -411,10 +420,7 @@ func (node *DataNode) Start() error { } connectEtcdFn := func() error { - etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath) - if err != nil { - return err - } + etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.DataNodeCfg.MetaRootPath) node.watchKv = etcdKV return nil } diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 921f19f221..239eecf06d 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -34,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/sessionutil" @@ -62,7 +63,11 @@ func TestMain(t *testing.M) { func TestDataNode(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) 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) err = node.Start() assert.Nil(t, err) @@ -164,6 +169,7 @@ func TestDataNode(t *testing.T) { dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments" node1 := newIDLEDataNodeMock(context.TODO()) + node1.SetEtcdClient(etcdCli) err = node1.Init() assert.Nil(t, err) err = node1.Start() @@ -475,7 +481,11 @@ func TestDataNode(t *testing.T) { func TestWatchChannel(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) 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) err = node.Start() assert.Nil(t, err) @@ -485,8 +495,7 @@ func TestWatchChannel(t *testing.T) { defer cancel() t.Run("test watch channel", func(t *testing.T) { - kv, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath) - require.NoError(t, err) + kv := etcdkv.NewEtcdKV(etcdCli, Params.DataNodeCfg.MetaRootPath) oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid" path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, oldInvalidCh) err = kv.Save(path, string([]byte{23})) diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index 002da5b847..b77e0cb665 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -42,6 +42,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/schemapb" + "github.com/milvus-io/milvus/internal/util/etcd" ) const ctxTimeInMillisecond = 5000 @@ -106,10 +107,11 @@ func makeNewChannelNames(names []string, suffix string) []string { } func clearEtcd(rootPath string) error { - etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, rootPath) + client, err := etcd.GetEtcdClient(&Params.BaseParams) if err != nil { return err } + etcdKV := etcdkv.NewEtcdKV(client, rootPath) err = etcdKV.RemoveWithPrefix("writer/segment") if err != nil { diff --git a/internal/distributed/connection_manager_test.go b/internal/distributed/connection_manager_test.go index 2b2974ff7f..f9b7c1baa0 100644 --- a/internal/distributed/connection_manager_test.go +++ b/internal/distributed/connection_manager_test.go @@ -28,6 +28,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/querypb" "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/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" @@ -62,7 +63,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() rootcoordpb.RegisterRootCoordServer(grpcServer, rootCoord) 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() assert.Eventually(t, func() bool { rootCoord, ok := cm.GetRootCoordClient() @@ -79,7 +80,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() querypb.RegisterQueryCoordServer(grpcServer, queryCoord) 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() assert.Eventually(t, func() bool { queryCoord, ok := cm.GetQueryCoordClient() @@ -96,7 +97,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() datapb.RegisterDataCoordServer(grpcServer, dataCoord) 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() assert.Eventually(t, func() bool { dataCoord, ok := cm.GetDataCoordClient() @@ -113,7 +114,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() indexpb.RegisterIndexCoordServer(grpcServer, indexCoord) 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() assert.Eventually(t, func() bool { indexCoord, ok := cm.GetIndexCoordClient() @@ -130,7 +131,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() querypb.RegisterQueryNodeServer(grpcServer, queryNode) 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() assert.Eventually(t, func() bool { queryNodes, ok := cm.GetQueryNodeClients() @@ -147,7 +148,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() datapb.RegisterDataNodeServer(grpcServer, dataNode) 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() assert.Eventually(t, func() bool { dataNodes, ok := cm.GetDataNodeClients() @@ -164,7 +165,7 @@ func TestConnectionManager(t *testing.T) { defer grpcServer.Stop() indexpb.RegisterIndexNodeServer(grpcServer, indexNode) 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() assert.Eventually(t, func() bool { indexNodes, ok := cm.GetIndexNodeClients() @@ -250,6 +251,10 @@ func initSession(ctx context.Context) *sessionutil.Session { log.Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath)) 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 } diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index 813b93ec3a..ee43229c4f 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -43,8 +44,8 @@ type Client struct { } // NewClient creates a new client instance -func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { - sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) +func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) { + sess := sessionutil.NewSession(ctx, metaRoot, etcdCli) if sess == nil { err := fmt.Errorf("new session error, maybe can not connect to etcd") log.Debug("DataCoordClient NewClient failed", zap.Error(err)) diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index 651a5d2cd0..b42c732dc7 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -21,9 +21,9 @@ import ( "errors" "testing" - "github.com/milvus-io/milvus/internal/util/mock" - "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" "google.golang.org/grpc" ) @@ -32,7 +32,9 @@ func Test_NewClient(t *testing.T) { proxy.Params.InitOnce() 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.NotNil(t, client) diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index 56b054bac9..449532b2d1 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -36,10 +36,12 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/paramtable" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -54,7 +56,9 @@ type Server struct { cancel context.CancelFunc wg sync.WaitGroup - dataCoord types.DataCoord + dataCoord types.DataCoordComponent + + etcdCli *clientv3.Client grpcErrChan chan error grpcServer *grpc.Server @@ -62,8 +66,7 @@ type Server struct { } // NewServer new data service grpc server -func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) (*Server, error) { - var err error +func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) *Server { ctx1, cancel := context.WithCancel(ctx) s := &Server{ @@ -71,11 +74,8 @@ func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord cancel: cancel, grpcErrChan: make(chan error), } - s.dataCoord, err = datacoord.CreateServer(s.ctx, factory, opts...) - if err != nil { - return nil, err - } - return s, nil + s.dataCoord = datacoord.CreateServer(s.ctx, factory, opts...) + return s } func (s *Server) init() error { @@ -89,7 +89,15 @@ func (s *Server) init() error { datacoord.Params.DataCoordCfg.Port = Params.Port 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 { log.Debug("DataCoord startGrpc failed", zap.Error(err)) return err @@ -178,6 +186,9 @@ func (s *Server) Stop() error { } s.cancel() + if s.etcdCli != nil { + defer s.etcdCli.Close() + } if s.grpcServer != nil { log.Debug("Graceful stop grpc server...") s.grpcServer.GracefulStop() diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index d111ce5d11..f156b6805d 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -26,6 +26,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/stretchr/testify/assert" + clientv3 "go.etcd.io/etcd/client/v3" ) /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// @@ -72,6 +73,9 @@ func (m *MockDataCoord) Register() error { return m.regErr } +func (m *MockDataCoord) SetEtcdClient(etcdClient *clientv3.Client) { +} + func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { 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) { ctx := context.Background() - server, err := NewServer(ctx, nil) - assert.Nil(t, err) + server := NewServer(ctx, nil) assert.NotNil(t, server) t.Run("Run", func(t *testing.T) { server.dataCoord = &MockDataCoord{} - err = server.Run() + err := server.Run() assert.Nil(t, err) }) @@ -335,21 +338,20 @@ func Test_NewServer(t *testing.T) { assert.NotNil(t, resp) }) - err = server.Stop() + err := server.Stop() assert.Nil(t, err) } func Test_Run(t *testing.T) { ctx := context.Background() - server, err := NewServer(ctx, nil) - assert.Nil(t, err) + server := NewServer(ctx, nil) assert.NotNil(t, server) server.dataCoord = &MockDataCoord{ regErr: errors.New("error"), } - err = server.Run() + err := server.Run() assert.Error(t, err) server.dataCoord = &MockDataCoord{ diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index 6aa72d7a5f..49fd89cbca 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -37,10 +37,12 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/paramtable" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -56,14 +58,14 @@ type Server struct { grpcServer *grpc.Server ctx context.Context cancel context.CancelFunc - - msFactory msgstream.Factory + etcdCli *clientv3.Client + msFactory msgstream.Factory rootCoord types.RootCoord dataCoord types.DataCoord - newRootCoordClient func(string, []string) (types.RootCoord, error) - newDataCoordClient func(string, []string) (types.DataCoord, error) + newRootCoordClient func(string, *clientv3.Client) (types.RootCoord, error) + newDataCoordClient func(string, *clientv3.Client) (types.DataCoord, error) closer io.Closer } @@ -76,11 +78,11 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) cancel: cancel, msFactory: factory, grpcErrChan: make(chan error), - newRootCoordClient: func(etcdMetaRoot string, etcdEndpoints []string) (types.RootCoord, error) { - return rcc.NewClient(ctx1, etcdMetaRoot, etcdEndpoints) + newRootCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.RootCoord, error) { + return rcc.NewClient(ctx1, etcdMetaRoot, client) }, - newDataCoordClient: func(etcdMetaRoot string, etcdEndpoints []string) (types.DataCoord, error) { - return dcc.NewClient(ctx1, etcdMetaRoot, etcdEndpoints) + newDataCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.DataCoord, error) { + 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 { return s.datanode.SetRootCoord(ms) } @@ -162,6 +168,9 @@ func (s *Server) Stop() error { } } s.cancel() + if s.etcdCli != nil { + defer s.etcdCli.Close() + } if s.grpcServer != nil { log.Debug("Graceful stop grpc server...") // make graceful stop has a timeout @@ -198,12 +207,19 @@ func (s *Server) init() error { dn.Params.DataNodeCfg.Port = Params.Port 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)) s.closer = closer addr := Params.IP + ":" + strconv.Itoa(Params.Port) log.Debug("DataNode address", zap.String("address", addr)) - err := s.startGrpc() + err = s.startGrpc() if err != nil { return err } @@ -211,7 +227,7 @@ func (s *Server) init() error { // --- RootCoord Client --- if s.newRootCoordClient != nil { 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 { log.Debug("DataNode newRootCoordClient failed", zap.Error(err)) panic(err) @@ -238,7 +254,7 @@ func (s *Server) init() error { // --- Data Server Client --- if s.newDataCoordClient != nil { 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 { log.Debug("DataNode newDataCoordClient failed", zap.Error(err)) panic(err) diff --git a/internal/distributed/datanode/service_test.go b/internal/distributed/datanode/service_test.go index 7afb25b254..358b9d4401 100644 --- a/internal/distributed/datanode/service_test.go +++ b/internal/distributed/datanode/service_test.go @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/typeutil" "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 } +func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) { +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// type mockDataCoord struct { types.DataCoord @@ -174,11 +178,11 @@ func Test_NewServer(t *testing.T) { assert.Nil(t, err) assert.NotNil(t, server) - server.newRootCoordClient = func(string, []string) (types.RootCoord, error) { + server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) { return &mockRootCoord{}, nil } - server.newDataCoordClient = func(string, []string) (types.DataCoord, error) { + server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) { return &mockDataCoord{}, nil } @@ -247,11 +251,11 @@ func Test_Run(t *testing.T) { regErr: errors.New("error"), } - server.newRootCoordClient = func(string, []string) (types.RootCoord, error) { + server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) { return &mockRootCoord{}, nil } - server.newDataCoordClient = func(string, []string) (types.DataCoord, error) { + server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) { return &mockDataCoord{}, nil } diff --git a/internal/distributed/indexcoord/client/client.go b/internal/distributed/indexcoord/client/client.go index 7ff0642eeb..8d8820491a 100644 --- a/internal/distributed/indexcoord/client/client.go +++ b/internal/distributed/indexcoord/client/client.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -43,8 +44,8 @@ type Client struct { } // NewClient creates a new IndexCoord client. -func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { - sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) +func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) { + sess := sessionutil.NewSession(ctx, metaRoot, etcdCli) if sess == nil { err := fmt.Errorf("new session error, maybe can not connect to etcd") log.Debug("IndexCoordClient NewClient failed", zap.Error(err)) diff --git a/internal/distributed/indexcoord/client/client_test.go b/internal/distributed/indexcoord/client/client_test.go index 19da519862..54c6901898 100644 --- a/internal/distributed/indexcoord/client/client_test.go +++ b/internal/distributed/indexcoord/client/client_test.go @@ -26,6 +26,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "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/stretchr/testify/assert" ) @@ -42,7 +43,9 @@ func TestIndexCoordClient(t *testing.T) { err = server.Run() 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.NotNil(t, icc) diff --git a/internal/distributed/indexcoord/service.go b/internal/distributed/indexcoord/service.go index b4aa2c5603..e27cd36e0b 100644 --- a/internal/distributed/indexcoord/service.go +++ b/internal/distributed/indexcoord/service.go @@ -32,10 +32,12 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/paramtable" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -49,7 +51,7 @@ type UniqueID = typeutil.UniqueID // Server is the grpc wrapper of IndexCoord. type Server struct { - indexcoord types.IndexCoord + indexcoord types.IndexCoordComponent grpcServer *grpc.Server grpcErrChan chan error @@ -58,6 +60,8 @@ type Server struct { loopCancel func() loopWg sync.WaitGroup + etcdCli *clientv3.Client + closer io.Closer } @@ -86,6 +90,14 @@ func (s *Server) init() error { closer := trace.InitTracing("IndexCoord") 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) go s.startGrpcLoop(indexcoord.Params.IndexCoordCfg.Port) // wait for grpc IndexCoord loop start @@ -126,7 +138,9 @@ func (s *Server) Stop() error { if s.indexcoord != nil { s.indexcoord.Stop() } - + if s.etcdCli != nil { + defer s.etcdCli.Close() + } s.loopCancel() if s.grpcServer != nil { log.Debug("Graceful stop grpc server...") @@ -138,7 +152,7 @@ func (s *Server) Stop() error { } // 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 return nil } diff --git a/internal/distributed/indexcoord/service_test.go b/internal/distributed/indexcoord/service_test.go index b83471ccde..2d2cf7f0e7 100644 --- a/internal/distributed/indexcoord/service_test.go +++ b/internal/distributed/indexcoord/service_test.go @@ -25,6 +25,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" + etcd "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) @@ -33,7 +34,11 @@ func TestIndexCoordinateServer(t *testing.T) { server, err := NewServer(ctx) assert.Nil(t, err) assert.NotNil(t, server) + Params.Init() + etcd, err := etcd.GetEtcdClient(&Params.BaseParamTable) + assert.NoError(t, err) indexCoordClient := &indexcoord.Mock{} + indexCoordClient.SetEtcdClient(etcd) err = server.SetClient(indexCoordClient) assert.Nil(t, err) err = server.Run() diff --git a/internal/distributed/indexnode/client/client_test.go b/internal/distributed/indexnode/client/client_test.go index 0565eaa725..d47a91a7af 100644 --- a/internal/distributed/indexnode/client/client_test.go +++ b/internal/distributed/indexnode/client/client_test.go @@ -26,13 +26,17 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/indexpb" "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/mock" + "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/typeutil" "github.com/stretchr/testify/assert" "google.golang.org/grpc" ) +var ParamsGlobal paramtable.GlobalParamTable + func Test_NewClient(t *testing.T) { ClientParams.InitOnce(typeutil.IndexNodeRole) ctx := context.Background() @@ -123,6 +127,10 @@ func TestIndexNodeClient(t *testing.T) { assert.NotNil(t, ins) inm := &indexnode.Mock{} + ParamsGlobal.InitOnce() + etcdCli, err := etcd.GetEtcdClient(&ParamsGlobal.BaseParams) + assert.NoError(t, err) + inm.SetEtcdClient(etcdCli) err = ins.SetClient(inm) assert.Nil(t, err) diff --git a/internal/distributed/indexnode/service.go b/internal/distributed/indexnode/service.go index 69189e0ed9..ef26df876b 100644 --- a/internal/distributed/indexnode/service.go +++ b/internal/distributed/indexnode/service.go @@ -33,10 +33,12 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/paramtable" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -46,7 +48,7 @@ var Params paramtable.GrpcServerConfig // Server is the grpc wrapper of IndexNode. type Server struct { - indexnode types.IndexNode + indexnode types.IndexNodeComponent grpcServer *grpc.Server grpcErrChan chan error @@ -55,7 +57,8 @@ type Server struct { loopCancel func() loopWg sync.WaitGroup - closer io.Closer + etcdCli *clientv3.Client + closer io.Closer } // Run initializes and starts IndexNode's grpc service. @@ -142,6 +145,13 @@ func (s *Server) init() error { 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() if err != nil { log.Error("IndexNode Init failed", zap.Error(err)) @@ -178,6 +188,9 @@ func (s *Server) Stop() error { if s.indexnode != nil { s.indexnode.Stop() } + if s.etcdCli != nil { + defer s.etcdCli.Close() + } if s.grpcServer != nil { log.Debug("Graceful stop grpc server...") s.grpcServer.GracefulStop() @@ -188,11 +201,16 @@ func (s *Server) Stop() error { } // 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 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. func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) { return s.indexnode.GetComponentStates(ctx) diff --git a/internal/distributed/indexnode/service_test.go b/internal/distributed/indexnode/service_test.go index bed22c50e6..c1c26dddce 100644 --- a/internal/distributed/indexnode/service_test.go +++ b/internal/distributed/indexnode/service_test.go @@ -27,9 +27,13 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/indexpb" "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" ) +var ParamsGlobal paramtable.GlobalParamTable + func TestIndexNodeServer(t *testing.T) { ctx := context.Background() server, err := NewServer(ctx) @@ -37,6 +41,10 @@ func TestIndexNodeServer(t *testing.T) { assert.NotNil(t, server) inm := &indexnode.Mock{} + ParamsGlobal.InitOnce() + etcdCli, err := etcd.GetEtcdClient(&ParamsGlobal.BaseParams) + assert.NoError(t, err) + inm.SetEtcdClient(etcdCli) err = server.SetClient(inm) assert.Nil(t, err) diff --git a/internal/distributed/proxy/service.go b/internal/distributed/proxy/service.go index 08f734a6bf..df092e1668 100644 --- a/internal/distributed/proxy/service.go +++ b/internal/distributed/proxy/service.go @@ -38,11 +38,13 @@ import ( "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/proxy" "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/paramtable" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" "github.com/opentracing/opentracing-go" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -59,6 +61,7 @@ type Server struct { grpcErrChan chan error + etcdCli *clientv3.Client rootCoordClient types.RootCoord dataCoordClient types.DataCoord queryCoordClient types.QueryCoord @@ -172,6 +175,13 @@ func (s *Server) init() error { s.closer = closer 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) go s.startGrpcLoop(Params.Port) log.Debug("waiting for grpc server of Proxy to be started") @@ -184,7 +194,7 @@ func (s *Server) init() error { if s.rootCoordClient == nil { var err error 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 { log.Warn("failed to create RootCoord client for Proxy", zap.Error(err)) return err @@ -213,7 +223,7 @@ func (s *Server) init() error { if s.dataCoordClient == nil { var err error 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 { log.Warn("failed to create DataCoord client for Proxy", zap.Error(err)) return err @@ -242,7 +252,7 @@ func (s *Server) init() error { if s.indexCoordClient == nil { var err error 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 { log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err)) return err @@ -271,7 +281,7 @@ func (s *Server) init() error { if s.queryCoordClient == nil { var err error 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 { log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err)) return err @@ -334,6 +344,10 @@ func (s *Server) Stop() error { } } + if s.etcdCli != nil { + defer s.etcdCli.Close() + } + if s.grpcServer != nil { log.Debug("Graceful stop grpc server...") s.grpcServer.GracefulStop() diff --git a/internal/distributed/proxy/service_test.go b/internal/distributed/proxy/service_test.go index 3a4dfb5b9b..67ca1bf40e 100644 --- a/internal/distributed/proxy/service_test.go +++ b/internal/distributed/proxy/service_test.go @@ -32,6 +32,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/types" "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) { return nil, nil } diff --git a/internal/distributed/querycoord/client/client.go b/internal/distributed/querycoord/client/client.go index 11bbc30e54..4c97175ec6 100644 --- a/internal/distributed/querycoord/client/client.go +++ b/internal/distributed/querycoord/client/client.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -43,8 +44,8 @@ type Client struct { } // NewClient creates a client for QueryCoord grpc call. -func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { - sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) +func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) { + sess := sessionutil.NewSession(ctx, metaRoot, etcdCli) if sess == nil { err := fmt.Errorf("new session error, maybe can not connect to etcd") log.Debug("QueryCoordClient NewClient failed", zap.Error(err)) diff --git a/internal/distributed/querycoord/client/client_test.go b/internal/distributed/querycoord/client/client_test.go index a26520d2f4..3fa5739d76 100644 --- a/internal/distributed/querycoord/client/client_test.go +++ b/internal/distributed/querycoord/client/client_test.go @@ -25,6 +25,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/proxy" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) @@ -32,7 +33,10 @@ func Test_NewClient(t *testing.T) { proxy.Params.InitOnce() 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.NotNil(t, client) diff --git a/internal/distributed/querycoord/service.go b/internal/distributed/querycoord/service.go index 7c18dc6104..0baecfa9dd 100644 --- a/internal/distributed/querycoord/service.go +++ b/internal/distributed/querycoord/service.go @@ -36,10 +36,12 @@ import ( "github.com/milvus-io/milvus/internal/proto/querypb" qc "github.com/milvus-io/milvus/internal/querycoord" "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/paramtable" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -60,6 +62,8 @@ type Server struct { msFactory msgstream.Factory + etcdCli *clientv3.Client + dataCoord types.DataCoord rootCoord types.RootCoord indexCoord types.IndexCoord @@ -111,17 +115,25 @@ func (s *Server) init() error { closer := trace.InitTracing("querycoord") 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) go s.startGrpcLoop(Params.Port) // wait for grpc server loop start - err := <-s.grpcErrChan + err = <-s.grpcErrChan if err != nil { return err } // --- Master Server Client --- 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 { log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err)) panic(err) @@ -152,7 +164,7 @@ func (s *Server) init() error { // --- Data service client --- 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 { log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err)) panic(err) @@ -180,7 +192,7 @@ func (s *Server) init() error { // --- IndexCoord --- 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 { log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err)) panic(err) @@ -273,6 +285,9 @@ func (s *Server) Stop() error { return err } } + if s.etcdCli != nil { + defer s.etcdCli.Close() + } err := s.queryCoord.Stop() s.loopCancel() if s.grpcServer != nil { @@ -282,6 +297,11 @@ func (s *Server) Stop() error { 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. func (s *Server) SetRootCoord(m types.RootCoord) error { s.queryCoord.SetRootCoord(m) diff --git a/internal/distributed/querycoord/service_test.go b/internal/distributed/querycoord/service_test.go index 39f5c266da..ab24f6c0e3 100644 --- a/internal/distributed/querycoord/service_test.go +++ b/internal/distributed/querycoord/service_test.go @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/querypb" "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) SetEtcdClient(client *clientv3.Client) { +} + func (m *MockQueryCoord) SetRootCoord(types.RootCoord) error { return nil } diff --git a/internal/distributed/querynode/service.go b/internal/distributed/querynode/service.go index ed44dc5856..9ad34be874 100644 --- a/internal/distributed/querynode/service.go +++ b/internal/distributed/querynode/service.go @@ -36,11 +36,13 @@ import ( "github.com/milvus-io/milvus/internal/proto/querypb" qn "github.com/milvus-io/milvus/internal/querynode" "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/paramtable" "github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -61,6 +63,7 @@ type Server struct { grpcServer *grpc.Server + etcdCli *clientv3.Client rootCoord types.RootCoord indexCoord types.IndexCoord @@ -93,17 +96,26 @@ func (s *Server) init() error { s.closer = closer 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) go s.startGrpcLoop(Params.Port) // wait for grpc server loop start - err := <-s.grpcErrChan + err = <-s.grpcErrChan if err != nil { return err } // --- RootCoord Client --- 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 { log.Debug("QueryNode new RootCoordClient failed", zap.Error(err)) panic(err) @@ -133,7 +145,7 @@ func (s *Server) init() error { // --- IndexCoord --- 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 { log.Debug("QueryNode new IndexCoordClient failed", zap.Error(err)) panic(err) @@ -260,6 +272,9 @@ func (s *Server) Stop() error { return err } } + if s.etcdCli != nil { + defer s.etcdCli.Close() + } s.cancel() if s.grpcServer != nil { @@ -275,6 +290,11 @@ func (s *Server) Stop() error { 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. func (s *Server) SetRootCoord(rootCoord types.RootCoord) error { return s.querynode.SetRootCoord(rootCoord) diff --git a/internal/distributed/querynode/service_test.go b/internal/distributed/querynode/service_test.go index 028ca6c728..6c356f30e2 100644 --- a/internal/distributed/querynode/service_test.go +++ b/internal/distributed/querynode/service_test.go @@ -28,6 +28,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/querypb" "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 } +func (m *MockQueryNode) SetEtcdClient(client *clientv3.Client) { +} + func (m *MockQueryNode) UpdateStateCode(code internalpb.StateCode) { } @@ -149,6 +153,9 @@ func (m *MockRootCoord) Register() error { return m.regErr } +func (m *MockRootCoord) SetEtcdClient(client *clientv3.Client) { +} + func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { return &internalpb.ComponentStates{ State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy}, @@ -182,6 +189,9 @@ func (m *MockIndexCoord) Register() error { return m.regErr } +func (m *MockIndexCoord) SetEtcdClient(client *clientv3.Client) { +} + func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { return &internalpb.ComponentStates{ State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy}, diff --git a/internal/distributed/rootcoord/client/client.go b/internal/distributed/rootcoord/client/client.go index c0177e37e8..4d7d5cc0f2 100644 --- a/internal/distributed/rootcoord/client/client.go +++ b/internal/distributed/rootcoord/client/client.go @@ -32,6 +32,7 @@ import ( "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -49,8 +50,8 @@ type Client struct { // metaRoot is the path in etcd for root coordinator registration // etcdEndpoints are the address list for etcd end points // timeout is default setting for each grpc call -func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) { - sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints) +func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) { + sess := sessionutil.NewSession(ctx, metaRoot, etcdCli) if sess == nil { err := fmt.Errorf("new session error, maybe can not connect to etcd") log.Debug("QueryCoordClient NewClient failed", zap.Error(err)) diff --git a/internal/distributed/rootcoord/client/client_test.go b/internal/distributed/rootcoord/client/client_test.go index 3655234698..2199d2b932 100644 --- a/internal/distributed/rootcoord/client/client_test.go +++ b/internal/distributed/rootcoord/client/client_test.go @@ -25,6 +25,7 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/proxy" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) @@ -32,7 +33,9 @@ func Test_NewClient(t *testing.T) { proxy.Params.InitOnce() 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.NotNil(t, client) diff --git a/internal/distributed/rootcoord/service.go b/internal/distributed/rootcoord/service.go index 68e75d6bb3..5dbc0b7470 100644 --- a/internal/distributed/rootcoord/service.go +++ b/internal/distributed/rootcoord/service.go @@ -25,10 +25,7 @@ import ( "time" 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" - qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/msgstream" "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/rootcoord" "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/paramtable" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" "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 @@ -62,13 +65,14 @@ type Server struct { ctx context.Context cancel context.CancelFunc + etcdCli *clientv3.Client dataCoord types.DataCoord indexCoord types.IndexCoord queryCoord types.QueryCoord - newIndexCoordClient func(string, []string) types.IndexCoord - newDataCoordClient func(string, []string) types.DataCoord - newQueryCoordClient func(string, []string) types.QueryCoord + newIndexCoordClient func(string, *clientv3.Client) types.IndexCoord + newDataCoordClient func(string, *clientv3.Client) types.DataCoord + newQueryCoordClient func(string, *clientv3.Client) types.QueryCoord closer io.Closer } @@ -106,22 +110,22 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) } func (s *Server) setClient() { - s.newDataCoordClient = func(etcdMetaRoot string, etcdEndpoints []string) types.DataCoord { - dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdEndpoints) + s.newDataCoordClient = func(etcdMetaRoot string, etcdCli *clientv3.Client) types.DataCoord { + dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdCli) if err != nil { panic(err) } return dsClient } - s.newIndexCoordClient = func(metaRootPath string, etcdEndpoints []string) types.IndexCoord { - isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdEndpoints) + s.newIndexCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.IndexCoord { + isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdCli) if err != nil { panic(err) } return isClient } - s.newQueryCoordClient = func(metaRootPath string, etcdEndpoints []string) types.QueryCoord { - qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdEndpoints) + s.newQueryCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.QueryCoord { + qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdCli) if err != nil { panic(err) } @@ -149,17 +153,25 @@ func (s *Server) init() error { rootcoord.Params.InitOnce() rootcoord.Params.RootCoordCfg.Address = Params.GetAddress() rootcoord.Params.RootCoordCfg.Port = Params.Port - log.Debug("grpc init done ...") + log.Debug("init params done..") closer := trace.InitTracing("root_coord") 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 { return err } + log.Debug("grpc init done ...") s.rootCoord.UpdateStateCode(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 { 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 { panic(err) } @@ -189,7 +201,7 @@ func (s *Server) init() error { } if s.newIndexCoordClient != nil { 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 { panic(err) } @@ -197,7 +209,7 @@ func (s *Server) init() error { } if s.newQueryCoordClient != nil { 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 { panic(err) } @@ -273,6 +285,9 @@ func (s *Server) Stop() error { log.Error("Failed to close opentracing", zap.Error(err)) } } + if s.etcdCli != nil { + defer s.etcdCli.Close() + } if s.indexCoord != nil { if err := s.indexCoord.Stop(); err != nil { log.Error("Failed to close indexCoord client", zap.Error(err)) diff --git a/internal/distributed/rootcoord/service_test.go b/internal/distributed/rootcoord/service_test.go index 875e07ae5e..63d6b002d3 100644 --- a/internal/distributed/rootcoord/service_test.go +++ b/internal/distributed/rootcoord/service_test.go @@ -41,6 +41,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/rootcoord" "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/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" @@ -98,7 +99,7 @@ func TestGrpcService(t *testing.T) { assert.Nil(t, err) svr.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing) - etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable) assert.Nil(t, err) sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) @@ -115,6 +116,7 @@ func TestGrpcService(t *testing.T) { rootcoord.Params.RootCoordCfg.Address = Params.GetAddress() + core.SetEtcdClient(etcdCli) err = core.Init() assert.Nil(t, err) @@ -214,7 +216,7 @@ func TestGrpcService(t *testing.T) { 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) err = cli.Init() @@ -794,6 +796,9 @@ type mockCore struct { func (m *mockCore) UpdateStateCode(internalpb.StateCode) { } +func (m *mockCore) SetEtcdClient(etcdClient *clientv3.Client) { +} + func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error { return nil } @@ -895,13 +900,13 @@ func TestRun(t *testing.T) { assert.NotNil(t, err) 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{} } - svr.newIndexCoordClient = func(string, []string) types.IndexCoord { + svr.newIndexCoordClient = func(string, *clientv3.Client) types.IndexCoord { return &mockIndex{} } - svr.newQueryCoordClient = func(string, []string) types.QueryCoord { + svr.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord { return &mockQuery{} } @@ -912,7 +917,7 @@ func TestRun(t *testing.T) { rootcoord.Params.Init() 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) sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) diff --git a/internal/indexcoord/index_coord.go b/internal/indexcoord/index_coord.go index 0578b0c11c..9539354340 100644 --- a/internal/indexcoord/index_coord.go +++ b/internal/indexcoord/index_coord.go @@ -50,6 +50,7 @@ import ( "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" ) // make sure IndexCoord implements types.IndexCoord @@ -79,7 +80,8 @@ type IndexCoord struct { idAllocator *allocator.GlobalIDAllocator - kv kv.BaseKV + etcdCli *clientv3.Client + kv kv.BaseKV metaTable *metaTable nodeManager *NodeManager @@ -129,17 +131,19 @@ func (i *IndexCoord) Register() error { log.Fatal("failed to stop server", zap.Error(err)) } // manually send signal to starter goroutine - syscall.Kill(syscall.Getpid(), syscall.SIGINT) + if i.session.TriggerKill { + syscall.Kill(syscall.Getpid(), syscall.SIGINT) + } }) return nil } 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 { 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) return nil } @@ -160,10 +164,7 @@ func (i *IndexCoord) Init() error { } connectEtcdFn := func() error { - etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath) - if err != nil { - return err - } + etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.IndexCoordCfg.MetaRootPath) metakv, err := NewMetaTable(etcdKV) if err != nil { return err @@ -207,12 +208,7 @@ func (i *IndexCoord) Init() error { //init idAllocator kvRootPath := Params.IndexCoordCfg.KvRootPath - etcdKV, err := tsoutil.NewTSOKVBase(Params.IndexCoordCfg.EtcdEndpoints, kvRootPath, "index_gid") - if err != nil { - log.Error("IndexCoord TSOKVBase initialize failed", zap.Error(err)) - initErr = err - return - } + etcdKV := tsoutil.NewTSOKVBase(i.etcdCli, kvRootPath, "index_gid") i.idAllocator = allocator.NewGlobalIDAllocator("idTimestamp", etcdKV) if err := i.idAllocator.Initialize(); err != nil { @@ -306,6 +302,10 @@ func (i *IndexCoord) Stop() error { return nil } +func (i *IndexCoord) SetEtcdClient(etcdClient *clientv3.Client) { + i.etcdCli = etcdClient +} + // UpdateStateCode updates the component state of IndexCoord. func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) { i.stateCode.Store(code) diff --git a/internal/indexcoord/index_coord_mock.go b/internal/indexcoord/index_coord_mock.go index cfa10191e3..c522b41bf9 100644 --- a/internal/indexcoord/index_coord_mock.go +++ b/internal/indexcoord/index_coord_mock.go @@ -28,12 +28,13 @@ import ( "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/util/sessionutil" "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. type Mock struct { - etcdKV *etcdkv.EtcdKV - + etcdKV *etcdkv.EtcdKV + etcdCli *clientv3.Client Failure bool } @@ -67,14 +68,24 @@ func (icm *Mock) Register() error { if icm.Failure { 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) - session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints) - session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true) + if err != nil { + return err + } + session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, icm.etcdCli) + session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true, false) session.Register() 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, // and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`. func (icm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { diff --git a/internal/indexcoord/index_coord_mock_test.go b/internal/indexcoord/index_coord_mock_test.go index 96645a529f..8b77951e3d 100644 --- a/internal/indexcoord/index_coord_mock_test.go +++ b/internal/indexcoord/index_coord_mock_test.go @@ -24,13 +24,19 @@ import ( "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) func TestIndexCoordMock(t *testing.T) { Params.Init() 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) err = icm.Init() assert.Nil(t, err) diff --git a/internal/indexcoord/index_coord_test.go b/internal/indexcoord/index_coord_test.go index 72a52d0796..158993b684 100644 --- a/internal/indexcoord/index_coord_test.go +++ b/internal/indexcoord/index_coord_test.go @@ -32,6 +32,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/sessionutil" "github.com/stretchr/testify/assert" @@ -41,7 +42,11 @@ import ( func TestIndexCoord(t *testing.T) { ctx := context.Background() 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) err = inm0.Register() assert.Nil(t, err) @@ -53,8 +58,8 @@ func TestIndexCoord(t *testing.T) { ic.durationInterval = time.Second ic.assignTaskInterval = 200 * time.Millisecond ic.taskLimit = 20 - Params.Init() + ic.SetEtcdClient(etcdCli) err = ic.Init() assert.Nil(t, err) err = ic.Register() @@ -72,6 +77,8 @@ func TestIndexCoord(t *testing.T) { Build: true, Failure: false, } + + inm.SetEtcdClient(etcdCli) err = in.SetClient(inm) assert.Nil(t, err) diff --git a/internal/indexcoord/meta_table_test.go b/internal/indexcoord/meta_table_test.go index b5d5867d6b..b8af1ce2d8 100644 --- a/internal/indexcoord/meta_table_test.go +++ b/internal/indexcoord/meta_table_test.go @@ -25,13 +25,17 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/schemapb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) func TestMetaTable(t *testing.T) { 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) + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.IndexCoordCfg.MetaRootPath) req := &indexpb.BuildIndexRequest{ IndexBuildID: 1, @@ -310,8 +314,11 @@ func TestMetaTable(t *testing.T) { func TestMetaTable_Error(t *testing.T) { Params.Init() - etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath) - assert.Nil(t, err) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + defer etcdCli.Close() + assert.NoError(t, err) + + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.IndexCoordCfg.MetaRootPath) t.Run("reloadFromKV error", func(t *testing.T) { value := "indexMeta-1" diff --git a/internal/indexcoord/metrics_info_test.go b/internal/indexcoord/metrics_info_test.go index 0ea3cbe6d7..373ff521c0 100644 --- a/internal/indexcoord/metrics_info_test.go +++ b/internal/indexcoord/metrics_info_test.go @@ -21,6 +21,7 @@ import ( "testing" "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/stretchr/testify/assert" ) @@ -31,6 +32,11 @@ func TestGetSystemInfoMetrics(t *testing.T) { assert.Nil(t, err) Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + defer etcdCli.Close() + assert.NoError(t, err) + + ic.SetEtcdClient(etcdCli) err = ic.Init() assert.Nil(t, err) err = ic.Start() diff --git a/internal/indexnode/indexnode.go b/internal/indexnode/indexnode.go index f9bf63c9ed..41e05e561c 100644 --- a/internal/indexnode/indexnode.go +++ b/internal/indexnode/indexnode.go @@ -51,10 +51,10 @@ import ( "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/metricsinfo" "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/trace" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -64,6 +64,9 @@ type UniqueID = typeutil.UniqueID // make sure IndexNode implements types.IndexNode var _ types.IndexNode = (*IndexNode)(nil) +// make sure IndexNode implements types.IndexNodeComponent +var _ types.IndexNodeComponent = (*IndexNode)(nil) + var Params paramtable.GlobalParamTable // IndexNode is a component that executes the task of building indexes. @@ -84,6 +87,7 @@ type IndexNode struct { startCallbacks []func() closeCallbacks []func() + etcdCli *clientv3.Client etcdKV *etcdkv.EtcdKV finishedTasks map[UniqueID]commonpb.IndexState @@ -122,7 +126,9 @@ func (i *IndexNode) Register() error { log.Fatal("failed to stop server", zap.Error(err)) } // manually send signal to starter goroutine - syscall.Kill(syscall.Getpid(), syscall.SIGINT) + if i.session.TriggerKill { + syscall.Kill(syscall.Getpid(), syscall.SIGINT) + } }) return nil } @@ -139,11 +145,11 @@ func (i *IndexNode) initKnowhere() { } 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 { 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.BaseParams.SetLogger(Params.IndexNodeCfg.NodeID) return nil @@ -165,18 +171,8 @@ func (i *IndexNode) Init() error { } log.Debug("IndexNode init session successful", zap.Int64("serverID", i.session.ServerID)) - connectEtcdFn := func() error { - etcdKV, err := etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath) - 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") + etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.IndexNodeCfg.MetaRootPath) + i.etcdKV = etcdKV option := &miniokv.Option{ Address: Params.IndexNodeCfg.MinIOAddress, @@ -250,6 +246,11 @@ func (i *IndexNode) UpdateStateCode(code internalpb.StateCode) { 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 { code := i.stateCode.Load().(internalpb.StateCode) return code == internalpb.StateCode_Healthy diff --git a/internal/indexnode/indexnode_mock.go b/internal/indexnode/indexnode_mock.go index 3e61b19883..2edb39f380 100644 --- a/internal/indexnode/indexnode_mock.go +++ b/internal/indexnode/indexnode_mock.go @@ -34,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/sessionutil" "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. @@ -46,7 +47,8 @@ type Mock struct { cancel context.CancelFunc wg sync.WaitGroup - etcdKV *etcdkv.EtcdKV + etcdCli *clientv3.Client + etcdKV *etcdkv.EtcdKV buildIndex chan *indexpb.CreateIndexRequest } @@ -182,16 +184,25 @@ func (inm *Mock) Register() error { return errors.New("IndexNode register failed") } 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 { return err } - session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints) - session.Init(typeutil.IndexNodeRole, "localhost:21121", false) + session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, inm.etcdCli) + session.Init(typeutil.IndexNodeRole, "localhost:21121", false, false) session.Register() 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, // and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`. func (inm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { diff --git a/internal/indexnode/indexnode_mock_test.go b/internal/indexnode/indexnode_mock_test.go index 9cb312d683..7e17d9dbed 100644 --- a/internal/indexnode/indexnode_mock_test.go +++ b/internal/indexnode/indexnode_mock_test.go @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) @@ -37,7 +38,11 @@ func TestIndexNodeMock(t *testing.T) { inm := Mock{ 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) err = inm.Init() assert.Nil(t, err) @@ -151,8 +156,11 @@ func TestIndexNodeMockFiled(t *testing.T) { Build: true, Err: false, } - - 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) err = inm.Init() assert.Nil(t, err) diff --git a/internal/indexnode/indexnode_test.go b/internal/indexnode/indexnode_test.go index 3c8d7f737b..7466761cc0 100644 --- a/internal/indexnode/indexnode_test.go +++ b/internal/indexnode/indexnode_test.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/proto/milvuspb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/log" "go.uber.org/zap" @@ -71,6 +72,11 @@ func TestIndexNode(t *testing.T) { assert.Nil(t, err) Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + in.SetEtcdClient(etcdCli) + defer etcdCli.Close() + err = in.Init() assert.Nil(t, err) @@ -477,6 +483,11 @@ func TestCreateIndexFailed(t *testing.T) { assert.Nil(t, err) Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + in.SetEtcdClient(etcdCli) + defer etcdCli.Close() + err = in.Init() assert.Nil(t, err) @@ -744,6 +755,11 @@ func TestIndexNode_Error(t *testing.T) { assert.Nil(t, err) Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + in.SetEtcdClient(etcdCli) + defer etcdCli.Close() + err = in.Init() assert.Nil(t, err) diff --git a/internal/kv/etcd/etcd_kv.go b/internal/kv/etcd/etcd_kv.go index f78b76551f..4d4f204f82 100644 --- a/internal/kv/etcd/etcd_kv.go +++ b/internal/kv/etcd/etcd_kv.go @@ -40,34 +40,17 @@ type EtcdKV struct { } // NewEtcdKV creates a new etcd kv. -func NewEtcdKV(etcdEndpoints []string, rootPath string) (*EtcdKV, error) { - client, err := clientv3.New(clientv3.Config{ - Endpoints: etcdEndpoints, - DialTimeout: 5 * time.Second, - }) - if err != nil { - return nil, err - } - +func NewEtcdKV(client *clientv3.Client, rootPath string) *EtcdKV { kv := &EtcdKV{ client: client, rootPath: rootPath, } - - 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, - } + return kv } // Close closes the connection to etcd. func (kv *EtcdKV) Close() { - kv.client.Close() + log.Debug("etcd kv closed", zap.String("path", kv.rootPath)) } // GetPath returns the path of the key. diff --git a/internal/kv/etcd/etcd_kv_test.go b/internal/kv/etcd/etcd_kv_test.go index 6ebfa0560e..76d3639c6a 100644 --- a/internal/kv/etcd/etcd_kv_test.go +++ b/internal/kv/etcd/etcd_kv_test.go @@ -18,18 +18,18 @@ package etcdkv_test import ( "os" - "strings" "testing" "time" 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/stretchr/testify/assert" "github.com/stretchr/testify/require" clientv3 "go.etcd.io/etcd/client/v3" ) -var Params paramtable.BaseTable +var Params paramtable.GlobalParamTable func TestMain(m *testing.M) { Params.Init() @@ -38,17 +38,12 @@ func TestMain(m *testing.M) { } func TestEtcdKV_Load(te *testing.T) { - endpoints, err := Params.Load("_EtcdEndpoints") - if err != nil { - panic(err) - } - - etcdEndPoints := strings.Split(endpoints, ",") - + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + defer etcdCli.Close() + assert.NoError(te, err) te.Run("EtcdKV SaveAndLoad", func(t *testing.T) { rootPath := "/etcd/test/root/saveandload" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - require.NoError(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) err = etcdKV.RemoveWithPrefix("") require.NoError(t, err) @@ -157,8 +152,7 @@ func TestEtcdKV_Load(te *testing.T) { te.Run("EtcdKV LoadWithRevision", func(t *testing.T) { rootPath := "/etcd/test/root/LoadWithRevision" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") @@ -202,8 +196,7 @@ func TestEtcdKV_Load(te *testing.T) { te.Run("EtcdKV MultiSaveAndMultiLoad", func(t *testing.T) { rootPath := "/etcd/test/root/multi_save_and_multi_load" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") @@ -311,9 +304,7 @@ func TestEtcdKV_Load(te *testing.T) { te.Run("EtcdKV MultiRemoveWithPrefix", func(t *testing.T) { rootPath := "/etcd/test/root/multi_remove_with_prefix" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - require.NoError(t, err) - + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") @@ -399,8 +390,7 @@ func TestEtcdKV_Load(te *testing.T) { te.Run("EtcdKV Watch", func(t *testing.T) { rootPath := "/etcd/test/root/watch" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") @@ -416,9 +406,7 @@ func TestEtcdKV_Load(te *testing.T) { te.Run("Etcd Revision", func(t *testing.T) { rootPath := "/etcd/test/root/watch" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - assert.Nil(t, err) - + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") @@ -467,8 +455,7 @@ func TestEtcdKV_Load(te *testing.T) { te.Run("Etcd Lease", func(t *testing.T) { rootPath := "/etcd/test/root/lease" - etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath) - assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") diff --git a/internal/kv/etcd/metakv_factory.go b/internal/kv/etcd/metakv_factory.go index c8c05691f0..ae5ec7226f 100644 --- a/internal/kv/etcd/metakv_factory.go +++ b/internal/kv/etcd/metakv_factory.go @@ -19,6 +19,7 @@ package etcdkv import ( "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/paramtable" "go.etcd.io/etcd/server/v3/embed" "go.uber.org/zap" @@ -49,9 +50,10 @@ func NewMetaKvFactory(rootPath string, param *paramtable.BaseParamTable) (kv.Met } return metaKv, err } - metaKv, err := NewEtcdKV(param.EtcdEndpoints, rootPath) + client, err := etcd.GetEtcdClient(param) if err != nil { return nil, err } + metaKv := NewEtcdKV(client, rootPath) return metaKv, err } diff --git a/internal/msgstream/mq_msgstream_test.go b/internal/msgstream/mq_msgstream_test.go index 441a9ec2c9..18c6d8ab41 100644 --- a/internal/msgstream/mq_msgstream_test.go +++ b/internal/msgstream/mq_msgstream_test.go @@ -38,6 +38,7 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/proto/commonpb" "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/mqclient" "github.com/milvus-io/milvus/internal/util/paramtable" @@ -73,10 +74,12 @@ func (f *fixture) setup() []parameters { endpoints = "localhost:2379" } etcdEndpoints := strings.Split(endpoints, ",") - f.etcdKV, err = etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root") + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + defer etcdCli.Close() if err != nil { log.Fatalf("New clientv3 error = %v", err) } + f.etcdKV = etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root") idAllocator := allocator.NewGlobalIDAllocator("dummy", f.etcdKV) _ = idAllocator.Initialize() err = rocksmq.InitRmq(rocksdbName, idAllocator) @@ -1342,10 +1345,11 @@ func initRmq(name string) *etcdkv.EtcdKV { endpoints = "localhost:2379" } etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root") + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) if err != nil { log.Fatalf("New clientv3 error = %v", err) } + etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root") idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) _ = idAllocator.Initialize() diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index 4a64476a31..2642859998 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -38,6 +38,7 @@ import ( "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -67,6 +68,7 @@ type Proxy struct { stateCode atomic.Value + etcdCli *clientv3.Client rootCoord types.RootCoord indexCoord types.IndexCoord dataCoord types.DataCoord @@ -116,7 +118,9 @@ func (node *Proxy) Register() error { if err := node.Stop(); err != nil { 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 //Params.initLogCfg() @@ -125,11 +129,11 @@ func (node *Proxy) Register() error { // initSession initialize the session of Proxy. 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 { 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.BaseParams.SetLogger(Params.ProxyCfg.ProxyID) return nil @@ -425,6 +429,11 @@ func (node *Proxy) AddCloseCallback(callbacks ...func()) { 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. func (node *Proxy) SetRootCoordClient(cli types.RootCoord) { node.rootCoord = cli diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index 766550355a..4565891041 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -29,6 +29,7 @@ import ( "testing" "time" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/sessionutil" "go.uber.org/zap" @@ -211,13 +212,9 @@ func runDataCoord(ctx context.Context, localMsg bool) *grpcdatacoordclient.Serve } factory := newMsgFactory(localMsg) - var err error - ds, err = grpcdatacoordclient.NewServer(ctx, factory) - if err != nil { - panic(err) - } + ds = grpcdatacoordclient.NewServer(ctx, factory) wg.Done() - err = ds.Run() + err := ds.Run() if err != nil { panic(err) } @@ -310,6 +307,11 @@ func runIndexNode(ctx context.Context, localMsg bool, alias string) *grpcindexno panic(err) } wg.Done() + etcd, err := etcd.GetEtcdClient(&indexnode.Params.BaseParams) + if err != nil { + panic(err) + } + in.SetEtcdClient(etcd) err = in.Run() if err != nil { panic(err) @@ -419,7 +421,11 @@ func TestProxy(t *testing.T) { Params.Init() 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) err = rootCoordClient.Init() assert.NoError(t, err) @@ -428,7 +434,7 @@ func TestProxy(t *testing.T) { proxy.SetRootCoordClient(rootCoordClient) 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) err = dataCoordClient.Init() assert.NoError(t, err) @@ -437,7 +443,7 @@ func TestProxy(t *testing.T) { proxy.SetDataCoordClient(dataCoordClient) 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) err = queryCoordClient.Init() assert.NoError(t, err) @@ -446,7 +452,7 @@ func TestProxy(t *testing.T) { proxy.SetQueryCoordClient(queryCoordClient) 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) err = indexCoordClient.Init() assert.NoError(t, err) @@ -456,7 +462,6 @@ func TestProxy(t *testing.T) { log.Info("Proxy set index coordinator client") proxy.UpdateStateCode(internalpb.StateCode_Initializing) - err = proxy.Init() assert.NoError(t, err) diff --git a/internal/querycoord/channel_allocator_test.go b/internal/querycoord/channel_allocator_test.go index 98b049ebfd..2b546888d4 100644 --- a/internal/querycoord/channel_allocator_test.go +++ b/internal/querycoord/channel_allocator_test.go @@ -25,6 +25,7 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/proto/datapb" "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/typeutil" ) @@ -32,10 +33,13 @@ import ( func TestShuffleChannelsToQueryNode(t *testing.T) { refreshParams() 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) - clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) - 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() meta, err := newMeta(baseCtx, kv, nil, nil) assert.Nil(t, err) diff --git a/internal/querycoord/cluster_test.go b/internal/querycoord/cluster_test.go index 983de36bb8..d4eabfaa44 100644 --- a/internal/querycoord/cluster_test.go +++ b/internal/querycoord/cluster_test.go @@ -41,6 +41,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/schemapb" "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/typeutil" ) @@ -390,13 +391,15 @@ func TestQueryNodeCluster_getMetrics(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) { refreshParams() baseCtx := context.Background() - kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) - assert.Nil(t, err) - clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) - 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() cluster := &queryNodeCluster{ ctx: baseCtx, @@ -422,10 +425,9 @@ func TestReloadClusterFromKV(t *testing.T) { t.Run("Test LoadOfflineNodes", func(t *testing.T) { refreshParams() - kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) - assert.Nil(t, err) - clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) - 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() cluster := &queryNodeCluster{ client: kv, @@ -459,10 +461,12 @@ func TestReloadClusterFromKV(t *testing.T) { func TestGrpcRequest(t *testing.T) { refreshParams() 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) - clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) - clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true) + defer etcdCli.Close() + 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() factory := msgstream.NewPmsFactory() m := map[string]interface{}{ diff --git a/internal/querycoord/index_checker_test.go b/internal/querycoord/index_checker_test.go index 60e7ee3a3b..de5df2f2ca 100644 --- a/internal/querycoord/index_checker_test.go +++ b/internal/querycoord/index_checker_test.go @@ -28,14 +28,17 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/tsoutil" ) func TestReloadFromKV(t *testing.T) { refreshParams() 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) + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) meta, err := newMeta(baseCtx, kv, nil, nil) assert.Nil(t, err) @@ -88,8 +91,10 @@ func TestReloadFromKV(t *testing.T) { func TestCheckIndexLoop(t *testing.T) { refreshParams() 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) + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) meta, err := newMeta(ctx, kv, nil, nil) assert.Nil(t, err) @@ -152,8 +157,11 @@ func TestCheckIndexLoop(t *testing.T) { func TestProcessHandoffAfterIndexDone(t *testing.T) { refreshParams() 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) + defer etcdCli.Close() + + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) meta, err := newMeta(ctx, kv, nil, nil) assert.Nil(t, err) taskScheduler := &TaskScheduler{ @@ -162,8 +170,7 @@ func TestProcessHandoffAfterIndexDone(t *testing.T) { client: kv, triggerTaskQueue: NewTaskQueue(), } - idAllocatorKV, err := tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID") - assert.Nil(t, err) + idAllocatorKV := tsoutil.NewTSOKVBase(etcdCli, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID") idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV) err = idAllocator.Initialize() assert.Nil(t, err) diff --git a/internal/querycoord/meta_test.go b/internal/querycoord/meta_test.go index 9c0c9271d6..6bc911051d 100644 --- a/internal/querycoord/meta_test.go +++ b/internal/querycoord/meta_test.go @@ -31,6 +31,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util" + "github.com/milvus-io/milvus/internal/util/etcd" ) func successResult() error { return nil } @@ -63,8 +64,10 @@ func (tk *testKv) Load(key string) (string, error) { func TestReplica_Release(t *testing.T) { refreshParams() - etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) + defer etcdCli.Close() + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) meta, err := newMeta(context.Background(), etcdKV, nil, nil) assert.Nil(t, err) err = meta.addCollection(1, querypb.LoadType_loadCollection, nil) @@ -93,8 +96,10 @@ func TestReplica_Release(t *testing.T) { func TestMetaFunc(t *testing.T) { 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) nodeID := defaultQueryNodeID segmentInfos := make(map[UniqueID]*querypb.SegmentInfo) @@ -285,8 +290,10 @@ func TestMetaFunc(t *testing.T) { func TestReloadMetaFromKV(t *testing.T) { 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) meta := &MetaReplica{ client: kv, collectionInfos: map[UniqueID]*querypb.CollectionInfo{}, diff --git a/internal/querycoord/mock_querynode_server_test.go b/internal/querycoord/mock_querynode_server_test.go index 8a3a4b1039..c4196c9cd4 100644 --- a/internal/querycoord/mock_querynode_server_test.go +++ b/internal/querycoord/mock_querynode_server_test.go @@ -31,6 +31,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "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/metricsinfo" "github.com/milvus-io/milvus/internal/util/retry" @@ -100,9 +101,13 @@ func newQueryNodeServerMock(ctx context.Context) *queryNodeServerMock { } func (qs *queryNodeServerMock) Register() error { - log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints)) - qs.session = sessionutil.NewSession(qs.ctx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) - qs.session.Init(typeutil.QueryNodeRole, qs.queryNodeIP+":"+strconv.FormatInt(qs.queryNodePort, 10), false) + log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath)) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + 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 log.Debug("query nodeID", zap.Int64("nodeID", qs.queryNodeID)) log.Debug("query node address", zap.String("address", qs.session.Address)) diff --git a/internal/querycoord/query_coord.go b/internal/querycoord/query_coord.go index 36f3447691..3f147727fb 100644 --- a/internal/querycoord/query_coord.go +++ b/internal/querycoord/query_coord.go @@ -40,11 +40,11 @@ import ( "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/metricsinfo" "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/tsoutil" "github.com/milvus-io/milvus/internal/util/typeutil" "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -84,6 +84,7 @@ type QueryCoord struct { metricsCacheManager *metricsinfo.MetricsCacheManager + etcdCli *clientv3.Client dataCoordClient types.DataCoord rootCoordClient types.RootCoord indexCoordClient types.IndexCoord @@ -106,17 +107,19 @@ func (qc *QueryCoord) Register() error { log.Fatal("failed to stop server", zap.Error(err)) } // manually send signal to starter goroutine - syscall.Kill(syscall.Getpid(), syscall.SIGINT) + if qc.session.TriggerKill { + syscall.Kill(syscall.Getpid(), syscall.SIGINT) + } }) return nil } 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 { 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.BaseParams.SetLogger(qc.session.ServerID) return nil @@ -124,17 +127,7 @@ func (qc *QueryCoord) initSession() error { // Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler func (qc *QueryCoord) Init() error { - log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), - 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 - } + log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.String("address", Params.QueryCoordCfg.Address)) var initError error qc.initOnce.Do(func() { err := qc.initSession() @@ -144,20 +137,12 @@ func (qc *QueryCoord) Init() error { return } log.Debug("queryCoord try to connect etcd") - initError = retry.Do(qc.loopCtx, connectEtcdFn, retry.Attempts(300)) - if initError != nil { - log.Debug("query coordinator try to connect etcd failed", zap.Error(initError)) - return - } + etcdKV := etcdkv.NewEtcdKV(qc.etcdCli, Params.QueryCoordCfg.MetaRootPath) + qc.kvClient = etcdKV log.Debug("query coordinator try to connect etcd success") // init id allocator - var idAllocatorKV *etcdkv.EtcdKV - 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 - } + idAllocatorKV := tsoutil.NewTSOKVBase(qc.etcdCli, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID") idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV) initError = idAllocator.Initialize() if initError != nil { @@ -294,6 +279,11 @@ func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord, 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 func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error { if rootCoord == nil { diff --git a/internal/querycoord/query_coord_test.go b/internal/querycoord/query_coord_test.go index 86fa4b10ba..04830b3c52 100644 --- a/internal/querycoord/query_coord_test.go +++ b/internal/querycoord/query_coord_test.go @@ -34,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/msgstream" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/sessionutil" "go.uber.org/zap" ) @@ -91,7 +92,11 @@ func startQueryCoord(ctx context.Context) (*QueryCoord, error) { coord.SetRootCoord(rootCoord) coord.SetDataCoord(dataCoord) coord.SetIndexCoord(indexCoord) - + etcd, err := etcd.GetEtcdClient(&Params.BaseParams) + if err != nil { + return nil, err + } + coord.SetEtcdClient(etcd) err = coord.Init() if err != nil { return nil, err @@ -132,7 +137,11 @@ func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) { coord.SetRootCoord(rootCoord) coord.SetDataCoord(dataCoord) - + etcd, err := etcd.GetEtcdClient(&Params.BaseParams) + if err != nil { + return nil, err + } + coord.SetEtcdClient(etcd) err = coord.Init() if err != nil { return nil, err @@ -147,11 +156,12 @@ func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) { func TestWatchNodeLoop(t *testing.T) { baseCtx := context.Background() - + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.Nil(t, err) t.Run("Test OfflineNodes", func(t *testing.T) { 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) session := &sessionutil.Session{ diff --git a/internal/querycoord/querynode_test.go b/internal/querycoord/querynode_test.go index b7e534701f..2458f64376 100644 --- a/internal/querycoord/querynode_test.go +++ b/internal/querycoord/querynode_test.go @@ -30,16 +30,20 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/querypb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/typeutil" ) //func waitQueryNodeOnline(cluster *queryNodeCluster, nodeID int64) 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 { return err } + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) + err = kv.Remove(fmt.Sprintf("session/"+typeutil.QueryNodeRole+"-%d", id)) if err != nil { return err @@ -48,10 +52,12 @@ func removeNodeSession(id int64) 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 { return err } + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) err = kv.RemoveWithPrefix("session") if err != nil { return err @@ -187,8 +193,10 @@ func TestQueryNode_getMetrics(t *testing.T) { func TestNewQueryNode(t *testing.T) { refreshParams() 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) + defer etcdCli.Close() + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) queryNode1, err := startQueryNodeServer(baseCtx) assert.Nil(t, err) @@ -211,8 +219,10 @@ func TestNewQueryNode(t *testing.T) { func TestReleaseCollectionOnOfflineNode(t *testing.T) { refreshParams() 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) + defer etcdCli.Close() + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) node, err := newQueryNode(baseCtx, "test", 100, kv) assert.Nil(t, err) @@ -280,8 +290,10 @@ func TestSealedSegmentChangeAfterQueryNodeStop(t *testing.T) { func TestGrpcRequestWithNodeOffline(t *testing.T) { refreshParams() 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) + defer etcdCli.Close() + kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath) nodeServer, err := startQueryNodeServer(baseCtx) assert.Nil(t, err) address := nodeServer.queryNodeIP diff --git a/internal/querycoord/segment_allocator_test.go b/internal/querycoord/segment_allocator_test.go index 63a07dbf7d..d02dd842ab 100644 --- a/internal/querycoord/segment_allocator_test.go +++ b/internal/querycoord/segment_allocator_test.go @@ -25,6 +25,7 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" minioKV "github.com/milvus-io/milvus/internal/kv/minio" "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/typeutil" ) @@ -32,10 +33,12 @@ import ( func TestShuffleSegmentsToQueryNode(t *testing.T) { refreshParams() 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) - clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints) - 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) meta, err := newMeta(baseCtx, kv, nil, nil) assert.Nil(t, err) cluster := &queryNodeCluster{ diff --git a/internal/querycoord/task_scheduler_test.go b/internal/querycoord/task_scheduler_test.go index ab3c14d552..41e3cccba1 100644 --- a/internal/querycoord/task_scheduler_test.go +++ b/internal/querycoord/task_scheduler_test.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "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/stretchr/testify/assert" ) @@ -208,8 +209,10 @@ func TestWatchQueryChannel_ClearEtcdInfoAfterAssignedNodeDown(t *testing.T) { func TestUnMarshalTask(t *testing.T) { 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) baseCtx, cancel := context.WithCancel(context.Background()) taskScheduler := &TaskScheduler{ ctx: baseCtx, @@ -454,7 +457,10 @@ func TestUnMarshalTask(t *testing.T) { func TestReloadTaskFromKV(t *testing.T) { 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) baseCtx, cancel := context.WithCancel(context.Background()) taskScheduler := &TaskScheduler{ diff --git a/internal/querynode/impl_test.go b/internal/querynode/impl_test.go index 5f6fea1c34..56ca1fba5f 100644 --- a/internal/querynode/impl_test.go +++ b/internal/querynode/impl_test.go @@ -20,6 +20,7 @@ import ( "context" "encoding/json" "math/rand" + "sync" "sync/atomic" "testing" @@ -29,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" 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/sessionutil" ) @@ -228,7 +230,10 @@ func TestImpl_GetSegmentInfo(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + var wg sync.WaitGroup + wg.Add(1) t.Run("test GetSegmentInfo", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -251,7 +256,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test no collection in historical", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -272,7 +279,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test no collection in streaming", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -293,7 +302,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test different segment type", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -335,7 +346,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test GetSegmentInfo with indexed segment", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -367,7 +380,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test GetSegmentInfo without streaming partition", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -386,7 +401,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test GetSegmentInfo without streaming segment", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -405,7 +422,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test GetSegmentInfo without historical partition", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -424,7 +443,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) }) + wg.Add(1) t.Run("test GetSegmentInfo without historical segment", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -442,6 +463,7 @@ func TestImpl_GetSegmentInfo(t *testing.T) { assert.NoError(t, err) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode) }) + wg.Wait() } func TestImpl_isHealthy(t *testing.T) { @@ -458,11 +480,17 @@ func TestImpl_GetMetrics(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) 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) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) - - node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints) + node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, etcdCli) metricReq := make(map[string]string) metricReq[metricsinfo.MetricTypeKey] = "system_info" @@ -481,7 +509,9 @@ func TestImpl_GetMetrics(t *testing.T) { assert.NoError(t, err) }) + wg.Add(1) t.Run("test ParseMetricType failed", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -499,13 +529,17 @@ func TestImpl_GetMetrics(t *testing.T) { _, err = node.GetMetrics(ctx, req) assert.NoError(t, err) }) + wg.Wait() } func TestImpl_ReleaseSegments(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + var wg sync.WaitGroup + wg.Add(1) t.Run("test valid", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -520,7 +554,9 @@ func TestImpl_ReleaseSegments(t *testing.T) { assert.NoError(t, err) }) + wg.Add(1) t.Run("test invalid query node", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -536,7 +572,9 @@ func TestImpl_ReleaseSegments(t *testing.T) { assert.NoError(t, err) }) + wg.Add(1) t.Run("test segment not exists", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) @@ -557,4 +595,5 @@ func TestImpl_ReleaseSegments(t *testing.T) { assert.NoError(t, err) assert.NotEqual(t, commonpb.ErrorCode_Success, status.ErrorCode) }) + wg.Wait() } diff --git a/internal/querynode/metrics_info_test.go b/internal/querynode/metrics_info_test.go index d16681f6ec..464b3883a0 100644 --- a/internal/querynode/metrics_info_test.go +++ b/internal/querynode/metrics_info_test.go @@ -24,6 +24,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/sessionutil" ) @@ -34,7 +35,10 @@ func TestGetSystemInfoMetrics(t *testing.T) { node, err := genSimpleQueryNode(ctx) 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{ Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels), diff --git a/internal/querynode/mock_test.go b/internal/querynode/mock_test.go index 580c230c1c..1038d87e36 100644 --- a/internal/querynode/mock_test.go +++ b/internal/querynode/mock_test.go @@ -42,6 +42,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util" + "github.com/milvus-io/milvus/internal/util/etcd" "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) { - etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) - return etcdKV, err + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + if err != nil { + return nil, err + } + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath) + return etcdKV, nil } func genFactory() (msgstream.Factory, error) { @@ -1275,14 +1280,12 @@ func genSimpleChangeInfo() *querypb.SealedSegmentsChangeInfo { func saveChangeInfo(key string, value string) error { log.Debug(".. [query node unittest] Saving change info") - kv, err := genEtcdKV() if err != nil { return err } key = util.ChangeInfoMetaPrefix + "/" + key - return kv.Save(key, value) } @@ -1293,16 +1296,17 @@ func genSimpleQueryNode(ctx context.Context) (*QueryNode, error) { return nil, err } node := NewQueryNode(ctx, fac) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + if err != nil { + return nil, err + } + node.etcdCli = etcdCli session := &sessionutil.Session{ ServerID: 1, } node.session = session - etcdKV, err := genEtcdKV() - if err != nil { - return nil, err - } - + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath) node.etcdKV = etcdKV node.tSafeReplica = newTSafeReplica() diff --git a/internal/querynode/query_collection_test.go b/internal/querynode/query_collection_test.go index 92638e929e..d9c0a8a3d0 100644 --- a/internal/querynode/query_collection_test.go +++ b/internal/querynode/query_collection_test.go @@ -40,6 +40,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/schemapb" "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/typeutil" ) @@ -131,8 +132,10 @@ func TestQueryCollection_withoutVChannel(t *testing.T) { factory := msgstream.NewPmsFactory() err := factory.SetParams(m) assert.Nil(t, err) - etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) + defer etcdCli.Close() + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath) schema := genTestCollectionSchema(0, false, 2) historicalReplica := newCollectionReplica(etcdKV) @@ -689,8 +692,10 @@ func TestQueryCollection_AddPopUnsolvedMsg(t *testing.T) { func TestQueryCollection_adjustByChangeInfo(t *testing.T) { ctx, cancel := context.WithCancel(context.TODO()) - + var wg sync.WaitGroup + wg.Add(1) t.Run("test adjustByChangeInfo", func(t *testing.T) { + defer wg.Done() qc, err := genSimpleQueryCollection(ctx, cancel) assert.Nil(t, err) @@ -710,7 +715,9 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) { assert.Len(t, ids, 0) }) + wg.Add(1) t.Run("test mismatch collectionID when adjustByChangeInfo", func(t *testing.T) { + defer wg.Done() qc, err := genSimpleQueryCollection(ctx, cancel) assert.Nil(t, err) @@ -723,7 +730,9 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) { qc.adjustByChangeInfo(segmentChangeInfos) }) + wg.Add(1) t.Run("test no segment when adjustByChangeInfo", func(t *testing.T) { + defer wg.Done() qc, err := genSimpleQueryCollection(ctx, cancel) assert.Nil(t, err) @@ -735,13 +744,17 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) { qc.adjustByChangeInfo(segmentChangeInfos) }) + wg.Wait() } func TestQueryCollection_search_while_release(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + var wgAll sync.WaitGroup + wgAll.Add(1) t.Run("test search while release collection", func(t *testing.T) { + defer wgAll.Done() queryCollection, err := genSimpleQueryCollection(ctx, cancel) assert.NoError(t, err) @@ -778,7 +791,9 @@ func TestQueryCollection_search_while_release(t *testing.T) { wg.Wait() }) + wgAll.Add(1) t.Run("test search while release partition", func(t *testing.T) { + defer wgAll.Done() queryCollection, err := genSimpleQueryCollection(ctx, cancel) assert.NoError(t, err) @@ -814,4 +829,5 @@ func TestQueryCollection_search_while_release(t *testing.T) { } wg.Wait() }) + wgAll.Wait() } diff --git a/internal/querynode/query_node.go b/internal/querynode/query_node.go index 4d2dc9e91d..333df920ed 100644 --- a/internal/querynode/query_node.go +++ b/internal/querynode/query_node.go @@ -55,6 +55,7 @@ import ( "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -99,6 +100,9 @@ type QueryNode struct { // segment loader loader *segmentLoader + // etcd client + etcdCli *clientv3.Client + // clients rootCoord types.RootCoord indexCoord types.IndexCoord @@ -129,11 +133,11 @@ func NewQueryNode(ctx context.Context, factory msgstream.Factory) *QueryNode { } 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 { 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.BaseParams.SetLogger(Params.QueryNodeCfg.QueryNodeID) 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)) } // 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 @@ -179,7 +185,7 @@ func (node *QueryNode) Init() error { var initError error = nil node.initOnce.Do(func() { //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() if err != nil { log.Error("QueryNode init session failed", zap.Error(err)) @@ -187,28 +193,9 @@ func (node *QueryNode) Init() error { return } Params.QueryNodeCfg.Refresh() - connectEtcdFn := func() error { - etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) - if err != nil { - 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.etcdKV = etcdkv.NewEtcdKV(node.etcdCli, Params.QueryNodeCfg.MetaRootPath) + log.Debug("queryNode try to connect etcd success", zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath)) node.tSafeReplica = newTSafeReplica() streamingReplica := newCollectionReplica(node.etcdKV) @@ -328,6 +315,11 @@ func (node *QueryNode) UpdateStateCode(code internalpb.StateCode) { 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. func (node *QueryNode) SetRootCoord(rc types.RootCoord) error { if rc == nil { diff --git a/internal/querynode/query_node_test.go b/internal/querynode/query_node_test.go index dcd81e12bb..29c8027053 100644 --- a/internal/querynode/query_node_test.go +++ b/internal/querynode/query_node_test.go @@ -21,6 +21,7 @@ import ( "math/rand" "os" "strconv" + "sync" "testing" "time" @@ -34,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/etcd" ) // mock of query coordinator client @@ -186,11 +188,11 @@ func newQueryNodeMock() *QueryNode { cancel() }() } - - etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) if err != nil { panic(err) } + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath) msFactory, err := newMessageStreamFactory() if err != nil { @@ -270,9 +272,14 @@ func TestQueryNode_register(t *testing.T) { node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) + etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + defer etcdcli.Close() + node.SetEtcdClient(etcdcli) err = node.initSession() assert.NoError(t, err) + node.session.TriggerKill = false err = node.Register() assert.NoError(t, err) } @@ -283,7 +290,10 @@ func TestQueryNode_init(t *testing.T) { node, err := genSimpleQueryNode(ctx) assert.NoError(t, err) - + etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + defer etcdcli.Close() + node.SetEtcdClient(etcdcli) err = node.Init() assert.Error(t, err) } @@ -322,7 +332,10 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + var wg sync.WaitGroup + wg.Add(1) t.Run("test cleanup segments", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) assert.NoError(t, err) @@ -330,7 +343,9 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) { assert.NoError(t, err) }) + wg.Add(1) t.Run("test cleanup segments no segment", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) assert.NoError(t, err) @@ -348,13 +363,16 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) { err = node.removeSegments(segmentChangeInfos) assert.Error(t, err) }) + wg.Wait() } func TestQueryNode_watchChangeInfo(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - + var wg sync.WaitGroup + wg.Add(1) t.Run("test watchChangeInfo", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) assert.NoError(t, err) @@ -369,7 +387,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) { time.Sleep(100 * time.Millisecond) }) + wg.Add(1) t.Run("test watchChangeInfo key error", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) assert.NoError(t, err) @@ -381,7 +401,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) { time.Sleep(100 * time.Millisecond) }) + wg.Add(1) t.Run("test watchChangeInfo unmarshal error", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) assert.NoError(t, err) @@ -393,7 +415,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) { time.Sleep(100 * time.Millisecond) }) + wg.Add(1) t.Run("test watchChangeInfo adjustByChangeInfo error", func(t *testing.T) { + defer wg.Done() node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx) assert.NoError(t, err) @@ -417,4 +441,5 @@ func TestQueryNode_watchChangeInfo(t *testing.T) { time.Sleep(100 * time.Millisecond) }) + wg.Wait() } diff --git a/internal/rootcoord/dml_channels_test.go b/internal/rootcoord/dml_channels_test.go index ba4542b5d3..e37162aaf7 100644 --- a/internal/rootcoord/dml_channels_test.go +++ b/internal/rootcoord/dml_channels_test.go @@ -19,6 +19,7 @@ package rootcoord import ( "context" "errors" + "sync" "testing" "github.com/milvus-io/milvus/internal/msgstream" @@ -82,12 +83,17 @@ func TestDmlChannels(t *testing.T) { } func TestDmChannelsFailure(t *testing.T) { + var wg sync.WaitGroup + wg.Add(1) t.Run("Test newDmlChannels", func(t *testing.T) { + defer wg.Done() mockFactory := &FailMessageStreamFactory{} assert.Panics(t, func() { newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) }) }) + wg.Add(1) t.Run("Test broadcast", func(t *testing.T) { + defer wg.Done() mockFactory := &FailMessageStreamFactory{errBroadcast: true} dml := newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) chanName0 := dml.getChannelName() @@ -101,6 +107,7 @@ func TestDmChannelsFailure(t *testing.T) { assert.Empty(t, v) assert.Error(t, err) }) + wg.Wait() } // FailMessageStreamFactory mock MessageStreamFactory failure diff --git a/internal/rootcoord/meta_snapshot_test.go b/internal/rootcoord/meta_snapshot_test.go index 3132841464..07ae180210 100644 --- a/internal/rootcoord/meta_snapshot_test.go +++ b/internal/rootcoord/meta_snapshot_test.go @@ -24,10 +24,9 @@ import ( "testing" "time" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/typeutil" "github.com/stretchr/testify/assert" - - clientv3 "go.etcd.io/etcd/client/v3" ) func TestMetaSnapshot(t *testing.T) { @@ -38,7 +37,7 @@ func TestMetaSnapshot(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) tsKey := "timestamp" - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) defer etcdCli.Close() @@ -178,7 +177,7 @@ func TestGetRevOnEtcd(t *testing.T) { tsKey := "timestamp" 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) defer etcdCli.Close() @@ -222,7 +221,7 @@ func TestLoad(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) tsKey := "timestamp" - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) defer etcdCli.Close() @@ -270,7 +269,7 @@ func TestMultiSave(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) tsKey := "timestamp" - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) defer etcdCli.Close() @@ -334,8 +333,9 @@ func TestMultiSaveAndRemoveWithPrefix(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) tsKey := "timestamp" - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) + defer etcdCli.Close() var vtso typeutil.Timestamp ftso := func() typeutil.Timestamp { @@ -411,7 +411,7 @@ func TestTsBackward(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) tsKey := "timestamp" - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) defer etcdCli.Close() @@ -438,7 +438,7 @@ func TestFix7150(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) tsKey := "timestamp" - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) defer etcdCli.Close() diff --git a/internal/rootcoord/meta_table_test.go b/internal/rootcoord/meta_table_test.go index 479a7c07d5..c6702ec60f 100644 --- a/internal/rootcoord/meta_table_test.go +++ b/internal/rootcoord/meta_table_test.go @@ -21,6 +21,7 @@ import ( "fmt" "math/rand" "path" + "sync" "testing" "time" @@ -31,9 +32,10 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "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/stretchr/testify/assert" - clientv3 "go.etcd.io/etcd/client/v3" + "github.com/stretchr/testify/require" ) type mockTestKV struct { @@ -220,13 +222,14 @@ func TestMetaTable(t *testing.T) { return vtso } - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) - assert.Nil(t, err) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + require.Nil(t, err) defer etcdCli.Close() + skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7) assert.Nil(t, err) assert.NotNil(t, skv) - txnKV := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath) + txnKV := etcdkv.NewEtcdKV(etcdCli, rootPath) mt, err := NewMetaTable(txnKV, skv) 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) { + defer wg.Done() ts := ftso() err = mt.AddCollection(collInfo, ts, nil, "") assert.NotNil(t, err) @@ -321,7 +327,9 @@ func TestMetaTable(t *testing.T) { assert.Equal(t, "false", flag) }) + wg.Add(1) t.Run("add alias", func(t *testing.T) { + defer wg.Done() ts := ftso() exists := mt.IsAlias(aliasName1) assert.False(t, exists) @@ -332,8 +340,9 @@ func TestMetaTable(t *testing.T) { exists = mt.IsAlias(aliasName1) assert.True(t, exists) }) - + wg.Add(1) t.Run("alter alias", func(t *testing.T) { + defer wg.Done() ts := ftso() err = mt.AlterAlias(aliasName1, collName, ts) assert.Nil(t, err) @@ -341,13 +350,17 @@ func TestMetaTable(t *testing.T) { assert.NotNil(t, err) }) + wg.Add(1) t.Run("delete alias", func(t *testing.T) { + defer wg.Done() ts := ftso() err = mt.DropAlias(aliasName1, ts) assert.Nil(t, err) }) + wg.Add(1) t.Run("add partition", func(t *testing.T) { + defer wg.Done() ts := ftso() err = mt.AddPartition(collID, partName, partID, ts, "") assert.Nil(t, err) @@ -365,7 +378,9 @@ func TestMetaTable(t *testing.T) { assert.Equal(t, "false", flag) }) + wg.Add(1) t.Run("add segment index", func(t *testing.T) { + defer wg.Done() segIdxInfo := pb.SegmentIndexInfo{ CollectionID: collID, PartitionID: partID, @@ -387,7 +402,9 @@ func TestMetaTable(t *testing.T) { 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) { + defer wg.Done() params := []*commonpb.KeyValuePair{ { 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) { + defer wg.Done() _, idx, err := mt.GetIndexByName(collName, "field110") assert.Nil(t, err) assert.Equal(t, 1, len(idx)) @@ -464,7 +483,9 @@ func TestMetaTable(t *testing.T) { assert.Zero(t, len(idx)) }) + wg.Add(1) t.Run("reload meta", func(t *testing.T) { + defer wg.Done() te := pb.TenantMeta{ ID: 100, } @@ -480,7 +501,9 @@ func TestMetaTable(t *testing.T) { assert.Nil(t, err) }) + wg.Add(1) t.Run("drop index", func(t *testing.T) { + defer wg.Done() idx, ok, err := mt.DropIndex(collName, "field110", "field110") assert.Nil(t, err) assert.True(t, ok) @@ -503,7 +526,9 @@ func TestMetaTable(t *testing.T) { assert.NotNil(t, err) }) + wg.Add(1) t.Run("drop partition", func(t *testing.T) { + defer wg.Done() ts := ftso() id, err := mt.DeletePartition(collID, partName, ts, "") assert.Nil(t, err) @@ -515,7 +540,9 @@ func TestMetaTable(t *testing.T) { assert.Equal(t, "false", flag) }) + wg.Add(1) t.Run("drop collection", func(t *testing.T) { + defer wg.Done() ts := ftso() err = mt.DeleteCollection(collIDInvalid, ts, "") 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 //////////////////////// - txnkv := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath) + txnkv := etcdkv.NewEtcdKV(etcdCli, rootPath) mockKV := &mockTestKV{} mt.snapshot = mockKV mockTxnKV := &mockTestTxnKV{ @@ -549,7 +576,9 @@ func TestMetaTable(t *testing.T) { } mt.txn = mockTxnKV + wg.Add(1) t.Run("add collection failed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { return nil, nil, nil } @@ -562,7 +591,9 @@ func TestMetaTable(t *testing.T) { assert.Panics(t, func() { mt.AddCollection(collInfo, 0, idxInfo, "") }) }) + wg.Add(1) t.Run("delete collection failed", func(t *testing.T) { + defer wg.Done() mockKV.multiSave = func(kvs map[string]string, ts typeutil.Timestamp) error { return nil } @@ -573,7 +604,9 @@ func TestMetaTable(t *testing.T) { assert.Panics(t, func() { mt.DeleteCollection(collInfo.ID, ts, "") }) }) + wg.Add(1) t.Run("get collection failed", func(t *testing.T) { + defer wg.Done() mockKV.save = func(key string, value string, ts typeutil.Timestamp) error { return nil } @@ -592,7 +625,9 @@ func TestMetaTable(t *testing.T) { }) + wg.Add(1) t.Run("add partition failed", func(t *testing.T) { + defer wg.Done() mockKV.save = func(key string, value string, ts typeutil.Timestamp) error { return nil } @@ -653,7 +688,9 @@ func TestMetaTable(t *testing.T) { assert.EqualError(t, err, fmt.Sprintf("partition id = %d already exists", partID)) }) + wg.Add(1) t.Run("has partition failed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { return nil, nil, nil } @@ -676,7 +713,9 @@ func TestMetaTable(t *testing.T) { assert.False(t, mt.HasPartition(collInfo.ID, partName, 0)) }) + wg.Add(1) t.Run("delete partition failed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { 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)) }) + wg.Add(1) t.Run("add index failed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { return nil, nil, nil } @@ -768,7 +809,9 @@ func TestMetaTable(t *testing.T) { assert.Panics(t, func() { mt.AddIndex(&segIdxInfo) }) }) + wg.Add(1) t.Run("drop index failed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { 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) }) }) + wg.Add(1) 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) { 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)) }) + wg.Add(1) t.Run("get field schema failed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { 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)) }) + wg.Add(1) t.Run("is segment indexed", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { return nil, nil, nil } @@ -935,7 +984,9 @@ func TestMetaTable(t *testing.T) { assert.False(t, mt.IsSegmentIndexed(idx.SegmentID, &field, nil)) }) + wg.Add(1) t.Run("get not indexed segments", func(t *testing.T) { + defer wg.Done() mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) { return nil, nil, nil } @@ -1028,7 +1079,9 @@ func TestMetaTable(t *testing.T) { //assert.EqualError(t, err, "multi save error") }) + wg.Add(1) 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) { return nil, nil, nil } @@ -1064,6 +1117,7 @@ func TestMetaTable(t *testing.T) { assert.NotNil(t, err) assert.EqualError(t, err, fmt.Sprintf("cannot find index, id = %d", idxInfo[0].IndexID)) }) + wg.Wait() } func TestMetaWithTimestamp(t *testing.T) { @@ -1088,15 +1142,14 @@ func TestMetaWithTimestamp(t *testing.T) { vtso++ return vtso } - - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) defer etcdCli.Close() skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7) assert.Nil(t, err) assert.NotNil(t, skv) - txnKV := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath) + txnKV := etcdkv.NewEtcdKV(etcdCli, rootPath) mt, err := NewMetaTable(txnKV, skv) assert.Nil(t, err) @@ -1246,7 +1299,7 @@ func TestFixIssue10540(t *testing.T) { Params.Init() 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) defer etcdCli.Close() diff --git a/internal/rootcoord/proxy_client_manager_test.go b/internal/rootcoord/proxy_client_manager_test.go index 44cb4342c1..34ec69017f 100644 --- a/internal/rootcoord/proxy_client_manager_test.go +++ b/internal/rootcoord/proxy_client_manager_test.go @@ -22,9 +22,9 @@ import ( "testing" "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/stretchr/testify/assert" - clientv3 "go.etcd.io/etcd/client/v3" ) func TestProxyClientManager_GetProxyClients(t *testing.T) { @@ -32,7 +32,8 @@ func TestProxyClientManager_GetProxyClients(t *testing.T) { core, err := NewCore(context.Background(), nil) 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) core.etcdCli = cli @@ -58,8 +59,9 @@ func TestProxyClientManager_AddProxyClient(t *testing.T) { core, err := NewCore(context.Background(), nil) assert.Nil(t, err) - cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + cli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) + defer cli.Close() core.etcdCli = cli core.SetNewProxyClient( @@ -84,8 +86,9 @@ func TestProxyClientManager_InvalidateCollectionMetaCache(t *testing.T) { core, err := NewCore(ctx, nil) assert.Nil(t, err) - cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + cli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) + defer cli.Close() core.etcdCli = cli pcm := newProxyClientManager(core) @@ -114,8 +117,9 @@ func TestProxyClientManager_ReleaseDQLMessageStream(t *testing.T) { core, err := NewCore(ctx, nil) assert.Nil(t, err) - cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints}) + cli, err := etcd.GetEtcdClient(&Params.BaseParams) assert.Nil(t, err) + defer cli.Close() core.etcdCli = cli pcm := newProxyClientManager(core) diff --git a/internal/rootcoord/proxy_manager.go b/internal/rootcoord/proxy_manager.go index 4bd9d96a4b..d978e5258f 100644 --- a/internal/rootcoord/proxy_manager.go +++ b/internal/rootcoord/proxy_manager.go @@ -46,20 +46,16 @@ type proxyManager struct { // newProxyManager helper function to create a proxyManager // etcdEndpoints is the address list of etcd // fns are the custom getSessions function list -func newProxyManager(ctx context.Context, etcdEndpoints []string, fns ...func([]*sessionutil.Session)) (*proxyManager, error) { - cli, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints}) - if err != nil { - return nil, err - } +func newProxyManager(ctx context.Context, client *clientv3.Client, fns ...func([]*sessionutil.Session)) *proxyManager { ctx2, cancel2 := context.WithCancel(ctx) p := &proxyManager{ ctx: ctx2, cancel: cancel2, lock: sync.Mutex{}, - etcdCli: cli, + etcdCli: client, } p.getSessions = append(p.getSessions, fns...) - return p, nil + return p } // AddSession adds functions to addSessions function list diff --git a/internal/rootcoord/proxy_manager_test.go b/internal/rootcoord/proxy_manager_test.go index cbf52baf3e..9ac79606b6 100644 --- a/internal/rootcoord/proxy_manager_test.go +++ b/internal/rootcoord/proxy_manager_test.go @@ -23,6 +23,7 @@ import ( "testing" "time" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" "github.com/stretchr/testify/assert" @@ -31,22 +32,23 @@ import ( func TestProxyManager(t *testing.T) { 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) defer cancel() sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) - cli.Delete(ctx, sessKey, clientv3.WithPrefix()) - defer cli.Delete(ctx, sessKey, clientv3.WithPrefix()) + etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) + defer etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) s1 := sessionutil.Session{ ServerID: 100, } b1, err := json.Marshal(&s1) assert.Nil(t, err) 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) s0 := sessionutil.Session{ @@ -55,7 +57,7 @@ func TestProxyManager(t *testing.T) { b0, err := json.Marshal(&s0) assert.Nil(t, err) 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) f1 := func(sess []*sessionutil.Session) { @@ -64,8 +66,7 @@ func TestProxyManager(t *testing.T) { assert.Equal(t, int64(99), sess[1].ServerID) t.Log("get sessions", sess[0], sess[1]) } - - pm, err := newProxyManager(ctx, Params.RootCoordCfg.EtcdEndpoints, f1) + pm := newProxyManager(ctx, etcdCli, f1) assert.Nil(t, err) fa := func(sess *sessionutil.Session) { assert.Equal(t, int64(101), sess.ServerID) @@ -88,10 +89,10 @@ func TestProxyManager(t *testing.T) { b2, err := json.Marshal(&s2) assert.Nil(t, err) 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) - _, err = cli.Delete(ctx, k1) + _, err = etcdCli.Delete(ctx, k1) assert.Nil(t, err) time.Sleep(100 * time.Millisecond) pm.Stop() diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index d1a7384bfd..5945315bb3 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -926,17 +926,23 @@ func (c *Core) Register() error { log.Fatal("failed to stop server", zap.Error(err)) } // manually send signal to starter goroutine - syscall.Kill(syscall.Getpid(), syscall.SIGINT) + if c.session.TriggerKill { + syscall.Kill(syscall.Getpid(), syscall.SIGINT) + } }) return nil } +func (c *Core) SetEtcdClient(etcdClient *clientv3.Client) { + c.etcdCli = etcdClient +} + 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 { 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) return nil } @@ -946,7 +952,7 @@ func (c *Core) Init() error { var initError error if c.kvBaseCreate == nil { 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() { @@ -956,10 +962,6 @@ func (c *Core) Init() error { return } 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 { log.Error("RootCoord failed to new EtcdKV", zap.Any("reason", initError)) return initError @@ -989,10 +991,7 @@ func (c *Core) Init() error { } log.Debug("RootCoord, Setting TSO and ID Allocator") - kv, initError := tsoutil.NewTSOKVBase(Params.RootCoordCfg.EtcdEndpoints, Params.RootCoordCfg.KvRootPath, "gid") - if initError != nil { - return - } + kv := tsoutil.NewTSOKVBase(c.etcdCli, Params.RootCoordCfg.KvRootPath, "gid") idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", kv) if initError = idAllocator.Initialize(); initError != nil { return @@ -1004,10 +1003,7 @@ func (c *Core) Init() error { return idAllocator.UpdateID() } - kv, initError = tsoutil.NewTSOKVBase(Params.RootCoordCfg.EtcdEndpoints, Params.RootCoordCfg.KvRootPath, "tso") - if initError != nil { - return - } + kv = tsoutil.NewTSOKVBase(c.etcdCli, Params.RootCoordCfg.KvRootPath, "tso") tsoAllocator := tso.NewGlobalTSOAllocator("timestamp", kv) if initError = tsoAllocator.Initialize(); initError != nil { return @@ -1033,15 +1029,12 @@ func (c *Core) Init() error { c.proxyClientManager = newProxyClientManager(c) log.Debug("RootCoord, set proxy manager") - c.proxyManager, initError = newProxyManager( + c.proxyManager = newProxyManager( c.ctx, - Params.RootCoordCfg.EtcdEndpoints, + c.etcdCli, c.chanTimeTick.getProxy, c.proxyClientManager.GetProxyClients, ) - if initError != nil { - return - } c.proxyManager.AddSession(c.chanTimeTick.addProxy, c.proxyClientManager.AddProxyClient) c.proxyManager.DelSession(c.chanTimeTick.delProxy, c.proxyClientManager.DelProxyClient) diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 775a5551ad..8b7c99ddae 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -37,6 +37,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/metricsinfo" "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/indexpb" @@ -48,10 +49,10 @@ import ( "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/schemapb" "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/typeutil" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" clientv3 "go.etcd.io/etcd/client/v3" ) @@ -433,9 +434,15 @@ func TestRootCoordInit(t *testing.T) { coreFactory := msgstream.NewPmsFactory() Params.Init() Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum + + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + defer etcdCli.Close() + core, err := NewCore(ctx, coreFactory) require.Nil(t, err) assert.Nil(t, err) + core.SetEtcdClient(etcdCli) randVal := rand.Int() Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath) @@ -443,12 +450,13 @@ func TestRootCoordInit(t *testing.T) { err = core.Init() assert.Nil(t, err) - + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) // inject kvBaseCreate fail core, err = NewCore(ctx, coreFactory) + core.SetEtcdClient(etcdCli) require.Nil(t, err) assert.Nil(t, err) randVal = rand.Int() @@ -462,11 +470,13 @@ func TestRootCoordInit(t *testing.T) { err = core.Init() assert.NotNil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) // inject metaKV create fail core, err = NewCore(ctx, coreFactory) + core.SetEtcdClient(etcdCli) require.Nil(t, err) assert.Nil(t, err) randVal = rand.Int() @@ -483,11 +493,13 @@ func TestRootCoordInit(t *testing.T) { err = core.Init() assert.NotNil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) // inject newSuffixSnapshot failure core, err = NewCore(ctx, coreFactory) + core.SetEtcdClient(etcdCli) require.Nil(t, err) assert.Nil(t, err) randVal = rand.Int() @@ -501,11 +513,13 @@ func TestRootCoordInit(t *testing.T) { err = core.Init() assert.NotNil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) // inject newMetaTable failure core, err = NewCore(ctx, coreFactory) + core.SetEtcdClient(etcdCli) require.Nil(t, err) assert.Nil(t, err) randVal = rand.Int() @@ -520,6 +534,7 @@ func TestRootCoordInit(t *testing.T) { err = core.Init() assert.NotNil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) @@ -544,7 +559,6 @@ func TestRootCoord(t *testing.T) { core, err := NewCore(ctx, coreFactory) assert.Nil(t, err) randVal := rand.Int() - Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal) Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal) 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.DeltaChannelName = fmt.Sprintf("rootcoord-delta-test-%d", randVal) - etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints, DialTimeout: 5 * time.Second}) - assert.Nil(t, err) + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + defer etcdCli.Close() + sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) assert.Nil(t, err) @@ -616,6 +632,8 @@ func TestRootCoord(t *testing.T) { dmlStream, _ := tmpFactory.NewMsgStream(ctx) clearMsgChan(1500*time.Millisecond, dmlStream.Chan()) + core.SetEtcdClient(etcdCli) + err = core.Init() assert.Nil(t, err) @@ -631,13 +649,18 @@ func TestRootCoord(t *testing.T) { err = core.Start() assert.Nil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) time.Sleep(100 * time.Millisecond) shardsNum := int32(8) + fmt.Printf("hello world2") + var wg sync.WaitGroup + wg.Add(1) t.Run("time tick", func(t *testing.T) { + defer wg.Done() ttmsg, ok := <-timeTickStream.Chan() assert.True(t, ok) 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) }) + wg.Add(1) t.Run("create collection", func(t *testing.T) { + defer wg.Done() schema := schemapb.CollectionSchema{ Name: collName, AutoID: true, @@ -789,7 +814,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) }) + wg.Add(1) t.Run("has collection", func(t *testing.T) { + defer wg.Done() req := &milvuspb.HasCollectionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_HasCollection, @@ -837,7 +864,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, true, rsp.Value) }) + wg.Add(1) t.Run("describe collection", func(t *testing.T) { + defer wg.Done() collMeta, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) req := &milvuspb.DescribeCollectionRequest{ @@ -860,7 +889,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, shardsNum, rsp.ShardsNum) }) + wg.Add(1) t.Run("show collection", func(t *testing.T) { + defer wg.Done() req := &milvuspb.ShowCollectionsRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_ShowCollections, @@ -877,7 +908,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, len(rsp.CollectionNames), 2) }) + wg.Add(1) t.Run("create partition", func(t *testing.T) { + defer wg.Done() req := &milvuspb.CreatePartitionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_CreatePartition, @@ -932,7 +965,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) }) + wg.Add(1) t.Run("has partition", func(t *testing.T) { + defer wg.Done() req := &milvuspb.HasPartitionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_HasPartition, @@ -950,7 +985,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, true, rsp.Value) }) + wg.Add(1) t.Run("show partition", func(t *testing.T) { + defer wg.Done() coll, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) req := &milvuspb.ShowPartitionsRequest{ @@ -971,7 +1008,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, 2, len(rsp.PartitionIDs)) }) + wg.Add(1) t.Run("show segment", func(t *testing.T) { + defer wg.Done() coll, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) partID := coll.PartitionIDs[1] @@ -996,7 +1035,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, 1, len(rsp.SegmentIDs)) }) + wg.Add(1) t.Run("create index", func(t *testing.T) { + defer wg.Done() req := &milvuspb.CreateIndexRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_CreateIndex, @@ -1038,7 +1079,9 @@ func TestRootCoord(t *testing.T) { assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode) }) + wg.Add(1) t.Run("describe segment", func(t *testing.T) { + defer wg.Done() coll, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) @@ -1058,7 +1101,9 @@ func TestRootCoord(t *testing.T) { t.Logf("index id = %d", rsp.IndexID) }) + wg.Add(1) t.Run("describe index", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DescribeIndexRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DescribeIndex, @@ -1079,7 +1124,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, "vector", rsp.IndexDescriptions[0].FieldName) }) + wg.Add(1) t.Run("describe index not exist", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DescribeIndexRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DescribeIndex, @@ -1098,7 +1145,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, 0, len(rsp.IndexDescriptions)) }) + wg.Add(1) t.Run("flush segment", func(t *testing.T) { + defer wg.Done() coll, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) partID := coll.PartitionIDs[1] @@ -1136,7 +1185,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName) }) + wg.Add(1) t.Run("over ride index", func(t *testing.T) { + defer wg.Done() req := &milvuspb.CreateIndexRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_CreateIndex, @@ -1180,7 +1231,9 @@ func TestRootCoord(t *testing.T) { }) + wg.Add(1) t.Run("drop index", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DropIndexRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DropIndex, @@ -1211,7 +1264,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, 0, len(idx)) }) + wg.Add(1) t.Run("drop partition", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DropPartitionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DropPartition, @@ -1267,7 +1322,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) }) + wg.Add(1) t.Run("remove DQL msgstream", func(t *testing.T) { + defer wg.Done() collMeta, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) @@ -1283,7 +1340,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) }) + wg.Add(1) t.Run("drop collection", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DropCollectionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DropCollection, @@ -1356,7 +1415,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) }) + wg.Add(1) t.Run("context_cancel", func(t *testing.T) { + defer wg.Done() ctx2, cancel2 := context.WithTimeout(ctx, time.Millisecond*100) defer cancel2() time.Sleep(100 * time.Millisecond) @@ -1516,7 +1577,9 @@ func TestRootCoord(t *testing.T) { time.Sleep(1 * time.Second) }) + wg.Add(1) t.Run("undefined req type", func(t *testing.T) { + defer wg.Done() st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{ Base: &commonpb.MsgBase{ 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) { + defer wg.Done() req := &rootcoordpb.AllocTimestampRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_Undefined, @@ -1689,7 +1754,9 @@ func TestRootCoord(t *testing.T) { assert.NotZero(t, rsp.Timestamp) }) + wg.Add(1) t.Run("alloc id", func(t *testing.T) { + defer wg.Done() req := &rootcoordpb.AllocIDRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_Undefined, @@ -1705,14 +1772,18 @@ func TestRootCoord(t *testing.T) { assert.NotZero(t, rsp.ID) }) + wg.Add(1) t.Run("get_channels", func(t *testing.T) { + defer wg.Done() _, err := core.GetTimeTickChannel(ctx) assert.Nil(t, err) _, err = core.GetStatisticsChannel(ctx) assert.Nil(t, err) }) + wg.Add(1) t.Run("channel timetick", func(t *testing.T) { + defer wg.Done() const ( proxyIDInvalid = 102 proxyName0 = "proxy_0" @@ -1825,7 +1896,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) + wg.Add(1) t.Run("create alias", func(t *testing.T) { + defer wg.Done() req := &milvuspb.CreateAliasRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_CreateAlias, @@ -1841,7 +1914,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode) }) + wg.Add(1) t.Run("describe collection2", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DescribeCollectionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DescribeCollection, @@ -1879,7 +1954,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) + wg.Add(1) t.Run("alter alias", func(t *testing.T) { + defer wg.Done() req := &milvuspb.AlterAliasRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_AlterAlias, @@ -1895,7 +1972,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode) }) + wg.Add(1) t.Run("drop collection with alias", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DropCollectionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DropAlias, @@ -1910,7 +1989,9 @@ func TestRootCoord(t *testing.T) { assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode) }) + wg.Add(1) t.Run("drop alias", func(t *testing.T) { + defer wg.Done() req := &milvuspb.DropAliasRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_DropAlias, @@ -1951,7 +2032,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) + wg.Add(1) t.Run("get metrics", func(t *testing.T) { + defer wg.Done() // not healthy stateSave := core.stateCode.Load().(internalpb.StateCode) core.UpdateStateCode(internalpb.StateCode_Abnormal) @@ -1985,7 +2068,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) }) + wg.Add(1) t.Run("get system info", func(t *testing.T) { + defer wg.Done() // normal case systemInfoMetricType := metricsinfo.SystemInfoMetrics req, err := metricsinfo.ConstructRequestByMetricType(systemInfoMetricType) @@ -2002,7 +2087,9 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, st.Status.ErrorCode) assert.NotEqual(t, internalpb.StateCode_Healthy, st.State.StateCode) + wg.Add(1) t.Run("state_not_healthy", func(t *testing.T) { + defer wg.Done() st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_CreateCollection, @@ -2159,7 +2246,9 @@ func TestRootCoord(t *testing.T) { }) + wg.Add(1) t.Run("alloc_error", func(t *testing.T) { + defer wg.Done() core.Stop() core.IDAllocator = func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) { return 0, 0, fmt.Errorf("id allocator error test") @@ -2195,9 +2284,9 @@ func TestRootCoord(t *testing.T) { assert.Nil(t, err) assert.NotEqual(t, commonpb.ErrorCode_Success, p2.Status.ErrorCode) }) + wg.Wait() err = core.Stop() assert.Nil(t, err) - } func TestRootCoord2(t *testing.T) { @@ -2215,6 +2304,11 @@ func TestRootCoord2(t *testing.T) { Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum core, err := NewCore(ctx, msFactory) assert.Nil(t, err) + + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.Nil(t, err) + defer etcdCli.Close() + randVal := rand.Int() Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal) @@ -2248,12 +2342,14 @@ func TestRootCoord2(t *testing.T) { return nil, nil } + core.SetEtcdClient(etcdCli) err = core.Init() assert.Nil(t, err) err = core.Start() assert.Nil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) @@ -2270,7 +2366,10 @@ func TestRootCoord2(t *testing.T) { time.Sleep(100 * time.Millisecond) + var wg sync.WaitGroup + wg.Add(1) t.Run("time tick", func(t *testing.T) { + defer wg.Done() ttmsg, ok := <-timeTickStream.Chan() assert.True(t, ok) 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)) }) + wg.Add(1) t.Run("create collection", func(t *testing.T) { + defer wg.Done() schema := schemapb.CollectionSchema{ Name: collName, } @@ -2314,7 +2415,9 @@ func TestRootCoord2(t *testing.T) { t.Log("time tick", m1.Base.Timestamp) }) + wg.Add(1) t.Run("describe collection", func(t *testing.T) { + defer wg.Done() collMeta, err := core.MetaTable.GetCollectionByName(collName, 0) assert.Nil(t, err) 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, rsp.ShardsNum) }) + wg.Wait() err = core.Stop() assert.Nil(t, err) } @@ -2516,12 +2620,17 @@ func TestCheckFlushedSegments(t *testing.T) { return nil, nil } + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.Nil(t, err) + defer etcdCli.Close() + core.SetEtcdClient(etcdCli) err = core.Init() assert.Nil(t, err) err = core.Start() assert.Nil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) @@ -2537,7 +2646,11 @@ func TestCheckFlushedSegments(t *testing.T) { timeTickStream.Start() time.Sleep(100 * time.Millisecond) + + var wg sync.WaitGroup + wg.Add(1) t.Run("check flushed segments", func(t *testing.T) { + defer wg.Done() ctx := context.Background() var collID int64 = 1 var partID int64 = 2 @@ -2620,6 +2733,7 @@ func TestCheckFlushedSegments(t *testing.T) { core.checkFlushedSegments(core.ctx) }) + wg.Wait() err = core.Stop() assert.Nil(t, err) } @@ -2641,7 +2755,6 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) { core, err := NewCore(ctx, msFactory) assert.Nil(t, err) randVal := rand.Int() - Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal) Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal) Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath) @@ -2673,12 +2786,18 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) { return nil, nil } + etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams) + assert.NoError(t, err) + defer etcdCli.Close() + + core.SetEtcdClient(etcdCli) err = core.Init() assert.Nil(t, err) err = core.Start() assert.Nil(t, err) + core.session.TriggerKill = false err = core.Register() assert.Nil(t, err) diff --git a/internal/rootcoord/suffix_snapshot_test.go b/internal/rootcoord/suffix_snapshot_test.go index 7e0ceb977b..93cd846a11 100644 --- a/internal/rootcoord/suffix_snapshot_test.go +++ b/internal/rootcoord/suffix_snapshot_test.go @@ -23,6 +23,7 @@ import ( "time" 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/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -262,8 +263,10 @@ func Test_SuffixSnapshotLoad(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) 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) defer etcdkv.Close() var vtso typeutil.Timestamp @@ -312,8 +315,10 @@ func Test_SuffixSnapshotMultiSave(t *testing.T) { Params.Init() rootPath := fmt.Sprintf("/test/meta/%d", randVal) 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) defer etcdkv.Close() var vtso typeutil.Timestamp @@ -386,7 +391,10 @@ func Test_SuffixSnapshotMultiSaveAndRemoveWithPrefix(t *testing.T) { rootPath := fmt.Sprintf("/test/meta/%d", randVal) 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) defer etcdkv.Close() diff --git a/internal/rootcoord/timeticksync_test.go b/internal/rootcoord/timeticksync_test.go index fcb20c1acc..bb4e58b762 100644 --- a/internal/rootcoord/timeticksync_test.go +++ b/internal/rootcoord/timeticksync_test.go @@ -18,6 +18,7 @@ package rootcoord import ( "context" + "sync" "testing" "github.com/stretchr/testify/assert" @@ -52,7 +53,10 @@ func TestTimetickSync(t *testing.T) { Params.RootCoordCfg.DeltaChannelName = "rootcoord-delta" ttSync := newTimeTickSync(ctx, session, factory, nil) + var wg sync.WaitGroup + wg.Add(1) t.Run("sendToChannel", func(t *testing.T) { + defer wg.Done() ttSync.sendToChannel() ttSync.proxyTimeTick[1] = nil @@ -67,14 +71,18 @@ func TestTimetickSync(t *testing.T) { ttSync.sendToChannel() }) + wg.Add(1) t.Run("RemoveDdlTimeTick", func(t *testing.T) { + defer wg.Done() ttSync.addDdlTimeTick(uint64(1), "1") ttSync.addDdlTimeTick(uint64(2), "2") ttSync.removeDdlTimeTick(uint64(1), "1") assert.Equal(t, ttSync.ddlMinTs, uint64(2)) }) + wg.Add(1) t.Run("UpdateTimeTick", func(t *testing.T) { + defer wg.Done() msg := &internalpb.ChannelTimeTickMsg{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_TimeTick, @@ -105,7 +113,9 @@ func TestTimetickSync(t *testing.T) { assert.Nil(t, err) }) + wg.Add(1) t.Run("minTimeTick", func(t *testing.T) { + defer wg.Done() tts := make([]uint64, 2) tts[0] = uint64(5) tts[1] = uint64(3) @@ -113,4 +123,5 @@ func TestTimetickSync(t *testing.T) { ret := minTimeTick(tts...) assert.Equal(t, ret, tts[1]) }) + wg.Wait() } diff --git a/internal/tso/global_allocator_test.go b/internal/tso/global_allocator_test.go index d24120bc74..ea56e7ed87 100644 --- a/internal/tso/global_allocator_test.go +++ b/internal/tso/global_allocator_test.go @@ -18,6 +18,7 @@ import ( "testing" "time" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/stretchr/testify/assert" ) @@ -30,8 +31,11 @@ func TestGlobalTSOAllocator_Initialize(t *testing.T) { endpoints = "localhost:2379" } etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest") - assert.NoError(t, err) + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + assert.Nil(t, err) + defer etcdCli.Close() + + etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest") gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) err = gTestTsoAllocator.Initialize() assert.Nil(t, err) @@ -67,8 +71,11 @@ func TestGlobalTSOAllocator_All(t *testing.T) { endpoints = "localhost:2379" } 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") + gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) t.Run("Initialize", func(t *testing.T) { err := gTestTsoAllocator.Initialize() @@ -153,7 +160,10 @@ func TestGlobalTSOAllocator_Fail(t *testing.T) { endpoints = "localhost:2379" } 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) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) t.Run("Initialize", func(t *testing.T) { @@ -194,7 +204,10 @@ func TestGlobalTSOAllocator_Update(t *testing.T) { endpoints = "localhost:2379" } 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) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) err = gTestTsoAllocator.Initialize() @@ -216,7 +229,10 @@ func TestGlobalTSOAllocator_load(t *testing.T) { endpoints = "localhost:2379" } 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) gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV) err = gTestTsoAllocator.Initialize() diff --git a/internal/types/types.go b/internal/types/types.go index a2f4827a57..575e5348e7 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -23,6 +23,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" ) // TimeTickProvider is the interface all services implement @@ -74,6 +75,9 @@ type DataNodeComponent interface { // GetStateCode return state code of this data node GetStateCode() internalpb.StateCode + // SetEtcdClient set etcd client for DataNode + SetEtcdClient(etcdClient *clientv3.Client) + // SetRootCoord set RootCoord for DataNode // `rootCoord` is a client of root coordinator. // @@ -248,6 +252,15 @@ type DataCoord interface { 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 type IndexNode interface { Component @@ -261,6 +274,18 @@ type IndexNode interface { 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 type IndexCoord interface { Component @@ -287,6 +312,18 @@ type IndexCoord interface { 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 type RootCoord interface { Component @@ -540,6 +577,10 @@ type RootCoord interface { type RootCoordComponent interface { RootCoord + // SetEtcdClient set EtcdClient for RootCoord + // `etcdClient` is a client of etcd + SetEtcdClient(etcdClient *clientv3.Client) + // UpdateStateCode updates state code for RootCoord // State includes: Initializing, Healthy and Abnormal UpdateStateCode(internalpb.StateCode) @@ -606,6 +647,10 @@ type Proxy interface { type ProxyComponent interface { Proxy + // SetEtcdClient set EtcdClient for Proxy + // `etcdClient` is a client of etcd + SetEtcdClient(etcdClient *clientv3.Client) + // SetRootCoord set RootCoord for Proxy // `rootCoord` is a client of root coordinator. SetRootCoordClient(rootCoord RootCoord) @@ -1021,6 +1066,9 @@ type QueryNodeComponent interface { // `stateCode` is current statement of this query node, indicating whether it's healthy. UpdateStateCode(stateCode internalpb.StateCode) + // SetEtcdClient set etcd client for QueryNode + SetEtcdClient(etcdClient *clientv3.Client) + // SetRootCoord set RootCoord for QueryNode // `rootCoord` is a client of root coordinator. Pass to segmentLoader. // @@ -1063,6 +1111,9 @@ type QueryCoord interface { type QueryCoordComponent interface { QueryCoord + // 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) diff --git a/internal/util/etcd/etcd_util.go b/internal/util/etcd/etcd_util.go new file mode 100644 index 0000000000..a3a57cc2b6 --- /dev/null +++ b/internal/util/etcd/etcd_util.go @@ -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, + }) +} diff --git a/internal/util/etcd/etcd_util_test.go b/internal/util/etcd/etcd_util_test.go new file mode 100644 index 0000000000..f3fbb7955d --- /dev/null +++ b/internal/util/etcd/etcd_util_test.go @@ -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") +} diff --git a/internal/util/grpcclient/client.go b/internal/util/grpcclient/client.go index b31eaf8c6e..d37194e9be 100644 --- a/internal/util/grpcclient/client.go +++ b/internal/util/grpcclient/client.go @@ -112,7 +112,7 @@ func (c *ClientBase) resetConnection(client interface{}) { func (c *ClientBase) connect(ctx context.Context) error { addr, err := c.getAddrFunc() 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 } diff --git a/internal/util/paramtable/global_param.go b/internal/util/paramtable/global_param.go index 65d825fb21..2a553c3202 100644 --- a/internal/util/paramtable/global_param.go +++ b/internal/util/paramtable/global_param.go @@ -289,7 +289,6 @@ type rootCoordConfig struct { Port int PulsarAddress string - EtcdEndpoints []string MetaRootPath string KvRootPath string @@ -316,7 +315,6 @@ func (p *rootCoordConfig) init(bp *BaseParamTable) { p.BaseParams = bp p.initPulsarAddress() - p.initEtcdEndpoints() p.initMetaRootPath() p.initKvRootPath() @@ -345,14 +343,6 @@ func (p *rootCoordConfig) initPulsarAddress() { 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { @@ -469,7 +459,6 @@ type proxyConfig struct { Alias string - EtcdEndpoints []string MetaRootPath string PulsarAddress string @@ -507,7 +496,6 @@ type proxyConfig struct { func (p *proxyConfig) init(bp *BaseParamTable) { p.BaseParams = bp - p.initEtcdEndpoints() p.initMetaRootPath() p.initPulsarAddress() 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { @@ -711,9 +691,8 @@ type queryCoordConfig struct { SearchResultChannelPrefix string // --- etcd --- - EtcdEndpoints []string - MetaRootPath string - KvRootPath string + MetaRootPath string + KvRootPath string //--- Minio --- MinioEndPoint string @@ -752,7 +731,6 @@ func (p *queryCoordConfig) init(bp *BaseParamTable) { p.initTimeTickChannelName() // --- etcd --- - p.initEtcdEndpoints() p.initMetaRootPath() p.initKvRootPath() @@ -824,14 +802,6 @@ func (p *queryCoordConfig) initTimeTickChannelName() { 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { @@ -980,7 +950,6 @@ type queryNodeConfig struct { PulsarAddress string RocksmqPath string - EtcdEndpoints []string MetaRootPath string Alias string @@ -1053,7 +1022,6 @@ func (p *queryNodeConfig) init(bp *BaseParamTable) { p.initPulsarAddress() p.initRocksmqPath() - p.initEtcdEndpoints() p.initMetaRootPath() p.initGracefulTime() @@ -1237,15 +1205,6 @@ func (p *queryNodeConfig) initStatsChannelName() { 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { @@ -1298,7 +1257,6 @@ type dataCoordConfig struct { Address string // --- ETCD --- - EtcdEndpoints []string MetaRootPath string KvRootPath string SegmentBinlogSubPath string @@ -1352,7 +1310,6 @@ type dataCoordConfig struct { func (p *dataCoordConfig) init(bp *BaseParamTable) { p.BaseParams = bp - p.initEtcdEndpoints() p.initMetaRootPath() p.initKvRootPath() p.initSegmentBinlogSubPath() @@ -1394,14 +1351,6 @@ func (p *dataCoordConfig) init(bp *BaseParamTable) { 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() { addr, err := p.BaseParams.Load("_PulsarAddress") if err != nil { @@ -1656,7 +1605,6 @@ type dataNodeConfig struct { MsgChannelSubName string // etcd - EtcdEndpoints []string MetaRootPath string ChannelWatchSubPath string @@ -1689,7 +1637,6 @@ func (p *dataNodeConfig) init(bp *BaseParamTable) { p.initTimeTickChannelName() p.initMsgChannelSubName() - p.initEtcdEndpoints() p.initMetaRootPath() p.initChannelWatchPath() @@ -1792,14 +1739,6 @@ func (p *dataNodeConfig) initMsgChannelSubName() { 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { @@ -1883,7 +1822,6 @@ type indexCoordConfig struct { Address string Port int - EtcdEndpoints []string KvRootPath string MetaRootPath string IndexStorageRootPath string @@ -1901,7 +1839,6 @@ type indexCoordConfig struct { func (p *indexCoordConfig) init(bp *BaseParamTable) { p.BaseParams = bp - p.initEtcdEndpoints() p.initMetaRootPath() p.initKvRootPath() p.initMinIOAddress() @@ -1912,14 +1849,6 @@ func (p *indexCoordConfig) init(bp *BaseParamTable) { 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { @@ -2013,7 +1942,6 @@ type indexNodeConfig struct { NodeID int64 Alias string - EtcdEndpoints []string MetaRootPath string IndexStorageRootPath string @@ -2037,7 +1965,6 @@ func (p *indexNodeConfig) init(bp *BaseParamTable) { p.initMinIOSecretAccessKey() p.initMinIOUseSSL() p.initMinioBucketName() - p.initEtcdEndpoints() p.initMetaRootPath() p.initIndexStorageRootPath() 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() { rootPath, err := p.BaseParams.Load("etcd.rootPath") if err != nil { diff --git a/internal/util/paramtable/global_param_test.go b/internal/util/paramtable/global_param_test.go index 3ee8f2b7f8..a86bafef76 100644 --- a/internal/util/paramtable/global_param_test.go +++ b/internal/util/paramtable/global_param_test.go @@ -40,9 +40,6 @@ func TestGlobalParamTable(t *testing.T) { assert.NotEqual(t, 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, "") 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) { Params := GlobalParams.ProxyCfg - t.Logf("EtcdEndPoints: %v", Params.EtcdEndpoints) - t.Logf("MetaRootPath: %s", Params.MetaRootPath) t.Logf("PulsarAddress: %s", Params.PulsarAddress) @@ -300,9 +295,6 @@ func TestGlobalParamTable(t *testing.T) { assert.Equal(t, name, "by-dev-dataNode-2") log.Println("MsgChannelSubName:", name) - endpoints := Params.EtcdEndpoints - log.Println("EtcdEndpoints:", endpoints) - path1 = Params.MetaRootPath log.Println("MetaRootPath:", path1) @@ -336,8 +328,6 @@ func TestGlobalParamTable(t *testing.T) { t.Logf("Port: %v", Params.Port) - t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints) - t.Logf("KvRootPath: %v", Params.KvRootPath) t.Logf("MetaRootPath: %v", Params.MetaRootPath) @@ -374,8 +364,6 @@ func TestGlobalParamTable(t *testing.T) { t.Logf("Alias: %v", Params.Alias) - t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints) - t.Logf("MetaRootPath: %v", Params.MetaRootPath) t.Logf("MinIOAddress: %v", Params.MinIOAddress) diff --git a/internal/util/rocksmq/server/rocksmq/global_rmq_test.go b/internal/util/rocksmq/server/rocksmq/global_rmq_test.go index e3d9005b00..71b5ccfcb0 100644 --- a/internal/util/rocksmq/server/rocksmq/global_rmq_test.go +++ b/internal/util/rocksmq/server/rocksmq/global_rmq_test.go @@ -21,6 +21,7 @@ import ( "github.com/milvus-io/milvus/internal/allocator" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/stretchr/testify/assert" ) @@ -32,10 +33,12 @@ func Test_InitRmq(t *testing.T) { endpoints = "localhost:2379" } etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root") + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + defer etcdCli.Close() if err != nil { log.Fatalf("New clientv3 error = %v", err) } + etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root") idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) _ = idAllocator.Initialize() diff --git a/internal/util/rocksmq/server/rocksmq/rocksmq_impl_test.go b/internal/util/rocksmq/server/rocksmq/rocksmq_impl_test.go index 869091679e..927927a707 100644 --- a/internal/util/rocksmq/server/rocksmq/rocksmq_impl_test.go +++ b/internal/util/rocksmq/server/rocksmq/rocksmq_impl_test.go @@ -26,6 +26,7 @@ import ( "github.com/milvus-io/milvus/internal/allocator" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" 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/stretchr/testify/assert" @@ -314,7 +315,10 @@ func TestRocksmq_Seek(t *testing.T) { func TestRocksmq_Loop(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -382,7 +386,10 @@ func TestRocksmq_Loop(t *testing.T) { func TestRocksmq_Goroutines(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -454,7 +461,10 @@ func TestRocksmq_Goroutines(t *testing.T) { */ func TestRocksmq_Throughout(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -509,7 +519,10 @@ func TestRocksmq_Throughout(t *testing.T) { func TestRocksmq_MultiChan(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -558,7 +571,10 @@ func TestRocksmq_MultiChan(t *testing.T) { func TestRocksmq_CopyData(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -621,7 +637,10 @@ func TestRocksmq_CopyData(t *testing.T) { func TestRocksmq_SeekToLatest(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -709,7 +728,10 @@ func TestRocksmq_SeekToLatest(t *testing.T) { func TestRocksmq_Reader(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -769,8 +791,10 @@ func TestRocksmq_Reader(t *testing.T) { func TestReader_CornerCase(t *testing.T) { 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") defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) _ = idAllocator.Initialize() @@ -826,7 +850,10 @@ func TestReader_CornerCase(t *testing.T) { func TestRocksmq_Close(t *testing.T) { 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) defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) @@ -863,8 +890,9 @@ func TestRocksmq_Close(t *testing.T) { func TestRocksmq_SeekWithNoConsumerError(t *testing.T) { ep := etcdEndpoints() - etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root") + etcdCli, err := etcd.GetRemoteEtcdClient(ep) assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root") defer etcdKV.Close() idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV) _ = idAllocator.Initialize() diff --git a/internal/util/sessionutil/session_util.go b/internal/util/sessionutil/session_util.go index f144bdca24..52a13020b4 100644 --- a/internal/util/sessionutil/session_util.go +++ b/internal/util/sessionutil/session_util.go @@ -55,10 +55,11 @@ type Session struct { // keepAliveCancel to cancel the etcd KeepAlive keepAliveCancel context.CancelFunc - ServerID int64 `json:"ServerID,omitempty"` - ServerName string `json:"ServerName,omitempty"` - Address string `json:"Address,omitempty"` - Exclusive bool `json:"Exclusive,omitempty"` + ServerID int64 `json:"ServerID,omitempty"` + ServerName string `json:"ServerName,omitempty"` + Address string `json:"Address,omitempty"` + Exclusive bool `json:"Exclusive,omitempty"` + TriggerKill bool liveCh <-chan bool etcdCli *clientv3.Client @@ -73,7 +74,7 @@ type Session struct { // ServerID, ServerName, Address, Exclusive will be assigned after Init(). // metaRoot is a path in etcd to save session information. // 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{ ctx: ctx, metaRoot: metaRoot, @@ -83,16 +84,12 @@ func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *S connectEtcdFn := func() error { 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) defer cancel2() - if _, err = etcdCli.Get(ctx2, "health"); err != nil { + if _, err := client.Get(ctx2, "health"); err != nil { return err } - session.etcdCli = etcdCli + session.etcdCli = client return nil } 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, // 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.Address = address s.Exclusive = exclusive + s.TriggerKill = triggerKill s.checkIDExist() serverID, err := s.getServerID() if err != nil { diff --git a/internal/util/sessionutil/session_util_test.go b/internal/util/sessionutil/session_util_test.go index 7d8d516053..a3fe708d33 100644 --- a/internal/util/sessionutil/session_util_test.go +++ b/internal/util/sessionutil/session_util_test.go @@ -12,6 +12,7 @@ import ( "time" 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/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -34,8 +35,10 @@ func TestGetServerIDConcurrently(t *testing.T) { } etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot) - assert.NoError(t, err) + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + require.NoError(t, err) + defer etcdCli.Close() + etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot) err = etcdKV.RemoveWithPrefix("") assert.NoError(t, err) @@ -45,7 +48,7 @@ func TestGetServerIDConcurrently(t *testing.T) { var wg sync.WaitGroup var muList = sync.Mutex{} - s := NewSession(ctx, metaRoot, etcdEndpoints) + s := NewSession(ctx, metaRoot, etcdCli) res := make([]int64, 0) getIDFunc := func() { @@ -79,16 +82,17 @@ func TestInit(t *testing.T) { metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot) - assert.NoError(t, err) + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + require.NoError(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot) err = etcdKV.RemoveWithPrefix("") assert.NoError(t, err) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") - s := NewSession(ctx, metaRoot, etcdEndpoints) - s.Init("inittest", "testAddr", false) + s := NewSession(ctx, metaRoot, etcdCli) + s.Init("inittest", "testAddr", false, false) assert.NotEqual(t, int64(0), s.leaseID) assert.NotEqual(t, int64(0), s.ServerID) s.Register() @@ -108,8 +112,10 @@ func TestUpdateSessions(t *testing.T) { etcdEndpoints := strings.Split(endpoints, ",") metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) - etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "") - assert.NoError(t, err) + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + require.NoError(t, err) + defer etcdCli.Close() + etcdKV := etcdkv.NewEtcdKV(etcdCli, "") defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") @@ -117,7 +123,7 @@ func TestUpdateSessions(t *testing.T) { var wg sync.WaitGroup var muList = sync.Mutex{} - s := NewSession(ctx, metaRoot, etcdEndpoints) + s := NewSession(ctx, metaRoot, etcdCli) sessions, rev, err := s.GetSessions("test") assert.Nil(t, err) @@ -127,8 +133,10 @@ func TestUpdateSessions(t *testing.T) { sList := []*Session{} getIDFunc := func() { - singleS := NewSession(ctx, metaRoot, etcdEndpoints) - singleS.Init("test", "testAddr", false) + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + require.NoError(t, err) + singleS := NewSession(ctx, metaRoot, etcdCli) + singleS.Init("test", "testAddr", false, false) singleS.Register() muList.Lock() sList = append(sList, singleS) @@ -220,12 +228,15 @@ func TestWatcherHandleWatchResp(t *testing.T) { etcdEndpoints := strings.Split(endpoints, ",") 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.RemoveWithPrefix("/by-dev/session-ut") - s := NewSession(ctx, metaRoot, etcdEndpoints) + s := NewSession(ctx, metaRoot, etcdCli) defer s.Revoke(time.Second) 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) { - s := NewSession(ctx, "/by-dev/session-ut", etcdEndpoints) + s := NewSession(ctx, "/by-dev/session-ut", etcdCli) s.etcdCli.Close() w := getWatcher(s, func(sessions map[string]*Session) error { return nil @@ -364,16 +375,18 @@ func TestSessionRevoke(t *testing.T) { metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot) etcdEndpoints := strings.Split(endpoints, ",") - etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot) - assert.NoError(t, err) + etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints) + defer etcdCli.Close() + require.NoError(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot) err = etcdKV.RemoveWithPrefix("") assert.NoError(t, err) defer etcdKV.Close() defer etcdKV.RemoveWithPrefix("") - s = NewSession(ctx, metaRoot, etcdEndpoints) - s.Init("revoketest", "testAddr", false) + s = NewSession(ctx, metaRoot, etcdCli) + s.Init("revoketest", "testAddr", false, false) assert.NotPanics(t, func() { s.Revoke(time.Second) }) diff --git a/internal/util/tsoutil/tso.go b/internal/util/tsoutil/tso.go index 566a87bacb..dac6530f8b 100644 --- a/internal/util/tsoutil/tso.go +++ b/internal/util/tsoutil/tso.go @@ -17,6 +17,7 @@ import ( etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/util/typeutil" + clientv3 "go.etcd.io/etcd/client/v3" ) const ( @@ -70,6 +71,6 @@ func AddPhysicalTimeOnTs(timeInMs int64, ts uint64) uint64 { } // NewTSOKVBase returns a etcdkv.EtcdKV object -func NewTSOKVBase(etcdEndpoints []string, tsoRoot, subPath string) (*etcdkv.EtcdKV, error) { - return etcdkv.NewEtcdKV(etcdEndpoints, path.Join(tsoRoot, subPath)) +func NewTSOKVBase(client *clientv3.Client, tsoRoot, subPath string) *etcdkv.EtcdKV { + return etcdkv.NewEtcdKV(client, path.Join(tsoRoot, subPath)) }