Support to send dd msg to all dml channels (#5597)

* send dd msg to dml channel

Signed-off-by: yudong.cai <yudong.cai@zilliz.com>

* add dml_channels.go

Signed-off-by: yudong.cai <yudong.cai@zilliz.com>

* fix race

Signed-off-by: yudong.cai <yudong.cai@zilliz.com>
pull/5779/head
Cai Yudong 2021-06-04 15:00:34 +08:00 committed by zhenshan.cao
parent e7521afed5
commit a948b71e06
16 changed files with 327 additions and 281 deletions

View File

@ -117,9 +117,6 @@ func (s *Server) Start() error {
if err = s.initMasterClient(); err != nil { if err = s.initMasterClient(); err != nil {
return err return err
} }
if err = s.getDDChannelFromMaster(); err != nil {
return err
}
if err = s.initMeta(); err != nil { if err = s.initMeta(); err != nil {
return err return err
@ -236,15 +233,6 @@ func (s *Server) initFlushMsgStream() error {
return nil return nil
} }
func (s *Server) getDDChannelFromMaster() error {
resp, err := s.masterClient.GetDdChannel(s.ctx)
if err = VerifyResponse(resp, err); err != nil {
return err
}
s.ddChannelName = resp.Value
return nil
}
func (s *Server) startServerLoop() { func (s *Server) startServerLoop() {
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx) s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx)
s.serverLoopWg.Add(5) s.serverLoopWg.Add(5)

View File

@ -176,15 +176,7 @@ func (c *GrpcClient) GetStatisticsChannel(ctx context.Context) (*milvuspb.String
return ret.(*milvuspb.StringResponse), err return ret.(*milvuspb.StringResponse), err
} }
// GetDdChannel receive ddl from rpc and time tick from proxy service, and put them into this channel //DDL request
func (c *GrpcClient) GetDdChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
ret, err := c.recall(func() (interface{}, error) {
return c.grpcClient.GetDdChannel(ctx, &internalpb.GetDdChannelRequest{})
})
return ret.(*milvuspb.StringResponse), err
}
// CreateCollection DDL request
func (c *GrpcClient) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) { func (c *GrpcClient) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
ret, err := c.recall(func() (interface{}, error) { ret, err := c.recall(func() (interface{}, error) {
return c.grpcClient.CreateCollection(ctx, in) return c.grpcClient.CreateCollection(ctx, in)

View File

@ -129,7 +129,6 @@ func TestGrpcService(t *testing.T) {
cms.Params.KvRootPath = fmt.Sprintf("/%d/test/kv", randVal) cms.Params.KvRootPath = fmt.Sprintf("/%d/test/kv", randVal)
cms.Params.MsgChannelSubName = fmt.Sprintf("msgChannel%d", randVal) cms.Params.MsgChannelSubName = fmt.Sprintf("msgChannel%d", randVal)
cms.Params.TimeTickChannel = fmt.Sprintf("timeTick%d", randVal) cms.Params.TimeTickChannel = fmt.Sprintf("timeTick%d", randVal)
cms.Params.DdChannel = fmt.Sprintf("ddChannel%d", randVal)
cms.Params.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal) cms.Params.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal)
cms.Params.DataServiceSegmentChannel = fmt.Sprintf("segmentChannel%d", randVal) cms.Params.DataServiceSegmentChannel = fmt.Sprintf("segmentChannel%d", randVal)
@ -286,13 +285,6 @@ func TestGrpcService(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
}) })
t.Run("get dd channel", func(t *testing.T) {
req := &internalpb.GetDdChannelRequest{}
rsp, err := svr.GetDdChannel(ctx, req)
assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
})
t.Run("alloc time stamp", func(t *testing.T) { t.Run("alloc time stamp", func(t *testing.T) {
req := &masterpb.AllocTimestampRequest{ req := &masterpb.AllocTimestampRequest{
Count: 1, Count: 1,

View File

@ -296,12 +296,7 @@ func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetSt
return s.masterService.GetStatisticsChannel(ctx) return s.masterService.GetStatisticsChannel(ctx)
} }
// GetDdChannel receive ddl from rpc and time tick from proxy service, and put them into this channel //DDL request
func (s *Server) GetDdChannel(ctx context.Context, req *internalpb.GetDdChannelRequest) (*milvuspb.StringResponse, error) {
return s.masterService.GetDdChannel(ctx)
}
// CreateCollection DDL request
func (s *Server) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) { func (s *Server) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
return s.masterService.CreateCollection(ctx, in) return s.masterService.CreateCollection(ctx, in)
} }

View File

@ -0,0 +1,129 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License.
package masterservice
import (
"fmt"
"sync"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"go.uber.org/zap"
)
type dmlChannels struct {
core *Core
lock sync.RWMutex
dml map[string]msgstream.MsgStream
}
func newDMLChannels(c *Core) *dmlChannels {
return &dmlChannels{
core: c,
lock: sync.RWMutex{},
dml: make(map[string]msgstream.MsgStream),
}
}
func (d *dmlChannels) GetNumChannles() int {
d.lock.RLock()
defer d.lock.RUnlock()
return len(d.dml)
}
func (d *dmlChannels) ProduceAll(pack *msgstream.MsgPack) {
d.lock.RLock()
defer d.lock.RUnlock()
for n, ms := range d.dml {
if err := ms.Produce(pack); err != nil {
log.Debug("msgstream produce error", zap.String("name", n), zap.Error(err))
}
}
}
func (d *dmlChannels) BroadcastAll(pack *msgstream.MsgPack) {
d.lock.RLock()
defer d.lock.RUnlock()
for n, ms := range d.dml {
if err := ms.Broadcast(pack); err != nil {
log.Debug("msgstream broadcast error", zap.String("name", n), zap.Error(err))
}
}
}
func (d *dmlChannels) Produce(name string, pack *msgstream.MsgPack) error {
d.lock.Lock()
defer d.lock.Unlock()
var err error
ms, ok := d.dml[name]
if !ok {
ms, err = d.core.msFactory.NewMsgStream(d.core.ctx)
if err != nil {
return fmt.Errorf("create mstream failed, name = %s, error=%w", name, err)
}
ms.AsProducer([]string{name})
d.dml[name] = ms
}
return ms.Produce(pack)
}
func (d *dmlChannels) Broadcast(name string, pack *msgstream.MsgPack) error {
d.lock.Lock()
defer d.lock.Unlock()
var err error
ms, ok := d.dml[name]
if !ok {
ms, err = d.core.msFactory.NewMsgStream(d.core.ctx)
if err != nil {
return fmt.Errorf("create msgtream failed, name = %s, error=%w", name, err)
}
ms.AsProducer([]string{name})
d.dml[name] = ms
}
return ms.Broadcast(pack)
}
func (d *dmlChannels) AddProducerChannles(names ...string) {
d.lock.Lock()
defer d.lock.Unlock()
var err error
for _, name := range names {
ms, ok := d.dml[name]
if !ok {
ms, err = d.core.msFactory.NewMsgStream(d.core.ctx)
if err != nil {
log.Debug("add msgstream failed", zap.String("name", name), zap.Error(err))
continue
}
ms.AsProducer([]string{name})
d.dml[name] = ms
}
}
}
func (d *dmlChannels) RemoveProducerChannels(names ...string) {
d.lock.Lock()
defer d.lock.Unlock()
for _, name := range names {
if ms, ok := d.dml[name]; ok {
ms.Close()
delete(d.dml, name)
}
}
}

View File

@ -143,6 +143,9 @@ type Core struct {
//dd request scheduler //dd request scheduler
ddReqQueue chan reqTask //dd request will be push into this chan ddReqQueue chan reqTask //dd request will be push into this chan
//dml channels
dmlChannels *dmlChannels
//ProxyNode manager //ProxyNode manager
proxyNodeManager *proxyNodeManager proxyNodeManager *proxyNodeManager
@ -528,14 +531,6 @@ func (c *Core) setMsgStreams() error {
timeTickStream.AsProducer([]string{Params.TimeTickChannel}) timeTickStream.AsProducer([]string{Params.TimeTickChannel})
log.Debug("masterservice AsProducer: " + Params.TimeTickChannel) log.Debug("masterservice AsProducer: " + Params.TimeTickChannel)
// master dd channel
if Params.DdChannel == "" {
return fmt.Errorf("DdChannel is empty")
}
ddStream, _ := c.msFactory.NewMsgStream(c.ctx)
ddStream.AsProducer([]string{Params.DdChannel})
log.Debug("masterservice AsProducer: " + Params.DdChannel)
c.SendTimeTick = func(t typeutil.Timestamp) error { c.SendTimeTick = func(t typeutil.Timestamp) error {
msgPack := ms.MsgPack{} msgPack := ms.MsgPack{}
baseMsg := ms.BaseMsg{ baseMsg := ms.BaseMsg{
@ -559,11 +554,25 @@ func (c *Core) setMsgStreams() error {
if err := timeTickStream.Broadcast(&msgPack); err != nil { if err := timeTickStream.Broadcast(&msgPack); err != nil {
return err return err
} }
if err := ddStream.Broadcast(&msgPack); err != nil {
return err
}
metrics.MasterDDChannelTimeTick.Set(float64(tsoutil.Mod24H(t))) metrics.MasterDDChannelTimeTick.Set(float64(tsoutil.Mod24H(t)))
return nil
c.dmlChannels.BroadcastAll(&msgPack)
pc := c.MetaTable.ListCollectionPhysicalChannels()
pt := make([]uint64, len(pc))
for i := 0; i < len(pt); i++ {
pt[i] = t
}
ttMsg := internalpb.ChannelTimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
MsgID: 0, //TODO
Timestamp: t,
SourceID: c.session.ServerID,
},
ChannelNames: pc,
Timestamps: pt,
}
return c.chanTimeTick.UpdateTimeTick(&ttMsg)
} }
c.SendDdCreateCollectionReq = func(ctx context.Context, req *internalpb.CreateCollectionRequest) error { c.SendDdCreateCollectionReq = func(ctx context.Context, req *internalpb.CreateCollectionRequest) error {
@ -574,14 +583,12 @@ func (c *Core) setMsgStreams() error {
EndTimestamp: req.Base.Timestamp, EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0}, HashValues: []uint32{0},
} }
collMsg := &ms.CreateCollectionMsg{ msg := &ms.CreateCollectionMsg{
BaseMsg: baseMsg, BaseMsg: baseMsg,
CreateCollectionRequest: *req, CreateCollectionRequest: *req,
} }
msgPack.Msgs = append(msgPack.Msgs, collMsg) msgPack.Msgs = append(msgPack.Msgs, msg)
if err := ddStream.Broadcast(&msgPack); err != nil { c.dmlChannels.BroadcastAll(&msgPack)
return err
}
return nil return nil
} }
@ -593,14 +600,12 @@ func (c *Core) setMsgStreams() error {
EndTimestamp: req.Base.Timestamp, EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0}, HashValues: []uint32{0},
} }
collMsg := &ms.DropCollectionMsg{ msg := &ms.DropCollectionMsg{
BaseMsg: baseMsg, BaseMsg: baseMsg,
DropCollectionRequest: *req, DropCollectionRequest: *req,
} }
msgPack.Msgs = append(msgPack.Msgs, collMsg) msgPack.Msgs = append(msgPack.Msgs, msg)
if err := ddStream.Broadcast(&msgPack); err != nil { c.dmlChannels.BroadcastAll(&msgPack)
return err
}
return nil return nil
} }
@ -612,14 +617,12 @@ func (c *Core) setMsgStreams() error {
EndTimestamp: req.Base.Timestamp, EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0}, HashValues: []uint32{0},
} }
collMsg := &ms.CreatePartitionMsg{ msg := &ms.CreatePartitionMsg{
BaseMsg: baseMsg, BaseMsg: baseMsg,
CreatePartitionRequest: *req, CreatePartitionRequest: *req,
} }
msgPack.Msgs = append(msgPack.Msgs, collMsg) msgPack.Msgs = append(msgPack.Msgs, msg)
if err := ddStream.Broadcast(&msgPack); err != nil { c.dmlChannels.BroadcastAll(&msgPack)
return err
}
return nil return nil
} }
@ -631,14 +634,12 @@ func (c *Core) setMsgStreams() error {
EndTimestamp: req.Base.Timestamp, EndTimestamp: req.Base.Timestamp,
HashValues: []uint32{0}, HashValues: []uint32{0},
} }
collMsg := &ms.DropPartitionMsg{ msg := &ms.DropPartitionMsg{
BaseMsg: baseMsg, BaseMsg: baseMsg,
DropPartitionRequest: *req, DropPartitionRequest: *req,
} }
msgPack.Msgs = append(msgPack.Msgs, collMsg) msgPack.Msgs = append(msgPack.Msgs, msg)
if err := ddStream.Broadcast(&msgPack); err != nil { c.dmlChannels.BroadcastAll(&msgPack)
return err
}
return nil return nil
} }
@ -968,7 +969,13 @@ func (c *Core) Init() error {
if initError = c.msFactory.SetParams(m); initError != nil { if initError = c.msFactory.SetParams(m); initError != nil {
return return
} }
c.dmlChannels = newDMLChannels(c)
pc := c.MetaTable.ListCollectionPhysicalChannels()
c.dmlChannels.AddProducerChannles(pc...)
c.chanTimeTick = newTimeTickSync(c) c.chanTimeTick = newTimeTickSync(c)
c.chanTimeTick.AddProxyNode(c.session)
c.proxyClientManager = newProxyClientManager(c) c.proxyClientManager = newProxyClientManager(c)
c.proxyNodeManager, initError = newProxyNodeManager( c.proxyNodeManager, initError = newProxyNodeManager(
@ -1096,7 +1103,6 @@ func (c *Core) Start() error {
} }
log.Debug("MasterService", zap.Int64("node id", c.session.ServerID)) log.Debug("MasterService", zap.Int64("node id", c.session.ServerID))
log.Debug("MasterService", zap.String("dd channel name", Params.DdChannel))
log.Debug("MasterService", zap.String("time tick channel name", Params.TimeTickChannel)) log.Debug("MasterService", zap.String("time tick channel name", Params.TimeTickChannel))
c.startOnce.Do(func() { c.startOnce.Do(func() {
@ -1163,16 +1169,6 @@ func (c *Core) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse
}, nil }, nil
} }
func (c *Core) GetDdChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Value: Params.DdChannel,
}, nil
}
func (c *Core) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) { func (c *Core) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{ return &milvuspb.StringResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{

View File

@ -174,7 +174,7 @@ func (idx *indexMock) getFileArray() []string {
return ret return ret
} }
func consumeMsgChan(timeout time.Duration, targetChan <-chan *msgstream.MsgPack) { func clearMsgChan(timeout time.Duration, targetChan <-chan *msgstream.MsgPack) {
ch := time.After(timeout) ch := time.After(timeout)
for { for {
select { select {
@ -232,6 +232,27 @@ func GenFlushedSegMsgPack(segID typeutil.UniqueID) *msgstream.MsgPack {
return &msgPack return &msgPack
} }
func getNotTtMsg(ctx context.Context, n int, ch <-chan *msgstream.MsgPack) []msgstream.TsMsg {
ret := make([]msgstream.TsMsg, 0, n)
for {
select {
case <-ctx.Done():
return nil
case msg, ok := <-ch:
if ok {
for _, v := range msg.Msgs {
if _, ok := v.(*msgstream.TimeTickMsg); !ok {
ret = append(ret, v)
}
}
if len(ret) >= n {
return ret
}
}
}
}
}
func TestMasterService(t *testing.T) { func TestMasterService(t *testing.T) {
const ( const (
dbName = "testDb" dbName = "testDb"
@ -250,7 +271,6 @@ func TestMasterService(t *testing.T) {
randVal := rand.Int() randVal := rand.Int()
Params.TimeTickChannel = fmt.Sprintf("master-time-tick-%d", randVal) Params.TimeTickChannel = fmt.Sprintf("master-time-tick-%d", randVal)
Params.DdChannel = fmt.Sprintf("master-dd-%d", randVal)
Params.StatisticsChannel = fmt.Sprintf("master-statistics-%d", randVal) Params.StatisticsChannel = fmt.Sprintf("master-statistics-%d", randVal)
Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath) Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath)
Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath) Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath)
@ -322,17 +342,15 @@ func TestMasterService(t *testing.T) {
timeTickStream.AsConsumer([]string{Params.TimeTickChannel}, Params.MsgChannelSubName) timeTickStream.AsConsumer([]string{Params.TimeTickChannel}, Params.MsgChannelSubName)
timeTickStream.Start() timeTickStream.Start()
ddStream, _ := tmpFactory.NewMsgStream(ctx) dmlStream, _ := tmpFactory.NewMsgStream(ctx)
ddStream.AsConsumer([]string{Params.DdChannel}, Params.MsgChannelSubName)
ddStream.Start()
// test dataServiceSegmentStream seek // test dataServiceSegmentStream seek
dataNodeSubName := Params.MsgChannelSubName + "dn" dataNodeSubName := Params.MsgChannelSubName + "dn"
flushedSegStream, _ := tmpFactory.NewMsgStream(ctx) flushedSegStream, _ := tmpFactory.NewMsgStream(ctx)
flushedSegStream.AsConsumer([]string{Params.DataServiceSegmentChannel}, dataNodeSubName) flushedSegStream.AsConsumer([]string{Params.DataServiceSegmentChannel}, dataNodeSubName)
flushedSegStream.Start() flushedSegStream.Start()
msgPack := GenFlushedSegMsgPack(9999) msgPackTmp := GenFlushedSegMsgPack(9999)
err = dataServiceSegmentStream.Produce(msgPack) err = dataServiceSegmentStream.Produce(msgPackTmp)
assert.Nil(t, err) assert.Nil(t, err)
flushedSegMsgPack := flushedSegStream.Consume() flushedSegMsgPack := flushedSegStream.Consume()
@ -360,27 +378,6 @@ func TestMasterService(t *testing.T) {
time.Sleep(time.Second) time.Sleep(time.Second)
getNotTtMsg := func(n int, ch <-chan *msgstream.MsgPack) []msgstream.TsMsg {
ret := make([]msgstream.TsMsg, 0, n)
for {
select {
case <-ctx.Done():
return nil
case msg, ok := <-ch:
if ok {
for _, v := range msg.Msgs {
if _, ok := v.(*msgstream.TimeTickMsg); !ok {
ret = append(ret, v)
}
}
if len(ret) >= n {
return ret
}
}
}
}
}
t.Run("time tick", func(t *testing.T) { t.Run("time tick", func(t *testing.T) {
ttmsg, ok := <-timeTickStream.Chan() ttmsg, ok := <-timeTickStream.Chan()
assert.True(t, ok) assert.True(t, ok)
@ -397,14 +394,6 @@ func TestMasterService(t *testing.T) {
assert.True(t, ok) assert.True(t, ok)
assert.Greater(t, ttm2.Base.Timestamp, uint64(0)) assert.Greater(t, ttm2.Base.Timestamp, uint64(0))
assert.Equal(t, ttm2.Base.Timestamp, ttm.Base.Timestamp+1) assert.Equal(t, ttm2.Base.Timestamp, ttm.Base.Timestamp+1)
ddmsg, ok := <-ddStream.Chan()
assert.True(t, ok)
assert.Equal(t, 1, len(ddmsg.Msgs))
ddm, ok := (ddmsg.Msgs[0]).(*msgstream.TimeTickMsg)
assert.True(t, ok)
assert.Greater(t, ddm.Base.Timestamp, uint64(0))
assert.Equal(t, ttm.Base.Timestamp, ddm.Base.Timestamp)
}) })
t.Run("create collection", func(t *testing.T) { t.Run("create collection", func(t *testing.T) {
@ -446,10 +435,17 @@ func TestMasterService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
msgs := getNotTtMsg(2, ddStream.Chan()) assert.Equal(t, 2, len(core.MetaTable.vChan2Chan))
assert.Equal(t, 2, len(msgs)) assert.Equal(t, 2, len(core.dmlChannels.dml))
createMsg, ok := (msgs[0]).(*msgstream.CreateCollectionMsg) pChan := core.MetaTable.ListCollectionPhysicalChannels()
dmlStream.AsConsumer([]string{pChan[0]}, Params.MsgChannelSubName)
dmlStream.Start()
// get CreateCollectionMsg
msgPack, ok := <-dmlStream.Chan()
assert.True(t, ok)
createMsg, ok := (msgPack.Msgs[0]).(*msgstream.CreateCollectionMsg)
assert.True(t, ok) assert.True(t, ok)
createMeta, err := core.MetaTable.GetCollectionByName(collName, 0) createMeta, err := core.MetaTable.GetCollectionByName(collName, 0)
assert.Nil(t, err) assert.Nil(t, err)
@ -463,11 +459,23 @@ func TestMasterService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, createMeta.PhysicalChannelNames[0], chanName) assert.Equal(t, createMeta.PhysicalChannelNames[0], chanName)
createPart, ok := (msgs[1]).(*msgstream.CreatePartitionMsg) // get CreatePartitionMsg
msgPack, ok = <-dmlStream.Chan()
assert.True(t, ok)
createPart, ok := (msgPack.Msgs[0]).(*msgstream.CreatePartitionMsg)
assert.True(t, ok) assert.True(t, ok)
assert.Equal(t, collName, createPart.CollectionName) assert.Equal(t, collName, createPart.CollectionName)
assert.Equal(t, createMeta.PartitionIDs[0], createPart.PartitionID) assert.Equal(t, createMeta.PartitionIDs[0], createPart.PartitionID)
// get TimeTickMsg
msgPack, ok = <-dmlStream.Chan()
assert.True(t, ok)
assert.Equal(t, 1, len(msgPack.Msgs))
ddm, ok := (msgPack.Msgs[0]).(*msgstream.TimeTickMsg)
assert.True(t, ok)
assert.Greater(t, ddm.Base.Timestamp, uint64(0))
// check invalid operation
req.Base.MsgID = 101 req.Base.MsgID = 101
req.Base.Timestamp = 101 req.Base.Timestamp = 101
req.Base.SourceID = 101 req.Base.SourceID = 101
@ -494,7 +502,7 @@ func TestMasterService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
msgs = getNotTtMsg(1, ddStream.Chan()) msgs := getNotTtMsg(ctx, 1, dmlStream.Chan())
createMsg, ok = (msgs[0]).(*msgstream.CreateCollectionMsg) createMsg, ok = (msgs[0]).(*msgstream.CreateCollectionMsg)
assert.True(t, ok) assert.True(t, ok)
createMeta, err = core.MetaTable.GetCollectionByName("testColl-again", 0) createMeta, err = core.MetaTable.GetCollectionByName("testColl-again", 0)
@ -623,7 +631,7 @@ func TestMasterService(t *testing.T) {
CollectionName: collName, CollectionName: collName,
PartitionName: partName, PartitionName: partName,
} }
consumeMsgChan(time.Second, ddStream.Chan()) clearMsgChan(10*time.Millisecond, dmlStream.Chan())
status, err := core.CreatePartition(ctx, req) status, err := core.CreatePartition(ctx, req)
assert.Nil(t, err) assert.Nil(t, err)
t.Log(status.Reason) t.Log(status.Reason)
@ -635,7 +643,7 @@ func TestMasterService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, partName, partMeta.PartitionName) assert.Equal(t, partName, partMeta.PartitionName)
msgs := getNotTtMsg(1, ddStream.Chan()) msgs := getNotTtMsg(ctx, 1, dmlStream.Chan())
assert.Equal(t, 1, len(msgs)) assert.Equal(t, 1, len(msgs))
partMsg, ok := (msgs[0]).(*msgstream.CreatePartitionMsg) partMsg, ok := (msgs[0]).(*msgstream.CreatePartitionMsg)
assert.True(t, ok) assert.True(t, ok)
@ -989,7 +997,7 @@ func TestMasterService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, Params.DefaultPartitionName, partMeta.PartitionName) assert.Equal(t, Params.DefaultPartitionName, partMeta.PartitionName)
msgs := getNotTtMsg(1, ddStream.Chan()) msgs := getNotTtMsg(ctx, 1, dmlStream.Chan())
assert.Equal(t, 1, len(msgs)) assert.Equal(t, 1, len(msgs))
dmsg, ok := (msgs[0]).(*msgstream.DropPartitionMsg) dmsg, ok := (msgs[0]).(*msgstream.DropPartitionMsg)
assert.True(t, ok) assert.True(t, ok)
@ -1038,7 +1046,7 @@ func TestMasterService(t *testing.T) {
_, err = core.MetaTable.GetChanNameByVirtualChan(vChanName) _, err = core.MetaTable.GetChanNameByVirtualChan(vChanName)
assert.NotNil(t, err) assert.NotNil(t, err)
msgs := getNotTtMsg(1, ddStream.Chan()) msgs := getNotTtMsg(ctx, 1, dmlStream.Chan())
assert.Equal(t, 1, len(msgs)) assert.Equal(t, 1, len(msgs))
dmsg, ok := (msgs[0]).(*msgstream.DropCollectionMsg) dmsg, ok := (msgs[0]).(*msgstream.DropCollectionMsg)
assert.True(t, ok) assert.True(t, ok)
@ -1067,7 +1075,6 @@ func TestMasterService(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.ErrorCode)
time.Sleep(time.Second) time.Sleep(time.Second)
//assert.Zero(t, len(ddStream.Chan()))
collArray = pnm.GetCollArray() collArray = pnm.GetCollArray()
assert.Equal(t, 3, len(collArray)) assert.Equal(t, 3, len(collArray))
assert.Equal(t, collName, collArray[2]) assert.Equal(t, collName, collArray[2])
@ -1441,8 +1448,6 @@ func TestMasterService(t *testing.T) {
t.Run("get_channels", func(t *testing.T) { t.Run("get_channels", func(t *testing.T) {
_, err := core.GetTimeTickChannel(ctx) _, err := core.GetTimeTickChannel(ctx)
assert.Nil(t, err) assert.Nil(t, err)
_, err = core.GetDdChannel(ctx)
assert.Nil(t, err)
_, err = core.GetStatisticsChannel(ctx) _, err = core.GetStatisticsChannel(ctx)
assert.Nil(t, err) assert.Nil(t, err)
}) })
@ -1490,7 +1495,7 @@ func TestMasterService(t *testing.T) {
s, _ := core.UpdateChannelTimeTick(ctx, msg0) s, _ := core.UpdateChannelTimeTick(ctx, msg0)
assert.Equal(t, commonpb.ErrorCode_Success, s.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, s.ErrorCode)
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
t.Log(core.chanTimeTick.proxyTimeTick) //t.Log(core.chanTimeTick.proxyTimeTick)
msg1 := &internalpb.ChannelTimeTickMsg{ msg1 := &internalpb.ChannelTimeTickMsg{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
@ -1514,8 +1519,11 @@ func TestMasterService(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, s.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, s.ErrorCode)
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
assert.Equal(t, 2, core.chanTimeTick.GetProxyNodeNum()) // 2 proxy nodes, 1 master
assert.Equal(t, 3, core.chanTimeTick.GetChanNum()) assert.Equal(t, 3, core.chanTimeTick.GetProxyNodeNum())
// 3 proxy node channels, 2 master channels
assert.Equal(t, 5, core.chanTimeTick.GetChanNum())
}) })
err = core.Stop() err = core.Stop()
@ -1734,7 +1742,6 @@ func TestMasterService2(t *testing.T) {
randVal := rand.Int() randVal := rand.Int()
Params.TimeTickChannel = fmt.Sprintf("master-time-tick-%d", randVal) Params.TimeTickChannel = fmt.Sprintf("master-time-tick-%d", randVal)
Params.DdChannel = fmt.Sprintf("master-dd-%d", randVal)
Params.StatisticsChannel = fmt.Sprintf("master-statistics-%d", randVal) Params.StatisticsChannel = fmt.Sprintf("master-statistics-%d", randVal)
Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath) Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath)
Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath) Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath)
@ -1788,28 +1795,8 @@ func TestMasterService2(t *testing.T) {
timeTickStream.AsConsumer([]string{Params.TimeTickChannel}, Params.MsgChannelSubName) timeTickStream.AsConsumer([]string{Params.TimeTickChannel}, Params.MsgChannelSubName)
timeTickStream.Start() timeTickStream.Start()
ddStream, _ := msFactory.NewMsgStream(ctx)
ddStream.AsConsumer([]string{Params.DdChannel}, Params.MsgChannelSubName)
ddStream.Start()
time.Sleep(time.Second) time.Sleep(time.Second)
getNotTTMsg := func(ch <-chan *msgstream.MsgPack, n int) []msgstream.TsMsg {
msg := make([]msgstream.TsMsg, 0, n)
for {
m, ok := <-ch
assert.True(t, ok)
for _, m := range m.Msgs {
if _, ok := (m).(*msgstream.TimeTickMsg); !ok {
msg = append(msg, m)
}
}
if len(msg) >= n {
return msg
}
}
}
t.Run("time tick", func(t *testing.T) { t.Run("time tick", func(t *testing.T) {
ttmsg, ok := <-timeTickStream.Chan() ttmsg, ok := <-timeTickStream.Chan()
assert.True(t, ok) assert.True(t, ok)
@ -1817,13 +1804,6 @@ func TestMasterService2(t *testing.T) {
ttm, ok := (ttmsg.Msgs[0]).(*msgstream.TimeTickMsg) ttm, ok := (ttmsg.Msgs[0]).(*msgstream.TimeTickMsg)
assert.True(t, ok) assert.True(t, ok)
assert.Greater(t, ttm.Base.Timestamp, typeutil.Timestamp(0)) assert.Greater(t, ttm.Base.Timestamp, typeutil.Timestamp(0))
ddmsg, ok := <-ddStream.Chan()
assert.True(t, ok)
assert.Equal(t, 1, len(ddmsg.Msgs))
ddm, ok := (ddmsg.Msgs[0]).(*msgstream.TimeTickMsg)
assert.True(t, ok)
assert.Greater(t, ddm.Base.Timestamp, typeutil.Timestamp(0))
}) })
t.Run("create collection", func(t *testing.T) { t.Run("create collection", func(t *testing.T) {
@ -1847,11 +1827,17 @@ func TestMasterService2(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
msg := getNotTTMsg(ddStream.Chan(), 2) pChan := core.MetaTable.ListCollectionPhysicalChannels()
assert.GreaterOrEqual(t, len(msg), 2) dmlStream, _ := msFactory.NewMsgStream(ctx)
m1, ok := (msg[0]).(*msgstream.CreateCollectionMsg) dmlStream.AsConsumer(pChan, Params.MsgChannelSubName)
dmlStream.Start()
msgs := getNotTtMsg(ctx, 2, dmlStream.Chan())
assert.Equal(t, 2, len(msgs))
m1, ok := (msgs[0]).(*msgstream.CreateCollectionMsg)
assert.True(t, ok) assert.True(t, ok)
m2, ok := (msg[1]).(*msgstream.CreatePartitionMsg) m2, ok := (msgs[1]).(*msgstream.CreatePartitionMsg)
assert.True(t, ok) assert.True(t, ok)
assert.Equal(t, m1.Base.Timestamp, m2.Base.Timestamp) assert.Equal(t, m1.Base.Timestamp, m2.Base.Timestamp)
t.Log("time tick", m1.Base.Timestamp) t.Log("time tick", m1.Base.Timestamp)

View File

@ -488,6 +488,30 @@ func (mt *metaTable) ListCollections(ts typeutil.Timestamp) (map[string]typeutil
return colls, nil return colls, nil
} }
// ListCollectionVirtualChannels list virtual channel of all the collection
func (mt *metaTable) ListCollectionVirtualChannels() []string {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
vlist := []string{}
for _, c := range mt.collID2Meta {
vlist = append(vlist, c.VirtualChannelNames...)
}
return vlist
}
// ListCollectionPhysicalChannels list physical channel of all the collection
func (mt *metaTable) ListCollectionPhysicalChannels() []string {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
plist := []string{}
for _, c := range mt.collID2Meta {
plist = append(plist, c.PhysicalChannelNames...)
}
return plist
}
func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string, partitionID typeutil.UniqueID, ddOpStr func(ts typeutil.Timestamp) (string, error)) (typeutil.Timestamp, error) { func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string, partitionID typeutil.UniqueID, ddOpStr func(ts typeutil.Timestamp) (string, error)) (typeutil.Timestamp, error) {
mt.ddLock.Lock() mt.ddLock.Lock()
defer mt.ddLock.Unlock() defer mt.ddLock.Unlock()

View File

@ -35,7 +35,6 @@ type ParamTable struct {
KvRootPath string KvRootPath string
MsgChannelSubName string MsgChannelSubName string
TimeTickChannel string TimeTickChannel string
DdChannel string
StatisticsChannel string StatisticsChannel string
DataServiceSegmentChannel string // data service create segment, or data node flush segment DataServiceSegmentChannel string // data service create segment, or data node flush segment
@ -68,7 +67,6 @@ func (p *ParamTable) Init() {
p.initMsgChannelSubName() p.initMsgChannelSubName()
p.initTimeTickChannel() p.initTimeTickChannel()
p.initDdChannelName()
p.initStatisticsChannelName() p.initStatisticsChannelName()
p.initSegmentInfoChannelName() p.initSegmentInfoChannelName()
@ -141,14 +139,6 @@ func (p *ParamTable) initTimeTickChannel() {
p.TimeTickChannel = channel p.TimeTickChannel = channel
} }
func (p *ParamTable) initDdChannelName() {
channel, err := p.Load("msgChannel.chanNamePrefix.dataDefinition")
if err != nil {
panic(err)
}
p.DdChannel = channel
}
func (p *ParamTable) initStatisticsChannelName() { func (p *ParamTable) initStatisticsChannelName() {
channel, err := p.Load("msgChannel.chanNamePrefix.masterStatistics") channel, err := p.Load("msgChannel.chanNamePrefix.masterStatistics")
if err != nil { if err != nil {

View File

@ -38,9 +38,6 @@ func TestParamTable(t *testing.T) {
assert.NotEqual(t, Params.TimeTickChannel, "") assert.NotEqual(t, Params.TimeTickChannel, "")
t.Logf("master time tick channel = %s", Params.TimeTickChannel) t.Logf("master time tick channel = %s", Params.TimeTickChannel)
assert.NotEqual(t, Params.DdChannel, "")
t.Logf("master dd channel = %s", Params.DdChannel)
assert.NotEqual(t, Params.StatisticsChannel, "") assert.NotEqual(t, Params.StatisticsChannel, "")
t.Logf("master statistics channel = %s", Params.StatisticsChannel) t.Logf("master statistics channel = %s", Params.StatisticsChannel)

View File

@ -236,6 +236,9 @@ func (t *CreateCollectionReqTask) Execute(ctx context.Context) error {
return err return err
} }
// add dml channel before send dd msg
t.core.dmlChannels.AddProducerChannles(chanNames...)
err = t.core.SendDdCreateCollectionReq(ctx, &ddCollReq) err = t.core.SendDdCreateCollectionReq(ctx, &ddCollReq)
if err != nil { if err != nil {
return err return err
@ -301,6 +304,9 @@ func (t *DropCollectionReqTask) Execute(ctx context.Context) error {
t.core.SendTimeTick(ts) t.core.SendTimeTick(ts)
// remove dml channel after send dd msg
t.core.dmlChannels.RemoveProducerChannels(collMeta.PhysicalChannelNames...)
//notify query service to release collection //notify query service to release collection
go func() { go func() {
if err = t.core.CallReleaseCollectionService(t.core.ctx, ts, 0, collMeta.ID); err != nil { if err = t.core.CallReleaseCollectionService(t.core.ctx, ts, 0, collMeta.ID); err != nil {

View File

@ -15,6 +15,7 @@ import (
"context" "context"
"fmt" "fmt"
"math/rand" "math/rand"
"sync"
"testing" "testing"
"github.com/milvus-io/milvus/internal/msgstream" "github.com/milvus-io/milvus/internal/msgstream"
@ -26,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -100,7 +102,6 @@ func BenchmarkAllocTimestamp(b *testing.B) {
randVal := rand.Int() randVal := rand.Int()
Params.TimeTickChannel = fmt.Sprintf("master-time-tick-%d", randVal) Params.TimeTickChannel = fmt.Sprintf("master-time-tick-%d", randVal)
Params.DdChannel = fmt.Sprintf("master-dd-%d", randVal)
Params.StatisticsChannel = fmt.Sprintf("master-statistics-%d", randVal) Params.StatisticsChannel = fmt.Sprintf("master-statistics-%d", randVal)
Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath) Params.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.MetaRootPath)
Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath) Params.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.KvRootPath)
@ -115,6 +116,17 @@ func BenchmarkAllocTimestamp(b *testing.B) {
err = core.SetQueryService(&tbq{}) err = core.SetQueryService(&tbq{})
assert.Nil(b, err) assert.Nil(b, err)
err = core.Register()
assert.Nil(b, err)
pnm := &proxyNodeMock{
collArray: make([]string, 0, 16),
mutex: sync.Mutex{},
}
core.NewProxyClient = func(*sessionutil.Session) (types.ProxyNode, error) {
return pnm, nil
}
err = core.Init() err = core.Init()
assert.Nil(b, err) assert.Nil(b, err)

View File

@ -30,7 +30,6 @@ type timetickSync struct {
core *Core core *Core
lock sync.Mutex lock sync.Mutex
proxyTimeTick map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg proxyTimeTick map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg
chanStream map[string]msgstream.MsgStream
sendChan chan map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg sendChan chan map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg
} }
@ -39,7 +38,6 @@ func newTimeTickSync(core *Core) *timetickSync {
lock: sync.Mutex{}, lock: sync.Mutex{},
core: core, core: core,
proxyTimeTick: make(map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg), proxyTimeTick: make(map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg),
chanStream: make(map[string]msgstream.MsgStream),
sendChan: make(chan map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg, 16), sendChan: make(chan map[typeutil.UniqueID]*internalpb.ChannelTimeTickMsg, 16),
} }
} }
@ -157,23 +155,7 @@ func (t *timetickSync) SendChannelTimeTick(chanName string, ts typeutil.Timestam
} }
msgPack.Msgs = append(msgPack.Msgs, timeTickMsg) msgPack.Msgs = append(msgPack.Msgs, timeTickMsg)
t.lock.Lock() err := t.core.dmlChannels.Broadcast(chanName, &msgPack)
defer t.lock.Unlock()
// send timetick msg to msg stream
var err error
var stream msgstream.MsgStream
stream, ok := t.chanStream[chanName]
if !ok {
stream, err = t.core.msFactory.NewMsgStream(t.core.ctx)
if err != nil {
return err
}
stream.AsProducer([]string{chanName})
t.chanStream[chanName] = stream
}
err = stream.Broadcast(&msgPack)
if err == nil { if err == nil {
metrics.MasterInsertChannelTimeTick.WithLabelValues(chanName).Set(float64(tsoutil.Mod24H(ts))) metrics.MasterInsertChannelTimeTick.WithLabelValues(chanName).Set(float64(tsoutil.Mod24H(ts)))
} }
@ -189,7 +171,5 @@ func (t *timetickSync) GetProxyNodeNum() int {
// GetChanNum return the num of channel // GetChanNum return the num of channel
func (t *timetickSync) GetChanNum() int { func (t *timetickSync) GetChanNum() int {
t.lock.Lock() return t.core.dmlChannels.GetNumChannles()
defer t.lock.Unlock()
return len(t.chanStream)
} }

View File

@ -94,8 +94,6 @@ service MasterService {
rpc AllocTimestamp(AllocTimestampRequest) returns (AllocTimestampResponse) {} rpc AllocTimestamp(AllocTimestampRequest) returns (AllocTimestampResponse) {}
rpc AllocID(AllocIDRequest) returns (AllocIDResponse) {} rpc AllocID(AllocIDRequest) returns (AllocIDResponse) {}
rpc UpdateChannelTimeTick(internal.ChannelTimeTickMsg) returns (common.Status) {} rpc UpdateChannelTimeTick(internal.ChannelTimeTickMsg) returns (common.Status) {}
rpc GetDdChannel(internal.GetDdChannelRequest) returns (milvus.StringResponse) {}
} }
message AllocTimestampRequest { message AllocTimestampRequest {

View File

@ -241,52 +241,51 @@ func init() {
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) } func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{ var fileDescriptor_f9c348dec43a6705 = []byte{
// 710 bytes of a gzipped FileDescriptorProto // 694 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0x5b, 0x4f, 0xdb, 0x3c, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0x5d, 0x4f, 0xdb, 0x3e,
0x18, 0xc7, 0x69, 0xe1, 0xe5, 0x15, 0x0f, 0x6d, 0x41, 0x1e, 0x4c, 0xa8, 0xe3, 0x82, 0x95, 0x1d, 0x14, 0xc6, 0x69, 0xe1, 0xcf, 0x5f, 0x1c, 0xda, 0x82, 0x3c, 0x98, 0x50, 0xc7, 0x05, 0x2b, 0x7b,
0x5a, 0x60, 0x2d, 0x02, 0xed, 0x03, 0x8c, 0x56, 0x82, 0x5e, 0x20, 0x6d, 0x2d, 0x4c, 0x3b, 0x08, 0x69, 0x81, 0xb5, 0x08, 0xb4, 0x0f, 0x30, 0x5a, 0x09, 0x7a, 0x81, 0xb4, 0xb5, 0x30, 0xed, 0x45,
0x21, 0x37, 0x58, 0xad, 0x45, 0x62, 0x87, 0xd8, 0x85, 0xdd, 0xee, 0xd3, 0xee, 0x6b, 0x4c, 0x89, 0x08, 0xb9, 0xc1, 0x6a, 0x2d, 0x12, 0x3b, 0xc4, 0x2e, 0xec, 0x76, 0xd2, 0x3e, 0xf8, 0x94, 0x38,
0x63, 0x37, 0x69, 0x0e, 0x4b, 0xb5, 0xdd, 0xc5, 0xf1, 0xcf, 0xff, 0xbf, 0x9f, 0x83, 0xf4, 0x18, 0x76, 0x93, 0xe6, 0x65, 0xa9, 0xb6, 0xbb, 0xba, 0xf9, 0xf9, 0x79, 0x7c, 0xce, 0x73, 0x24, 0x1b,
0x2a, 0x0e, 0x16, 0x92, 0x78, 0x6d, 0xd7, 0xe3, 0x92, 0xa3, 0x67, 0x0e, 0xb5, 0x1f, 0xa7, 0x42, 0x2a, 0x0e, 0x16, 0x92, 0x78, 0x6d, 0xd7, 0xe3, 0x92, 0xa3, 0x67, 0x0e, 0xb5, 0x1f, 0xa7, 0x42,
0xad, 0xda, 0x6a, 0xab, 0x5e, 0xb1, 0xb8, 0xe3, 0x70, 0xa6, 0x7e, 0xd6, 0x2b, 0x51, 0xa4, 0x5e, 0xad, 0xda, 0xea, 0x53, 0xbd, 0x62, 0x71, 0xc7, 0xe1, 0x4c, 0xfd, 0x59, 0xaf, 0x44, 0x91, 0x7a,
0xa3, 0x4c, 0x12, 0x8f, 0x61, 0x5b, 0xad, 0x1b, 0xb7, 0xb0, 0xfd, 0xc1, 0xb6, 0xb9, 0x75, 0x45, 0x8d, 0x32, 0x49, 0x3c, 0x86, 0x6d, 0xb5, 0x6e, 0xdc, 0xc2, 0xf6, 0x07, 0xdb, 0xe6, 0xd6, 0x15,
0x1d, 0x22, 0x24, 0x76, 0xdc, 0x01, 0x79, 0x98, 0x12, 0x21, 0xd1, 0x31, 0xac, 0x8c, 0xb0, 0x20, 0x75, 0x88, 0x90, 0xd8, 0x71, 0x07, 0xe4, 0x61, 0x4a, 0x84, 0x44, 0xc7, 0xb0, 0x32, 0xc2, 0x82,
0x3b, 0xa5, 0xbd, 0x52, 0x73, 0xfd, 0x64, 0xb7, 0x1d, 0x33, 0x0a, 0x0d, 0x2e, 0xc5, 0xf8, 0x0c, 0xec, 0x94, 0xf6, 0x4a, 0xcd, 0xf5, 0x93, 0xdd, 0x76, 0xcc, 0x28, 0x34, 0xb8, 0x14, 0xe3, 0x33,
0x0b, 0x32, 0x08, 0x48, 0xb4, 0x05, 0xff, 0x59, 0x7c, 0xca, 0xe4, 0xce, 0xf2, 0x5e, 0xa9, 0x59, 0x2c, 0xc8, 0x20, 0x20, 0xd1, 0x16, 0xfc, 0x67, 0xf1, 0x29, 0x93, 0x3b, 0xcb, 0x7b, 0xa5, 0x66,
0x1d, 0xa8, 0x45, 0xe3, 0x67, 0x09, 0x9e, 0xcf, 0x3b, 0x08, 0x97, 0x33, 0x41, 0xd0, 0x29, 0xac, 0x75, 0xa0, 0x16, 0x8d, 0x9f, 0x25, 0x78, 0x3e, 0xef, 0x20, 0x5c, 0xce, 0x04, 0x41, 0xa7, 0xb0,
0x0a, 0x89, 0xe5, 0x54, 0x84, 0x26, 0x2f, 0x52, 0x4d, 0x86, 0x01, 0x32, 0x08, 0x51, 0xb4, 0x0b, 0x2a, 0x24, 0x96, 0x53, 0x11, 0x9a, 0xbc, 0x48, 0x35, 0x19, 0x06, 0xc8, 0x20, 0x44, 0xd1, 0x2e,
0x6b, 0x52, 0x2b, 0xed, 0x94, 0xf7, 0x4a, 0xcd, 0x95, 0xc1, 0xec, 0x47, 0xc6, 0x1d, 0xbe, 0x40, 0xac, 0x49, 0xad, 0xb4, 0x53, 0xde, 0x2b, 0x35, 0x57, 0x06, 0xb3, 0x3f, 0x32, 0xce, 0xf0, 0x05,
0x2d, 0xb8, 0x42, 0xbf, 0xf7, 0x0f, 0xa2, 0x2b, 0x47, 0x95, 0x6d, 0xd8, 0x30, 0xca, 0x7f, 0x13, 0x6a, 0xc1, 0x11, 0xfa, 0xbd, 0x7f, 0x50, 0x5d, 0x39, 0xaa, 0x6c, 0xc3, 0x86, 0x51, 0xfe, 0x9b,
0x55, 0x0d, 0xca, 0xfd, 0x5e, 0x20, 0xbd, 0x3c, 0x28, 0xf7, 0x7b, 0xe9, 0x71, 0x9c, 0xfc, 0xda, 0xaa, 0x6a, 0x50, 0xee, 0xf7, 0x02, 0xe9, 0xe5, 0x41, 0xb9, 0xdf, 0x4b, 0xaf, 0xe3, 0xe4, 0xd7,
0x84, 0xea, 0x65, 0x50, 0xe3, 0x21, 0xf1, 0x1e, 0xa9, 0x45, 0x90, 0x0b, 0xe8, 0x9c, 0xc8, 0x2e, 0x26, 0x54, 0x2f, 0x83, 0x8c, 0x87, 0xc4, 0x7b, 0xa4, 0x16, 0x41, 0x2e, 0xa0, 0x73, 0x22, 0xbb,
0x77, 0x5c, 0xce, 0x08, 0x93, 0xbe, 0x2a, 0x11, 0xe8, 0x38, 0x6e, 0x69, 0x4a, 0x9e, 0x44, 0xc3, 0xdc, 0x71, 0x39, 0x23, 0x4c, 0xfa, 0xaa, 0x44, 0xa0, 0xe3, 0xb8, 0xa5, 0x89, 0x3c, 0x89, 0x86,
0x7c, 0xd4, 0xdf, 0x64, 0x9c, 0x98, 0xc3, 0x1b, 0x4b, 0xc8, 0x09, 0x1c, 0xfd, 0x62, 0x5e, 0x51, 0xfd, 0xa8, 0xbf, 0xc9, 0xd8, 0x31, 0x87, 0x37, 0x96, 0x90, 0x13, 0x38, 0xfa, 0x61, 0x5e, 0x51,
0xeb, 0xbe, 0x3b, 0xc1, 0x8c, 0x11, 0x3b, 0xcf, 0x71, 0x0e, 0xd5, 0x8e, 0xfb, 0xf1, 0x13, 0xe1, 0xeb, 0xbe, 0x3b, 0xc1, 0x8c, 0x11, 0x3b, 0xcf, 0x71, 0x0e, 0xd5, 0x8e, 0xfb, 0xf1, 0x1d, 0xe1,
0x62, 0x28, 0x3d, 0xca, 0xc6, 0x3a, 0x97, 0x8d, 0x25, 0xf4, 0x00, 0x5b, 0xe7, 0x24, 0x70, 0xa7, 0x62, 0x28, 0x3d, 0xca, 0xc6, 0xba, 0x97, 0x8d, 0x25, 0xf4, 0x00, 0x5b, 0xe7, 0x24, 0x70, 0xa7,
0x42, 0x52, 0x4b, 0x68, 0xc3, 0x93, 0x6c, 0xc3, 0x04, 0xbc, 0xa0, 0xe5, 0x2d, 0x6c, 0x76, 0x3d, 0x42, 0x52, 0x4b, 0x68, 0xc3, 0x93, 0x6c, 0xc3, 0x04, 0xbc, 0xa0, 0xe5, 0x2d, 0x6c, 0x76, 0x3d,
0x82, 0x25, 0xe9, 0x72, 0xdb, 0x26, 0x96, 0xa4, 0x9c, 0xa1, 0xa3, 0xd4, 0xa3, 0xf3, 0x98, 0x36, 0x82, 0x25, 0xe9, 0x72, 0xdb, 0x26, 0x96, 0xa4, 0x9c, 0xa1, 0xa3, 0xd4, 0xad, 0xf3, 0x98, 0x36,
0xca, 0x2b, 0x79, 0x63, 0x09, 0x7d, 0x87, 0x5a, 0xcf, 0xe3, 0x6e, 0x44, 0xfe, 0x20, 0x55, 0x3e, 0xca, 0x8b, 0xbc, 0xb1, 0x84, 0xbe, 0x43, 0xad, 0xe7, 0x71, 0x37, 0x22, 0x7f, 0x90, 0x2a, 0x1f,
0x0e, 0x15, 0x14, 0xbf, 0x85, 0xea, 0x05, 0x16, 0x11, 0xed, 0x56, 0xaa, 0x76, 0x8c, 0xd1, 0xd2, 0x87, 0x0a, 0x8a, 0xdf, 0x42, 0xf5, 0x02, 0x8b, 0x88, 0x76, 0x2b, 0x55, 0x3b, 0xc6, 0x68, 0xe9,
0x2f, 0x53, 0xd1, 0x33, 0xce, 0xed, 0x48, 0x7a, 0x9e, 0x00, 0xf5, 0x88, 0xb0, 0x3c, 0x3a, 0x8a, 0x97, 0xa9, 0xe8, 0x19, 0xe7, 0x76, 0xa4, 0x3d, 0x4f, 0x80, 0x7a, 0x44, 0x58, 0x1e, 0x1d, 0x45,
0x26, 0xa8, 0x9d, 0x1e, 0x41, 0x02, 0xd4, 0x56, 0x9d, 0xc2, 0xbc, 0x31, 0x66, 0xb0, 0x31, 0x9c, 0x1b, 0xd4, 0x4e, 0xaf, 0x20, 0x01, 0x6a, 0xab, 0x4e, 0x61, 0xde, 0x18, 0x33, 0xd8, 0x18, 0x4e,
0xf0, 0xa7, 0xd9, 0x9e, 0x40, 0x87, 0xe9, 0x15, 0x8d, 0x53, 0xda, 0xf2, 0xa8, 0x18, 0x6c, 0xfc, 0xf8, 0xd3, 0xec, 0x9b, 0x40, 0x87, 0xe9, 0x89, 0xc6, 0x29, 0x6d, 0x79, 0x54, 0x0c, 0x36, 0x7e,
0x6e, 0x60, 0x43, 0x15, 0xf8, 0x23, 0xf6, 0x24, 0x0d, 0xa2, 0x3c, 0xcc, 0x69, 0x03, 0x43, 0x15, 0x37, 0xb0, 0xa1, 0x02, 0xfe, 0x88, 0x3d, 0x49, 0x83, 0x2a, 0x0f, 0x73, 0xc6, 0xc0, 0x50, 0x05,
0x2c, 0xd4, 0x57, 0xa8, 0xfa, 0x05, 0x9e, 0x89, 0xb7, 0x32, 0x9b, 0x60, 0x51, 0xe9, 0x1b, 0xa8, 0x83, 0xfa, 0x0a, 0x55, 0x3f, 0xe0, 0x99, 0x78, 0x2b, 0x73, 0x08, 0x16, 0x95, 0xbe, 0x81, 0xca,
0x5c, 0x60, 0x31, 0x53, 0x6e, 0x66, 0xb5, 0x40, 0x42, 0xb8, 0x50, 0x07, 0xdc, 0x43, 0xcd, 0xcf, 0x05, 0x16, 0x33, 0xe5, 0x66, 0xd6, 0x08, 0x24, 0x84, 0x0b, 0x4d, 0xc0, 0x3d, 0xd4, 0xfc, 0xae,
0x9a, 0x39, 0x2c, 0x32, 0xfa, 0x37, 0x0e, 0x69, 0x8b, 0xc3, 0x42, 0x6c, 0xb4, 0xea, 0xba, 0x2b, 0x99, 0xcd, 0x22, 0x63, 0x7e, 0xe3, 0x90, 0xb6, 0x38, 0x2c, 0xc4, 0x46, 0x53, 0xd7, 0x53, 0x31,
0x86, 0x64, 0xec, 0x10, 0x26, 0x33, 0xaa, 0x30, 0x47, 0xe5, 0x57, 0x3d, 0x01, 0x1b, 0x3f, 0x02, 0x24, 0x63, 0x87, 0x30, 0x99, 0x91, 0xc2, 0x1c, 0x95, 0x9f, 0x7a, 0x02, 0x36, 0x7e, 0x04, 0x2a,
0x15, 0xff, 0x2e, 0xe1, 0x86, 0xc8, 0xc8, 0x5d, 0x14, 0xd1, 0x4e, 0xad, 0x02, 0xa4, 0xb1, 0xb9, 0xfe, 0x59, 0xc2, 0x0f, 0x22, 0xa3, 0x77, 0x51, 0x44, 0x3b, 0xb5, 0x0a, 0x90, 0xc6, 0xe6, 0x1a,
0x86, 0x75, 0xd5, 0x36, 0x7d, 0x76, 0x47, 0x7e, 0xa0, 0xb7, 0x39, 0x8d, 0x15, 0x10, 0x05, 0x2b, 0xd6, 0xd5, 0xd8, 0xf4, 0xd9, 0x1d, 0xf9, 0x81, 0xde, 0xe6, 0x0c, 0x56, 0x40, 0x14, 0x4c, 0x7e,
0x3f, 0x81, 0xaa, 0x0e, 0x4d, 0x09, 0xb7, 0x72, 0xc3, 0x8f, 0x49, 0x1f, 0x14, 0x41, 0x4d, 0x00, 0x02, 0x55, 0x5d, 0x9a, 0x12, 0x6e, 0xe5, 0x96, 0x1f, 0x93, 0x3e, 0x28, 0x82, 0x9a, 0x02, 0x3e,
0x9f, 0x60, 0xcd, 0x6f, 0x4d, 0xe5, 0xf2, 0x3a, 0xb3, 0x75, 0x17, 0xb9, 0xfc, 0x7d, 0x38, 0xa6, 0xc1, 0x9a, 0x3f, 0x9a, 0xca, 0xe5, 0x75, 0xe6, 0xe8, 0x2e, 0x72, 0xf8, 0xfb, 0xf0, 0x9a, 0x36,
0xcd, 0x4b, 0x21, 0xd1, 0x57, 0xea, 0xe9, 0x93, 0xfa, 0x60, 0x49, 0xf4, 0x55, 0x3a, 0x6b, 0xee, 0x2f, 0x85, 0xc4, 0x5c, 0xa9, 0xa7, 0x4f, 0xea, 0x83, 0x25, 0x31, 0x57, 0xe9, 0xac, 0x39, 0xff,
0xff, 0x19, 0xfe, 0x0f, 0x27, 0x37, 0xda, 0xcf, 0x3e, 0x69, 0x5e, 0x0c, 0xf5, 0x57, 0xf9, 0x90, 0x67, 0xf8, 0x3f, 0xbc, 0xb9, 0xd1, 0x7e, 0xf6, 0x4e, 0xf3, 0x62, 0xa8, 0xbf, 0xca, 0x87, 0x8c,
0xd1, 0xc5, 0xb0, 0x7d, 0xed, 0xde, 0xf9, 0x63, 0x41, 0x0d, 0x1f, 0x3d, 0xfe, 0xe6, 0x2b, 0x31, 0x2e, 0x86, 0xed, 0x6b, 0xf7, 0xce, 0xbf, 0x16, 0xd4, 0xe5, 0xa3, 0xaf, 0xbf, 0xf9, 0x24, 0x66,
0x1b, 0xb1, 0x71, 0xee, 0x52, 0x8c, 0xff, 0x94, 0x27, 0x0c, 0x95, 0x73, 0x22, 0x7b, 0x77, 0x7a, 0x57, 0x6c, 0x9c, 0xbb, 0x14, 0xe3, 0x3f, 0xf4, 0xe9, 0xec, 0xfd, 0xb7, 0xd3, 0x31, 0x95, 0x93,
0x16, 0x1e, 0x64, 0xcf, 0x42, 0x03, 0x2d, 0x36, 0x03, 0xcf, 0xde, 0x7f, 0x3b, 0x1d, 0x53, 0x39, 0xe9, 0xc8, 0xff, 0xd2, 0x51, 0xe8, 0x3b, 0xca, 0xc3, 0x5f, 0x1d, 0xad, 0xdc, 0x09, 0x76, 0x77,
0x99, 0x8e, 0x7c, 0xf3, 0x8e, 0xa2, 0xde, 0x51, 0x1e, 0x7e, 0x75, 0xb4, 0x45, 0x27, 0x50, 0xe9, 0xd4, 0x49, 0xdd, 0xd1, 0x68, 0x35, 0x58, 0x9f, 0xfe, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x99, 0x60,
0xa8, 0x64, 0xb8, 0xa3, 0xd1, 0x6a, 0xb0, 0x3e, 0xfd, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xee, 0x13, 0x10, 0x62, 0x42, 0x0a, 0x00, 0x00,
0xfc, 0x7b, 0xa5, 0x0a, 0x00, 0x00,
} }
// Reference imports to suppress errors if they are not otherwise used. // Reference imports to suppress errors if they are not otherwise used.
@ -367,7 +366,6 @@ type MasterServiceClient interface {
AllocTimestamp(ctx context.Context, in *AllocTimestampRequest, opts ...grpc.CallOption) (*AllocTimestampResponse, error) AllocTimestamp(ctx context.Context, in *AllocTimestampRequest, opts ...grpc.CallOption) (*AllocTimestampResponse, error)
AllocID(ctx context.Context, in *AllocIDRequest, opts ...grpc.CallOption) (*AllocIDResponse, error) AllocID(ctx context.Context, in *AllocIDRequest, opts ...grpc.CallOption) (*AllocIDResponse, error)
UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg, opts ...grpc.CallOption) (*commonpb.Status, error) UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg, opts ...grpc.CallOption) (*commonpb.Status, error)
GetDdChannel(ctx context.Context, in *internalpb.GetDdChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
} }
type masterServiceClient struct { type masterServiceClient struct {
@ -558,15 +556,6 @@ func (c *masterServiceClient) UpdateChannelTimeTick(ctx context.Context, in *int
return out, nil return out, nil
} }
func (c *masterServiceClient) GetDdChannel(ctx context.Context, in *internalpb.GetDdChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
out := new(milvuspb.StringResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetDdChannel", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// MasterServiceServer is the server API for MasterService service. // MasterServiceServer is the server API for MasterService service.
type MasterServiceServer interface { type MasterServiceServer interface {
GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error)
@ -635,7 +624,6 @@ type MasterServiceServer interface {
AllocTimestamp(context.Context, *AllocTimestampRequest) (*AllocTimestampResponse, error) AllocTimestamp(context.Context, *AllocTimestampRequest) (*AllocTimestampResponse, error)
AllocID(context.Context, *AllocIDRequest) (*AllocIDResponse, error) AllocID(context.Context, *AllocIDRequest) (*AllocIDResponse, error)
UpdateChannelTimeTick(context.Context, *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) UpdateChannelTimeTick(context.Context, *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error)
GetDdChannel(context.Context, *internalpb.GetDdChannelRequest) (*milvuspb.StringResponse, error)
} }
// UnimplementedMasterServiceServer can be embedded to have forward compatible implementations. // UnimplementedMasterServiceServer can be embedded to have forward compatible implementations.
@ -702,9 +690,6 @@ func (*UnimplementedMasterServiceServer) AllocID(ctx context.Context, req *Alloc
func (*UnimplementedMasterServiceServer) UpdateChannelTimeTick(ctx context.Context, req *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) { func (*UnimplementedMasterServiceServer) UpdateChannelTimeTick(ctx context.Context, req *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method UpdateChannelTimeTick not implemented") return nil, status.Errorf(codes.Unimplemented, "method UpdateChannelTimeTick not implemented")
} }
func (*UnimplementedMasterServiceServer) GetDdChannel(ctx context.Context, req *internalpb.GetDdChannelRequest) (*milvuspb.StringResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetDdChannel not implemented")
}
func RegisterMasterServiceServer(s *grpc.Server, srv MasterServiceServer) { func RegisterMasterServiceServer(s *grpc.Server, srv MasterServiceServer) {
s.RegisterService(&_MasterService_serviceDesc, srv) s.RegisterService(&_MasterService_serviceDesc, srv)
@ -1070,24 +1055,6 @@ func _MasterService_UpdateChannelTimeTick_Handler(srv interface{}, ctx context.C
return interceptor(ctx, in, info, handler) return interceptor(ctx, in, info, handler)
} }
func _MasterService_GetDdChannel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(internalpb.GetDdChannelRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetDdChannel(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetDdChannel",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetDdChannel(ctx, req.(*internalpb.GetDdChannelRequest))
}
return interceptor(ctx, in, info, handler)
}
var _MasterService_serviceDesc = grpc.ServiceDesc{ var _MasterService_serviceDesc = grpc.ServiceDesc{
ServiceName: "milvus.proto.master.MasterService", ServiceName: "milvus.proto.master.MasterService",
HandlerType: (*MasterServiceServer)(nil), HandlerType: (*MasterServiceServer)(nil),
@ -1172,10 +1139,6 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "UpdateChannelTimeTick", MethodName: "UpdateChannelTimeTick",
Handler: _MasterService_UpdateChannelTimeTick_Handler, Handler: _MasterService_UpdateChannelTimeTick_Handler,
}, },
{
MethodName: "GetDdChannel",
Handler: _MasterService_GetDdChannel_Handler,
},
}, },
Streams: []grpc.StreamDesc{}, Streams: []grpc.StreamDesc{},
Metadata: "master.proto", Metadata: "master.proto",

View File

@ -111,8 +111,6 @@ type MasterService interface {
//segment //segment
DescribeSegment(ctx context.Context, req *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) DescribeSegment(ctx context.Context, req *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error)
ShowSegments(ctx context.Context, req *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) ShowSegments(ctx context.Context, req *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error)
GetDdChannel(ctx context.Context) (*milvuspb.StringResponse, error)
} }
// MasterComponent is used by grpc server of master service // MasterComponent is used by grpc server of master service