mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-02 03:48:37 +08:00
Support embedded etcd (#14333)
Signed-off-by: xiaofan-luan <xiaofan.luan@zilliz.com>
This commit is contained in:
parent
673806b1c1
commit
69087ff8dd
@ -34,10 +34,7 @@ type DataCoord struct {
|
|||||||
|
|
||||||
// NewDataCoord creates a new DataCoord
|
// NewDataCoord creates a new DataCoord
|
||||||
func NewDataCoord(ctx context.Context, factory msgstream.Factory) (*DataCoord, error) {
|
func NewDataCoord(ctx context.Context, factory msgstream.Factory) (*DataCoord, error) {
|
||||||
s, err := grpcdatacoordclient.NewServer(ctx, factory)
|
s := grpcdatacoordclient.NewServer(ctx, factory)
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
return &DataCoord{
|
return &DataCoord{
|
||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
|
@ -232,7 +232,7 @@ func main() {
|
|||||||
os.Exit(-1)
|
os.Exit(-1)
|
||||||
}
|
}
|
||||||
|
|
||||||
var localMsg = false
|
var local = false
|
||||||
role := roles.MilvusRoles{}
|
role := roles.MilvusRoles{}
|
||||||
switch serverType {
|
switch serverType {
|
||||||
case typeutil.RootCoordRole:
|
case typeutil.RootCoordRole:
|
||||||
@ -260,7 +260,7 @@ func main() {
|
|||||||
role.EnableDataNode = true
|
role.EnableDataNode = true
|
||||||
role.EnableIndexCoord = true
|
role.EnableIndexCoord = true
|
||||||
role.EnableIndexNode = true
|
role.EnableIndexNode = true
|
||||||
localMsg = true
|
local = true
|
||||||
case roleMixture:
|
case roleMixture:
|
||||||
role.EnableRootCoord = enableRootCoord
|
role.EnableRootCoord = enableRootCoord
|
||||||
role.EnableQueryCoord = enableQueryCoord
|
role.EnableQueryCoord = enableQueryCoord
|
||||||
@ -291,7 +291,7 @@ func main() {
|
|||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
defer removePidFile(fd)
|
defer removePidFile(fd)
|
||||||
role.Run(localMsg, svrAlias)
|
role.Run(local, svrAlias)
|
||||||
case "stop":
|
case "stop":
|
||||||
if err := stopPid(filename, runtimeDir); err != nil {
|
if err := stopPid(filename, runtimeDir); err != nil {
|
||||||
fmt.Fprintf(os.Stderr, "%s\n\n", err.Error())
|
fmt.Fprintf(os.Stderr, "%s\n\n", err.Error())
|
||||||
|
@ -43,13 +43,17 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/querycoord"
|
"github.com/milvus-io/milvus/internal/querycoord"
|
||||||
"github.com/milvus-io/milvus/internal/querynode"
|
"github.com/milvus-io/milvus/internal/querynode"
|
||||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/healthz"
|
"github.com/milvus-io/milvus/internal/util/healthz"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/rocksmq/server/rocksmq"
|
"github.com/milvus-io/milvus/internal/util/rocksmq/server/rocksmq"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var Params paramtable.GlobalParamTable
|
||||||
|
|
||||||
func newMsgFactory(localMsg bool) msgstream.Factory {
|
func newMsgFactory(localMsg bool) msgstream.Factory {
|
||||||
if localMsg {
|
if localMsg {
|
||||||
return msgstream.NewRmsFactory()
|
return msgstream.NewRmsFactory()
|
||||||
@ -338,7 +342,7 @@ func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, alias st
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Run Milvus components.
|
// Run Milvus components.
|
||||||
func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
func (mr *MilvusRoles) Run(local bool, alias string) {
|
||||||
if os.Getenv(metricsinfo.DeployModeEnvKey) == metricsinfo.StandaloneDeployMode {
|
if os.Getenv(metricsinfo.DeployModeEnvKey) == metricsinfo.StandaloneDeployMode {
|
||||||
closer := trace.InitTracing("standalone")
|
closer := trace.InitTracing("standalone")
|
||||||
if closer != nil {
|
if closer != nil {
|
||||||
@ -349,7 +353,8 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
|
||||||
// only standalone enable localMsg
|
// only standalone enable localMsg
|
||||||
if localMsg {
|
if local {
|
||||||
|
Params.Init()
|
||||||
if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode); err != nil {
|
if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode); err != nil {
|
||||||
log.Error("Failed to set deploy mode: ", zap.Error(err))
|
log.Error("Failed to set deploy mode: ", zap.Error(err))
|
||||||
}
|
}
|
||||||
@ -358,6 +363,12 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
defer stopRocksmq()
|
defer stopRocksmq()
|
||||||
|
|
||||||
|
if Params.BaseParams.UseEmbedEtcd {
|
||||||
|
// start etcd server
|
||||||
|
etcd.InitEtcdServer(&Params.BaseParams)
|
||||||
|
defer etcd.StopEtcdServer()
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode); err != nil {
|
if err := os.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode); err != nil {
|
||||||
log.Error("Failed to set deploy mode: ", zap.Error(err))
|
log.Error("Failed to set deploy mode: ", zap.Error(err))
|
||||||
@ -366,7 +377,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var rc *components.RootCoord
|
var rc *components.RootCoord
|
||||||
if mr.EnableRootCoord {
|
if mr.EnableRootCoord {
|
||||||
rc = mr.runRootCoord(ctx, localMsg)
|
rc = mr.runRootCoord(ctx, local)
|
||||||
if rc != nil {
|
if rc != nil {
|
||||||
defer rc.Stop()
|
defer rc.Stop()
|
||||||
}
|
}
|
||||||
@ -375,7 +386,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
var pn *components.Proxy
|
var pn *components.Proxy
|
||||||
if mr.EnableProxy {
|
if mr.EnableProxy {
|
||||||
pctx := logutil.WithModule(ctx, "Proxy")
|
pctx := logutil.WithModule(ctx, "Proxy")
|
||||||
pn = mr.runProxy(pctx, localMsg, alias)
|
pn = mr.runProxy(pctx, local, alias)
|
||||||
if pn != nil {
|
if pn != nil {
|
||||||
defer pn.Stop()
|
defer pn.Stop()
|
||||||
}
|
}
|
||||||
@ -383,7 +394,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var qs *components.QueryCoord
|
var qs *components.QueryCoord
|
||||||
if mr.EnableQueryCoord {
|
if mr.EnableQueryCoord {
|
||||||
qs = mr.runQueryCoord(ctx, localMsg)
|
qs = mr.runQueryCoord(ctx, local)
|
||||||
if qs != nil {
|
if qs != nil {
|
||||||
defer qs.Stop()
|
defer qs.Stop()
|
||||||
}
|
}
|
||||||
@ -391,7 +402,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var qn *components.QueryNode
|
var qn *components.QueryNode
|
||||||
if mr.EnableQueryNode {
|
if mr.EnableQueryNode {
|
||||||
qn = mr.runQueryNode(ctx, localMsg, alias)
|
qn = mr.runQueryNode(ctx, local, alias)
|
||||||
if qn != nil {
|
if qn != nil {
|
||||||
defer qn.Stop()
|
defer qn.Stop()
|
||||||
}
|
}
|
||||||
@ -399,7 +410,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var ds *components.DataCoord
|
var ds *components.DataCoord
|
||||||
if mr.EnableDataCoord {
|
if mr.EnableDataCoord {
|
||||||
ds = mr.runDataCoord(ctx, localMsg)
|
ds = mr.runDataCoord(ctx, local)
|
||||||
if ds != nil {
|
if ds != nil {
|
||||||
defer ds.Stop()
|
defer ds.Stop()
|
||||||
}
|
}
|
||||||
@ -407,7 +418,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var dn *components.DataNode
|
var dn *components.DataNode
|
||||||
if mr.EnableDataNode {
|
if mr.EnableDataNode {
|
||||||
dn = mr.runDataNode(ctx, localMsg, alias)
|
dn = mr.runDataNode(ctx, local, alias)
|
||||||
if dn != nil {
|
if dn != nil {
|
||||||
defer dn.Stop()
|
defer dn.Stop()
|
||||||
}
|
}
|
||||||
@ -415,7 +426,7 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var is *components.IndexCoord
|
var is *components.IndexCoord
|
||||||
if mr.EnableIndexCoord {
|
if mr.EnableIndexCoord {
|
||||||
is = mr.runIndexCoord(ctx, localMsg)
|
is = mr.runIndexCoord(ctx, local)
|
||||||
if is != nil {
|
if is != nil {
|
||||||
defer is.Stop()
|
defer is.Stop()
|
||||||
}
|
}
|
||||||
@ -423,13 +434,13 @@ func (mr *MilvusRoles) Run(localMsg bool, alias string) {
|
|||||||
|
|
||||||
var in *components.IndexNode
|
var in *components.IndexNode
|
||||||
if mr.EnableIndexNode {
|
if mr.EnableIndexNode {
|
||||||
in = mr.runIndexNode(ctx, localMsg, alias)
|
in = mr.runIndexNode(ctx, local, alias)
|
||||||
if in != nil {
|
if in != nil {
|
||||||
defer in.Stop()
|
defer in.Stop()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if localMsg {
|
if local {
|
||||||
standaloneHealthzHandler := func(w http.ResponseWriter, r *http.Request) {
|
standaloneHealthzHandler := func(w http.ResponseWriter, r *http.Request) {
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
@ -10,6 +10,7 @@ import (
|
|||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
@ -27,11 +28,14 @@ var (
|
|||||||
|
|
||||||
func main() {
|
func main() {
|
||||||
flag.Parse()
|
flag.Parse()
|
||||||
etcdkv, err := etcdkv.NewEtcdKV([]string{*etcdAddr}, *rootPath)
|
|
||||||
|
etcdCli, err := etcd.GetRemoteEtcdClient([]string{*etcdAddr})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatal("failed to connect to etcd", zap.Error(err))
|
log.Fatal("failed to connect to etcd", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
etcdkv := etcdkv.NewEtcdKV(etcdCli, *rootPath)
|
||||||
|
|
||||||
keys, values, err := etcdkv.LoadWithPrefix("/")
|
keys, values, err := etcdkv.LoadWithPrefix("/")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatal("failed to list ", zap.Error(err))
|
log.Fatal("failed to list ", zap.Error(err))
|
||||||
|
@ -17,24 +17,24 @@
|
|||||||
package allocator
|
package allocator
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"os"
|
|
||||||
"strings"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
var gTestIDAllocator *GlobalIDAllocator
|
var gTestIDAllocator *GlobalIDAllocator
|
||||||
|
|
||||||
|
var Params paramtable.GlobalParamTable
|
||||||
|
|
||||||
func TestGlobalTSOAllocator_All(t *testing.T) {
|
func TestGlobalTSOAllocator_All(t *testing.T) {
|
||||||
endpoints := os.Getenv("ETCD_ENDPOINTS")
|
Params.Init()
|
||||||
if endpoints == "" {
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
endpoints = "localhost:2379"
|
|
||||||
}
|
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "gidTest")
|
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "gidTest")
|
||||||
|
|
||||||
gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", etcdKV)
|
gTestIDAllocator = NewGlobalIDAllocator("idTimestamp", etcdKV)
|
||||||
|
|
||||||
|
@ -45,6 +45,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/minio/minio-go/v7"
|
"github.com/minio/minio-go/v7"
|
||||||
"github.com/minio/minio-go/v7/pkg/credentials"
|
"github.com/minio/minio-go/v7/pkg/credentials"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -82,7 +83,7 @@ const (
|
|||||||
)
|
)
|
||||||
|
|
||||||
type dataNodeCreatorFunc func(ctx context.Context, addr string) (types.DataNode, error)
|
type dataNodeCreatorFunc func(ctx context.Context, addr string) (types.DataNode, error)
|
||||||
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error)
|
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdClient *clientv3.Client) (types.RootCoord, error)
|
||||||
|
|
||||||
// makes sure Server implements `DataCoord`
|
// makes sure Server implements `DataCoord`
|
||||||
var _ types.DataCoord = (*Server)(nil)
|
var _ types.DataCoord = (*Server)(nil)
|
||||||
@ -100,6 +101,7 @@ type Server struct {
|
|||||||
isServing ServerState
|
isServing ServerState
|
||||||
helper ServerHelper
|
helper ServerHelper
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
kvClient *etcdkv.EtcdKV
|
kvClient *etcdkv.EtcdKV
|
||||||
meta *meta
|
meta *meta
|
||||||
segmentManager Manager
|
segmentManager Manager
|
||||||
@ -177,7 +179,7 @@ func SetSegmentManager(manager Manager) Option {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// CreateServer creates a `Server` instance
|
// CreateServer creates a `Server` instance
|
||||||
func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option) (*Server, error) {
|
func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option) *Server {
|
||||||
rand.Seed(time.Now().UnixNano())
|
rand.Seed(time.Now().UnixNano())
|
||||||
s := &Server{
|
s := &Server{
|
||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
@ -194,15 +196,15 @@ func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option
|
|||||||
for _, opt := range opts {
|
for _, opt := range opts {
|
||||||
opt(s)
|
opt(s)
|
||||||
}
|
}
|
||||||
return s, nil
|
return s
|
||||||
}
|
}
|
||||||
|
|
||||||
func defaultDataNodeCreatorFunc(ctx context.Context, addr string) (types.DataNode, error) {
|
func defaultDataNodeCreatorFunc(ctx context.Context, addr string) (types.DataNode, error) {
|
||||||
return datanodeclient.NewClient(ctx, addr)
|
return datanodeclient.NewClient(ctx, addr)
|
||||||
}
|
}
|
||||||
|
|
||||||
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, client *clientv3.Client) (types.RootCoord, error) {
|
||||||
return rootcoordclient.NewClient(ctx, metaRootPath, etcdEndpoints)
|
return rootcoordclient.NewClient(ctx, metaRootPath, client)
|
||||||
}
|
}
|
||||||
|
|
||||||
// QuitSignal returns signal when server quits
|
// QuitSignal returns signal when server quits
|
||||||
@ -219,17 +221,19 @@ func (s *Server) Register() error {
|
|||||||
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
|
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if s.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) initSession() error {
|
func (s *Server) initSession() error {
|
||||||
s.session = sessionutil.NewSession(s.ctx, Params.DataCoordCfg.MetaRootPath, Params.DataCoordCfg.EtcdEndpoints)
|
s.session = sessionutil.NewSession(s.ctx, Params.DataCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if s.session == nil {
|
if s.session == nil {
|
||||||
return errors.New("failed to initialize session")
|
return errors.New("failed to initialize session")
|
||||||
}
|
}
|
||||||
s.session.Init(typeutil.DataCoordRole, Params.DataCoordCfg.Address, true)
|
s.session.Init(typeutil.DataCoordRole, Params.DataCoordCfg.Address, true, true)
|
||||||
Params.DataCoordCfg.NodeID = s.session.ServerID
|
Params.DataCoordCfg.NodeID = s.session.ServerID
|
||||||
Params.BaseParams.SetLogger(Params.DataCoordCfg.NodeID)
|
Params.BaseParams.SetLogger(Params.DataCoordCfg.NodeID)
|
||||||
return nil
|
return nil
|
||||||
@ -311,6 +315,11 @@ func (s *Server) initCluster() error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient sets etcd client for datacoord.
|
||||||
|
func (s *Server) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
s.etcdCli = client
|
||||||
|
}
|
||||||
|
|
||||||
func (s *Server) createCompactionHandler() {
|
func (s *Server) createCompactionHandler() {
|
||||||
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh)
|
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh)
|
||||||
s.compactionHandler.start()
|
s.compactionHandler.start()
|
||||||
@ -407,20 +416,17 @@ func (s *Server) startSegmentManager() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) initMeta() error {
|
func (s *Server) initMeta() error {
|
||||||
connectEtcdFn := func() error {
|
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.DataCoordCfg.MetaRootPath)
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.DataCoordCfg.EtcdEndpoints, Params.DataCoordCfg.MetaRootPath)
|
s.kvClient = etcdKV
|
||||||
if err != nil {
|
reloadEtcdFn := func() error {
|
||||||
return err
|
var err error
|
||||||
}
|
|
||||||
|
|
||||||
s.kvClient = etcdKV
|
|
||||||
s.meta, err = newMeta(s.kvClient)
|
s.meta, err = newMeta(s.kvClient)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
return retry.Do(s.ctx, connectEtcdFn, retry.Attempts(connEtcdMaxRetryTime))
|
return retry.Do(s.ctx, reloadEtcdFn, retry.Attempts(connEtcdMaxRetryTime))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) startServerLoop() {
|
func (s *Server) startServerLoop() {
|
||||||
@ -724,7 +730,7 @@ func (s *Server) handleFlushingSegments(ctx context.Context) {
|
|||||||
|
|
||||||
func (s *Server) initRootCoordClient() error {
|
func (s *Server) initRootCoordClient() error {
|
||||||
var err error
|
var err error
|
||||||
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.DataCoordCfg.MetaRootPath, Params.DataCoordCfg.EtcdEndpoints); err != nil {
|
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.DataCoordCfg.MetaRootPath, s.etcdCli); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
if err = s.rootCoordClient.Init(); err != nil {
|
if err = s.rootCoordClient.Init(); err != nil {
|
||||||
|
@ -37,8 +37,8 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
@ -613,8 +613,7 @@ func TestGetFlushedSegments(t *testing.T) {
|
|||||||
|
|
||||||
func TestService_WatchServices(t *testing.T) {
|
func TestService_WatchServices(t *testing.T) {
|
||||||
factory := msgstream.NewPmsFactory()
|
factory := msgstream.NewPmsFactory()
|
||||||
svr, err := CreateServer(context.TODO(), factory)
|
svr := CreateServer(context.TODO(), factory)
|
||||||
assert.Nil(t, err)
|
|
||||||
svr.serverLoopWg.Add(1)
|
svr.serverLoopWg.Add(1)
|
||||||
|
|
||||||
ech := make(chan *sessionutil.SessionEvent)
|
ech := make(chan *sessionutil.SessionEvent)
|
||||||
@ -1522,7 +1521,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
|
|
||||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||||
return newMockRootCoordService(), nil
|
return newMockRootCoordService(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1565,7 +1564,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
|
|
||||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||||
return newMockRootCoordService(), nil
|
return newMockRootCoordService(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1593,7 +1592,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
|
|
||||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||||
return newMockRootCoordService(), nil
|
return newMockRootCoordService(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1621,7 +1620,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
|
|
||||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||||
return newMockRootCoordService(), nil
|
return newMockRootCoordService(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1699,7 +1698,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
|||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
|
|
||||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||||
return newMockRootCoordService(), nil
|
return newMockRootCoordService(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1954,7 +1953,7 @@ func TestOptions(t *testing.T) {
|
|||||||
t.Run("SetRootCoordCreator", func(t *testing.T) {
|
t.Run("SetRootCoordCreator", func(t *testing.T) {
|
||||||
svr := newTestServer(t, nil)
|
svr := newTestServer(t, nil)
|
||||||
defer closeTestServer(t, svr)
|
defer closeTestServer(t, svr)
|
||||||
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, endpoints []string) (types.RootCoord, error) {
|
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, etcdClient *clientv3.Client) (types.RootCoord, error) {
|
||||||
return nil, errors.New("dummy")
|
return nil, errors.New("dummy")
|
||||||
}
|
}
|
||||||
opt := SetRootCoordCreator(crt)
|
opt := SetRootCoordCreator(crt)
|
||||||
@ -1992,8 +1991,7 @@ func TestOptions(t *testing.T) {
|
|||||||
|
|
||||||
factory := msgstream.NewPmsFactory()
|
factory := msgstream.NewPmsFactory()
|
||||||
|
|
||||||
svr, err := CreateServer(context.TODO(), factory, opt)
|
svr := CreateServer(context.TODO(), factory, opt)
|
||||||
assert.Nil(t, err)
|
|
||||||
dn, err := svr.dataNodeCreator(context.Background(), "")
|
dn, err := svr.dataNodeCreator(context.Background(), "")
|
||||||
assert.Nil(t, dn)
|
assert.Nil(t, dn)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -2246,18 +2244,18 @@ func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Se
|
|||||||
err = factory.SetParams(m)
|
err = factory.SetParams(m)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
etcdCli, err := initEtcd(Params.DataCoordCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
sessKey := path.Join(Params.DataCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
sessKey := path.Join(Params.DataCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||||
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
|
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
svr, err := CreateServer(context.TODO(), factory, opts...)
|
svr := CreateServer(context.TODO(), factory, opts...)
|
||||||
assert.Nil(t, err)
|
svr.SetEtcdClient(etcdCli)
|
||||||
svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) {
|
svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) {
|
||||||
return newMockDataNodeClient(0, receiveCh)
|
return newMockDataNodeClient(0, receiveCh)
|
||||||
}
|
}
|
||||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||||
return newMockRootCoordService(), nil
|
return newMockRootCoordService(), nil
|
||||||
}
|
}
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -2276,20 +2274,3 @@ func closeTestServer(t *testing.T, svr *Server) {
|
|||||||
err = svr.CleanMeta()
|
err = svr.CleanMeta()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
func initEtcd(etcdEndpoints []string) (*clientv3.Client, error) {
|
|
||||||
var etcdCli *clientv3.Client
|
|
||||||
connectEtcdFn := func() error {
|
|
||||||
etcd, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints, DialTimeout: 5 * time.Second})
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
etcdCli = etcd
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
err := retry.Do(context.TODO(), connectEtcdFn, retry.Attempts(300))
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
return etcdCli, nil
|
|
||||||
}
|
|
||||||
|
@ -83,6 +83,7 @@ var Params paramtable.GlobalParamTable
|
|||||||
// services in datanode package.
|
// services in datanode package.
|
||||||
//
|
//
|
||||||
// DataNode implements `types.Component`, `types.DataNode` interfaces.
|
// DataNode implements `types.Component`, `types.DataNode` interfaces.
|
||||||
|
// `etcdCli` is a connection of etcd
|
||||||
// `rootCoord` is a grpc client of root coordinator.
|
// `rootCoord` is a grpc client of root coordinator.
|
||||||
// `dataCoord` is a grpc client of data service.
|
// `dataCoord` is a grpc client of data service.
|
||||||
// `NodeID` is unique to each datanode.
|
// `NodeID` is unique to each datanode.
|
||||||
@ -109,6 +110,7 @@ type DataNode struct {
|
|||||||
segmentCache *Cache
|
segmentCache *Cache
|
||||||
compactionExecutor *compactionExecutor
|
compactionExecutor *compactionExecutor
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
rootCoord types.RootCoord
|
rootCoord types.RootCoord
|
||||||
dataCoord types.DataCoord
|
dataCoord types.DataCoord
|
||||||
|
|
||||||
@ -144,6 +146,11 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
|
|||||||
return node
|
return node
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Set etcd client
|
||||||
|
func (node *DataNode) SetEtcdClient(etcdCli *clientv3.Client) {
|
||||||
|
node.etcdCli = etcdCli
|
||||||
|
}
|
||||||
|
|
||||||
// SetRootCoord sets RootCoord's grpc client, error is returned if repeatedly set.
|
// SetRootCoord sets RootCoord's grpc client, error is returned if repeatedly set.
|
||||||
func (node *DataNode) SetRootCoord(rc types.RootCoord) error {
|
func (node *DataNode) SetRootCoord(rc types.RootCoord) error {
|
||||||
switch {
|
switch {
|
||||||
@ -182,18 +189,20 @@ func (node *DataNode) Register() error {
|
|||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if node.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (node *DataNode) initSession() error {
|
func (node *DataNode) initSession() error {
|
||||||
node.session = sessionutil.NewSession(node.ctx, Params.DataNodeCfg.MetaRootPath, Params.DataNodeCfg.EtcdEndpoints)
|
node.session = sessionutil.NewSession(node.ctx, Params.DataNodeCfg.MetaRootPath, node.etcdCli)
|
||||||
if node.session == nil {
|
if node.session == nil {
|
||||||
return errors.New("failed to initialize session")
|
return errors.New("failed to initialize session")
|
||||||
}
|
}
|
||||||
node.session.Init(typeutil.DataNodeRole, Params.DataNodeCfg.IP+":"+strconv.Itoa(Params.DataNodeCfg.Port), false)
|
node.session.Init(typeutil.DataNodeRole, Params.DataNodeCfg.IP+":"+strconv.Itoa(Params.DataNodeCfg.Port), false, true)
|
||||||
Params.DataNodeCfg.NodeID = node.session.ServerID
|
Params.DataNodeCfg.NodeID = node.session.ServerID
|
||||||
node.NodeID = node.session.ServerID
|
node.NodeID = node.session.ServerID
|
||||||
Params.BaseParams.SetLogger(Params.DataNodeCfg.NodeID)
|
Params.BaseParams.SetLogger(Params.DataNodeCfg.NodeID)
|
||||||
@ -411,10 +420,7 @@ func (node *DataNode) Start() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
connectEtcdFn := func() error {
|
connectEtcdFn := func() error {
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath)
|
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.DataNodeCfg.MetaRootPath)
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
node.watchKv = etcdKV
|
node.watchKv = etcdKV
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -34,6 +34,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
|
|
||||||
@ -62,7 +63,11 @@ func TestMain(t *testing.M) {
|
|||||||
func TestDataNode(t *testing.T) {
|
func TestDataNode(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
node := newIDLEDataNodeMock(ctx)
|
node := newIDLEDataNodeMock(ctx)
|
||||||
err := node.Init()
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
node.SetEtcdClient(etcdCli)
|
||||||
|
err = node.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = node.Start()
|
err = node.Start()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -164,6 +169,7 @@ func TestDataNode(t *testing.T) {
|
|||||||
dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments"
|
dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments"
|
||||||
|
|
||||||
node1 := newIDLEDataNodeMock(context.TODO())
|
node1 := newIDLEDataNodeMock(context.TODO())
|
||||||
|
node1.SetEtcdClient(etcdCli)
|
||||||
err = node1.Init()
|
err = node1.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = node1.Start()
|
err = node1.Start()
|
||||||
@ -475,7 +481,11 @@ func TestDataNode(t *testing.T) {
|
|||||||
func TestWatchChannel(t *testing.T) {
|
func TestWatchChannel(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
node := newIDLEDataNodeMock(ctx)
|
node := newIDLEDataNodeMock(ctx)
|
||||||
err := node.Init()
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
node.SetEtcdClient(etcdCli)
|
||||||
|
err = node.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = node.Start()
|
err = node.Start()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -485,8 +495,7 @@ func TestWatchChannel(t *testing.T) {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
t.Run("test watch channel", func(t *testing.T) {
|
t.Run("test watch channel", func(t *testing.T) {
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.DataNodeCfg.MetaRootPath)
|
||||||
require.NoError(t, err)
|
|
||||||
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
|
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
|
||||||
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, oldInvalidCh)
|
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, oldInvalidCh)
|
||||||
err = kv.Save(path, string([]byte{23}))
|
err = kv.Save(path, string([]byte{23}))
|
||||||
|
@ -42,6 +42,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
)
|
)
|
||||||
|
|
||||||
const ctxTimeInMillisecond = 5000
|
const ctxTimeInMillisecond = 5000
|
||||||
@ -106,10 +107,11 @@ func makeNewChannelNames(names []string, suffix string) []string {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func clearEtcd(rootPath string) error {
|
func clearEtcd(rootPath string) error {
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, rootPath)
|
client, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(client, rootPath)
|
||||||
|
|
||||||
err = etcdKV.RemoveWithPrefix("writer/segment")
|
err = etcdKV.RemoveWithPrefix("writer/segment")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -28,6 +28,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
@ -62,7 +63,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
rootcoordpb.RegisterRootCoordServer(grpcServer, rootCoord)
|
rootcoordpb.RegisterRootCoordServer(grpcServer, rootCoord)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.RootCoordRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.RootCoordRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
rootCoord, ok := cm.GetRootCoordClient()
|
rootCoord, ok := cm.GetRootCoordClient()
|
||||||
@ -79,7 +80,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
querypb.RegisterQueryCoordServer(grpcServer, queryCoord)
|
querypb.RegisterQueryCoordServer(grpcServer, queryCoord)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.QueryCoordRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.QueryCoordRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
queryCoord, ok := cm.GetQueryCoordClient()
|
queryCoord, ok := cm.GetQueryCoordClient()
|
||||||
@ -96,7 +97,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
datapb.RegisterDataCoordServer(grpcServer, dataCoord)
|
datapb.RegisterDataCoordServer(grpcServer, dataCoord)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.DataCoordRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.DataCoordRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
dataCoord, ok := cm.GetDataCoordClient()
|
dataCoord, ok := cm.GetDataCoordClient()
|
||||||
@ -113,7 +114,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
indexpb.RegisterIndexCoordServer(grpcServer, indexCoord)
|
indexpb.RegisterIndexCoordServer(grpcServer, indexCoord)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.IndexCoordRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.IndexCoordRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
indexCoord, ok := cm.GetIndexCoordClient()
|
indexCoord, ok := cm.GetIndexCoordClient()
|
||||||
@ -130,7 +131,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
querypb.RegisterQueryNodeServer(grpcServer, queryNode)
|
querypb.RegisterQueryNodeServer(grpcServer, queryNode)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.QueryNodeRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.QueryNodeRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
queryNodes, ok := cm.GetQueryNodeClients()
|
queryNodes, ok := cm.GetQueryNodeClients()
|
||||||
@ -147,7 +148,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
datapb.RegisterDataNodeServer(grpcServer, dataNode)
|
datapb.RegisterDataNodeServer(grpcServer, dataNode)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.DataNodeRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.DataNodeRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
dataNodes, ok := cm.GetDataNodeClients()
|
dataNodes, ok := cm.GetDataNodeClients()
|
||||||
@ -164,7 +165,7 @@ func TestConnectionManager(t *testing.T) {
|
|||||||
defer grpcServer.Stop()
|
defer grpcServer.Stop()
|
||||||
indexpb.RegisterIndexNodeServer(grpcServer, indexNode)
|
indexpb.RegisterIndexNodeServer(grpcServer, indexNode)
|
||||||
go grpcServer.Serve(lis)
|
go grpcServer.Serve(lis)
|
||||||
session.Init(typeutil.IndexNodeRole, "127.0.0.1:9999", true)
|
session.Init(typeutil.IndexNodeRole, "127.0.0.1:9999", true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
assert.Eventually(t, func() bool {
|
assert.Eventually(t, func() bool {
|
||||||
indexNodes, ok := cm.GetIndexNodeClients()
|
indexNodes, ok := cm.GetIndexNodeClients()
|
||||||
@ -250,6 +251,10 @@ func initSession(ctx context.Context) *sessionutil.Session {
|
|||||||
log.Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath))
|
log.Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath))
|
||||||
log.Debug("etcdPoints", zap.Any("etcdPoints", etcdEndpoints))
|
log.Debug("etcdPoints", zap.Any("etcdPoints", etcdEndpoints))
|
||||||
|
|
||||||
session := sessionutil.NewSession(ctx, metaRootPath, etcdEndpoints)
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
if err != nil {
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
|
session := sessionutil.NewSession(ctx, metaRootPath, etcdCli)
|
||||||
return session
|
return session
|
||||||
}
|
}
|
||||||
|
@ -30,6 +30,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
@ -43,8 +44,8 @@ type Client struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewClient creates a new client instance
|
// NewClient creates a new client instance
|
||||||
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) {
|
func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
|
||||||
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints)
|
sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
|
||||||
if sess == nil {
|
if sess == nil {
|
||||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||||
log.Debug("DataCoordClient NewClient failed", zap.Error(err))
|
log.Debug("DataCoordClient NewClient failed", zap.Error(err))
|
||||||
|
@ -21,9 +21,9 @@ import (
|
|||||||
"errors"
|
"errors"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/util/mock"
|
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proxy"
|
"github.com/milvus-io/milvus/internal/proxy"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/mock"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
@ -32,7 +32,9 @@ func Test_NewClient(t *testing.T) {
|
|||||||
proxy.Params.InitOnce()
|
proxy.Params.InitOnce()
|
||||||
|
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, client)
|
assert.NotNil(t, client)
|
||||||
|
|
||||||
|
@ -36,10 +36,12 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -54,7 +56,9 @@ type Server struct {
|
|||||||
cancel context.CancelFunc
|
cancel context.CancelFunc
|
||||||
|
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
dataCoord types.DataCoord
|
dataCoord types.DataCoordComponent
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
|
|
||||||
grpcErrChan chan error
|
grpcErrChan chan error
|
||||||
grpcServer *grpc.Server
|
grpcServer *grpc.Server
|
||||||
@ -62,8 +66,7 @@ type Server struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewServer new data service grpc server
|
// NewServer new data service grpc server
|
||||||
func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) (*Server, error) {
|
func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) *Server {
|
||||||
var err error
|
|
||||||
ctx1, cancel := context.WithCancel(ctx)
|
ctx1, cancel := context.WithCancel(ctx)
|
||||||
|
|
||||||
s := &Server{
|
s := &Server{
|
||||||
@ -71,11 +74,8 @@ func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord
|
|||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
grpcErrChan: make(chan error),
|
grpcErrChan: make(chan error),
|
||||||
}
|
}
|
||||||
s.dataCoord, err = datacoord.CreateServer(s.ctx, factory, opts...)
|
s.dataCoord = datacoord.CreateServer(s.ctx, factory, opts...)
|
||||||
if err != nil {
|
return s
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
return s, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) init() error {
|
func (s *Server) init() error {
|
||||||
@ -89,7 +89,15 @@ func (s *Server) init() error {
|
|||||||
datacoord.Params.DataCoordCfg.Port = Params.Port
|
datacoord.Params.DataCoordCfg.Port = Params.Port
|
||||||
datacoord.Params.DataCoordCfg.Address = Params.GetAddress()
|
datacoord.Params.DataCoordCfg.Address = Params.GetAddress()
|
||||||
|
|
||||||
err := s.startGrpc()
|
etcdCli, err := etcd.GetEtcdClient(&datacoord.Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("DataCoord connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.dataCoord.SetEtcdClient(etcdCli)
|
||||||
|
|
||||||
|
err = s.startGrpc()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("DataCoord startGrpc failed", zap.Error(err))
|
log.Debug("DataCoord startGrpc failed", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
@ -178,6 +186,9 @@ func (s *Server) Stop() error {
|
|||||||
}
|
}
|
||||||
s.cancel()
|
s.cancel()
|
||||||
|
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
log.Debug("Graceful stop grpc server...")
|
log.Debug("Graceful stop grpc server...")
|
||||||
s.grpcServer.GracefulStop()
|
s.grpcServer.GracefulStop()
|
||||||
|
@ -26,6 +26,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
@ -72,6 +73,9 @@ func (m *MockDataCoord) Register() error {
|
|||||||
return m.regErr
|
return m.regErr
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockDataCoord) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||||
return m.states, m.err
|
return m.states, m.err
|
||||||
}
|
}
|
||||||
@ -163,13 +167,12 @@ func (m *MockDataCoord) DropVirtualChannel(ctx context.Context, req *datapb.Drop
|
|||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
func Test_NewServer(t *testing.T) {
|
func Test_NewServer(t *testing.T) {
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
server, err := NewServer(ctx, nil)
|
server := NewServer(ctx, nil)
|
||||||
assert.Nil(t, err)
|
|
||||||
assert.NotNil(t, server)
|
assert.NotNil(t, server)
|
||||||
|
|
||||||
t.Run("Run", func(t *testing.T) {
|
t.Run("Run", func(t *testing.T) {
|
||||||
server.dataCoord = &MockDataCoord{}
|
server.dataCoord = &MockDataCoord{}
|
||||||
err = server.Run()
|
err := server.Run()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
@ -335,21 +338,20 @@ func Test_NewServer(t *testing.T) {
|
|||||||
assert.NotNil(t, resp)
|
assert.NotNil(t, resp)
|
||||||
})
|
})
|
||||||
|
|
||||||
err = server.Stop()
|
err := server.Stop()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
func Test_Run(t *testing.T) {
|
func Test_Run(t *testing.T) {
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
server, err := NewServer(ctx, nil)
|
server := NewServer(ctx, nil)
|
||||||
assert.Nil(t, err)
|
|
||||||
assert.NotNil(t, server)
|
assert.NotNil(t, server)
|
||||||
|
|
||||||
server.dataCoord = &MockDataCoord{
|
server.dataCoord = &MockDataCoord{
|
||||||
regErr: errors.New("error"),
|
regErr: errors.New("error"),
|
||||||
}
|
}
|
||||||
|
|
||||||
err = server.Run()
|
err := server.Run()
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
|
|
||||||
server.dataCoord = &MockDataCoord{
|
server.dataCoord = &MockDataCoord{
|
||||||
|
@ -37,10 +37,12 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -56,14 +58,14 @@ type Server struct {
|
|||||||
grpcServer *grpc.Server
|
grpcServer *grpc.Server
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
cancel context.CancelFunc
|
cancel context.CancelFunc
|
||||||
|
etcdCli *clientv3.Client
|
||||||
msFactory msgstream.Factory
|
msFactory msgstream.Factory
|
||||||
|
|
||||||
rootCoord types.RootCoord
|
rootCoord types.RootCoord
|
||||||
dataCoord types.DataCoord
|
dataCoord types.DataCoord
|
||||||
|
|
||||||
newRootCoordClient func(string, []string) (types.RootCoord, error)
|
newRootCoordClient func(string, *clientv3.Client) (types.RootCoord, error)
|
||||||
newDataCoordClient func(string, []string) (types.DataCoord, error)
|
newDataCoordClient func(string, *clientv3.Client) (types.DataCoord, error)
|
||||||
|
|
||||||
closer io.Closer
|
closer io.Closer
|
||||||
}
|
}
|
||||||
@ -76,11 +78,11 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
msFactory: factory,
|
msFactory: factory,
|
||||||
grpcErrChan: make(chan error),
|
grpcErrChan: make(chan error),
|
||||||
newRootCoordClient: func(etcdMetaRoot string, etcdEndpoints []string) (types.RootCoord, error) {
|
newRootCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.RootCoord, error) {
|
||||||
return rcc.NewClient(ctx1, etcdMetaRoot, etcdEndpoints)
|
return rcc.NewClient(ctx1, etcdMetaRoot, client)
|
||||||
},
|
},
|
||||||
newDataCoordClient: func(etcdMetaRoot string, etcdEndpoints []string) (types.DataCoord, error) {
|
newDataCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.DataCoord, error) {
|
||||||
return dcc.NewClient(ctx1, etcdMetaRoot, etcdEndpoints)
|
return dcc.NewClient(ctx1, etcdMetaRoot, client)
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -131,6 +133,10 @@ func (s *Server) startGrpcLoop(listener net.Listener) {
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *Server) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
s.datanode.SetEtcdClient(client)
|
||||||
|
}
|
||||||
|
|
||||||
func (s *Server) SetRootCoordInterface(ms types.RootCoord) error {
|
func (s *Server) SetRootCoordInterface(ms types.RootCoord) error {
|
||||||
return s.datanode.SetRootCoord(ms)
|
return s.datanode.SetRootCoord(ms)
|
||||||
}
|
}
|
||||||
@ -162,6 +168,9 @@ func (s *Server) Stop() error {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
s.cancel()
|
s.cancel()
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
log.Debug("Graceful stop grpc server...")
|
log.Debug("Graceful stop grpc server...")
|
||||||
// make graceful stop has a timeout
|
// make graceful stop has a timeout
|
||||||
@ -198,12 +207,19 @@ func (s *Server) init() error {
|
|||||||
dn.Params.DataNodeCfg.Port = Params.Port
|
dn.Params.DataNodeCfg.Port = Params.Port
|
||||||
dn.Params.DataNodeCfg.IP = Params.IP
|
dn.Params.DataNodeCfg.IP = Params.IP
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&dn.Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("DataNode connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.SetEtcdClient(s.etcdCli)
|
||||||
closer := trace.InitTracing(fmt.Sprintf("data_node ip: %s, port: %d", Params.IP, Params.Port))
|
closer := trace.InitTracing(fmt.Sprintf("data_node ip: %s, port: %d", Params.IP, Params.Port))
|
||||||
s.closer = closer
|
s.closer = closer
|
||||||
addr := Params.IP + ":" + strconv.Itoa(Params.Port)
|
addr := Params.IP + ":" + strconv.Itoa(Params.Port)
|
||||||
log.Debug("DataNode address", zap.String("address", addr))
|
log.Debug("DataNode address", zap.String("address", addr))
|
||||||
|
|
||||||
err := s.startGrpc()
|
err = s.startGrpc()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -211,7 +227,7 @@ func (s *Server) init() error {
|
|||||||
// --- RootCoord Client ---
|
// --- RootCoord Client ---
|
||||||
if s.newRootCoordClient != nil {
|
if s.newRootCoordClient != nil {
|
||||||
log.Debug("Init root coord client ...")
|
log.Debug("Init root coord client ...")
|
||||||
rootCoordClient, err := s.newRootCoordClient(dn.Params.DataNodeCfg.MetaRootPath, dn.Params.DataNodeCfg.EtcdEndpoints)
|
rootCoordClient, err := s.newRootCoordClient(dn.Params.DataNodeCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("DataNode newRootCoordClient failed", zap.Error(err))
|
log.Debug("DataNode newRootCoordClient failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -238,7 +254,7 @@ func (s *Server) init() error {
|
|||||||
// --- Data Server Client ---
|
// --- Data Server Client ---
|
||||||
if s.newDataCoordClient != nil {
|
if s.newDataCoordClient != nil {
|
||||||
log.Debug("DataNode Init data service client ...")
|
log.Debug("DataNode Init data service client ...")
|
||||||
dataCoordClient, err := s.newDataCoordClient(dn.Params.DataNodeCfg.MetaRootPath, dn.Params.DataNodeCfg.EtcdEndpoints)
|
dataCoordClient, err := s.newDataCoordClient(dn.Params.DataNodeCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("DataNode newDataCoordClient failed", zap.Error(err))
|
log.Debug("DataNode newDataCoordClient failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
|
@ -29,6 +29,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
@ -107,6 +108,9 @@ func (m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPla
|
|||||||
return m.status, m.err
|
return m.status, m.err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
type mockDataCoord struct {
|
type mockDataCoord struct {
|
||||||
types.DataCoord
|
types.DataCoord
|
||||||
@ -174,11 +178,11 @@ func Test_NewServer(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, server)
|
assert.NotNil(t, server)
|
||||||
|
|
||||||
server.newRootCoordClient = func(string, []string) (types.RootCoord, error) {
|
server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) {
|
||||||
return &mockRootCoord{}, nil
|
return &mockRootCoord{}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
server.newDataCoordClient = func(string, []string) (types.DataCoord, error) {
|
server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) {
|
||||||
return &mockDataCoord{}, nil
|
return &mockDataCoord{}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -247,11 +251,11 @@ func Test_Run(t *testing.T) {
|
|||||||
regErr: errors.New("error"),
|
regErr: errors.New("error"),
|
||||||
}
|
}
|
||||||
|
|
||||||
server.newRootCoordClient = func(string, []string) (types.RootCoord, error) {
|
server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) {
|
||||||
return &mockRootCoord{}, nil
|
return &mockRootCoord{}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
server.newDataCoordClient = func(string, []string) (types.DataCoord, error) {
|
server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) {
|
||||||
return &mockDataCoord{}, nil
|
return &mockDataCoord{}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -30,6 +30,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
@ -43,8 +44,8 @@ type Client struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewClient creates a new IndexCoord client.
|
// NewClient creates a new IndexCoord client.
|
||||||
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) {
|
func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
|
||||||
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints)
|
sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
|
||||||
if sess == nil {
|
if sess == nil {
|
||||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||||
log.Debug("IndexCoordClient NewClient failed", zap.Error(err))
|
log.Debug("IndexCoordClient NewClient failed", zap.Error(err))
|
||||||
|
@ -26,6 +26,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
@ -42,7 +43,9 @@ func TestIndexCoordClient(t *testing.T) {
|
|||||||
err = server.Run()
|
err = server.Run()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
icc, err := NewClient(ctx, indexcoord.Params.IndexCoordCfg.MetaRootPath, indexcoord.Params.IndexCoordCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
icc, err := NewClient(ctx, indexcoord.Params.IndexCoordCfg.MetaRootPath, etcdCli)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, icc)
|
assert.NotNil(t, icc)
|
||||||
|
|
||||||
|
@ -32,10 +32,12 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -49,7 +51,7 @@ type UniqueID = typeutil.UniqueID
|
|||||||
|
|
||||||
// Server is the grpc wrapper of IndexCoord.
|
// Server is the grpc wrapper of IndexCoord.
|
||||||
type Server struct {
|
type Server struct {
|
||||||
indexcoord types.IndexCoord
|
indexcoord types.IndexCoordComponent
|
||||||
|
|
||||||
grpcServer *grpc.Server
|
grpcServer *grpc.Server
|
||||||
grpcErrChan chan error
|
grpcErrChan chan error
|
||||||
@ -58,6 +60,8 @@ type Server struct {
|
|||||||
loopCancel func()
|
loopCancel func()
|
||||||
loopWg sync.WaitGroup
|
loopWg sync.WaitGroup
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
|
|
||||||
closer io.Closer
|
closer io.Closer
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -86,6 +90,14 @@ func (s *Server) init() error {
|
|||||||
closer := trace.InitTracing("IndexCoord")
|
closer := trace.InitTracing("IndexCoord")
|
||||||
s.closer = closer
|
s.closer = closer
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("IndexCoord connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.indexcoord.SetEtcdClient(s.etcdCli)
|
||||||
|
|
||||||
s.loopWg.Add(1)
|
s.loopWg.Add(1)
|
||||||
go s.startGrpcLoop(indexcoord.Params.IndexCoordCfg.Port)
|
go s.startGrpcLoop(indexcoord.Params.IndexCoordCfg.Port)
|
||||||
// wait for grpc IndexCoord loop start
|
// wait for grpc IndexCoord loop start
|
||||||
@ -126,7 +138,9 @@ func (s *Server) Stop() error {
|
|||||||
if s.indexcoord != nil {
|
if s.indexcoord != nil {
|
||||||
s.indexcoord.Stop()
|
s.indexcoord.Stop()
|
||||||
}
|
}
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
s.loopCancel()
|
s.loopCancel()
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
log.Debug("Graceful stop grpc server...")
|
log.Debug("Graceful stop grpc server...")
|
||||||
@ -138,7 +152,7 @@ func (s *Server) Stop() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// SetClient sets the IndexCoord's instance.
|
// SetClient sets the IndexCoord's instance.
|
||||||
func (s *Server) SetClient(indexCoordClient types.IndexCoord) error {
|
func (s *Server) SetClient(indexCoordClient types.IndexCoordComponent) error {
|
||||||
s.indexcoord = indexCoordClient
|
s.indexcoord = indexCoordClient
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -25,6 +25,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
|
etcd "github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -33,7 +34,11 @@ func TestIndexCoordinateServer(t *testing.T) {
|
|||||||
server, err := NewServer(ctx)
|
server, err := NewServer(ctx)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, server)
|
assert.NotNil(t, server)
|
||||||
|
Params.Init()
|
||||||
|
etcd, err := etcd.GetEtcdClient(&Params.BaseParamTable)
|
||||||
|
assert.NoError(t, err)
|
||||||
indexCoordClient := &indexcoord.Mock{}
|
indexCoordClient := &indexcoord.Mock{}
|
||||||
|
indexCoordClient.SetEtcdClient(etcd)
|
||||||
err = server.SetClient(indexCoordClient)
|
err = server.SetClient(indexCoordClient)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = server.Run()
|
err = server.Run()
|
||||||
|
@ -26,13 +26,17 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/mock"
|
"github.com/milvus-io/milvus/internal/util/mock"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var ParamsGlobal paramtable.GlobalParamTable
|
||||||
|
|
||||||
func Test_NewClient(t *testing.T) {
|
func Test_NewClient(t *testing.T) {
|
||||||
ClientParams.InitOnce(typeutil.IndexNodeRole)
|
ClientParams.InitOnce(typeutil.IndexNodeRole)
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
@ -123,6 +127,10 @@ func TestIndexNodeClient(t *testing.T) {
|
|||||||
assert.NotNil(t, ins)
|
assert.NotNil(t, ins)
|
||||||
|
|
||||||
inm := &indexnode.Mock{}
|
inm := &indexnode.Mock{}
|
||||||
|
ParamsGlobal.InitOnce()
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&ParamsGlobal.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
inm.SetEtcdClient(etcdCli)
|
||||||
err = ins.SetClient(inm)
|
err = ins.SetClient(inm)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
@ -33,10 +33,12 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -46,7 +48,7 @@ var Params paramtable.GrpcServerConfig
|
|||||||
|
|
||||||
// Server is the grpc wrapper of IndexNode.
|
// Server is the grpc wrapper of IndexNode.
|
||||||
type Server struct {
|
type Server struct {
|
||||||
indexnode types.IndexNode
|
indexnode types.IndexNodeComponent
|
||||||
|
|
||||||
grpcServer *grpc.Server
|
grpcServer *grpc.Server
|
||||||
grpcErrChan chan error
|
grpcErrChan chan error
|
||||||
@ -55,7 +57,8 @@ type Server struct {
|
|||||||
loopCancel func()
|
loopCancel func()
|
||||||
loopWg sync.WaitGroup
|
loopWg sync.WaitGroup
|
||||||
|
|
||||||
closer io.Closer
|
etcdCli *clientv3.Client
|
||||||
|
closer io.Closer
|
||||||
}
|
}
|
||||||
|
|
||||||
// Run initializes and starts IndexNode's grpc service.
|
// Run initializes and starts IndexNode's grpc service.
|
||||||
@ -142,6 +145,13 @@ func (s *Server) init() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&indexnode.Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("IndexNode connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.indexnode.SetEtcdClient(etcdCli)
|
||||||
err = s.indexnode.Init()
|
err = s.indexnode.Init()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("IndexNode Init failed", zap.Error(err))
|
log.Error("IndexNode Init failed", zap.Error(err))
|
||||||
@ -178,6 +188,9 @@ func (s *Server) Stop() error {
|
|||||||
if s.indexnode != nil {
|
if s.indexnode != nil {
|
||||||
s.indexnode.Stop()
|
s.indexnode.Stop()
|
||||||
}
|
}
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
log.Debug("Graceful stop grpc server...")
|
log.Debug("Graceful stop grpc server...")
|
||||||
s.grpcServer.GracefulStop()
|
s.grpcServer.GracefulStop()
|
||||||
@ -188,11 +201,16 @@ func (s *Server) Stop() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// SetClient sets the IndexNode's instance.
|
// SetClient sets the IndexNode's instance.
|
||||||
func (s *Server) SetClient(indexNodeClient types.IndexNode) error {
|
func (s *Server) SetClient(indexNodeClient types.IndexNodeComponent) error {
|
||||||
s.indexnode = indexNodeClient
|
s.indexnode = indexNodeClient
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient sets the etcd client for QueryNode component.
|
||||||
|
func (s *Server) SetEtcdClient(etcdCli *clientv3.Client) {
|
||||||
|
s.indexnode.SetEtcdClient(etcdCli)
|
||||||
|
}
|
||||||
|
|
||||||
// GetComponentStates gets the component states of IndexNode.
|
// GetComponentStates gets the component states of IndexNode.
|
||||||
func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
||||||
return s.indexnode.GetComponentStates(ctx)
|
return s.indexnode.GetComponentStates(ctx)
|
||||||
|
@ -27,9 +27,13 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var ParamsGlobal paramtable.GlobalParamTable
|
||||||
|
|
||||||
func TestIndexNodeServer(t *testing.T) {
|
func TestIndexNodeServer(t *testing.T) {
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
server, err := NewServer(ctx)
|
server, err := NewServer(ctx)
|
||||||
@ -37,6 +41,10 @@ func TestIndexNodeServer(t *testing.T) {
|
|||||||
assert.NotNil(t, server)
|
assert.NotNil(t, server)
|
||||||
|
|
||||||
inm := &indexnode.Mock{}
|
inm := &indexnode.Mock{}
|
||||||
|
ParamsGlobal.InitOnce()
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&ParamsGlobal.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
inm.SetEtcdClient(etcdCli)
|
||||||
err = server.SetClient(inm)
|
err = server.SetClient(inm)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
@ -38,11 +38,13 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||||
"github.com/milvus-io/milvus/internal/proxy"
|
"github.com/milvus-io/milvus/internal/proxy"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/opentracing/opentracing-go"
|
"github.com/opentracing/opentracing-go"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -59,6 +61,7 @@ type Server struct {
|
|||||||
|
|
||||||
grpcErrChan chan error
|
grpcErrChan chan error
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
rootCoordClient types.RootCoord
|
rootCoordClient types.RootCoord
|
||||||
dataCoordClient types.DataCoord
|
dataCoordClient types.DataCoord
|
||||||
queryCoordClient types.QueryCoord
|
queryCoordClient types.QueryCoord
|
||||||
@ -172,6 +175,13 @@ func (s *Server) init() error {
|
|||||||
s.closer = closer
|
s.closer = closer
|
||||||
log.Debug("init Proxy's tracer done", zap.String("service name", serviceName))
|
log.Debug("init Proxy's tracer done", zap.String("service name", serviceName))
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("Proxy connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.proxy.SetEtcdClient(s.etcdCli)
|
||||||
s.wg.Add(1)
|
s.wg.Add(1)
|
||||||
go s.startGrpcLoop(Params.Port)
|
go s.startGrpcLoop(Params.Port)
|
||||||
log.Debug("waiting for grpc server of Proxy to be started")
|
log.Debug("waiting for grpc server of Proxy to be started")
|
||||||
@ -184,7 +194,7 @@ func (s *Server) init() error {
|
|||||||
if s.rootCoordClient == nil {
|
if s.rootCoordClient == nil {
|
||||||
var err error
|
var err error
|
||||||
log.Debug("create RootCoord client for Proxy")
|
log.Debug("create RootCoord client for Proxy")
|
||||||
s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("failed to create RootCoord client for Proxy", zap.Error(err))
|
log.Warn("failed to create RootCoord client for Proxy", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
@ -213,7 +223,7 @@ func (s *Server) init() error {
|
|||||||
if s.dataCoordClient == nil {
|
if s.dataCoordClient == nil {
|
||||||
var err error
|
var err error
|
||||||
log.Debug("create DataCoord client for Proxy")
|
log.Debug("create DataCoord client for Proxy")
|
||||||
s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("failed to create DataCoord client for Proxy", zap.Error(err))
|
log.Warn("failed to create DataCoord client for Proxy", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
@ -242,7 +252,7 @@ func (s *Server) init() error {
|
|||||||
if s.indexCoordClient == nil {
|
if s.indexCoordClient == nil {
|
||||||
var err error
|
var err error
|
||||||
log.Debug("create IndexCoord client for Proxy")
|
log.Debug("create IndexCoord client for Proxy")
|
||||||
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err))
|
log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
@ -271,7 +281,7 @@ func (s *Server) init() error {
|
|||||||
if s.queryCoordClient == nil {
|
if s.queryCoordClient == nil {
|
||||||
var err error
|
var err error
|
||||||
log.Debug("create QueryCoord client for Proxy")
|
log.Debug("create QueryCoord client for Proxy")
|
||||||
s.queryCoordClient, err = qcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
s.queryCoordClient, err = qcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err))
|
log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
@ -334,6 +344,10 @@ func (s *Server) Stop() error {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
|
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
log.Debug("Graceful stop grpc server...")
|
log.Debug("Graceful stop grpc server...")
|
||||||
s.grpcServer.GracefulStop()
|
s.grpcServer.GracefulStop()
|
||||||
|
@ -32,6 +32,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
@ -616,6 +617,9 @@ func (m *MockProxy) UpdateStateCode(stateCode internalpb.StateCode) {
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockProxy) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *MockProxy) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
|
func (m *MockProxy) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
|
||||||
return nil, nil
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
@ -30,6 +30,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
@ -43,8 +44,8 @@ type Client struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewClient creates a client for QueryCoord grpc call.
|
// NewClient creates a client for QueryCoord grpc call.
|
||||||
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) {
|
func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
|
||||||
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints)
|
sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
|
||||||
if sess == nil {
|
if sess == nil {
|
||||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||||
log.Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
log.Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
||||||
|
@ -25,6 +25,7 @@ import (
|
|||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proxy"
|
"github.com/milvus-io/milvus/internal/proxy"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -32,7 +33,10 @@ func Test_NewClient(t *testing.T) {
|
|||||||
proxy.Params.InitOnce()
|
proxy.Params.InitOnce()
|
||||||
|
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, client)
|
assert.NotNil(t, client)
|
||||||
|
|
||||||
|
@ -36,10 +36,12 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
qc "github.com/milvus-io/milvus/internal/querycoord"
|
qc "github.com/milvus-io/milvus/internal/querycoord"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -60,6 +62,8 @@ type Server struct {
|
|||||||
|
|
||||||
msFactory msgstream.Factory
|
msFactory msgstream.Factory
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
|
|
||||||
dataCoord types.DataCoord
|
dataCoord types.DataCoord
|
||||||
rootCoord types.RootCoord
|
rootCoord types.RootCoord
|
||||||
indexCoord types.IndexCoord
|
indexCoord types.IndexCoord
|
||||||
@ -111,17 +115,25 @@ func (s *Server) init() error {
|
|||||||
closer := trace.InitTracing("querycoord")
|
closer := trace.InitTracing("querycoord")
|
||||||
s.closer = closer
|
s.closer = closer
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("QueryCoord connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.SetEtcdClient(etcdCli)
|
||||||
|
|
||||||
s.wg.Add(1)
|
s.wg.Add(1)
|
||||||
go s.startGrpcLoop(Params.Port)
|
go s.startGrpcLoop(Params.Port)
|
||||||
// wait for grpc server loop start
|
// wait for grpc server loop start
|
||||||
err := <-s.grpcErrChan
|
err = <-s.grpcErrChan
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// --- Master Server Client ---
|
// --- Master Server Client ---
|
||||||
if s.rootCoord == nil {
|
if s.rootCoord == nil {
|
||||||
s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints)
|
s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err))
|
log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -152,7 +164,7 @@ func (s *Server) init() error {
|
|||||||
|
|
||||||
// --- Data service client ---
|
// --- Data service client ---
|
||||||
if s.dataCoord == nil {
|
if s.dataCoord == nil {
|
||||||
s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints)
|
s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err))
|
log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -180,7 +192,7 @@ func (s *Server) init() error {
|
|||||||
|
|
||||||
// --- IndexCoord ---
|
// --- IndexCoord ---
|
||||||
if s.indexCoord == nil {
|
if s.indexCoord == nil {
|
||||||
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints)
|
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
|
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -273,6 +285,9 @@ func (s *Server) Stop() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
err := s.queryCoord.Stop()
|
err := s.queryCoord.Stop()
|
||||||
s.loopCancel()
|
s.loopCancel()
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
@ -282,6 +297,11 @@ func (s *Server) Stop() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetRootCoord sets root coordinator's client
|
||||||
|
func (s *Server) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
s.queryCoord.SetEtcdClient(etcdClient)
|
||||||
|
}
|
||||||
|
|
||||||
// SetRootCoord sets the RootCoord's client for QueryCoord component.
|
// SetRootCoord sets the RootCoord's client for QueryCoord component.
|
||||||
func (s *Server) SetRootCoord(m types.RootCoord) error {
|
func (s *Server) SetRootCoord(m types.RootCoord) error {
|
||||||
s.queryCoord.SetRootCoord(m)
|
s.queryCoord.SetRootCoord(m)
|
||||||
|
@ -29,6 +29,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
@ -69,6 +70,9 @@ func (m *MockQueryCoord) Register() error {
|
|||||||
func (m *MockQueryCoord) UpdateStateCode(code internalpb.StateCode) {
|
func (m *MockQueryCoord) UpdateStateCode(code internalpb.StateCode) {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockQueryCoord) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *MockQueryCoord) SetRootCoord(types.RootCoord) error {
|
func (m *MockQueryCoord) SetRootCoord(types.RootCoord) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -36,11 +36,13 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
qn "github.com/milvus-io/milvus/internal/querynode"
|
qn "github.com/milvus-io/milvus/internal/querynode"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
@ -61,6 +63,7 @@ type Server struct {
|
|||||||
|
|
||||||
grpcServer *grpc.Server
|
grpcServer *grpc.Server
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
rootCoord types.RootCoord
|
rootCoord types.RootCoord
|
||||||
indexCoord types.IndexCoord
|
indexCoord types.IndexCoord
|
||||||
|
|
||||||
@ -93,17 +96,26 @@ func (s *Server) init() error {
|
|||||||
s.closer = closer
|
s.closer = closer
|
||||||
|
|
||||||
log.Debug("QueryNode", zap.Int("port", Params.Port))
|
log.Debug("QueryNode", zap.Int("port", Params.Port))
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("QueryNode connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.SetEtcdClient(etcdCli)
|
||||||
|
log.Debug("QueryNode connect to etcd successfully")
|
||||||
s.wg.Add(1)
|
s.wg.Add(1)
|
||||||
go s.startGrpcLoop(Params.Port)
|
go s.startGrpcLoop(Params.Port)
|
||||||
// wait for grpc server loop start
|
// wait for grpc server loop start
|
||||||
err := <-s.grpcErrChan
|
err = <-s.grpcErrChan
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// --- RootCoord Client ---
|
// --- RootCoord Client ---
|
||||||
if s.rootCoord == nil {
|
if s.rootCoord == nil {
|
||||||
s.rootCoord, err = rcc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, qn.Params.QueryNodeCfg.EtcdEndpoints)
|
s.rootCoord, err = rcc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("QueryNode new RootCoordClient failed", zap.Error(err))
|
log.Debug("QueryNode new RootCoordClient failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -133,7 +145,7 @@ func (s *Server) init() error {
|
|||||||
|
|
||||||
// --- IndexCoord ---
|
// --- IndexCoord ---
|
||||||
if s.indexCoord == nil {
|
if s.indexCoord == nil {
|
||||||
s.indexCoord, err = icc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, qn.Params.QueryNodeCfg.EtcdEndpoints)
|
s.indexCoord, err = icc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, s.etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("QueryNode new IndexCoordClient failed", zap.Error(err))
|
log.Debug("QueryNode new IndexCoordClient failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -260,6 +272,9 @@ func (s *Server) Stop() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
|
|
||||||
s.cancel()
|
s.cancel()
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
@ -275,6 +290,11 @@ func (s *Server) Stop() error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient sets the etcd client for QueryNode component.
|
||||||
|
func (s *Server) SetEtcdClient(etcdCli *clientv3.Client) {
|
||||||
|
s.querynode.SetEtcdClient(etcdCli)
|
||||||
|
}
|
||||||
|
|
||||||
// SetRootCoord sets the RootCoord's client for QueryNode component.
|
// SetRootCoord sets the RootCoord's client for QueryNode component.
|
||||||
func (s *Server) SetRootCoord(rootCoord types.RootCoord) error {
|
func (s *Server) SetRootCoord(rootCoord types.RootCoord) error {
|
||||||
return s.querynode.SetRootCoord(rootCoord)
|
return s.querynode.SetRootCoord(rootCoord)
|
||||||
|
@ -28,6 +28,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||||
@ -112,6 +113,9 @@ func (m *MockQueryNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetrics
|
|||||||
return m.metricResp, m.err
|
return m.metricResp, m.err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockQueryNode) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *MockQueryNode) UpdateStateCode(code internalpb.StateCode) {
|
func (m *MockQueryNode) UpdateStateCode(code internalpb.StateCode) {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -149,6 +153,9 @@ func (m *MockRootCoord) Register() error {
|
|||||||
return m.regErr
|
return m.regErr
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockRootCoord) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||||
return &internalpb.ComponentStates{
|
return &internalpb.ComponentStates{
|
||||||
State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy},
|
State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy},
|
||||||
@ -182,6 +189,9 @@ func (m *MockIndexCoord) Register() error {
|
|||||||
return m.regErr
|
return m.regErr
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *MockIndexCoord) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||||
return &internalpb.ComponentStates{
|
return &internalpb.ComponentStates{
|
||||||
State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy},
|
State: &internalpb.ComponentInfo{StateCode: internalpb.StateCode_Healthy},
|
||||||
|
@ -32,6 +32,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
)
|
)
|
||||||
@ -49,8 +50,8 @@ type Client struct {
|
|||||||
// metaRoot is the path in etcd for root coordinator registration
|
// metaRoot is the path in etcd for root coordinator registration
|
||||||
// etcdEndpoints are the address list for etcd end points
|
// etcdEndpoints are the address list for etcd end points
|
||||||
// timeout is default setting for each grpc call
|
// timeout is default setting for each grpc call
|
||||||
func NewClient(ctx context.Context, metaRoot string, etcdEndpoints []string) (*Client, error) {
|
func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
|
||||||
sess := sessionutil.NewSession(ctx, metaRoot, etcdEndpoints)
|
sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
|
||||||
if sess == nil {
|
if sess == nil {
|
||||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||||
log.Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
log.Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
||||||
|
@ -25,6 +25,7 @@ import (
|
|||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proxy"
|
"github.com/milvus-io/milvus/internal/proxy"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -32,7 +33,9 @@ func Test_NewClient(t *testing.T) {
|
|||||||
proxy.Params.InitOnce()
|
proxy.Params.InitOnce()
|
||||||
|
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, etcdCli)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, client)
|
assert.NotNil(t, client)
|
||||||
|
|
||||||
|
@ -25,10 +25,7 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||||
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
|
|
||||||
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
|
|
||||||
pnc "github.com/milvus-io/milvus/internal/distributed/proxy/client"
|
pnc "github.com/milvus-io/milvus/internal/distributed/proxy/client"
|
||||||
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
|
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
@ -39,14 +36,20 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/keepalive"
|
"google.golang.org/grpc/keepalive"
|
||||||
|
|
||||||
|
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
|
||||||
|
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
|
||||||
|
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
|
||||||
)
|
)
|
||||||
|
|
||||||
var Params paramtable.GrpcServerConfig
|
var Params paramtable.GrpcServerConfig
|
||||||
@ -62,13 +65,14 @@ type Server struct {
|
|||||||
ctx context.Context
|
ctx context.Context
|
||||||
cancel context.CancelFunc
|
cancel context.CancelFunc
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
dataCoord types.DataCoord
|
dataCoord types.DataCoord
|
||||||
indexCoord types.IndexCoord
|
indexCoord types.IndexCoord
|
||||||
queryCoord types.QueryCoord
|
queryCoord types.QueryCoord
|
||||||
|
|
||||||
newIndexCoordClient func(string, []string) types.IndexCoord
|
newIndexCoordClient func(string, *clientv3.Client) types.IndexCoord
|
||||||
newDataCoordClient func(string, []string) types.DataCoord
|
newDataCoordClient func(string, *clientv3.Client) types.DataCoord
|
||||||
newQueryCoordClient func(string, []string) types.QueryCoord
|
newQueryCoordClient func(string, *clientv3.Client) types.QueryCoord
|
||||||
|
|
||||||
closer io.Closer
|
closer io.Closer
|
||||||
}
|
}
|
||||||
@ -106,22 +110,22 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) setClient() {
|
func (s *Server) setClient() {
|
||||||
s.newDataCoordClient = func(etcdMetaRoot string, etcdEndpoints []string) types.DataCoord {
|
s.newDataCoordClient = func(etcdMetaRoot string, etcdCli *clientv3.Client) types.DataCoord {
|
||||||
dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdEndpoints)
|
dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
return dsClient
|
return dsClient
|
||||||
}
|
}
|
||||||
s.newIndexCoordClient = func(metaRootPath string, etcdEndpoints []string) types.IndexCoord {
|
s.newIndexCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.IndexCoord {
|
||||||
isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdEndpoints)
|
isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
return isClient
|
return isClient
|
||||||
}
|
}
|
||||||
s.newQueryCoordClient = func(metaRootPath string, etcdEndpoints []string) types.QueryCoord {
|
s.newQueryCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.QueryCoord {
|
||||||
qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdEndpoints)
|
qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdCli)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -149,17 +153,25 @@ func (s *Server) init() error {
|
|||||||
rootcoord.Params.InitOnce()
|
rootcoord.Params.InitOnce()
|
||||||
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
|
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
|
||||||
rootcoord.Params.RootCoordCfg.Port = Params.Port
|
rootcoord.Params.RootCoordCfg.Port = Params.Port
|
||||||
log.Debug("grpc init done ...")
|
log.Debug("init params done..")
|
||||||
|
|
||||||
closer := trace.InitTracing("root_coord")
|
closer := trace.InitTracing("root_coord")
|
||||||
s.closer = closer
|
s.closer = closer
|
||||||
|
|
||||||
log.Debug("init params done")
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
|
||||||
|
if err != nil {
|
||||||
|
log.Debug("RootCoord connect to etcd failed", zap.Error(err))
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
s.etcdCli = etcdCli
|
||||||
|
s.rootCoord.SetEtcdClient(s.etcdCli)
|
||||||
|
log.Debug("etcd connect done ...")
|
||||||
|
|
||||||
err := s.startGrpc(Params.Port)
|
err = s.startGrpc(Params.Port)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
log.Debug("grpc init done ...")
|
||||||
|
|
||||||
s.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
|
s.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||||
log.Debug("RootCoord", zap.Any("State", internalpb.StateCode_Initializing))
|
log.Debug("RootCoord", zap.Any("State", internalpb.StateCode_Initializing))
|
||||||
@ -181,7 +193,7 @@ func (s *Server) init() error {
|
|||||||
|
|
||||||
if s.newDataCoordClient != nil {
|
if s.newDataCoordClient != nil {
|
||||||
log.Debug("RootCoord start to create DataCoord client")
|
log.Debug("RootCoord start to create DataCoord client")
|
||||||
dataCoord := s.newDataCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
dataCoord := s.newDataCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil {
|
if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -189,7 +201,7 @@ func (s *Server) init() error {
|
|||||||
}
|
}
|
||||||
if s.newIndexCoordClient != nil {
|
if s.newIndexCoordClient != nil {
|
||||||
log.Debug("RootCoord start to create IndexCoord client")
|
log.Debug("RootCoord start to create IndexCoord client")
|
||||||
indexCoord := s.newIndexCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
indexCoord := s.newIndexCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil {
|
if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -197,7 +209,7 @@ func (s *Server) init() error {
|
|||||||
}
|
}
|
||||||
if s.newQueryCoordClient != nil {
|
if s.newQueryCoordClient != nil {
|
||||||
log.Debug("RootCoord start to create QueryCoord client")
|
log.Debug("RootCoord start to create QueryCoord client")
|
||||||
queryCoord := s.newQueryCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
queryCoord := s.newQueryCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, s.etcdCli)
|
||||||
if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil {
|
if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -273,6 +285,9 @@ func (s *Server) Stop() error {
|
|||||||
log.Error("Failed to close opentracing", zap.Error(err))
|
log.Error("Failed to close opentracing", zap.Error(err))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if s.etcdCli != nil {
|
||||||
|
defer s.etcdCli.Close()
|
||||||
|
}
|
||||||
if s.indexCoord != nil {
|
if s.indexCoord != nil {
|
||||||
if err := s.indexCoord.Stop(); err != nil {
|
if err := s.indexCoord.Stop(); err != nil {
|
||||||
log.Error("Failed to close indexCoord client", zap.Error(err))
|
log.Error("Failed to close indexCoord client", zap.Error(err))
|
||||||
|
@ -41,6 +41,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
@ -98,7 +99,7 @@ func TestGrpcService(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
|
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||||
|
|
||||||
etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||||
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||||
@ -115,6 +116,7 @@ func TestGrpcService(t *testing.T) {
|
|||||||
|
|
||||||
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
|
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
|
||||||
|
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -214,7 +216,7 @@ func TestGrpcService(t *testing.T) {
|
|||||||
|
|
||||||
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Healthy)
|
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Healthy)
|
||||||
|
|
||||||
cli, err := rcc.NewClient(context.Background(), rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
cli, err := rcc.NewClient(context.Background(), rootcoord.Params.RootCoordCfg.MetaRootPath, etcdCli)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
err = cli.Init()
|
err = cli.Init()
|
||||||
@ -794,6 +796,9 @@ type mockCore struct {
|
|||||||
func (m *mockCore) UpdateStateCode(internalpb.StateCode) {
|
func (m *mockCore) UpdateStateCode(internalpb.StateCode) {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *mockCore) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
}
|
||||||
|
|
||||||
func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error {
|
func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -895,13 +900,13 @@ func TestRun(t *testing.T) {
|
|||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
assert.EqualError(t, err, "listen tcp: address 1000000: invalid port")
|
assert.EqualError(t, err, "listen tcp: address 1000000: invalid port")
|
||||||
|
|
||||||
svr.newDataCoordClient = func(string, []string) types.DataCoord {
|
svr.newDataCoordClient = func(string, *clientv3.Client) types.DataCoord {
|
||||||
return &mockDataCoord{}
|
return &mockDataCoord{}
|
||||||
}
|
}
|
||||||
svr.newIndexCoordClient = func(string, []string) types.IndexCoord {
|
svr.newIndexCoordClient = func(string, *clientv3.Client) types.IndexCoord {
|
||||||
return &mockIndex{}
|
return &mockIndex{}
|
||||||
}
|
}
|
||||||
svr.newQueryCoordClient = func(string, []string) types.QueryCoord {
|
svr.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord {
|
||||||
return &mockQuery{}
|
return &mockQuery{}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -912,7 +917,7 @@ func TestRun(t *testing.T) {
|
|||||||
rootcoord.Params.Init()
|
rootcoord.Params.Init()
|
||||||
rootcoord.Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
|
rootcoord.Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
|
||||||
|
|
||||||
etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParamTable)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||||
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||||
|
@ -50,6 +50,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
// make sure IndexCoord implements types.IndexCoord
|
// make sure IndexCoord implements types.IndexCoord
|
||||||
@ -79,7 +80,8 @@ type IndexCoord struct {
|
|||||||
|
|
||||||
idAllocator *allocator.GlobalIDAllocator
|
idAllocator *allocator.GlobalIDAllocator
|
||||||
|
|
||||||
kv kv.BaseKV
|
etcdCli *clientv3.Client
|
||||||
|
kv kv.BaseKV
|
||||||
|
|
||||||
metaTable *metaTable
|
metaTable *metaTable
|
||||||
nodeManager *NodeManager
|
nodeManager *NodeManager
|
||||||
@ -129,17 +131,19 @@ func (i *IndexCoord) Register() error {
|
|||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if i.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (i *IndexCoord) initSession() error {
|
func (i *IndexCoord) initSession() error {
|
||||||
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints)
|
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexCoordCfg.MetaRootPath, i.etcdCli)
|
||||||
if i.session == nil {
|
if i.session == nil {
|
||||||
return errors.New("failed to initialize session")
|
return errors.New("failed to initialize session")
|
||||||
}
|
}
|
||||||
i.session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true)
|
i.session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true, true)
|
||||||
Params.BaseParams.SetLogger(i.session.ServerID)
|
Params.BaseParams.SetLogger(i.session.ServerID)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -160,10 +164,7 @@ func (i *IndexCoord) Init() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
connectEtcdFn := func() error {
|
connectEtcdFn := func() error {
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.IndexCoordCfg.MetaRootPath)
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
metakv, err := NewMetaTable(etcdKV)
|
metakv, err := NewMetaTable(etcdKV)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
@ -207,12 +208,7 @@ func (i *IndexCoord) Init() error {
|
|||||||
|
|
||||||
//init idAllocator
|
//init idAllocator
|
||||||
kvRootPath := Params.IndexCoordCfg.KvRootPath
|
kvRootPath := Params.IndexCoordCfg.KvRootPath
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(Params.IndexCoordCfg.EtcdEndpoints, kvRootPath, "index_gid")
|
etcdKV := tsoutil.NewTSOKVBase(i.etcdCli, kvRootPath, "index_gid")
|
||||||
if err != nil {
|
|
||||||
log.Error("IndexCoord TSOKVBase initialize failed", zap.Error(err))
|
|
||||||
initErr = err
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
i.idAllocator = allocator.NewGlobalIDAllocator("idTimestamp", etcdKV)
|
i.idAllocator = allocator.NewGlobalIDAllocator("idTimestamp", etcdKV)
|
||||||
if err := i.idAllocator.Initialize(); err != nil {
|
if err := i.idAllocator.Initialize(); err != nil {
|
||||||
@ -306,6 +302,10 @@ func (i *IndexCoord) Stop() error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (i *IndexCoord) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
i.etcdCli = etcdClient
|
||||||
|
}
|
||||||
|
|
||||||
// UpdateStateCode updates the component state of IndexCoord.
|
// UpdateStateCode updates the component state of IndexCoord.
|
||||||
func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) {
|
func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) {
|
||||||
i.stateCode.Store(code)
|
i.stateCode.Store(code)
|
||||||
|
@ -28,12 +28,13 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Mock is an alternative to IndexCoord, it will return specific results based on specific parameters.
|
// Mock is an alternative to IndexCoord, it will return specific results based on specific parameters.
|
||||||
type Mock struct {
|
type Mock struct {
|
||||||
etcdKV *etcdkv.EtcdKV
|
etcdKV *etcdkv.EtcdKV
|
||||||
|
etcdCli *clientv3.Client
|
||||||
Failure bool
|
Failure bool
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -67,14 +68,24 @@ func (icm *Mock) Register() error {
|
|||||||
if icm.Failure {
|
if icm.Failure {
|
||||||
return errors.New("IndexCoordinate register failed")
|
return errors.New("IndexCoordinate register failed")
|
||||||
}
|
}
|
||||||
icm.etcdKV, _ = etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
icm.etcdKV = etcdkv.NewEtcdKV(icm.etcdCli, Params.IndexCoordCfg.MetaRootPath)
|
||||||
err := icm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole)
|
err := icm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole)
|
||||||
session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints)
|
if err != nil {
|
||||||
session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true)
|
return err
|
||||||
|
}
|
||||||
|
session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, icm.etcdCli)
|
||||||
|
session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (icm *Mock) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
icm.etcdCli = client
|
||||||
|
}
|
||||||
|
|
||||||
|
func (icm *Mock) UpdateStateCode(stateCode internalpb.StateCode) {
|
||||||
|
}
|
||||||
|
|
||||||
// GetComponentStates gets the component states of the mocked IndexCoord, if Param `Failure` is true, it will return an error,
|
// GetComponentStates gets the component states of the mocked IndexCoord, if Param `Failure` is true, it will return an error,
|
||||||
// and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`.
|
// and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`.
|
||||||
func (icm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
func (icm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||||
|
@ -24,13 +24,19 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestIndexCoordMock(t *testing.T) {
|
func TestIndexCoordMock(t *testing.T) {
|
||||||
Params.Init()
|
Params.Init()
|
||||||
icm := Mock{}
|
icm := Mock{}
|
||||||
err := icm.Register()
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
icm.SetEtcdClient(etcdCli)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
err = icm.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = icm.Init()
|
err = icm.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
@ -32,6 +32,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
@ -41,7 +42,11 @@ import (
|
|||||||
func TestIndexCoord(t *testing.T) {
|
func TestIndexCoord(t *testing.T) {
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
inm0 := &indexnode.Mock{}
|
inm0 := &indexnode.Mock{}
|
||||||
err := inm0.Init()
|
Params.Init()
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
inm0.SetEtcdClient(etcdCli)
|
||||||
|
err = inm0.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = inm0.Register()
|
err = inm0.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -53,8 +58,8 @@ func TestIndexCoord(t *testing.T) {
|
|||||||
ic.durationInterval = time.Second
|
ic.durationInterval = time.Second
|
||||||
ic.assignTaskInterval = 200 * time.Millisecond
|
ic.assignTaskInterval = 200 * time.Millisecond
|
||||||
ic.taskLimit = 20
|
ic.taskLimit = 20
|
||||||
Params.Init()
|
|
||||||
|
|
||||||
|
ic.SetEtcdClient(etcdCli)
|
||||||
err = ic.Init()
|
err = ic.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = ic.Register()
|
err = ic.Register()
|
||||||
@ -72,6 +77,8 @@ func TestIndexCoord(t *testing.T) {
|
|||||||
Build: true,
|
Build: true,
|
||||||
Failure: false,
|
Failure: false,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
inm.SetEtcdClient(etcdCli)
|
||||||
err = in.SetClient(inm)
|
err = in.SetClient(inm)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
@ -25,13 +25,17 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestMetaTable(t *testing.T) {
|
func TestMetaTable(t *testing.T) {
|
||||||
Params.Init()
|
Params.Init()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
assert.NoError(t, err)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.IndexCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
req := &indexpb.BuildIndexRequest{
|
req := &indexpb.BuildIndexRequest{
|
||||||
IndexBuildID: 1,
|
IndexBuildID: 1,
|
||||||
@ -310,8 +314,11 @@ func TestMetaTable(t *testing.T) {
|
|||||||
|
|
||||||
func TestMetaTable_Error(t *testing.T) {
|
func TestMetaTable_Error(t *testing.T) {
|
||||||
Params.Init()
|
Params.Init()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
defer etcdCli.Close()
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.IndexCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
t.Run("reloadFromKV error", func(t *testing.T) {
|
t.Run("reloadFromKV error", func(t *testing.T) {
|
||||||
value := "indexMeta-1"
|
value := "indexMeta-1"
|
||||||
|
@ -21,6 +21,7 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/indexnode"
|
"github.com/milvus-io/milvus/internal/indexnode"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
@ -31,6 +32,11 @@ func TestGetSystemInfoMetrics(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
Params.Init()
|
Params.Init()
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
ic.SetEtcdClient(etcdCli)
|
||||||
err = ic.Init()
|
err = ic.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = ic.Start()
|
err = ic.Start()
|
||||||
|
@ -51,10 +51,10 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -64,6 +64,9 @@ type UniqueID = typeutil.UniqueID
|
|||||||
// make sure IndexNode implements types.IndexNode
|
// make sure IndexNode implements types.IndexNode
|
||||||
var _ types.IndexNode = (*IndexNode)(nil)
|
var _ types.IndexNode = (*IndexNode)(nil)
|
||||||
|
|
||||||
|
// make sure IndexNode implements types.IndexNodeComponent
|
||||||
|
var _ types.IndexNodeComponent = (*IndexNode)(nil)
|
||||||
|
|
||||||
var Params paramtable.GlobalParamTable
|
var Params paramtable.GlobalParamTable
|
||||||
|
|
||||||
// IndexNode is a component that executes the task of building indexes.
|
// IndexNode is a component that executes the task of building indexes.
|
||||||
@ -84,6 +87,7 @@ type IndexNode struct {
|
|||||||
startCallbacks []func()
|
startCallbacks []func()
|
||||||
closeCallbacks []func()
|
closeCallbacks []func()
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
etcdKV *etcdkv.EtcdKV
|
etcdKV *etcdkv.EtcdKV
|
||||||
finishedTasks map[UniqueID]commonpb.IndexState
|
finishedTasks map[UniqueID]commonpb.IndexState
|
||||||
|
|
||||||
@ -122,7 +126,9 @@ func (i *IndexNode) Register() error {
|
|||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if i.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -139,11 +145,11 @@ func (i *IndexNode) initKnowhere() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (i *IndexNode) initSession() error {
|
func (i *IndexNode) initSession() error {
|
||||||
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints)
|
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexNodeCfg.MetaRootPath, i.etcdCli)
|
||||||
if i.session == nil {
|
if i.session == nil {
|
||||||
return errors.New("failed to initialize session")
|
return errors.New("failed to initialize session")
|
||||||
}
|
}
|
||||||
i.session.Init(typeutil.IndexNodeRole, Params.IndexNodeCfg.IP+":"+strconv.Itoa(Params.IndexNodeCfg.Port), false)
|
i.session.Init(typeutil.IndexNodeRole, Params.IndexNodeCfg.IP+":"+strconv.Itoa(Params.IndexNodeCfg.Port), false, true)
|
||||||
Params.IndexNodeCfg.NodeID = i.session.ServerID
|
Params.IndexNodeCfg.NodeID = i.session.ServerID
|
||||||
Params.BaseParams.SetLogger(Params.IndexNodeCfg.NodeID)
|
Params.BaseParams.SetLogger(Params.IndexNodeCfg.NodeID)
|
||||||
return nil
|
return nil
|
||||||
@ -165,18 +171,8 @@ func (i *IndexNode) Init() error {
|
|||||||
}
|
}
|
||||||
log.Debug("IndexNode init session successful", zap.Int64("serverID", i.session.ServerID))
|
log.Debug("IndexNode init session successful", zap.Int64("serverID", i.session.ServerID))
|
||||||
|
|
||||||
connectEtcdFn := func() error {
|
etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.IndexNodeCfg.MetaRootPath)
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath)
|
i.etcdKV = etcdKV
|
||||||
i.etcdKV = etcdKV
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
err = retry.Do(i.loopCtx, connectEtcdFn, retry.Attempts(300))
|
|
||||||
if err != nil {
|
|
||||||
log.Error("IndexNode failed to connect to etcd", zap.Error(err))
|
|
||||||
initErr = err
|
|
||||||
return
|
|
||||||
}
|
|
||||||
log.Debug("IndexNode connected to etcd successfully")
|
|
||||||
|
|
||||||
option := &miniokv.Option{
|
option := &miniokv.Option{
|
||||||
Address: Params.IndexNodeCfg.MinIOAddress,
|
Address: Params.IndexNodeCfg.MinIOAddress,
|
||||||
@ -250,6 +246,11 @@ func (i *IndexNode) UpdateStateCode(code internalpb.StateCode) {
|
|||||||
i.stateCode.Store(code)
|
i.stateCode.Store(code)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient assigns parameter client to its member etcdCli
|
||||||
|
func (node *IndexNode) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
node.etcdCli = client
|
||||||
|
}
|
||||||
|
|
||||||
func (i *IndexNode) isHealthy() bool {
|
func (i *IndexNode) isHealthy() bool {
|
||||||
code := i.stateCode.Load().(internalpb.StateCode)
|
code := i.stateCode.Load().(internalpb.StateCode)
|
||||||
return code == internalpb.StateCode_Healthy
|
return code == internalpb.StateCode_Healthy
|
||||||
|
@ -34,6 +34,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Mock is an alternative to IndexNode, it will return specific results based on specific parameters.
|
// Mock is an alternative to IndexNode, it will return specific results based on specific parameters.
|
||||||
@ -46,7 +47,8 @@ type Mock struct {
|
|||||||
cancel context.CancelFunc
|
cancel context.CancelFunc
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
|
|
||||||
etcdKV *etcdkv.EtcdKV
|
etcdCli *clientv3.Client
|
||||||
|
etcdKV *etcdkv.EtcdKV
|
||||||
|
|
||||||
buildIndex chan *indexpb.CreateIndexRequest
|
buildIndex chan *indexpb.CreateIndexRequest
|
||||||
}
|
}
|
||||||
@ -182,16 +184,25 @@ func (inm *Mock) Register() error {
|
|||||||
return errors.New("IndexNode register failed")
|
return errors.New("IndexNode register failed")
|
||||||
}
|
}
|
||||||
Params.Init()
|
Params.Init()
|
||||||
inm.etcdKV, _ = etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath)
|
inm.etcdKV = etcdkv.NewEtcdKV(inm.etcdCli, Params.IndexNodeCfg.MetaRootPath)
|
||||||
if err := inm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexNodeRole); err != nil {
|
if err := inm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexNodeRole); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints)
|
session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, inm.etcdCli)
|
||||||
session.Init(typeutil.IndexNodeRole, "localhost:21121", false)
|
session.Init(typeutil.IndexNodeRole, "localhost:21121", false, false)
|
||||||
session.Register()
|
session.Register()
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetClient sets the IndexNode's instance.
|
||||||
|
func (inm *Mock) UpdateStateCode(stateCode internalpb.StateCode) {
|
||||||
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient assigns parameter client to its member etcdCli
|
||||||
|
func (inm *Mock) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
inm.etcdCli = client
|
||||||
|
}
|
||||||
|
|
||||||
// GetComponentStates gets the component states of the mocked IndexNode, if the internal member `Err` is true, it will return an error,
|
// GetComponentStates gets the component states of the mocked IndexNode, if the internal member `Err` is true, it will return an error,
|
||||||
// and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`.
|
// and the state is `StateCode_Abnormal`. Under normal circumstances the state is `StateCode_Healthy`.
|
||||||
func (inm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
func (inm *Mock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||||
|
@ -29,6 +29,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -37,7 +38,11 @@ func TestIndexNodeMock(t *testing.T) {
|
|||||||
inm := Mock{
|
inm := Mock{
|
||||||
Build: true,
|
Build: true,
|
||||||
}
|
}
|
||||||
err := inm.Register()
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
inm.SetEtcdClient(etcdCli)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
err = inm.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = inm.Init()
|
err = inm.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -151,8 +156,11 @@ func TestIndexNodeMockFiled(t *testing.T) {
|
|||||||
Build: true,
|
Build: true,
|
||||||
Err: false,
|
Err: false,
|
||||||
}
|
}
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
err := inm.Register()
|
assert.NoError(t, err)
|
||||||
|
inm.SetEtcdClient(etcdCli)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
err = inm.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = inm.Init()
|
err = inm.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
@ -27,6 +27,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/common"
|
"github.com/milvus-io/milvus/internal/common"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
@ -71,6 +72,11 @@ func TestIndexNode(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
Params.Init()
|
Params.Init()
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
in.SetEtcdClient(etcdCli)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
err = in.Init()
|
err = in.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -477,6 +483,11 @@ func TestCreateIndexFailed(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
Params.Init()
|
Params.Init()
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
in.SetEtcdClient(etcdCli)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
err = in.Init()
|
err = in.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -744,6 +755,11 @@ func TestIndexNode_Error(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
Params.Init()
|
Params.Init()
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
in.SetEtcdClient(etcdCli)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
err = in.Init()
|
err = in.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
@ -40,34 +40,17 @@ type EtcdKV struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewEtcdKV creates a new etcd kv.
|
// NewEtcdKV creates a new etcd kv.
|
||||||
func NewEtcdKV(etcdEndpoints []string, rootPath string) (*EtcdKV, error) {
|
func NewEtcdKV(client *clientv3.Client, rootPath string) *EtcdKV {
|
||||||
client, err := clientv3.New(clientv3.Config{
|
|
||||||
Endpoints: etcdEndpoints,
|
|
||||||
DialTimeout: 5 * time.Second,
|
|
||||||
})
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
kv := &EtcdKV{
|
kv := &EtcdKV{
|
||||||
client: client,
|
client: client,
|
||||||
rootPath: rootPath,
|
rootPath: rootPath,
|
||||||
}
|
}
|
||||||
|
return kv
|
||||||
return kv, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// NewEtcdKVWithClient creates a new etcd kv with a client.
|
|
||||||
func NewEtcdKVWithClient(cli *clientv3.Client, rootPath string) *EtcdKV {
|
|
||||||
return &EtcdKV{
|
|
||||||
client: cli,
|
|
||||||
rootPath: rootPath,
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Close closes the connection to etcd.
|
// Close closes the connection to etcd.
|
||||||
func (kv *EtcdKV) Close() {
|
func (kv *EtcdKV) Close() {
|
||||||
kv.client.Close()
|
log.Debug("etcd kv closed", zap.String("path", kv.rootPath))
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetPath returns the path of the key.
|
// GetPath returns the path of the key.
|
||||||
|
@ -18,18 +18,18 @@ package etcdkv_test
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"os"
|
"os"
|
||||||
"strings"
|
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
clientv3 "go.etcd.io/etcd/client/v3"
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
var Params paramtable.BaseTable
|
var Params paramtable.GlobalParamTable
|
||||||
|
|
||||||
func TestMain(m *testing.M) {
|
func TestMain(m *testing.M) {
|
||||||
Params.Init()
|
Params.Init()
|
||||||
@ -38,17 +38,12 @@ func TestMain(m *testing.M) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestEtcdKV_Load(te *testing.T) {
|
func TestEtcdKV_Load(te *testing.T) {
|
||||||
endpoints, err := Params.Load("_EtcdEndpoints")
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
if err != nil {
|
defer etcdCli.Close()
|
||||||
panic(err)
|
assert.NoError(te, err)
|
||||||
}
|
|
||||||
|
|
||||||
etcdEndPoints := strings.Split(endpoints, ",")
|
|
||||||
|
|
||||||
te.Run("EtcdKV SaveAndLoad", func(t *testing.T) {
|
te.Run("EtcdKV SaveAndLoad", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/saveandload"
|
rootPath := "/etcd/test/root/saveandload"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
require.NoError(t, err)
|
|
||||||
err = etcdKV.RemoveWithPrefix("")
|
err = etcdKV.RemoveWithPrefix("")
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
@ -157,8 +152,7 @@ func TestEtcdKV_Load(te *testing.T) {
|
|||||||
|
|
||||||
te.Run("EtcdKV LoadWithRevision", func(t *testing.T) {
|
te.Run("EtcdKV LoadWithRevision", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/LoadWithRevision"
|
rootPath := "/etcd/test/root/LoadWithRevision"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
assert.Nil(t, err)
|
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
@ -202,8 +196,7 @@ func TestEtcdKV_Load(te *testing.T) {
|
|||||||
|
|
||||||
te.Run("EtcdKV MultiSaveAndMultiLoad", func(t *testing.T) {
|
te.Run("EtcdKV MultiSaveAndMultiLoad", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/multi_save_and_multi_load"
|
rootPath := "/etcd/test/root/multi_save_and_multi_load"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
assert.Nil(t, err)
|
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
@ -311,9 +304,7 @@ func TestEtcdKV_Load(te *testing.T) {
|
|||||||
|
|
||||||
te.Run("EtcdKV MultiRemoveWithPrefix", func(t *testing.T) {
|
te.Run("EtcdKV MultiRemoveWithPrefix", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/multi_remove_with_prefix"
|
rootPath := "/etcd/test/root/multi_remove_with_prefix"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
require.NoError(t, err)
|
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
|
|
||||||
@ -399,8 +390,7 @@ func TestEtcdKV_Load(te *testing.T) {
|
|||||||
|
|
||||||
te.Run("EtcdKV Watch", func(t *testing.T) {
|
te.Run("EtcdKV Watch", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/watch"
|
rootPath := "/etcd/test/root/watch"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
assert.Nil(t, err)
|
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
@ -416,9 +406,7 @@ func TestEtcdKV_Load(te *testing.T) {
|
|||||||
|
|
||||||
te.Run("Etcd Revision", func(t *testing.T) {
|
te.Run("Etcd Revision", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/watch"
|
rootPath := "/etcd/test/root/watch"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
assert.Nil(t, err)
|
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
|
|
||||||
@ -467,8 +455,7 @@ func TestEtcdKV_Load(te *testing.T) {
|
|||||||
|
|
||||||
te.Run("Etcd Lease", func(t *testing.T) {
|
te.Run("Etcd Lease", func(t *testing.T) {
|
||||||
rootPath := "/etcd/test/root/lease"
|
rootPath := "/etcd/test/root/lease"
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndPoints, rootPath)
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
assert.Nil(t, err)
|
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
|
@ -19,6 +19,7 @@ package etcdkv
|
|||||||
import (
|
import (
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"go.etcd.io/etcd/server/v3/embed"
|
"go.etcd.io/etcd/server/v3/embed"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
@ -49,9 +50,10 @@ func NewMetaKvFactory(rootPath string, param *paramtable.BaseParamTable) (kv.Met
|
|||||||
}
|
}
|
||||||
return metaKv, err
|
return metaKv, err
|
||||||
}
|
}
|
||||||
metaKv, err := NewEtcdKV(param.EtcdEndpoints, rootPath)
|
client, err := etcd.GetEtcdClient(param)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
metaKv := NewEtcdKV(client, rootPath)
|
||||||
return metaKv, err
|
return metaKv, err
|
||||||
}
|
}
|
||||||
|
@ -38,6 +38,7 @@ import (
|
|||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/mqclient"
|
"github.com/milvus-io/milvus/internal/util/mqclient"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
@ -73,10 +74,12 @@ func (f *fixture) setup() []parameters {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
f.etcdKV, err = etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
defer etcdCli.Close()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatalf("New clientv3 error = %v", err)
|
log.Fatalf("New clientv3 error = %v", err)
|
||||||
}
|
}
|
||||||
|
f.etcdKV = etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", f.etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", f.etcdKV)
|
||||||
_ = idAllocator.Initialize()
|
_ = idAllocator.Initialize()
|
||||||
err = rocksmq.InitRmq(rocksdbName, idAllocator)
|
err = rocksmq.InitRmq(rocksdbName, idAllocator)
|
||||||
@ -1342,10 +1345,11 @@ func initRmq(name string) *etcdkv.EtcdKV {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatalf("New clientv3 error = %v", err)
|
log.Fatalf("New clientv3 error = %v", err)
|
||||||
}
|
}
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
_ = idAllocator.Initialize()
|
_ = idAllocator.Initialize()
|
||||||
|
|
||||||
|
@ -38,6 +38,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -67,6 +68,7 @@ type Proxy struct {
|
|||||||
|
|
||||||
stateCode atomic.Value
|
stateCode atomic.Value
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
rootCoord types.RootCoord
|
rootCoord types.RootCoord
|
||||||
indexCoord types.IndexCoord
|
indexCoord types.IndexCoord
|
||||||
dataCoord types.DataCoord
|
dataCoord types.DataCoord
|
||||||
@ -116,7 +118,9 @@ func (node *Proxy) Register() error {
|
|||||||
if err := node.Stop(); err != nil {
|
if err := node.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if node.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
// TODO Reset the logger
|
// TODO Reset the logger
|
||||||
//Params.initLogCfg()
|
//Params.initLogCfg()
|
||||||
@ -125,11 +129,11 @@ func (node *Proxy) Register() error {
|
|||||||
|
|
||||||
// initSession initialize the session of Proxy.
|
// initSession initialize the session of Proxy.
|
||||||
func (node *Proxy) initSession() error {
|
func (node *Proxy) initSession() error {
|
||||||
node.session = sessionutil.NewSession(node.ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
node.session = sessionutil.NewSession(node.ctx, Params.ProxyCfg.MetaRootPath, node.etcdCli)
|
||||||
if node.session == nil {
|
if node.session == nil {
|
||||||
return errors.New("new session failed, maybe etcd cannot be connected")
|
return errors.New("new session failed, maybe etcd cannot be connected")
|
||||||
}
|
}
|
||||||
node.session.Init(typeutil.ProxyRole, Params.ProxyCfg.NetworkAddress, false)
|
node.session.Init(typeutil.ProxyRole, Params.ProxyCfg.NetworkAddress, false, true)
|
||||||
Params.ProxyCfg.ProxyID = node.session.ServerID
|
Params.ProxyCfg.ProxyID = node.session.ServerID
|
||||||
Params.BaseParams.SetLogger(Params.ProxyCfg.ProxyID)
|
Params.BaseParams.SetLogger(Params.ProxyCfg.ProxyID)
|
||||||
return nil
|
return nil
|
||||||
@ -425,6 +429,11 @@ func (node *Proxy) AddCloseCallback(callbacks ...func()) {
|
|||||||
node.closeCallbacks = append(node.closeCallbacks, callbacks...)
|
node.closeCallbacks = append(node.closeCallbacks, callbacks...)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient sets etcd client for proxy.
|
||||||
|
func (node *Proxy) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
node.etcdCli = client
|
||||||
|
}
|
||||||
|
|
||||||
// SetRootCoordClient sets RootCoord client for proxy.
|
// SetRootCoordClient sets RootCoord client for proxy.
|
||||||
func (node *Proxy) SetRootCoordClient(cli types.RootCoord) {
|
func (node *Proxy) SetRootCoordClient(cli types.RootCoord) {
|
||||||
node.rootCoord = cli
|
node.rootCoord = cli
|
||||||
|
@ -29,6 +29,7 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
|
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
@ -211,13 +212,9 @@ func runDataCoord(ctx context.Context, localMsg bool) *grpcdatacoordclient.Serve
|
|||||||
}
|
}
|
||||||
|
|
||||||
factory := newMsgFactory(localMsg)
|
factory := newMsgFactory(localMsg)
|
||||||
var err error
|
ds = grpcdatacoordclient.NewServer(ctx, factory)
|
||||||
ds, err = grpcdatacoordclient.NewServer(ctx, factory)
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
wg.Done()
|
wg.Done()
|
||||||
err = ds.Run()
|
err := ds.Run()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
@ -310,6 +307,11 @@ func runIndexNode(ctx context.Context, localMsg bool, alias string) *grpcindexno
|
|||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
wg.Done()
|
wg.Done()
|
||||||
|
etcd, err := etcd.GetEtcdClient(&indexnode.Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
|
in.SetEtcdClient(etcd)
|
||||||
err = in.Run()
|
err = in.Run()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -419,7 +421,11 @@ func TestProxy(t *testing.T) {
|
|||||||
Params.Init()
|
Params.Init()
|
||||||
log.Info("Initialize parameter table of Proxy")
|
log.Info("Initialize parameter table of Proxy")
|
||||||
|
|
||||||
rootCoordClient, err := rcc.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdcli.Close()
|
||||||
|
assert.NoError(t, err)
|
||||||
|
proxy.SetEtcdClient(etcdcli)
|
||||||
|
rootCoordClient, err := rcc.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
err = rootCoordClient.Init()
|
err = rootCoordClient.Init()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -428,7 +434,7 @@ func TestProxy(t *testing.T) {
|
|||||||
proxy.SetRootCoordClient(rootCoordClient)
|
proxy.SetRootCoordClient(rootCoordClient)
|
||||||
log.Info("Proxy set root coordinator client")
|
log.Info("Proxy set root coordinator client")
|
||||||
|
|
||||||
dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
err = dataCoordClient.Init()
|
err = dataCoordClient.Init()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -437,7 +443,7 @@ func TestProxy(t *testing.T) {
|
|||||||
proxy.SetDataCoordClient(dataCoordClient)
|
proxy.SetDataCoordClient(dataCoordClient)
|
||||||
log.Info("Proxy set data coordinator client")
|
log.Info("Proxy set data coordinator client")
|
||||||
|
|
||||||
queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
err = queryCoordClient.Init()
|
err = queryCoordClient.Init()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -446,7 +452,7 @@ func TestProxy(t *testing.T) {
|
|||||||
proxy.SetQueryCoordClient(queryCoordClient)
|
proxy.SetQueryCoordClient(queryCoordClient)
|
||||||
log.Info("Proxy set query coordinator client")
|
log.Info("Proxy set query coordinator client")
|
||||||
|
|
||||||
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, etcdcli)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
err = indexCoordClient.Init()
|
err = indexCoordClient.Init()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -456,7 +462,6 @@ func TestProxy(t *testing.T) {
|
|||||||
log.Info("Proxy set index coordinator client")
|
log.Info("Proxy set index coordinator client")
|
||||||
|
|
||||||
proxy.UpdateStateCode(internalpb.StateCode_Initializing)
|
proxy.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||||
|
|
||||||
err = proxy.Init()
|
err = proxy.Init()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
@ -25,6 +25,7 @@ import (
|
|||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
@ -32,10 +33,13 @@ import (
|
|||||||
func TestShuffleChannelsToQueryNode(t *testing.T) {
|
func TestShuffleChannelsToQueryNode(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
|
||||||
|
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||||
clusterSession.Register()
|
clusterSession.Register()
|
||||||
meta, err := newMeta(baseCtx, kv, nil, nil)
|
meta, err := newMeta(baseCtx, kv, nil, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
@ -41,6 +41,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
@ -390,13 +391,15 @@ func TestQueryNodeCluster_getMetrics(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestReloadClusterFromKV(t *testing.T) {
|
func TestReloadClusterFromKV(t *testing.T) {
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
assert.Nil(t, err)
|
||||||
t.Run("Test LoadOnlineNodes", func(t *testing.T) {
|
t.Run("Test LoadOnlineNodes", func(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx := context.Background()
|
baseCtx := context.Background()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
assert.Nil(t, err)
|
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
|
||||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
|
||||||
clusterSession.Register()
|
clusterSession.Register()
|
||||||
cluster := &queryNodeCluster{
|
cluster := &queryNodeCluster{
|
||||||
ctx: baseCtx,
|
ctx: baseCtx,
|
||||||
@ -422,10 +425,9 @@ func TestReloadClusterFromKV(t *testing.T) {
|
|||||||
|
|
||||||
t.Run("Test LoadOfflineNodes", func(t *testing.T) {
|
t.Run("Test LoadOfflineNodes", func(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
assert.Nil(t, err)
|
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
|
||||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
|
||||||
clusterSession.Register()
|
clusterSession.Register()
|
||||||
cluster := &queryNodeCluster{
|
cluster := &queryNodeCluster{
|
||||||
client: kv,
|
client: kv,
|
||||||
@ -459,10 +461,12 @@ func TestReloadClusterFromKV(t *testing.T) {
|
|||||||
func TestGrpcRequest(t *testing.T) {
|
func TestGrpcRequest(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
defer etcdCli.Close()
|
||||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
|
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
|
||||||
|
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||||
clusterSession.Register()
|
clusterSession.Register()
|
||||||
factory := msgstream.NewPmsFactory()
|
factory := msgstream.NewPmsFactory()
|
||||||
m := map[string]interface{}{
|
m := map[string]interface{}{
|
||||||
|
@ -28,14 +28,17 @@ import (
|
|||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestReloadFromKV(t *testing.T) {
|
func TestReloadFromKV(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
meta, err := newMeta(baseCtx, kv, nil, nil)
|
meta, err := newMeta(baseCtx, kv, nil, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -88,8 +91,10 @@ func TestReloadFromKV(t *testing.T) {
|
|||||||
func TestCheckIndexLoop(t *testing.T) {
|
func TestCheckIndexLoop(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
meta, err := newMeta(ctx, kv, nil, nil)
|
meta, err := newMeta(ctx, kv, nil, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -152,8 +157,11 @@ func TestCheckIndexLoop(t *testing.T) {
|
|||||||
func TestProcessHandoffAfterIndexDone(t *testing.T) {
|
func TestProcessHandoffAfterIndexDone(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
meta, err := newMeta(ctx, kv, nil, nil)
|
meta, err := newMeta(ctx, kv, nil, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
taskScheduler := &TaskScheduler{
|
taskScheduler := &TaskScheduler{
|
||||||
@ -162,8 +170,7 @@ func TestProcessHandoffAfterIndexDone(t *testing.T) {
|
|||||||
client: kv,
|
client: kv,
|
||||||
triggerTaskQueue: NewTaskQueue(),
|
triggerTaskQueue: NewTaskQueue(),
|
||||||
}
|
}
|
||||||
idAllocatorKV, err := tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
|
idAllocatorKV := tsoutil.NewTSOKVBase(etcdCli, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
|
||||||
assert.Nil(t, err)
|
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
||||||
err = idAllocator.Initialize()
|
err = idAllocator.Initialize()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
@ -31,6 +31,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/internal/util"
|
"github.com/milvus-io/milvus/internal/util"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
)
|
)
|
||||||
|
|
||||||
func successResult() error { return nil }
|
func successResult() error { return nil }
|
||||||
@ -63,8 +64,10 @@ func (tk *testKv) Load(key string) (string, error) {
|
|||||||
|
|
||||||
func TestReplica_Release(t *testing.T) {
|
func TestReplica_Release(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
meta, err := newMeta(context.Background(), etcdKV, nil, nil)
|
meta, err := newMeta(context.Background(), etcdKV, nil, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
err = meta.addCollection(1, querypb.LoadType_loadCollection, nil)
|
err = meta.addCollection(1, querypb.LoadType_loadCollection, nil)
|
||||||
@ -93,8 +96,10 @@ func TestReplica_Release(t *testing.T) {
|
|||||||
|
|
||||||
func TestMetaFunc(t *testing.T) {
|
func TestMetaFunc(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
nodeID := defaultQueryNodeID
|
nodeID := defaultQueryNodeID
|
||||||
segmentInfos := make(map[UniqueID]*querypb.SegmentInfo)
|
segmentInfos := make(map[UniqueID]*querypb.SegmentInfo)
|
||||||
@ -285,8 +290,10 @@ func TestMetaFunc(t *testing.T) {
|
|||||||
|
|
||||||
func TestReloadMetaFromKV(t *testing.T) {
|
func TestReloadMetaFromKV(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
meta := &MetaReplica{
|
meta := &MetaReplica{
|
||||||
client: kv,
|
client: kv,
|
||||||
collectionInfos: map[UniqueID]*querypb.CollectionInfo{},
|
collectionInfos: map[UniqueID]*querypb.CollectionInfo{},
|
||||||
|
@ -31,6 +31,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
@ -100,9 +101,13 @@ func newQueryNodeServerMock(ctx context.Context) *queryNodeServerMock {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (qs *queryNodeServerMock) Register() error {
|
func (qs *queryNodeServerMock) Register() error {
|
||||||
log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints))
|
log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath))
|
||||||
qs.session = sessionutil.NewSession(qs.ctx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
qs.session.Init(typeutil.QueryNodeRole, qs.queryNodeIP+":"+strconv.FormatInt(qs.queryNodePort, 10), false)
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
qs.session = sessionutil.NewSession(qs.ctx, Params.QueryCoordCfg.MetaRootPath, etcdCli)
|
||||||
|
qs.session.Init(typeutil.QueryNodeRole, qs.queryNodeIP+":"+strconv.FormatInt(qs.queryNodePort, 10), false, false)
|
||||||
qs.queryNodeID = qs.session.ServerID
|
qs.queryNodeID = qs.session.ServerID
|
||||||
log.Debug("query nodeID", zap.Int64("nodeID", qs.queryNodeID))
|
log.Debug("query nodeID", zap.Int64("nodeID", qs.queryNodeID))
|
||||||
log.Debug("query node address", zap.String("address", qs.session.Address))
|
log.Debug("query node address", zap.String("address", qs.session.Address))
|
||||||
|
@ -40,11 +40,11 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -84,6 +84,7 @@ type QueryCoord struct {
|
|||||||
|
|
||||||
metricsCacheManager *metricsinfo.MetricsCacheManager
|
metricsCacheManager *metricsinfo.MetricsCacheManager
|
||||||
|
|
||||||
|
etcdCli *clientv3.Client
|
||||||
dataCoordClient types.DataCoord
|
dataCoordClient types.DataCoord
|
||||||
rootCoordClient types.RootCoord
|
rootCoordClient types.RootCoord
|
||||||
indexCoordClient types.IndexCoord
|
indexCoordClient types.IndexCoord
|
||||||
@ -106,17 +107,19 @@ func (qc *QueryCoord) Register() error {
|
|||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if qc.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (qc *QueryCoord) initSession() error {
|
func (qc *QueryCoord) initSession() error {
|
||||||
qc.session = sessionutil.NewSession(qc.loopCtx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
qc.session = sessionutil.NewSession(qc.loopCtx, Params.QueryCoordCfg.MetaRootPath, qc.etcdCli)
|
||||||
if qc.session == nil {
|
if qc.session == nil {
|
||||||
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
||||||
}
|
}
|
||||||
qc.session.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
qc.session.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, true)
|
||||||
Params.QueryCoordCfg.NodeID = uint64(qc.session.ServerID)
|
Params.QueryCoordCfg.NodeID = uint64(qc.session.ServerID)
|
||||||
Params.BaseParams.SetLogger(qc.session.ServerID)
|
Params.BaseParams.SetLogger(qc.session.ServerID)
|
||||||
return nil
|
return nil
|
||||||
@ -124,17 +127,7 @@ func (qc *QueryCoord) initSession() error {
|
|||||||
|
|
||||||
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
|
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
|
||||||
func (qc *QueryCoord) Init() error {
|
func (qc *QueryCoord) Init() error {
|
||||||
log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath),
|
log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.String("address", Params.QueryCoordCfg.Address))
|
||||||
zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints), zap.String("address", Params.QueryCoordCfg.Address))
|
|
||||||
//connect etcd
|
|
||||||
connectEtcdFn := func() error {
|
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
qc.kvClient = etcdKV
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
var initError error
|
var initError error
|
||||||
qc.initOnce.Do(func() {
|
qc.initOnce.Do(func() {
|
||||||
err := qc.initSession()
|
err := qc.initSession()
|
||||||
@ -144,20 +137,12 @@ func (qc *QueryCoord) Init() error {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
log.Debug("queryCoord try to connect etcd")
|
log.Debug("queryCoord try to connect etcd")
|
||||||
initError = retry.Do(qc.loopCtx, connectEtcdFn, retry.Attempts(300))
|
etcdKV := etcdkv.NewEtcdKV(qc.etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
if initError != nil {
|
qc.kvClient = etcdKV
|
||||||
log.Debug("query coordinator try to connect etcd failed", zap.Error(initError))
|
|
||||||
return
|
|
||||||
}
|
|
||||||
log.Debug("query coordinator try to connect etcd success")
|
log.Debug("query coordinator try to connect etcd success")
|
||||||
|
|
||||||
// init id allocator
|
// init id allocator
|
||||||
var idAllocatorKV *etcdkv.EtcdKV
|
idAllocatorKV := tsoutil.NewTSOKVBase(qc.etcdCli, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
|
||||||
idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
|
|
||||||
if initError != nil {
|
|
||||||
log.Debug("query coordinator idAllocatorKV initialize failed", zap.Error(initError))
|
|
||||||
return
|
|
||||||
}
|
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
||||||
initError = idAllocator.Initialize()
|
initError = idAllocator.Initialize()
|
||||||
if initError != nil {
|
if initError != nil {
|
||||||
@ -294,6 +279,11 @@ func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord,
|
|||||||
return service, nil
|
return service, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetRootCoord sets root coordinator's client
|
||||||
|
func (qc *QueryCoord) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
qc.etcdCli = etcdClient
|
||||||
|
}
|
||||||
|
|
||||||
// SetRootCoord sets root coordinator's client
|
// SetRootCoord sets root coordinator's client
|
||||||
func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error {
|
func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error {
|
||||||
if rootCoord == nil {
|
if rootCoord == nil {
|
||||||
|
@ -34,6 +34,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
@ -91,7 +92,11 @@ func startQueryCoord(ctx context.Context) (*QueryCoord, error) {
|
|||||||
coord.SetRootCoord(rootCoord)
|
coord.SetRootCoord(rootCoord)
|
||||||
coord.SetDataCoord(dataCoord)
|
coord.SetDataCoord(dataCoord)
|
||||||
coord.SetIndexCoord(indexCoord)
|
coord.SetIndexCoord(indexCoord)
|
||||||
|
etcd, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
coord.SetEtcdClient(etcd)
|
||||||
err = coord.Init()
|
err = coord.Init()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
@ -132,7 +137,11 @@ func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) {
|
|||||||
|
|
||||||
coord.SetRootCoord(rootCoord)
|
coord.SetRootCoord(rootCoord)
|
||||||
coord.SetDataCoord(dataCoord)
|
coord.SetDataCoord(dataCoord)
|
||||||
|
etcd, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
coord.SetEtcdClient(etcd)
|
||||||
err = coord.Init()
|
err = coord.Init()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
@ -147,11 +156,12 @@ func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) {
|
|||||||
|
|
||||||
func TestWatchNodeLoop(t *testing.T) {
|
func TestWatchNodeLoop(t *testing.T) {
|
||||||
baseCtx := context.Background()
|
baseCtx := context.Background()
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
t.Run("Test OfflineNodes", func(t *testing.T) {
|
t.Run("Test OfflineNodes", func(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
|
||||||
assert.Nil(t, err)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
kvs := make(map[string]string)
|
kvs := make(map[string]string)
|
||||||
session := &sessionutil.Session{
|
session := &sessionutil.Session{
|
||||||
|
@ -30,16 +30,20 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
//func waitQueryNodeOnline(cluster *queryNodeCluster, nodeID int64)
|
//func waitQueryNodeOnline(cluster *queryNodeCluster, nodeID int64)
|
||||||
|
|
||||||
func removeNodeSession(id int64) error {
|
func removeNodeSession(id int64) error {
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
err = kv.Remove(fmt.Sprintf("session/"+typeutil.QueryNodeRole+"-%d", id))
|
err = kv.Remove(fmt.Sprintf("session/"+typeutil.QueryNodeRole+"-%d", id))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
@ -48,10 +52,12 @@ func removeNodeSession(id int64) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func removeAllSession() error {
|
func removeAllSession() error {
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
err = kv.RemoveWithPrefix("session")
|
err = kv.RemoveWithPrefix("session")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
@ -187,8 +193,10 @@ func TestQueryNode_getMetrics(t *testing.T) {
|
|||||||
func TestNewQueryNode(t *testing.T) {
|
func TestNewQueryNode(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
queryNode1, err := startQueryNodeServer(baseCtx)
|
queryNode1, err := startQueryNodeServer(baseCtx)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -211,8 +219,10 @@ func TestNewQueryNode(t *testing.T) {
|
|||||||
func TestReleaseCollectionOnOfflineNode(t *testing.T) {
|
func TestReleaseCollectionOnOfflineNode(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
|
|
||||||
node, err := newQueryNode(baseCtx, "test", 100, kv)
|
node, err := newQueryNode(baseCtx, "test", 100, kv)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -280,8 +290,10 @@ func TestSealedSegmentChangeAfterQueryNodeStop(t *testing.T) {
|
|||||||
func TestGrpcRequestWithNodeOffline(t *testing.T) {
|
func TestGrpcRequestWithNodeOffline(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
nodeServer, err := startQueryNodeServer(baseCtx)
|
nodeServer, err := startQueryNodeServer(baseCtx)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
address := nodeServer.queryNodeIP
|
address := nodeServer.queryNodeIP
|
||||||
|
@ -25,6 +25,7 @@ import (
|
|||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
minioKV "github.com/milvus-io/milvus/internal/kv/minio"
|
minioKV "github.com/milvus-io/milvus/internal/kv/minio"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
@ -32,10 +33,12 @@ import (
|
|||||||
func TestShuffleSegmentsToQueryNode(t *testing.T) {
|
func TestShuffleSegmentsToQueryNode(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer etcdCli.Close()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, etcdCli)
|
||||||
|
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||||
meta, err := newMeta(baseCtx, kv, nil, nil)
|
meta, err := newMeta(baseCtx, kv, nil, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
cluster := &queryNodeCluster{
|
cluster := &queryNodeCluster{
|
||||||
|
@ -27,6 +27,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
@ -208,8 +209,10 @@ func TestWatchQueryChannel_ClearEtcdInfoAfterAssignedNodeDown(t *testing.T) {
|
|||||||
|
|
||||||
func TestUnMarshalTask(t *testing.T) {
|
func TestUnMarshalTask(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
taskScheduler := &TaskScheduler{
|
taskScheduler := &TaskScheduler{
|
||||||
ctx: baseCtx,
|
ctx: baseCtx,
|
||||||
@ -454,7 +457,10 @@ func TestUnMarshalTask(t *testing.T) {
|
|||||||
|
|
||||||
func TestReloadTaskFromKV(t *testing.T) {
|
func TestReloadTaskFromKV(t *testing.T) {
|
||||||
refreshParams()
|
refreshParams()
|
||||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
kv := etcdkv.NewEtcdKV(etcdCli, Params.QueryCoordCfg.MetaRootPath)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
baseCtx, cancel := context.WithCancel(context.Background())
|
baseCtx, cancel := context.WithCancel(context.Background())
|
||||||
taskScheduler := &TaskScheduler{
|
taskScheduler := &TaskScheduler{
|
||||||
|
@ -20,6 +20,7 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
|
"sync"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
@ -29,6 +30,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
|
queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
)
|
)
|
||||||
@ -228,7 +230,10 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetSegmentInfo", func(t *testing.T) {
|
t.Run("test GetSegmentInfo", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -251,7 +256,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test no collection in historical", func(t *testing.T) {
|
t.Run("test no collection in historical", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -272,7 +279,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test no collection in streaming", func(t *testing.T) {
|
t.Run("test no collection in streaming", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -293,7 +302,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test different segment type", func(t *testing.T) {
|
t.Run("test different segment type", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -335,7 +346,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetSegmentInfo with indexed segment", func(t *testing.T) {
|
t.Run("test GetSegmentInfo with indexed segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -367,7 +380,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetSegmentInfo without streaming partition", func(t *testing.T) {
|
t.Run("test GetSegmentInfo without streaming partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -386,7 +401,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetSegmentInfo without streaming segment", func(t *testing.T) {
|
t.Run("test GetSegmentInfo without streaming segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -405,7 +422,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetSegmentInfo without historical partition", func(t *testing.T) {
|
t.Run("test GetSegmentInfo without historical partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -424,7 +443,9 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetSegmentInfo without historical segment", func(t *testing.T) {
|
t.Run("test GetSegmentInfo without historical segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -442,6 +463,7 @@ func TestImpl_GetSegmentInfo(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, rsp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestImpl_isHealthy(t *testing.T) {
|
func TestImpl_isHealthy(t *testing.T) {
|
||||||
@ -458,11 +480,17 @@ func TestImpl_GetMetrics(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test GetMetrics", func(t *testing.T) {
|
t.Run("test GetMetrics", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, etcdCli)
|
||||||
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints)
|
|
||||||
|
|
||||||
metricReq := make(map[string]string)
|
metricReq := make(map[string]string)
|
||||||
metricReq[metricsinfo.MetricTypeKey] = "system_info"
|
metricReq[metricsinfo.MetricTypeKey] = "system_info"
|
||||||
@ -481,7 +509,9 @@ func TestImpl_GetMetrics(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test ParseMetricType failed", func(t *testing.T) {
|
t.Run("test ParseMetricType failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -499,13 +529,17 @@ func TestImpl_GetMetrics(t *testing.T) {
|
|||||||
_, err = node.GetMetrics(ctx, req)
|
_, err = node.GetMetrics(ctx, req)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestImpl_ReleaseSegments(t *testing.T) {
|
func TestImpl_ReleaseSegments(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test valid", func(t *testing.T) {
|
t.Run("test valid", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -520,7 +554,9 @@ func TestImpl_ReleaseSegments(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test invalid query node", func(t *testing.T) {
|
t.Run("test invalid query node", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -536,7 +572,9 @@ func TestImpl_ReleaseSegments(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test segment not exists", func(t *testing.T) {
|
t.Run("test segment not exists", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -557,4 +595,5 @@ func TestImpl_ReleaseSegments(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.NotEqual(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
assert.NotEqual(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
@ -24,6 +24,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -34,7 +35,10 @@ func TestGetSystemInfoMetrics(t *testing.T) {
|
|||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, etcdCli)
|
||||||
|
|
||||||
req := &milvuspb.GetMetricsRequest{
|
req := &milvuspb.GetMetricsRequest{
|
||||||
Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels),
|
Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels),
|
||||||
|
@ -42,6 +42,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util"
|
"github.com/milvus-io/milvus/internal/util"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -370,8 +371,12 @@ func genMinioKV(ctx context.Context) (*minioKV.MinIOKV, error) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func genEtcdKV() (*etcdkv.EtcdKV, error) {
|
func genEtcdKV() (*etcdkv.EtcdKV, error) {
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
return etcdKV, err
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
|
||||||
|
return etcdKV, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func genFactory() (msgstream.Factory, error) {
|
func genFactory() (msgstream.Factory, error) {
|
||||||
@ -1275,14 +1280,12 @@ func genSimpleChangeInfo() *querypb.SealedSegmentsChangeInfo {
|
|||||||
|
|
||||||
func saveChangeInfo(key string, value string) error {
|
func saveChangeInfo(key string, value string) error {
|
||||||
log.Debug(".. [query node unittest] Saving change info")
|
log.Debug(".. [query node unittest] Saving change info")
|
||||||
|
|
||||||
kv, err := genEtcdKV()
|
kv, err := genEtcdKV()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
key = util.ChangeInfoMetaPrefix + "/" + key
|
key = util.ChangeInfoMetaPrefix + "/" + key
|
||||||
|
|
||||||
return kv.Save(key, value)
|
return kv.Save(key, value)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1293,16 +1296,17 @@ func genSimpleQueryNode(ctx context.Context) (*QueryNode, error) {
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
node := NewQueryNode(ctx, fac)
|
node := NewQueryNode(ctx, fac)
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
node.etcdCli = etcdCli
|
||||||
session := &sessionutil.Session{
|
session := &sessionutil.Session{
|
||||||
ServerID: 1,
|
ServerID: 1,
|
||||||
}
|
}
|
||||||
node.session = session
|
node.session = session
|
||||||
|
|
||||||
etcdKV, err := genEtcdKV()
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
node.etcdKV = etcdKV
|
node.etcdKV = etcdKV
|
||||||
|
|
||||||
node.tSafeReplica = newTSafeReplica()
|
node.tSafeReplica = newTSafeReplica()
|
||||||
|
@ -40,6 +40,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/segcorepb"
|
"github.com/milvus-io/milvus/internal/proto/segcorepb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
@ -131,8 +132,10 @@ func TestQueryCollection_withoutVChannel(t *testing.T) {
|
|||||||
factory := msgstream.NewPmsFactory()
|
factory := msgstream.NewPmsFactory()
|
||||||
err := factory.SetParams(m)
|
err := factory.SetParams(m)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
|
||||||
|
|
||||||
schema := genTestCollectionSchema(0, false, 2)
|
schema := genTestCollectionSchema(0, false, 2)
|
||||||
historicalReplica := newCollectionReplica(etcdKV)
|
historicalReplica := newCollectionReplica(etcdKV)
|
||||||
@ -689,8 +692,10 @@ func TestQueryCollection_AddPopUnsolvedMsg(t *testing.T) {
|
|||||||
|
|
||||||
func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
|
func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.TODO())
|
ctx, cancel := context.WithCancel(context.TODO())
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test adjustByChangeInfo", func(t *testing.T) {
|
t.Run("test adjustByChangeInfo", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
qc, err := genSimpleQueryCollection(ctx, cancel)
|
qc, err := genSimpleQueryCollection(ctx, cancel)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -710,7 +715,9 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
|
|||||||
assert.Len(t, ids, 0)
|
assert.Len(t, ids, 0)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test mismatch collectionID when adjustByChangeInfo", func(t *testing.T) {
|
t.Run("test mismatch collectionID when adjustByChangeInfo", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
qc, err := genSimpleQueryCollection(ctx, cancel)
|
qc, err := genSimpleQueryCollection(ctx, cancel)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -723,7 +730,9 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
|
|||||||
qc.adjustByChangeInfo(segmentChangeInfos)
|
qc.adjustByChangeInfo(segmentChangeInfos)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test no segment when adjustByChangeInfo", func(t *testing.T) {
|
t.Run("test no segment when adjustByChangeInfo", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
qc, err := genSimpleQueryCollection(ctx, cancel)
|
qc, err := genSimpleQueryCollection(ctx, cancel)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -735,13 +744,17 @@ func TestQueryCollection_adjustByChangeInfo(t *testing.T) {
|
|||||||
|
|
||||||
qc.adjustByChangeInfo(segmentChangeInfos)
|
qc.adjustByChangeInfo(segmentChangeInfos)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestQueryCollection_search_while_release(t *testing.T) {
|
func TestQueryCollection_search_while_release(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
|
var wgAll sync.WaitGroup
|
||||||
|
wgAll.Add(1)
|
||||||
t.Run("test search while release collection", func(t *testing.T) {
|
t.Run("test search while release collection", func(t *testing.T) {
|
||||||
|
defer wgAll.Done()
|
||||||
queryCollection, err := genSimpleQueryCollection(ctx, cancel)
|
queryCollection, err := genSimpleQueryCollection(ctx, cancel)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -778,7 +791,9 @@ func TestQueryCollection_search_while_release(t *testing.T) {
|
|||||||
wg.Wait()
|
wg.Wait()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wgAll.Add(1)
|
||||||
t.Run("test search while release partition", func(t *testing.T) {
|
t.Run("test search while release partition", func(t *testing.T) {
|
||||||
|
defer wgAll.Done()
|
||||||
queryCollection, err := genSimpleQueryCollection(ctx, cancel)
|
queryCollection, err := genSimpleQueryCollection(ctx, cancel)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -814,4 +829,5 @@ func TestQueryCollection_search_while_release(t *testing.T) {
|
|||||||
}
|
}
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
})
|
})
|
||||||
|
wgAll.Wait()
|
||||||
}
|
}
|
||||||
|
@ -55,6 +55,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -99,6 +100,9 @@ type QueryNode struct {
|
|||||||
// segment loader
|
// segment loader
|
||||||
loader *segmentLoader
|
loader *segmentLoader
|
||||||
|
|
||||||
|
// etcd client
|
||||||
|
etcdCli *clientv3.Client
|
||||||
|
|
||||||
// clients
|
// clients
|
||||||
rootCoord types.RootCoord
|
rootCoord types.RootCoord
|
||||||
indexCoord types.IndexCoord
|
indexCoord types.IndexCoord
|
||||||
@ -129,11 +133,11 @@ func NewQueryNode(ctx context.Context, factory msgstream.Factory) *QueryNode {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (node *QueryNode) initSession() error {
|
func (node *QueryNode) initSession() error {
|
||||||
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, Params.QueryNodeCfg.EtcdEndpoints)
|
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.QueryNodeCfg.MetaRootPath, node.etcdCli)
|
||||||
if node.session == nil {
|
if node.session == nil {
|
||||||
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
||||||
}
|
}
|
||||||
node.session.Init(typeutil.QueryNodeRole, Params.QueryNodeCfg.QueryNodeIP+":"+strconv.FormatInt(Params.QueryNodeCfg.QueryNodePort, 10), false)
|
node.session.Init(typeutil.QueryNodeRole, Params.QueryNodeCfg.QueryNodeIP+":"+strconv.FormatInt(Params.QueryNodeCfg.QueryNodePort, 10), false, true)
|
||||||
Params.QueryNodeCfg.QueryNodeID = node.session.ServerID
|
Params.QueryNodeCfg.QueryNodeID = node.session.ServerID
|
||||||
Params.BaseParams.SetLogger(Params.QueryNodeCfg.QueryNodeID)
|
Params.BaseParams.SetLogger(Params.QueryNodeCfg.QueryNodeID)
|
||||||
log.Debug("QueryNode", zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID), zap.String("node address", node.session.Address))
|
log.Debug("QueryNode", zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID), zap.String("node address", node.session.Address))
|
||||||
@ -150,7 +154,9 @@ func (node *QueryNode) Register() error {
|
|||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if node.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
//TODO Reset the logger
|
//TODO Reset the logger
|
||||||
@ -179,7 +185,7 @@ func (node *QueryNode) Init() error {
|
|||||||
var initError error = nil
|
var initError error = nil
|
||||||
node.initOnce.Do(func() {
|
node.initOnce.Do(func() {
|
||||||
//ctx := context.Background()
|
//ctx := context.Background()
|
||||||
log.Debug("QueryNode session info", zap.String("metaPath", Params.QueryNodeCfg.MetaRootPath), zap.Strings("etcdEndPoints", Params.QueryNodeCfg.EtcdEndpoints))
|
log.Debug("QueryNode session info", zap.String("metaPath", Params.QueryNodeCfg.MetaRootPath))
|
||||||
err := node.initSession()
|
err := node.initSession()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("QueryNode init session failed", zap.Error(err))
|
log.Error("QueryNode init session failed", zap.Error(err))
|
||||||
@ -187,28 +193,9 @@ func (node *QueryNode) Init() error {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
Params.QueryNodeCfg.Refresh()
|
Params.QueryNodeCfg.Refresh()
|
||||||
connectEtcdFn := func() error {
|
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath)
|
node.etcdKV = etcdkv.NewEtcdKV(node.etcdCli, Params.QueryNodeCfg.MetaRootPath)
|
||||||
if err != nil {
|
log.Debug("queryNode try to connect etcd success", zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath))
|
||||||
return err
|
|
||||||
}
|
|
||||||
node.etcdKV = etcdKV
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
log.Debug("queryNode try to connect etcd",
|
|
||||||
zap.Any("EtcdEndpoints", Params.QueryNodeCfg.EtcdEndpoints),
|
|
||||||
zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath),
|
|
||||||
)
|
|
||||||
err = retry.Do(node.queryNodeLoopCtx, connectEtcdFn, retry.Attempts(300))
|
|
||||||
if err != nil {
|
|
||||||
log.Debug("queryNode try to connect etcd failed", zap.Error(err))
|
|
||||||
initError = err
|
|
||||||
return
|
|
||||||
}
|
|
||||||
log.Debug("queryNode try to connect etcd success",
|
|
||||||
zap.Any("EtcdEndpoints", Params.QueryNodeCfg.EtcdEndpoints),
|
|
||||||
zap.Any("MetaRootPath", Params.QueryNodeCfg.MetaRootPath),
|
|
||||||
)
|
|
||||||
node.tSafeReplica = newTSafeReplica()
|
node.tSafeReplica = newTSafeReplica()
|
||||||
|
|
||||||
streamingReplica := newCollectionReplica(node.etcdKV)
|
streamingReplica := newCollectionReplica(node.etcdKV)
|
||||||
@ -328,6 +315,11 @@ func (node *QueryNode) UpdateStateCode(code internalpb.StateCode) {
|
|||||||
node.stateCode.Store(code)
|
node.stateCode.Store(code)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SetEtcdClient assigns parameter client to its member etcdCli
|
||||||
|
func (node *QueryNode) SetEtcdClient(client *clientv3.Client) {
|
||||||
|
node.etcdCli = client
|
||||||
|
}
|
||||||
|
|
||||||
// SetRootCoord assigns parameter rc to its member rootCoord.
|
// SetRootCoord assigns parameter rc to its member rootCoord.
|
||||||
func (node *QueryNode) SetRootCoord(rc types.RootCoord) error {
|
func (node *QueryNode) SetRootCoord(rc types.RootCoord) error {
|
||||||
if rc == nil {
|
if rc == nil {
|
||||||
|
@ -21,6 +21,7 @@ import (
|
|||||||
"math/rand"
|
"math/rand"
|
||||||
"os"
|
"os"
|
||||||
"strconv"
|
"strconv"
|
||||||
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
@ -34,6 +35,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
)
|
)
|
||||||
|
|
||||||
// mock of query coordinator client
|
// mock of query coordinator client
|
||||||
@ -186,11 +188,11 @@ func newQueryNodeMock() *QueryNode {
|
|||||||
cancel()
|
cancel()
|
||||||
}()
|
}()
|
||||||
}
|
}
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryNodeCfg.EtcdEndpoints, Params.QueryNodeCfg.MetaRootPath)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.QueryNodeCfg.MetaRootPath)
|
||||||
|
|
||||||
msFactory, err := newMessageStreamFactory()
|
msFactory, err := newMessageStreamFactory()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -270,9 +272,14 @@ func TestQueryNode_register(t *testing.T) {
|
|||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdcli.Close()
|
||||||
|
node.SetEtcdClient(etcdcli)
|
||||||
err = node.initSession()
|
err = node.initSession()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
node.session.TriggerKill = false
|
||||||
err = node.Register()
|
err = node.Register()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
}
|
}
|
||||||
@ -283,7 +290,10 @@ func TestQueryNode_init(t *testing.T) {
|
|||||||
|
|
||||||
node, err := genSimpleQueryNode(ctx)
|
node, err := genSimpleQueryNode(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
etcdcli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdcli.Close()
|
||||||
|
node.SetEtcdClient(etcdcli)
|
||||||
err = node.Init()
|
err = node.Init()
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
}
|
}
|
||||||
@ -322,7 +332,10 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) {
|
|||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test cleanup segments", func(t *testing.T) {
|
t.Run("test cleanup segments", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -330,7 +343,9 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test cleanup segments no segment", func(t *testing.T) {
|
t.Run("test cleanup segments no segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -348,13 +363,16 @@ func TestQueryNode_adjustByChangeInfo(t *testing.T) {
|
|||||||
err = node.removeSegments(segmentChangeInfos)
|
err = node.removeSegments(segmentChangeInfos)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestQueryNode_watchChangeInfo(t *testing.T) {
|
func TestQueryNode_watchChangeInfo(t *testing.T) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test watchChangeInfo", func(t *testing.T) {
|
t.Run("test watchChangeInfo", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -369,7 +387,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
|
|||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test watchChangeInfo key error", func(t *testing.T) {
|
t.Run("test watchChangeInfo key error", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -381,7 +401,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
|
|||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test watchChangeInfo unmarshal error", func(t *testing.T) {
|
t.Run("test watchChangeInfo unmarshal error", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -393,7 +415,9 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
|
|||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("test watchChangeInfo adjustByChangeInfo error", func(t *testing.T) {
|
t.Run("test watchChangeInfo adjustByChangeInfo error", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
node, err := genSimpleQueryNodeToTestWatchChangeInfo(ctx)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -417,4 +441,5 @@ func TestQueryNode_watchChangeInfo(t *testing.T) {
|
|||||||
|
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
@ -19,6 +19,7 @@ package rootcoord
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
@ -82,12 +83,17 @@ func TestDmlChannels(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestDmChannelsFailure(t *testing.T) {
|
func TestDmChannelsFailure(t *testing.T) {
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("Test newDmlChannels", func(t *testing.T) {
|
t.Run("Test newDmlChannels", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockFactory := &FailMessageStreamFactory{}
|
mockFactory := &FailMessageStreamFactory{}
|
||||||
assert.Panics(t, func() { newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) })
|
assert.Panics(t, func() { newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1) })
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("Test broadcast", func(t *testing.T) {
|
t.Run("Test broadcast", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockFactory := &FailMessageStreamFactory{errBroadcast: true}
|
mockFactory := &FailMessageStreamFactory{errBroadcast: true}
|
||||||
dml := newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1)
|
dml := newDmlChannels(context.TODO(), mockFactory, "test-newdmlchannel-root", 1)
|
||||||
chanName0 := dml.getChannelName()
|
chanName0 := dml.getChannelName()
|
||||||
@ -101,6 +107,7 @@ func TestDmChannelsFailure(t *testing.T) {
|
|||||||
assert.Empty(t, v)
|
assert.Empty(t, v)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
// FailMessageStreamFactory mock MessageStreamFactory failure
|
// FailMessageStreamFactory mock MessageStreamFactory failure
|
||||||
|
@ -24,10 +24,9 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
|
||||||
clientv3 "go.etcd.io/etcd/client/v3"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestMetaSnapshot(t *testing.T) {
|
func TestMetaSnapshot(t *testing.T) {
|
||||||
@ -38,7 +37,7 @@ func TestMetaSnapshot(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
@ -178,7 +177,7 @@ func TestGetRevOnEtcd(t *testing.T) {
|
|||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
key := path.Join(rootPath, tsKey)
|
key := path.Join(rootPath, tsKey)
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
@ -222,7 +221,7 @@ func TestLoad(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
@ -270,7 +269,7 @@ func TestMultiSave(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
@ -334,8 +333,9 @@ func TestMultiSaveAndRemoveWithPrefix(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
var vtso typeutil.Timestamp
|
var vtso typeutil.Timestamp
|
||||||
ftso := func() typeutil.Timestamp {
|
ftso := func() typeutil.Timestamp {
|
||||||
@ -411,7 +411,7 @@ func TestTsBackward(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
@ -438,7 +438,7 @@ func TestFix7150(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
tsKey := "timestamp"
|
tsKey := "timestamp"
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
@ -21,6 +21,7 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"path"
|
"path"
|
||||||
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
@ -31,9 +32,10 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
|
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
clientv3 "go.etcd.io/etcd/client/v3"
|
"github.com/stretchr/testify/require"
|
||||||
)
|
)
|
||||||
|
|
||||||
type mockTestKV struct {
|
type mockTestKV struct {
|
||||||
@ -220,13 +222,14 @@ func TestMetaTable(t *testing.T) {
|
|||||||
return vtso
|
return vtso
|
||||||
}
|
}
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
require.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7)
|
skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, skv)
|
assert.NotNil(t, skv)
|
||||||
txnKV := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath)
|
txnKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
mt, err := NewMetaTable(txnKV, skv)
|
mt, err := NewMetaTable(txnKV, skv)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -293,7 +296,10 @@ func TestMetaTable(t *testing.T) {
|
|||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add collection", func(t *testing.T) {
|
t.Run("add collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
err = mt.AddCollection(collInfo, ts, nil, "")
|
err = mt.AddCollection(collInfo, ts, nil, "")
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
@ -321,7 +327,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Equal(t, "false", flag)
|
assert.Equal(t, "false", flag)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add alias", func(t *testing.T) {
|
t.Run("add alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
exists := mt.IsAlias(aliasName1)
|
exists := mt.IsAlias(aliasName1)
|
||||||
assert.False(t, exists)
|
assert.False(t, exists)
|
||||||
@ -332,8 +340,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
exists = mt.IsAlias(aliasName1)
|
exists = mt.IsAlias(aliasName1)
|
||||||
assert.True(t, exists)
|
assert.True(t, exists)
|
||||||
})
|
})
|
||||||
|
wg.Add(1)
|
||||||
t.Run("alter alias", func(t *testing.T) {
|
t.Run("alter alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
err = mt.AlterAlias(aliasName1, collName, ts)
|
err = mt.AlterAlias(aliasName1, collName, ts)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -341,13 +350,17 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("delete alias", func(t *testing.T) {
|
t.Run("delete alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
err = mt.DropAlias(aliasName1, ts)
|
err = mt.DropAlias(aliasName1, ts)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add partition", func(t *testing.T) {
|
t.Run("add partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
err = mt.AddPartition(collID, partName, partID, ts, "")
|
err = mt.AddPartition(collID, partName, partID, ts, "")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -365,7 +378,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Equal(t, "false", flag)
|
assert.Equal(t, "false", flag)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add segment index", func(t *testing.T) {
|
t.Run("add segment index", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
segIdxInfo := pb.SegmentIndexInfo{
|
segIdxInfo := pb.SegmentIndexInfo{
|
||||||
CollectionID: collID,
|
CollectionID: collID,
|
||||||
PartitionID: partID,
|
PartitionID: partID,
|
||||||
@ -387,7 +402,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.EqualError(t, err, fmt.Sprintf("index id = %d exist", segIdxInfo.IndexID))
|
assert.EqualError(t, err, fmt.Sprintf("index id = %d exist", segIdxInfo.IndexID))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get not indexed segments", func(t *testing.T) {
|
t.Run("get not indexed segments", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
params := []*commonpb.KeyValuePair{
|
params := []*commonpb.KeyValuePair{
|
||||||
{
|
{
|
||||||
Key: "field110-i1",
|
Key: "field110-i1",
|
||||||
@ -442,7 +459,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
|
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get index by name", func(t *testing.T) {
|
t.Run("get index by name", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
_, idx, err := mt.GetIndexByName(collName, "field110")
|
_, idx, err := mt.GetIndexByName(collName, "field110")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.Equal(t, 1, len(idx))
|
assert.Equal(t, 1, len(idx))
|
||||||
@ -464,7 +483,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Zero(t, len(idx))
|
assert.Zero(t, len(idx))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("reload meta", func(t *testing.T) {
|
t.Run("reload meta", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
te := pb.TenantMeta{
|
te := pb.TenantMeta{
|
||||||
ID: 100,
|
ID: 100,
|
||||||
}
|
}
|
||||||
@ -480,7 +501,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop index", func(t *testing.T) {
|
t.Run("drop index", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
idx, ok, err := mt.DropIndex(collName, "field110", "field110")
|
idx, ok, err := mt.DropIndex(collName, "field110", "field110")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.True(t, ok)
|
assert.True(t, ok)
|
||||||
@ -503,7 +526,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop partition", func(t *testing.T) {
|
t.Run("drop partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
id, err := mt.DeletePartition(collID, partName, ts, "")
|
id, err := mt.DeletePartition(collID, partName, ts, "")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -515,7 +540,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Equal(t, "false", flag)
|
assert.Equal(t, "false", flag)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop collection", func(t *testing.T) {
|
t.Run("drop collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ts := ftso()
|
ts := ftso()
|
||||||
err = mt.DeleteCollection(collIDInvalid, ts, "")
|
err = mt.DeleteCollection(collIDInvalid, ts, "")
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
@ -535,7 +562,7 @@ func TestMetaTable(t *testing.T) {
|
|||||||
})
|
})
|
||||||
|
|
||||||
/////////////////////////// these tests should run at last, it only used to hit the error lines ////////////////////////
|
/////////////////////////// these tests should run at last, it only used to hit the error lines ////////////////////////
|
||||||
txnkv := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath)
|
txnkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
mockKV := &mockTestKV{}
|
mockKV := &mockTestKV{}
|
||||||
mt.snapshot = mockKV
|
mt.snapshot = mockKV
|
||||||
mockTxnKV := &mockTestTxnKV{
|
mockTxnKV := &mockTestTxnKV{
|
||||||
@ -549,7 +576,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
}
|
}
|
||||||
mt.txn = mockTxnKV
|
mt.txn = mockTxnKV
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add collection failed", func(t *testing.T) {
|
t.Run("add collection failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -562,7 +591,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Panics(t, func() { mt.AddCollection(collInfo, 0, idxInfo, "") })
|
assert.Panics(t, func() { mt.AddCollection(collInfo, 0, idxInfo, "") })
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("delete collection failed", func(t *testing.T) {
|
t.Run("delete collection failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.multiSave = func(kvs map[string]string, ts typeutil.Timestamp) error {
|
mockKV.multiSave = func(kvs map[string]string, ts typeutil.Timestamp) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -573,7 +604,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Panics(t, func() { mt.DeleteCollection(collInfo.ID, ts, "") })
|
assert.Panics(t, func() { mt.DeleteCollection(collInfo.ID, ts, "") })
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get collection failed", func(t *testing.T) {
|
t.Run("get collection failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.save = func(key string, value string, ts typeutil.Timestamp) error {
|
mockKV.save = func(key string, value string, ts typeutil.Timestamp) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -592,7 +625,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
|
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add partition failed", func(t *testing.T) {
|
t.Run("add partition failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.save = func(key string, value string, ts typeutil.Timestamp) error {
|
mockKV.save = func(key string, value string, ts typeutil.Timestamp) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -653,7 +688,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.EqualError(t, err, fmt.Sprintf("partition id = %d already exists", partID))
|
assert.EqualError(t, err, fmt.Sprintf("partition id = %d already exists", partID))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("has partition failed", func(t *testing.T) {
|
t.Run("has partition failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -676,7 +713,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.False(t, mt.HasPartition(collInfo.ID, partName, 0))
|
assert.False(t, mt.HasPartition(collInfo.ID, partName, 0))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("delete partition failed", func(t *testing.T) {
|
t.Run("delete partition failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -714,7 +753,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.EqualError(t, err, fmt.Sprintf("can't find collection id = %d", collInfo.ID))
|
assert.EqualError(t, err, fmt.Sprintf("can't find collection id = %d", collInfo.ID))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("add index failed", func(t *testing.T) {
|
t.Run("add index failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -768,7 +809,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Panics(t, func() { mt.AddIndex(&segIdxInfo) })
|
assert.Panics(t, func() { mt.AddIndex(&segIdxInfo) })
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop index failed", func(t *testing.T) {
|
t.Run("drop index failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -833,7 +876,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.Panics(t, func() { mt.DropIndex(collInfo.Schema.Name, collInfo.Schema.Fields[0].Name, idxInfo[0].IndexName) })
|
assert.Panics(t, func() { mt.DropIndex(collInfo.Schema.Name, collInfo.Schema.Fields[0].Name, idxInfo[0].IndexName) })
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get segment index info by id", func(t *testing.T) {
|
t.Run("get segment index info by id", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -882,7 +927,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.EqualError(t, err, fmt.Sprintf("can't find index name = %s on segment = %d, with filed id = 11", idxInfo[0].IndexName, segIdxInfo.SegmentID))
|
assert.EqualError(t, err, fmt.Sprintf("can't find index name = %s on segment = %d, with filed id = 11", idxInfo[0].IndexName, segIdxInfo.SegmentID))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get field schema failed", func(t *testing.T) {
|
t.Run("get field schema failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -913,7 +960,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.EqualError(t, err, fmt.Sprintf("collection %s not found", collInfo.Schema.Name))
|
assert.EqualError(t, err, fmt.Sprintf("collection %s not found", collInfo.Schema.Name))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("is segment indexed", func(t *testing.T) {
|
t.Run("is segment indexed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -935,7 +984,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.False(t, mt.IsSegmentIndexed(idx.SegmentID, &field, nil))
|
assert.False(t, mt.IsSegmentIndexed(idx.SegmentID, &field, nil))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get not indexed segments", func(t *testing.T) {
|
t.Run("get not indexed segments", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -1028,7 +1079,9 @@ func TestMetaTable(t *testing.T) {
|
|||||||
//assert.EqualError(t, err, "multi save error")
|
//assert.EqualError(t, err, "multi save error")
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get index by name failed", func(t *testing.T) {
|
t.Run("get index by name failed", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
mockKV.loadWithPrefix = func(key string, ts typeutil.Timestamp) ([]string, []string, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil
|
||||||
}
|
}
|
||||||
@ -1064,6 +1117,7 @@ func TestMetaTable(t *testing.T) {
|
|||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
assert.EqualError(t, err, fmt.Sprintf("cannot find index, id = %d", idxInfo[0].IndexID))
|
assert.EqualError(t, err, fmt.Sprintf("cannot find index, id = %d", idxInfo[0].IndexID))
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestMetaWithTimestamp(t *testing.T) {
|
func TestMetaWithTimestamp(t *testing.T) {
|
||||||
@ -1088,15 +1142,14 @@ func TestMetaWithTimestamp(t *testing.T) {
|
|||||||
vtso++
|
vtso++
|
||||||
return vtso
|
return vtso
|
||||||
}
|
}
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7)
|
skv, err := newMetaSnapshot(etcdCli, rootPath, TimestampPrefix, 7)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotNil(t, skv)
|
assert.NotNil(t, skv)
|
||||||
txnKV := etcdkv.NewEtcdKVWithClient(etcdCli, rootPath)
|
txnKV := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
mt, err := NewMetaTable(txnKV, skv)
|
mt, err := NewMetaTable(txnKV, skv)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -1246,7 +1299,7 @@ func TestFixIssue10540(t *testing.T) {
|
|||||||
Params.Init()
|
Params.Init()
|
||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdCli.Close()
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
@ -22,9 +22,9 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
clientv3 "go.etcd.io/etcd/client/v3"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestProxyClientManager_GetProxyClients(t *testing.T) {
|
func TestProxyClientManager_GetProxyClients(t *testing.T) {
|
||||||
@ -32,7 +32,8 @@ func TestProxyClientManager_GetProxyClients(t *testing.T) {
|
|||||||
|
|
||||||
core, err := NewCore(context.Background(), nil)
|
core, err := NewCore(context.Background(), nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
cli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
defer cli.Close()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
core.etcdCli = cli
|
core.etcdCli = cli
|
||||||
|
|
||||||
@ -58,8 +59,9 @@ func TestProxyClientManager_AddProxyClient(t *testing.T) {
|
|||||||
|
|
||||||
core, err := NewCore(context.Background(), nil)
|
core, err := NewCore(context.Background(), nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
cli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer cli.Close()
|
||||||
core.etcdCli = cli
|
core.etcdCli = cli
|
||||||
|
|
||||||
core.SetNewProxyClient(
|
core.SetNewProxyClient(
|
||||||
@ -84,8 +86,9 @@ func TestProxyClientManager_InvalidateCollectionMetaCache(t *testing.T) {
|
|||||||
|
|
||||||
core, err := NewCore(ctx, nil)
|
core, err := NewCore(ctx, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
cli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer cli.Close()
|
||||||
core.etcdCli = cli
|
core.etcdCli = cli
|
||||||
|
|
||||||
pcm := newProxyClientManager(core)
|
pcm := newProxyClientManager(core)
|
||||||
@ -114,8 +117,9 @@ func TestProxyClientManager_ReleaseDQLMessageStream(t *testing.T) {
|
|||||||
|
|
||||||
core, err := NewCore(ctx, nil)
|
core, err := NewCore(ctx, nil)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
cli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer cli.Close()
|
||||||
core.etcdCli = cli
|
core.etcdCli = cli
|
||||||
|
|
||||||
pcm := newProxyClientManager(core)
|
pcm := newProxyClientManager(core)
|
||||||
|
@ -46,20 +46,16 @@ type proxyManager struct {
|
|||||||
// newProxyManager helper function to create a proxyManager
|
// newProxyManager helper function to create a proxyManager
|
||||||
// etcdEndpoints is the address list of etcd
|
// etcdEndpoints is the address list of etcd
|
||||||
// fns are the custom getSessions function list
|
// fns are the custom getSessions function list
|
||||||
func newProxyManager(ctx context.Context, etcdEndpoints []string, fns ...func([]*sessionutil.Session)) (*proxyManager, error) {
|
func newProxyManager(ctx context.Context, client *clientv3.Client, fns ...func([]*sessionutil.Session)) *proxyManager {
|
||||||
cli, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints})
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
ctx2, cancel2 := context.WithCancel(ctx)
|
ctx2, cancel2 := context.WithCancel(ctx)
|
||||||
p := &proxyManager{
|
p := &proxyManager{
|
||||||
ctx: ctx2,
|
ctx: ctx2,
|
||||||
cancel: cancel2,
|
cancel: cancel2,
|
||||||
lock: sync.Mutex{},
|
lock: sync.Mutex{},
|
||||||
etcdCli: cli,
|
etcdCli: client,
|
||||||
}
|
}
|
||||||
p.getSessions = append(p.getSessions, fns...)
|
p.getSessions = append(p.getSessions, fns...)
|
||||||
return p, nil
|
return p
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddSession adds functions to addSessions function list
|
// AddSession adds functions to addSessions function list
|
||||||
|
@ -23,6 +23,7 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
@ -31,22 +32,23 @@ import (
|
|||||||
|
|
||||||
func TestProxyManager(t *testing.T) {
|
func TestProxyManager(t *testing.T) {
|
||||||
Params.Init()
|
Params.Init()
|
||||||
cli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints})
|
|
||||||
assert.Nil(t, err)
|
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||||
cli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||||
defer cli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
defer etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||||
s1 := sessionutil.Session{
|
s1 := sessionutil.Session{
|
||||||
ServerID: 100,
|
ServerID: 100,
|
||||||
}
|
}
|
||||||
b1, err := json.Marshal(&s1)
|
b1, err := json.Marshal(&s1)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
k1 := path.Join(sessKey, typeutil.ProxyRole+"-100")
|
k1 := path.Join(sessKey, typeutil.ProxyRole+"-100")
|
||||||
_, err = cli.Put(ctx, k1, string(b1))
|
_, err = etcdCli.Put(ctx, k1, string(b1))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
s0 := sessionutil.Session{
|
s0 := sessionutil.Session{
|
||||||
@ -55,7 +57,7 @@ func TestProxyManager(t *testing.T) {
|
|||||||
b0, err := json.Marshal(&s0)
|
b0, err := json.Marshal(&s0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
k0 := path.Join(sessKey, typeutil.ProxyRole+"-99")
|
k0 := path.Join(sessKey, typeutil.ProxyRole+"-99")
|
||||||
_, err = cli.Put(ctx, k0, string(b0))
|
_, err = etcdCli.Put(ctx, k0, string(b0))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
f1 := func(sess []*sessionutil.Session) {
|
f1 := func(sess []*sessionutil.Session) {
|
||||||
@ -64,8 +66,7 @@ func TestProxyManager(t *testing.T) {
|
|||||||
assert.Equal(t, int64(99), sess[1].ServerID)
|
assert.Equal(t, int64(99), sess[1].ServerID)
|
||||||
t.Log("get sessions", sess[0], sess[1])
|
t.Log("get sessions", sess[0], sess[1])
|
||||||
}
|
}
|
||||||
|
pm := newProxyManager(ctx, etcdCli, f1)
|
||||||
pm, err := newProxyManager(ctx, Params.RootCoordCfg.EtcdEndpoints, f1)
|
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
fa := func(sess *sessionutil.Session) {
|
fa := func(sess *sessionutil.Session) {
|
||||||
assert.Equal(t, int64(101), sess.ServerID)
|
assert.Equal(t, int64(101), sess.ServerID)
|
||||||
@ -88,10 +89,10 @@ func TestProxyManager(t *testing.T) {
|
|||||||
b2, err := json.Marshal(&s2)
|
b2, err := json.Marshal(&s2)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
k2 := path.Join(sessKey, typeutil.ProxyRole+"-101")
|
k2 := path.Join(sessKey, typeutil.ProxyRole+"-101")
|
||||||
_, err = cli.Put(ctx, k2, string(b2))
|
_, err = etcdCli.Put(ctx, k2, string(b2))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
_, err = cli.Delete(ctx, k1)
|
_, err = etcdCli.Delete(ctx, k1)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
pm.Stop()
|
pm.Stop()
|
||||||
|
@ -926,17 +926,23 @@ func (c *Core) Register() error {
|
|||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
if c.session.TriggerKill {
|
||||||
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
||||||
|
}
|
||||||
})
|
})
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *Core) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||||
|
c.etcdCli = etcdClient
|
||||||
|
}
|
||||||
|
|
||||||
func (c *Core) initSession() error {
|
func (c *Core) initSession() error {
|
||||||
c.session = sessionutil.NewSession(c.ctx, Params.RootCoordCfg.MetaRootPath, Params.RootCoordCfg.EtcdEndpoints)
|
c.session = sessionutil.NewSession(c.ctx, Params.RootCoordCfg.MetaRootPath, c.etcdCli)
|
||||||
if c.session == nil {
|
if c.session == nil {
|
||||||
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
||||||
}
|
}
|
||||||
c.session.Init(typeutil.RootCoordRole, Params.RootCoordCfg.Address, true)
|
c.session.Init(typeutil.RootCoordRole, Params.RootCoordCfg.Address, true, true)
|
||||||
Params.BaseParams.SetLogger(c.session.ServerID)
|
Params.BaseParams.SetLogger(c.session.ServerID)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -946,7 +952,7 @@ func (c *Core) Init() error {
|
|||||||
var initError error
|
var initError error
|
||||||
if c.kvBaseCreate == nil {
|
if c.kvBaseCreate == nil {
|
||||||
c.kvBaseCreate = func(root string) (kv.TxnKV, error) {
|
c.kvBaseCreate = func(root string) (kv.TxnKV, error) {
|
||||||
return etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, root)
|
return etcdkv.NewEtcdKV(c.etcdCli, root), nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
c.initOnce.Do(func() {
|
c.initOnce.Do(func() {
|
||||||
@ -956,10 +962,6 @@ func (c *Core) Init() error {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
connectEtcdFn := func() error {
|
connectEtcdFn := func() error {
|
||||||
if c.etcdCli, initError = clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints, DialTimeout: 5 * time.Second}); initError != nil {
|
|
||||||
log.Error("RootCoord failed to new Etcd client", zap.Any("reason", initError))
|
|
||||||
return initError
|
|
||||||
}
|
|
||||||
if c.kvBase, initError = c.kvBaseCreate(Params.RootCoordCfg.KvRootPath); initError != nil {
|
if c.kvBase, initError = c.kvBaseCreate(Params.RootCoordCfg.KvRootPath); initError != nil {
|
||||||
log.Error("RootCoord failed to new EtcdKV", zap.Any("reason", initError))
|
log.Error("RootCoord failed to new EtcdKV", zap.Any("reason", initError))
|
||||||
return initError
|
return initError
|
||||||
@ -989,10 +991,7 @@ func (c *Core) Init() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
log.Debug("RootCoord, Setting TSO and ID Allocator")
|
log.Debug("RootCoord, Setting TSO and ID Allocator")
|
||||||
kv, initError := tsoutil.NewTSOKVBase(Params.RootCoordCfg.EtcdEndpoints, Params.RootCoordCfg.KvRootPath, "gid")
|
kv := tsoutil.NewTSOKVBase(c.etcdCli, Params.RootCoordCfg.KvRootPath, "gid")
|
||||||
if initError != nil {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", kv)
|
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", kv)
|
||||||
if initError = idAllocator.Initialize(); initError != nil {
|
if initError = idAllocator.Initialize(); initError != nil {
|
||||||
return
|
return
|
||||||
@ -1004,10 +1003,7 @@ func (c *Core) Init() error {
|
|||||||
return idAllocator.UpdateID()
|
return idAllocator.UpdateID()
|
||||||
}
|
}
|
||||||
|
|
||||||
kv, initError = tsoutil.NewTSOKVBase(Params.RootCoordCfg.EtcdEndpoints, Params.RootCoordCfg.KvRootPath, "tso")
|
kv = tsoutil.NewTSOKVBase(c.etcdCli, Params.RootCoordCfg.KvRootPath, "tso")
|
||||||
if initError != nil {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
tsoAllocator := tso.NewGlobalTSOAllocator("timestamp", kv)
|
tsoAllocator := tso.NewGlobalTSOAllocator("timestamp", kv)
|
||||||
if initError = tsoAllocator.Initialize(); initError != nil {
|
if initError = tsoAllocator.Initialize(); initError != nil {
|
||||||
return
|
return
|
||||||
@ -1033,15 +1029,12 @@ func (c *Core) Init() error {
|
|||||||
c.proxyClientManager = newProxyClientManager(c)
|
c.proxyClientManager = newProxyClientManager(c)
|
||||||
|
|
||||||
log.Debug("RootCoord, set proxy manager")
|
log.Debug("RootCoord, set proxy manager")
|
||||||
c.proxyManager, initError = newProxyManager(
|
c.proxyManager = newProxyManager(
|
||||||
c.ctx,
|
c.ctx,
|
||||||
Params.RootCoordCfg.EtcdEndpoints,
|
c.etcdCli,
|
||||||
c.chanTimeTick.getProxy,
|
c.chanTimeTick.getProxy,
|
||||||
c.proxyClientManager.GetProxyClients,
|
c.proxyClientManager.GetProxyClients,
|
||||||
)
|
)
|
||||||
if initError != nil {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
c.proxyManager.AddSession(c.chanTimeTick.addProxy, c.proxyClientManager.AddProxyClient)
|
c.proxyManager.AddSession(c.chanTimeTick.addProxy, c.proxyClientManager.AddProxyClient)
|
||||||
c.proxyManager.DelSession(c.chanTimeTick.delProxy, c.proxyClientManager.DelProxyClient)
|
c.proxyManager.DelSession(c.chanTimeTick.delProxy, c.proxyClientManager.DelProxyClient)
|
||||||
|
|
||||||
|
@ -37,6 +37,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||||
@ -48,10 +49,10 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
|
||||||
clientv3 "go.etcd.io/etcd/client/v3"
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -433,9 +434,15 @@ func TestRootCoordInit(t *testing.T) {
|
|||||||
coreFactory := msgstream.NewPmsFactory()
|
coreFactory := msgstream.NewPmsFactory()
|
||||||
Params.Init()
|
Params.Init()
|
||||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
core, err := NewCore(ctx, coreFactory)
|
core, err := NewCore(ctx, coreFactory)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
randVal := rand.Int()
|
randVal := rand.Int()
|
||||||
|
|
||||||
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
|
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
|
||||||
@ -443,12 +450,13 @@ func TestRootCoordInit(t *testing.T) {
|
|||||||
|
|
||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
// inject kvBaseCreate fail
|
// inject kvBaseCreate fail
|
||||||
core, err = NewCore(ctx, coreFactory)
|
core, err = NewCore(ctx, coreFactory)
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
randVal = rand.Int()
|
randVal = rand.Int()
|
||||||
@ -462,11 +470,13 @@ func TestRootCoordInit(t *testing.T) {
|
|||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
// inject metaKV create fail
|
// inject metaKV create fail
|
||||||
core, err = NewCore(ctx, coreFactory)
|
core, err = NewCore(ctx, coreFactory)
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
randVal = rand.Int()
|
randVal = rand.Int()
|
||||||
@ -483,11 +493,13 @@ func TestRootCoordInit(t *testing.T) {
|
|||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
// inject newSuffixSnapshot failure
|
// inject newSuffixSnapshot failure
|
||||||
core, err = NewCore(ctx, coreFactory)
|
core, err = NewCore(ctx, coreFactory)
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
randVal = rand.Int()
|
randVal = rand.Int()
|
||||||
@ -501,11 +513,13 @@ func TestRootCoordInit(t *testing.T) {
|
|||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
// inject newMetaTable failure
|
// inject newMetaTable failure
|
||||||
core, err = NewCore(ctx, coreFactory)
|
core, err = NewCore(ctx, coreFactory)
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
randVal = rand.Int()
|
randVal = rand.Int()
|
||||||
@ -520,6 +534,7 @@ func TestRootCoordInit(t *testing.T) {
|
|||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -544,7 +559,6 @@ func TestRootCoord(t *testing.T) {
|
|||||||
core, err := NewCore(ctx, coreFactory)
|
core, err := NewCore(ctx, coreFactory)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
randVal := rand.Int()
|
randVal := rand.Int()
|
||||||
|
|
||||||
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
|
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
|
||||||
Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal)
|
Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal)
|
||||||
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
|
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
|
||||||
@ -553,8 +567,10 @@ func TestRootCoord(t *testing.T) {
|
|||||||
Params.RootCoordCfg.DmlChannelName = fmt.Sprintf("rootcoord-dml-test-%d", randVal)
|
Params.RootCoordCfg.DmlChannelName = fmt.Sprintf("rootcoord-dml-test-%d", randVal)
|
||||||
Params.RootCoordCfg.DeltaChannelName = fmt.Sprintf("rootcoord-delta-test-%d", randVal)
|
Params.RootCoordCfg.DeltaChannelName = fmt.Sprintf("rootcoord-delta-test-%d", randVal)
|
||||||
|
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: Params.RootCoordCfg.EtcdEndpoints, DialTimeout: 5 * time.Second})
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
assert.Nil(t, err)
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
sessKey := path.Join(Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||||
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -616,6 +632,8 @@ func TestRootCoord(t *testing.T) {
|
|||||||
dmlStream, _ := tmpFactory.NewMsgStream(ctx)
|
dmlStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||||
clearMsgChan(1500*time.Millisecond, dmlStream.Chan())
|
clearMsgChan(1500*time.Millisecond, dmlStream.Chan())
|
||||||
|
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
|
|
||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -631,13 +649,18 @@ func TestRootCoord(t *testing.T) {
|
|||||||
err = core.Start()
|
err = core.Start()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
shardsNum := int32(8)
|
shardsNum := int32(8)
|
||||||
|
|
||||||
|
fmt.Printf("hello world2")
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("time tick", func(t *testing.T) {
|
t.Run("time tick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ttmsg, ok := <-timeTickStream.Chan()
|
ttmsg, ok := <-timeTickStream.Chan()
|
||||||
assert.True(t, ok)
|
assert.True(t, ok)
|
||||||
assert.Equal(t, 1, len(ttmsg.Msgs))
|
assert.Equal(t, 1, len(ttmsg.Msgs))
|
||||||
@ -655,7 +678,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, ttm2.Base.Timestamp, ttm.Base.Timestamp+1)
|
assert.Equal(t, ttm2.Base.Timestamp, ttm.Base.Timestamp+1)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("create collection", func(t *testing.T) {
|
t.Run("create collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
schema := schemapb.CollectionSchema{
|
schema := schemapb.CollectionSchema{
|
||||||
Name: collName,
|
Name: collName,
|
||||||
AutoID: true,
|
AutoID: true,
|
||||||
@ -789,7 +814,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("has collection", func(t *testing.T) {
|
t.Run("has collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.HasCollectionRequest{
|
req := &milvuspb.HasCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasCollection,
|
MsgType: commonpb.MsgType_HasCollection,
|
||||||
@ -837,7 +864,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, true, rsp.Value)
|
assert.Equal(t, true, rsp.Value)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("describe collection", func(t *testing.T) {
|
t.Run("describe collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
|
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
req := &milvuspb.DescribeCollectionRequest{
|
req := &milvuspb.DescribeCollectionRequest{
|
||||||
@ -860,7 +889,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, shardsNum, rsp.ShardsNum)
|
assert.Equal(t, shardsNum, rsp.ShardsNum)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("show collection", func(t *testing.T) {
|
t.Run("show collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.ShowCollectionsRequest{
|
req := &milvuspb.ShowCollectionsRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_ShowCollections,
|
MsgType: commonpb.MsgType_ShowCollections,
|
||||||
@ -877,7 +908,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, len(rsp.CollectionNames), 2)
|
assert.Equal(t, len(rsp.CollectionNames), 2)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("create partition", func(t *testing.T) {
|
t.Run("create partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.CreatePartitionRequest{
|
req := &milvuspb.CreatePartitionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_CreatePartition,
|
MsgType: commonpb.MsgType_CreatePartition,
|
||||||
@ -932,7 +965,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("has partition", func(t *testing.T) {
|
t.Run("has partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.HasPartitionRequest{
|
req := &milvuspb.HasPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasPartition,
|
MsgType: commonpb.MsgType_HasPartition,
|
||||||
@ -950,7 +985,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, true, rsp.Value)
|
assert.Equal(t, true, rsp.Value)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("show partition", func(t *testing.T) {
|
t.Run("show partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
req := &milvuspb.ShowPartitionsRequest{
|
req := &milvuspb.ShowPartitionsRequest{
|
||||||
@ -971,7 +1008,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, 2, len(rsp.PartitionIDs))
|
assert.Equal(t, 2, len(rsp.PartitionIDs))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("show segment", func(t *testing.T) {
|
t.Run("show segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
partID := coll.PartitionIDs[1]
|
partID := coll.PartitionIDs[1]
|
||||||
@ -996,7 +1035,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, 1, len(rsp.SegmentIDs))
|
assert.Equal(t, 1, len(rsp.SegmentIDs))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("create index", func(t *testing.T) {
|
t.Run("create index", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.CreateIndexRequest{
|
req := &milvuspb.CreateIndexRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_CreateIndex,
|
MsgType: commonpb.MsgType_CreateIndex,
|
||||||
@ -1038,7 +1079,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("describe segment", func(t *testing.T) {
|
t.Run("describe segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -1058,7 +1101,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
t.Logf("index id = %d", rsp.IndexID)
|
t.Logf("index id = %d", rsp.IndexID)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("describe index", func(t *testing.T) {
|
t.Run("describe index", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DescribeIndexRequest{
|
req := &milvuspb.DescribeIndexRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DescribeIndex,
|
MsgType: commonpb.MsgType_DescribeIndex,
|
||||||
@ -1079,7 +1124,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, "vector", rsp.IndexDescriptions[0].FieldName)
|
assert.Equal(t, "vector", rsp.IndexDescriptions[0].FieldName)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("describe index not exist", func(t *testing.T) {
|
t.Run("describe index not exist", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DescribeIndexRequest{
|
req := &milvuspb.DescribeIndexRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DescribeIndex,
|
MsgType: commonpb.MsgType_DescribeIndex,
|
||||||
@ -1098,7 +1145,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, 0, len(rsp.IndexDescriptions))
|
assert.Equal(t, 0, len(rsp.IndexDescriptions))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("flush segment", func(t *testing.T) {
|
t.Run("flush segment", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
coll, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
partID := coll.PartitionIDs[1]
|
partID := coll.PartitionIDs[1]
|
||||||
@ -1136,7 +1185,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
|
assert.Equal(t, Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("over ride index", func(t *testing.T) {
|
t.Run("over ride index", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.CreateIndexRequest{
|
req := &milvuspb.CreateIndexRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_CreateIndex,
|
MsgType: commonpb.MsgType_CreateIndex,
|
||||||
@ -1180,7 +1231,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
|
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop index", func(t *testing.T) {
|
t.Run("drop index", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DropIndexRequest{
|
req := &milvuspb.DropIndexRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DropIndex,
|
MsgType: commonpb.MsgType_DropIndex,
|
||||||
@ -1211,7 +1264,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, 0, len(idx))
|
assert.Equal(t, 0, len(idx))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop partition", func(t *testing.T) {
|
t.Run("drop partition", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DropPartitionRequest{
|
req := &milvuspb.DropPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DropPartition,
|
MsgType: commonpb.MsgType_DropPartition,
|
||||||
@ -1267,7 +1322,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("remove DQL msgstream", func(t *testing.T) {
|
t.Run("remove DQL msgstream", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
|
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -1283,7 +1340,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop collection", func(t *testing.T) {
|
t.Run("drop collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DropCollectionRequest{
|
req := &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DropCollection,
|
MsgType: commonpb.MsgType_DropCollection,
|
||||||
@ -1356,7 +1415,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("context_cancel", func(t *testing.T) {
|
t.Run("context_cancel", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ctx2, cancel2 := context.WithTimeout(ctx, time.Millisecond*100)
|
ctx2, cancel2 := context.WithTimeout(ctx, time.Millisecond*100)
|
||||||
defer cancel2()
|
defer cancel2()
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
@ -1516,7 +1577,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
time.Sleep(1 * time.Second)
|
time.Sleep(1 * time.Second)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("undefined req type", func(t *testing.T) {
|
t.Run("undefined req type", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_Undefined,
|
MsgType: commonpb.MsgType_Undefined,
|
||||||
@ -1673,7 +1736,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
|
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("alloc time tick", func(t *testing.T) {
|
t.Run("alloc time tick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &rootcoordpb.AllocTimestampRequest{
|
req := &rootcoordpb.AllocTimestampRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_Undefined,
|
MsgType: commonpb.MsgType_Undefined,
|
||||||
@ -1689,7 +1754,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.NotZero(t, rsp.Timestamp)
|
assert.NotZero(t, rsp.Timestamp)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("alloc id", func(t *testing.T) {
|
t.Run("alloc id", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &rootcoordpb.AllocIDRequest{
|
req := &rootcoordpb.AllocIDRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_Undefined,
|
MsgType: commonpb.MsgType_Undefined,
|
||||||
@ -1705,14 +1772,18 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.NotZero(t, rsp.ID)
|
assert.NotZero(t, rsp.ID)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get_channels", func(t *testing.T) {
|
t.Run("get_channels", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
_, err := core.GetTimeTickChannel(ctx)
|
_, err := core.GetTimeTickChannel(ctx)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
_, err = core.GetStatisticsChannel(ctx)
|
_, err = core.GetStatisticsChannel(ctx)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("channel timetick", func(t *testing.T) {
|
t.Run("channel timetick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
const (
|
const (
|
||||||
proxyIDInvalid = 102
|
proxyIDInvalid = 102
|
||||||
proxyName0 = "proxy_0"
|
proxyName0 = "proxy_0"
|
||||||
@ -1825,7 +1896,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("create alias", func(t *testing.T) {
|
t.Run("create alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.CreateAliasRequest{
|
req := &milvuspb.CreateAliasRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_CreateAlias,
|
MsgType: commonpb.MsgType_CreateAlias,
|
||||||
@ -1841,7 +1914,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("describe collection2", func(t *testing.T) {
|
t.Run("describe collection2", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DescribeCollectionRequest{
|
req := &milvuspb.DescribeCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DescribeCollection,
|
MsgType: commonpb.MsgType_DescribeCollection,
|
||||||
@ -1879,7 +1954,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("alter alias", func(t *testing.T) {
|
t.Run("alter alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.AlterAliasRequest{
|
req := &milvuspb.AlterAliasRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_AlterAlias,
|
MsgType: commonpb.MsgType_AlterAlias,
|
||||||
@ -1895,7 +1972,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop collection with alias", func(t *testing.T) {
|
t.Run("drop collection with alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DropCollectionRequest{
|
req := &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DropAlias,
|
MsgType: commonpb.MsgType_DropAlias,
|
||||||
@ -1910,7 +1989,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
assert.NotEqual(t, commonpb.ErrorCode_Success, rsp.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("drop alias", func(t *testing.T) {
|
t.Run("drop alias", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
req := &milvuspb.DropAliasRequest{
|
req := &milvuspb.DropAliasRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DropAlias,
|
MsgType: commonpb.MsgType_DropAlias,
|
||||||
@ -1951,7 +2032,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get metrics", func(t *testing.T) {
|
t.Run("get metrics", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
// not healthy
|
// not healthy
|
||||||
stateSave := core.stateCode.Load().(internalpb.StateCode)
|
stateSave := core.stateCode.Load().(internalpb.StateCode)
|
||||||
core.UpdateStateCode(internalpb.StateCode_Abnormal)
|
core.UpdateStateCode(internalpb.StateCode_Abnormal)
|
||||||
@ -1985,7 +2068,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("get system info", func(t *testing.T) {
|
t.Run("get system info", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
// normal case
|
// normal case
|
||||||
systemInfoMetricType := metricsinfo.SystemInfoMetrics
|
systemInfoMetricType := metricsinfo.SystemInfoMetrics
|
||||||
req, err := metricsinfo.ConstructRequestByMetricType(systemInfoMetricType)
|
req, err := metricsinfo.ConstructRequestByMetricType(systemInfoMetricType)
|
||||||
@ -2002,7 +2087,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Equal(t, commonpb.ErrorCode_Success, st.Status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, st.Status.ErrorCode)
|
||||||
assert.NotEqual(t, internalpb.StateCode_Healthy, st.State.StateCode)
|
assert.NotEqual(t, internalpb.StateCode_Healthy, st.State.StateCode)
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("state_not_healthy", func(t *testing.T) {
|
t.Run("state_not_healthy", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
st, err := core.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_CreateCollection,
|
MsgType: commonpb.MsgType_CreateCollection,
|
||||||
@ -2159,7 +2246,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
|
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("alloc_error", func(t *testing.T) {
|
t.Run("alloc_error", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
core.Stop()
|
core.Stop()
|
||||||
core.IDAllocator = func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) {
|
core.IDAllocator = func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) {
|
||||||
return 0, 0, fmt.Errorf("id allocator error test")
|
return 0, 0, fmt.Errorf("id allocator error test")
|
||||||
@ -2195,9 +2284,9 @@ func TestRootCoord(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.NotEqual(t, commonpb.ErrorCode_Success, p2.Status.ErrorCode)
|
assert.NotEqual(t, commonpb.ErrorCode_Success, p2.Status.ErrorCode)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
err = core.Stop()
|
err = core.Stop()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRootCoord2(t *testing.T) {
|
func TestRootCoord2(t *testing.T) {
|
||||||
@ -2215,6 +2304,11 @@ func TestRootCoord2(t *testing.T) {
|
|||||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||||
core, err := NewCore(ctx, msFactory)
|
core, err := NewCore(ctx, msFactory)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
randVal := rand.Int()
|
randVal := rand.Int()
|
||||||
|
|
||||||
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
|
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
|
||||||
@ -2248,12 +2342,14 @@ func TestRootCoord2(t *testing.T) {
|
|||||||
return nil, nil
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
err = core.Start()
|
err = core.Start()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -2270,7 +2366,10 @@ func TestRootCoord2(t *testing.T) {
|
|||||||
|
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("time tick", func(t *testing.T) {
|
t.Run("time tick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ttmsg, ok := <-timeTickStream.Chan()
|
ttmsg, ok := <-timeTickStream.Chan()
|
||||||
assert.True(t, ok)
|
assert.True(t, ok)
|
||||||
assert.Equal(t, 1, len(ttmsg.Msgs))
|
assert.Equal(t, 1, len(ttmsg.Msgs))
|
||||||
@ -2279,7 +2378,9 @@ func TestRootCoord2(t *testing.T) {
|
|||||||
assert.Greater(t, ttm.Base.Timestamp, typeutil.Timestamp(0))
|
assert.Greater(t, ttm.Base.Timestamp, typeutil.Timestamp(0))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("create collection", func(t *testing.T) {
|
t.Run("create collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
schema := schemapb.CollectionSchema{
|
schema := schemapb.CollectionSchema{
|
||||||
Name: collName,
|
Name: collName,
|
||||||
}
|
}
|
||||||
@ -2314,7 +2415,9 @@ func TestRootCoord2(t *testing.T) {
|
|||||||
t.Log("time tick", m1.Base.Timestamp)
|
t.Log("time tick", m1.Base.Timestamp)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("describe collection", func(t *testing.T) {
|
t.Run("describe collection", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
|
collMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
req := &milvuspb.DescribeCollectionRequest{
|
req := &milvuspb.DescribeCollectionRequest{
|
||||||
@ -2336,6 +2439,7 @@ func TestRootCoord2(t *testing.T) {
|
|||||||
assert.Equal(t, common.DefaultShardsNum, int32(len(rsp.PhysicalChannelNames)))
|
assert.Equal(t, common.DefaultShardsNum, int32(len(rsp.PhysicalChannelNames)))
|
||||||
assert.Equal(t, common.DefaultShardsNum, rsp.ShardsNum)
|
assert.Equal(t, common.DefaultShardsNum, rsp.ShardsNum)
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
err = core.Stop()
|
err = core.Stop()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
@ -2516,12 +2620,17 @@ func TestCheckFlushedSegments(t *testing.T) {
|
|||||||
return nil, nil
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
err = core.Start()
|
err = core.Start()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
@ -2537,7 +2646,11 @@ func TestCheckFlushedSegments(t *testing.T) {
|
|||||||
timeTickStream.Start()
|
timeTickStream.Start()
|
||||||
|
|
||||||
time.Sleep(100 * time.Millisecond)
|
time.Sleep(100 * time.Millisecond)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("check flushed segments", func(t *testing.T) {
|
t.Run("check flushed segments", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
var collID int64 = 1
|
var collID int64 = 1
|
||||||
var partID int64 = 2
|
var partID int64 = 2
|
||||||
@ -2620,6 +2733,7 @@ func TestCheckFlushedSegments(t *testing.T) {
|
|||||||
core.checkFlushedSegments(core.ctx)
|
core.checkFlushedSegments(core.ctx)
|
||||||
|
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
err = core.Stop()
|
err = core.Stop()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
@ -2641,7 +2755,6 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) {
|
|||||||
core, err := NewCore(ctx, msFactory)
|
core, err := NewCore(ctx, msFactory)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
randVal := rand.Int()
|
randVal := rand.Int()
|
||||||
|
|
||||||
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
|
Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
|
||||||
Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal)
|
Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("rootcoord-statistics-%d", randVal)
|
||||||
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
|
Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.RootCoordCfg.MetaRootPath)
|
||||||
@ -2673,12 +2786,18 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) {
|
|||||||
return nil, nil
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
core.SetEtcdClient(etcdCli)
|
||||||
err = core.Init()
|
err = core.Init()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
err = core.Start()
|
err = core.Start()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
core.session.TriggerKill = false
|
||||||
err = core.Register()
|
err = core.Register()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
@ -23,6 +23,7 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
@ -262,8 +263,10 @@ func Test_SuffixSnapshotLoad(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
sep := "_ts"
|
sep := "_ts"
|
||||||
|
|
||||||
etcdkv, err := etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, rootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
defer etcdkv.Close()
|
defer etcdkv.Close()
|
||||||
|
|
||||||
var vtso typeutil.Timestamp
|
var vtso typeutil.Timestamp
|
||||||
@ -312,8 +315,10 @@ func Test_SuffixSnapshotMultiSave(t *testing.T) {
|
|||||||
Params.Init()
|
Params.Init()
|
||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
sep := "_ts"
|
sep := "_ts"
|
||||||
etcdkv, err := etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, rootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
defer etcdkv.Close()
|
defer etcdkv.Close()
|
||||||
|
|
||||||
var vtso typeutil.Timestamp
|
var vtso typeutil.Timestamp
|
||||||
@ -386,7 +391,10 @@ func Test_SuffixSnapshotMultiSaveAndRemoveWithPrefix(t *testing.T) {
|
|||||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||||
sep := "_ts"
|
sep := "_ts"
|
||||||
|
|
||||||
etcdkv, err := etcdkv.NewEtcdKV(Params.RootCoordCfg.EtcdEndpoints, rootPath)
|
etcdCli, err := etcd.GetEtcdClient(&Params.BaseParams)
|
||||||
|
require.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdkv := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
defer etcdkv.Close()
|
defer etcdkv.Close()
|
||||||
|
|
||||||
|
@ -18,6 +18,7 @@ package rootcoord
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
@ -52,7 +53,10 @@ func TestTimetickSync(t *testing.T) {
|
|||||||
Params.RootCoordCfg.DeltaChannelName = "rootcoord-delta"
|
Params.RootCoordCfg.DeltaChannelName = "rootcoord-delta"
|
||||||
ttSync := newTimeTickSync(ctx, session, factory, nil)
|
ttSync := newTimeTickSync(ctx, session, factory, nil)
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
wg.Add(1)
|
||||||
t.Run("sendToChannel", func(t *testing.T) {
|
t.Run("sendToChannel", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ttSync.sendToChannel()
|
ttSync.sendToChannel()
|
||||||
|
|
||||||
ttSync.proxyTimeTick[1] = nil
|
ttSync.proxyTimeTick[1] = nil
|
||||||
@ -67,14 +71,18 @@ func TestTimetickSync(t *testing.T) {
|
|||||||
ttSync.sendToChannel()
|
ttSync.sendToChannel()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("RemoveDdlTimeTick", func(t *testing.T) {
|
t.Run("RemoveDdlTimeTick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
ttSync.addDdlTimeTick(uint64(1), "1")
|
ttSync.addDdlTimeTick(uint64(1), "1")
|
||||||
ttSync.addDdlTimeTick(uint64(2), "2")
|
ttSync.addDdlTimeTick(uint64(2), "2")
|
||||||
ttSync.removeDdlTimeTick(uint64(1), "1")
|
ttSync.removeDdlTimeTick(uint64(1), "1")
|
||||||
assert.Equal(t, ttSync.ddlMinTs, uint64(2))
|
assert.Equal(t, ttSync.ddlMinTs, uint64(2))
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("UpdateTimeTick", func(t *testing.T) {
|
t.Run("UpdateTimeTick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
msg := &internalpb.ChannelTimeTickMsg{
|
msg := &internalpb.ChannelTimeTickMsg{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_TimeTick,
|
MsgType: commonpb.MsgType_TimeTick,
|
||||||
@ -105,7 +113,9 @@ func TestTimetickSync(t *testing.T) {
|
|||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
t.Run("minTimeTick", func(t *testing.T) {
|
t.Run("minTimeTick", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
tts := make([]uint64, 2)
|
tts := make([]uint64, 2)
|
||||||
tts[0] = uint64(5)
|
tts[0] = uint64(5)
|
||||||
tts[1] = uint64(3)
|
tts[1] = uint64(3)
|
||||||
@ -113,4 +123,5 @@ func TestTimetickSync(t *testing.T) {
|
|||||||
ret := minTimeTick(tts...)
|
ret := minTimeTick(tts...)
|
||||||
assert.Equal(t, ret, tts[1])
|
assert.Equal(t, ret, tts[1])
|
||||||
})
|
})
|
||||||
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
@ -18,6 +18,7 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
@ -30,8 +31,11 @@ func TestGlobalTSOAllocator_Initialize(t *testing.T) {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
assert.NoError(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
|
||||||
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
||||||
err = gTestTsoAllocator.Initialize()
|
err = gTestTsoAllocator.Initialize()
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -67,8 +71,11 @@ func TestGlobalTSOAllocator_All(t *testing.T) {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
|
||||||
|
|
||||||
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
||||||
t.Run("Initialize", func(t *testing.T) {
|
t.Run("Initialize", func(t *testing.T) {
|
||||||
err := gTestTsoAllocator.Initialize()
|
err := gTestTsoAllocator.Initialize()
|
||||||
@ -153,7 +160,10 @@ func TestGlobalTSOAllocator_Fail(t *testing.T) {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
||||||
t.Run("Initialize", func(t *testing.T) {
|
t.Run("Initialize", func(t *testing.T) {
|
||||||
@ -194,7 +204,10 @@ func TestGlobalTSOAllocator_Update(t *testing.T) {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
||||||
err = gTestTsoAllocator.Initialize()
|
err = gTestTsoAllocator.Initialize()
|
||||||
@ -216,7 +229,10 @@ func TestGlobalTSOAllocator_load(t *testing.T) {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := tsoutil.NewTSOKVBase(etcdEndpoints, "/test/root/kv", "tsoTest")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := tsoutil.NewTSOKVBase(etcdCli, "/test/root/kv", "tsoTest")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
gTestTsoAllocator = NewGlobalTSOAllocator("timestamp", etcdKV)
|
||||||
err = gTestTsoAllocator.Initialize()
|
err = gTestTsoAllocator.Initialize()
|
||||||
|
@ -23,6 +23,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
// TimeTickProvider is the interface all services implement
|
// TimeTickProvider is the interface all services implement
|
||||||
@ -74,6 +75,9 @@ type DataNodeComponent interface {
|
|||||||
// GetStateCode return state code of this data node
|
// GetStateCode return state code of this data node
|
||||||
GetStateCode() internalpb.StateCode
|
GetStateCode() internalpb.StateCode
|
||||||
|
|
||||||
|
// SetEtcdClient set etcd client for DataNode
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
// SetRootCoord set RootCoord for DataNode
|
// SetRootCoord set RootCoord for DataNode
|
||||||
// `rootCoord` is a client of root coordinator.
|
// `rootCoord` is a client of root coordinator.
|
||||||
//
|
//
|
||||||
@ -248,6 +252,15 @@ type DataCoord interface {
|
|||||||
DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error)
|
DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// DataCoordComponent defines the interface of DataCoord component.
|
||||||
|
type DataCoordComponent interface {
|
||||||
|
DataCoord
|
||||||
|
|
||||||
|
// SetEtcdClient set EtcdClient for DataCoord
|
||||||
|
// `etcdClient` is a client of etcd
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
}
|
||||||
|
|
||||||
// IndexNode is the interface `indexnode` package implements
|
// IndexNode is the interface `indexnode` package implements
|
||||||
type IndexNode interface {
|
type IndexNode interface {
|
||||||
Component
|
Component
|
||||||
@ -261,6 +274,18 @@ type IndexNode interface {
|
|||||||
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// IndexNodeComponent is used by grpc server of IndexNode
|
||||||
|
type IndexNodeComponent interface {
|
||||||
|
IndexNode
|
||||||
|
|
||||||
|
// SetEtcdClient set etcd client for QueryCoord
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
|
// UpdateStateCode updates state code for QueryCoord
|
||||||
|
// `stateCode` is current statement of this query coord, indicating whether it's healthy.
|
||||||
|
UpdateStateCode(stateCode internalpb.StateCode)
|
||||||
|
}
|
||||||
|
|
||||||
// IndexCoord is the interface `indexcoord` package implements
|
// IndexCoord is the interface `indexcoord` package implements
|
||||||
type IndexCoord interface {
|
type IndexCoord interface {
|
||||||
Component
|
Component
|
||||||
@ -287,6 +312,18 @@ type IndexCoord interface {
|
|||||||
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// IndexCoordComponent is used by grpc server of IndexCoord
|
||||||
|
type IndexCoordComponent interface {
|
||||||
|
IndexCoord
|
||||||
|
|
||||||
|
// SetEtcdClient set etcd client for QueryCoord
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
|
// UpdateStateCode updates state code for QueryCoord
|
||||||
|
// `stateCode` is current statement of this query coord, indicating whether it's healthy.
|
||||||
|
UpdateStateCode(stateCode internalpb.StateCode)
|
||||||
|
}
|
||||||
|
|
||||||
// RootCoord is the interface `rootcoord` package implements
|
// RootCoord is the interface `rootcoord` package implements
|
||||||
type RootCoord interface {
|
type RootCoord interface {
|
||||||
Component
|
Component
|
||||||
@ -540,6 +577,10 @@ type RootCoord interface {
|
|||||||
type RootCoordComponent interface {
|
type RootCoordComponent interface {
|
||||||
RootCoord
|
RootCoord
|
||||||
|
|
||||||
|
// SetEtcdClient set EtcdClient for RootCoord
|
||||||
|
// `etcdClient` is a client of etcd
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
// UpdateStateCode updates state code for RootCoord
|
// UpdateStateCode updates state code for RootCoord
|
||||||
// State includes: Initializing, Healthy and Abnormal
|
// State includes: Initializing, Healthy and Abnormal
|
||||||
UpdateStateCode(internalpb.StateCode)
|
UpdateStateCode(internalpb.StateCode)
|
||||||
@ -606,6 +647,10 @@ type Proxy interface {
|
|||||||
type ProxyComponent interface {
|
type ProxyComponent interface {
|
||||||
Proxy
|
Proxy
|
||||||
|
|
||||||
|
// SetEtcdClient set EtcdClient for Proxy
|
||||||
|
// `etcdClient` is a client of etcd
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
// SetRootCoord set RootCoord for Proxy
|
// SetRootCoord set RootCoord for Proxy
|
||||||
// `rootCoord` is a client of root coordinator.
|
// `rootCoord` is a client of root coordinator.
|
||||||
SetRootCoordClient(rootCoord RootCoord)
|
SetRootCoordClient(rootCoord RootCoord)
|
||||||
@ -1021,6 +1066,9 @@ type QueryNodeComponent interface {
|
|||||||
// `stateCode` is current statement of this query node, indicating whether it's healthy.
|
// `stateCode` is current statement of this query node, indicating whether it's healthy.
|
||||||
UpdateStateCode(stateCode internalpb.StateCode)
|
UpdateStateCode(stateCode internalpb.StateCode)
|
||||||
|
|
||||||
|
// SetEtcdClient set etcd client for QueryNode
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
// SetRootCoord set RootCoord for QueryNode
|
// SetRootCoord set RootCoord for QueryNode
|
||||||
// `rootCoord` is a client of root coordinator. Pass to segmentLoader.
|
// `rootCoord` is a client of root coordinator. Pass to segmentLoader.
|
||||||
//
|
//
|
||||||
@ -1063,6 +1111,9 @@ type QueryCoord interface {
|
|||||||
type QueryCoordComponent interface {
|
type QueryCoordComponent interface {
|
||||||
QueryCoord
|
QueryCoord
|
||||||
|
|
||||||
|
// SetEtcdClient set etcd client for QueryCoord
|
||||||
|
SetEtcdClient(etcdClient *clientv3.Client)
|
||||||
|
|
||||||
// UpdateStateCode updates state code for QueryCoord
|
// UpdateStateCode updates state code for QueryCoord
|
||||||
// `stateCode` is current statement of this query coord, indicating whether it's healthy.
|
// `stateCode` is current statement of this query coord, indicating whether it's healthy.
|
||||||
UpdateStateCode(stateCode internalpb.StateCode)
|
UpdateStateCode(stateCode internalpb.StateCode)
|
||||||
|
80
internal/util/etcd/etcd_util.go
Normal file
80
internal/util/etcd/etcd_util.go
Normal file
@ -0,0 +1,80 @@
|
|||||||
|
// Licensed to the LF AI & Data foundation under one
|
||||||
|
// or more contributor license agreements. See the NOTICE file
|
||||||
|
// distributed with this work for additional information
|
||||||
|
// regarding copyright ownership. The ASF licenses this file
|
||||||
|
// to you under the Apache License, Version 2.0 (the
|
||||||
|
// "License"); you may not use this file except in compliance
|
||||||
|
// with the License. You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package etcd
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
|
"go.etcd.io/etcd/server/v3/embed"
|
||||||
|
"go.etcd.io/etcd/server/v3/etcdserver/api/v3client"
|
||||||
|
)
|
||||||
|
|
||||||
|
var EtcdServer *embed.Etcd
|
||||||
|
|
||||||
|
func InitEtcdServer(pt *paramtable.BaseParamTable) error {
|
||||||
|
if pt.UseEmbedEtcd {
|
||||||
|
path := pt.EtcdConfigPath
|
||||||
|
fmt.Println("path", path, "data", pt.EtcdDataDir)
|
||||||
|
var cfg *embed.Config
|
||||||
|
if len(path) > 0 {
|
||||||
|
cfgFromFile, err := embed.ConfigFromFile(path)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
cfg = cfgFromFile
|
||||||
|
} else {
|
||||||
|
cfg = embed.NewConfig()
|
||||||
|
}
|
||||||
|
cfg.Dir = pt.EtcdDataDir
|
||||||
|
e, err := embed.StartEtcd(cfg)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
EtcdServer = e
|
||||||
|
log.Info("finish init embedded etcd")
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func StopEtcdServer() {
|
||||||
|
if EtcdServer != nil {
|
||||||
|
EtcdServer.Close()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func GetEtcdClient(pt *paramtable.BaseParamTable) (*clientv3.Client, error) {
|
||||||
|
if pt.UseEmbedEtcd {
|
||||||
|
return GetEmbedEtcdClient()
|
||||||
|
}
|
||||||
|
return GetRemoteEtcdClient(pt.EtcdEndpoints)
|
||||||
|
}
|
||||||
|
|
||||||
|
func GetEmbedEtcdClient() (*clientv3.Client, error) {
|
||||||
|
client := v3client.New(EtcdServer.Server)
|
||||||
|
return client, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func GetRemoteEtcdClient(endpoints []string) (*clientv3.Client, error) {
|
||||||
|
return clientv3.New(clientv3.Config{
|
||||||
|
Endpoints: endpoints,
|
||||||
|
DialTimeout: 5 * time.Second,
|
||||||
|
})
|
||||||
|
}
|
55
internal/util/etcd/etcd_util_test.go
Normal file
55
internal/util/etcd/etcd_util_test.go
Normal file
@ -0,0 +1,55 @@
|
|||||||
|
// Licensed to the LF AI & Data foundation under one
|
||||||
|
// or more contributor license agreements. See the NOTICE file
|
||||||
|
// distributed with this work for additional information
|
||||||
|
// regarding copyright ownership. The ASF licenses this file
|
||||||
|
// to you under the Apache License, Version 2.0 (the
|
||||||
|
// "License"); you may not use this file except in compliance
|
||||||
|
// with the License. You may obtain a copy of the License at
|
||||||
|
//
|
||||||
|
// http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
//
|
||||||
|
// Unless required by applicable law or agreed to in writing, software
|
||||||
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
// See the License for the specific language governing permissions and
|
||||||
|
// limitations under the License.
|
||||||
|
|
||||||
|
package etcd
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"os"
|
||||||
|
"path"
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
)
|
||||||
|
|
||||||
|
var Params paramtable.GlobalParamTable
|
||||||
|
|
||||||
|
func TestEtcd(t *testing.T) {
|
||||||
|
Params.Init()
|
||||||
|
Params.BaseParams.UseEmbedEtcd = true
|
||||||
|
Params.BaseParams.EtcdDataDir = "/tmp/data"
|
||||||
|
err := InitEtcdServer(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
defer os.RemoveAll(Params.BaseParams.EtcdDataDir)
|
||||||
|
defer StopEtcdServer()
|
||||||
|
|
||||||
|
// port is binded
|
||||||
|
err = InitEtcdServer(&Params.BaseParams)
|
||||||
|
assert.Error(t, err)
|
||||||
|
|
||||||
|
etcdCli, err := GetEtcdClient(&Params.BaseParams)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
key := path.Join("test", "test")
|
||||||
|
_, err = etcdCli.Put(context.TODO(), key, "value")
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
resp, err := etcdCli.Get(context.TODO(), key)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.False(t, resp.Count < 1)
|
||||||
|
assert.Equal(t, string(resp.Kvs[0].Value), "value")
|
||||||
|
}
|
@ -112,7 +112,7 @@ func (c *ClientBase) resetConnection(client interface{}) {
|
|||||||
func (c *ClientBase) connect(ctx context.Context) error {
|
func (c *ClientBase) connect(ctx context.Context) error {
|
||||||
addr, err := c.getAddrFunc()
|
addr, err := c.getAddrFunc()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("failed to get cclient address", zap.Error(err))
|
log.Error("failed to get client address", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -289,7 +289,6 @@ type rootCoordConfig struct {
|
|||||||
Port int
|
Port int
|
||||||
|
|
||||||
PulsarAddress string
|
PulsarAddress string
|
||||||
EtcdEndpoints []string
|
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
KvRootPath string
|
KvRootPath string
|
||||||
|
|
||||||
@ -316,7 +315,6 @@ func (p *rootCoordConfig) init(bp *BaseParamTable) {
|
|||||||
p.BaseParams = bp
|
p.BaseParams = bp
|
||||||
|
|
||||||
p.initPulsarAddress()
|
p.initPulsarAddress()
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initKvRootPath()
|
p.initKvRootPath()
|
||||||
|
|
||||||
@ -345,14 +343,6 @@ func (p *rootCoordConfig) initPulsarAddress() {
|
|||||||
p.PulsarAddress = addr
|
p.PulsarAddress = addr
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *rootCoordConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *rootCoordConfig) initMetaRootPath() {
|
func (p *rootCoordConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -469,7 +459,6 @@ type proxyConfig struct {
|
|||||||
|
|
||||||
Alias string
|
Alias string
|
||||||
|
|
||||||
EtcdEndpoints []string
|
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
PulsarAddress string
|
PulsarAddress string
|
||||||
|
|
||||||
@ -507,7 +496,6 @@ type proxyConfig struct {
|
|||||||
func (p *proxyConfig) init(bp *BaseParamTable) {
|
func (p *proxyConfig) init(bp *BaseParamTable) {
|
||||||
p.BaseParams = bp
|
p.BaseParams = bp
|
||||||
|
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initPulsarAddress()
|
p.initPulsarAddress()
|
||||||
p.initRocksmqPath()
|
p.initRocksmqPath()
|
||||||
@ -654,14 +642,6 @@ func (p *proxyConfig) initPulsarMaxMessageSize() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *proxyConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *proxyConfig) initMetaRootPath() {
|
func (p *proxyConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -711,9 +691,8 @@ type queryCoordConfig struct {
|
|||||||
SearchResultChannelPrefix string
|
SearchResultChannelPrefix string
|
||||||
|
|
||||||
// --- etcd ---
|
// --- etcd ---
|
||||||
EtcdEndpoints []string
|
MetaRootPath string
|
||||||
MetaRootPath string
|
KvRootPath string
|
||||||
KvRootPath string
|
|
||||||
|
|
||||||
//--- Minio ---
|
//--- Minio ---
|
||||||
MinioEndPoint string
|
MinioEndPoint string
|
||||||
@ -752,7 +731,6 @@ func (p *queryCoordConfig) init(bp *BaseParamTable) {
|
|||||||
p.initTimeTickChannelName()
|
p.initTimeTickChannelName()
|
||||||
|
|
||||||
// --- etcd ---
|
// --- etcd ---
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initKvRootPath()
|
p.initKvRootPath()
|
||||||
|
|
||||||
@ -824,14 +802,6 @@ func (p *queryCoordConfig) initTimeTickChannelName() {
|
|||||||
p.TimeTickChannelName = strings.Join(s, "-")
|
p.TimeTickChannelName = strings.Join(s, "-")
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *queryCoordConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *queryCoordConfig) initMetaRootPath() {
|
func (p *queryCoordConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -980,7 +950,6 @@ type queryNodeConfig struct {
|
|||||||
|
|
||||||
PulsarAddress string
|
PulsarAddress string
|
||||||
RocksmqPath string
|
RocksmqPath string
|
||||||
EtcdEndpoints []string
|
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
|
|
||||||
Alias string
|
Alias string
|
||||||
@ -1053,7 +1022,6 @@ func (p *queryNodeConfig) init(bp *BaseParamTable) {
|
|||||||
|
|
||||||
p.initPulsarAddress()
|
p.initPulsarAddress()
|
||||||
p.initRocksmqPath()
|
p.initRocksmqPath()
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
|
|
||||||
p.initGracefulTime()
|
p.initGracefulTime()
|
||||||
@ -1237,15 +1205,6 @@ func (p *queryNodeConfig) initStatsChannelName() {
|
|||||||
p.StatsChannelName = strings.Join(s, "-")
|
p.StatsChannelName = strings.Join(s, "-")
|
||||||
}
|
}
|
||||||
|
|
||||||
// ETCD configs
|
|
||||||
func (p *queryNodeConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *queryNodeConfig) initMetaRootPath() {
|
func (p *queryNodeConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -1298,7 +1257,6 @@ type dataCoordConfig struct {
|
|||||||
Address string
|
Address string
|
||||||
|
|
||||||
// --- ETCD ---
|
// --- ETCD ---
|
||||||
EtcdEndpoints []string
|
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
KvRootPath string
|
KvRootPath string
|
||||||
SegmentBinlogSubPath string
|
SegmentBinlogSubPath string
|
||||||
@ -1352,7 +1310,6 @@ type dataCoordConfig struct {
|
|||||||
func (p *dataCoordConfig) init(bp *BaseParamTable) {
|
func (p *dataCoordConfig) init(bp *BaseParamTable) {
|
||||||
p.BaseParams = bp
|
p.BaseParams = bp
|
||||||
|
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initKvRootPath()
|
p.initKvRootPath()
|
||||||
p.initSegmentBinlogSubPath()
|
p.initSegmentBinlogSubPath()
|
||||||
@ -1394,14 +1351,6 @@ func (p *dataCoordConfig) init(bp *BaseParamTable) {
|
|||||||
p.initGCDropTolerance()
|
p.initGCDropTolerance()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *dataCoordConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *dataCoordConfig) initPulsarAddress() {
|
func (p *dataCoordConfig) initPulsarAddress() {
|
||||||
addr, err := p.BaseParams.Load("_PulsarAddress")
|
addr, err := p.BaseParams.Load("_PulsarAddress")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -1656,7 +1605,6 @@ type dataNodeConfig struct {
|
|||||||
MsgChannelSubName string
|
MsgChannelSubName string
|
||||||
|
|
||||||
// etcd
|
// etcd
|
||||||
EtcdEndpoints []string
|
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
ChannelWatchSubPath string
|
ChannelWatchSubPath string
|
||||||
|
|
||||||
@ -1689,7 +1637,6 @@ func (p *dataNodeConfig) init(bp *BaseParamTable) {
|
|||||||
p.initTimeTickChannelName()
|
p.initTimeTickChannelName()
|
||||||
p.initMsgChannelSubName()
|
p.initMsgChannelSubName()
|
||||||
|
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initChannelWatchPath()
|
p.initChannelWatchPath()
|
||||||
|
|
||||||
@ -1792,14 +1739,6 @@ func (p *dataNodeConfig) initMsgChannelSubName() {
|
|||||||
p.MsgChannelSubName = strings.Join(s, "-")
|
p.MsgChannelSubName = strings.Join(s, "-")
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *dataNodeConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *dataNodeConfig) initMetaRootPath() {
|
func (p *dataNodeConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -1883,7 +1822,6 @@ type indexCoordConfig struct {
|
|||||||
Address string
|
Address string
|
||||||
Port int
|
Port int
|
||||||
|
|
||||||
EtcdEndpoints []string
|
|
||||||
KvRootPath string
|
KvRootPath string
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
IndexStorageRootPath string
|
IndexStorageRootPath string
|
||||||
@ -1901,7 +1839,6 @@ type indexCoordConfig struct {
|
|||||||
func (p *indexCoordConfig) init(bp *BaseParamTable) {
|
func (p *indexCoordConfig) init(bp *BaseParamTable) {
|
||||||
p.BaseParams = bp
|
p.BaseParams = bp
|
||||||
|
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initKvRootPath()
|
p.initKvRootPath()
|
||||||
p.initMinIOAddress()
|
p.initMinIOAddress()
|
||||||
@ -1912,14 +1849,6 @@ func (p *indexCoordConfig) init(bp *BaseParamTable) {
|
|||||||
p.initIndexStorageRootPath()
|
p.initIndexStorageRootPath()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *indexCoordConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *indexCoordConfig) initMetaRootPath() {
|
func (p *indexCoordConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -2013,7 +1942,6 @@ type indexNodeConfig struct {
|
|||||||
NodeID int64
|
NodeID int64
|
||||||
Alias string
|
Alias string
|
||||||
|
|
||||||
EtcdEndpoints []string
|
|
||||||
MetaRootPath string
|
MetaRootPath string
|
||||||
IndexStorageRootPath string
|
IndexStorageRootPath string
|
||||||
|
|
||||||
@ -2037,7 +1965,6 @@ func (p *indexNodeConfig) init(bp *BaseParamTable) {
|
|||||||
p.initMinIOSecretAccessKey()
|
p.initMinIOSecretAccessKey()
|
||||||
p.initMinIOUseSSL()
|
p.initMinIOUseSSL()
|
||||||
p.initMinioBucketName()
|
p.initMinioBucketName()
|
||||||
p.initEtcdEndpoints()
|
|
||||||
p.initMetaRootPath()
|
p.initMetaRootPath()
|
||||||
p.initIndexStorageRootPath()
|
p.initIndexStorageRootPath()
|
||||||
p.initKnowhereSimdType()
|
p.initKnowhereSimdType()
|
||||||
@ -2083,14 +2010,6 @@ func (p *indexNodeConfig) initMinIOUseSSL() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *indexNodeConfig) initEtcdEndpoints() {
|
|
||||||
endpoints, err := p.BaseParams.Load("_EtcdEndpoints")
|
|
||||||
if err != nil {
|
|
||||||
panic(err)
|
|
||||||
}
|
|
||||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *indexNodeConfig) initMetaRootPath() {
|
func (p *indexNodeConfig) initMetaRootPath() {
|
||||||
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
rootPath, err := p.BaseParams.Load("etcd.rootPath")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -40,9 +40,6 @@ func TestGlobalParamTable(t *testing.T) {
|
|||||||
assert.NotEqual(t, Params.PulsarAddress, "")
|
assert.NotEqual(t, Params.PulsarAddress, "")
|
||||||
t.Logf("pulsar address = %s", Params.PulsarAddress)
|
t.Logf("pulsar address = %s", Params.PulsarAddress)
|
||||||
|
|
||||||
assert.NotZero(t, len(Params.EtcdEndpoints))
|
|
||||||
t.Logf("etcd endpoints = %s", Params.EtcdEndpoints)
|
|
||||||
|
|
||||||
assert.NotEqual(t, Params.MetaRootPath, "")
|
assert.NotEqual(t, Params.MetaRootPath, "")
|
||||||
t.Logf("meta root path = %s", Params.MetaRootPath)
|
t.Logf("meta root path = %s", Params.MetaRootPath)
|
||||||
|
|
||||||
@ -88,8 +85,6 @@ func TestGlobalParamTable(t *testing.T) {
|
|||||||
t.Run("test proxyConfig", func(t *testing.T) {
|
t.Run("test proxyConfig", func(t *testing.T) {
|
||||||
Params := GlobalParams.ProxyCfg
|
Params := GlobalParams.ProxyCfg
|
||||||
|
|
||||||
t.Logf("EtcdEndPoints: %v", Params.EtcdEndpoints)
|
|
||||||
|
|
||||||
t.Logf("MetaRootPath: %s", Params.MetaRootPath)
|
t.Logf("MetaRootPath: %s", Params.MetaRootPath)
|
||||||
|
|
||||||
t.Logf("PulsarAddress: %s", Params.PulsarAddress)
|
t.Logf("PulsarAddress: %s", Params.PulsarAddress)
|
||||||
@ -300,9 +295,6 @@ func TestGlobalParamTable(t *testing.T) {
|
|||||||
assert.Equal(t, name, "by-dev-dataNode-2")
|
assert.Equal(t, name, "by-dev-dataNode-2")
|
||||||
log.Println("MsgChannelSubName:", name)
|
log.Println("MsgChannelSubName:", name)
|
||||||
|
|
||||||
endpoints := Params.EtcdEndpoints
|
|
||||||
log.Println("EtcdEndpoints:", endpoints)
|
|
||||||
|
|
||||||
path1 = Params.MetaRootPath
|
path1 = Params.MetaRootPath
|
||||||
log.Println("MetaRootPath:", path1)
|
log.Println("MetaRootPath:", path1)
|
||||||
|
|
||||||
@ -336,8 +328,6 @@ func TestGlobalParamTable(t *testing.T) {
|
|||||||
|
|
||||||
t.Logf("Port: %v", Params.Port)
|
t.Logf("Port: %v", Params.Port)
|
||||||
|
|
||||||
t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints)
|
|
||||||
|
|
||||||
t.Logf("KvRootPath: %v", Params.KvRootPath)
|
t.Logf("KvRootPath: %v", Params.KvRootPath)
|
||||||
|
|
||||||
t.Logf("MetaRootPath: %v", Params.MetaRootPath)
|
t.Logf("MetaRootPath: %v", Params.MetaRootPath)
|
||||||
@ -374,8 +364,6 @@ func TestGlobalParamTable(t *testing.T) {
|
|||||||
|
|
||||||
t.Logf("Alias: %v", Params.Alias)
|
t.Logf("Alias: %v", Params.Alias)
|
||||||
|
|
||||||
t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints)
|
|
||||||
|
|
||||||
t.Logf("MetaRootPath: %v", Params.MetaRootPath)
|
t.Logf("MetaRootPath: %v", Params.MetaRootPath)
|
||||||
|
|
||||||
t.Logf("MinIOAddress: %v", Params.MinIOAddress)
|
t.Logf("MinIOAddress: %v", Params.MinIOAddress)
|
||||||
|
@ -21,6 +21,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/allocator"
|
"github.com/milvus-io/milvus/internal/allocator"
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -32,10 +33,12 @@ func Test_InitRmq(t *testing.T) {
|
|||||||
endpoints = "localhost:2379"
|
endpoints = "localhost:2379"
|
||||||
}
|
}
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
defer etcdCli.Close()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatalf("New clientv3 error = %v", err)
|
log.Fatalf("New clientv3 error = %v", err)
|
||||||
}
|
}
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
_ = idAllocator.Initialize()
|
_ = idAllocator.Initialize()
|
||||||
|
|
||||||
|
@ -26,6 +26,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/allocator"
|
"github.com/milvus-io/milvus/internal/allocator"
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
rocksdbkv "github.com/milvus-io/milvus/internal/kv/rocksdb"
|
rocksdbkv "github.com/milvus-io/milvus/internal/kv/rocksdb"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
@ -314,7 +315,10 @@ func TestRocksmq_Seek(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_Loop(t *testing.T) {
|
func TestRocksmq_Loop(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -382,7 +386,10 @@ func TestRocksmq_Loop(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_Goroutines(t *testing.T) {
|
func TestRocksmq_Goroutines(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -454,7 +461,10 @@ func TestRocksmq_Goroutines(t *testing.T) {
|
|||||||
*/
|
*/
|
||||||
func TestRocksmq_Throughout(t *testing.T) {
|
func TestRocksmq_Throughout(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -509,7 +519,10 @@ func TestRocksmq_Throughout(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_MultiChan(t *testing.T) {
|
func TestRocksmq_MultiChan(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -558,7 +571,10 @@ func TestRocksmq_MultiChan(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_CopyData(t *testing.T) {
|
func TestRocksmq_CopyData(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -621,7 +637,10 @@ func TestRocksmq_CopyData(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_SeekToLatest(t *testing.T) {
|
func TestRocksmq_SeekToLatest(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -709,7 +728,10 @@ func TestRocksmq_SeekToLatest(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_Reader(t *testing.T) {
|
func TestRocksmq_Reader(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -769,8 +791,10 @@ func TestRocksmq_Reader(t *testing.T) {
|
|||||||
|
|
||||||
func TestReader_CornerCase(t *testing.T) {
|
func TestReader_CornerCase(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
_ = idAllocator.Initialize()
|
_ = idAllocator.Initialize()
|
||||||
@ -826,7 +850,10 @@ func TestReader_CornerCase(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_Close(t *testing.T) {
|
func TestRocksmq_Close(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
@ -863,8 +890,9 @@ func TestRocksmq_Close(t *testing.T) {
|
|||||||
|
|
||||||
func TestRocksmq_SeekWithNoConsumerError(t *testing.T) {
|
func TestRocksmq_SeekWithNoConsumerError(t *testing.T) {
|
||||||
ep := etcdEndpoints()
|
ep := etcdEndpoints()
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(ep, "/etcd/test/root")
|
etcdCli, err := etcd.GetRemoteEtcdClient(ep)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/etcd/test/root")
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||||
_ = idAllocator.Initialize()
|
_ = idAllocator.Initialize()
|
||||||
|
@ -55,10 +55,11 @@ type Session struct {
|
|||||||
// keepAliveCancel to cancel the etcd KeepAlive
|
// keepAliveCancel to cancel the etcd KeepAlive
|
||||||
keepAliveCancel context.CancelFunc
|
keepAliveCancel context.CancelFunc
|
||||||
|
|
||||||
ServerID int64 `json:"ServerID,omitempty"`
|
ServerID int64 `json:"ServerID,omitempty"`
|
||||||
ServerName string `json:"ServerName,omitempty"`
|
ServerName string `json:"ServerName,omitempty"`
|
||||||
Address string `json:"Address,omitempty"`
|
Address string `json:"Address,omitempty"`
|
||||||
Exclusive bool `json:"Exclusive,omitempty"`
|
Exclusive bool `json:"Exclusive,omitempty"`
|
||||||
|
TriggerKill bool
|
||||||
|
|
||||||
liveCh <-chan bool
|
liveCh <-chan bool
|
||||||
etcdCli *clientv3.Client
|
etcdCli *clientv3.Client
|
||||||
@ -73,7 +74,7 @@ type Session struct {
|
|||||||
// ServerID, ServerName, Address, Exclusive will be assigned after Init().
|
// ServerID, ServerName, Address, Exclusive will be assigned after Init().
|
||||||
// metaRoot is a path in etcd to save session information.
|
// metaRoot is a path in etcd to save session information.
|
||||||
// etcdEndpoints is to init etcdCli when NewSession
|
// etcdEndpoints is to init etcdCli when NewSession
|
||||||
func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *Session {
|
func NewSession(ctx context.Context, metaRoot string, client *clientv3.Client) *Session {
|
||||||
session := &Session{
|
session := &Session{
|
||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
metaRoot: metaRoot,
|
metaRoot: metaRoot,
|
||||||
@ -83,16 +84,12 @@ func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *S
|
|||||||
|
|
||||||
connectEtcdFn := func() error {
|
connectEtcdFn := func() error {
|
||||||
log.Debug("Session try to connect to etcd")
|
log.Debug("Session try to connect to etcd")
|
||||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: etcdEndpoints, DialTimeout: 5 * time.Second})
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
ctx2, cancel2 := context.WithTimeout(session.ctx, 5*time.Second)
|
ctx2, cancel2 := context.WithTimeout(session.ctx, 5*time.Second)
|
||||||
defer cancel2()
|
defer cancel2()
|
||||||
if _, err = etcdCli.Get(ctx2, "health"); err != nil {
|
if _, err := client.Get(ctx2, "health"); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
session.etcdCli = etcdCli
|
session.etcdCli = client
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
err := retry.Do(ctx, connectEtcdFn, retry.Attempts(300))
|
err := retry.Do(ctx, connectEtcdFn, retry.Attempts(300))
|
||||||
@ -107,10 +104,11 @@ func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *S
|
|||||||
|
|
||||||
// Init will initialize base struct of the Session, including ServerName, ServerID,
|
// Init will initialize base struct of the Session, including ServerName, ServerID,
|
||||||
// Address, Exclusive. ServerID is obtained in getServerID.
|
// Address, Exclusive. ServerID is obtained in getServerID.
|
||||||
func (s *Session) Init(serverName, address string, exclusive bool) {
|
func (s *Session) Init(serverName, address string, exclusive bool, triggerKill bool) {
|
||||||
s.ServerName = serverName
|
s.ServerName = serverName
|
||||||
s.Address = address
|
s.Address = address
|
||||||
s.Exclusive = exclusive
|
s.Exclusive = exclusive
|
||||||
|
s.TriggerKill = triggerKill
|
||||||
s.checkIDExist()
|
s.checkIDExist()
|
||||||
serverID, err := s.getServerID()
|
serverID, err := s.getServerID()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -12,6 +12,7 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
@ -34,8 +35,10 @@ func TestGetServerIDConcurrently(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot)
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
assert.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot)
|
||||||
err = etcdKV.RemoveWithPrefix("")
|
err = etcdKV.RemoveWithPrefix("")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
@ -45,7 +48,7 @@ func TestGetServerIDConcurrently(t *testing.T) {
|
|||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
var muList = sync.Mutex{}
|
var muList = sync.Mutex{}
|
||||||
|
|
||||||
s := NewSession(ctx, metaRoot, etcdEndpoints)
|
s := NewSession(ctx, metaRoot, etcdCli)
|
||||||
res := make([]int64, 0)
|
res := make([]int64, 0)
|
||||||
|
|
||||||
getIDFunc := func() {
|
getIDFunc := func() {
|
||||||
@ -79,16 +82,17 @@ func TestInit(t *testing.T) {
|
|||||||
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
||||||
|
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot)
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
assert.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot)
|
||||||
err = etcdKV.RemoveWithPrefix("")
|
err = etcdKV.RemoveWithPrefix("")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
|
|
||||||
s := NewSession(ctx, metaRoot, etcdEndpoints)
|
s := NewSession(ctx, metaRoot, etcdCli)
|
||||||
s.Init("inittest", "testAddr", false)
|
s.Init("inittest", "testAddr", false, false)
|
||||||
assert.NotEqual(t, int64(0), s.leaseID)
|
assert.NotEqual(t, int64(0), s.leaseID)
|
||||||
assert.NotEqual(t, int64(0), s.ServerID)
|
assert.NotEqual(t, int64(0), s.ServerID)
|
||||||
s.Register()
|
s.Register()
|
||||||
@ -108,8 +112,10 @@ func TestUpdateSessions(t *testing.T) {
|
|||||||
|
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "")
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
assert.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "")
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
@ -117,7 +123,7 @@ func TestUpdateSessions(t *testing.T) {
|
|||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
var muList = sync.Mutex{}
|
var muList = sync.Mutex{}
|
||||||
|
|
||||||
s := NewSession(ctx, metaRoot, etcdEndpoints)
|
s := NewSession(ctx, metaRoot, etcdCli)
|
||||||
|
|
||||||
sessions, rev, err := s.GetSessions("test")
|
sessions, rev, err := s.GetSessions("test")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
@ -127,8 +133,10 @@ func TestUpdateSessions(t *testing.T) {
|
|||||||
sList := []*Session{}
|
sList := []*Session{}
|
||||||
|
|
||||||
getIDFunc := func() {
|
getIDFunc := func() {
|
||||||
singleS := NewSession(ctx, metaRoot, etcdEndpoints)
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
singleS.Init("test", "testAddr", false)
|
require.NoError(t, err)
|
||||||
|
singleS := NewSession(ctx, metaRoot, etcdCli)
|
||||||
|
singleS.Init("test", "testAddr", false, false)
|
||||||
singleS.Register()
|
singleS.Register()
|
||||||
muList.Lock()
|
muList.Lock()
|
||||||
sList = append(sList, singleS)
|
sList = append(sList, singleS)
|
||||||
@ -220,12 +228,15 @@ func TestWatcherHandleWatchResp(t *testing.T) {
|
|||||||
|
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, "/by-dev/session-ut")
|
|
||||||
require.NoError(t, err)
|
|
||||||
|
|
||||||
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
|
require.NoError(t, err)
|
||||||
|
defer etcdCli.Close()
|
||||||
|
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, "/by-dev/session-ut")
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("/by-dev/session-ut")
|
defer etcdKV.RemoveWithPrefix("/by-dev/session-ut")
|
||||||
s := NewSession(ctx, metaRoot, etcdEndpoints)
|
s := NewSession(ctx, metaRoot, etcdCli)
|
||||||
defer s.Revoke(time.Second)
|
defer s.Revoke(time.Second)
|
||||||
|
|
||||||
getWatcher := func(s *Session, rewatch Rewatch) *sessionWatcher {
|
getWatcher := func(s *Session, rewatch Rewatch) *sessionWatcher {
|
||||||
@ -317,7 +328,7 @@ func TestWatcherHandleWatchResp(t *testing.T) {
|
|||||||
})
|
})
|
||||||
|
|
||||||
t.Run("err handled but list failed", func(t *testing.T) {
|
t.Run("err handled but list failed", func(t *testing.T) {
|
||||||
s := NewSession(ctx, "/by-dev/session-ut", etcdEndpoints)
|
s := NewSession(ctx, "/by-dev/session-ut", etcdCli)
|
||||||
s.etcdCli.Close()
|
s.etcdCli.Close()
|
||||||
w := getWatcher(s, func(sessions map[string]*Session) error {
|
w := getWatcher(s, func(sessions map[string]*Session) error {
|
||||||
return nil
|
return nil
|
||||||
@ -364,16 +375,18 @@ func TestSessionRevoke(t *testing.T) {
|
|||||||
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
metaRoot := fmt.Sprintf("%d/%s", rand.Int(), DefaultServiceRoot)
|
||||||
|
|
||||||
etcdEndpoints := strings.Split(endpoints, ",")
|
etcdEndpoints := strings.Split(endpoints, ",")
|
||||||
etcdKV, err := etcdkv.NewEtcdKV(etcdEndpoints, metaRoot)
|
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||||
assert.NoError(t, err)
|
defer etcdCli.Close()
|
||||||
|
require.NoError(t, err)
|
||||||
|
etcdKV := etcdkv.NewEtcdKV(etcdCli, metaRoot)
|
||||||
err = etcdKV.RemoveWithPrefix("")
|
err = etcdKV.RemoveWithPrefix("")
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
defer etcdKV.Close()
|
defer etcdKV.Close()
|
||||||
defer etcdKV.RemoveWithPrefix("")
|
defer etcdKV.RemoveWithPrefix("")
|
||||||
|
|
||||||
s = NewSession(ctx, metaRoot, etcdEndpoints)
|
s = NewSession(ctx, metaRoot, etcdCli)
|
||||||
s.Init("revoketest", "testAddr", false)
|
s.Init("revoketest", "testAddr", false, false)
|
||||||
assert.NotPanics(t, func() {
|
assert.NotPanics(t, func() {
|
||||||
s.Revoke(time.Second)
|
s.Revoke(time.Second)
|
||||||
})
|
})
|
||||||
|
@ -17,6 +17,7 @@ import (
|
|||||||
|
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
@ -70,6 +71,6 @@ func AddPhysicalTimeOnTs(timeInMs int64, ts uint64) uint64 {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewTSOKVBase returns a etcdkv.EtcdKV object
|
// NewTSOKVBase returns a etcdkv.EtcdKV object
|
||||||
func NewTSOKVBase(etcdEndpoints []string, tsoRoot, subPath string) (*etcdkv.EtcdKV, error) {
|
func NewTSOKVBase(client *clientv3.Client, tsoRoot, subPath string) *etcdkv.EtcdKV {
|
||||||
return etcdkv.NewEtcdKV(etcdEndpoints, path.Join(tsoRoot, subPath))
|
return etcdkv.NewEtcdKV(client, path.Join(tsoRoot, subPath))
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user