mirror of https://github.com/milvus-io/milvus.git
Rename dataservice to datacoordinator (#5912)
Signed-off-by: sunby <bingyi.sun@zilliz.com>pull/5939/head
parent
688a587f4f
commit
7f2d7c1da3
|
@ -35,6 +35,7 @@ pipeline {
|
|||
IMAGE_REPO = "dockerhub-mirror-sh.zilliz.cc/milvusdb"
|
||||
DOCKER_BUILDKIT = 1
|
||||
ARTIFACTS = "${env.WORKSPACE}/artifacts"
|
||||
MILVUS_HELM_BRANCH = "datacoord"
|
||||
}
|
||||
stages {
|
||||
stage('Test') {
|
||||
|
|
|
@ -44,6 +44,7 @@ pipeline {
|
|||
DOCKER_CREDENTIALS_ID = "ba070c98-c8cc-4f7c-b657-897715f359fc"
|
||||
DOKCER_REGISTRY_URL = "registry.zilliz.com"
|
||||
TARGET_REPO = "${DOKCER_REGISTRY_URL}/milvus"
|
||||
MILVUS_HELM_BRANCH = "datacoord"
|
||||
}
|
||||
stages {
|
||||
stage('Test') {
|
||||
|
|
|
@ -56,7 +56,7 @@ indexService:
|
|||
indexNode:
|
||||
port: 21121
|
||||
|
||||
dataService:
|
||||
dataCoord:
|
||||
address: localhost
|
||||
port: 13333
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ const (
|
|||
//
|
||||
// DataNode struct implements `types.Component`, `types.DataNode` interfaces.
|
||||
// `rootCoord` holds a grpc client of root coordinator.
|
||||
// `dataService` holds a grpc client of data service.
|
||||
// `dataCoord` holds a grpc client of data service.
|
||||
// `NodeID` is unique to each data node.
|
||||
// `State` is current statement of this data node, indicating whether it's healthy.
|
||||
//
|
||||
|
@ -77,8 +77,8 @@ type DataNode struct {
|
|||
clearSignal chan UniqueID // collection ID
|
||||
segmentCache *Cache
|
||||
|
||||
rootCoord types.RootCoord
|
||||
dataService types.DataService
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
|
||||
session *sessionutil.Session
|
||||
|
||||
|
@ -98,7 +98,7 @@ func NewDataNode(ctx context.Context, factory msgstream.Factory) *DataNode {
|
|||
watchDm: make(chan struct{}, 1),
|
||||
|
||||
rootCoord: nil,
|
||||
dataService: nil,
|
||||
dataCoord: nil,
|
||||
msFactory: factory,
|
||||
segmentCache: newCache(),
|
||||
|
||||
|
@ -121,13 +121,13 @@ func (node *DataNode) SetRootCoordInterface(rc types.RootCoord) error {
|
|||
}
|
||||
}
|
||||
|
||||
// SetDataServiceInterface sets data service's grpc client, error is returned if repeatedly set.
|
||||
func (node *DataNode) SetDataServiceInterface(ds types.DataService) error {
|
||||
// SetDataCoordInterface sets data service's grpc client, error is returned if repeatedly set.
|
||||
func (node *DataNode) SetDataCoordInterface(ds types.DataCoord) error {
|
||||
switch {
|
||||
case ds == nil, node.dataService != nil:
|
||||
case ds == nil, node.dataCoord != nil:
|
||||
return errors.New("Nil parameter or repeatly set")
|
||||
default:
|
||||
node.dataService = ds
|
||||
node.dataCoord = ds
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
@ -173,7 +173,7 @@ func (node *DataNode) NewDataSyncService(vchan *datapb.VchannelInfo) error {
|
|||
)
|
||||
|
||||
flushChan := make(chan *flushMsg, 100)
|
||||
dataSyncService, err := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataService)
|
||||
dataSyncService, err := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataCoord)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ type dataSyncService struct {
|
|||
idAllocator allocatorInterface
|
||||
msFactory msgstream.Factory
|
||||
collectionID UniqueID
|
||||
dataService types.DataService
|
||||
dataCoord types.DataCoord
|
||||
clearSignal chan<- UniqueID
|
||||
}
|
||||
|
||||
|
@ -46,7 +46,7 @@ func newDataSyncService(ctx context.Context,
|
|||
factory msgstream.Factory,
|
||||
vchan *datapb.VchannelInfo,
|
||||
clearSignal chan<- UniqueID,
|
||||
dataService types.DataService,
|
||||
dataCoord types.DataCoord,
|
||||
|
||||
) (*dataSyncService, error) {
|
||||
|
||||
|
@ -61,7 +61,7 @@ func newDataSyncService(ctx context.Context,
|
|||
idAllocator: alloc,
|
||||
msFactory: factory,
|
||||
collectionID: vchan.GetCollectionID(),
|
||||
dataService: dataService,
|
||||
dataCoord: dataCoord,
|
||||
clearSignal: clearSignal,
|
||||
}
|
||||
|
||||
|
@ -139,7 +139,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
|||
StartPositions: fu.startPositions,
|
||||
Flushed: fu.flushed,
|
||||
}
|
||||
rsp, err := dsService.dataService.SaveBinlogPaths(dsService.ctx, req)
|
||||
rsp, err := dsService.dataCoord.SaveBinlogPaths(dsService.ctx, req)
|
||||
if err != nil {
|
||||
return fmt.Errorf(err.Error())
|
||||
}
|
||||
|
|
|
@ -66,7 +66,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
}
|
||||
|
||||
signalCh := make(chan UniqueID, 100)
|
||||
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataServiceFactory{})
|
||||
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataCoordFactory{})
|
||||
|
||||
assert.Nil(t, err)
|
||||
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||
|
|
|
@ -522,7 +522,7 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
}
|
||||
}
|
||||
|
||||
// iMsg is Flush() msg from data cooperator
|
||||
// iMsg is Flush() msg from datacoord
|
||||
select {
|
||||
case fmsg := <-ibNode.flushChan:
|
||||
currentSegID := fmsg.segmentID
|
||||
|
|
|
@ -56,8 +56,8 @@ func newIDLEDataNodeMock(ctx context.Context) *DataNode {
|
|||
|
||||
node.SetRootCoordInterface(rc)
|
||||
|
||||
ds := &DataServiceFactory{}
|
||||
node.SetDataServiceInterface(ds)
|
||||
ds := &DataCoordFactory{}
|
||||
node.SetDataCoordInterface(ds)
|
||||
|
||||
return node
|
||||
}
|
||||
|
@ -88,8 +88,8 @@ func newHEALTHDataNodeMock(dmChannelName string) *DataNode {
|
|||
|
||||
node.SetRootCoordInterface(ms)
|
||||
|
||||
ds := &DataServiceFactory{}
|
||||
node.SetDataServiceInterface(ds)
|
||||
ds := &DataCoordFactory{}
|
||||
node.SetDataCoordInterface(ds)
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
|
@ -163,11 +163,11 @@ type RootCoordFactory struct {
|
|||
collectionID UniqueID
|
||||
}
|
||||
|
||||
type DataServiceFactory struct {
|
||||
types.DataService
|
||||
type DataCoordFactory struct {
|
||||
types.DataCoord
|
||||
}
|
||||
|
||||
func (ds *DataServiceFactory) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||
func (ds *DataCoordFactory) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -187,7 +187,7 @@ func (s *Server) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsert
|
|||
tMeta := &datapb.SegmentFieldBinlogMeta{}
|
||||
for _, v := range values {
|
||||
if err := proto.UnmarshalText(v, tMeta); err != nil {
|
||||
resp.Status.Reason = fmt.Errorf("DataService GetInsertBinlogPaths UnmarshalText datapb.SegmentFieldBinlogMeta err:%w", err).Error()
|
||||
resp.Status.Reason = fmt.Errorf("DataCoord GetInsertBinlogPaths UnmarshalText datapb.SegmentFieldBinlogMeta err:%w", err).Error()
|
||||
return resp, nil
|
||||
}
|
||||
m[tMeta.FieldID] = append(m[tMeta.FieldID], tMeta.BinlogPath)
|
||||
|
@ -278,7 +278,6 @@ func (s *Server) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoR
|
|||
return resp, nil
|
||||
}
|
||||
|
||||
// SaveBinlogPaths implement DataServiceServer
|
||||
func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||
resp := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
|
@ -333,7 +332,7 @@ func (s *Server) GetComponentStates(ctx context.Context) (*internalpb.ComponentS
|
|||
resp := &internalpb.ComponentStates{
|
||||
State: &internalpb.ComponentInfo{
|
||||
NodeID: Params.NodeID,
|
||||
Role: "dataservice",
|
||||
Role: "datacoord",
|
||||
StateCode: 0,
|
||||
},
|
||||
Status: &commonpb.Status{
|
||||
|
|
|
@ -25,7 +25,7 @@ import (
|
|||
)
|
||||
|
||||
const (
|
||||
metaPrefix = "dataservice-meta"
|
||||
metaPrefix = "datacoord-meta"
|
||||
segmentPrefix = metaPrefix + "/s"
|
||||
)
|
||||
|
||||
|
@ -81,7 +81,7 @@ func (m *meta) reloadFromKV() error {
|
|||
segmentInfo := &datapb.SegmentInfo{}
|
||||
err = proto.UnmarshalText(value, segmentInfo)
|
||||
if err != nil {
|
||||
return fmt.Errorf("DataService reloadFromKV UnMarshalText datapb.SegmentInfo err:%w", err)
|
||||
return fmt.Errorf("DataCoord reloadFromKV UnMarshalText datapb.SegmentInfo err:%w", err)
|
||||
}
|
||||
m.segments[segmentInfo.ID] = segmentInfo
|
||||
}
|
||||
|
|
|
@ -336,7 +336,7 @@ func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID Unique
|
|||
return nil, err
|
||||
}
|
||||
|
||||
log.Debug("dataservice: estimateTotalRows: ",
|
||||
log.Debug("datacoord: estimateTotalRows: ",
|
||||
zap.Int64("CollectionID", segmentInfo.CollectionID),
|
||||
zap.Int64("SegmentID", segmentInfo.ID),
|
||||
zap.Int("Rows", totalRows),
|
||||
|
|
|
@ -71,23 +71,23 @@ func (m *mockRootCoord) ShowCollections(ctx context.Context, req *milvuspb.ShowC
|
|||
}, nil
|
||||
}
|
||||
|
||||
type mockDataService struct {
|
||||
types.DataService
|
||||
type mockDataCoord struct {
|
||||
types.DataCoord
|
||||
}
|
||||
|
||||
func (m *mockDataService) Init() error {
|
||||
func (m *mockDataCoord) Init() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockDataService) Start() error {
|
||||
func (m *mockDataCoord) Start() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockDataService) Stop() error {
|
||||
func (m *mockDataCoord) Stop() error {
|
||||
return fmt.Errorf("stop error")
|
||||
}
|
||||
|
||||
func (m *mockDataService) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||
func (m *mockDataCoord) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||
return &internalpb.ComponentStates{
|
||||
State: &internalpb.ComponentInfo{
|
||||
StateCode: internalpb.StateCode_Healthy,
|
||||
|
@ -114,8 +114,8 @@ func TestRun(t *testing.T) {
|
|||
dnServer.newRootCoordClient = func() (types.RootCoord, error) {
|
||||
return &mockRootCoord{}, nil
|
||||
}
|
||||
dnServer.newDataServiceClient = func(string, []string, time.Duration) types.DataService {
|
||||
return &mockDataService{}
|
||||
dnServer.newDataCoordClient = func(string, []string, time.Duration) types.DataCoord {
|
||||
return &mockDataCoord{}
|
||||
}
|
||||
|
||||
grpcPort := rand.Int()%100 + 10000
|
||||
|
|
|
@ -31,15 +31,15 @@ type ParamTable struct {
|
|||
Port int
|
||||
listener net.Listener
|
||||
|
||||
MasterAddress string
|
||||
DataServiceAddress string
|
||||
MasterAddress string
|
||||
DataCoordAddress string
|
||||
}
|
||||
|
||||
func (pt *ParamTable) Init() {
|
||||
once.Do(func() {
|
||||
pt.BaseTable.Init()
|
||||
pt.initMasterAddress()
|
||||
pt.initDataServiceAddress()
|
||||
pt.initDataCoordAddress()
|
||||
pt.initPort()
|
||||
})
|
||||
}
|
||||
|
@ -72,10 +72,10 @@ func (pt *ParamTable) initMasterAddress() {
|
|||
pt.MasterAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initDataServiceAddress() {
|
||||
ret, err := pt.Load("_DataServiceAddress")
|
||||
func (pt *ParamTable) initDataCoordAddress() {
|
||||
ret, err := pt.Load("_DataCoordAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.DataServiceAddress = ret
|
||||
pt.DataCoordAddress = ret
|
||||
}
|
||||
|
|
|
@ -30,8 +30,8 @@ func TestParamTable(t *testing.T) {
|
|||
assert.NotNil(t, Params.listener)
|
||||
t.Logf("DataNode listener:%d", Params.listener)
|
||||
|
||||
assert.NotEqual(t, Params.DataServiceAddress, "")
|
||||
t.Logf("DataServiceAddress:%s", Params.DataServiceAddress)
|
||||
assert.NotEqual(t, Params.DataCoordAddress, "")
|
||||
t.Logf("DataCoordAddress:%s", Params.DataCoordAddress)
|
||||
|
||||
assert.NotEqual(t, Params.MasterAddress, "")
|
||||
t.Logf("MasterAddress:%s", Params.MasterAddress)
|
||||
|
|
|
@ -51,11 +51,11 @@ type Server struct {
|
|||
|
||||
msFactory msgstream.Factory
|
||||
|
||||
rootCoord types.RootCoord
|
||||
dataService types.DataService
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
|
||||
newRootCoordClient func() (types.RootCoord, error)
|
||||
newDataServiceClient func(string, []string, time.Duration) types.DataService
|
||||
newRootCoordClient func() (types.RootCoord, error)
|
||||
newDataCoordClient func(string, []string, time.Duration) types.DataCoord
|
||||
|
||||
closer io.Closer
|
||||
}
|
||||
|
@ -71,7 +71,7 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||
newRootCoordClient: func() (types.RootCoord, error) {
|
||||
return rcc.NewClient(ctx1, dn.Params.MetaRootPath, dn.Params.EtcdEndpoints, 3*time.Second)
|
||||
},
|
||||
newDataServiceClient: func(etcdMetaRoot string, etcdEndpoints []string, timeout time.Duration) types.DataService {
|
||||
newDataCoordClient: func(etcdMetaRoot string, etcdEndpoints []string, timeout time.Duration) types.DataCoord {
|
||||
return dsc.NewClient(etcdMetaRoot, etcdEndpoints, timeout)
|
||||
},
|
||||
}
|
||||
|
@ -117,8 +117,8 @@ func (s *Server) SetRootCoordInterface(ms types.RootCoord) error {
|
|||
return s.datanode.SetRootCoordInterface(ms)
|
||||
}
|
||||
|
||||
func (s *Server) SetDataServiceInterface(ds types.DataService) error {
|
||||
return s.datanode.SetDataServiceInterface(ds)
|
||||
func (s *Server) SetDataCoordInterface(ds types.DataCoord) error {
|
||||
return s.datanode.SetDataCoordInterface(ds)
|
||||
}
|
||||
|
||||
func (s *Server) Run() error {
|
||||
|
@ -202,25 +202,25 @@ func (s *Server) init() error {
|
|||
}
|
||||
|
||||
// --- Data Server Client ---
|
||||
if s.newDataServiceClient != nil {
|
||||
log.Debug("Data service address", zap.String("address", Params.DataServiceAddress))
|
||||
if s.newDataCoordClient != nil {
|
||||
log.Debug("Data service address", zap.String("address", Params.DataCoordAddress))
|
||||
log.Debug("DataNode Init data service client ...")
|
||||
dataServiceClient := s.newDataServiceClient(dn.Params.MetaRootPath, dn.Params.EtcdEndpoints, 10*time.Second)
|
||||
if err = dataServiceClient.Init(); err != nil {
|
||||
log.Debug("DataNode newDataServiceClient failed", zap.Error(err))
|
||||
dataCoordClient := s.newDataCoordClient(dn.Params.MetaRootPath, dn.Params.EtcdEndpoints, 10*time.Second)
|
||||
if err = dataCoordClient.Init(); err != nil {
|
||||
log.Debug("DataNode newDataCoord failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
if err = dataServiceClient.Start(); err != nil {
|
||||
log.Debug("DataNode dataServiceClient Start failed", zap.Error(err))
|
||||
if err = dataCoordClient.Start(); err != nil {
|
||||
log.Debug("DataNode dataCoordClient Start failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
err = funcutil.WaitForComponentInitOrHealthy(ctx, dataServiceClient, "DataService", 1000000, time.Millisecond*200)
|
||||
err = funcutil.WaitForComponentInitOrHealthy(ctx, dataCoordClient, "DataCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
log.Debug("DataNode wait dataServiceClient ready failed", zap.Error(err))
|
||||
log.Debug("DataNode wait dataCoordClient ready failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("DataNode dataService is ready")
|
||||
if err = s.SetDataServiceInterface(dataServiceClient); err != nil {
|
||||
log.Debug("DataNode dataCoord is ready")
|
||||
if err = s.SetDataCoordInterface(dataCoordClient); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import (
|
|||
)
|
||||
|
||||
type Client struct {
|
||||
grpcClient datapb.DataServiceClient
|
||||
grpcClient datapb.DataCoordClient
|
||||
conn *grpc.ClientConn
|
||||
ctx context.Context
|
||||
addr string
|
||||
|
@ -44,17 +44,17 @@ type Client struct {
|
|||
reconnTry int
|
||||
}
|
||||
|
||||
func getDataServiceAddress(sess *sessionutil.Session) (string, error) {
|
||||
func getDataCoordAddress(sess *sessionutil.Session) (string, error) {
|
||||
key := typeutil.DataCoordRole
|
||||
msess, _, err := sess.GetSessions(key)
|
||||
if err != nil {
|
||||
log.Debug("DataServiceClient, getSessions failed", zap.Any("key", key), zap.Error(err))
|
||||
log.Debug("DataCoordClient, getSessions failed", zap.Any("key", key), zap.Error(err))
|
||||
return "", err
|
||||
}
|
||||
ms, ok := msess[key]
|
||||
if !ok {
|
||||
log.Debug("DataServiceClient, not existed in msess ", zap.Any("key", key), zap.Any("len of msess", len(msess)))
|
||||
return "", fmt.Errorf("number of dataservice is incorrect, %d", len(msess))
|
||||
log.Debug("DataCoordClient, not existed in msess ", zap.Any("key", key), zap.Any("len of msess", len(msess)))
|
||||
return "", fmt.Errorf("number of datacoord is incorrect, %d", len(msess))
|
||||
}
|
||||
return ms.Address, nil
|
||||
}
|
||||
|
@ -81,23 +81,23 @@ func (c *Client) Init() error {
|
|||
|
||||
func (c *Client) connect() error {
|
||||
var err error
|
||||
getDataServiceAddressFn := func() error {
|
||||
c.addr, err = getDataServiceAddress(c.sess)
|
||||
getDataCoordAddressFn := func() error {
|
||||
c.addr, err = getDataCoordAddress(c.sess)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
err = retry.Retry(c.reconnTry, 3*time.Second, getDataServiceAddressFn)
|
||||
err = retry.Retry(c.reconnTry, 3*time.Second, getDataCoordAddressFn)
|
||||
if err != nil {
|
||||
log.Debug("DataServiceClient try reconnect getDataServiceAddressFn failed", zap.Error(err))
|
||||
log.Debug("DataCoordClient try reconnect getDataCoordAddressFn failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
connectGrpcFunc := func() error {
|
||||
ctx, cancelFunc := context.WithTimeout(c.ctx, c.timeout)
|
||||
defer cancelFunc()
|
||||
opts := trace.GetInterceptorOpts()
|
||||
log.Debug("DataServiceClient try reconnect ", zap.String("address", c.addr))
|
||||
log.Debug("DataCoordClient try reconnect ", zap.String("address", c.addr))
|
||||
conn, err := grpc.DialContext(ctx, c.addr, grpc.WithInsecure(), grpc.WithBlock(),
|
||||
grpc.WithUnaryInterceptor(
|
||||
grpc_opentracing.UnaryClientInterceptor(opts...)),
|
||||
|
@ -112,10 +112,10 @@ func (c *Client) connect() error {
|
|||
|
||||
err = retry.Retry(c.reconnTry, 500*time.Millisecond, connectGrpcFunc)
|
||||
if err != nil {
|
||||
log.Debug("DataService try reconnect failed", zap.Error(err))
|
||||
log.Debug("DataCoord try reconnect failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
c.grpcClient = datapb.NewDataServiceClient(c.conn)
|
||||
c.grpcClient = datapb.NewDataCoordClient(c.conn)
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -39,7 +39,7 @@ func (pt *ParamTable) Init() {
|
|||
|
||||
func (pt *ParamTable) initParams() {
|
||||
pt.initMasterAddress()
|
||||
pt.initDataServiceAddress()
|
||||
pt.initDataCoordAddress()
|
||||
}
|
||||
|
||||
func (pt *ParamTable) LoadFromEnv() {
|
||||
|
@ -47,7 +47,7 @@ func (pt *ParamTable) LoadFromEnv() {
|
|||
}
|
||||
|
||||
func (pt *ParamTable) initPort() {
|
||||
pt.Port = pt.ParseInt("dataService.port")
|
||||
pt.Port = pt.ParseInt("dataCoord.port")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMasterAddress() {
|
||||
|
@ -58,8 +58,8 @@ func (pt *ParamTable) initMasterAddress() {
|
|||
pt.MasterAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initDataServiceAddress() {
|
||||
ret, err := pt.Load("_DataServiceAddress")
|
||||
func (pt *ParamTable) initDataCoordAddress() {
|
||||
ret, err := pt.Load("_DataCoordAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
|
|
@ -41,9 +41,9 @@ import (
|
|||
)
|
||||
|
||||
type Server struct {
|
||||
dataService *dataservice.Server
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
dataCoord *dataservice.Server
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
grpcErrChan chan error
|
||||
wg sync.WaitGroup
|
||||
|
@ -64,7 +64,7 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||
cancel: cancel,
|
||||
grpcErrChan: make(chan error),
|
||||
}
|
||||
s.dataService, err = dataservice.CreateServer(s.ctx, factory)
|
||||
s.dataCoord, err = dataservice.CreateServer(s.ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -82,21 +82,21 @@ func (s *Server) init() error {
|
|||
dataservice.Params.IP = Params.IP
|
||||
dataservice.Params.Port = Params.Port
|
||||
|
||||
err := s.dataService.Register()
|
||||
err := s.dataCoord.Register()
|
||||
if err != nil {
|
||||
log.Debug("DataService Register etcd failed", zap.Error(err))
|
||||
log.Debug("DataCoord Register etcd failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("DataService Register etcd success")
|
||||
log.Debug("DataCoord Register etcd success")
|
||||
|
||||
err = s.startGrpc()
|
||||
if err != nil {
|
||||
log.Debug("DataService startGrpc failed", zap.Error(err))
|
||||
log.Debug("DataCoord startGrpc failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if err := s.dataService.Init(); err != nil {
|
||||
log.Error("dataService init error", zap.Error(err))
|
||||
if err := s.dataCoord.Init(); err != nil {
|
||||
log.Error("dataCoord init error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -134,7 +134,7 @@ func (s *Server) startGrpcLoop(grpcPort int) {
|
|||
grpc.StreamInterceptor(
|
||||
grpc_opentracing.StreamServerInterceptor(opts...)))
|
||||
//grpc.UnaryInterceptor(grpc_prometheus.UnaryServerInterceptor))
|
||||
datapb.RegisterDataServiceServer(s.grpcServer, s)
|
||||
datapb.RegisterDataCoordServer(s.grpcServer, s)
|
||||
grpc_prometheus.Register(s.grpcServer)
|
||||
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
|
||||
if err := s.grpcServer.Serve(lis); err != nil {
|
||||
|
@ -143,7 +143,7 @@ func (s *Server) startGrpcLoop(grpcPort int) {
|
|||
}
|
||||
|
||||
func (s *Server) start() error {
|
||||
return s.dataService.Start()
|
||||
return s.dataCoord.Start()
|
||||
}
|
||||
|
||||
func (s *Server) Stop() error {
|
||||
|
@ -159,7 +159,7 @@ func (s *Server) Stop() error {
|
|||
s.grpcServer.GracefulStop()
|
||||
}
|
||||
|
||||
err = s.dataService.Stop()
|
||||
err = s.dataCoord.Stop()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -173,7 +173,7 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("DataService init done ...")
|
||||
log.Debug("DataCoord init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
|
@ -182,54 +182,54 @@ func (s *Server) Run() error {
|
|||
}
|
||||
|
||||
func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
||||
return s.dataService.GetComponentStates(ctx)
|
||||
return s.dataCoord.GetComponentStates(ctx)
|
||||
}
|
||||
|
||||
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
return s.dataService.GetTimeTickChannel(ctx)
|
||||
return s.dataCoord.GetTimeTickChannel(ctx)
|
||||
}
|
||||
|
||||
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
return s.dataService.GetStatisticsChannel(ctx)
|
||||
return s.dataCoord.GetStatisticsChannel(ctx)
|
||||
}
|
||||
|
||||
func (s *Server) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return s.dataService.GetSegmentInfo(ctx, req)
|
||||
return s.dataCoord.GetSegmentInfo(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*commonpb.Status, error) {
|
||||
return s.dataService.Flush(ctx, req)
|
||||
return s.dataCoord.Flush(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
|
||||
return s.dataService.AssignSegmentID(ctx, req)
|
||||
return s.dataCoord.AssignSegmentID(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
|
||||
return s.dataService.GetSegmentStates(ctx, req)
|
||||
return s.dataCoord.GetSegmentStates(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
|
||||
return s.dataService.GetInsertBinlogPaths(ctx, req)
|
||||
return s.dataCoord.GetInsertBinlogPaths(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) {
|
||||
return s.dataService.GetCollectionStatistics(ctx, req)
|
||||
return s.dataCoord.GetCollectionStatistics(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) {
|
||||
return s.dataService.GetPartitionStatistics(ctx, req)
|
||||
return s.dataCoord.GetPartitionStatistics(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) GetSegmentInfoChannel(ctx context.Context, req *datapb.GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
return s.dataService.GetSegmentInfoChannel(ctx)
|
||||
return s.dataCoord.GetSegmentInfoChannel(ctx)
|
||||
}
|
||||
|
||||
//SaveBinlogPaths implement DataServiceServer, saves segment, collection binlog according to datanode request
|
||||
//SaveBinlogPaths implement DataCoordServer, saves segment, collection binlog according to datanode request
|
||||
func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||
return s.dataService.SaveBinlogPaths(ctx, req)
|
||||
return s.dataCoord.SaveBinlogPaths(ctx, req)
|
||||
}
|
||||
|
||||
func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
|
||||
return s.dataService.GetRecoveryInfo(ctx, req)
|
||||
return s.dataCoord.GetRecoveryInfo(ctx, req)
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ type ParamTable struct {
|
|||
IndexServerAddress string
|
||||
MasterAddress string
|
||||
|
||||
DataServiceAddress string
|
||||
DataCoordAddress string
|
||||
QueryServiceAddress string
|
||||
|
||||
IP string
|
||||
|
@ -54,7 +54,7 @@ func (pt *ParamTable) initParams() {
|
|||
pt.initPort()
|
||||
pt.initMasterAddress()
|
||||
pt.initIndexServerAddress()
|
||||
pt.initDataServiceAddress()
|
||||
pt.initDataCoordAddress()
|
||||
pt.initQueryServiceAddress()
|
||||
}
|
||||
|
||||
|
@ -77,12 +77,12 @@ func (pt *ParamTable) initMasterAddress() {
|
|||
}
|
||||
|
||||
// todo remove and use load from env
|
||||
func (pt *ParamTable) initDataServiceAddress() {
|
||||
ret, err := pt.Load("_DataServiceAddress")
|
||||
func (pt *ParamTable) initDataCoordAddress() {
|
||||
ret, err := pt.Load("_DataCoordAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.DataServiceAddress = ret
|
||||
pt.DataCoordAddress = ret
|
||||
}
|
||||
|
||||
// todo remove and use load from env
|
||||
|
|
|
@ -55,7 +55,7 @@ type Server struct {
|
|||
grpcErrChan chan error
|
||||
|
||||
rootCoordClient *rcc.GrpcClient
|
||||
dataServiceClient *grpcdataserviceclient.Client
|
||||
dataCoordClient *grpcdataserviceclient.Client
|
||||
queryServiceClient *grpcqueryserviceclient.Client
|
||||
indexCoordClient *grpcindexcoordclient.Client
|
||||
|
||||
|
@ -188,15 +188,15 @@ func (s *Server) init() error {
|
|||
s.proxynode.SetRootCoordClient(s.rootCoordClient)
|
||||
log.Debug("set rootcoord client ...")
|
||||
|
||||
dataServiceAddr := Params.DataServiceAddress
|
||||
log.Debug("ProxyNode", zap.String("data service address", dataServiceAddr))
|
||||
s.dataServiceClient = grpcdataserviceclient.NewClient(proxynode.Params.MetaRootPath, proxynode.Params.EtcdEndpoints, timeout)
|
||||
err = s.dataServiceClient.Init()
|
||||
dataCoordAddr := Params.DataCoordAddress
|
||||
log.Debug("ProxyNode", zap.String("data service address", dataCoordAddr))
|
||||
s.dataCoordClient = grpcdataserviceclient.NewClient(proxynode.Params.MetaRootPath, proxynode.Params.EtcdEndpoints, timeout)
|
||||
err = s.dataCoordClient.Init()
|
||||
if err != nil {
|
||||
log.Debug("ProxyNode dataServiceClient init failed ", zap.Error(err))
|
||||
log.Debug("ProxyNode dataCoordClient init failed ", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
s.proxynode.SetDataServiceClient(s.dataServiceClient)
|
||||
s.proxynode.SetDataCoordClient(s.dataCoordClient)
|
||||
log.Debug("set data service address ...")
|
||||
|
||||
indexServiceAddr := Params.IndexServerAddress
|
||||
|
|
|
@ -110,11 +110,11 @@ func (m *MasterServiceMock) DescribeSegment(in *milvuspb.DescribeSegmentRequest)
|
|||
return rsp, nil
|
||||
}
|
||||
|
||||
type DataServiceMock struct {
|
||||
type DataCoordMock struct {
|
||||
Count int
|
||||
}
|
||||
|
||||
func (data *DataServiceMock) GetInsertBinlogPaths(req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
|
||||
func (data *DataCoordMock) GetInsertBinlogPaths(req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
|
||||
if data.Count < 10 {
|
||||
data.Count++
|
||||
return nil, errors.New("binlog not exist")
|
||||
|
@ -136,7 +136,7 @@ func (data *DataServiceMock) GetInsertBinlogPaths(req *datapb.GetInsertBinlogPat
|
|||
return rsp, nil
|
||||
}
|
||||
|
||||
func (data *DataServiceMock) GetSegmentStates(req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
|
||||
func (data *DataCoordMock) GetSegmentStates(req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
|
||||
segmentGrowingInfo := &datapb.SegmentStateInfo{
|
||||
State: commonpb.SegmentState_Growing,
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ type ParamTable struct {
|
|||
|
||||
IndexServiceAddress string
|
||||
MasterAddress string
|
||||
DataServiceAddress string
|
||||
DataCoordAddress string
|
||||
QueryServiceAddress string
|
||||
}
|
||||
|
||||
|
@ -40,7 +40,7 @@ func (pt *ParamTable) Init() {
|
|||
pt.initPort()
|
||||
pt.initMasterAddress()
|
||||
pt.initIndexServiceAddress()
|
||||
pt.initDataServiceAddress()
|
||||
pt.initDataCoordAddress()
|
||||
pt.initQueryServiceAddress()
|
||||
|
||||
})
|
||||
|
@ -70,12 +70,12 @@ func (pt *ParamTable) initIndexServiceAddress() {
|
|||
pt.IndexServiceAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initDataServiceAddress() {
|
||||
ret, err := pt.Load("_DataServiceAddress")
|
||||
func (pt *ParamTable) initDataCoordAddress() {
|
||||
ret, err := pt.Load("_DataCoordAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.DataServiceAddress = ret
|
||||
pt.DataCoordAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initQueryServiceAddress() {
|
||||
|
|
|
@ -23,8 +23,8 @@ func TestParamTable(t *testing.T) {
|
|||
assert.NotEqual(t, Params.IndexServiceAddress, "")
|
||||
t.Logf("IndexServiceAddress:%s", Params.IndexServiceAddress)
|
||||
|
||||
assert.NotEqual(t, Params.DataServiceAddress, "")
|
||||
t.Logf("DataServiceAddress:%s", Params.DataServiceAddress)
|
||||
assert.NotEqual(t, Params.DataCoordAddress, "")
|
||||
t.Logf("DataCoordAddress:%s", Params.DataCoordAddress)
|
||||
|
||||
assert.NotEqual(t, Params.MasterAddress, "")
|
||||
t.Logf("MasterAddress:%s", Params.MasterAddress)
|
||||
|
|
|
@ -56,7 +56,7 @@ type Server struct {
|
|||
|
||||
grpcServer *grpc.Server
|
||||
|
||||
dataService *dsc.Client
|
||||
dataCoord *dsc.Client
|
||||
rootCoord *rcc.GrpcClient
|
||||
indexCoord *isc.Client
|
||||
queryService *qsc.Client
|
||||
|
@ -189,26 +189,26 @@ func (s *Server) init() error {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
// --- DataService ---
|
||||
log.Debug("QueryNode start to new DataServiceClient", zap.Any("DataServiceAddress", Params.DataServiceAddress))
|
||||
dataService := dsc.NewClient(qn.Params.MetaRootPath, qn.Params.EtcdEndpoints, 3*time.Second)
|
||||
if err = dataService.Init(); err != nil {
|
||||
log.Debug("QueryNode DataServiceClient Init failed", zap.Error(err))
|
||||
// --- DataCoord ---
|
||||
log.Debug("QueryNode start to new DataCoordClient", zap.Any("DataCoordAddress", Params.DataCoordAddress))
|
||||
dataCoord := dsc.NewClient(qn.Params.MetaRootPath, qn.Params.EtcdEndpoints, 3*time.Second)
|
||||
if err = dataCoord.Init(); err != nil {
|
||||
log.Debug("QueryNode DataCoordClient Init failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
if err = dataService.Start(); err != nil {
|
||||
log.Debug("QueryNode DataServiceClient Start failed", zap.Error(err))
|
||||
if err = dataCoord.Start(); err != nil {
|
||||
log.Debug("QueryNode DataCoordClient Start failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("QueryNode start to wait for DataService ready")
|
||||
err = funcutil.WaitForComponentInitOrHealthy(s.ctx, dataService, "DataService", 1000000, time.Millisecond*200)
|
||||
log.Debug("QueryNode start to wait for DataCoord ready")
|
||||
err = funcutil.WaitForComponentInitOrHealthy(s.ctx, dataCoord, "DataCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
log.Debug("QueryNode wait for DataService ready failed", zap.Error(err))
|
||||
log.Debug("QueryNode wait for DataCoord ready failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("QueryNode report DataService is ready")
|
||||
log.Debug("QueryNode report DataCoord is ready")
|
||||
|
||||
if err := s.SetDataService(dataService); err != nil {
|
||||
if err := s.SetDataCoord(dataCoord); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
|
@ -314,8 +314,8 @@ func (s *Server) SetIndexCoord(indexCoord types.IndexCoord) error {
|
|||
return s.querynode.SetIndexCoord(indexCoord)
|
||||
}
|
||||
|
||||
func (s *Server) SetDataService(dataService types.DataService) error {
|
||||
return s.querynode.SetDataService(dataService)
|
||||
func (s *Server) SetDataCoord(dataCoord types.DataCoord) error {
|
||||
return s.querynode.SetDataCoord(dataCoord)
|
||||
}
|
||||
|
||||
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
|
|
|
@ -26,7 +26,7 @@ type ParamTable struct {
|
|||
|
||||
IndexServiceAddress string
|
||||
MasterAddress string
|
||||
DataServiceAddress string
|
||||
DataCoordAddress string
|
||||
}
|
||||
|
||||
func (pt *ParamTable) Init() {
|
||||
|
@ -35,7 +35,7 @@ func (pt *ParamTable) Init() {
|
|||
pt.initPort()
|
||||
pt.initMasterAddress()
|
||||
pt.initIndexServiceAddress()
|
||||
pt.initDataServiceAddress()
|
||||
pt.initDataCoordAddress()
|
||||
|
||||
})
|
||||
}
|
||||
|
@ -56,12 +56,12 @@ func (pt *ParamTable) initIndexServiceAddress() {
|
|||
pt.IndexServiceAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initDataServiceAddress() {
|
||||
ret, err := pt.Load("_DataServiceAddress")
|
||||
func (pt *ParamTable) initDataCoordAddress() {
|
||||
ret, err := pt.Load("_DataCoordAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.DataServiceAddress = ret
|
||||
pt.DataCoordAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initPort() {
|
||||
|
|
|
@ -23,8 +23,8 @@ func TestParamTable(t *testing.T) {
|
|||
assert.NotEqual(t, Params.IndexServiceAddress, "")
|
||||
t.Logf("IndexServiceAddress:%s", Params.IndexServiceAddress)
|
||||
|
||||
assert.NotEqual(t, Params.DataServiceAddress, "")
|
||||
t.Logf("DataServiceAddress:%s", Params.DataServiceAddress)
|
||||
assert.NotEqual(t, Params.DataCoordAddress, "")
|
||||
t.Logf("DataCoordAddress:%s", Params.DataCoordAddress)
|
||||
|
||||
assert.NotEqual(t, Params.MasterAddress, "")
|
||||
t.Logf("MasterAddress:%s", Params.MasterAddress)
|
||||
|
|
|
@ -50,8 +50,8 @@ type Server struct {
|
|||
|
||||
msFactory msgstream.Factory
|
||||
|
||||
dataService *dsc.Client
|
||||
rootCoord *rcc.GrpcClient
|
||||
dataCoord *dsc.Client
|
||||
rootCoord *rcc.GrpcClient
|
||||
|
||||
closer io.Closer
|
||||
}
|
||||
|
@ -136,27 +136,27 @@ func (s *Server) init() error {
|
|||
log.Debug("QueryService report RootCoord ready")
|
||||
|
||||
// --- Data service client ---
|
||||
log.Debug("QueryService try to new DataService client", zap.Any("DataServiceAddress", Params.DataServiceAddress))
|
||||
log.Debug("QueryService try to new DataCoord client", zap.Any("DataCoordAddress", Params.DataCoordAddress))
|
||||
|
||||
dataService := dsc.NewClient(qs.Params.MetaRootPath, qs.Params.EtcdEndpoints, 3*time.Second)
|
||||
if err = dataService.Init(); err != nil {
|
||||
log.Debug("QueryService DataServiceClient Init failed", zap.Error(err))
|
||||
dataCoord := dsc.NewClient(qs.Params.MetaRootPath, qs.Params.EtcdEndpoints, 3*time.Second)
|
||||
if err = dataCoord.Init(); err != nil {
|
||||
log.Debug("QueryService DataCoordClient Init failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
if err = dataService.Start(); err != nil {
|
||||
log.Debug("QueryService DataServiceClient Start failed", zap.Error(err))
|
||||
if err = dataCoord.Start(); err != nil {
|
||||
log.Debug("QueryService DataCoordClient Start failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("QueryService try to wait for DataService ready")
|
||||
err = funcutil.WaitForComponentInitOrHealthy(s.loopCtx, dataService, "DataService", 1000000, time.Millisecond*200)
|
||||
log.Debug("QueryService try to wait for DataCoord ready")
|
||||
err = funcutil.WaitForComponentInitOrHealthy(s.loopCtx, dataCoord, "DataCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
log.Debug("QueryService wait for DataService ready failed", zap.Error(err))
|
||||
log.Debug("QueryService wait for DataCoord ready failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
if err := s.SetDataService(dataService); err != nil {
|
||||
if err := s.SetDataCoord(dataCoord); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("QueryService report DataService ready")
|
||||
log.Debug("QueryService report DataCoord ready")
|
||||
|
||||
s.queryservice.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||
log.Debug("QueryService", zap.Any("State", internalpb.StateCode_Initializing))
|
||||
|
@ -220,8 +220,8 @@ func (s *Server) SetRootCoord(m types.RootCoord) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) SetDataService(d types.DataService) error {
|
||||
s.queryservice.SetDataService(d)
|
||||
func (s *Server) SetDataCoord(d types.DataCoord) error {
|
||||
s.queryservice.SetDataCoord(d)
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ type ParamTable struct {
|
|||
|
||||
IndexServiceAddress string
|
||||
QueryServiceAddress string
|
||||
DataServiceAddress string
|
||||
DataCoordAddress string
|
||||
}
|
||||
|
||||
func (p *ParamTable) Init() {
|
||||
|
@ -42,7 +42,7 @@ func (p *ParamTable) Init() {
|
|||
p.initPort()
|
||||
p.initIndexServiceAddress()
|
||||
p.initQueryServiceAddress()
|
||||
p.initDataServiceAddress()
|
||||
p.initDataCoordAddress()
|
||||
|
||||
})
|
||||
}
|
||||
|
@ -75,10 +75,10 @@ func (p *ParamTable) initQueryServiceAddress() {
|
|||
p.QueryServiceAddress = ret
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDataServiceAddress() {
|
||||
ret, err := p.Load("_DataServiceAddress")
|
||||
func (p *ParamTable) initDataCoordAddress() {
|
||||
ret, err := p.Load("_DataCoordAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.DataServiceAddress = ret
|
||||
p.DataCoordAddress = ret
|
||||
}
|
||||
|
|
|
@ -29,8 +29,8 @@ func TestParamTable(t *testing.T) {
|
|||
assert.NotEqual(t, Params.IndexServiceAddress, "")
|
||||
t.Logf("IndexServiceAddress:%s", Params.IndexServiceAddress)
|
||||
|
||||
assert.NotEqual(t, Params.DataServiceAddress, "")
|
||||
t.Logf("DataServiceAddress:%s", Params.DataServiceAddress)
|
||||
assert.NotEqual(t, Params.DataCoordAddress, "")
|
||||
t.Logf("DataCoordAddress:%s", Params.DataCoordAddress)
|
||||
|
||||
assert.NotEqual(t, Params.QueryServiceAddress, "")
|
||||
t.Logf("QueryServiceAddress:%s", Params.QueryServiceAddress)
|
||||
|
|
|
@ -53,12 +53,12 @@ type Server struct {
|
|||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
dataCoord types.DataService
|
||||
dataCoord types.DataCoord
|
||||
indexCoord types.IndexCoord
|
||||
queryCoord types.QueryService
|
||||
|
||||
newIndexCoordClient func(string, []string, time.Duration) types.IndexCoord
|
||||
newDataCoordClient func(string, []string, time.Duration) types.DataService
|
||||
newDataCoordClient func(string, []string, time.Duration) types.DataCoord
|
||||
newQueryCoordClient func(string, []string, time.Duration) types.QueryService
|
||||
|
||||
closer io.Closer
|
||||
|
@ -83,7 +83,7 @@ func NewServer(ctx context.Context, factory msgstream.Factory) (*Server, error)
|
|||
func (s *Server) setClient() {
|
||||
ctx := context.Background()
|
||||
|
||||
s.newDataCoordClient = func(etcdMetaRoot string, etcdEndpoints []string, timeout time.Duration) types.DataService {
|
||||
s.newDataCoordClient = func(etcdMetaRoot string, etcdEndpoints []string, timeout time.Duration) types.DataCoord {
|
||||
dsClient := dsc.NewClient(etcdMetaRoot, etcdEndpoints, timeout)
|
||||
if err := dsClient.Init(); err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -130,7 +130,7 @@ func TestGrpcService(t *testing.T) {
|
|||
rootcoord.Params.MsgChannelSubName = fmt.Sprintf("msgChannel%d", randVal)
|
||||
rootcoord.Params.TimeTickChannel = fmt.Sprintf("timeTick%d", randVal)
|
||||
rootcoord.Params.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal)
|
||||
rootcoord.Params.DataServiceSegmentChannel = fmt.Sprintf("segmentChannel%d", randVal)
|
||||
rootcoord.Params.DataCoordSegmentChannel = fmt.Sprintf("segmentChannel%d", randVal)
|
||||
|
||||
rootcoord.Params.MaxPartitionNum = 64
|
||||
rootcoord.Params.DefaultPartitionName = "_default"
|
||||
|
@ -821,7 +821,7 @@ type mockCore struct {
|
|||
func (m *mockCore) UpdateStateCode(internalpb.StateCode) {
|
||||
}
|
||||
|
||||
func (m *mockCore) SetDataCoord(context.Context, types.DataService) error {
|
||||
func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error {
|
||||
return nil
|
||||
}
|
||||
func (m *mockCore) SetIndexCoord(types.IndexCoord) error {
|
||||
|
@ -852,7 +852,7 @@ func (m *mockCore) SetNewProxyClient(func(sess *sessionutil.Session) (types.Prox
|
|||
}
|
||||
|
||||
type mockDataCoord struct {
|
||||
types.DataService
|
||||
types.DataCoord
|
||||
}
|
||||
|
||||
func (m *mockDataCoord) Init() error {
|
||||
|
@ -922,7 +922,7 @@ func TestRun(t *testing.T) {
|
|||
assert.NotNil(t, err)
|
||||
assert.EqualError(t, err, "listen tcp: address 1000000: invalid port")
|
||||
|
||||
svr.newDataCoordClient = func(string, []string, time.Duration) types.DataService {
|
||||
svr.newDataCoordClient = func(string, []string, time.Duration) types.DataCoord {
|
||||
return &mockDataCoord{}
|
||||
}
|
||||
svr.newIndexCoordClient = func(string, []string, time.Duration) types.IndexCoord {
|
||||
|
|
|
@ -9,7 +9,7 @@ import "internal.proto";
|
|||
import "milvus.proto";
|
||||
import "schema.proto";
|
||||
|
||||
service DataService {
|
||||
service DataCoord {
|
||||
rpc GetComponentStates(internal.GetComponentStatesRequest) returns (internal.ComponentStates) {}
|
||||
rpc GetTimeTickChannel(internal.GetTimeTickChannelRequest) returns(milvus.StringResponse) {}
|
||||
rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){}
|
||||
|
|
|
@ -2256,127 +2256,127 @@ func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad
|
|||
var fileDescriptor_3385cd32ad6cfe64 = []byte{
|
||||
// 1943 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0x5b, 0x6f, 0x1b, 0x59,
|
||||
0xb9, 0xe3, 0x89, 0x93, 0xf8, 0x9b, 0xb1, 0x93, 0x1c, 0x42, 0xd6, 0x72, 0xdb, 0x34, 0x1d, 0x76,
|
||||
0xbb, 0xd9, 0x22, 0x92, 0xad, 0x8b, 0xc4, 0xa5, 0x2c, 0x68, 0x5b, 0xb7, 0x91, 0x45, 0x52, 0xc2,
|
||||
0x49, 0x77, 0x57, 0x62, 0x85, 0xac, 0x89, 0xe7, 0xc4, 0x19, 0x3a, 0x17, 0xef, 0x9c, 0x71, 0x9a,
|
||||
0x3e, 0xed, 0xaa, 0x3c, 0x20, 0x10, 0xe2, 0xf2, 0xc0, 0x3b, 0x42, 0x42, 0x02, 0xf1, 0xc2, 0x3f,
|
||||
0xe0, 0x95, 0x9f, 0x85, 0xce, 0x65, 0xee, 0x63, 0x7b, 0xea, 0xd2, 0xcd, 0x9b, 0xcf, 0x99, 0xef,
|
||||
0x76, 0xbe, 0xfb, 0xf7, 0x19, 0x90, 0x65, 0x86, 0xe6, 0x80, 0x92, 0xe0, 0xc2, 0x1e, 0x92, 0xbd,
|
||||
0x71, 0xe0, 0x87, 0x3e, 0xda, 0x70, 0x6d, 0xe7, 0x62, 0x42, 0xc5, 0x69, 0x8f, 0x01, 0x74, 0xf4,
|
||||
0xa1, 0xef, 0xba, 0xbe, 0x27, 0xae, 0x3a, 0x2d, 0xdb, 0x0b, 0x49, 0xe0, 0x99, 0x8e, 0x3c, 0xeb,
|
||||
0x69, 0x84, 0x8e, 0x4e, 0x87, 0xe7, 0xc4, 0x35, 0xc5, 0xc9, 0xb8, 0x04, 0xfd, 0x89, 0x33, 0xa1,
|
||||
0xe7, 0x98, 0x7c, 0x31, 0x21, 0x34, 0x44, 0x1f, 0xc2, 0xd2, 0xa9, 0x49, 0x49, 0x5b, 0xd9, 0x51,
|
||||
0x76, 0xb5, 0xee, 0x8d, 0xbd, 0x0c, 0x2f, 0xc9, 0xe5, 0x88, 0x8e, 0x1e, 0x9a, 0x94, 0x60, 0x0e,
|
||||
0x89, 0x10, 0x2c, 0x59, 0xa7, 0xfd, 0x5e, 0xbb, 0xb6, 0xa3, 0xec, 0xaa, 0x98, 0xff, 0x46, 0x06,
|
||||
0xe8, 0x43, 0xdf, 0x71, 0xc8, 0x30, 0xb4, 0x7d, 0xaf, 0xdf, 0x6b, 0x2f, 0xf1, 0x6f, 0x99, 0x3b,
|
||||
0xe3, 0xcf, 0x0a, 0xac, 0x9f, 0x90, 0x91, 0x4b, 0xbc, 0xb0, 0xdf, 0x8b, 0xd8, 0x6f, 0x42, 0x7d,
|
||||
0xe8, 0x4f, 0xbc, 0x90, 0xf3, 0x6f, 0x62, 0x71, 0x40, 0xb7, 0x41, 0x1f, 0x9e, 0x9b, 0x9e, 0x47,
|
||||
0x9c, 0x81, 0x67, 0xba, 0x84, 0xb3, 0x6a, 0x60, 0x4d, 0xde, 0x3d, 0x35, 0x5d, 0x52, 0xe0, 0xa8,
|
||||
0x16, 0x39, 0xa2, 0x1d, 0xd0, 0xc6, 0x66, 0x10, 0xda, 0x19, 0xa1, 0xd2, 0x57, 0xc6, 0x5f, 0x15,
|
||||
0xd8, 0xfa, 0x98, 0x52, 0x7b, 0xe4, 0x15, 0x24, 0xdb, 0x82, 0x65, 0xcf, 0xb7, 0x48, 0xbf, 0xc7,
|
||||
0x45, 0x53, 0xb1, 0x3c, 0xa1, 0xeb, 0xd0, 0x18, 0x13, 0x12, 0x0c, 0x02, 0xdf, 0x89, 0x04, 0x5b,
|
||||
0x65, 0x17, 0xd8, 0x77, 0x08, 0xfa, 0x39, 0x6c, 0xd0, 0x1c, 0x21, 0xda, 0x56, 0x77, 0xd4, 0x5d,
|
||||
0xad, 0xfb, 0xad, 0xbd, 0x82, 0x19, 0xf7, 0xf2, 0x4c, 0x71, 0x11, 0xdb, 0xf8, 0xaa, 0x06, 0xdf,
|
||||
0x88, 0xe1, 0x84, 0xac, 0xec, 0x37, 0xd3, 0x1c, 0x25, 0xa3, 0x58, 0x3c, 0x71, 0xa8, 0xa2, 0xb9,
|
||||
0x58, 0xe5, 0x6a, 0x5a, 0xe5, 0x15, 0x2c, 0x98, 0xd7, 0x67, 0xbd, 0xa0, 0x4f, 0x74, 0x0b, 0x34,
|
||||
0x72, 0x39, 0xb6, 0x03, 0x32, 0x08, 0x6d, 0x97, 0xb4, 0x97, 0x77, 0x94, 0xdd, 0x25, 0x0c, 0xe2,
|
||||
0xea, 0x99, 0xed, 0x12, 0x74, 0x1f, 0x96, 0x69, 0x68, 0x86, 0x13, 0xda, 0x5e, 0xe1, 0x0e, 0x77,
|
||||
0xbd, 0xd4, 0xe1, 0x4e, 0x38, 0x08, 0x96, 0xa0, 0xc6, 0xdf, 0x14, 0x78, 0xa7, 0x60, 0x25, 0x3a,
|
||||
0xf6, 0x3d, 0x4a, 0x10, 0x86, 0x75, 0xfe, 0xf2, 0x44, 0x33, 0xb4, 0xad, 0x70, 0x85, 0xdf, 0x99,
|
||||
0xa5, 0xf0, 0x04, 0x1c, 0x17, 0xf0, 0x53, 0x42, 0xd6, 0xaa, 0x0b, 0xf9, 0x1c, 0xde, 0x39, 0x20,
|
||||
0xa1, 0x64, 0xc0, 0xbe, 0x11, 0xba, 0x78, 0x8c, 0x6d, 0x03, 0xc4, 0x9e, 0xc0, 0xa4, 0x50, 0x77,
|
||||
0x55, 0x9c, 0xba, 0x31, 0xfe, 0x5d, 0x8b, 0x63, 0x89, 0xb3, 0xea, 0x7b, 0x67, 0x3e, 0xba, 0x01,
|
||||
0x8d, 0x18, 0x44, 0x7a, 0x45, 0x72, 0x81, 0xbe, 0x07, 0x75, 0x26, 0xa9, 0x70, 0x89, 0x56, 0xf7,
|
||||
0x76, 0xf9, 0x9b, 0x52, 0x34, 0xb1, 0x80, 0x47, 0x7d, 0x68, 0xd1, 0xd0, 0x0c, 0xc2, 0xc1, 0xd8,
|
||||
0xa7, 0xdc, 0xce, 0xdc, 0x71, 0xb4, 0xae, 0x91, 0xa5, 0x10, 0xe7, 0xa0, 0x23, 0x3a, 0x3a, 0x96,
|
||||
0x90, 0xb8, 0xc9, 0x31, 0xa3, 0x23, 0x7a, 0x0c, 0x3a, 0xf1, 0xac, 0x84, 0xd0, 0x52, 0x65, 0x42,
|
||||
0x1a, 0xf1, 0xac, 0x98, 0x4c, 0x62, 0x9f, 0x7a, 0x75, 0xfb, 0xfc, 0x5e, 0x81, 0x76, 0xd1, 0x40,
|
||||
0xd2, 0x8b, 0x12, 0x8a, 0x4a, 0x65, 0x8a, 0xe8, 0x81, 0x40, 0x22, 0xc2, 0x40, 0x33, 0x23, 0x3c,
|
||||
0x36, 0x12, 0x96, 0x28, 0x86, 0x0d, 0xdf, 0x4c, 0xa4, 0xe1, 0x5f, 0xde, 0x9a, 0xb3, 0xfc, 0x5a,
|
||||
0x81, 0xad, 0x3c, 0xaf, 0x37, 0x79, 0xf7, 0x77, 0xa1, 0x6e, 0x7b, 0x67, 0x7e, 0xf4, 0xec, 0xed,
|
||||
0x19, 0x71, 0xc6, 0x78, 0x09, 0x60, 0xc3, 0x85, 0xeb, 0x07, 0x24, 0xec, 0x7b, 0x94, 0x04, 0xe1,
|
||||
0x43, 0xdb, 0x73, 0xfc, 0xd1, 0xb1, 0x19, 0x9e, 0xbf, 0x41, 0x8c, 0x64, 0xdc, 0xbd, 0x96, 0x73,
|
||||
0x77, 0xe3, 0x1f, 0x0a, 0xdc, 0x28, 0xe7, 0x27, 0x9f, 0xde, 0x81, 0xd5, 0x33, 0x9b, 0x38, 0x16,
|
||||
0xd3, 0x99, 0xc2, 0x75, 0x16, 0x9f, 0x59, 0xac, 0x8c, 0x19, 0xb0, 0x7c, 0xe1, 0xed, 0x29, 0x0e,
|
||||
0x7a, 0x12, 0x06, 0xb6, 0x37, 0x3a, 0xb4, 0x69, 0x88, 0x05, 0x7c, 0x4a, 0x9f, 0x6a, 0x75, 0xcf,
|
||||
0xfc, 0x9d, 0x02, 0xdb, 0x07, 0x24, 0x7c, 0x14, 0xa7, 0x5a, 0xf6, 0xdd, 0xa6, 0xa1, 0x3d, 0xa4,
|
||||
0x6f, 0xb7, 0x4a, 0x97, 0xd4, 0x4c, 0xe3, 0x8f, 0x0a, 0xdc, 0x9a, 0x2a, 0x8c, 0x54, 0x9d, 0x4c,
|
||||
0x25, 0x51, 0xa2, 0x2d, 0x4f, 0x25, 0x3f, 0x25, 0x2f, 0x3f, 0x35, 0x9d, 0x09, 0x39, 0x36, 0xed,
|
||||
0x40, 0xa4, 0x92, 0x05, 0x13, 0xeb, 0xbf, 0x14, 0xb8, 0x79, 0x40, 0xc2, 0xe3, 0xa8, 0xcc, 0x5c,
|
||||
0xa1, 0x76, 0x2a, 0x74, 0x14, 0x7f, 0x10, 0xc6, 0x2c, 0x95, 0xf6, 0x4a, 0xd4, 0xb7, 0xcd, 0xe3,
|
||||
0x20, 0x15, 0x90, 0x8f, 0x44, 0x2f, 0x20, 0x95, 0x67, 0xfc, 0xa5, 0x06, 0xfa, 0xa7, 0xb2, 0x3f,
|
||||
0xe0, 0x65, 0x24, 0xaf, 0x07, 0xa5, 0x5c, 0x0f, 0xa9, 0x96, 0xa2, 0xac, 0xcb, 0x38, 0x80, 0x26,
|
||||
0x25, 0xe4, 0xf9, 0x22, 0x45, 0x43, 0x67, 0x88, 0x71, 0xb2, 0x3f, 0x84, 0x8d, 0x89, 0x77, 0xc6,
|
||||
0x5a, 0x56, 0x62, 0xc9, 0x57, 0xd0, 0xf6, 0x52, 0xa5, 0xcc, 0x53, 0x44, 0x44, 0xbb, 0xb0, 0x96,
|
||||
0xa7, 0x55, 0xe7, 0xc1, 0x9f, 0xbf, 0x36, 0x7e, 0xab, 0xc0, 0xd6, 0x67, 0x66, 0x38, 0x3c, 0xef,
|
||||
0xb9, 0x52, 0x63, 0x6f, 0xe0, 0x6f, 0x1f, 0x41, 0xe3, 0x42, 0x6a, 0x27, 0x4a, 0x2a, 0xb7, 0x4a,
|
||||
0x84, 0x4f, 0xdb, 0x01, 0x27, 0x18, 0xac, 0x4d, 0xdd, 0xe4, 0x5d, 0x7b, 0x24, 0xdd, 0xd7, 0xef,
|
||||
0xf9, 0xd9, 0x22, 0xb3, 0x54, 0x28, 0x32, 0x97, 0x00, 0x52, 0xb8, 0x23, 0x3a, 0x5a, 0x40, 0xae,
|
||||
0xef, 0xc3, 0x8a, 0xa4, 0x26, 0x9d, 0x7b, 0x9e, 0x71, 0x23, 0x70, 0xe3, 0x04, 0xb6, 0xe4, 0xfd,
|
||||
0x13, 0x96, 0xbf, 0x45, 0xae, 0x3f, 0x22, 0xa1, 0x89, 0xda, 0xb0, 0x22, 0x53, 0xba, 0x74, 0xe2,
|
||||
0xe8, 0xc8, 0xfa, 0xd4, 0x53, 0x0e, 0x37, 0x60, 0x79, 0x5b, 0xfa, 0x2f, 0x9c, 0xc6, 0x65, 0xc2,
|
||||
0xf8, 0x25, 0x34, 0x7b, 0xbd, 0xc3, 0x14, 0xad, 0x3b, 0xb0, 0x66, 0x59, 0xce, 0x20, 0x8d, 0xa5,
|
||||
0x70, 0xac, 0xa6, 0x65, 0x39, 0x49, 0x7d, 0x41, 0xef, 0x42, 0x2b, 0xa4, 0x83, 0x22, 0x71, 0x3d,
|
||||
0xa4, 0x09, 0x94, 0x71, 0x04, 0x2d, 0x2e, 0x2c, 0x37, 0xea, 0x1c, 0x59, 0x6f, 0x83, 0x9e, 0x22,
|
||||
0x27, 0xdc, 0xa7, 0x81, 0xb5, 0x44, 0x58, 0x5e, 0x41, 0xa2, 0x76, 0x30, 0xa1, 0x38, 0xbb, 0x1d,
|
||||
0xbc, 0x09, 0x60, 0xd3, 0x81, 0x74, 0x7a, 0x2e, 0xe3, 0x2a, 0x6e, 0xd8, 0xf4, 0x89, 0xb8, 0x40,
|
||||
0x3f, 0x80, 0x65, 0xce, 0x5f, 0x84, 0x47, 0x21, 0x49, 0x71, 0x6b, 0x64, 0x5f, 0x80, 0x25, 0x82,
|
||||
0xf1, 0x09, 0xe8, 0xbd, 0xde, 0x61, 0x22, 0x47, 0x95, 0x7c, 0x52, 0xe1, 0x8d, 0x5f, 0x42, 0x2b,
|
||||
0x29, 0x4a, 0x3c, 0x51, 0xb5, 0xa0, 0x16, 0x93, 0xab, 0xf5, 0x7b, 0xe8, 0x23, 0x58, 0x16, 0xa3,
|
||||
0xae, 0xf4, 0xa0, 0xf7, 0xb2, 0x32, 0xcb, 0x31, 0x38, 0x55, 0xd9, 0xf8, 0x05, 0x96, 0x48, 0xcc,
|
||||
0xc3, 0xe3, 0x44, 0x2e, 0x86, 0x36, 0x15, 0xa7, 0x6e, 0x8c, 0xff, 0xa8, 0xa0, 0xa5, 0x1c, 0xb0,
|
||||
0xc0, 0x3e, 0xff, 0xce, 0xda, 0xfc, 0xfa, 0xa1, 0x16, 0x27, 0xa8, 0xf7, 0xa0, 0x65, 0xf3, 0x9e,
|
||||
0x65, 0x20, 0xa3, 0x9f, 0x17, 0x99, 0x06, 0x6e, 0x8a, 0x5b, 0x99, 0x8a, 0xd0, 0x36, 0x68, 0xde,
|
||||
0xc4, 0x1d, 0xf8, 0x67, 0x83, 0xc0, 0x7f, 0x41, 0xe5, 0x28, 0xd6, 0xf0, 0x26, 0xee, 0xcf, 0xce,
|
||||
0xb0, 0xff, 0x82, 0x26, 0xdd, 0xfe, 0xf2, 0x6b, 0x76, 0xfb, 0x8f, 0x41, 0xb7, 0x5c, 0x27, 0x49,
|
||||
0xdb, 0x2b, 0xd5, 0x5b, 0x74, 0xcb, 0x75, 0xe2, 0xac, 0xbd, 0x0d, 0x9a, 0x6b, 0x5e, 0x32, 0xe1,
|
||||
0x06, 0xde, 0xc4, 0x6d, 0xaf, 0x0a, 0xf9, 0x5c, 0xf3, 0x12, 0xfb, 0x2f, 0x9e, 0x4e, 0x5c, 0xb4,
|
||||
0x0b, 0xeb, 0x8e, 0x49, 0xc3, 0x41, 0x7a, 0x5a, 0x6c, 0xf0, 0x69, 0xb1, 0xc5, 0xee, 0x1f, 0x27,
|
||||
0x13, 0x63, 0x71, 0xfc, 0x80, 0x05, 0xc7, 0x0f, 0xe3, 0x3e, 0x68, 0xfd, 0x5e, 0x97, 0xb9, 0x13,
|
||||
0xeb, 0xd9, 0x0a, 0x06, 0xdc, 0x84, 0xfa, 0x71, 0xca, 0xfb, 0xea, 0x91, 0xdf, 0x6d, 0x26, 0x7a,
|
||||
0x4a, 0xcd, 0x32, 0x45, 0xb9, 0x94, 0x45, 0xc7, 0xa2, 0xd9, 0x9d, 0xec, 0x6f, 0x54, 0xd8, 0x3a,
|
||||
0x31, 0x2f, 0xc8, 0xdb, 0x6f, 0x9a, 0x2b, 0x15, 0x82, 0x43, 0xd8, 0xe0, 0x81, 0xde, 0x4d, 0xc9,
|
||||
0x33, 0xa3, 0x1e, 0xa7, 0x14, 0x8e, 0x8b, 0x88, 0xe8, 0x27, 0xac, 0x91, 0x20, 0xc3, 0xe7, 0xc7,
|
||||
0xbe, 0x1d, 0xd5, 0x62, 0xad, 0x7b, 0xb3, 0x84, 0xce, 0xa3, 0x18, 0x0a, 0xa7, 0x31, 0xd0, 0x31,
|
||||
0xac, 0x65, 0xcd, 0x40, 0xdb, 0xcb, 0x9c, 0xc8, 0xfb, 0x33, 0xa7, 0xb1, 0x44, 0xfb, 0xb8, 0x95,
|
||||
0x31, 0x06, 0xe5, 0x99, 0x58, 0xa6, 0xc5, 0x15, 0x9e, 0x16, 0xa3, 0x23, 0x4b, 0xb3, 0x90, 0xc8,
|
||||
0x31, 0x27, 0xc1, 0xfe, 0x18, 0x56, 0x63, 0xcf, 0xa8, 0x55, 0xf6, 0x8c, 0x18, 0x27, 0x1f, 0xe1,
|
||||
0x6a, 0x2e, 0xc2, 0x8d, 0x57, 0x0a, 0x34, 0x7b, 0x66, 0x68, 0x3e, 0xf5, 0x2d, 0xf2, 0x6c, 0xc1,
|
||||
0xa2, 0x5b, 0x61, 0x5b, 0x74, 0x03, 0x1a, 0x2c, 0x38, 0x69, 0x68, 0xba, 0x63, 0x2e, 0xc4, 0x12,
|
||||
0x4e, 0x2e, 0xd8, 0x68, 0xd9, 0x94, 0x29, 0x49, 0xb4, 0x9d, 0xac, 0xbf, 0xe0, 0xa4, 0x44, 0x71,
|
||||
0xe4, 0xbf, 0xd1, 0x0f, 0xb3, 0xab, 0x87, 0x77, 0x4b, 0xcd, 0xcb, 0x89, 0xf0, 0x86, 0x2b, 0x93,
|
||||
0x8f, 0xaa, 0xcc, 0x2c, 0x5f, 0x29, 0xa0, 0x47, 0xaa, 0xe0, 0xa9, 0xb9, 0x0d, 0x2b, 0xa6, 0x65,
|
||||
0x05, 0x84, 0x52, 0x29, 0x47, 0x74, 0x64, 0x5f, 0x2e, 0x48, 0x40, 0x23, 0xa3, 0xa8, 0x38, 0x3a,
|
||||
0xa2, 0x1f, 0xc1, 0x6a, 0xdc, 0xa1, 0x89, 0x8d, 0xdd, 0xce, 0x74, 0x39, 0x65, 0x8f, 0x1d, 0x63,
|
||||
0x18, 0x01, 0xb4, 0xa4, 0x73, 0x09, 0xef, 0xa6, 0x73, 0xbc, 0xe3, 0x21, 0xe8, 0x67, 0x49, 0xb7,
|
||||
0x32, 0x6b, 0x94, 0x4e, 0x35, 0x35, 0x38, 0x83, 0x63, 0x7c, 0x0c, 0x5a, 0xea, 0xe3, 0x8c, 0x0e,
|
||||
0xa2, 0x0d, 0x2b, 0xa7, 0x29, 0x3e, 0x0d, 0x1c, 0x1d, 0x8d, 0xff, 0x2a, 0x7c, 0x6b, 0x85, 0xc9,
|
||||
0xd0, 0xbf, 0x20, 0xc1, 0xcb, 0x37, 0xdf, 0x0d, 0x3c, 0x48, 0x69, 0xb1, 0x62, 0x9f, 0x1b, 0x23,
|
||||
0xa0, 0x07, 0x89, 0x9c, 0xea, 0xd4, 0xae, 0x23, 0xab, 0xe6, 0xe4, 0x29, 0x7f, 0x12, 0x5b, 0x8e,
|
||||
0xec, 0x53, 0x16, 0x4d, 0x93, 0xff, 0x97, 0x5a, 0x7e, 0xf7, 0x1e, 0x6c, 0x14, 0xfc, 0x1a, 0xb5,
|
||||
0x00, 0x3e, 0xf1, 0x86, 0xbe, 0x3b, 0x76, 0x48, 0x48, 0xd6, 0xaf, 0x21, 0x1d, 0x56, 0x1f, 0x45,
|
||||
0x27, 0xa5, 0xfb, 0x4f, 0x0d, 0x34, 0xe6, 0xca, 0x27, 0xe2, 0x1f, 0x00, 0x34, 0x06, 0xc4, 0xa7,
|
||||
0x71, 0x77, 0xec, 0x7b, 0xf1, 0xda, 0x0a, 0x7d, 0x38, 0x25, 0x93, 0x14, 0x41, 0xa5, 0x0a, 0x3a,
|
||||
0x77, 0xa6, 0x60, 0xe4, 0xc0, 0x8d, 0x6b, 0xc8, 0xe5, 0x1c, 0x59, 0xe9, 0x7d, 0x66, 0x0f, 0x9f,
|
||||
0x47, 0xfd, 0xc6, 0x0c, 0x8e, 0x39, 0xd0, 0x88, 0x63, 0x6e, 0x1b, 0x26, 0x0f, 0x62, 0x65, 0x12,
|
||||
0xf9, 0x98, 0x71, 0x0d, 0x7d, 0x01, 0x9b, 0x6c, 0x3c, 0x8d, 0xa7, 0xe4, 0x88, 0x61, 0x77, 0x3a,
|
||||
0xc3, 0x02, 0xf0, 0x6b, 0xb2, 0x3c, 0x80, 0x3a, 0xef, 0x4e, 0x51, 0x99, 0x6b, 0xa6, 0xff, 0x1c,
|
||||
0xe9, 0xcc, 0x72, 0x79, 0xe3, 0x1a, 0xfa, 0x15, 0xac, 0xe5, 0xd6, 0xd2, 0xe8, 0x83, 0x12, 0x92,
|
||||
0xe5, 0x7f, 0x30, 0x74, 0xee, 0x56, 0x01, 0x8d, 0x85, 0x1e, 0x41, 0x2b, 0x3b, 0xc6, 0xa3, 0xdd,
|
||||
0x12, 0xfc, 0xd2, 0x95, 0x62, 0xe7, 0x83, 0x0a, 0x90, 0x31, 0x23, 0x17, 0xd6, 0xf3, 0x6b, 0x52,
|
||||
0x74, 0x77, 0x26, 0x81, 0xac, 0xa7, 0x7d, 0xbb, 0x12, 0x6c, 0xcc, 0xee, 0x25, 0xb7, 0x7f, 0x61,
|
||||
0x4d, 0x87, 0xf6, 0xca, 0xc9, 0x4c, 0xdb, 0x1f, 0x76, 0xf6, 0x2b, 0xc3, 0xc7, 0xac, 0x5f, 0x89,
|
||||
0xe4, 0x57, 0xb6, 0xea, 0x42, 0xf7, 0xca, 0xc9, 0xcd, 0xd8, 0xd1, 0x75, 0xba, 0xaf, 0x83, 0x12,
|
||||
0x0b, 0xf1, 0x25, 0xcf, 0x5a, 0x25, 0xeb, 0xa2, 0x7c, 0xc8, 0x45, 0xf4, 0xa6, 0xef, 0xc1, 0x3a,
|
||||
0xf7, 0x5e, 0x03, 0x23, 0x16, 0xc0, 0xcf, 0x2f, 0xa2, 0xa3, 0x08, 0xdc, 0x9f, 0xeb, 0x35, 0x8b,
|
||||
0x85, 0xdf, 0xe7, 0xb0, 0x96, 0x6b, 0x67, 0x4b, 0xa3, 0xa6, 0xbc, 0xe5, 0xad, 0x10, 0x92, 0xb9,
|
||||
0x22, 0x80, 0xa6, 0x78, 0x7f, 0x49, 0xa1, 0xe8, 0xdc, 0xad, 0x02, 0x1a, 0x3d, 0xa4, 0xfb, 0x77,
|
||||
0x15, 0x56, 0xa3, 0xb6, 0xe3, 0x0a, 0x12, 0xf5, 0x15, 0x64, 0xce, 0xcf, 0x61, 0x2d, 0xb7, 0x12,
|
||||
0x2b, 0xd5, 0x6e, 0xf9, 0xda, 0x6c, 0x9e, 0xe9, 0x3e, 0x83, 0x66, 0x66, 0xc7, 0x85, 0xde, 0x9f,
|
||||
0x96, 0x9e, 0x73, 0x5b, 0xb0, 0x39, 0x84, 0x1f, 0xde, 0xff, 0xc5, 0xbd, 0x91, 0x1d, 0x9e, 0x4f,
|
||||
0x4e, 0xd9, 0x97, 0x7d, 0x01, 0xfa, 0x1d, 0xdb, 0x97, 0xbf, 0xf6, 0x23, 0x05, 0xed, 0x73, 0xec,
|
||||
0x7d, 0xc6, 0x66, 0x7c, 0x7a, 0xba, 0xcc, 0x4f, 0xf7, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xf7,
|
||||
0x57, 0x50, 0x90, 0x91, 0x1f, 0x00, 0x00,
|
||||
0xb9, 0xe3, 0x89, 0x93, 0xf8, 0xf3, 0xd8, 0x49, 0x0e, 0x21, 0x6b, 0xb9, 0x6d, 0x9a, 0x0e, 0xbb,
|
||||
0x5d, 0x6f, 0x11, 0xc9, 0xd6, 0x45, 0xe2, 0x52, 0x16, 0xb4, 0x8d, 0xdb, 0xc8, 0x22, 0x29, 0xe1,
|
||||
0xa4, 0xbb, 0x2b, 0xb1, 0x42, 0xd6, 0xd8, 0x73, 0xe2, 0x0c, 0x9d, 0x8b, 0x77, 0xce, 0x38, 0x4d,
|
||||
0x9f, 0x76, 0x55, 0x1e, 0x10, 0x08, 0x71, 0x79, 0xe0, 0x1d, 0x21, 0x21, 0x90, 0x78, 0xe1, 0x1f,
|
||||
0xf0, 0xca, 0xcf, 0x42, 0xe7, 0x32, 0xf7, 0xb1, 0x3d, 0x75, 0xe9, 0xf6, 0xcd, 0xe7, 0xcc, 0x77,
|
||||
0x3b, 0xdf, 0xfd, 0xfb, 0x0c, 0xc8, 0x34, 0x02, 0x63, 0x40, 0x89, 0x7f, 0x69, 0x8d, 0xc8, 0xfe,
|
||||
0xc4, 0xf7, 0x02, 0x0f, 0x6d, 0x39, 0x96, 0x7d, 0x39, 0xa5, 0xe2, 0xb4, 0xcf, 0x00, 0xda, 0xda,
|
||||
0xc8, 0x73, 0x1c, 0xcf, 0x15, 0x57, 0xed, 0xa6, 0xe5, 0x06, 0xc4, 0x77, 0x0d, 0x5b, 0x9e, 0xb5,
|
||||
0x24, 0x42, 0x5b, 0xa3, 0xa3, 0x0b, 0xe2, 0x18, 0xe2, 0xa4, 0x5f, 0x81, 0xf6, 0xd8, 0x9e, 0xd2,
|
||||
0x0b, 0x4c, 0xbe, 0x98, 0x12, 0x1a, 0xa0, 0x0f, 0x61, 0x65, 0x68, 0x50, 0xd2, 0x52, 0xf6, 0x94,
|
||||
0x4e, 0xbd, 0x7b, 0x63, 0x3f, 0xc5, 0x4b, 0x72, 0x39, 0xa1, 0xe3, 0x87, 0x06, 0x25, 0x98, 0x43,
|
||||
0x22, 0x04, 0x2b, 0xe6, 0xb0, 0xdf, 0x6b, 0x55, 0xf6, 0x94, 0x8e, 0x8a, 0xf9, 0x6f, 0xa4, 0x83,
|
||||
0x36, 0xf2, 0x6c, 0x9b, 0x8c, 0x02, 0xcb, 0x73, 0xfb, 0xbd, 0xd6, 0x0a, 0xff, 0x96, 0xba, 0xd3,
|
||||
0xff, 0xac, 0xc0, 0xe6, 0x19, 0x19, 0x3b, 0xc4, 0x0d, 0xfa, 0xbd, 0x90, 0xfd, 0x36, 0x54, 0x47,
|
||||
0xde, 0xd4, 0x0d, 0x38, 0xff, 0x06, 0x16, 0x07, 0x74, 0x1b, 0xb4, 0xd1, 0x85, 0xe1, 0xba, 0xc4,
|
||||
0x1e, 0xb8, 0x86, 0x43, 0x38, 0xab, 0x1a, 0xae, 0xcb, 0xbb, 0x27, 0x86, 0x43, 0x72, 0x1c, 0xd5,
|
||||
0x3c, 0x47, 0xb4, 0x07, 0xf5, 0x89, 0xe1, 0x07, 0x56, 0x4a, 0xa8, 0xe4, 0x95, 0xfe, 0x57, 0x05,
|
||||
0x76, 0x3e, 0xa6, 0xd4, 0x1a, 0xbb, 0x39, 0xc9, 0x76, 0x60, 0xd5, 0xf5, 0x4c, 0xd2, 0xef, 0x71,
|
||||
0xd1, 0x54, 0x2c, 0x4f, 0xe8, 0x3a, 0xd4, 0x26, 0x84, 0xf8, 0x03, 0xdf, 0xb3, 0x43, 0xc1, 0xd6,
|
||||
0xd9, 0x05, 0xf6, 0x6c, 0x82, 0x7e, 0x0e, 0x5b, 0x34, 0x43, 0x88, 0xb6, 0xd4, 0x3d, 0xb5, 0x53,
|
||||
0xef, 0x7e, 0x6b, 0x3f, 0x67, 0xc6, 0xfd, 0x2c, 0x53, 0x9c, 0xc7, 0xd6, 0xbf, 0xaa, 0xc0, 0x37,
|
||||
0x22, 0x38, 0x21, 0x2b, 0xfb, 0xcd, 0x34, 0x47, 0xc9, 0x38, 0x12, 0x4f, 0x1c, 0xca, 0x68, 0x2e,
|
||||
0x52, 0xb9, 0x9a, 0x54, 0x79, 0x09, 0x0b, 0x66, 0xf5, 0x59, 0xcd, 0xe9, 0x13, 0xdd, 0x82, 0x3a,
|
||||
0xb9, 0x9a, 0x58, 0x3e, 0x19, 0x04, 0x96, 0x43, 0x5a, 0xab, 0x7b, 0x4a, 0x67, 0x05, 0x83, 0xb8,
|
||||
0x7a, 0x6a, 0x39, 0x04, 0xdd, 0x87, 0x55, 0x1a, 0x18, 0xc1, 0x94, 0xb6, 0xd6, 0xb8, 0xc3, 0x5d,
|
||||
0x2f, 0x74, 0xb8, 0x33, 0x0e, 0x82, 0x25, 0xa8, 0xfe, 0x37, 0x05, 0xde, 0xc9, 0x59, 0x89, 0x4e,
|
||||
0x3c, 0x97, 0x12, 0x84, 0x61, 0x93, 0xbf, 0x3c, 0xd6, 0x0c, 0x6d, 0x29, 0x5c, 0xe1, 0x77, 0xe6,
|
||||
0x29, 0x3c, 0x06, 0xc7, 0x39, 0xfc, 0x84, 0x90, 0x95, 0xf2, 0x42, 0x3e, 0x83, 0x77, 0x8e, 0x48,
|
||||
0x20, 0x19, 0xb0, 0x6f, 0x84, 0x2e, 0x1f, 0x63, 0xbb, 0x00, 0x91, 0x27, 0x30, 0x29, 0xd4, 0x8e,
|
||||
0x8a, 0x13, 0x37, 0xfa, 0xbf, 0x2b, 0x51, 0x2c, 0x71, 0x56, 0x7d, 0xf7, 0xdc, 0x43, 0x37, 0xa0,
|
||||
0x16, 0x81, 0x48, 0xaf, 0x88, 0x2f, 0xd0, 0xf7, 0xa0, 0xca, 0x24, 0x15, 0x2e, 0xd1, 0xec, 0xde,
|
||||
0x2e, 0x7e, 0x53, 0x82, 0x26, 0x16, 0xf0, 0xa8, 0x0f, 0x4d, 0x1a, 0x18, 0x7e, 0x30, 0x98, 0x78,
|
||||
0x94, 0xdb, 0x99, 0x3b, 0x4e, 0xbd, 0xab, 0xa7, 0x29, 0x44, 0x39, 0xe8, 0x84, 0x8e, 0x4f, 0x25,
|
||||
0x24, 0x6e, 0x70, 0xcc, 0xf0, 0x88, 0x1e, 0x81, 0x46, 0x5c, 0x33, 0x26, 0xb4, 0x52, 0x9a, 0x50,
|
||||
0x9d, 0xb8, 0x66, 0x44, 0x26, 0xb6, 0x4f, 0xb5, 0xbc, 0x7d, 0x7e, 0xaf, 0x40, 0x2b, 0x6f, 0x20,
|
||||
0xe9, 0x45, 0x31, 0x45, 0xa5, 0x34, 0x45, 0xf4, 0x40, 0x20, 0x11, 0x61, 0xa0, 0xb9, 0x11, 0x1e,
|
||||
0x19, 0x09, 0x4b, 0x14, 0xdd, 0x82, 0x6f, 0xc6, 0xd2, 0xf0, 0x2f, 0x6f, 0xcc, 0x59, 0x7e, 0xad,
|
||||
0xc0, 0x4e, 0x96, 0xd7, 0xeb, 0xbc, 0xfb, 0xbb, 0x50, 0xb5, 0xdc, 0x73, 0x2f, 0x7c, 0xf6, 0xee,
|
||||
0x9c, 0x38, 0x63, 0xbc, 0x04, 0xb0, 0xee, 0xc0, 0xf5, 0x23, 0x12, 0xf4, 0x5d, 0x4a, 0xfc, 0xe0,
|
||||
0xa1, 0xe5, 0xda, 0xde, 0xf8, 0xd4, 0x08, 0x2e, 0x5e, 0x23, 0x46, 0x52, 0xee, 0x5e, 0xc9, 0xb8,
|
||||
0xbb, 0xfe, 0x4f, 0x05, 0x6e, 0x14, 0xf3, 0x93, 0x4f, 0x6f, 0xc3, 0xfa, 0xb9, 0x45, 0x6c, 0x93,
|
||||
0xe9, 0x4c, 0xe1, 0x3a, 0x8b, 0xce, 0x2c, 0x56, 0x26, 0x0c, 0x58, 0xbe, 0xf0, 0xf6, 0x0c, 0x07,
|
||||
0x3d, 0x0b, 0x7c, 0xcb, 0x1d, 0x1f, 0x5b, 0x34, 0xc0, 0x02, 0x3e, 0xa1, 0x4f, 0xb5, 0xbc, 0x67,
|
||||
0xfe, 0x4e, 0x81, 0xdd, 0x23, 0x12, 0x1c, 0x46, 0xa9, 0x96, 0x7d, 0xb7, 0x68, 0x60, 0x8d, 0xe8,
|
||||
0x9b, 0xad, 0xd2, 0x05, 0x35, 0x53, 0xff, 0xa3, 0x02, 0xb7, 0x66, 0x0a, 0x23, 0x55, 0x27, 0x53,
|
||||
0x49, 0x98, 0x68, 0x8b, 0x53, 0xc9, 0x4f, 0xc9, 0x8b, 0x4f, 0x0d, 0x7b, 0x4a, 0x4e, 0x0d, 0xcb,
|
||||
0x17, 0xa9, 0x64, 0xc9, 0xc4, 0xfa, 0x2f, 0x05, 0x6e, 0x1e, 0x91, 0xe0, 0x34, 0x2c, 0x33, 0x6f,
|
||||
0x51, 0x3b, 0x25, 0x3a, 0x8a, 0x3f, 0x08, 0x63, 0x16, 0x4a, 0xfb, 0x56, 0xd4, 0xb7, 0xcb, 0xe3,
|
||||
0x20, 0x11, 0x90, 0x87, 0xa2, 0x17, 0x90, 0xca, 0xd3, 0xff, 0x52, 0x01, 0xed, 0x53, 0xd9, 0x1f,
|
||||
0xf0, 0x32, 0x92, 0xd5, 0x83, 0x52, 0xac, 0x87, 0x44, 0x4b, 0x51, 0xd4, 0x65, 0x1c, 0x41, 0x83,
|
||||
0x12, 0xf2, 0x6c, 0x99, 0xa2, 0xa1, 0x31, 0xc4, 0x28, 0xd9, 0x1f, 0xc3, 0xd6, 0xd4, 0x3d, 0x67,
|
||||
0x2d, 0x2b, 0x31, 0xe5, 0x2b, 0x68, 0x6b, 0xa5, 0x54, 0xe6, 0xc9, 0x23, 0xa2, 0x0e, 0x6c, 0x64,
|
||||
0x69, 0x55, 0x79, 0xf0, 0x67, 0xaf, 0xf5, 0xdf, 0x2a, 0xb0, 0xf3, 0x99, 0x11, 0x8c, 0x2e, 0x7a,
|
||||
0x8e, 0xd4, 0xd8, 0x6b, 0xf8, 0xdb, 0x47, 0x50, 0xbb, 0x94, 0xda, 0x09, 0x93, 0xca, 0xad, 0x02,
|
||||
0xe1, 0x93, 0x76, 0xc0, 0x31, 0x06, 0x6b, 0x53, 0xb7, 0x79, 0xd7, 0x1e, 0x4a, 0xf7, 0xf5, 0x7b,
|
||||
0x7e, 0xba, 0xc8, 0xac, 0xe4, 0x8a, 0xcc, 0x15, 0x80, 0x14, 0xee, 0x84, 0x8e, 0x97, 0x90, 0xeb,
|
||||
0xfb, 0xb0, 0x26, 0xa9, 0x49, 0xe7, 0x5e, 0x64, 0xdc, 0x10, 0x5c, 0x3f, 0x83, 0x1d, 0x79, 0xff,
|
||||
0x98, 0xe5, 0x6f, 0x91, 0xeb, 0x4f, 0x48, 0x60, 0xa0, 0x16, 0xac, 0xc9, 0x94, 0x2e, 0x9d, 0x38,
|
||||
0x3c, 0xb2, 0x3e, 0x75, 0xc8, 0xe1, 0x06, 0x2c, 0x6f, 0x4b, 0xff, 0x85, 0x61, 0x54, 0x26, 0xf4,
|
||||
0x5f, 0x42, 0xa3, 0xd7, 0x3b, 0x4e, 0xd0, 0xba, 0x03, 0x1b, 0xa6, 0x69, 0x0f, 0x92, 0x58, 0x0a,
|
||||
0xc7, 0x6a, 0x98, 0xa6, 0x1d, 0xd7, 0x17, 0xf4, 0x2e, 0x34, 0x03, 0x3a, 0xc8, 0x13, 0xd7, 0x02,
|
||||
0x1a, 0x43, 0xe9, 0x27, 0xd0, 0xe4, 0xc2, 0x72, 0xa3, 0x2e, 0x90, 0xf5, 0x36, 0x68, 0x09, 0x72,
|
||||
0xc2, 0x7d, 0x6a, 0xb8, 0x1e, 0x0b, 0xcb, 0x2b, 0x48, 0xd8, 0x0e, 0xc6, 0x14, 0xe7, 0xb7, 0x83,
|
||||
0x37, 0x01, 0x2c, 0x3a, 0x90, 0x4e, 0xcf, 0x65, 0x5c, 0xc7, 0x35, 0x8b, 0x3e, 0x16, 0x17, 0xe8,
|
||||
0x07, 0xb0, 0xca, 0xf9, 0x8b, 0xf0, 0xc8, 0x25, 0x29, 0x6e, 0x8d, 0xf4, 0x0b, 0xb0, 0x44, 0xd0,
|
||||
0x3f, 0x01, 0xad, 0xd7, 0x3b, 0x8e, 0xe5, 0x28, 0x93, 0x4f, 0x4a, 0xbc, 0xf1, 0x4b, 0x68, 0xc6,
|
||||
0x45, 0x89, 0x27, 0xaa, 0x26, 0x54, 0x22, 0x72, 0x95, 0x7e, 0x0f, 0x7d, 0x04, 0xab, 0x62, 0xd4,
|
||||
0x95, 0x1e, 0xf4, 0x5e, 0x5a, 0x66, 0x39, 0x06, 0x27, 0x2a, 0x1b, 0xbf, 0xc0, 0x12, 0x89, 0x79,
|
||||
0x78, 0x94, 0xc8, 0xc5, 0xd0, 0xa6, 0xe2, 0xc4, 0x8d, 0xfe, 0x1f, 0x15, 0xea, 0x09, 0x07, 0xcc,
|
||||
0xb1, 0xcf, 0xbe, 0xb3, 0xb2, 0xb8, 0x7e, 0xa8, 0xf9, 0x09, 0xea, 0x3d, 0x68, 0x5a, 0xbc, 0x67,
|
||||
0x19, 0xc8, 0xe8, 0xe7, 0x45, 0xa6, 0x86, 0x1b, 0xe2, 0x56, 0xa6, 0x22, 0xb4, 0x0b, 0x75, 0x77,
|
||||
0xea, 0x0c, 0xbc, 0xf3, 0x81, 0xef, 0x3d, 0xa7, 0x72, 0x14, 0xab, 0xb9, 0x53, 0xe7, 0x67, 0xe7,
|
||||
0xd8, 0x7b, 0x4e, 0xe3, 0x6e, 0x7f, 0xf5, 0x15, 0xbb, 0xfd, 0x47, 0xa0, 0x99, 0x8e, 0x1d, 0xa7,
|
||||
0xed, 0xb5, 0xf2, 0x2d, 0xba, 0xe9, 0xd8, 0x51, 0xd6, 0xde, 0x85, 0xba, 0x63, 0x5c, 0x31, 0xe1,
|
||||
0x06, 0xee, 0xd4, 0x69, 0xad, 0x0b, 0xf9, 0x1c, 0xe3, 0x0a, 0x7b, 0xcf, 0x9f, 0x4c, 0x1d, 0xd4,
|
||||
0x81, 0x4d, 0xdb, 0xa0, 0xc1, 0x20, 0x39, 0x2d, 0xd6, 0xf8, 0xb4, 0xd8, 0x64, 0xf7, 0x8f, 0xe2,
|
||||
0x89, 0x31, 0x3f, 0x7e, 0xc0, 0x92, 0xe3, 0x87, 0x7e, 0x1f, 0xea, 0xfd, 0x5e, 0x97, 0xb9, 0x13,
|
||||
0xeb, 0xd9, 0x72, 0x06, 0xdc, 0x86, 0xea, 0x69, 0xc2, 0xfb, 0xaa, 0xa1, 0xdf, 0x6d, 0xc7, 0x7a,
|
||||
0x4a, 0xcc, 0x32, 0x79, 0xb9, 0x94, 0x65, 0xc7, 0xa2, 0xf9, 0x9d, 0xec, 0x6f, 0x54, 0xd8, 0x39,
|
||||
0x33, 0x2e, 0xc9, 0x9b, 0x6f, 0x9a, 0x4b, 0x15, 0x82, 0x63, 0xd8, 0xe2, 0x81, 0xde, 0x4d, 0xc8,
|
||||
0x33, 0xa7, 0x1e, 0x27, 0x14, 0x8e, 0xf3, 0x88, 0xe8, 0x27, 0xac, 0x91, 0x20, 0xa3, 0x67, 0xa7,
|
||||
0x9e, 0x15, 0xd6, 0xe2, 0x7a, 0xf7, 0x66, 0x01, 0x9d, 0xc3, 0x08, 0x0a, 0x27, 0x31, 0xd0, 0x29,
|
||||
0x6c, 0xa4, 0xcd, 0x40, 0x5b, 0xab, 0x9c, 0xc8, 0xfb, 0x73, 0xa7, 0xb1, 0x58, 0xfb, 0xb8, 0x99,
|
||||
0x32, 0x06, 0xe5, 0x99, 0x58, 0xa6, 0xc5, 0x35, 0x9e, 0x16, 0xc3, 0x23, 0x4b, 0xb3, 0x10, 0xcb,
|
||||
0xb1, 0x20, 0xc1, 0xfe, 0x18, 0xd6, 0x23, 0xcf, 0xa8, 0x94, 0xf6, 0x8c, 0x08, 0x27, 0x1b, 0xe1,
|
||||
0x6a, 0x26, 0xc2, 0xf5, 0x97, 0x0a, 0x34, 0x7a, 0x46, 0x60, 0x3c, 0xf1, 0x4c, 0xf2, 0x74, 0xc9,
|
||||
0xa2, 0x5b, 0x62, 0x5b, 0x74, 0x03, 0x6a, 0x2c, 0x38, 0x69, 0x60, 0x38, 0x13, 0x2e, 0xc4, 0x0a,
|
||||
0x8e, 0x2f, 0xd8, 0x68, 0xd9, 0x90, 0x29, 0x49, 0xb4, 0x9d, 0xac, 0xbf, 0xe0, 0xa4, 0x44, 0x71,
|
||||
0xe4, 0xbf, 0xd1, 0x0f, 0xd3, 0xab, 0x87, 0x77, 0x0b, 0xcd, 0xcb, 0x89, 0xf0, 0x86, 0x2b, 0x95,
|
||||
0x8f, 0xca, 0xcc, 0x2c, 0x5f, 0x29, 0xa0, 0x85, 0xaa, 0xe0, 0xa9, 0xb9, 0x05, 0x6b, 0x86, 0x69,
|
||||
0xfa, 0x84, 0x52, 0x29, 0x47, 0x78, 0x64, 0x5f, 0x2e, 0x89, 0x4f, 0x43, 0xa3, 0xa8, 0x38, 0x3c,
|
||||
0xa2, 0x1f, 0xc1, 0x7a, 0xd4, 0xa1, 0x89, 0x8d, 0xdd, 0xde, 0x6c, 0x39, 0x65, 0x8f, 0x1d, 0x61,
|
||||
0xe8, 0x3e, 0x34, 0xa5, 0x73, 0x09, 0xef, 0xa6, 0x0b, 0xbc, 0xe3, 0x21, 0x68, 0xe7, 0x71, 0xb7,
|
||||
0x32, 0x6f, 0x94, 0x4e, 0x34, 0x35, 0x38, 0x85, 0xa3, 0x7f, 0x0c, 0xf5, 0xc4, 0xc7, 0x39, 0x1d,
|
||||
0x44, 0x0b, 0xd6, 0x86, 0x09, 0x3e, 0x35, 0x1c, 0x1e, 0xf5, 0xff, 0x2a, 0x7c, 0x6b, 0x85, 0xc9,
|
||||
0xc8, 0xbb, 0x24, 0xfe, 0x8b, 0xd7, 0xdf, 0x0d, 0x3c, 0x48, 0x68, 0xb1, 0x64, 0x9f, 0x1b, 0x21,
|
||||
0xa0, 0x07, 0xb1, 0x9c, 0xea, 0xcc, 0xae, 0x23, 0xad, 0xe6, 0xf8, 0x29, 0x7f, 0x12, 0x5b, 0x8e,
|
||||
0xf4, 0x53, 0x96, 0x4d, 0x93, 0xff, 0x97, 0x5a, 0x7e, 0xf7, 0x1e, 0x6c, 0xe5, 0xfc, 0x1a, 0x35,
|
||||
0x01, 0x3e, 0x71, 0x47, 0x9e, 0x33, 0xb1, 0x49, 0x40, 0x36, 0xaf, 0x21, 0x0d, 0xd6, 0x0f, 0xc3,
|
||||
0x93, 0xd2, 0xfd, 0x47, 0x1d, 0x6a, 0xcc, 0x95, 0x0f, 0x3d, 0xcf, 0x37, 0xd1, 0x04, 0x10, 0x9f,
|
||||
0xc5, 0x9d, 0x89, 0xe7, 0x46, 0x4b, 0x2b, 0xf4, 0xe1, 0x8c, 0x3c, 0x92, 0x07, 0x95, 0x0a, 0x68,
|
||||
0xdf, 0x99, 0x81, 0x91, 0x01, 0xd7, 0xaf, 0x21, 0x87, 0x73, 0x64, 0x85, 0xf7, 0xa9, 0x35, 0x7a,
|
||||
0x16, 0x76, 0x1b, 0x73, 0x38, 0x66, 0x40, 0x43, 0x8e, 0x99, 0x5d, 0x98, 0x3c, 0x88, 0x85, 0x49,
|
||||
0xe8, 0x61, 0xfa, 0x35, 0xf4, 0x05, 0x6c, 0xb3, 0xe1, 0x34, 0x9a, 0x91, 0x43, 0x86, 0xdd, 0xd9,
|
||||
0x0c, 0x73, 0xc0, 0xaf, 0xc8, 0xf2, 0x08, 0xaa, 0xbc, 0x37, 0x45, 0x45, 0x8e, 0x99, 0xfc, 0x6b,
|
||||
0xa4, 0x3d, 0xcf, 0xe1, 0xf5, 0x6b, 0xe8, 0x57, 0xb0, 0x91, 0x59, 0x4a, 0xa3, 0x0f, 0x0a, 0x48,
|
||||
0x16, 0xff, 0xbd, 0xd0, 0xbe, 0x5b, 0x06, 0x34, 0x12, 0x7a, 0x0c, 0xcd, 0xf4, 0x10, 0x8f, 0x3a,
|
||||
0x05, 0xf8, 0x85, 0x0b, 0xc5, 0xf6, 0x07, 0x25, 0x20, 0x23, 0x46, 0x0e, 0x6c, 0x66, 0x97, 0xa4,
|
||||
0xe8, 0xee, 0x5c, 0x02, 0x69, 0x4f, 0xfb, 0x76, 0x29, 0xd8, 0x88, 0xdd, 0x0b, 0x6e, 0xff, 0xdc,
|
||||
0x92, 0x0e, 0xed, 0x17, 0x93, 0x99, 0xb5, 0x3d, 0x6c, 0x1f, 0x94, 0x86, 0x8f, 0x58, 0xbf, 0x14,
|
||||
0xa9, 0xaf, 0x68, 0xd1, 0x85, 0xee, 0x15, 0x93, 0x9b, 0xb3, 0xa1, 0x6b, 0x77, 0x5f, 0x05, 0x25,
|
||||
0x12, 0xe2, 0x4b, 0x9e, 0xb3, 0x0a, 0x96, 0x45, 0xd9, 0x90, 0x0b, 0xe9, 0xcd, 0xde, 0x82, 0xb5,
|
||||
0xef, 0xbd, 0x02, 0x46, 0x24, 0x80, 0x97, 0x5d, 0x43, 0x87, 0x11, 0x78, 0xb0, 0xd0, 0x6b, 0x96,
|
||||
0x0b, 0xbf, 0xcf, 0x61, 0x23, 0xd3, 0xcc, 0x16, 0x46, 0x4d, 0x71, 0xc3, 0x5b, 0x22, 0x24, 0x33,
|
||||
0x25, 0x00, 0xcd, 0xf0, 0xfe, 0x82, 0x32, 0xd1, 0xbe, 0x5b, 0x06, 0x34, 0x7c, 0x48, 0xf7, 0xef,
|
||||
0x2a, 0xac, 0x87, 0x4d, 0xc7, 0x5b, 0x48, 0xd4, 0x6f, 0x21, 0x73, 0x7e, 0x0e, 0x1b, 0x99, 0x85,
|
||||
0x58, 0xa1, 0x76, 0x8b, 0x97, 0x66, 0x8b, 0x4c, 0xf7, 0x19, 0x34, 0x52, 0x1b, 0x2e, 0xf4, 0xfe,
|
||||
0xac, 0xf4, 0x9c, 0xd9, 0x81, 0x2d, 0x20, 0xfc, 0xf0, 0xfe, 0x2f, 0xee, 0x8d, 0xad, 0xe0, 0x62,
|
||||
0x3a, 0x64, 0x5f, 0x0e, 0x04, 0xe8, 0x77, 0x2c, 0x4f, 0xfe, 0x3a, 0x08, 0x15, 0x74, 0xc0, 0xb1,
|
||||
0x0f, 0x18, 0x9b, 0xc9, 0x70, 0xb8, 0xca, 0x4f, 0xf7, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x30,
|
||||
0x3a, 0xc3, 0x19, 0x8f, 0x1f, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
|
@ -2387,10 +2387,10 @@ var _ grpc.ClientConn
|
|||
// is compatible with the grpc package it is being compiled against.
|
||||
const _ = grpc.SupportPackageIsVersion4
|
||||
|
||||
// DataServiceClient is the client API for DataService service.
|
||||
// DataCoordClient is the client API for DataCoord service.
|
||||
//
|
||||
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
|
||||
type DataServiceClient interface {
|
||||
type DataCoordClient interface {
|
||||
GetComponentStates(ctx context.Context, in *internalpb.GetComponentStatesRequest, opts ...grpc.CallOption) (*internalpb.ComponentStates, error)
|
||||
GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
|
||||
GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
|
||||
|
@ -2406,133 +2406,133 @@ type DataServiceClient interface {
|
|||
GetRecoveryInfo(ctx context.Context, in *GetRecoveryInfoRequest, opts ...grpc.CallOption) (*GetRecoveryInfoResponse, error)
|
||||
}
|
||||
|
||||
type dataServiceClient struct {
|
||||
type dataCoordClient struct {
|
||||
cc *grpc.ClientConn
|
||||
}
|
||||
|
||||
func NewDataServiceClient(cc *grpc.ClientConn) DataServiceClient {
|
||||
return &dataServiceClient{cc}
|
||||
func NewDataCoordClient(cc *grpc.ClientConn) DataCoordClient {
|
||||
return &dataCoordClient{cc}
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetComponentStates(ctx context.Context, in *internalpb.GetComponentStatesRequest, opts ...grpc.CallOption) (*internalpb.ComponentStates, error) {
|
||||
func (c *dataCoordClient) GetComponentStates(ctx context.Context, in *internalpb.GetComponentStatesRequest, opts ...grpc.CallOption) (*internalpb.ComponentStates, error) {
|
||||
out := new(internalpb.ComponentStates)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetComponentStates", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetComponentStates", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
func (c *dataCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
out := new(milvuspb.StringResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetTimeTickChannel", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetTimeTickChannel", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
func (c *dataCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
out := new(milvuspb.StringResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetStatisticsChannel", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetStatisticsChannel", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
func (c *dataCoordClient) Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/Flush", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/Flush", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) AssignSegmentID(ctx context.Context, in *AssignSegmentIDRequest, opts ...grpc.CallOption) (*AssignSegmentIDResponse, error) {
|
||||
func (c *dataCoordClient) AssignSegmentID(ctx context.Context, in *AssignSegmentIDRequest, opts ...grpc.CallOption) (*AssignSegmentIDResponse, error) {
|
||||
out := new(AssignSegmentIDResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/AssignSegmentID", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/AssignSegmentID", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) {
|
||||
func (c *dataCoordClient) GetSegmentInfo(ctx context.Context, in *GetSegmentInfoRequest, opts ...grpc.CallOption) (*GetSegmentInfoResponse, error) {
|
||||
out := new(GetSegmentInfoResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetSegmentInfo", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetSegmentInfo", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetSegmentStates(ctx context.Context, in *GetSegmentStatesRequest, opts ...grpc.CallOption) (*GetSegmentStatesResponse, error) {
|
||||
func (c *dataCoordClient) GetSegmentStates(ctx context.Context, in *GetSegmentStatesRequest, opts ...grpc.CallOption) (*GetSegmentStatesResponse, error) {
|
||||
out := new(GetSegmentStatesResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetSegmentStates", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetSegmentStates", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetInsertBinlogPaths(ctx context.Context, in *GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*GetInsertBinlogPathsResponse, error) {
|
||||
func (c *dataCoordClient) GetInsertBinlogPaths(ctx context.Context, in *GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*GetInsertBinlogPathsResponse, error) {
|
||||
out := new(GetInsertBinlogPathsResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetInsertBinlogPaths", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetInsertBinlogPaths", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetCollectionStatistics(ctx context.Context, in *GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*GetCollectionStatisticsResponse, error) {
|
||||
func (c *dataCoordClient) GetCollectionStatistics(ctx context.Context, in *GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*GetCollectionStatisticsResponse, error) {
|
||||
out := new(GetCollectionStatisticsResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetCollectionStatistics", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetCollectionStatistics", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetPartitionStatistics(ctx context.Context, in *GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*GetPartitionStatisticsResponse, error) {
|
||||
func (c *dataCoordClient) GetPartitionStatistics(ctx context.Context, in *GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*GetPartitionStatisticsResponse, error) {
|
||||
out := new(GetPartitionStatisticsResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetPartitionStatistics", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetPartitionStatistics", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetSegmentInfoChannel(ctx context.Context, in *GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
func (c *dataCoordClient) GetSegmentInfoChannel(ctx context.Context, in *GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
out := new(milvuspb.StringResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetSegmentInfoChannel", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetSegmentInfoChannel", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) SaveBinlogPaths(ctx context.Context, in *SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
func (c *dataCoordClient) SaveBinlogPaths(ctx context.Context, in *SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/SaveBinlogPaths", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/SaveBinlogPaths", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *dataServiceClient) GetRecoveryInfo(ctx context.Context, in *GetRecoveryInfoRequest, opts ...grpc.CallOption) (*GetRecoveryInfoResponse, error) {
|
||||
func (c *dataCoordClient) GetRecoveryInfo(ctx context.Context, in *GetRecoveryInfoRequest, opts ...grpc.CallOption) (*GetRecoveryInfoResponse, error) {
|
||||
out := new(GetRecoveryInfoResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataService/GetRecoveryInfo", in, out, opts...)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetRecoveryInfo", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// DataServiceServer is the server API for DataService service.
|
||||
type DataServiceServer interface {
|
||||
// DataCoordServer is the server API for DataCoord service.
|
||||
type DataCoordServer interface {
|
||||
GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error)
|
||||
GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error)
|
||||
GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error)
|
||||
|
@ -2548,343 +2548,343 @@ type DataServiceServer interface {
|
|||
GetRecoveryInfo(context.Context, *GetRecoveryInfoRequest) (*GetRecoveryInfoResponse, error)
|
||||
}
|
||||
|
||||
// UnimplementedDataServiceServer can be embedded to have forward compatible implementations.
|
||||
type UnimplementedDataServiceServer struct {
|
||||
// UnimplementedDataCoordServer can be embedded to have forward compatible implementations.
|
||||
type UnimplementedDataCoordServer struct {
|
||||
}
|
||||
|
||||
func (*UnimplementedDataServiceServer) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
||||
func (*UnimplementedDataCoordServer) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetTimeTickChannel not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetStatisticsChannel not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) Flush(ctx context.Context, req *FlushRequest) (*commonpb.Status, error) {
|
||||
func (*UnimplementedDataCoordServer) Flush(ctx context.Context, req *FlushRequest) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Flush not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) AssignSegmentID(ctx context.Context, req *AssignSegmentIDRequest) (*AssignSegmentIDResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) AssignSegmentID(ctx context.Context, req *AssignSegmentIDRequest) (*AssignSegmentIDResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method AssignSegmentID not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetSegmentInfo(ctx context.Context, req *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetSegmentInfo(ctx context.Context, req *GetSegmentInfoRequest) (*GetSegmentInfoResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetSegmentInfo not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetSegmentStates(ctx context.Context, req *GetSegmentStatesRequest) (*GetSegmentStatesResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetSegmentStates(ctx context.Context, req *GetSegmentStatesRequest) (*GetSegmentStatesResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetSegmentStates not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetInsertBinlogPaths(ctx context.Context, req *GetInsertBinlogPathsRequest) (*GetInsertBinlogPathsResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetInsertBinlogPaths(ctx context.Context, req *GetInsertBinlogPathsRequest) (*GetInsertBinlogPathsResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetInsertBinlogPaths not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetCollectionStatistics(ctx context.Context, req *GetCollectionStatisticsRequest) (*GetCollectionStatisticsResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetCollectionStatistics(ctx context.Context, req *GetCollectionStatisticsRequest) (*GetCollectionStatisticsResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetPartitionStatistics(ctx context.Context, req *GetPartitionStatisticsRequest) (*GetPartitionStatisticsResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetPartitionStatistics(ctx context.Context, req *GetPartitionStatisticsRequest) (*GetPartitionStatisticsResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetSegmentInfoChannel(ctx context.Context, req *GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetSegmentInfoChannel(ctx context.Context, req *GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetSegmentInfoChannel not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) SaveBinlogPaths(ctx context.Context, req *SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||
func (*UnimplementedDataCoordServer) SaveBinlogPaths(ctx context.Context, req *SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method SaveBinlogPaths not implemented")
|
||||
}
|
||||
func (*UnimplementedDataServiceServer) GetRecoveryInfo(ctx context.Context, req *GetRecoveryInfoRequest) (*GetRecoveryInfoResponse, error) {
|
||||
func (*UnimplementedDataCoordServer) GetRecoveryInfo(ctx context.Context, req *GetRecoveryInfoRequest) (*GetRecoveryInfoResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetRecoveryInfo not implemented")
|
||||
}
|
||||
|
||||
func RegisterDataServiceServer(s *grpc.Server, srv DataServiceServer) {
|
||||
s.RegisterService(&_DataService_serviceDesc, srv)
|
||||
func RegisterDataCoordServer(s *grpc.Server, srv DataCoordServer) {
|
||||
s.RegisterService(&_DataCoord_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _DataService_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.GetComponentStatesRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetComponentStates(ctx, in)
|
||||
return srv.(DataCoordServer).GetComponentStates(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetComponentStates",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetComponentStates",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetComponentStates(ctx, req.(*internalpb.GetComponentStatesRequest))
|
||||
return srv.(DataCoordServer).GetComponentStates(ctx, req.(*internalpb.GetComponentStatesRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetTimeTickChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.GetTimeTickChannelRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetTimeTickChannel(ctx, in)
|
||||
return srv.(DataCoordServer).GetTimeTickChannel(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetTimeTickChannel",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetTimeTickChannel",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetTimeTickChannel(ctx, req.(*internalpb.GetTimeTickChannelRequest))
|
||||
return srv.(DataCoordServer).GetTimeTickChannel(ctx, req.(*internalpb.GetTimeTickChannelRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetStatisticsChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.GetStatisticsChannelRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetStatisticsChannel(ctx, in)
|
||||
return srv.(DataCoordServer).GetStatisticsChannel(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetStatisticsChannel",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetStatisticsChannel",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest))
|
||||
return srv.(DataCoordServer).GetStatisticsChannel(ctx, req.(*internalpb.GetStatisticsChannelRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_Flush_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_Flush_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(FlushRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).Flush(ctx, in)
|
||||
return srv.(DataCoordServer).Flush(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/Flush",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/Flush",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).Flush(ctx, req.(*FlushRequest))
|
||||
return srv.(DataCoordServer).Flush(ctx, req.(*FlushRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_AssignSegmentID_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_AssignSegmentID_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(AssignSegmentIDRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).AssignSegmentID(ctx, in)
|
||||
return srv.(DataCoordServer).AssignSegmentID(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/AssignSegmentID",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/AssignSegmentID",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).AssignSegmentID(ctx, req.(*AssignSegmentIDRequest))
|
||||
return srv.(DataCoordServer).AssignSegmentID(ctx, req.(*AssignSegmentIDRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetSegmentInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetSegmentInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetSegmentInfoRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetSegmentInfo(ctx, in)
|
||||
return srv.(DataCoordServer).GetSegmentInfo(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetSegmentInfo",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetSegmentInfo",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetSegmentInfo(ctx, req.(*GetSegmentInfoRequest))
|
||||
return srv.(DataCoordServer).GetSegmentInfo(ctx, req.(*GetSegmentInfoRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetSegmentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetSegmentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetSegmentStatesRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetSegmentStates(ctx, in)
|
||||
return srv.(DataCoordServer).GetSegmentStates(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetSegmentStates",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetSegmentStates",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetSegmentStates(ctx, req.(*GetSegmentStatesRequest))
|
||||
return srv.(DataCoordServer).GetSegmentStates(ctx, req.(*GetSegmentStatesRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetInsertBinlogPaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetInsertBinlogPaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetInsertBinlogPathsRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetInsertBinlogPaths(ctx, in)
|
||||
return srv.(DataCoordServer).GetInsertBinlogPaths(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetInsertBinlogPaths",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetInsertBinlogPaths",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetInsertBinlogPaths(ctx, req.(*GetInsertBinlogPathsRequest))
|
||||
return srv.(DataCoordServer).GetInsertBinlogPaths(ctx, req.(*GetInsertBinlogPathsRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetCollectionStatisticsRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetCollectionStatistics(ctx, in)
|
||||
return srv.(DataCoordServer).GetCollectionStatistics(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetCollectionStatistics",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetCollectionStatistics",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetCollectionStatistics(ctx, req.(*GetCollectionStatisticsRequest))
|
||||
return srv.(DataCoordServer).GetCollectionStatistics(ctx, req.(*GetCollectionStatisticsRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetPartitionStatisticsRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetPartitionStatistics(ctx, in)
|
||||
return srv.(DataCoordServer).GetPartitionStatistics(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetPartitionStatistics",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetPartitionStatistics",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetPartitionStatistics(ctx, req.(*GetPartitionStatisticsRequest))
|
||||
return srv.(DataCoordServer).GetPartitionStatistics(ctx, req.(*GetPartitionStatisticsRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetSegmentInfoChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetSegmentInfoChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetSegmentInfoChannelRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetSegmentInfoChannel(ctx, in)
|
||||
return srv.(DataCoordServer).GetSegmentInfoChannel(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetSegmentInfoChannel",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetSegmentInfoChannel",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetSegmentInfoChannel(ctx, req.(*GetSegmentInfoChannelRequest))
|
||||
return srv.(DataCoordServer).GetSegmentInfoChannel(ctx, req.(*GetSegmentInfoChannelRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_SaveBinlogPaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_SaveBinlogPaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(SaveBinlogPathsRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).SaveBinlogPaths(ctx, in)
|
||||
return srv.(DataCoordServer).SaveBinlogPaths(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/SaveBinlogPaths",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/SaveBinlogPaths",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).SaveBinlogPaths(ctx, req.(*SaveBinlogPathsRequest))
|
||||
return srv.(DataCoordServer).SaveBinlogPaths(ctx, req.(*SaveBinlogPathsRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _DataService_GetRecoveryInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
func _DataCoord_GetRecoveryInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(GetRecoveryInfoRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(DataServiceServer).GetRecoveryInfo(ctx, in)
|
||||
return srv.(DataCoordServer).GetRecoveryInfo(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.data.DataService/GetRecoveryInfo",
|
||||
FullMethod: "/milvus.proto.data.DataCoord/GetRecoveryInfo",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(DataServiceServer).GetRecoveryInfo(ctx, req.(*GetRecoveryInfoRequest))
|
||||
return srv.(DataCoordServer).GetRecoveryInfo(ctx, req.(*GetRecoveryInfoRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
var _DataService_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "milvus.proto.data.DataService",
|
||||
HandlerType: (*DataServiceServer)(nil),
|
||||
var _DataCoord_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "milvus.proto.data.DataCoord",
|
||||
HandlerType: (*DataCoordServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
{
|
||||
MethodName: "GetComponentStates",
|
||||
Handler: _DataService_GetComponentStates_Handler,
|
||||
Handler: _DataCoord_GetComponentStates_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetTimeTickChannel",
|
||||
Handler: _DataService_GetTimeTickChannel_Handler,
|
||||
Handler: _DataCoord_GetTimeTickChannel_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetStatisticsChannel",
|
||||
Handler: _DataService_GetStatisticsChannel_Handler,
|
||||
Handler: _DataCoord_GetStatisticsChannel_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "Flush",
|
||||
Handler: _DataService_Flush_Handler,
|
||||
Handler: _DataCoord_Flush_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "AssignSegmentID",
|
||||
Handler: _DataService_AssignSegmentID_Handler,
|
||||
Handler: _DataCoord_AssignSegmentID_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetSegmentInfo",
|
||||
Handler: _DataService_GetSegmentInfo_Handler,
|
||||
Handler: _DataCoord_GetSegmentInfo_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetSegmentStates",
|
||||
Handler: _DataService_GetSegmentStates_Handler,
|
||||
Handler: _DataCoord_GetSegmentStates_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetInsertBinlogPaths",
|
||||
Handler: _DataService_GetInsertBinlogPaths_Handler,
|
||||
Handler: _DataCoord_GetInsertBinlogPaths_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetCollectionStatistics",
|
||||
Handler: _DataService_GetCollectionStatistics_Handler,
|
||||
Handler: _DataCoord_GetCollectionStatistics_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetPartitionStatistics",
|
||||
Handler: _DataService_GetPartitionStatistics_Handler,
|
||||
Handler: _DataCoord_GetPartitionStatistics_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetSegmentInfoChannel",
|
||||
Handler: _DataService_GetSegmentInfoChannel_Handler,
|
||||
Handler: _DataCoord_GetSegmentInfoChannel_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "SaveBinlogPaths",
|
||||
Handler: _DataService_SaveBinlogPaths_Handler,
|
||||
Handler: _DataCoord_SaveBinlogPaths_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetRecoveryInfo",
|
||||
Handler: _DataService_GetRecoveryInfo_Handler,
|
||||
Handler: _DataCoord_GetRecoveryInfo_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
|
|
|
@ -116,7 +116,7 @@ func (node *ProxyNode) CreateCollection(ctx context.Context, request *milvuspb.C
|
|||
Condition: NewTaskCondition(ctx),
|
||||
CreateCollectionRequest: request,
|
||||
rootCoord: node.rootCoord,
|
||||
dataServiceClient: node.dataService,
|
||||
dataCoordClient: node.dataCoord,
|
||||
}
|
||||
|
||||
err := node.sched.DdQueue.Enqueue(cct)
|
||||
|
@ -388,7 +388,7 @@ func (node *ProxyNode) GetCollectionStatistics(ctx context.Context, request *mil
|
|||
ctx: ctx,
|
||||
Condition: NewTaskCondition(ctx),
|
||||
GetCollectionStatisticsRequest: request,
|
||||
dataService: node.dataService,
|
||||
dataCoord: node.dataCoord,
|
||||
}
|
||||
|
||||
err := node.sched.DdQueue.Enqueue(g)
|
||||
|
@ -714,7 +714,7 @@ func (node *ProxyNode) GetPartitionStatistics(ctx context.Context, request *milv
|
|||
ctx: ctx,
|
||||
Condition: NewTaskCondition(ctx),
|
||||
GetPartitionStatisticsRequest: request,
|
||||
dataService: node.dataService,
|
||||
dataCoord: node.dataCoord,
|
||||
}
|
||||
|
||||
err := node.sched.DdQueue.Enqueue(g)
|
||||
|
@ -962,7 +962,7 @@ func (node *ProxyNode) GetIndexBuildProgress(ctx context.Context, request *milvu
|
|||
GetIndexBuildProgressRequest: request,
|
||||
indexCoord: node.indexCoord,
|
||||
rootCoord: node.rootCoord,
|
||||
dataService: node.dataService,
|
||||
dataCoord: node.dataCoord,
|
||||
}
|
||||
|
||||
err := node.sched.DdQueue.Enqueue(gibpt)
|
||||
|
@ -1064,10 +1064,10 @@ func (node *ProxyNode) GetIndexState(ctx context.Context, request *milvuspb.GetI
|
|||
|
||||
func (node *ProxyNode) Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.MutationResult, error) {
|
||||
it := &InsertTask{
|
||||
ctx: ctx,
|
||||
Condition: NewTaskCondition(ctx),
|
||||
dataService: node.dataService,
|
||||
req: request,
|
||||
ctx: ctx,
|
||||
Condition: NewTaskCondition(ctx),
|
||||
dataCoord: node.dataCoord,
|
||||
req: request,
|
||||
BaseInsertTask: BaseInsertTask{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
HashValues: request.HashKeys,
|
||||
|
@ -1292,7 +1292,7 @@ func (node *ProxyNode) Flush(ctx context.Context, request *milvuspb.FlushRequest
|
|||
ctx: ctx,
|
||||
Condition: NewTaskCondition(ctx),
|
||||
FlushRequest: request,
|
||||
dataService: node.dataService,
|
||||
dataCoord: node.dataCoord,
|
||||
}
|
||||
|
||||
err := node.sched.DdQueue.Enqueue(ft)
|
||||
|
@ -1472,7 +1472,7 @@ func (node *ProxyNode) GetPersistentSegmentInfo(ctx context.Context, req *milvus
|
|||
resp.Status.Reason = fmt.Errorf("getSegmentsOfCollection, err:%w", err).Error()
|
||||
return resp, nil
|
||||
}
|
||||
infoResp, err := node.dataService.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
||||
infoResp, err := node.dataCoord.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_SegmentInfo,
|
||||
MsgID: 0,
|
||||
|
@ -1483,7 +1483,7 @@ func (node *ProxyNode) GetPersistentSegmentInfo(ctx context.Context, req *milvus
|
|||
})
|
||||
log.Debug("GetPersistentSegmentInfo ", zap.Any("infos", infoResp.Infos), zap.Any("status", infoResp.Status))
|
||||
if err != nil {
|
||||
resp.Status.Reason = fmt.Errorf("dataService:GetSegmentInfo, err:%w", err).Error()
|
||||
resp.Status.Reason = fmt.Errorf("dataCoord:GetSegmentInfo, err:%w", err).Error()
|
||||
return resp, nil
|
||||
}
|
||||
if infoResp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
|
|
|
@ -54,7 +54,7 @@ type ProxyNode struct {
|
|||
|
||||
rootCoord types.RootCoord
|
||||
indexCoord types.IndexCoord
|
||||
dataService types.DataService
|
||||
dataCoord types.DataCoord
|
||||
queryService types.QueryService
|
||||
|
||||
chMgr channelsMgr
|
||||
|
@ -100,15 +100,15 @@ func (node *ProxyNode) Register() error {
|
|||
}
|
||||
|
||||
func (node *ProxyNode) Init() error {
|
||||
// wait for dataservice state changed to Healthy
|
||||
if node.dataService != nil {
|
||||
log.Debug("ProxyNode wait for dataService ready")
|
||||
err := funcutil.WaitForComponentHealthy(node.ctx, node.dataService, "DataService", 1000000, time.Millisecond*200)
|
||||
// wait for datacoord state changed to Healthy
|
||||
if node.dataCoord != nil {
|
||||
log.Debug("ProxyNode wait for dataCoord ready")
|
||||
err := funcutil.WaitForComponentHealthy(node.ctx, node.dataCoord, "DataCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
log.Debug("ProxyNode wait for dataService ready failed", zap.Error(err))
|
||||
log.Debug("ProxyNode wait for dataCoord ready failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("ProxyNode dataService is ready")
|
||||
log.Debug("ProxyNode dataCoord is ready")
|
||||
}
|
||||
|
||||
// wait for queryService state changed to Healthy
|
||||
|
@ -152,12 +152,6 @@ func (node *ProxyNode) Init() error {
|
|||
log.Debug("ProxyNode CreateQueryChannel success", zap.Any("RetrieveResultChannelNames", Params.RetrieveResultChannelNames))
|
||||
}
|
||||
|
||||
// todo
|
||||
//Params.InsertChannelNames, err = node.dataService.GetInsertChannels()
|
||||
//if err != nil {
|
||||
// return err
|
||||
//}
|
||||
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"PulsarBufSize": 1024}
|
||||
|
@ -180,7 +174,7 @@ func (node *ProxyNode) Init() error {
|
|||
}
|
||||
node.tsoAllocator = tsoAllocator
|
||||
|
||||
segAssigner, err := NewSegIDAssigner(node.ctx, node.dataService, node.lastTick)
|
||||
segAssigner, err := NewSegIDAssigner(node.ctx, node.dataCoord, node.lastTick)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -416,8 +410,8 @@ func (node *ProxyNode) SetIndexCoordClient(cli types.IndexCoord) {
|
|||
node.indexCoord = cli
|
||||
}
|
||||
|
||||
func (node *ProxyNode) SetDataServiceClient(cli types.DataService) {
|
||||
node.dataService = cli
|
||||
func (node *ProxyNode) SetDataCoordClient(cli types.DataCoord) {
|
||||
node.dataCoord = cli
|
||||
}
|
||||
|
||||
func (node *ProxyNode) SetQueryServiceClient(cli types.QueryService) {
|
||||
|
|
|
@ -139,11 +139,11 @@ type SegIDAssigner struct {
|
|||
getTickFunc func() Timestamp
|
||||
PeerID UniqueID
|
||||
|
||||
dataService types.DataService
|
||||
dataCoord types.DataCoord
|
||||
countPerRPC uint32
|
||||
}
|
||||
|
||||
func NewSegIDAssigner(ctx context.Context, dataService types.DataService, getTickFunc func() Timestamp) (*SegIDAssigner, error) {
|
||||
func NewSegIDAssigner(ctx context.Context, dataCoord types.DataCoord, getTickFunc func() Timestamp) (*SegIDAssigner, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
sa := &SegIDAssigner{
|
||||
Allocator: Allocator{
|
||||
|
@ -152,7 +152,7 @@ func NewSegIDAssigner(ctx context.Context, dataService types.DataService, getTic
|
|||
Role: "SegmentIDAllocator",
|
||||
},
|
||||
countPerRPC: SegCountPerRPC,
|
||||
dataService: dataService,
|
||||
dataCoord: dataCoord,
|
||||
assignInfos: make(map[UniqueID]*list.List),
|
||||
getTickFunc: getTickFunc,
|
||||
}
|
||||
|
@ -167,8 +167,8 @@ func NewSegIDAssigner(ctx context.Context, dataService types.DataService, getTic
|
|||
return sa, nil
|
||||
}
|
||||
|
||||
func (sa *SegIDAssigner) SetServiceClient(client types.DataService) {
|
||||
sa.dataService = client
|
||||
func (sa *SegIDAssigner) SetServiceClient(client types.DataCoord) {
|
||||
sa.dataCoord = client
|
||||
}
|
||||
|
||||
func (sa *SegIDAssigner) collectExpired() {
|
||||
|
@ -299,7 +299,7 @@ func (sa *SegIDAssigner) syncSegments() (bool, error) {
|
|||
}
|
||||
|
||||
sa.segReqs = []*datapb.SegmentIDRequest{}
|
||||
resp, err := sa.dataService.AssignSegmentID(context.Background(), req)
|
||||
resp, err := sa.dataCoord.AssignSegmentID(context.Background(), req)
|
||||
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("syncSegmentID Failed:%w", err)
|
||||
|
|
|
@ -108,8 +108,8 @@ type InsertTask struct {
|
|||
req *milvuspb.InsertRequest
|
||||
Condition
|
||||
ctx context.Context
|
||||
dataService types.DataService
|
||||
result *milvuspb.MutationResult
|
||||
dataCoord types.DataCoord
|
||||
rowIDAllocator *allocator.IDAllocator
|
||||
segIDAssigner *SegIDAssigner
|
||||
chMgr channelsMgr
|
||||
|
@ -916,11 +916,11 @@ func (it *InsertTask) PostExecute(ctx context.Context) error {
|
|||
type CreateCollectionTask struct {
|
||||
Condition
|
||||
*milvuspb.CreateCollectionRequest
|
||||
ctx context.Context
|
||||
rootCoord types.RootCoord
|
||||
dataServiceClient types.DataService
|
||||
result *commonpb.Status
|
||||
schema *schemapb.CollectionSchema
|
||||
ctx context.Context
|
||||
rootCoord types.RootCoord
|
||||
dataCoordClient types.DataCoord
|
||||
result *commonpb.Status
|
||||
schema *schemapb.CollectionSchema
|
||||
}
|
||||
|
||||
func (cct *CreateCollectionTask) TraceCtx() context.Context {
|
||||
|
@ -2247,9 +2247,9 @@ func (dct *DescribeCollectionTask) PostExecute(ctx context.Context) error {
|
|||
type GetCollectionStatisticsTask struct {
|
||||
Condition
|
||||
*milvuspb.GetCollectionStatisticsRequest
|
||||
ctx context.Context
|
||||
dataService types.DataService
|
||||
result *milvuspb.GetCollectionStatisticsResponse
|
||||
ctx context.Context
|
||||
dataCoord types.DataCoord
|
||||
result *milvuspb.GetCollectionStatisticsResponse
|
||||
}
|
||||
|
||||
func (g *GetCollectionStatisticsTask) TraceCtx() context.Context {
|
||||
|
@ -2310,7 +2310,7 @@ func (g *GetCollectionStatisticsTask) Execute(ctx context.Context) error {
|
|||
CollectionID: collID,
|
||||
}
|
||||
|
||||
result, _ := g.dataService.GetCollectionStatistics(ctx, req)
|
||||
result, _ := g.dataCoord.GetCollectionStatistics(ctx, req)
|
||||
if result == nil {
|
||||
return errors.New("get collection statistics resp is nil")
|
||||
}
|
||||
|
@ -2334,9 +2334,9 @@ func (g *GetCollectionStatisticsTask) PostExecute(ctx context.Context) error {
|
|||
type GetPartitionStatisticsTask struct {
|
||||
Condition
|
||||
*milvuspb.GetPartitionStatisticsRequest
|
||||
ctx context.Context
|
||||
dataService types.DataService
|
||||
result *milvuspb.GetPartitionStatisticsResponse
|
||||
ctx context.Context
|
||||
dataCoord types.DataCoord
|
||||
result *milvuspb.GetPartitionStatisticsResponse
|
||||
}
|
||||
|
||||
func (g *GetPartitionStatisticsTask) TraceCtx() context.Context {
|
||||
|
@ -2402,7 +2402,7 @@ func (g *GetPartitionStatisticsTask) Execute(ctx context.Context) error {
|
|||
PartitionID: partitionID,
|
||||
}
|
||||
|
||||
result, _ := g.dataService.GetPartitionStatistics(ctx, req)
|
||||
result, _ := g.dataCoord.GetPartitionStatistics(ctx, req)
|
||||
if result == nil {
|
||||
return errors.New("get partition statistics resp is nil")
|
||||
}
|
||||
|
@ -3079,11 +3079,11 @@ func (dit *DropIndexTask) PostExecute(ctx context.Context) error {
|
|||
type GetIndexBuildProgressTask struct {
|
||||
Condition
|
||||
*milvuspb.GetIndexBuildProgressRequest
|
||||
ctx context.Context
|
||||
indexCoord types.IndexCoord
|
||||
rootCoord types.RootCoord
|
||||
dataService types.DataService
|
||||
result *milvuspb.GetIndexBuildProgressResponse
|
||||
ctx context.Context
|
||||
indexCoord types.IndexCoord
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
result *milvuspb.GetIndexBuildProgressResponse
|
||||
}
|
||||
|
||||
func (gibpt *GetIndexBuildProgressTask) TraceCtx() context.Context {
|
||||
|
@ -3259,7 +3259,7 @@ func (gibpt *GetIndexBuildProgressTask) Execute(ctx context.Context) error {
|
|||
}
|
||||
}
|
||||
|
||||
infoResp, err := gibpt.dataService.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
||||
infoResp, err := gibpt.dataCoord.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_SegmentInfo,
|
||||
MsgID: 0,
|
||||
|
@ -3526,9 +3526,9 @@ func (gist *GetIndexStateTask) PostExecute(ctx context.Context) error {
|
|||
type FlushTask struct {
|
||||
Condition
|
||||
*milvuspb.FlushRequest
|
||||
ctx context.Context
|
||||
dataService types.DataService
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
dataCoord types.DataCoord
|
||||
result *commonpb.Status
|
||||
}
|
||||
|
||||
func (ft *FlushTask) TraceCtx() context.Context {
|
||||
|
@ -3591,7 +3591,7 @@ func (ft *FlushTask) Execute(ctx context.Context) error {
|
|||
CollectionID: collID,
|
||||
}
|
||||
var status *commonpb.Status
|
||||
status, _ = ft.dataService.Flush(ctx, flushReq)
|
||||
status, _ = ft.dataCoord.Flush(ctx, flushReq)
|
||||
if status == nil {
|
||||
return errors.New("flush resp is nil")
|
||||
}
|
||||
|
|
|
@ -35,12 +35,12 @@ type historical struct {
|
|||
|
||||
func newHistorical(ctx context.Context,
|
||||
rootCoord types.RootCoord,
|
||||
dataService types.DataService,
|
||||
dataCoord types.DataCoord,
|
||||
indexCoord types.IndexCoord,
|
||||
factory msgstream.Factory,
|
||||
etcdKV *etcdkv.EtcdKV) *historical {
|
||||
replica := newCollectionReplica(etcdKV)
|
||||
loader := newSegmentLoader(ctx, rootCoord, indexCoord, dataService, replica, etcdKV)
|
||||
loader := newSegmentLoader(ctx, rootCoord, indexCoord, dataCoord, replica, etcdKV)
|
||||
ss := newStatsService(ctx, replica, loader.indexLoader.fieldStatsChan, factory)
|
||||
|
||||
return &historical{
|
||||
|
|
|
@ -27,15 +27,16 @@ import "C"
|
|||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.uber.org/zap"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
|
@ -64,7 +65,7 @@ type QueryNode struct {
|
|||
rootCoord types.RootCoord
|
||||
queryService types.QueryService
|
||||
indexCoord types.IndexCoord
|
||||
dataService types.DataService
|
||||
dataCoord types.DataCoord
|
||||
|
||||
msFactory msgstream.Factory
|
||||
scheduler *taskScheduler
|
||||
|
@ -142,7 +143,7 @@ func (node *QueryNode) Init() error {
|
|||
|
||||
node.historical = newHistorical(node.queryNodeLoopCtx,
|
||||
node.rootCoord,
|
||||
node.dataService,
|
||||
node.dataCoord,
|
||||
node.indexCoord,
|
||||
node.msFactory,
|
||||
node.etcdKV)
|
||||
|
@ -195,7 +196,7 @@ func (node *QueryNode) Init() error {
|
|||
log.Error("null indexCoord detected")
|
||||
}
|
||||
|
||||
if node.dataService == nil {
|
||||
if node.dataCoord == nil {
|
||||
log.Error("null data service detected")
|
||||
}
|
||||
|
||||
|
@ -284,10 +285,10 @@ func (node *QueryNode) SetIndexCoord(index types.IndexCoord) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (node *QueryNode) SetDataService(data types.DataService) error {
|
||||
func (node *QueryNode) SetDataCoord(data types.DataCoord) error {
|
||||
if data == nil {
|
||||
return errors.New("null data service interface")
|
||||
}
|
||||
node.dataService = data
|
||||
node.dataCoord = data
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ const (
|
|||
type segmentLoader struct {
|
||||
historicalReplica ReplicaInterface
|
||||
|
||||
dataService types.DataService
|
||||
dataCoord types.DataCoord
|
||||
|
||||
minioKV kv.BaseKV // minio minioKV
|
||||
etcdKV *etcdkv.EtcdKV
|
||||
|
@ -178,14 +178,14 @@ func (loader *segmentLoader) loadSegmentInternal(collectionID UniqueID,
|
|||
|
||||
func (loader *segmentLoader) GetSegmentStates(segmentID UniqueID) (*datapb.GetSegmentStatesResponse, error) {
|
||||
ctx := context.TODO()
|
||||
if loader.dataService == nil {
|
||||
if loader.dataCoord == nil {
|
||||
return nil, errors.New("null data service client")
|
||||
}
|
||||
|
||||
segmentStatesRequest := &datapb.GetSegmentStatesRequest{
|
||||
SegmentIDs: []int64{segmentID},
|
||||
}
|
||||
statesResponse, err := loader.dataService.GetSegmentStates(ctx, segmentStatesRequest)
|
||||
statesResponse, err := loader.dataCoord.GetSegmentStates(ctx, segmentStatesRequest)
|
||||
if err != nil || statesResponse.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -302,7 +302,7 @@ func (loader *segmentLoader) loadSegmentFieldsData(segment *Segment, binlogPaths
|
|||
return nil
|
||||
}
|
||||
|
||||
func newSegmentLoader(ctx context.Context, rootCoord types.RootCoord, indexCoord types.IndexCoord, dataService types.DataService, replica ReplicaInterface, etcdKV *etcdkv.EtcdKV) *segmentLoader {
|
||||
func newSegmentLoader(ctx context.Context, rootCoord types.RootCoord, indexCoord types.IndexCoord, dataCoord types.DataCoord, replica ReplicaInterface, etcdKV *etcdkv.EtcdKV) *segmentLoader {
|
||||
option := &minioKV.Option{
|
||||
Address: Params.MinioEndPoint,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
|
@ -321,7 +321,7 @@ func newSegmentLoader(ctx context.Context, rootCoord types.RootCoord, indexCoord
|
|||
return &segmentLoader{
|
||||
historicalReplica: replica,
|
||||
|
||||
dataService: dataService,
|
||||
dataCoord: dataCoord,
|
||||
|
||||
minioKV: client,
|
||||
etcdKV: etcdKV,
|
||||
|
|
|
@ -154,7 +154,7 @@ func (qs *QueryService) LoadCollection(ctx context.Context, req *querypb.LoadCol
|
|||
},
|
||||
LoadCollectionRequest: req,
|
||||
rootCoord: qs.rootCoordClient,
|
||||
dataService: qs.dataServiceClient,
|
||||
dataCoord: qs.dataCoordClient,
|
||||
cluster: qs.cluster,
|
||||
meta: qs.meta,
|
||||
}
|
||||
|
@ -269,7 +269,7 @@ func (qs *QueryService) LoadPartitions(ctx context.Context, req *querypb.LoadPar
|
|||
triggerCondition: querypb.TriggerCondition_grpcRequest,
|
||||
},
|
||||
LoadPartitionsRequest: req,
|
||||
dataService: qs.dataServiceClient,
|
||||
dataCoord: qs.dataCoordClient,
|
||||
cluster: qs.cluster,
|
||||
meta: qs.meta,
|
||||
}
|
||||
|
|
|
@ -101,7 +101,7 @@ func (rc *RootCoordMock) ShowSegments(ctx context.Context, in *milvuspb.ShowSegm
|
|||
}
|
||||
|
||||
type DataMock struct {
|
||||
types.DataService
|
||||
types.DataCoord
|
||||
SegmentIDs []UniqueID
|
||||
SegmentStates map[UniqueID]*datapb.SegmentStateInfo
|
||||
}
|
||||
|
|
|
@ -55,8 +55,8 @@ type QueryService struct {
|
|||
cluster *queryNodeCluster
|
||||
scheduler *TaskScheduler
|
||||
|
||||
dataServiceClient types.DataService
|
||||
rootCoordClient types.RootCoord
|
||||
dataCoordClient types.DataCoord
|
||||
rootCoordClient types.RootCoord
|
||||
|
||||
session *sessionutil.Session
|
||||
eventChan <-chan *sessionutil.SessionEvent
|
||||
|
@ -94,7 +94,7 @@ func (qs *QueryService) Init() error {
|
|||
return err
|
||||
}
|
||||
|
||||
qs.scheduler, err = NewTaskScheduler(qs.loopCtx, metaKV, qs.cluster, etcdKV, qs.rootCoordClient, qs.dataServiceClient)
|
||||
qs.scheduler, err = NewTaskScheduler(qs.loopCtx, metaKV, qs.cluster, etcdKV, qs.rootCoordClient, qs.dataCoordClient)
|
||||
return err
|
||||
}
|
||||
log.Debug("queryService try to connect etcd")
|
||||
|
@ -165,8 +165,8 @@ func (qs *QueryService) SetRootCoord(rootCoord types.RootCoord) {
|
|||
qs.rootCoordClient = rootCoord
|
||||
}
|
||||
|
||||
func (qs *QueryService) SetDataService(dataService types.DataService) {
|
||||
qs.dataServiceClient = dataService
|
||||
func (qs *QueryService) SetDataCoord(dataCoord types.DataCoord) {
|
||||
qs.dataCoordClient = dataCoord
|
||||
}
|
||||
|
||||
func (qs *QueryService) watchNodeLoop() {
|
||||
|
@ -210,7 +210,7 @@ func (qs *QueryService) watchNodeLoop() {
|
|||
},
|
||||
LoadBalanceRequest: loadBalanceSegment,
|
||||
rootCoord: qs.rootCoordClient,
|
||||
dataService: qs.dataServiceClient,
|
||||
dataCoord: qs.dataCoordClient,
|
||||
cluster: qs.cluster,
|
||||
meta: qs.meta,
|
||||
}
|
||||
|
@ -253,7 +253,7 @@ func (qs *QueryService) watchNodeLoop() {
|
|||
},
|
||||
LoadBalanceRequest: loadBalanceSegment,
|
||||
rootCoord: qs.rootCoordClient,
|
||||
dataService: qs.dataServiceClient,
|
||||
dataCoord: qs.dataCoordClient,
|
||||
cluster: qs.cluster,
|
||||
meta: qs.meta,
|
||||
}
|
||||
|
|
|
@ -61,7 +61,7 @@ func TestQueryService_load(t *testing.T) {
|
|||
service.Init()
|
||||
service.Start()
|
||||
service.SetRootCoord(NewRootCoordMock())
|
||||
service.SetDataService(NewDataMock())
|
||||
service.SetDataCoord(NewDataMock())
|
||||
registerNodeRequest := &querypb.RegisterNodeRequest{
|
||||
Address: &commonpb.Address{},
|
||||
}
|
||||
|
|
|
@ -126,10 +126,10 @@ func (bt *BaseTask) SetState(state taskState) {
|
|||
type LoadCollectionTask struct {
|
||||
BaseTask
|
||||
*querypb.LoadCollectionRequest
|
||||
rootCoord types.RootCoord
|
||||
dataService types.DataService
|
||||
cluster *queryNodeCluster
|
||||
meta *meta
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
cluster *queryNodeCluster
|
||||
meta *meta
|
||||
}
|
||||
|
||||
func (lct *LoadCollectionTask) Marshal() string {
|
||||
|
@ -187,7 +187,7 @@ func (lct *LoadCollectionTask) Execute(ctx context.Context) error {
|
|||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
}
|
||||
recoveryInfo, err := lct.dataService.GetRecoveryInfo(lct.ctx, getRecoveryInfoRequest)
|
||||
recoveryInfo, err := lct.dataCoord.GetRecoveryInfo(lct.ctx, getRecoveryInfoRequest)
|
||||
if err != nil {
|
||||
status.Reason = err.Error()
|
||||
lct.result = status
|
||||
|
@ -418,9 +418,9 @@ func (rct *ReleaseCollectionTask) PostExecute(ctx context.Context) error {
|
|||
type LoadPartitionTask struct {
|
||||
BaseTask
|
||||
*querypb.LoadPartitionsRequest
|
||||
dataService types.DataService
|
||||
cluster *queryNodeCluster
|
||||
meta *meta
|
||||
dataCoord types.DataCoord
|
||||
cluster *queryNodeCluster
|
||||
meta *meta
|
||||
}
|
||||
|
||||
func (lpt *LoadPartitionTask) Marshal() string {
|
||||
|
@ -465,7 +465,7 @@ func (lpt *LoadPartitionTask) Execute(ctx context.Context) error {
|
|||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
}
|
||||
recoveryInfo, err := lpt.dataService.GetRecoveryInfo(lpt.ctx, getRecoveryInfoRequest)
|
||||
recoveryInfo, err := lpt.dataCoord.GetRecoveryInfo(lpt.ctx, getRecoveryInfoRequest)
|
||||
if err != nil {
|
||||
status.Reason = err.Error()
|
||||
lpt.result = status
|
||||
|
@ -1047,10 +1047,10 @@ type HandoffTask struct {
|
|||
type LoadBalanceTask struct {
|
||||
BaseTask
|
||||
*querypb.LoadBalanceRequest
|
||||
rootCoord types.RootCoord
|
||||
dataService types.DataService
|
||||
cluster *queryNodeCluster
|
||||
meta *meta
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
cluster *queryNodeCluster
|
||||
meta *meta
|
||||
}
|
||||
|
||||
func (lbt *LoadBalanceTask) Marshal() string {
|
||||
|
@ -1108,7 +1108,7 @@ func (lbt *LoadBalanceTask) Execute(ctx context.Context) error {
|
|||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
}
|
||||
recoveryInfo, err := lbt.dataService.GetRecoveryInfo(lbt.ctx, getRecoveryInfo)
|
||||
recoveryInfo, err := lbt.dataCoord.GetRecoveryInfo(lbt.ctx, getRecoveryInfo)
|
||||
if err != nil {
|
||||
status.Reason = err.Error()
|
||||
lbt.result = status
|
||||
|
|
|
@ -126,15 +126,15 @@ type TaskScheduler struct {
|
|||
taskIDAllocator func() (UniqueID, error)
|
||||
client *etcdkv.EtcdKV
|
||||
|
||||
rootCoord types.RootCoord
|
||||
dataService types.DataService
|
||||
rootCoord types.RootCoord
|
||||
dataCoord types.DataCoord
|
||||
|
||||
wg sync.WaitGroup
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
}
|
||||
|
||||
func NewTaskScheduler(ctx context.Context, meta *meta, cluster *queryNodeCluster, kv *etcdkv.EtcdKV, rootCoord types.RootCoord, dataService types.DataService) (*TaskScheduler, error) {
|
||||
func NewTaskScheduler(ctx context.Context, meta *meta, cluster *queryNodeCluster, kv *etcdkv.EtcdKV, rootCoord types.RootCoord, dataCoord types.DataCoord) (*TaskScheduler, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
taskChan := make(chan task, 1024)
|
||||
s := &TaskScheduler{
|
||||
|
@ -145,7 +145,7 @@ func NewTaskScheduler(ctx context.Context, meta *meta, cluster *queryNodeCluster
|
|||
activateTaskChan: taskChan,
|
||||
client: kv,
|
||||
rootCoord: rootCoord,
|
||||
dataService: dataService,
|
||||
dataCoord: dataCoord,
|
||||
}
|
||||
s.triggerTaskQueue = NewTaskQueue()
|
||||
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryService task id"))
|
||||
|
@ -259,7 +259,7 @@ func (scheduler *TaskScheduler) unmarshalTask(t string) (task, error) {
|
|||
},
|
||||
LoadCollectionRequest: &loadReq,
|
||||
rootCoord: scheduler.rootCoord,
|
||||
dataService: scheduler.dataService,
|
||||
dataCoord: scheduler.dataCoord,
|
||||
cluster: scheduler.cluster,
|
||||
meta: scheduler.meta,
|
||||
}
|
||||
|
@ -277,7 +277,7 @@ func (scheduler *TaskScheduler) unmarshalTask(t string) (task, error) {
|
|||
triggerCondition: querypb.TriggerCondition_grpcRequest,
|
||||
},
|
||||
LoadPartitionsRequest: &loadReq,
|
||||
dataService: scheduler.dataService,
|
||||
dataCoord: scheduler.dataCoord,
|
||||
cluster: scheduler.cluster,
|
||||
meta: scheduler.meta,
|
||||
}
|
||||
|
@ -394,7 +394,7 @@ func (scheduler *TaskScheduler) unmarshalTask(t string) (task, error) {
|
|||
},
|
||||
LoadBalanceRequest: &loadReq,
|
||||
rootCoord: scheduler.rootCoord,
|
||||
dataService: scheduler.dataService,
|
||||
dataCoord: scheduler.dataCoord,
|
||||
cluster: scheduler.cluster,
|
||||
meta: scheduler.meta,
|
||||
}
|
||||
|
|
|
@ -757,7 +757,7 @@ func (mt *metaTable) AddSegment(segInfos []*datapb.SegmentInfo, msgStartPos stri
|
|||
meta[k] = v
|
||||
}
|
||||
|
||||
// AddSegment is invoked from DataService
|
||||
// AddSegment is invoked from DataCoord
|
||||
if msgStartPos != "" && msgEndPos != "" {
|
||||
meta[SegInfoMsgStartPosPrefix] = msgStartPos
|
||||
meta[SegInfoMsgEndPosPrefix] = msgEndPos
|
||||
|
|
|
@ -29,14 +29,14 @@ type ParamTable struct {
|
|||
Address string
|
||||
Port int
|
||||
|
||||
PulsarAddress string
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
KvRootPath string
|
||||
MsgChannelSubName string
|
||||
TimeTickChannel string
|
||||
StatisticsChannel string
|
||||
DataServiceSegmentChannel string // data service create segment, or data node flush segment
|
||||
PulsarAddress string
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
KvRootPath string
|
||||
MsgChannelSubName string
|
||||
TimeTickChannel string
|
||||
StatisticsChannel string
|
||||
DataCoordSegmentChannel string // data service create segment, or data node flush segment
|
||||
|
||||
MaxPartitionNum int64
|
||||
DefaultPartitionName string
|
||||
|
@ -152,7 +152,7 @@ func (p *ParamTable) initSegmentInfoChannelName() {
|
|||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.DataServiceSegmentChannel = channel
|
||||
p.DataCoordSegmentChannel = channel
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMaxPartitionNum() {
|
||||
|
|
|
@ -642,12 +642,12 @@ func (c *Core) setMsgStreams() error {
|
|||
return c.dmlChannels.BroadcastAll(channelNames, &msgPack)
|
||||
}
|
||||
|
||||
if Params.DataServiceSegmentChannel == "" {
|
||||
return fmt.Errorf("DataServiceSegmentChannel is empty")
|
||||
if Params.DataCoordSegmentChannel == "" {
|
||||
return fmt.Errorf("DataCoordSegmentChannel is empty")
|
||||
}
|
||||
|
||||
// data service will put msg into this channel when create segment
|
||||
dsChanName := Params.DataServiceSegmentChannel
|
||||
dsChanName := Params.DataCoordSegmentChannel
|
||||
dsSubName := Params.MsgChannelSubName + "ds"
|
||||
dsStream, err := c.startMsgStreamAndSeek(dsChanName, dsSubName, SegInfoMsgEndPosPrefix)
|
||||
if err != nil {
|
||||
|
@ -656,7 +656,7 @@ func (c *Core) setMsgStreams() error {
|
|||
c.DataCoordSegmentChan = (*dsStream).Chan()
|
||||
|
||||
// data node will put msg into this channel when flush segment
|
||||
dnChanName := Params.DataServiceSegmentChannel
|
||||
dnChanName := Params.DataCoordSegmentChannel
|
||||
dnSubName := Params.MsgChannelSubName + "dn"
|
||||
dnStream, err := c.startMsgStreamAndSeek(dnChanName, dnSubName, FlushedSegMsgEndPosPrefix)
|
||||
if err != nil {
|
||||
|
@ -676,13 +676,13 @@ func (c *Core) SetNewProxyClient(f func(sess *sessionutil.Session) (types.ProxyN
|
|||
}
|
||||
}
|
||||
|
||||
func (c *Core) SetDataCoord(ctx context.Context, s types.DataService) error {
|
||||
func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
|
||||
rsp, err := s.GetSegmentInfoChannel(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
Params.DataServiceSegmentChannel = rsp.Value
|
||||
log.Debug("data service segment", zap.String("channel name", Params.DataServiceSegmentChannel))
|
||||
Params.DataCoordSegmentChannel = rsp.Value
|
||||
log.Debug("data service segment", zap.String("channel name", Params.DataCoordSegmentChannel))
|
||||
|
||||
c.CallGetBinlogFilePathsService = func(segID typeutil.UniqueID, fieldID typeutil.UniqueID) (retFiles []string, retErr error) {
|
||||
defer func() {
|
||||
|
|
|
@ -64,7 +64,7 @@ func (p *proxyNodeMock) GetCollArray() []string {
|
|||
}
|
||||
|
||||
type dataMock struct {
|
||||
types.DataService
|
||||
types.DataCoord
|
||||
randVal int
|
||||
}
|
||||
|
||||
|
@ -275,7 +275,7 @@ func TestMasterService(t *testing.T) {
|
|||
Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath)
|
||||
Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath)
|
||||
Params.MsgChannelSubName = fmt.Sprintf("subname-%d", randVal)
|
||||
Params.DataServiceSegmentChannel = fmt.Sprintf("data-service-segment-%d", randVal)
|
||||
Params.DataCoordSegmentChannel = fmt.Sprintf("data-service-segment-%d", randVal)
|
||||
|
||||
err = core.Register()
|
||||
assert.Nil(t, err)
|
||||
|
@ -336,8 +336,8 @@ func TestMasterService(t *testing.T) {
|
|||
err = tmpFactory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
|
||||
dataServiceSegmentStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||
dataServiceSegmentStream.AsProducer([]string{Params.DataServiceSegmentChannel})
|
||||
dataCoordSegmentStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||
dataCoordSegmentStream.AsProducer([]string{Params.DataCoordSegmentChannel})
|
||||
|
||||
timeTickStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||
timeTickStream.AsConsumer([]string{Params.TimeTickChannel}, Params.MsgChannelSubName)
|
||||
|
@ -345,13 +345,13 @@ func TestMasterService(t *testing.T) {
|
|||
|
||||
dmlStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||
|
||||
// test dataServiceSegmentStream seek
|
||||
// test dataCoordSegmentStream seek
|
||||
dataNodeSubName := Params.MsgChannelSubName + "dn"
|
||||
flushedSegStream, _ := tmpFactory.NewMsgStream(ctx)
|
||||
flushedSegStream.AsConsumer([]string{Params.DataServiceSegmentChannel}, dataNodeSubName)
|
||||
flushedSegStream.AsConsumer([]string{Params.DataCoordSegmentChannel}, dataNodeSubName)
|
||||
flushedSegStream.Start()
|
||||
msgPackTmp := GenFlushedSegMsgPack(9999)
|
||||
err = dataServiceSegmentStream.Produce(msgPackTmp)
|
||||
err = dataCoordSegmentStream.Produce(msgPackTmp)
|
||||
assert.Nil(t, err)
|
||||
|
||||
flushedSegMsgPack := flushedSegStream.Consume()
|
||||
|
@ -732,7 +732,7 @@ func TestMasterService(t *testing.T) {
|
|||
PartitionID: part.PartitionID,
|
||||
}
|
||||
segInfoMsgPack := GenSegInfoMsgPack(seg)
|
||||
err = dataServiceSegmentStream.Broadcast(segInfoMsgPack)
|
||||
err = dataCoordSegmentStream.Broadcast(segInfoMsgPack)
|
||||
assert.Nil(t, err)
|
||||
time.Sleep(time.Second)
|
||||
|
||||
|
@ -873,7 +873,7 @@ func TestMasterService(t *testing.T) {
|
|||
PartitionID: part.PartitionID,
|
||||
}
|
||||
segInfoMsgPack := GenSegInfoMsgPack(seg)
|
||||
err = dataServiceSegmentStream.Broadcast(segInfoMsgPack)
|
||||
err = dataCoordSegmentStream.Broadcast(segInfoMsgPack)
|
||||
assert.Nil(t, err)
|
||||
time.Sleep(time.Second)
|
||||
|
||||
|
@ -882,7 +882,7 @@ func TestMasterService(t *testing.T) {
|
|||
assert.Equal(t, 2, len(part.SegmentIDs))
|
||||
|
||||
flushedSegMsgPack := GenFlushedSegMsgPack(segID)
|
||||
err = dataServiceSegmentStream.Broadcast(flushedSegMsgPack)
|
||||
err = dataCoordSegmentStream.Broadcast(flushedSegMsgPack)
|
||||
assert.Nil(t, err)
|
||||
time.Sleep(time.Second)
|
||||
|
||||
|
@ -1800,8 +1800,8 @@ func TestMasterService2(t *testing.T) {
|
|||
err = msFactory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
|
||||
dataServiceSegmentStream, _ := msFactory.NewMsgStream(ctx)
|
||||
dataServiceSegmentStream.AsProducer([]string{Params.DataServiceSegmentChannel})
|
||||
dataCoordSegmentStream, _ := msFactory.NewMsgStream(ctx)
|
||||
dataCoordSegmentStream.AsProducer([]string{Params.DataCoordSegmentChannel})
|
||||
|
||||
timeTickStream, _ := msFactory.NewMsgStream(ctx)
|
||||
timeTickStream.AsConsumer([]string{Params.TimeTickChannel}, Params.MsgChannelSubName)
|
||||
|
|
|
@ -31,7 +31,7 @@ import (
|
|||
)
|
||||
|
||||
type tbd struct {
|
||||
types.DataService
|
||||
types.DataCoord
|
||||
}
|
||||
|
||||
func (*tbd) GetInsertBinlogPaths(context.Context, *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
|
||||
|
|
|
@ -45,7 +45,7 @@ type DataNode interface {
|
|||
FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error)
|
||||
}
|
||||
|
||||
type DataService interface {
|
||||
type DataCoord interface {
|
||||
Component
|
||||
TimeTickProvider
|
||||
|
||||
|
@ -116,7 +116,7 @@ type RootCoordComponent interface {
|
|||
RootCoord
|
||||
|
||||
UpdateStateCode(internalpb.StateCode)
|
||||
SetDataCoord(context.Context, DataService) error
|
||||
SetDataCoord(context.Context, DataCoord) error
|
||||
SetIndexCoord(IndexCoord) error
|
||||
SetQueryCoord(QueryService) error
|
||||
SetNewProxyClient(func(sess *sessionutil.Session) (ProxyNode, error))
|
||||
|
|
|
@ -170,19 +170,19 @@ func (gp *BaseTable) tryloadFromEnv() {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
dataServiceAddress := os.Getenv("DATA_SERVICE_ADDRESS")
|
||||
if dataServiceAddress == "" {
|
||||
serviceHost, err := gp.Load("dataService.address")
|
||||
dataCoordAddress := os.Getenv("DATA_SERVICE_ADDRESS")
|
||||
if dataCoordAddress == "" {
|
||||
serviceHost, err := gp.Load("dataCoord.address")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
port, err := gp.Load("dataService.port")
|
||||
port, err := gp.Load("dataCoord.port")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
dataServiceAddress = serviceHost + ":" + port
|
||||
dataCoordAddress = serviceHost + ":" + port
|
||||
}
|
||||
err = gp.Save("_DataServiceAddress", dataServiceAddress)
|
||||
err = gp.Save("_DataCoordAddress", dataCoordAddress)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue