mirror of https://github.com/milvus-io/milvus.git
Add dependency factory (#16204)
Signed-off-by: godchen0212 <qingxiang.chen@zilliz.com>pull/16422/head
parent
062e364308
commit
bb7a0766fe
|
@ -21,9 +21,9 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
)
|
||||
|
||||
// DataCoord implements grpc server of DataCoord server
|
||||
|
@ -33,7 +33,7 @@ type DataCoord struct {
|
|||
}
|
||||
|
||||
// NewDataCoord creates a new DataCoord
|
||||
func NewDataCoord(ctx context.Context, factory msgstream.Factory) (*DataCoord, error) {
|
||||
func NewDataCoord(ctx context.Context, factory dependency.Factory) (*DataCoord, error) {
|
||||
s := grpcdatacoordclient.NewServer(ctx, factory)
|
||||
|
||||
return &DataCoord{
|
||||
|
|
|
@ -20,10 +20,10 @@ import (
|
|||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
)
|
||||
|
||||
// DataNode implements DataNode grpc server
|
||||
|
@ -33,7 +33,7 @@ type DataNode struct {
|
|||
}
|
||||
|
||||
// NewDataNode creates a new DataNode
|
||||
func NewDataNode(ctx context.Context, factory msgstream.Factory) (*DataNode, error) {
|
||||
func NewDataNode(ctx context.Context, factory dependency.Factory) (*DataNode, error) {
|
||||
svr, err := grpcdatanode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
|
|
@ -19,10 +19,10 @@ package components
|
|||
import (
|
||||
"context"
|
||||
|
||||
grpcindexcoord "github.com/milvus-io/milvus/internal/distributed/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
|
||||
grpcindexcoord "github.com/milvus-io/milvus/internal/distributed/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
)
|
||||
|
||||
// IndexCoord implements IndexCoord grpc server
|
||||
|
@ -31,10 +31,10 @@ type IndexCoord struct {
|
|||
}
|
||||
|
||||
// NewIndexCoord creates a new IndexCoord
|
||||
func NewIndexCoord(ctx context.Context) (*IndexCoord, error) {
|
||||
func NewIndexCoord(ctx context.Context, factory dependency.Factory) (*IndexCoord, error) {
|
||||
var err error
|
||||
s := &IndexCoord{}
|
||||
svr, err := grpcindexcoord.NewServer(ctx)
|
||||
svr, err := grpcindexcoord.NewServer(ctx, factory)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
|
|
@ -21,6 +21,7 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
grpcindexnode "github.com/milvus-io/milvus/internal/distributed/indexnode"
|
||||
)
|
||||
|
@ -31,10 +32,10 @@ type IndexNode struct {
|
|||
}
|
||||
|
||||
// NewIndexNode creates a new IndexNode
|
||||
func NewIndexNode(ctx context.Context) (*IndexNode, error) {
|
||||
func NewIndexNode(ctx context.Context, factory dependency.Factory) (*IndexNode, error) {
|
||||
var err error
|
||||
n := &IndexNode{}
|
||||
svr, err := grpcindexnode.NewServer(ctx)
|
||||
svr, err := grpcindexnode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -21,9 +21,9 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
grpcproxy "github.com/milvus-io/milvus/internal/distributed/proxy"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
)
|
||||
|
||||
// Proxy implements Proxy grpc server
|
||||
|
@ -32,7 +32,7 @@ type Proxy struct {
|
|||
}
|
||||
|
||||
// NewProxy creates a new Proxy
|
||||
func NewProxy(ctx context.Context, factory msgstream.Factory) (*Proxy, error) {
|
||||
func NewProxy(ctx context.Context, factory dependency.Factory) (*Proxy, error) {
|
||||
var err error
|
||||
n := &Proxy{}
|
||||
|
||||
|
|
|
@ -21,9 +21,9 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
grpcquerycoord "github.com/milvus-io/milvus/internal/distributed/querycoord"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
)
|
||||
|
||||
// QueryCoord implements QueryCoord grpc server
|
||||
|
@ -33,7 +33,7 @@ type QueryCoord struct {
|
|||
}
|
||||
|
||||
// NewQueryCoord creates a new QueryCoord
|
||||
func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord, error) {
|
||||
func NewQueryCoord(ctx context.Context, factory dependency.Factory) (*QueryCoord, error) {
|
||||
svr, err := grpcquerycoord.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -21,9 +21,9 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
grpcquerynode "github.com/milvus-io/milvus/internal/distributed/querynode"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
)
|
||||
|
||||
// QueryNode implements QueryNode grpc server
|
||||
|
@ -33,7 +33,7 @@ type QueryNode struct {
|
|||
}
|
||||
|
||||
// NewQueryNode creates a new QueryNode
|
||||
func NewQueryNode(ctx context.Context, factory msgstream.Factory) (*QueryNode, error) {
|
||||
func NewQueryNode(ctx context.Context, factory dependency.Factory) (*QueryNode, error) {
|
||||
svr, err := grpcquerynode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
|
|
@ -22,10 +22,11 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
|
||||
rc "github.com/milvus-io/milvus/internal/distributed/rootcoord"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
)
|
||||
|
||||
// RootCoord implements RoodCoord grpc server
|
||||
|
@ -38,7 +39,7 @@ type RootCoord struct {
|
|||
}
|
||||
|
||||
// NewRootCoord creates a new RoorCoord
|
||||
func NewRootCoord(ctx context.Context, factory msgstream.Factory) (*RootCoord, error) {
|
||||
func NewRootCoord(ctx context.Context, factory dependency.Factory) (*RootCoord, error) {
|
||||
svr, err := rc.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
|
|
@ -28,9 +28,12 @@ import (
|
|||
"time"
|
||||
|
||||
rocksmqimpl "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
|
||||
"github.com/milvus-io/milvus/cmd/components"
|
||||
"github.com/milvus-io/milvus/internal/datacoord"
|
||||
"github.com/milvus-io/milvus/internal/datanode"
|
||||
|
@ -38,7 +41,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proxy"
|
||||
"github.com/milvus-io/milvus/internal/querycoord"
|
||||
|
@ -46,12 +48,11 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/healthz"
|
||||
logutil "github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
|
||||
var Params paramtable.ComponentParam
|
||||
|
@ -65,21 +66,6 @@ func init() {
|
|||
Registry.MustRegister(prometheus.NewGoCollector())
|
||||
}
|
||||
|
||||
func newMsgFactory(localMsg bool) msgstream.Factory {
|
||||
if localMsg {
|
||||
if Params.RocksmqEnable() {
|
||||
return msgstream.NewRmsFactory()
|
||||
}
|
||||
return msgstream.NewPmsFactory()
|
||||
}
|
||||
return msgstream.NewPmsFactory()
|
||||
}
|
||||
|
||||
func initRocksmq() error {
|
||||
err := rocksmqimpl.InitRocksMQ()
|
||||
return err
|
||||
}
|
||||
|
||||
func stopRocksmq() {
|
||||
rocksmqimpl.CloseRocksMQ()
|
||||
}
|
||||
|
@ -115,8 +101,7 @@ func (mr *MilvusRoles) runRootCoord(ctx context.Context, localMsg bool) *compone
|
|||
} else {
|
||||
rootcoord.Params.SetLogConfig(typeutil.RootCoordRole)
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
rc, err = components.NewRootCoord(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -148,7 +133,7 @@ func (mr *MilvusRoles) runProxy(ctx context.Context, localMsg bool, alias string
|
|||
proxy.Params.SetLogConfig(typeutil.ProxyRole)
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
pn, err = components.NewProxy(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -179,7 +164,7 @@ func (mr *MilvusRoles) runQueryCoord(ctx context.Context, localMsg bool) *compon
|
|||
querycoord.Params.SetLogConfig(typeutil.QueryCoordRole)
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
qs, err = components.NewQueryCoord(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -211,7 +196,7 @@ func (mr *MilvusRoles) runQueryNode(ctx context.Context, localMsg bool, alias st
|
|||
querynode.Params.SetLogConfig(typeutil.QueryNodeRole)
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
qn, err = components.NewQueryNode(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -242,7 +227,7 @@ func (mr *MilvusRoles) runDataCoord(ctx context.Context, localMsg bool) *compone
|
|||
datacoord.Params.SetLogConfig(typeutil.DataCoordRole)
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
|
||||
dctx := logutil.WithModule(ctx, "DataCoord")
|
||||
var err error
|
||||
|
@ -276,7 +261,7 @@ func (mr *MilvusRoles) runDataNode(ctx context.Context, localMsg bool, alias str
|
|||
datanode.Params.SetLogConfig(typeutil.DataNodeRole)
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
dn, err = components.NewDataNode(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -307,8 +292,10 @@ func (mr *MilvusRoles) runIndexCoord(ctx context.Context, localMsg bool) *compon
|
|||
indexcoord.Params.SetLogConfig(typeutil.IndexCoordRole)
|
||||
}
|
||||
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
|
||||
var err error
|
||||
is, err = components.NewIndexCoord(ctx)
|
||||
is, err = components.NewIndexCoord(ctx, factory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -338,8 +325,10 @@ func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, alias st
|
|||
indexnode.Params.SetLogConfig(typeutil.IndexNodeRole)
|
||||
}
|
||||
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
|
||||
var err error
|
||||
in, err = components.NewIndexNode(ctx)
|
||||
in, err = components.NewIndexNode(ctx, factory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -366,7 +355,9 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
|
|||
}
|
||||
Params.Init()
|
||||
|
||||
if err := initRocksmq(); err != nil {
|
||||
path, _ := Params.Load("_RocksmqPath")
|
||||
err := rocksmqimpl.InitRocksMQ(path)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
defer stopRocksmq()
|
||||
|
|
|
@ -32,6 +32,9 @@ etcd:
|
|||
use:
|
||||
embed: false # Whether to enable embedded Etcd (an in-process EtcdServer).
|
||||
|
||||
localStorage:
|
||||
path: /var/lib/milvus/data/
|
||||
|
||||
# Related configuration of minio, which is responsible for data persistence for Milvus.
|
||||
minio:
|
||||
address: localhost # Address of MinIO/S3
|
||||
|
@ -159,8 +162,9 @@ queryNode:
|
|||
segcore:
|
||||
chunkRows: 32768 # The number of vectors in a chunk.
|
||||
|
||||
chunkManager:
|
||||
localFileCacheLimit: 2147483648 # 2 GB, 2 * 1024 *1024 *1024
|
||||
cache:
|
||||
enabled: true
|
||||
memoryLimit: 2147483648 # 2 GB, 2 * 1024 *1024 *1024
|
||||
|
||||
|
||||
indexCoord:
|
||||
|
@ -226,11 +230,6 @@ dataNode:
|
|||
# Max buffer size to flush for a single segment.
|
||||
insertBufSize: 16777216 # Bytes, 16 MB
|
||||
|
||||
# Configure whether to store the vector and the local path when querying/searching in Querynode.
|
||||
localStorage:
|
||||
path: /var/lib/milvus/data/
|
||||
enabled: true
|
||||
|
||||
# Configures the system log output.
|
||||
log:
|
||||
level: info # Only supports debug, info, warn, error, panic, or fatal. Default 'info'.
|
||||
|
@ -275,3 +274,7 @@ common:
|
|||
# Valid values: [auto, avx512, avx2, avx, sse4_2]
|
||||
# This configuration is only used by querynode and indexnode, it selects CPU instruction set for Searching and Index-building.
|
||||
simdType: auto
|
||||
|
||||
storage:
|
||||
vector: minio
|
||||
|
||||
|
|
|
@ -26,9 +26,11 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"stathat.com/c/consistent"
|
||||
|
@ -227,7 +229,7 @@ func TestChannelManager_StateTransfer(t *testing.T) {
|
|||
|
||||
metakv.RemoveWithPrefix("")
|
||||
ctx, cancel := context.WithCancel(context.TODO())
|
||||
factory := msgstream.NewRmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
_, err := factory.NewMsgStream(context.TODO())
|
||||
require.NoError(t, err)
|
||||
chManager, err := NewChannelManager(metakv, newMockHandler(), withMsgstreamFactory(factory))
|
||||
|
@ -445,7 +447,7 @@ func TestChannelManager(t *testing.T) {
|
|||
{false, UniqueID(9), "invalid-chan"},
|
||||
}
|
||||
|
||||
factory := msgstream.NewRmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
_, err := factory.NewMsgStream(context.TODO())
|
||||
require.NoError(t, err)
|
||||
chManager, err := NewChannelManager(metakv, newMockHandler(), withMsgstreamFactory(factory))
|
||||
|
|
|
@ -27,6 +27,11 @@ import (
|
|||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio-go/v7"
|
||||
"github.com/minio/minio-go/v7/pkg/credentials"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
datanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client"
|
||||
rootcoordclient "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
|
@ -38,6 +43,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
@ -46,10 +52,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/timerecord"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/minio/minio-go/v7"
|
||||
"github.com/minio/minio-go/v7/pkg/credentials"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -122,8 +124,8 @@ type Server struct {
|
|||
|
||||
metricsCacheManager *metricsinfo.MetricsCacheManager
|
||||
|
||||
flushCh chan UniqueID
|
||||
msFactory msgstream.Factory
|
||||
flushCh chan UniqueID
|
||||
factory dependency.Factory
|
||||
|
||||
session *sessionutil.Session
|
||||
eventCh <-chan *sessionutil.SessionEvent
|
||||
|
@ -182,12 +184,12 @@ func SetSegmentManager(manager Manager) Option {
|
|||
}
|
||||
|
||||
// CreateServer creates a `Server` instance
|
||||
func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option) *Server {
|
||||
func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Option) *Server {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
s := &Server{
|
||||
ctx: ctx,
|
||||
quitCh: make(chan struct{}),
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
flushCh: make(chan UniqueID, 1024),
|
||||
dataNodeCreator: defaultDataNodeCreatorFunc,
|
||||
rootCoordClientCreator: defaultRootCoordCreatorFunc,
|
||||
|
@ -259,10 +261,7 @@ func (s *Server) Init() error {
|
|||
// 4. set server state to Healthy
|
||||
func (s *Server) Start() error {
|
||||
var err error
|
||||
err = s.msFactory.Init(&Params)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
s.factory.Init(&Params)
|
||||
if err = s.initRootCoordClient(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -307,7 +306,7 @@ func (s *Server) initCluster() error {
|
|||
}
|
||||
|
||||
var err error
|
||||
s.channelManager, err = NewChannelManager(s.kvClient, s.handler, withMsgstreamFactory(s.msFactory), withStateChecker())
|
||||
s.channelManager, err = NewChannelManager(s.kvClient, s.handler, withMsgstreamFactory(s.factory), withStateChecker())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -443,7 +442,7 @@ func (s *Server) startServerLoop() {
|
|||
// startDataNodeTtLoop start a goroutine to recv data node tt msg from msgstream
|
||||
// tt msg stands for the currently consumed timestamp for each channel
|
||||
func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
||||
ttMsgStream, err := s.msFactory.NewMsgStream(ctx)
|
||||
ttMsgStream, err := s.factory.NewMsgStream(ctx)
|
||||
if err != nil {
|
||||
log.Error("DataCoord failed to create timetick channel", zap.Error(err))
|
||||
return
|
||||
|
|
|
@ -30,6 +30,12 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
|
@ -39,14 +45,10 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
|
@ -617,7 +619,7 @@ func TestService_WatchServices(t *testing.T) {
|
|||
sc := make(chan os.Signal, 1)
|
||||
signal.Notify(sc, syscall.SIGINT)
|
||||
defer signal.Reset(syscall.SIGINT)
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
svr := CreateServer(context.TODO(), factory)
|
||||
svr.session = &sessionutil.Session{
|
||||
TriggerKill: true,
|
||||
|
@ -1078,7 +1080,7 @@ func TestDataNodeTtChannel(t *testing.T) {
|
|||
Partitions: []int64{0},
|
||||
})
|
||||
|
||||
ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO())
|
||||
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
|
||||
ttMsgStream.Start()
|
||||
|
@ -1146,7 +1148,7 @@ func TestDataNodeTtChannel(t *testing.T) {
|
|||
Schema: newTestSchema(),
|
||||
Partitions: []int64{0},
|
||||
})
|
||||
ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO())
|
||||
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
|
||||
ttMsgStream.Start()
|
||||
|
@ -1228,7 +1230,7 @@ func TestDataNodeTtChannel(t *testing.T) {
|
|||
Partitions: []int64{0},
|
||||
})
|
||||
|
||||
ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO())
|
||||
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
|
||||
ttMsgStream.Start()
|
||||
|
@ -2013,7 +2015,7 @@ func TestOptions(t *testing.T) {
|
|||
})
|
||||
assert.NotNil(t, opt)
|
||||
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
svr := CreateServer(context.TODO(), factory, opt)
|
||||
dn, err := svr.dataNodeCreator(context.Background(), "")
|
||||
|
@ -2491,9 +2493,7 @@ func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Se
|
|||
var err error
|
||||
Params.Init()
|
||||
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int())
|
||||
factory := msgstream.NewPmsFactory()
|
||||
err = factory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
|
|
|
@ -35,12 +35,15 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
|
@ -48,15 +51,13 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -117,11 +118,11 @@ type DataNode struct {
|
|||
|
||||
closer io.Closer
|
||||
|
||||
msFactory msgstream.Factory
|
||||
factory dependency.Factory
|
||||
}
|
||||
|
||||
// NewDataNode will return a DataNode with abnormal state.
|
||||
func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
|
||||
func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx2, cancel2 := context.WithCancel(ctx)
|
||||
node := &DataNode{
|
||||
|
@ -131,7 +132,7 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
|
|||
|
||||
rootCoord: nil,
|
||||
dataCoord: nil,
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
segmentCache: newCache(),
|
||||
compactionExecutor: newCompactionExecutor(),
|
||||
|
||||
|
@ -217,11 +218,7 @@ func (node *DataNode) Init() error {
|
|||
return err
|
||||
}
|
||||
|
||||
if err := node.msFactory.Init(&Params); err != nil {
|
||||
log.Warn("DataNode Init msFactory SetParams failed, use default",
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
node.factory.Init(&Params)
|
||||
log.Info("DataNode Init successfully",
|
||||
zap.String("MsgChannelSubName", Params.CommonCfg.DataNodeSubName))
|
||||
|
||||
|
@ -464,13 +461,7 @@ func (node *DataNode) Start() error {
|
|||
return errors.New("DataNode fail to connect etcd")
|
||||
}
|
||||
|
||||
chunkManager, err := storage.NewMinioChunkManager(node.ctx,
|
||||
storage.Address(Params.MinioCfg.Address),
|
||||
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(Params.MinioCfg.UseSSL),
|
||||
storage.BucketName(Params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
chunkManager, err := node.factory.NewVectorStorageChunkManager(node.ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -33,6 +33,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
|
@ -209,10 +210,8 @@ func TestDataNode(t *testing.T) {
|
|||
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
|
||||
|
||||
// pulsar produce
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err = msFactory.Init(&Params)
|
||||
assert.NoError(t, err)
|
||||
insertStream, err := msFactory.NewMsgStream(node1.ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
insertStream, err := factory.NewMsgStream(node1.ctx)
|
||||
assert.NoError(t, err)
|
||||
insertStream.AsProducer([]string{dmChannelName})
|
||||
insertStream.Start()
|
||||
|
@ -540,12 +539,10 @@ func TestWatchChannel(t *testing.T) {
|
|||
bs, err = proto.Marshal(&info)
|
||||
assert.NoError(t, err)
|
||||
|
||||
msFactory := node.msFactory
|
||||
defer func() { node.msFactory = msFactory }()
|
||||
msFactory := node.factory
|
||||
defer func() { node.factory = msFactory }()
|
||||
|
||||
node.msFactory = &FailMessageStreamFactory{
|
||||
node.msFactory,
|
||||
}
|
||||
node.factory = &FailMessageStreamFactory{}
|
||||
node.handleWatchInfo(e, ch, bs)
|
||||
<-chPut
|
||||
exist = node.flowgraphManager.exist(ch)
|
||||
|
|
|
@ -21,6 +21,8 @@ import (
|
|||
"errors"
|
||||
"fmt"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
|
@ -28,7 +30,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// dataSyncService controls a flowgraph for a specific collection
|
||||
|
|
|
@ -35,6 +35,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
)
|
||||
|
||||
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
|
||||
|
@ -197,9 +198,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
|
||||
allocFactory := NewAllocatorFactory(1)
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
insertChannelName := "data_sync_service_test_dml"
|
||||
ddlChannelName := "data_sync_service_test_ddl"
|
||||
|
@ -229,7 +228,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
}
|
||||
|
||||
signalCh := make(chan string, 100)
|
||||
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataCoordFactory{}, newCache(), cm, newCompactionExecutor())
|
||||
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, factory, vchan, signalCh, &DataCoordFactory{}, newCache(), cm, newCompactionExecutor())
|
||||
|
||||
assert.Nil(t, err)
|
||||
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||
|
@ -276,10 +275,10 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
|
||||
// pulsar produce
|
||||
assert.NoError(t, err)
|
||||
insertStream, _ := msFactory.NewMsgStream(ctx)
|
||||
insertStream, _ := factory.NewMsgStream(ctx)
|
||||
insertStream.AsProducer([]string{insertChannelName})
|
||||
|
||||
ddStream, _ := msFactory.NewMsgStream(ctx)
|
||||
ddStream, _ := factory.NewMsgStream(ctx)
|
||||
ddStream.AsProducer([]string{ddlChannelName})
|
||||
|
||||
var insertMsgStream msgstream.MsgStream = insertStream
|
||||
|
|
|
@ -28,6 +28,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
)
|
||||
|
||||
|
@ -144,7 +145,7 @@ func TestFlowGraph_DDNode_Operate(to *testing.T) {
|
|||
|
||||
for _, test := range tests {
|
||||
te.Run(test.description, func(t *testing.T) {
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
deltaStream, err := factory.NewMsgStream(context.Background())
|
||||
assert.Nil(t, err)
|
||||
ddn := ddNode{
|
||||
|
@ -204,7 +205,7 @@ func TestFlowGraph_DDNode_Operate(to *testing.T) {
|
|||
for _, test := range tests {
|
||||
te.Run(test.description, func(t *testing.T) {
|
||||
fs := &datapb.SegmentInfo{ID: test.ddnFlushedSegment}
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
deltaStream, err := factory.NewMsgStream(context.Background())
|
||||
assert.Nil(t, err)
|
||||
// Prepare ddNode states
|
||||
|
@ -250,7 +251,7 @@ func TestFlowGraph_DDNode_Operate(to *testing.T) {
|
|||
|
||||
for _, test := range tests {
|
||||
te.Run(test.description, func(t *testing.T) {
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
deltaStream, err := factory.NewMsgStream(context.Background())
|
||||
assert.Nil(t, err)
|
||||
// Prepare ddNode states
|
||||
|
@ -312,7 +313,7 @@ func TestFlowGraph_DDNode_filterMessages(te *testing.T) {
|
|||
s := &datapb.SegmentInfo{ID: id}
|
||||
fs = append(fs, s)
|
||||
}
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
deltaStream, err := factory.NewMsgStream(context.Background())
|
||||
assert.Nil(t, err)
|
||||
// Prepare ddNode states
|
||||
|
@ -376,7 +377,7 @@ func TestFlowGraph_DDNode_isFlushed(te *testing.T) {
|
|||
s := &datapb.SegmentInfo{ID: id}
|
||||
fs = append(fs, s)
|
||||
}
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
deltaStream, err := factory.NewMsgStream(context.Background())
|
||||
assert.Nil(t, err)
|
||||
ddn := &ddNode{flushedSegments: fs, deltaMsgStream: deltaStream}
|
||||
|
|
|
@ -23,10 +23,12 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
|
@ -79,9 +81,7 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
|
|||
err = replica.addNewSegment(1, collMeta.ID, 0, insertChannelName, &internalpb.MsgPosition{}, &internalpb.MsgPosition{})
|
||||
require.NoError(t, err)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
fm := NewRendezvousFlushManager(&allocator{}, cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc)
|
||||
|
||||
|
@ -89,7 +89,7 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
|
|||
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
msFactory: msFactory,
|
||||
msFactory: factory,
|
||||
allocator: NewAllocatorFactory(),
|
||||
vChannelName: "string",
|
||||
}
|
||||
|
@ -104,7 +104,7 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
|
|||
assert.Error(t, err)*/
|
||||
|
||||
c.msFactory = &CDFMsFactory{
|
||||
Factory: msFactory,
|
||||
Factory: factory,
|
||||
cd: 0,
|
||||
}
|
||||
|
||||
|
@ -167,16 +167,14 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
err = replica.addNewSegment(1, collMeta.ID, 0, insertChannelName, &internalpb.MsgPosition{}, &internalpb.MsgPosition{})
|
||||
require.NoError(t, err)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
fm := NewRendezvousFlushManager(NewAllocatorFactory(), cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc)
|
||||
|
||||
flushChan := make(chan flushMsg, 100)
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
msFactory: msFactory,
|
||||
msFactory: factory,
|
||||
allocator: NewAllocatorFactory(),
|
||||
vChannelName: "string",
|
||||
}
|
||||
|
@ -247,12 +245,12 @@ func TestFlushSegment(t *testing.T) {
|
|||
}
|
||||
flushMap.Store(segmentID, insertData)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
factory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarBufSize": 1024}
|
||||
err = msFactory.SetParams(m)
|
||||
err = factory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
flushChan := make(chan flushMsg, 100)
|
||||
|
||||
|
@ -264,7 +262,7 @@ func TestFlushSegment(t *testing.T) {
|
|||
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
msFactory: msFactory,
|
||||
factory: factory,
|
||||
allocator: NewAllocatorFactory(),
|
||||
vChannelName: "string",
|
||||
}
|
||||
|
@ -369,9 +367,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
|||
|
||||
colRep.metaService = newMetaService(mockRootCoord, collMeta.ID)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
flushPacks := []*segmentFlushPack{}
|
||||
fpMut := sync.Mutex{}
|
||||
|
@ -394,7 +390,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
|||
flushChan := make(chan flushMsg, 100)
|
||||
c := &nodeConfig{
|
||||
replica: colRep,
|
||||
msFactory: msFactory,
|
||||
msFactory: factory,
|
||||
allocator: NewAllocatorFactory(),
|
||||
vChannelName: "string",
|
||||
}
|
||||
|
@ -653,17 +649,14 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
|
|||
err = replica.addNewSegment(1, collMeta.ID, 0, insertChannelName, &internalpb.MsgPosition{}, &internalpb.MsgPosition{Timestamp: 101})
|
||||
require.NoError(t, err)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
fm := NewRendezvousFlushManager(&allocator{}, cm, replica, func(*segmentFlushPack) {}, emptyFlushAndDropFunc)
|
||||
|
||||
flushChan := make(chan flushMsg, 100)
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
msFactory: msFactory,
|
||||
msFactory: factory,
|
||||
allocator: NewAllocatorFactory(),
|
||||
vChannelName: "string",
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ func (fm *flowgraphManager) addAndStart(dn *DataNode, vchan *datapb.VchannelInfo
|
|||
|
||||
var alloc allocatorInterface = newAllocator(dn.rootCoord)
|
||||
|
||||
dataSyncService, err := newDataSyncService(dn.ctx, make(chan flushMsg, 100), replica, alloc, dn.msFactory, vchan, dn.clearSignal, dn.dataCoord, dn.segmentCache, dn.chunkManager, dn.compactionExecutor)
|
||||
dataSyncService, err := newDataSyncService(dn.ctx, make(chan flushMsg, 100), replica, alloc, dn.factory, vchan, dn.clearSignal, dn.dataCoord, dn.segmentCache, dn.chunkManager, dn.compactionExecutor)
|
||||
if err != nil {
|
||||
log.Warn("new data sync service fail", zap.String("vChannelName", vchan.GetChannelName()), zap.Error(err))
|
||||
return err
|
||||
|
|
|
@ -33,6 +33,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
s "github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
||||
|
@ -53,8 +54,8 @@ const debug = false
|
|||
var emptyFlushAndDropFunc flushAndDropFunc = func(_ []*segmentFlushPack) {}
|
||||
|
||||
func newIDLEDataNodeMock(ctx context.Context, pkType schemapb.DataType) *DataNode {
|
||||
msFactory := msgstream.NewRmsFactory()
|
||||
node := NewDataNode(ctx, msFactory)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
node := NewDataNode(ctx, factory)
|
||||
|
||||
rc := &RootCoordFactory{
|
||||
ID: 0,
|
||||
|
@ -85,8 +86,8 @@ func newHEALTHDataNodeMock(dmChannelName string) *DataNode {
|
|||
}()
|
||||
}
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
node := NewDataNode(ctx, msFactory)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
node := NewDataNode(ctx, factory)
|
||||
|
||||
ms := &RootCoordFactory{
|
||||
ID: 0,
|
||||
|
@ -912,7 +913,7 @@ func (m *RootCoordFactory) GetComponentStates(ctx context.Context) (*internalpb.
|
|||
|
||||
// FailMessageStreamFactory mock MessageStreamFactory failure
|
||||
type FailMessageStreamFactory struct {
|
||||
msgstream.Factory
|
||||
dependency.Factory
|
||||
}
|
||||
|
||||
func (f *FailMessageStreamFactory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
|
|
|
@ -26,24 +26,25 @@ import (
|
|||
"time"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datacoord"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
)
|
||||
|
||||
// Params is the parameters for DataCoord grpc server
|
||||
|
@ -65,7 +66,7 @@ type Server struct {
|
|||
}
|
||||
|
||||
// NewServer new data service grpc server
|
||||
func NewServer(ctx context.Context, factory msgstream.Factory, opts ...datacoord.Option) *Server {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory, opts ...datacoord.Option) *Server {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
|
||||
s := &Server{
|
||||
|
|
|
@ -27,26 +27,27 @@ import (
|
|||
"time"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
dn "github.com/milvus-io/milvus/internal/datanode"
|
||||
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
|
||||
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
)
|
||||
|
||||
// Params contains parameters for datanode grpc server.
|
||||
|
@ -60,7 +61,7 @@ type Server struct {
|
|||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
etcdCli *clientv3.Client
|
||||
msFactory msgstream.Factory
|
||||
factory dependency.Factory
|
||||
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
|
@ -72,12 +73,12 @@ type Server struct {
|
|||
}
|
||||
|
||||
// NewServer new DataNode grpc server
|
||||
func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
var s = &Server{
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
grpcErrChan: make(chan error),
|
||||
newRootCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.RootCoord, error) {
|
||||
return rcc.NewClient(ctx1, etcdMetaRoot, client)
|
||||
|
@ -87,7 +88,7 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||
},
|
||||
}
|
||||
|
||||
s.datanode = dn.NewDataNode(s.ctx, s.msFactory)
|
||||
s.datanode = dn.NewDataNode(s.ctx, s.factory)
|
||||
|
||||
return s, nil
|
||||
}
|
||||
|
|
|
@ -20,21 +20,24 @@ import (
|
|||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
grpcindexcoord "github.com/milvus-io/milvus/internal/distributed/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestIndexCoordClient(t *testing.T) {
|
||||
ClientParams.InitOnce(typeutil.IndexCoordRole)
|
||||
ctx := context.Background()
|
||||
server, err := grpcindexcoord.NewServer(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
server, err := grpcindexcoord.NewServer(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
icm := &indexcoord.Mock{}
|
||||
err = server.SetClient(icm)
|
||||
|
|
|
@ -30,6 +30,7 @@ import (
|
|||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -37,6 +38,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
@ -241,9 +243,9 @@ func (s *Server) startGrpcLoop(grpcPort int) {
|
|||
}
|
||||
|
||||
// NewServer create a new IndexCoord grpc server.
|
||||
func NewServer(ctx context.Context) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
serverImp, err := indexcoord.NewIndexCoord(ctx)
|
||||
serverImp, err := indexcoord.NewIndexCoord(ctx, factory)
|
||||
if err != nil {
|
||||
defer cancel()
|
||||
return nil, err
|
||||
|
|
|
@ -20,18 +20,21 @@ import (
|
|||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
etcd "github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
)
|
||||
|
||||
func TestIndexCoordinateServer(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
server, err := NewServer(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
server, err := NewServer(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, server)
|
||||
Params.Init()
|
||||
|
|
|
@ -21,18 +21,20 @@ import (
|
|||
"errors"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
grpcindexnode "github.com/milvus-io/milvus/internal/distributed/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/mock"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
var ParamsGlobal paramtable.ComponentParam
|
||||
|
@ -122,7 +124,8 @@ func Test_NewClient(t *testing.T) {
|
|||
func TestIndexNodeClient(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
ins, err := grpcindexnode.NewServer(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
ins, err := grpcindexnode.NewServer(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, ins)
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import (
|
|||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -38,6 +39,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
@ -241,9 +243,9 @@ func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsReq
|
|||
}
|
||||
|
||||
// NewServer create a new IndexNode grpc server.
|
||||
func NewServer(ctx context.Context) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
node, err := indexnode.NewIndexNode(ctx1)
|
||||
node, err := indexnode.NewIndexNode(ctx1, factory)
|
||||
if err != nil {
|
||||
defer cancel()
|
||||
return nil, err
|
||||
|
|
|
@ -20,21 +20,24 @@ import (
|
|||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
var ParamsGlobal paramtable.ComponentParam
|
||||
|
||||
func TestIndexNodeServer(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
server, err := NewServer(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
server, err := NewServer(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, server)
|
||||
|
||||
|
|
|
@ -28,30 +28,31 @@ import (
|
|||
|
||||
"github.com/gin-gonic/gin"
|
||||
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"
|
||||
"github.com/milvus-io/milvus/internal/distributed/proxy/httpserver"
|
||||
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
|
||||
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/internal/proxy"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/health/grpc_health_v1"
|
||||
"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"
|
||||
"github.com/milvus-io/milvus/internal/distributed/proxy/httpserver"
|
||||
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
|
||||
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/internal/proxy"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
)
|
||||
|
||||
var Params paramtable.GrpcServerConfig
|
||||
|
@ -80,7 +81,7 @@ type Server struct {
|
|||
}
|
||||
|
||||
// NewServer create a Proxy server.
|
||||
func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
|
||||
var err error
|
||||
server := &Server{
|
||||
|
|
|
@ -25,26 +25,27 @@ import (
|
|||
"time"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
|
||||
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
|
||||
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
qc "github.com/milvus-io/milvus/internal/querycoord"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
)
|
||||
|
||||
var Params paramtable.GrpcServerConfig
|
||||
|
@ -60,7 +61,7 @@ type Server struct {
|
|||
|
||||
queryCoord types.QueryCoordComponent
|
||||
|
||||
msFactory msgstream.Factory
|
||||
factory dependency.Factory
|
||||
|
||||
etcdCli *clientv3.Client
|
||||
|
||||
|
@ -72,7 +73,7 @@ type Server struct {
|
|||
}
|
||||
|
||||
// NewServer create a new QueryCoord grpc server.
|
||||
func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
svr, err := qc.NewQueryCoord(ctx1, factory)
|
||||
if err != nil {
|
||||
|
@ -84,7 +85,7 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||
queryCoord: svr,
|
||||
loopCtx: ctx1,
|
||||
loopCancel: cancel,
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
grpcErrChan: make(chan error),
|
||||
}, nil
|
||||
}
|
||||
|
|
|
@ -32,13 +32,13 @@ import (
|
|||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
qn "github.com/milvus-io/milvus/internal/querynode"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
@ -68,7 +68,7 @@ type Server struct {
|
|||
}
|
||||
|
||||
// NewServer create a new QueryNode grpc server.
|
||||
func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
|
||||
s := &Server{
|
||||
|
|
|
@ -25,9 +25,13 @@ import (
|
|||
"time"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
pnc "github.com/milvus-io/milvus/internal/distributed/proxy/client"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
|
@ -36,16 +40,13 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
|
||||
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
|
||||
|
@ -93,7 +94,7 @@ func (s *Server) AlterAlias(ctx context.Context, request *milvuspb.AlterAliasReq
|
|||
}
|
||||
|
||||
// NewServer create a new RootCoord grpc server.
|
||||
func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error) {
|
||||
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
s := &Server{
|
||||
ctx: ctx1,
|
||||
|
|
|
@ -29,8 +29,9 @@ import (
|
|||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
|
@ -41,11 +42,11 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
type proxyMock struct {
|
||||
|
@ -75,8 +76,8 @@ func TestGrpcService(t *testing.T) {
|
|||
t.Log("newParams.Address:", Params.GetAddress())
|
||||
|
||||
ctx := context.Background()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
svr, err := NewServer(ctx, msFactory)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
svr, err := NewServer(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
|
||||
rootcoord.Params.Init()
|
||||
|
|
|
@ -30,11 +30,14 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
|
@ -52,7 +55,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
)
|
||||
|
||||
// make sure IndexCoord implements types.IndexCoord
|
||||
|
@ -82,6 +84,7 @@ type IndexCoord struct {
|
|||
|
||||
idAllocator *allocator.GlobalIDAllocator
|
||||
|
||||
factory dependency.Factory
|
||||
etcdCli *clientv3.Client
|
||||
chunkManager storage.ChunkManager
|
||||
|
||||
|
@ -109,7 +112,7 @@ type IndexCoord struct {
|
|||
type UniqueID = typeutil.UniqueID
|
||||
|
||||
// NewIndexCoord creates a new IndexCoord component.
|
||||
func NewIndexCoord(ctx context.Context) (*IndexCoord, error) {
|
||||
func NewIndexCoord(ctx context.Context, factory dependency.Factory) (*IndexCoord, error) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
i := &IndexCoord{
|
||||
|
@ -119,6 +122,7 @@ func NewIndexCoord(ctx context.Context) (*IndexCoord, error) {
|
|||
durationInterval: time.Second * 10,
|
||||
assignTaskInterval: time.Second * 3,
|
||||
taskLimit: 20,
|
||||
factory: factory,
|
||||
}
|
||||
i.UpdateStateCode(internalpb.StateCode_Abnormal)
|
||||
return i, nil
|
||||
|
@ -160,6 +164,8 @@ func (i *IndexCoord) Init() error {
|
|||
i.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||
log.Debug("IndexCoord init", zap.Any("stateCode", i.stateCode.Load().(internalpb.StateCode)))
|
||||
|
||||
i.factory.Init(&Params)
|
||||
|
||||
err := i.initSession()
|
||||
if err != nil {
|
||||
log.Error(err.Error())
|
||||
|
@ -221,13 +227,7 @@ func (i *IndexCoord) Init() error {
|
|||
return
|
||||
}
|
||||
|
||||
chunkManager, err := storage.NewMinioChunkManager(i.loopCtx,
|
||||
storage.Address(Params.MinioCfg.Address),
|
||||
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(Params.MinioCfg.UseSSL),
|
||||
storage.BucketName(Params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
chunkManager, err := i.factory.NewVectorStorageChunkManager(i.loopCtx)
|
||||
|
||||
if err != nil {
|
||||
log.Error("IndexCoord new minio chunkManager failed", zap.Error(err))
|
||||
|
|
|
@ -26,6 +26,9 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
grpcindexnode "github.com/milvus-io/milvus/internal/distributed/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
|
@ -35,11 +38,10 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func TestIndexCoord(t *testing.T) {
|
||||
|
@ -55,7 +57,8 @@ func TestIndexCoord(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
err = inm0.Start()
|
||||
assert.Nil(t, err)
|
||||
ic, err := NewIndexCoord(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
ic, err := NewIndexCoord(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
ic.reqTimeoutInterval = time.Second * 10
|
||||
ic.durationInterval = time.Second
|
||||
|
@ -73,7 +76,7 @@ func TestIndexCoord(t *testing.T) {
|
|||
err = inm0.Stop()
|
||||
assert.Nil(t, err)
|
||||
|
||||
in, err := grpcindexnode.NewServer(ctx)
|
||||
in, err := grpcindexnode.NewServer(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, in)
|
||||
inm := &indexnode.Mock{
|
||||
|
|
|
@ -20,15 +20,18 @@ import (
|
|||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestGetSystemInfoMetrics(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
ic, err := NewIndexCoord(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
ic, err := NewIndexCoord(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
Params.Init()
|
||||
|
||||
|
|
|
@ -44,6 +44,10 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
|
@ -58,8 +62,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// UniqueID is an alias of int64, is used as a unique identifier for the request.
|
||||
|
@ -85,6 +87,7 @@ type IndexNode struct {
|
|||
|
||||
once sync.Once
|
||||
|
||||
factory dependency.Factory
|
||||
chunkManager storage.ChunkManager
|
||||
session *sessionutil.Session
|
||||
|
||||
|
@ -102,13 +105,14 @@ type IndexNode struct {
|
|||
}
|
||||
|
||||
// NewIndexNode creates a new IndexNode component.
|
||||
func NewIndexNode(ctx context.Context) (*IndexNode, error) {
|
||||
func NewIndexNode(ctx context.Context, factory dependency.Factory) (*IndexNode, error) {
|
||||
log.Debug("New IndexNode ...")
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
b := &IndexNode{
|
||||
loopCtx: ctx1,
|
||||
loopCancel: cancel,
|
||||
factory: factory,
|
||||
}
|
||||
b.UpdateStateCode(internalpb.StateCode_Abnormal)
|
||||
sc, err := NewTaskScheduler(b.loopCtx, b.chunkManager)
|
||||
|
@ -168,6 +172,8 @@ func (i *IndexNode) Init() error {
|
|||
i.initOnce.Do(func() {
|
||||
Params.Init()
|
||||
|
||||
i.factory.Init(&Params)
|
||||
|
||||
i.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||
log.Debug("IndexNode init", zap.Any("State", i.stateCode.Load().(internalpb.StateCode)))
|
||||
err := i.initSession()
|
||||
|
@ -181,13 +187,7 @@ func (i *IndexNode) Init() error {
|
|||
etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
i.etcdKV = etcdKV
|
||||
|
||||
chunkManager, err := storage.NewMinioChunkManager(i.loopCtx,
|
||||
storage.Address(Params.MinioCfg.Address),
|
||||
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(Params.MinioCfg.UseSSL),
|
||||
storage.BucketName(Params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
chunkManager, err := i.factory.NewVectorStorageChunkManager(i.loopCtx)
|
||||
|
||||
if err != nil {
|
||||
log.Error("IndexNode NewMinIOKV failed", zap.Error(err))
|
||||
|
|
|
@ -30,6 +30,8 @@ import (
|
|||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -39,10 +41,10 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestIndexNode(t *testing.T) {
|
||||
|
@ -63,7 +65,8 @@ func TestIndexNode(t *testing.T) {
|
|||
floatVectorBinlogPath := "float_vector_binlog"
|
||||
binaryVectorBinlogPath := "binary_vector_binlog"
|
||||
|
||||
in, err := NewIndexNode(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
in, err := NewIndexNode(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
Params.Init()
|
||||
|
||||
|
@ -81,6 +84,7 @@ func TestIndexNode(t *testing.T) {
|
|||
err = in.Register()
|
||||
assert.Nil(t, err)
|
||||
|
||||
in.chunkManager = storage.NewLocalChunkManager(storage.RootPath("/tmp/lib/milvus"))
|
||||
t.Run("CreateIndex FloatVector", func(t *testing.T) {
|
||||
var insertCodec storage.InsertCodec
|
||||
|
||||
|
@ -476,7 +480,8 @@ func TestCreateIndexFailed(t *testing.T) {
|
|||
metaPath2 := "FloatVector2"
|
||||
floatVectorBinlogPath := "float_vector_binlog"
|
||||
|
||||
in, err := NewIndexNode(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
in, err := NewIndexNode(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
Params.Init()
|
||||
|
||||
|
@ -494,6 +499,7 @@ func TestCreateIndexFailed(t *testing.T) {
|
|||
err = in.Register()
|
||||
assert.Nil(t, err)
|
||||
|
||||
in.chunkManager = storage.NewLocalChunkManager(storage.RootPath("/tmp/lib/milvus"))
|
||||
t.Run("CreateIndex error", func(t *testing.T) {
|
||||
var insertCodec storage.InsertCodec
|
||||
|
||||
|
@ -748,7 +754,8 @@ func TestCreateIndexFailed(t *testing.T) {
|
|||
func TestIndexNode_Error(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
in, err := NewIndexNode(ctx)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
in, err := NewIndexNode(ctx, factory)
|
||||
assert.Nil(t, err)
|
||||
Params.Init()
|
||||
|
||||
|
@ -768,6 +775,7 @@ func TestIndexNode_Error(t *testing.T) {
|
|||
|
||||
in.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||
|
||||
in.chunkManager = storage.NewLocalChunkManager(storage.RootPath("/tmp/lib/milvus"))
|
||||
t.Run("CreateIndex", func(t *testing.T) {
|
||||
status, err := in.CreateIndex(ctx, &indexpb.CreateIndexRequest{})
|
||||
assert.Nil(t, err)
|
||||
|
|
|
@ -34,6 +34,7 @@ import (
|
|||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -498,6 +499,7 @@ func (it *IndexBuildTask) saveIndex(ctx context.Context, blobs []*storage.Blob)
|
|||
it.savePaths = make([]string, blobCnt)
|
||||
saveIndexFile := func(idx int) error {
|
||||
blob := blobs[idx]
|
||||
log.Info("xxxxxxxxxxxxxxxxxxxxxxxxxx")
|
||||
savePath := getSavePathByKey(blob.Key)
|
||||
saveIndexFileFn := func() error {
|
||||
v, err := it.etcdKV.Load(it.req.MetaPath)
|
||||
|
@ -505,12 +507,14 @@ func (it *IndexBuildTask) saveIndex(ctx context.Context, blobs []*storage.Blob)
|
|||
log.Warn("IndexNode load meta failed", zap.Any("path", it.req.MetaPath), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("ggggggggggggggggggggggg")
|
||||
indexMeta := indexpb.IndexMeta{}
|
||||
err = proto.Unmarshal([]byte(v), &indexMeta)
|
||||
if err != nil {
|
||||
log.Warn("IndexNode Unmarshal indexMeta error ", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("hhhhhhhhhhhhhhhhhhhhhh")
|
||||
//log.Debug("IndexNode Unmarshal indexMeta success ", zap.Any("meta", indexMeta))
|
||||
if indexMeta.Version > it.req.Version {
|
||||
log.Warn("IndexNode try saveIndexFile failed req.Version is low", zap.Any("req.Version", it.req.Version),
|
||||
|
|
|
@ -47,16 +47,15 @@ func InitRmq(rocksdbName string, idAllocator allocator.GIDAllocator) error {
|
|||
}
|
||||
|
||||
// InitRocksMQ init global rocksmq single instance
|
||||
func InitRocksMQ() error {
|
||||
func InitRocksMQ(path string) error {
|
||||
var finalErr error
|
||||
once.Do(func() {
|
||||
params.Init()
|
||||
rocksdbName, _ := params.Load("_RocksmqPath")
|
||||
log.Debug("initializing global rmq", zap.String("path", rocksdbName))
|
||||
log.Debug("initializing global rmq", zap.String("path", path))
|
||||
var fi os.FileInfo
|
||||
fi, finalErr = os.Stat(rocksdbName)
|
||||
fi, finalErr = os.Stat(path)
|
||||
if os.IsNotExist(finalErr) {
|
||||
finalErr = os.MkdirAll(rocksdbName, os.ModePerm)
|
||||
finalErr = os.MkdirAll(path, os.ModePerm)
|
||||
if finalErr != nil {
|
||||
return
|
||||
}
|
||||
|
@ -97,7 +96,7 @@ func InitRocksMQ() error {
|
|||
}
|
||||
log.Debug("", zap.Any("RocksmqRetentionTimeInMinutes", rawRmqRetentionTimeInMinutes),
|
||||
zap.Any("RocksmqRetentionSizeInMB", RocksmqRetentionSizeInMB), zap.Any("RocksmqPageSize", RocksmqPageSize))
|
||||
Rmq, finalErr = NewRocksMQ(rocksdbName, nil)
|
||||
Rmq, finalErr = NewRocksMQ(path, nil)
|
||||
})
|
||||
return finalErr
|
||||
}
|
||||
|
|
|
@ -21,8 +21,9 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
)
|
||||
|
||||
func Test_InitRmq(t *testing.T) {
|
||||
|
@ -51,12 +52,9 @@ func Test_InitRmq(t *testing.T) {
|
|||
}
|
||||
|
||||
func Test_InitRocksMQ(t *testing.T) {
|
||||
// Params.Init()
|
||||
rmqPath := "/tmp/milvus/rdb_data_global"
|
||||
err := os.Setenv("ROCKSMQ_PATH", rmqPath)
|
||||
assert.Nil(t, err)
|
||||
defer os.RemoveAll("/tmp/milvus")
|
||||
err = InitRocksMQ()
|
||||
err := InitRocksMQ(rmqPath)
|
||||
defer Rmq.stopRetention()
|
||||
assert.NoError(t, err)
|
||||
defer CloseRocksMQ()
|
||||
|
@ -86,8 +84,7 @@ func Test_InitRocksMQError(t *testing.T) {
|
|||
f, err := os.Create(dummyPath)
|
||||
defer f.Close()
|
||||
assert.NoError(t, err)
|
||||
os.Setenv("ROCKSMQ_PATH", dummyPath)
|
||||
defer os.RemoveAll(dir)
|
||||
err = InitRocksMQ()
|
||||
err = InitRocksMQ(dummyPath)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
|
|
@ -19,11 +19,15 @@ package msgstream
|
|||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
||||
rmqimplserver "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
|
||||
puslarmqwrapper "github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper/pulsar"
|
||||
rmqwrapper "github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper/rmq"
|
||||
)
|
||||
|
@ -37,12 +41,12 @@ type PmsFactory struct {
|
|||
PulsarBufSize int64
|
||||
}
|
||||
|
||||
// Init is used to set parameters for PmsFactory
|
||||
func (f *PmsFactory) Init(params *paramtable.ComponentParam) error {
|
||||
f.PulsarBufSize = 1024
|
||||
f.ReceiveBufSize = 1024
|
||||
f.PulsarAddress = params.PulsarCfg.Address
|
||||
return nil
|
||||
func NewPmsFactory(config *paramtable.PulsarConfig) *PmsFactory {
|
||||
return &PmsFactory{
|
||||
PulsarBufSize: 1024,
|
||||
ReceiveBufSize: 1024,
|
||||
PulsarAddress: config.Address,
|
||||
}
|
||||
}
|
||||
|
||||
// NewMsgStream is used to generate a new Msgstream object
|
||||
|
@ -68,16 +72,6 @@ func (f *PmsFactory) NewQueryMsgStream(ctx context.Context) (MsgStream, error) {
|
|||
return f.NewMsgStream(ctx)
|
||||
}
|
||||
|
||||
// NewPmsFactory is used to generate a new PmsFactory object
|
||||
func NewPmsFactory() Factory {
|
||||
f := &PmsFactory{
|
||||
dispatcherFactory: ProtoUDFactory{},
|
||||
ReceiveBufSize: 64,
|
||||
PulsarBufSize: 64,
|
||||
}
|
||||
return f
|
||||
}
|
||||
|
||||
// RmsFactory is a rocksmq msgstream factory that implemented Factory interface(msgstream.go)
|
||||
type RmsFactory struct {
|
||||
dispatcherFactory ProtoUDFactory
|
||||
|
@ -86,13 +80,6 @@ type RmsFactory struct {
|
|||
RmqBufSize int64
|
||||
}
|
||||
|
||||
// Init is used to set parameters for RmsFactory
|
||||
func (f *RmsFactory) Init(params *paramtable.ComponentParam) error {
|
||||
f.RmqBufSize = 1024
|
||||
f.ReceiveBufSize = 1024
|
||||
return nil
|
||||
}
|
||||
|
||||
// NewMsgStream is used to generate a new Msgstream object
|
||||
func (f *RmsFactory) NewMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
rmqClient, err := rmqwrapper.NewClientWithDefaultOptions()
|
||||
|
@ -121,13 +108,15 @@ func (f *RmsFactory) NewQueryMsgStream(ctx context.Context) (MsgStream, error) {
|
|||
}
|
||||
|
||||
// NewRmsFactory is used to generate a new RmsFactory object
|
||||
func NewRmsFactory() Factory {
|
||||
func NewRmsFactory(path string) *RmsFactory {
|
||||
f := &RmsFactory{
|
||||
dispatcherFactory: ProtoUDFactory{},
|
||||
ReceiveBufSize: 1024,
|
||||
RmqBufSize: 1024,
|
||||
}
|
||||
|
||||
rmqimplserver.InitRocksMQ()
|
||||
err := rmqimplserver.InitRocksMQ(path)
|
||||
if err != nil {
|
||||
log.Error("init rmq error", zap.Error(err))
|
||||
}
|
||||
return f
|
||||
}
|
||||
|
|
|
@ -25,9 +25,7 @@ import (
|
|||
)
|
||||
|
||||
func TestPmsFactory(t *testing.T) {
|
||||
pmsFactory := NewPmsFactory()
|
||||
|
||||
pmsFactory.Init(&Params)
|
||||
pmsFactory := NewPmsFactory(&Params.PulsarCfg)
|
||||
|
||||
ctx := context.Background()
|
||||
_, err := pmsFactory.NewMsgStream(ctx)
|
||||
|
@ -40,19 +38,10 @@ func TestPmsFactory(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func TestPmsFactory_Init(t *testing.T) {
|
||||
rmsFactory := NewRmsFactory()
|
||||
err := rmsFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func TestRmsFactory(t *testing.T) {
|
||||
os.Setenv("ROCKSMQ_PATH", "/tmp/milvus")
|
||||
defer os.Unsetenv("ROCKSMQ_PATH")
|
||||
|
||||
rmsFactory := NewRmsFactory()
|
||||
|
||||
rmsFactory.Init(&Params)
|
||||
rmsFactory := NewRmsFactory("tmp/milvus")
|
||||
|
||||
ctx := context.Background()
|
||||
_, err := rmsFactory.NewMsgStream(ctx)
|
||||
|
@ -64,9 +53,3 @@ func TestRmsFactory(t *testing.T) {
|
|||
_, err = rmsFactory.NewQueryMsgStream(ctx)
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func TestRmsFactory_Init(t *testing.T) {
|
||||
rmsFactory := NewRmsFactory()
|
||||
err := rmsFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import (
|
|||
rocksmqimplserver "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
|
||||
pulsarwrapper "github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper/pulsar"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
@ -36,11 +37,9 @@ import (
|
|||
var Params paramtable.BaseTable
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
Params.Init()
|
||||
path := "/tmp/milvus/rdb_data"
|
||||
os.Setenv("ROCKSMQ_PATH", path)
|
||||
defer os.RemoveAll(path)
|
||||
_ = rocksmqimplserver.InitRocksMQ()
|
||||
_ = rocksmqimplserver.InitRocksMQ(path)
|
||||
exitCode := m.Run()
|
||||
defer rocksmqimplserver.CloseRocksMQ()
|
||||
os.Exit(exitCode)
|
||||
|
|
|
@ -19,8 +19,6 @@ package msgstream
|
|||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
@ -75,9 +73,7 @@ type MsgStream interface {
|
|||
GetLatestMsgID(channel string) (MessageID, error)
|
||||
}
|
||||
|
||||
// Factory is an interface that can be used to generate a new msgstream object
|
||||
type Factory interface {
|
||||
Init(params *paramtable.ComponentParam) error
|
||||
NewMsgStream(ctx context.Context) (MsgStream, error)
|
||||
NewTtMsgStream(ctx context.Context) (MsgStream, error)
|
||||
NewQueryMsgStream(ctx context.Context) (MsgStream, error)
|
||||
|
|
|
@ -27,21 +27,22 @@ import (
|
|||
"syscall"
|
||||
"time"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// UniqueID is alias of typeutil.UniqueID
|
||||
|
@ -90,7 +91,7 @@ type Proxy struct {
|
|||
|
||||
session *sessionutil.Session
|
||||
|
||||
msFactory msgstream.Factory
|
||||
factory dependency.Factory
|
||||
|
||||
searchResultCh chan *internalpb.SearchResults
|
||||
retrieveResultCh chan *internalpb.RetrieveResults
|
||||
|
@ -101,14 +102,14 @@ type Proxy struct {
|
|||
}
|
||||
|
||||
// NewProxy returns a Proxy struct.
|
||||
func NewProxy(ctx context.Context, factory msgstream.Factory) (*Proxy, error) {
|
||||
func NewProxy(ctx context.Context, factory dependency.Factory) (*Proxy, error) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
n := 1024 // better to be configurable
|
||||
node := &Proxy{
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
searchResultCh: make(chan *internalpb.SearchResults, n),
|
||||
retrieveResultCh: make(chan *internalpb.RetrieveResults, n),
|
||||
}
|
||||
|
@ -182,15 +183,8 @@ func (node *Proxy) Init() error {
|
|||
log.Debug("create query channel for Proxy done", zap.String("QueryResultChannel", resp.QueryResultChannel))
|
||||
}
|
||||
|
||||
log.Debug("set parameters for ms factory", zap.String("role", typeutil.ProxyRole), zap.Any("parameters", Params.ServiceParam))
|
||||
if err := node.msFactory.Init(&Params); err != nil {
|
||||
log.Warn("failed to set parameters for ms factory",
|
||||
zap.Error(err),
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
zap.Any("parameters", Params.ServiceParam))
|
||||
return err
|
||||
}
|
||||
log.Debug("set parameters for ms factory done", zap.String("role", typeutil.ProxyRole), zap.Any("parameters", Params.ServiceParam))
|
||||
node.factory.Init(&Params)
|
||||
log.Debug("init parameters for factory", zap.String("role", typeutil.ProxyRole), zap.Any("parameters", Params.ServiceParam))
|
||||
|
||||
log.Debug("create id allocator", zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", Params.ProxyCfg.ProxyID))
|
||||
idAllocator, err := allocator.NewIDAllocator(node.ctx, node.rootCoord, Params.ProxyCfg.ProxyID)
|
||||
|
@ -229,12 +223,12 @@ func (node *Proxy) Init() error {
|
|||
log.Debug("create channels manager", zap.String("role", typeutil.ProxyRole))
|
||||
dmlChannelsFunc := getDmlChannelsFunc(node.ctx, node.rootCoord)
|
||||
dqlChannelsFunc := getDqlChannelsFunc(node.ctx, node.session.ServerID, node.queryCoord)
|
||||
chMgr := newChannelsMgrImpl(dmlChannelsFunc, defaultInsertRepackFunc, dqlChannelsFunc, nil, node.msFactory)
|
||||
chMgr := newChannelsMgrImpl(dmlChannelsFunc, defaultInsertRepackFunc, dqlChannelsFunc, nil, node.factory)
|
||||
node.chMgr = chMgr
|
||||
log.Debug("create channels manager done", zap.String("role", typeutil.ProxyRole))
|
||||
|
||||
log.Debug("create task scheduler", zap.String("role", typeutil.ProxyRole))
|
||||
node.sched, err = newTaskScheduler(node.ctx, node.idAllocator, node.tsoAllocator, node.msFactory,
|
||||
node.sched, err = newTaskScheduler(node.ctx, node.idAllocator, node.tsoAllocator, node.factory,
|
||||
schedOptWithSearchResultCh(node.searchResultCh),
|
||||
schedOptWithRetrieveResultCh(node.retrieveResultCh))
|
||||
if err != nil {
|
||||
|
|
|
@ -31,13 +31,16 @@ import (
|
|||
"time"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
|
@ -92,10 +95,9 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/indexnode"
|
||||
"github.com/milvus-io/milvus/internal/querynode"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/querycoord"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/querycoord"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
|
@ -116,13 +118,6 @@ func init() {
|
|||
Registry.MustRegister(prometheus.NewGoCollector())
|
||||
}
|
||||
|
||||
func newMsgFactory(localMsg bool) msgstream.Factory {
|
||||
if localMsg {
|
||||
return msgstream.NewRmsFactory()
|
||||
}
|
||||
return msgstream.NewPmsFactory()
|
||||
}
|
||||
|
||||
func runRootCoord(ctx context.Context, localMsg bool) *grpcrootcoord.Server {
|
||||
var rc *grpcrootcoord.Server
|
||||
var wg sync.WaitGroup
|
||||
|
@ -135,7 +130,7 @@ func runRootCoord(ctx context.Context, localMsg bool) *grpcrootcoord.Server {
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
rc, err = grpcrootcoord.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -166,7 +161,7 @@ func runQueryCoord(ctx context.Context, localMsg bool) *grpcquerycoord.Server {
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
qs, err = grpcquerycoord.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -198,7 +193,7 @@ func runQueryNode(ctx context.Context, localMsg bool, alias string) *grpcqueryno
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
qn, err = grpcquerynode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -229,7 +224,7 @@ func runDataCoord(ctx context.Context, localMsg bool) *grpcdatacoordclient.Serve
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
ds = grpcdatacoordclient.NewServer(ctx, factory)
|
||||
wg.Done()
|
||||
err := ds.Run()
|
||||
|
@ -257,7 +252,7 @@ func runDataNode(ctx context.Context, localMsg bool, alias string) *grpcdatanode
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
dn, err = grpcdatanode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -288,8 +283,9 @@ func runIndexCoord(ctx context.Context, localMsg bool) *grpcindexcoord.Server {
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
is, err = grpcindexcoord.NewServer(ctx)
|
||||
is, err = grpcindexcoord.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -319,8 +315,9 @@ func runIndexNode(ctx context.Context, localMsg bool, alias string) *grpcindexno
|
|||
defer log.Sync()
|
||||
}
|
||||
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
var err error
|
||||
in, err = grpcindexnode.NewServer(ctx)
|
||||
in, err = grpcindexnode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -439,7 +436,7 @@ func TestProxy(t *testing.T) {
|
|||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
localMsg := true
|
||||
factory := newMsgFactory(localMsg)
|
||||
factory := dependency.NewDefaultFactory(localMsg)
|
||||
alias := "TestProxy"
|
||||
|
||||
rc := runRootCoord(ctx, localMsg)
|
||||
|
|
|
@ -25,8 +25,8 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
)
|
||||
|
||||
|
@ -48,7 +48,7 @@ func Test_HandlerReloadFromKV(t *testing.T) {
|
|||
err = kv.Save(channelInfoKey, string(channelInfoBytes))
|
||||
assert.Nil(t, err)
|
||||
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
handler, err := newChannelUnsubscribeHandler(baseCtx, kv, factory)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, 1, len(handler.downNodeChan))
|
||||
|
@ -63,7 +63,7 @@ func Test_AddUnsubscribeChannelInfo(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
defer etcdCli.Close()
|
||||
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
handler, err := newChannelUnsubscribeHandler(baseCtx, kv, factory)
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
@ -95,8 +95,7 @@ func Test_HandleChannelUnsubscribeLoop(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
defer etcdCli.Close()
|
||||
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory.Init(&Params)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
handler, err := newChannelUnsubscribeHandler(baseCtx, kv, factory)
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import (
|
|||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
@ -33,7 +34,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
|
@ -409,7 +409,7 @@ func TestReloadClusterFromKV(t *testing.T) {
|
|||
clusterSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||
clusterSession.Register()
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
handler, err := newChannelUnsubscribeHandler(ctx, kv, factory)
|
||||
assert.Nil(t, err)
|
||||
meta, err := newMeta(ctx, kv, factory, nil)
|
||||
|
@ -456,8 +456,7 @@ func TestGrpcRequest(t *testing.T) {
|
|||
clusterSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||
clusterSession.Register()
|
||||
factory := msgstream.NewPmsFactory()
|
||||
err = factory.Init(&Params)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
assert.Nil(t, err)
|
||||
idAllocator := func() (UniqueID, error) {
|
||||
return 0, nil
|
||||
|
@ -647,9 +646,7 @@ func TestSetNodeState(t *testing.T) {
|
|||
clusterSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||
clusterSession.Register()
|
||||
factory := msgstream.NewPmsFactory()
|
||||
err = factory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
idAllocator := func() (UniqueID, error) {
|
||||
return 0, nil
|
||||
}
|
||||
|
|
|
@ -30,23 +30,24 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -98,7 +99,7 @@ type QueryCoord struct {
|
|||
|
||||
stateCode atomic.Value
|
||||
|
||||
msFactory msgstream.Factory
|
||||
factory dependency.Factory
|
||||
chunkManager storage.ChunkManager
|
||||
}
|
||||
|
||||
|
@ -160,14 +161,14 @@ func (qc *QueryCoord) Init() error {
|
|||
}
|
||||
|
||||
// init meta
|
||||
qc.meta, initError = newMeta(qc.loopCtx, qc.kvClient, qc.msFactory, qc.idAllocator)
|
||||
qc.meta, initError = newMeta(qc.loopCtx, qc.kvClient, qc.factory, qc.idAllocator)
|
||||
if initError != nil {
|
||||
log.Error("query coordinator init meta failed", zap.Error(initError))
|
||||
return
|
||||
}
|
||||
|
||||
// init channelUnsubscribeHandler
|
||||
qc.handler, initError = newChannelUnsubscribeHandler(qc.loopCtx, qc.kvClient, qc.msFactory)
|
||||
qc.handler, initError = newChannelUnsubscribeHandler(qc.loopCtx, qc.kvClient, qc.factory)
|
||||
if initError != nil {
|
||||
log.Error("query coordinator init channelUnsubscribeHandler failed", zap.Error(initError))
|
||||
return
|
||||
|
@ -180,13 +181,7 @@ func (qc *QueryCoord) Init() error {
|
|||
return
|
||||
}
|
||||
|
||||
qc.chunkManager, initError = storage.NewMinioChunkManager(qc.loopCtx,
|
||||
storage.Address(Params.MinioCfg.Address),
|
||||
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(Params.MinioCfg.UseSSL),
|
||||
storage.BucketName(Params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
qc.chunkManager, initError = qc.factory.NewVectorStorageChunkManager(qc.loopCtx)
|
||||
|
||||
if initError != nil {
|
||||
log.Error("query coordinator init cluster failed", zap.Error(initError))
|
||||
|
@ -222,10 +217,7 @@ func (qc *QueryCoord) Init() error {
|
|||
|
||||
// Start function starts the goroutines to watch the meta and node updates
|
||||
func (qc *QueryCoord) Start() error {
|
||||
err := qc.msFactory.Init(&Params)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
qc.factory.Init(&Params)
|
||||
qc.scheduler.Start()
|
||||
log.Debug("start scheduler ...")
|
||||
|
||||
|
@ -289,13 +281,13 @@ func (qc *QueryCoord) UpdateStateCode(code internalpb.StateCode) {
|
|||
}
|
||||
|
||||
// NewQueryCoord creates a QueryCoord object.
|
||||
func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord, error) {
|
||||
func NewQueryCoord(ctx context.Context, factory dependency.Factory) (*QueryCoord, error) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
service := &QueryCoord{
|
||||
loopCtx: ctx1,
|
||||
loopCancel: cancel,
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
newNodeFn: newQueryNode,
|
||||
}
|
||||
|
||||
|
|
|
@ -31,14 +31,15 @@ import (
|
|||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
var queryCoordTestDir = "/tmp/milvus_test/query_coord"
|
||||
|
@ -65,7 +66,7 @@ func TestMain(m *testing.M) {
|
|||
os.Exit(exitCode)
|
||||
}
|
||||
|
||||
func NewQueryCoordTest(ctx context.Context, factory msgstream.Factory) (*QueryCoord, error) {
|
||||
func NewQueryCoordTest(ctx context.Context, factory dependency.Factory) (*QueryCoord, error) {
|
||||
queryCoord, err := NewQueryCoord(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -75,7 +76,7 @@ func NewQueryCoordTest(ctx context.Context, factory msgstream.Factory) (*QueryCo
|
|||
}
|
||||
|
||||
func startQueryCoord(ctx context.Context) (*QueryCoord, error) {
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
coord, err := NewQueryCoordTest(ctx, factory)
|
||||
if err != nil {
|
||||
|
@ -121,7 +122,7 @@ func createDefaultPartition(ctx context.Context, queryCoord *QueryCoord) error {
|
|||
}
|
||||
|
||||
func startUnHealthyQueryCoord(ctx context.Context) (*QueryCoord, error) {
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
coord, err := NewQueryCoordTest(ctx, factory)
|
||||
if err != nil {
|
||||
|
|
|
@ -23,8 +23,8 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
@ -39,7 +39,7 @@ func TestShuffleSegmentsToQueryNode(t *testing.T) {
|
|||
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true, false)
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
meta, err := newMeta(baseCtx, kv, factory, nil)
|
||||
assert.Nil(t, err)
|
||||
handler, err := newChannelUnsubscribeHandler(baseCtx, kv, factory)
|
||||
|
|
|
@ -33,7 +33,7 @@ func TestDataSyncService_DMLFlowGraphs(t *testing.T) {
|
|||
historicalReplica, err := genSimpleReplica()
|
||||
assert.NoError(t, err)
|
||||
|
||||
fac, err := genFactory()
|
||||
fac := genFactory()
|
||||
assert.NoError(t, err)
|
||||
|
||||
tSafe := newTSafeReplica()
|
||||
|
@ -87,7 +87,7 @@ func TestDataSyncService_DeltaFlowGraphs(t *testing.T) {
|
|||
historicalReplica, err := genSimpleReplica()
|
||||
assert.NoError(t, err)
|
||||
|
||||
fac, err := genFactory()
|
||||
fac := genFactory()
|
||||
assert.NoError(t, err)
|
||||
|
||||
tSafe := newTSafeReplica()
|
||||
|
|
|
@ -34,8 +34,7 @@ func TestQueryNodeFlowGraph_consumerFlowGraph(t *testing.T) {
|
|||
streamingReplica, err := genSimpleReplica()
|
||||
assert.NoError(t, err)
|
||||
|
||||
fac, err := genFactory()
|
||||
assert.NoError(t, err)
|
||||
fac := genFactory()
|
||||
|
||||
fg, err := newQueryNodeFlowGraph(ctx,
|
||||
defaultCollectionID,
|
||||
|
@ -58,8 +57,7 @@ func TestQueryNodeFlowGraph_seekQueryNodeFlowGraph(t *testing.T) {
|
|||
streamingReplica, err := genSimpleReplica()
|
||||
assert.NoError(t, err)
|
||||
|
||||
fac, err := genFactory()
|
||||
assert.NoError(t, err)
|
||||
fac := genFactory()
|
||||
|
||||
tSafe := newTSafeReplica()
|
||||
|
||||
|
|
|
@ -24,8 +24,7 @@ import (
|
|||
"math/rand"
|
||||
"strconv"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -627,33 +626,13 @@ func genEtcdKV() (*etcdkv.EtcdKV, error) {
|
|||
return etcdKV, nil
|
||||
}
|
||||
|
||||
func genFactory() (msgstream.Factory, error) {
|
||||
const receiveBufSize = 1024
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err := msFactory.Init(&Params)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return msFactory, nil
|
||||
}
|
||||
|
||||
func genInvalidFactory() (msgstream.Factory, error) {
|
||||
const receiveBufSize = 1024
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err := msFactory.Init(&Params)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return msFactory, nil
|
||||
func genFactory() dependency.Factory {
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
return factory
|
||||
}
|
||||
|
||||
func genQueryMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
fac, err := genFactory()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
fac := genFactory()
|
||||
stream, err := fac.NewQueryMsgStream(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -699,7 +678,7 @@ func genVectorChunkManager(ctx context.Context) (*storage.VectorChunkManager, er
|
|||
vcm, err := storage.NewVectorChunkManager(lcm, rcm, &etcdpb.CollectionMeta{
|
||||
ID: defaultCollectionID,
|
||||
Schema: schema,
|
||||
}, Params.QueryNodeCfg.LocalFileCacheLimit, false)
|
||||
}, Params.QueryNodeCfg.CacheMemoryLimit, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -1201,7 +1180,7 @@ func genSimpleReplica() (ReplicaInterface, error) {
|
|||
return r, err
|
||||
}
|
||||
|
||||
func genSimpleSegmentLoaderWithMqFactory(ctx context.Context, historicalReplica ReplicaInterface, streamingReplica ReplicaInterface, factory msgstream.Factory) (*segmentLoader, error) {
|
||||
func genSimpleSegmentLoaderWithMqFactory(historicalReplica ReplicaInterface, streamingReplica ReplicaInterface, factory msgstream.Factory) (*segmentLoader, error) {
|
||||
kv, err := genEtcdKV()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -1210,10 +1189,6 @@ func genSimpleSegmentLoaderWithMqFactory(ctx context.Context, historicalReplica
|
|||
return newSegmentLoader(historicalReplica, streamingReplica, kv, cm, factory), nil
|
||||
}
|
||||
|
||||
func genSimpleSegmentLoader(ctx context.Context, historicalReplica ReplicaInterface, streamingReplica ReplicaInterface) (*segmentLoader, error) {
|
||||
return genSimpleSegmentLoaderWithMqFactory(ctx, historicalReplica, streamingReplica, msgstream.NewPmsFactory())
|
||||
}
|
||||
|
||||
func genSimpleHistorical(ctx context.Context, tSafeReplica TSafeReplicaInterface) (*historical, error) {
|
||||
replica, err := genSimpleReplica()
|
||||
if err != nil {
|
||||
|
@ -1249,10 +1224,7 @@ func genSimpleStreaming(ctx context.Context, tSafeReplica TSafeReplicaInterface)
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
fac, err := genFactory()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
fac := genFactory()
|
||||
replica, err := genSimpleReplica()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -1684,7 +1656,7 @@ func saveChangeInfo(key string, value string) error {
|
|||
return kv.Save(key, value)
|
||||
}
|
||||
|
||||
func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac msgstream.Factory) (*QueryNode, error) {
|
||||
func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac dependency.Factory) (*QueryNode, error) {
|
||||
node := NewQueryNode(ctx, fac)
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
if err != nil {
|
||||
|
@ -1707,12 +1679,12 @@ func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac msgstream.Factory)
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, streaming.replica, historical.replica, node.tSafeReplica, node.msFactory)
|
||||
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, streaming.replica, historical.replica, node.tSafeReplica, node.factory)
|
||||
|
||||
node.streaming = streaming
|
||||
node.historical = historical
|
||||
|
||||
loader, err := genSimpleSegmentLoaderWithMqFactory(node.queryNodeLoopCtx, historical.replica, streaming.replica, fac)
|
||||
loader, err := genSimpleSegmentLoaderWithMqFactory(historical.replica, streaming.replica, fac)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -1721,7 +1693,15 @@ func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac msgstream.Factory)
|
|||
// start task scheduler
|
||||
go node.scheduler.Start()
|
||||
|
||||
qs := newQueryService(ctx, node.historical, node.streaming, node.msFactory)
|
||||
vectorStorage, err := node.factory.NewVectorStorageChunkManager(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
cacheStorage, err := node.factory.NewCacheStorageChunkManager(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
qs := newQueryService(ctx, node.historical, node.streaming, vectorStorage, cacheStorage, fac)
|
||||
defer qs.close()
|
||||
node.queryService = qs
|
||||
|
||||
|
@ -1732,10 +1712,7 @@ func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac msgstream.Factory)
|
|||
|
||||
// node
|
||||
func genSimpleQueryNode(ctx context.Context) (*QueryNode, error) {
|
||||
fac, err := genFactory()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
fac := genFactory()
|
||||
return genSimpleQueryNodeWithMQFactory(ctx, fac)
|
||||
}
|
||||
|
||||
|
@ -1855,14 +1832,11 @@ func genFieldData(fieldName string, fieldID int64, fieldType schemapb.DataType,
|
|||
}
|
||||
|
||||
type mockMsgStreamFactory struct {
|
||||
dependency.Factory
|
||||
mockMqStream msgstream.MsgStream
|
||||
}
|
||||
|
||||
var _ msgstream.Factory = &mockMsgStreamFactory{}
|
||||
|
||||
func (mm *mockMsgStreamFactory) Init(params *paramtable.ComponentParam) error {
|
||||
return nil
|
||||
}
|
||||
var _ dependency.Factory = &mockMsgStreamFactory{}
|
||||
|
||||
func (mm *mockMsgStreamFactory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return mm.mockMqStream, nil
|
||||
|
@ -1875,6 +1849,12 @@ func (mm *mockMsgStreamFactory) NewTtMsgStream(ctx context.Context) (msgstream.M
|
|||
func (mm *mockMsgStreamFactory) NewQueryMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return nil, nil
|
||||
}
|
||||
func (mm *mockMsgStreamFactory) NewCacheStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
|
||||
return nil, nil
|
||||
}
|
||||
func (mm *mockMsgStreamFactory) NewVectorStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
type readAtFunc func(path string, offset int64, length int64) ([]byte, error)
|
||||
type readFunc func(path string) ([]byte, error)
|
||||
|
|
|
@ -90,8 +90,6 @@ type queryCollection struct {
|
|||
localChunkManager storage.ChunkManager
|
||||
remoteChunkManager storage.ChunkManager
|
||||
vectorChunkManager *storage.VectorChunkManager
|
||||
localCacheEnabled bool
|
||||
localCacheSize int64
|
||||
|
||||
globalSegmentManager *globalSealedSegmentManager
|
||||
}
|
||||
|
@ -112,7 +110,6 @@ func newQueryCollection(releaseCtx context.Context,
|
|||
factory msgstream.Factory,
|
||||
localChunkManager storage.ChunkManager,
|
||||
remoteChunkManager storage.ChunkManager,
|
||||
localCacheEnabled bool,
|
||||
opts ...qcOpt,
|
||||
) (*queryCollection, error) {
|
||||
|
||||
|
@ -142,8 +139,6 @@ func newQueryCollection(releaseCtx context.Context,
|
|||
|
||||
localChunkManager: localChunkManager,
|
||||
remoteChunkManager: remoteChunkManager,
|
||||
localCacheEnabled: localCacheEnabled,
|
||||
localCacheSize: Params.QueryNodeCfg.LocalFileCacheLimit,
|
||||
globalSegmentManager: newGlobalSealedSegmentManager(collectionID),
|
||||
}
|
||||
|
||||
|
@ -1303,7 +1298,7 @@ func (q *queryCollection) retrieve(msg queryMsg) error {
|
|||
&etcdpb.CollectionMeta{
|
||||
ID: collection.id,
|
||||
Schema: collection.schema,
|
||||
}, q.localCacheSize, q.localCacheEnabled)
|
||||
}, Params.QueryNodeCfg.CacheMemoryLimit, Params.QueryNodeCfg.CacheEnabled)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/segcorepb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
@ -57,10 +58,7 @@ func genSimpleQueryCollection(ctx context.Context, cancel context.CancelFunc) (*
|
|||
return nil, err
|
||||
}
|
||||
|
||||
fac, err := genFactory()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
fac := genFactory()
|
||||
|
||||
localCM, err := genLocalChunkManager()
|
||||
if err != nil {
|
||||
|
@ -79,7 +77,7 @@ func genSimpleQueryCollection(ctx context.Context, cancel context.CancelFunc) (*
|
|||
fac,
|
||||
localCM,
|
||||
remoteCM,
|
||||
false)
|
||||
)
|
||||
return queryCollection, err
|
||||
}
|
||||
|
||||
|
@ -125,9 +123,7 @@ func updateTSafe(queryCollection *queryCollection, timestamp Timestamp) error {
|
|||
|
||||
func TestQueryCollection_withoutVChannel(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
factory := msgstream.NewPmsFactory()
|
||||
err := factory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
defer etcdCli.Close()
|
||||
|
@ -168,7 +164,7 @@ func TestQueryCollection_withoutVChannel(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
queryCollection, err := newQueryCollection(ctx, cancel, 0, historical, streaming, factory, nil, nil, false)
|
||||
queryCollection, err := newQueryCollection(ctx, cancel, 0, historical, streaming, factory, nil, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// producerChannels := []string{"testResultChannel"}
|
||||
|
|
|
@ -45,21 +45,22 @@ import (
|
|||
"unsafe"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"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/typeutil"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// make sure QueryNode implements types.QueryNode
|
||||
|
@ -108,25 +109,26 @@ type QueryNode struct {
|
|||
// etcd client
|
||||
etcdCli *clientv3.Client
|
||||
|
||||
msFactory msgstream.Factory
|
||||
factory dependency.Factory
|
||||
scheduler *taskScheduler
|
||||
|
||||
session *sessionutil.Session
|
||||
eventCh <-chan *sessionutil.SessionEvent
|
||||
sessionManager *SessionManager
|
||||
|
||||
chunkManager storage.ChunkManager
|
||||
etcdKV *etcdkv.EtcdKV
|
||||
vectorStorage storage.ChunkManager
|
||||
cacheStorage storage.ChunkManager
|
||||
etcdKV *etcdkv.EtcdKV
|
||||
}
|
||||
|
||||
// NewQueryNode will return a QueryNode with abnormal state.
|
||||
func NewQueryNode(ctx context.Context, factory msgstream.Factory) *QueryNode {
|
||||
func NewQueryNode(ctx context.Context, factory dependency.Factory) *QueryNode {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
node := &QueryNode{
|
||||
queryNodeLoopCtx: ctx1,
|
||||
queryNodeLoopCancel: cancel,
|
||||
queryService: nil,
|
||||
msFactory: factory,
|
||||
factory: factory,
|
||||
}
|
||||
|
||||
node.scheduler = newTaskScheduler(ctx1)
|
||||
|
@ -272,16 +274,18 @@ func (node *QueryNode) Init() error {
|
|||
return
|
||||
}
|
||||
|
||||
node.chunkManager, err = storage.NewMinioChunkManager(node.queryNodeLoopCtx,
|
||||
storage.Address(Params.MinioCfg.Address),
|
||||
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(Params.MinioCfg.UseSSL),
|
||||
storage.BucketName(Params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
node.factory.Init(&Params)
|
||||
|
||||
node.vectorStorage, err = node.factory.NewVectorStorageChunkManager(node.queryNodeLoopCtx)
|
||||
if err != nil {
|
||||
log.Error("QueryNode init session failed", zap.Error(err))
|
||||
log.Error("QueryNode init vector storage failed", zap.Error(err))
|
||||
initError = err
|
||||
return
|
||||
}
|
||||
|
||||
node.cacheStorage, err = node.factory.NewCacheStorageChunkManager(node.queryNodeLoopCtx)
|
||||
if err != nil {
|
||||
log.Error("QueryNode init cache storage failed", zap.Error(err))
|
||||
initError = err
|
||||
return
|
||||
}
|
||||
|
@ -299,7 +303,7 @@ func (node *QueryNode) Init() error {
|
|||
)
|
||||
node.streaming = newStreaming(node.queryNodeLoopCtx,
|
||||
streamingReplica,
|
||||
node.msFactory,
|
||||
node.factory,
|
||||
node.etcdKV,
|
||||
node.tSafeReplica,
|
||||
)
|
||||
|
@ -308,17 +312,27 @@ func (node *QueryNode) Init() error {
|
|||
node.historical.replica,
|
||||
node.streaming.replica,
|
||||
node.etcdKV,
|
||||
node.chunkManager,
|
||||
node.msFactory)
|
||||
node.vectorStorage,
|
||||
node.factory)
|
||||
|
||||
//node.statsService = newStatsService(node.queryNodeLoopCtx, node.historical.replica, node.msFactory)
|
||||
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, streamingReplica, historicalReplica, node.tSafeReplica, node.msFactory)
|
||||
//node.statsService = newStatsService(node.queryNodeLoopCtx, node.historical.replica, node.factory)
|
||||
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, streamingReplica, historicalReplica, node.tSafeReplica, node.factory)
|
||||
|
||||
node.InitSegcore()
|
||||
|
||||
// TODO: add session creator to node
|
||||
node.sessionManager = NewSessionManager(withSessionCreator(defaultSessionCreator()))
|
||||
|
||||
// init services and manager
|
||||
// TODO: pass node.streaming.replica to search service
|
||||
node.queryService = newQueryService(node.queryNodeLoopCtx,
|
||||
node.historical,
|
||||
node.streaming,
|
||||
node.vectorStorage,
|
||||
node.cacheStorage,
|
||||
node.factory,
|
||||
qsOptWithSessionManager(node.sessionManager))
|
||||
|
||||
log.Debug("query node init successfully",
|
||||
zap.Any("queryNodeID", Params.QueryNodeCfg.QueryNodeID),
|
||||
zap.Any("IP", Params.QueryNodeCfg.QueryNodeIP),
|
||||
|
@ -331,19 +345,6 @@ func (node *QueryNode) Init() error {
|
|||
|
||||
// Start mainly start QueryNode's query service.
|
||||
func (node *QueryNode) Start() error {
|
||||
err := node.msFactory.Init(&Params)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// init services and manager
|
||||
// TODO: pass node.streaming.replica to search service
|
||||
node.queryService = newQueryService(node.queryNodeLoopCtx,
|
||||
node.historical,
|
||||
node.streaming,
|
||||
node.msFactory,
|
||||
qsOptWithSessionManager(node.sessionManager))
|
||||
|
||||
// start task scheduler
|
||||
go node.scheduler.Start()
|
||||
|
||||
|
|
|
@ -28,11 +28,11 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
|
@ -197,39 +197,31 @@ func newQueryNodeMock() *QueryNode {
|
|||
}
|
||||
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
|
||||
msFactory, err := newMessageStreamFactory()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
svr := NewQueryNode(ctx, msFactory)
|
||||
factory := newMessageStreamFactory()
|
||||
svr := NewQueryNode(ctx, factory)
|
||||
tsReplica := newTSafeReplica()
|
||||
streamingReplica := newCollectionReplica(etcdKV)
|
||||
historicalReplica := newCollectionReplica(etcdKV)
|
||||
svr.historical = newHistorical(svr.queryNodeLoopCtx, historicalReplica, tsReplica)
|
||||
svr.streaming = newStreaming(ctx, streamingReplica, msFactory, etcdKV, tsReplica)
|
||||
svr.dataSyncService = newDataSyncService(ctx, svr.streaming.replica, svr.historical.replica, tsReplica, msFactory)
|
||||
svr.statsService = newStatsService(ctx, svr.historical.replica, msFactory)
|
||||
svr.chunkManager = storage.NewLocalChunkManager(storage.RootPath(defaultLocalStorage))
|
||||
svr.loader = newSegmentLoader(svr.historical.replica, svr.streaming.replica, etcdKV, svr.chunkManager, msgstream.NewPmsFactory())
|
||||
svr.streaming = newStreaming(ctx, streamingReplica, factory, etcdKV, tsReplica)
|
||||
svr.dataSyncService = newDataSyncService(ctx, svr.streaming.replica, svr.historical.replica, tsReplica, factory)
|
||||
svr.statsService = newStatsService(ctx, svr.historical.replica, factory)
|
||||
svr.vectorStorage, err = factory.NewVectorStorageChunkManager(ctx)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
svr.cacheStorage, err = factory.NewCacheStorageChunkManager(ctx)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
svr.loader = newSegmentLoader(svr.historical.replica, svr.streaming.replica, etcdKV, svr.vectorStorage, factory)
|
||||
svr.etcdKV = etcdKV
|
||||
|
||||
return svr
|
||||
}
|
||||
|
||||
func makeNewChannelNames(names []string, suffix string) []string {
|
||||
var ret []string
|
||||
for _, name := range names {
|
||||
ret = append(ret, name+suffix)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func newMessageStreamFactory() (msgstream.Factory, error) {
|
||||
const receiveBufSize = 1024
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err := msFactory.Init(&Params)
|
||||
return msFactory, err
|
||||
func newMessageStreamFactory() dependency.Factory {
|
||||
return dependency.NewDefaultFactory(true)
|
||||
}
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
|
|
|
@ -21,7 +21,6 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
@ -45,9 +44,8 @@ type queryService struct {
|
|||
|
||||
sessionManager *SessionManager
|
||||
|
||||
localChunkManager storage.ChunkManager
|
||||
remoteChunkManager storage.ChunkManager
|
||||
localCacheEnabled bool
|
||||
cacheStorage storage.ChunkManager
|
||||
vectorStorage storage.ChunkManager
|
||||
}
|
||||
|
||||
type qsOpt func(*queryService)
|
||||
|
@ -61,31 +59,14 @@ func qsOptWithSessionManager(s *SessionManager) qsOpt {
|
|||
func newQueryService(ctx context.Context,
|
||||
historical *historical,
|
||||
streaming *streaming,
|
||||
vectorStorage storage.ChunkManager,
|
||||
cacheStorage storage.ChunkManager,
|
||||
factory msgstream.Factory,
|
||||
opts ...qsOpt,
|
||||
) *queryService {
|
||||
|
||||
queryServiceCtx, queryServiceCancel := context.WithCancel(ctx)
|
||||
|
||||
//TODO godchen: change this to configuration
|
||||
path, err := Params.Load("localStorage.Path")
|
||||
if err != nil {
|
||||
path = "/tmp/milvus/data"
|
||||
}
|
||||
enabled, _ := Params.Load("localStorage.enabled")
|
||||
localCacheEnabled, _ := strconv.ParseBool(enabled)
|
||||
|
||||
localChunkManager := storage.NewLocalChunkManager(storage.RootPath(path))
|
||||
|
||||
remoteChunkManager, err := storage.NewMinioChunkManager(
|
||||
ctx,
|
||||
storage.Address(Params.MinioCfg.Address),
|
||||
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(Params.MinioCfg.UseSSL),
|
||||
storage.BucketName(Params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
|
||||
qs := &queryService{
|
||||
ctx: queryServiceCtx,
|
||||
cancel: queryServiceCancel,
|
||||
|
@ -95,11 +76,9 @@ func newQueryService(ctx context.Context,
|
|||
|
||||
queryCollections: make(map[UniqueID]*queryCollection),
|
||||
|
||||
factory: factory,
|
||||
|
||||
localChunkManager: localChunkManager,
|
||||
remoteChunkManager: remoteChunkManager,
|
||||
localCacheEnabled: localCacheEnabled,
|
||||
vectorStorage: vectorStorage,
|
||||
cacheStorage: cacheStorage,
|
||||
factory: factory,
|
||||
}
|
||||
|
||||
for _, opt := range opts {
|
||||
|
@ -137,9 +116,8 @@ func (q *queryService) addQueryCollection(collectionID UniqueID) error {
|
|||
q.historical,
|
||||
q.streaming,
|
||||
q.factory,
|
||||
q.localChunkManager,
|
||||
q.remoteChunkManager,
|
||||
q.localCacheEnabled,
|
||||
q.cacheStorage,
|
||||
q.vectorStorage,
|
||||
qcOptWithSessionManager(q.sessionManager),
|
||||
)
|
||||
if err != nil {
|
||||
|
|
|
@ -63,13 +63,10 @@ func loadFields(segment *Segment, DIM int, N int) error {
|
|||
|
||||
func sendSearchRequest(ctx context.Context, DIM int) error {
|
||||
// init message stream
|
||||
msFactory, err := newMessageStreamFactory()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
factory := newMessageStreamFactory()
|
||||
searchProducerChannels := []string{"test-query"}
|
||||
|
||||
searchStream, _ := msFactory.NewMsgStream(ctx)
|
||||
searchStream, _ := factory.NewMsgStream(ctx)
|
||||
searchStream.AsProducer(searchProducerChannels)
|
||||
searchStream.Start()
|
||||
|
||||
|
@ -142,17 +139,18 @@ func TestSearch_Search(t *testing.T) {
|
|||
node := newQueryNodeMock()
|
||||
initTestMeta(t, node, collectionID, UniqueID(0))
|
||||
|
||||
msFactory, err := newMessageStreamFactory()
|
||||
assert.NoError(t, err)
|
||||
|
||||
// start search service
|
||||
fac := genFactory()
|
||||
node.queryService = newQueryService(node.queryNodeLoopCtx,
|
||||
node.historical,
|
||||
node.streaming,
|
||||
msFactory)
|
||||
node.vectorStorage,
|
||||
node.cacheStorage,
|
||||
fac,
|
||||
)
|
||||
|
||||
// load segment
|
||||
err = node.historical.replica.addSegment(segmentID, defaultPartitionID, collectionID, "", segmentTypeSealed, true)
|
||||
err := node.historical.replica.addSegment(segmentID, defaultPartitionID, collectionID, "", segmentTypeSealed, true)
|
||||
assert.NoError(t, err)
|
||||
segment, err := node.historical.replica.getSegmentByID(segmentID)
|
||||
assert.NoError(t, err)
|
||||
|
@ -185,21 +183,22 @@ func TestSearch_SearchMultiSegments(t *testing.T) {
|
|||
node := newQueryNodeMock()
|
||||
initTestMeta(t, node, collectionID, UniqueID(0))
|
||||
|
||||
msFactory, err := newMessageStreamFactory()
|
||||
assert.NoError(t, err)
|
||||
|
||||
// start search service
|
||||
fac := genFactory()
|
||||
node.queryService = newQueryService(node.queryNodeLoopCtx,
|
||||
node.historical,
|
||||
node.streaming,
|
||||
msFactory)
|
||||
node.vectorStorage,
|
||||
node.cacheStorage,
|
||||
fac,
|
||||
)
|
||||
node.queryService.addQueryCollection(collectionID)
|
||||
//err = node.queryService.addQueryCollection(collectionID)
|
||||
//TODO: Why error
|
||||
//assert.Error(t, err)
|
||||
|
||||
// load segments
|
||||
err = node.historical.replica.addSegment(segmentID1, defaultPartitionID, collectionID, "", segmentTypeSealed, true)
|
||||
err := node.historical.replica.addSegment(segmentID1, defaultPartitionID, collectionID, "", segmentTypeSealed, true)
|
||||
assert.NoError(t, err)
|
||||
segment1, err := node.historical.replica.getSegmentByID(segmentID1)
|
||||
assert.NoError(t, err)
|
||||
|
@ -233,11 +232,14 @@ func TestQueryService_addQueryCollection(t *testing.T) {
|
|||
str, err := genSimpleStreaming(ctx, tSafe)
|
||||
assert.NoError(t, err)
|
||||
|
||||
fac, err := genFactory()
|
||||
assert.NoError(t, err)
|
||||
fac := genFactory()
|
||||
|
||||
vectorStorage, err := fac.NewVectorStorageChunkManager(ctx)
|
||||
assert.NoError(t, err)
|
||||
cacheStorage, err := fac.NewCacheStorageChunkManager(ctx)
|
||||
assert.NoError(t, err)
|
||||
// start search service
|
||||
qs := newQueryService(ctx, his, str, fac)
|
||||
qs := newQueryService(ctx, his, str, vectorStorage, cacheStorage, fac)
|
||||
assert.NotNil(t, qs)
|
||||
|
||||
err = qs.addQueryCollection(defaultCollectionID)
|
||||
|
|
|
@ -21,6 +21,8 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
|
@ -29,9 +31,8 @@ func TestStatsService_start(t *testing.T) {
|
|||
node := newQueryNodeMock()
|
||||
initTestMeta(t, node, 0, 0)
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
msFactory.Init(&Params)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.historical.replica, msFactory)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.historical.replica, factory)
|
||||
node.statsService.start()
|
||||
node.Stop()
|
||||
}
|
||||
|
@ -48,17 +49,15 @@ func TestSegmentManagement_sendSegmentStatistic(t *testing.T) {
|
|||
// start pulsar
|
||||
producerChannels := []string{Params.CommonCfg.QueryNodeStats}
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
statsStream, err := msFactory.NewMsgStream(node.queryNodeLoopCtx)
|
||||
statsStream, err := factory.NewMsgStream(node.queryNodeLoopCtx)
|
||||
assert.Nil(t, err)
|
||||
statsStream.AsProducer(producerChannels)
|
||||
|
||||
var statsMsgStream msgstream.MsgStream = statsStream
|
||||
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.historical.replica, msFactory)
|
||||
node.statsService = newStatsService(node.queryNodeLoopCtx, node.historical.replica, factory)
|
||||
node.statsService.statsStream = statsMsgStream
|
||||
node.statsService.statsStream.Start()
|
||||
|
||||
|
|
|
@ -176,7 +176,7 @@ func TestTask_AddQueryChannel(t *testing.T) {
|
|||
|
||||
position := &internalpb.MsgPosition{
|
||||
ChannelName: genQueryChannel(),
|
||||
MsgID: []byte{1, 2, 3},
|
||||
MsgID: []byte{1, 2, 3, 4, 5, 6, 7, 8},
|
||||
MsgGroup: defaultSubName,
|
||||
Timestamp: 0,
|
||||
}
|
||||
|
@ -336,7 +336,7 @@ func TestTask_watchDmChannelsTask(t *testing.T) {
|
|||
SeekPosition: &msgstream.MsgPosition{
|
||||
ChannelName: tmpChannel,
|
||||
Timestamp: 0,
|
||||
MsgID: []byte{1, 2, 3},
|
||||
MsgID: []byte{1, 2, 3, 4, 5, 6, 7, 8},
|
||||
},
|
||||
FlushedSegments: []*datapb.SegmentInfo{
|
||||
{
|
||||
|
@ -368,7 +368,7 @@ func TestTask_watchDmChannelsTask(t *testing.T) {
|
|||
SeekPosition: &msgstream.MsgPosition{
|
||||
ChannelName: tmpChannel,
|
||||
Timestamp: 0,
|
||||
MsgID: []byte{1, 2, 3},
|
||||
MsgID: []byte{1, 2, 3, 4, 5, 6, 7, 8},
|
||||
},
|
||||
DroppedSegments: []*datapb.SegmentInfo{
|
||||
{
|
||||
|
@ -494,7 +494,7 @@ func TestTask_watchDeltaChannelsTask(t *testing.T) {
|
|||
ChannelName: defaultDeltaChannel,
|
||||
SeekPosition: &internalpb.MsgPosition{
|
||||
ChannelName: defaultDeltaChannel,
|
||||
MsgID: []byte{1, 2, 3},
|
||||
MsgID: []byte{1, 2, 3, 4, 5, 6, 7, 8},
|
||||
MsgGroup: defaultSubName,
|
||||
Timestamp: 0,
|
||||
},
|
||||
|
|
|
@ -24,6 +24,7 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
@ -38,12 +39,9 @@ func TestDmlChannels(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
factory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
Params.Init()
|
||||
|
||||
err := factory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
|
||||
dml := newDmlChannels(ctx, factory, dmlChanPrefix, totalDmlChannelNum)
|
||||
chanNames := dml.listChannels()
|
||||
assert.Equal(t, 0, len(chanNames))
|
||||
|
|
|
@ -29,7 +29,12 @@ import (
|
|||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
|
@ -57,8 +62,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// UniqueID is an alias of typeutil.UniqueID.
|
||||
|
@ -178,7 +181,7 @@ type Core struct {
|
|||
|
||||
session *sessionutil.Session
|
||||
|
||||
msFactory ms.Factory
|
||||
factory dependency.Factory
|
||||
|
||||
//import manager
|
||||
importManager *importManager
|
||||
|
@ -187,14 +190,14 @@ type Core struct {
|
|||
// --------------------- function --------------------------
|
||||
|
||||
// NewCore creates a new rootcoord core
|
||||
func NewCore(c context.Context, factory ms.Factory) (*Core, error) {
|
||||
func NewCore(c context.Context, factory dependency.Factory) (*Core, error) {
|
||||
ctx, cancel := context.WithCancel(c)
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
core := &Core{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
ddlLock: sync.Mutex{},
|
||||
msFactory: factory,
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
ddlLock: sync.Mutex{},
|
||||
factory: factory,
|
||||
}
|
||||
core.UpdateStateCode(internalpb.StateCode_Abnormal)
|
||||
return core, nil
|
||||
|
@ -475,7 +478,7 @@ func (c *Core) setMsgStreams() error {
|
|||
if Params.CommonCfg.RootCoordTimeTick == "" {
|
||||
return fmt.Errorf("timeTickChannel is empty")
|
||||
}
|
||||
timeTickStream, _ := c.msFactory.NewMsgStream(c.ctx)
|
||||
timeTickStream, _ := c.factory.NewMsgStream(c.ctx)
|
||||
metrics.RootCoordNumOfMsgStream.Inc()
|
||||
timeTickStream.AsProducer([]string{Params.CommonCfg.RootCoordTimeTick})
|
||||
log.Debug("RootCoord register timetick producer success", zap.String("channel name", Params.CommonCfg.RootCoordTimeTick))
|
||||
|
@ -1126,12 +1129,10 @@ func (c *Core) Init() error {
|
|||
return tsoAllocator.GetLastSavedTime()
|
||||
}
|
||||
|
||||
if initError = c.msFactory.Init(&Params); initError != nil {
|
||||
return
|
||||
}
|
||||
c.factory.Init(&Params)
|
||||
|
||||
chanMap := c.MetaTable.ListCollectionPhysicalChannels()
|
||||
c.chanTimeTick = newTimeTickSync(c.ctx, c.session.ServerID, c.msFactory, chanMap)
|
||||
c.chanTimeTick = newTimeTickSync(c.ctx, c.session.ServerID, c.factory, chanMap)
|
||||
c.chanTimeTick.addSession(c.session)
|
||||
c.proxyClientManager = newProxyClientManager(c)
|
||||
|
||||
|
|
|
@ -28,6 +28,10 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
|
@ -45,15 +49,13 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -504,7 +506,7 @@ func TestRootCoordInit(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
coreFactory := msgstream.NewPmsFactory()
|
||||
coreFactory := dependency.NewDefaultFactory(true)
|
||||
Params.Init()
|
||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||
|
||||
|
@ -637,7 +639,7 @@ func TestRootCoord_Base(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
coreFactory := msgstream.NewPmsFactory()
|
||||
coreFactory := dependency.NewDefaultFactory(true)
|
||||
Params.Init()
|
||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||
Params.RootCoordCfg.ImportIndexCheckInterval = 0.1
|
||||
|
@ -702,10 +704,7 @@ func TestRootCoord_Base(t *testing.T) {
|
|||
err = core.SetQueryCoord(qm)
|
||||
assert.NoError(t, err)
|
||||
|
||||
tmpFactory := msgstream.NewPmsFactory()
|
||||
|
||||
err = tmpFactory.Init(&Params)
|
||||
assert.NoError(t, err)
|
||||
tmpFactory := dependency.NewDefaultFactory(true)
|
||||
|
||||
timeTickStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||
timeTickStream.AsConsumer([]string{Params.CommonCfg.RootCoordTimeTick}, Params.CommonCfg.RootCoordSubName)
|
||||
|
@ -2611,7 +2610,8 @@ func TestRootCoord2(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
msFactory := dependency.NewDefaultFactory(true)
|
||||
|
||||
Params.Init()
|
||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||
core, err := NewCore(ctx, msFactory)
|
||||
|
@ -2665,9 +2665,6 @@ func TestRootCoord2(t *testing.T) {
|
|||
err = core.Register()
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = msFactory.Init(&Params)
|
||||
assert.NoError(t, err)
|
||||
|
||||
timeTickStream, _ := msFactory.NewMsgStream(ctx)
|
||||
timeTickStream.AsConsumer([]string{Params.CommonCfg.RootCoordTimeTick}, Params.CommonCfg.RootCoordSubName)
|
||||
timeTickStream.Start()
|
||||
|
@ -2910,7 +2907,7 @@ func TestCheckFlushedSegments(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
msFactory := dependency.NewDefaultFactory(true)
|
||||
Params.Init()
|
||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||
core, err := NewCore(ctx, msFactory)
|
||||
|
@ -2962,9 +2959,6 @@ func TestCheckFlushedSegments(t *testing.T) {
|
|||
err = core.Register()
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = msFactory.Init(&Params)
|
||||
assert.NoError(t, err)
|
||||
|
||||
timeTickStream, _ := msFactory.NewMsgStream(ctx)
|
||||
timeTickStream.AsConsumer([]string{Params.CommonCfg.RootCoordTimeTick}, Params.CommonCfg.RootCoordSubName)
|
||||
timeTickStream.Start()
|
||||
|
@ -3072,7 +3066,7 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
msFactory := dependency.NewDefaultFactory(true)
|
||||
Params.Init()
|
||||
Params.RootCoordCfg.DmlChannelNum = TestDMLChannelNum
|
||||
|
||||
|
@ -3125,9 +3119,6 @@ func TestRootCoord_CheckZeroShardsNum(t *testing.T) {
|
|||
err = core.Register()
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = msFactory.Init(&Params)
|
||||
assert.NoError(t, err)
|
||||
|
||||
timeTickStream, _ := msFactory.NewMsgStream(ctx)
|
||||
timeTickStream.AsConsumer([]string{Params.CommonCfg.RootCoordTimeTick}, Params.CommonCfg.RootCoordSubName)
|
||||
timeTickStream.Start()
|
||||
|
|
|
@ -23,7 +23,8 @@ import (
|
|||
"sync"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
|
@ -31,8 +32,8 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
type tbd struct {
|
||||
|
@ -80,9 +81,9 @@ func BenchmarkAllocTimestamp(b *testing.B) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
Params.Init()
|
||||
core, err := NewCore(ctx, msFactory)
|
||||
core, err := NewCore(ctx, factory)
|
||||
|
||||
assert.Nil(b, err)
|
||||
|
||||
|
@ -120,9 +121,6 @@ func BenchmarkAllocTimestamp(b *testing.B) {
|
|||
err = core.Start()
|
||||
assert.Nil(b, err)
|
||||
|
||||
err = msFactory.Init(&Params)
|
||||
assert.Nil(b, err)
|
||||
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
req := rootcoordpb.AllocTimestampRequest{
|
||||
|
|
|
@ -23,18 +23,16 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
)
|
||||
|
||||
func TestTimetickSync(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
sourceID := int64(100)
|
||||
|
||||
factory := msgstream.NewPmsFactory()
|
||||
err := factory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
//chanMap := map[typeutil.UniqueID][]string{
|
||||
// int64(1): {"rootcoord-dml_0"},
|
||||
|
|
|
@ -20,8 +20,8 @@ import (
|
|||
"errors"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -98,8 +98,8 @@ func (itr *InsertBinlogIterator) Next() (interface{}, error) {
|
|||
}
|
||||
|
||||
v := &Value{
|
||||
ID: itr.data.Data[rootcoord.RowIDField].GetRow(itr.pos).(int64),
|
||||
Timestamp: itr.data.Data[rootcoord.TimeStampField].GetRow(itr.pos).(int64),
|
||||
ID: itr.data.Data[common.RowIDField].GetRow(itr.pos).(int64),
|
||||
Timestamp: itr.data.Data[common.TimeStampField].GetRow(itr.pos).(int64),
|
||||
PK: pk,
|
||||
IsDeleted: false,
|
||||
Value: m,
|
||||
|
@ -114,11 +114,11 @@ func (itr *InsertBinlogIterator) Dispose() {
|
|||
}
|
||||
|
||||
func (itr *InsertBinlogIterator) hasNext() bool {
|
||||
_, ok := itr.data.Data[rootcoord.RowIDField]
|
||||
_, ok := itr.data.Data[common.RowIDField]
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return itr.pos < itr.data.Data[rootcoord.RowIDField].RowNum()
|
||||
return itr.pos < itr.data.Data[common.RowIDField].RowNum()
|
||||
}
|
||||
|
||||
func (itr *InsertBinlogIterator) isDisposed() bool {
|
||||
|
|
|
@ -19,17 +19,17 @@ package storage
|
|||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func generateTestData(t *testing.T, num int) []*Blob {
|
||||
schema := &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||
{FieldID: rootcoord.TimeStampField, Name: "ts", DataType: schemapb.DataType_Int64},
|
||||
{FieldID: rootcoord.RowIDField, Name: "rowid", DataType: schemapb.DataType_Int64},
|
||||
{FieldID: common.TimeStampField, Name: "ts", DataType: schemapb.DataType_Int64},
|
||||
{FieldID: common.RowIDField, Name: "rowid", DataType: schemapb.DataType_Int64},
|
||||
{FieldID: 101, Name: "int32", DataType: schemapb.DataType_Int32},
|
||||
{FieldID: 102, Name: "floatVector", DataType: schemapb.DataType_FloatVector},
|
||||
{FieldID: 103, Name: "binaryVector", DataType: schemapb.DataType_BinaryVector},
|
||||
|
@ -59,9 +59,9 @@ func generateTestData(t *testing.T, num int) []*Blob {
|
|||
}
|
||||
|
||||
data := &InsertData{Data: map[FieldID]FieldData{
|
||||
rootcoord.RowIDField: &Int64FieldData{Data: field0},
|
||||
rootcoord.TimeStampField: &Int64FieldData{Data: field1},
|
||||
101: &Int32FieldData{Data: field101},
|
||||
common.RowIDField: &Int64FieldData{Data: field0},
|
||||
common.TimeStampField: &Int64FieldData{Data: field1},
|
||||
101: &Int32FieldData{Data: field101},
|
||||
102: &FloatVectorFieldData{
|
||||
NumRows: []int64{int64(num)},
|
||||
Data: field102,
|
||||
|
@ -91,7 +91,7 @@ func TestInsertlogIterator(t *testing.T) {
|
|||
|
||||
t.Run("test dispose", func(t *testing.T) {
|
||||
blobs := generateTestData(t, 1)
|
||||
itr, err := NewInsertBinlogIterator(blobs, rootcoord.RowIDField, schemapb.DataType_Int64)
|
||||
itr, err := NewInsertBinlogIterator(blobs, common.RowIDField, schemapb.DataType_Int64)
|
||||
assert.Nil(t, err)
|
||||
|
||||
itr.Dispose()
|
||||
|
@ -102,7 +102,7 @@ func TestInsertlogIterator(t *testing.T) {
|
|||
|
||||
t.Run("not empty iterator", func(t *testing.T) {
|
||||
blobs := generateTestData(t, 3)
|
||||
itr, err := NewInsertBinlogIterator(blobs, rootcoord.RowIDField, schemapb.DataType_Int64)
|
||||
itr, err := NewInsertBinlogIterator(blobs, common.RowIDField, schemapb.DataType_Int64)
|
||||
assert.Nil(t, err)
|
||||
|
||||
for i := 1; i <= 3; i++ {
|
||||
|
@ -125,11 +125,11 @@ func TestInsertlogIterator(t *testing.T) {
|
|||
int64(i),
|
||||
false,
|
||||
map[FieldID]interface{}{
|
||||
rootcoord.TimeStampField: int64(i),
|
||||
rootcoord.RowIDField: int64(i),
|
||||
101: int32(i),
|
||||
102: f102,
|
||||
103: []byte{byte(i)},
|
||||
common.TimeStampField: int64(i),
|
||||
common.RowIDField: int64(i),
|
||||
101: int32(i),
|
||||
102: f102,
|
||||
103: []byte{byte(i)},
|
||||
},
|
||||
}
|
||||
assert.EqualValues(t, expected, value)
|
||||
|
@ -157,7 +157,7 @@ func TestMergeIterator(t *testing.T) {
|
|||
|
||||
t.Run("empty and non-empty iterators", func(t *testing.T) {
|
||||
blobs := generateTestData(t, 3)
|
||||
insertItr, err := NewInsertBinlogIterator(blobs, rootcoord.RowIDField, schemapb.DataType_Int64)
|
||||
insertItr, err := NewInsertBinlogIterator(blobs, common.RowIDField, schemapb.DataType_Int64)
|
||||
assert.Nil(t, err)
|
||||
iterators := []Iterator{
|
||||
&InsertBinlogIterator{data: &InsertData{}},
|
||||
|
@ -185,11 +185,11 @@ func TestMergeIterator(t *testing.T) {
|
|||
int64(i),
|
||||
false,
|
||||
map[FieldID]interface{}{
|
||||
rootcoord.TimeStampField: int64(i),
|
||||
rootcoord.RowIDField: int64(i),
|
||||
101: int32(i),
|
||||
102: f102,
|
||||
103: []byte{byte(i)},
|
||||
common.TimeStampField: int64(i),
|
||||
common.RowIDField: int64(i),
|
||||
101: int32(i),
|
||||
102: f102,
|
||||
103: []byte{byte(i)},
|
||||
},
|
||||
}
|
||||
assert.EqualValues(t, expected, value)
|
||||
|
@ -201,9 +201,9 @@ func TestMergeIterator(t *testing.T) {
|
|||
|
||||
t.Run("non-empty iterators", func(t *testing.T) {
|
||||
blobs := generateTestData(t, 3)
|
||||
itr1, err := NewInsertBinlogIterator(blobs, rootcoord.RowIDField, schemapb.DataType_Int64)
|
||||
itr1, err := NewInsertBinlogIterator(blobs, common.RowIDField, schemapb.DataType_Int64)
|
||||
assert.Nil(t, err)
|
||||
itr2, err := NewInsertBinlogIterator(blobs, rootcoord.RowIDField, schemapb.DataType_Int64)
|
||||
itr2, err := NewInsertBinlogIterator(blobs, common.RowIDField, schemapb.DataType_Int64)
|
||||
assert.Nil(t, err)
|
||||
iterators := []Iterator{itr1, itr2}
|
||||
itr := NewMergeIterator(iterators)
|
||||
|
@ -223,11 +223,11 @@ func TestMergeIterator(t *testing.T) {
|
|||
int64(i),
|
||||
false,
|
||||
map[FieldID]interface{}{
|
||||
rootcoord.TimeStampField: int64(i),
|
||||
rootcoord.RowIDField: int64(i),
|
||||
101: int32(i),
|
||||
102: f102,
|
||||
103: []byte{byte(i)},
|
||||
common.TimeStampField: int64(i),
|
||||
common.RowIDField: int64(i),
|
||||
101: int32(i),
|
||||
102: f102,
|
||||
103: []byte{byte(i)},
|
||||
},
|
||||
}
|
||||
for j := 0; j < 2; j++ {
|
||||
|
@ -246,7 +246,7 @@ func TestMergeIterator(t *testing.T) {
|
|||
|
||||
t.Run("test dispose", func(t *testing.T) {
|
||||
blobs := generateTestData(t, 3)
|
||||
itr1, err := NewInsertBinlogIterator(blobs, rootcoord.RowIDField, schemapb.DataType_Int64)
|
||||
itr1, err := NewInsertBinlogIterator(blobs, common.RowIDField, schemapb.DataType_Int64)
|
||||
assert.Nil(t, err)
|
||||
itr := NewMergeIterator([]Iterator{itr1})
|
||||
|
||||
|
|
|
@ -27,12 +27,12 @@ import (
|
|||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
)
|
||||
|
||||
|
@ -274,7 +274,7 @@ func (insertCodec *InsertCodec) Serialize(partitionID UniqueID, segmentID Unique
|
|||
blobs := make([]*Blob, 0)
|
||||
statsBlobs := make([]*Blob, 0)
|
||||
var writer *InsertBinlogWriter
|
||||
timeFieldData, ok := data.Data[rootcoord.TimeStampField]
|
||||
timeFieldData, ok := data.Data[common.TimeStampField]
|
||||
if !ok {
|
||||
return nil, nil, fmt.Errorf("data doesn't contains timestamp field")
|
||||
}
|
||||
|
@ -646,7 +646,7 @@ func (insertCodec *InsertCodec) DeserializeAll(blobs []*Blob) (
|
|||
}
|
||||
eventReader.Close()
|
||||
}
|
||||
if fieldID == rootcoord.TimeStampField {
|
||||
if fieldID == common.TimeStampField {
|
||||
blobInfo := BlobInfo{
|
||||
Length: totalLength,
|
||||
}
|
||||
|
|
|
@ -17,8 +17,8 @@
|
|||
package storage
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
)
|
||||
|
||||
// DataSorter sorts insert data
|
||||
|
@ -29,7 +29,7 @@ type DataSorter struct {
|
|||
|
||||
// getRowIDFieldData returns auto generated row id Field
|
||||
func (ds *DataSorter) getRowIDFieldData() FieldData {
|
||||
if data, ok := ds.InsertData.Data[rootcoord.RowIDField]; ok {
|
||||
if data, ok := ds.InsertData.Data[common.RowIDField]; ok {
|
||||
return data
|
||||
}
|
||||
return nil
|
||||
|
|
|
@ -0,0 +1,48 @@
|
|||
package storage
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
)
|
||||
|
||||
type ChunkManagerFactory struct {
|
||||
cacheStorage string
|
||||
vectorStorage string
|
||||
config *config
|
||||
}
|
||||
|
||||
func NewChunkManagerFactory(cacheStorage, vectorStorage string, opts ...Option) *ChunkManagerFactory {
|
||||
c := newDefaultConfig()
|
||||
for _, opt := range opts {
|
||||
opt(c)
|
||||
}
|
||||
return &ChunkManagerFactory{
|
||||
cacheStorage: cacheStorage,
|
||||
vectorStorage: vectorStorage,
|
||||
config: c,
|
||||
}
|
||||
}
|
||||
|
||||
func (f *ChunkManagerFactory) newChunkManager(ctx context.Context, engine string) (ChunkManager, error) {
|
||||
switch engine {
|
||||
case "local":
|
||||
return NewLocalChunkManager(RootPath(f.config.rootPath)), nil
|
||||
case "minio":
|
||||
return newMinioChunkManagerWithConfig(ctx, f.config)
|
||||
default:
|
||||
return nil, errors.New("no chunk manager implemented with engine: " + engine)
|
||||
}
|
||||
}
|
||||
|
||||
func (f *ChunkManagerFactory) NewCacheStorageChunkManager(ctx context.Context) (ChunkManager, error) {
|
||||
return f.newChunkManager(ctx, f.cacheStorage)
|
||||
}
|
||||
|
||||
func (f *ChunkManagerFactory) NewVectorStorageChunkManager(ctx context.Context) (ChunkManager, error) {
|
||||
return f.newChunkManager(ctx, f.vectorStorage)
|
||||
}
|
||||
|
||||
type Factory interface {
|
||||
NewCacheStorageChunkManager(ctx context.Context) (ChunkManager, error)
|
||||
NewVectorStorageChunkManager(ctx context.Context) (ChunkManager, error)
|
||||
}
|
|
@ -25,8 +25,9 @@ import (
|
|||
"path/filepath"
|
||||
"strings"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/errorutil"
|
||||
"golang.org/x/exp/mmap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/errorutil"
|
||||
)
|
||||
|
||||
// LocalChunkManager is responsible for read and write local file.
|
||||
|
|
|
@ -50,6 +50,10 @@ func NewMinioChunkManager(ctx context.Context, opts ...Option) (*MinioChunkManag
|
|||
for _, opt := range opts {
|
||||
opt(c)
|
||||
}
|
||||
|
||||
return newMinioChunkManagerWithConfig(ctx, c)
|
||||
}
|
||||
func newMinioChunkManagerWithConfig(ctx context.Context, c *config) (*MinioChunkManager, error) {
|
||||
minIOClient, err := minio.New(c.address, &minio.Options{
|
||||
Creds: credentials.NewStaticV4(c.accessKeyID, c.secretAccessKeyID, ""),
|
||||
Secure: c.useSSL,
|
||||
|
|
|
@ -25,7 +25,6 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
)
|
||||
|
||||
func TestStatsWriter_Int64PrimaryKey(t *testing.T) {
|
||||
|
@ -58,7 +57,7 @@ func TestStatsWriter_Int64PrimaryKey(t *testing.T) {
|
|||
msgs := &Int64FieldData{
|
||||
Data: []int64{},
|
||||
}
|
||||
err = sw.generatePrimaryKeyStats(rootcoord.RowIDField, schemapb.DataType_Int64, msgs)
|
||||
err = sw.generatePrimaryKeyStats(common.RowIDField, schemapb.DataType_Int64, msgs)
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
|
@ -86,7 +85,7 @@ func TestStatsWriter_VarCharPrimaryKey(t *testing.T) {
|
|||
msgs := &Int64FieldData{
|
||||
Data: []int64{},
|
||||
}
|
||||
err = sw.generatePrimaryKeyStats(rootcoord.RowIDField, schemapb.DataType_Int64, msgs)
|
||||
err = sw.generatePrimaryKeyStats(common.RowIDField, schemapb.DataType_Int64, msgs)
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,71 @@
|
|||
package dependency
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
)
|
||||
|
||||
type DefaultFactory struct {
|
||||
standAlone bool
|
||||
chunkManagerFactory storage.Factory
|
||||
msgStreamFactory msgstream.Factory
|
||||
}
|
||||
|
||||
func NewDefaultFactory(standAlone bool) *DefaultFactory {
|
||||
return &DefaultFactory{
|
||||
standAlone: standAlone,
|
||||
msgStreamFactory: msgstream.NewRmsFactory("/tmp/milvus/rocksmq/"),
|
||||
chunkManagerFactory: storage.NewChunkManagerFactory("local", "local",
|
||||
storage.RootPath("/tmp/milvus")),
|
||||
}
|
||||
}
|
||||
|
||||
func (f *DefaultFactory) Init(params *paramtable.ComponentParam) {
|
||||
if f.standAlone {
|
||||
path, _ := params.Load("_RocksmqPath")
|
||||
f.msgStreamFactory = msgstream.NewRmsFactory(path)
|
||||
f.chunkManagerFactory = storage.NewChunkManagerFactory("local", "local",
|
||||
storage.RootPath(params.LocalStorageCfg.Path))
|
||||
} else {
|
||||
f.msgStreamFactory = msgstream.NewPmsFactory(¶ms.PulsarCfg)
|
||||
f.chunkManagerFactory = storage.NewChunkManagerFactory("local", "minio",
|
||||
storage.RootPath(params.LocalStorageCfg.Path),
|
||||
storage.Address(params.MinioCfg.Address),
|
||||
storage.AccessKeyID(params.MinioCfg.AccessKeyID),
|
||||
storage.SecretAccessKeyID(params.MinioCfg.SecretAccessKey),
|
||||
storage.UseSSL(params.MinioCfg.UseSSL),
|
||||
storage.BucketName(params.MinioCfg.BucketName),
|
||||
storage.CreateBucket(true))
|
||||
}
|
||||
}
|
||||
|
||||
func (f *DefaultFactory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return f.msgStreamFactory.NewMsgStream(ctx)
|
||||
|
||||
}
|
||||
func (f *DefaultFactory) NewTtMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return f.msgStreamFactory.NewTtMsgStream(ctx)
|
||||
}
|
||||
func (f *DefaultFactory) NewQueryMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return f.msgStreamFactory.NewQueryMsgStream(ctx)
|
||||
}
|
||||
|
||||
func (f *DefaultFactory) NewCacheStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
|
||||
return f.chunkManagerFactory.NewCacheStorageChunkManager(ctx)
|
||||
}
|
||||
|
||||
func (f *DefaultFactory) NewVectorStorageChunkManager(ctx context.Context) (storage.ChunkManager, error) {
|
||||
return f.chunkManagerFactory.NewVectorStorageChunkManager(ctx)
|
||||
}
|
||||
|
||||
type Factory interface {
|
||||
Init(p *paramtable.ComponentParam)
|
||||
NewMsgStream(ctx context.Context) (msgstream.MsgStream, error)
|
||||
NewTtMsgStream(ctx context.Context) (msgstream.MsgStream, error)
|
||||
NewQueryMsgStream(ctx context.Context) (msgstream.MsgStream, error)
|
||||
NewCacheStorageChunkManager(ctx context.Context) (storage.ChunkManager, error)
|
||||
NewVectorStorageChunkManager(ctx context.Context) (storage.ChunkManager, error)
|
||||
}
|
|
@ -21,26 +21,22 @@ import (
|
|||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
)
|
||||
|
||||
func TestInputNode(t *testing.T) {
|
||||
os.Setenv("ROCKSMQ_PATH", "/tmp/MilvusTest/FlowGraph/TestInputNode")
|
||||
msFactory := msgstream.NewRmsFactory()
|
||||
var Params paramtable.ComponentParam
|
||||
err := msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
msgStream, _ := msFactory.NewMsgStream(context.TODO())
|
||||
msgStream, _ := factory.NewMsgStream(context.TODO())
|
||||
channels := []string{"cc"}
|
||||
msgStream.AsConsumer(channels, "sub")
|
||||
msgStream.Start()
|
||||
|
||||
msgPack := generateMsgPack()
|
||||
produceStream, _ := msFactory.NewMsgStream(context.TODO())
|
||||
produceStream, _ := factory.NewMsgStream(context.TODO())
|
||||
produceStream.AsProducer(channels)
|
||||
produceStream.Produce(&msgPack)
|
||||
|
||||
|
|
|
@ -24,12 +24,12 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
)
|
||||
|
||||
func generateMsgPack() msgstream.MsgPack {
|
||||
|
@ -58,16 +58,13 @@ func generateMsgPack() msgstream.MsgPack {
|
|||
|
||||
func TestNodeCtx_Start(t *testing.T) {
|
||||
os.Setenv("ROCKSMQ_PATH", "/tmp/MilvusTest/FlowGraph/TestNodeStart")
|
||||
msFactory := msgstream.NewRmsFactory()
|
||||
var Params paramtable.ComponentParam
|
||||
err := msFactory.Init(&Params)
|
||||
assert.Nil(t, err)
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
msgStream, _ := msFactory.NewMsgStream(context.TODO())
|
||||
msgStream, _ := factory.NewMsgStream(context.TODO())
|
||||
channels := []string{"cc"}
|
||||
msgStream.AsConsumer(channels, "sub")
|
||||
|
||||
produceStream, _ := msFactory.NewMsgStream(context.TODO())
|
||||
produceStream, _ := factory.NewMsgStream(context.TODO())
|
||||
produceStream.AsProducer(channels)
|
||||
|
||||
msgPack := generateMsgPack()
|
||||
|
|
|
@ -21,13 +21,14 @@ import (
|
|||
"sync"
|
||||
"syscall"
|
||||
|
||||
"github.com/spf13/cast"
|
||||
"github.com/spf13/viper"
|
||||
|
||||
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/spf13/cast"
|
||||
"github.com/spf13/viper"
|
||||
)
|
||||
|
||||
// UniqueID is type alias of typeutil.UniqueID
|
||||
|
|
|
@ -20,8 +20,9 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -607,7 +608,8 @@ type queryNodeConfig struct {
|
|||
OverloadedMemoryThresholdPercentage float64
|
||||
|
||||
// cache limit
|
||||
LocalFileCacheLimit int64
|
||||
CacheEnabled bool
|
||||
CacheMemoryLimit int64
|
||||
}
|
||||
|
||||
func (p *queryNodeConfig) init(base *BaseTable) {
|
||||
|
@ -629,7 +631,8 @@ func (p *queryNodeConfig) init(base *BaseTable) {
|
|||
|
||||
p.initOverloadedMemoryThresholdPercentage()
|
||||
|
||||
p.initLocalFileCacheLimit()
|
||||
p.initCacheMemoryLimit()
|
||||
p.initCacheEnabled()
|
||||
}
|
||||
|
||||
// InitAlias initializes an alias for the QueryNode role.
|
||||
|
@ -704,13 +707,21 @@ func (p *queryNodeConfig) initOverloadedMemoryThresholdPercentage() {
|
|||
p.OverloadedMemoryThresholdPercentage = float64(thresholdPercentage) / 100
|
||||
}
|
||||
|
||||
func (p *queryNodeConfig) initLocalFileCacheLimit() {
|
||||
overloadedMemoryThresholdPercentage := p.Base.LoadWithDefault("querynoe.chunkManager.localFileCacheLimit", "90")
|
||||
localFileCacheLimit, err := strconv.ParseInt(overloadedMemoryThresholdPercentage, 10, 64)
|
||||
func (p *queryNodeConfig) initCacheMemoryLimit() {
|
||||
overloadedMemoryThresholdPercentage := p.Base.LoadWithDefault("queryNode.cache.memoryLimit", "2147483648")
|
||||
cacheMemoryLimit, err := strconv.ParseInt(overloadedMemoryThresholdPercentage, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.CacheMemoryLimit = cacheMemoryLimit
|
||||
}
|
||||
func (p *queryNodeConfig) initCacheEnabled() {
|
||||
var err error
|
||||
cacheEnabled := p.Base.LoadWithDefault("queryNode.cache.enabled", "true")
|
||||
p.CacheEnabled, err = strconv.ParseBool(cacheEnabled)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.LocalFileCacheLimit = localFileCacheLimit
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
|
|
|
@ -36,15 +36,17 @@ const (
|
|||
type ServiceParam struct {
|
||||
BaseTable
|
||||
|
||||
EtcdCfg EtcdConfig
|
||||
PulsarCfg PulsarConfig
|
||||
RocksmqCfg RocksmqConfig
|
||||
MinioCfg MinioConfig
|
||||
LocalStorageCfg LocalStorageConfig
|
||||
EtcdCfg EtcdConfig
|
||||
PulsarCfg PulsarConfig
|
||||
RocksmqCfg RocksmqConfig
|
||||
MinioCfg MinioConfig
|
||||
}
|
||||
|
||||
func (p *ServiceParam) Init() {
|
||||
p.BaseTable.Init()
|
||||
|
||||
p.LocalStorageCfg.init(&p.BaseTable)
|
||||
p.EtcdCfg.init(&p.BaseTable)
|
||||
p.PulsarCfg.init(&p.BaseTable)
|
||||
p.RocksmqCfg.init(&p.BaseTable)
|
||||
|
@ -145,6 +147,21 @@ func (p *EtcdConfig) initEtcdLogPath() {
|
|||
p.EtcdLogPath = p.Base.LoadWithDefault("etcd.log.path", defaultEtcdLogPath)
|
||||
}
|
||||
|
||||
type LocalStorageConfig struct {
|
||||
Base *BaseTable
|
||||
|
||||
Path string
|
||||
}
|
||||
|
||||
func (p *LocalStorageConfig) init(base *BaseTable) {
|
||||
p.Base = base
|
||||
p.initPath()
|
||||
}
|
||||
|
||||
func (p *LocalStorageConfig) initPath() {
|
||||
p.Path = p.Base.LoadWithDefault("localStorage.path", "/var/lib/milvus/data")
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
// --- pulsar ---
|
||||
type PulsarConfig struct {
|
||||
|
|
Loading…
Reference in New Issue