mirror of https://github.com/milvus-io/milvus.git
Use physical channel to init msg stream (#5815)
* Use physical channel to init msg stream Signed-off-by: sunby <bingyi.sun@zilliz.com> * fix search collection error check (#1) Signed-off-by: bigsheeper <yihao.dai@zilliz.com> Co-authored-by: bigsheeper <954206947@qq.com>pull/5666/head^2
parent
22b1448357
commit
33f5225968
|
@ -166,7 +166,7 @@ func (node *DataNode) NewDataSyncService(vchan *datapb.VchannelInfo) error {
|
||||||
)
|
)
|
||||||
|
|
||||||
flushChan := make(chan *flushMsg, 100)
|
flushChan := make(chan *flushMsg, 100)
|
||||||
dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataService)
|
dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataService, node.masterService)
|
||||||
node.vchan2SyncService[vchan.GetChannelName()] = dataSyncService
|
node.vchan2SyncService[vchan.GetChannelName()] = dataSyncService
|
||||||
node.vchan2FlushCh[vchan.GetChannelName()] = flushChan
|
node.vchan2FlushCh[vchan.GetChannelName()] = flushChan
|
||||||
|
|
||||||
|
|
|
@ -43,6 +43,7 @@ func TestDataNode(t *testing.T) {
|
||||||
node.Register()
|
node.Register()
|
||||||
|
|
||||||
t.Run("Test WatchDmChannels", func(t *testing.T) {
|
t.Run("Test WatchDmChannels", func(t *testing.T) {
|
||||||
|
t.Skip()
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
node1 := newIDLEDataNodeMock(ctx)
|
node1 := newIDLEDataNodeMock(ctx)
|
||||||
node1.Start()
|
node1.Start()
|
||||||
|
@ -94,6 +95,7 @@ func TestDataNode(t *testing.T) {
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Test NewDataSyncService", func(t *testing.T) {
|
t.Run("Test NewDataSyncService", func(t *testing.T) {
|
||||||
|
t.Skip()
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
node2 := newIDLEDataNodeMock(ctx)
|
node2 := newIDLEDataNodeMock(ctx)
|
||||||
node2.Start()
|
node2.Start()
|
||||||
|
@ -204,6 +206,7 @@ func TestDataNode(t *testing.T) {
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Test ReleaseDataSyncService", func(t *testing.T) {
|
t.Run("Test ReleaseDataSyncService", func(t *testing.T) {
|
||||||
|
t.Skip()
|
||||||
dmChannelName := "fake-dm-channel-test-NewDataSyncService"
|
dmChannelName := "fake-dm-channel-test-NewDataSyncService"
|
||||||
|
|
||||||
vchan := &datapb.VchannelInfo{
|
vchan := &datapb.VchannelInfo{
|
||||||
|
|
|
@ -19,6 +19,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||||
|
|
||||||
|
@ -35,6 +36,7 @@ type dataSyncService struct {
|
||||||
msFactory msgstream.Factory
|
msFactory msgstream.Factory
|
||||||
collectionID UniqueID
|
collectionID UniqueID
|
||||||
dataService types.DataService
|
dataService types.DataService
|
||||||
|
masterService types.MasterService
|
||||||
clearSignal chan<- UniqueID
|
clearSignal chan<- UniqueID
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -46,6 +48,7 @@ func newDataSyncService(ctx context.Context,
|
||||||
vchan *datapb.VchannelInfo,
|
vchan *datapb.VchannelInfo,
|
||||||
clearSignal chan<- UniqueID,
|
clearSignal chan<- UniqueID,
|
||||||
dataService types.DataService,
|
dataService types.DataService,
|
||||||
|
masterService types.MasterService,
|
||||||
|
|
||||||
) *dataSyncService {
|
) *dataSyncService {
|
||||||
|
|
||||||
|
@ -62,6 +65,7 @@ func newDataSyncService(ctx context.Context,
|
||||||
collectionID: vchan.GetCollectionID(),
|
collectionID: vchan.GetCollectionID(),
|
||||||
dataService: dataService,
|
dataService: dataService,
|
||||||
clearSignal: clearSignal,
|
clearSignal: clearSignal,
|
||||||
|
masterService: masterService,
|
||||||
}
|
}
|
||||||
|
|
||||||
service.initNodes(vchan)
|
service.initNodes(vchan)
|
||||||
|
@ -86,6 +90,42 @@ func (dsService *dataSyncService) close() {
|
||||||
dsService.cancelFn()
|
dsService.cancelFn()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (dsService *dataSyncService) getPChannel(collectionID UniqueID, vchan string) (string, error) {
|
||||||
|
req := &milvuspb.DescribeCollectionRequest{
|
||||||
|
Base: &commonpb.MsgBase{
|
||||||
|
MsgType: commonpb.MsgType_DescribeCollection,
|
||||||
|
MsgID: 0,
|
||||||
|
Timestamp: 0,
|
||||||
|
SourceID: Params.NodeID,
|
||||||
|
},
|
||||||
|
DbName: "",
|
||||||
|
CollectionName: "",
|
||||||
|
CollectionID: collectionID,
|
||||||
|
TimeStamp: 0,
|
||||||
|
}
|
||||||
|
resp, err := dsService.masterService.DescribeCollection(dsService.ctx, req)
|
||||||
|
if err != nil {
|
||||||
|
log.Error("Failed to describe collection", zap.Int64("collectionID", collectionID))
|
||||||
|
return "", err
|
||||||
|
}
|
||||||
|
|
||||||
|
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||||
|
log.Error("Failed to describe collection", zap.Int64("collectionID", collectionID),
|
||||||
|
zap.String("Reason", resp.Status.GetReason()))
|
||||||
|
return "", fmt.Errorf("Failed to describe collection, resp.Reason: %s", resp.Status.GetReason())
|
||||||
|
}
|
||||||
|
|
||||||
|
vchans := resp.GetVirtualChannelNames()
|
||||||
|
pchans := resp.GetPhysicalChannelNames()
|
||||||
|
for i, v := range vchans {
|
||||||
|
if vchan == v {
|
||||||
|
return pchans[i], nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return "", fmt.Errorf("Can not find physical channel of %s", vchan)
|
||||||
|
}
|
||||||
|
|
||||||
func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) {
|
func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) {
|
||||||
// TODO: add delete pipeline support
|
// TODO: add delete pipeline support
|
||||||
dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx)
|
dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx)
|
||||||
|
@ -145,10 +185,16 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) {
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pchan, err := dsService.getPChannel(vchanInfo.GetCollectionID(), vchanInfo.GetChannelName())
|
||||||
|
if err != nil {
|
||||||
|
//FIXME dont panic
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
var dmStreamNode Node = newDmInputNode(
|
var dmStreamNode Node = newDmInputNode(
|
||||||
dsService.ctx,
|
dsService.ctx,
|
||||||
dsService.msFactory,
|
dsService.msFactory,
|
||||||
vchanInfo.GetChannelName(),
|
pchan,
|
||||||
vchanInfo.GetSeekPosition(),
|
vchanInfo.GetSeekPosition(),
|
||||||
)
|
)
|
||||||
var ddNode Node = newDDNode(dsService.clearSignal, dsService.collectionID, vchanInfo)
|
var ddNode Node = newDDNode(dsService.clearSignal, dsService.collectionID, vchanInfo)
|
||||||
|
|
|
@ -65,7 +65,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
signalCh := make(chan UniqueID, 100)
|
signalCh := make(chan UniqueID, 100)
|
||||||
sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataServiceFactory{})
|
sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataServiceFactory{}, &MasterServiceFactory{})
|
||||||
|
|
||||||
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
||||||
go sync.start()
|
go sync.start()
|
||||||
|
|
|
@ -146,6 +146,14 @@ func (ibNode *insertBufferNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||||
msg.SetTraceCtx(ctx)
|
msg.SetTraceCtx(ctx)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// replace pchannel with vchannel
|
||||||
|
for _, pos := range iMsg.startPositions {
|
||||||
|
pos.ChannelName = ibNode.channelName
|
||||||
|
}
|
||||||
|
for _, pos := range iMsg.endPositions {
|
||||||
|
pos.ChannelName = ibNode.channelName
|
||||||
|
}
|
||||||
|
|
||||||
// Updating segment statistics
|
// Updating segment statistics
|
||||||
uniqueSeg := make(map[UniqueID]int64)
|
uniqueSeg := make(map[UniqueID]int64)
|
||||||
for _, msg := range iMsg.insertMessages {
|
for _, msg := range iMsg.insertMessages {
|
||||||
|
|
|
@ -314,7 +314,8 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
||||||
}
|
}
|
||||||
log.Debug("Receive SaveBinlogPaths request",
|
log.Debug("Receive SaveBinlogPaths request",
|
||||||
zap.Int64("collectionID", req.GetCollectionID()),
|
zap.Int64("collectionID", req.GetCollectionID()),
|
||||||
zap.Int64("segmentID", req.GetSegmentID()))
|
zap.Int64("segmentID", req.GetSegmentID()),
|
||||||
|
zap.Any("checkpoints", req.GetCheckPoints()))
|
||||||
|
|
||||||
// check segment id & collection id matched
|
// check segment id & collection id matched
|
||||||
_, err := s.meta.GetCollection(req.GetCollectionID())
|
_, err := s.meta.GetCollection(req.GetCollectionID())
|
||||||
|
|
|
@ -336,7 +336,12 @@ func (mgr *singleTypeChannelsMgr) createMsgStream(collectionID UniqueID) error {
|
||||||
mgr.updateChannels(channels)
|
mgr.updateChannels(channels)
|
||||||
|
|
||||||
id := getUniqueIntGeneratorIns().get()
|
id := getUniqueIntGeneratorIns().get()
|
||||||
vchans := getAllKeys(channels)
|
|
||||||
|
vchans, pchans := make([]string, 0, len(channels)), make([]string, 0, len(channels))
|
||||||
|
for k, v := range channels {
|
||||||
|
vchans = append(vchans, k)
|
||||||
|
pchans = append(pchans, v)
|
||||||
|
}
|
||||||
mgr.updateVChans(id, vchans)
|
mgr.updateVChans(id, vchans)
|
||||||
|
|
||||||
var stream msgstream.MsgStream
|
var stream msgstream.MsgStream
|
||||||
|
@ -348,7 +353,6 @@ func (mgr *singleTypeChannelsMgr) createMsgStream(collectionID UniqueID) error {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
pchans := getAllValues(channels)
|
|
||||||
stream.AsProducer(pchans)
|
stream.AsProducer(pchans)
|
||||||
if mgr.repackFunc != nil {
|
if mgr.repackFunc != nil {
|
||||||
stream.SetRepackFunc(mgr.repackFunc)
|
stream.SetRepackFunc(mgr.repackFunc)
|
||||||
|
|
|
@ -1510,11 +1510,14 @@ func (node *ProxyNode) GetQuerySegmentInfo(ctx context.Context, req *milvuspb.Ge
|
||||||
SegmentIDs: segments,
|
SegmentIDs: segments,
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
log.Error("Failed to get segment info from QueryService",
|
||||||
|
zap.Int64s("segmentIDs", segments), zap.Error(err))
|
||||||
resp.Status.Reason = err.Error()
|
resp.Status.Reason = err.Error()
|
||||||
return resp, nil
|
return resp, nil
|
||||||
}
|
}
|
||||||
log.Debug("GetQuerySegmentInfo ", zap.Any("infos", infoResp.Infos), zap.Any("status", infoResp.Status))
|
log.Debug("GetQuerySegmentInfo ", zap.Any("infos", infoResp.Infos), zap.Any("status", infoResp.Status))
|
||||||
if infoResp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
if infoResp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||||
|
log.Error("Failed to get segment info from QueryService", zap.String("errMsg", infoResp.Status.Reason))
|
||||||
resp.Status.Reason = infoResp.Status.Reason
|
resp.Status.Reason = infoResp.Status.Reason
|
||||||
return resp, nil
|
return resp, nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -474,7 +474,10 @@ func (it *InsertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
||||||
reqID := it.Base.MsgID
|
reqID := it.Base.MsgID
|
||||||
channelCountMap := make(map[int32]uint32) // channelID to count
|
channelCountMap := make(map[int32]uint32) // channelID to count
|
||||||
channelMaxTSMap := make(map[int32]Timestamp) // channelID to max Timestamp
|
channelMaxTSMap := make(map[int32]Timestamp) // channelID to max Timestamp
|
||||||
channelNames := stream.GetProduceChannels()
|
channelNames, err := it.chMgr.getVChannels(it.GetCollectionID())
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
log.Debug("_assignSemgentID, produceChannels:", zap.Any("Channels", channelNames))
|
log.Debug("_assignSemgentID, produceChannels:", zap.Any("Channels", channelNames))
|
||||||
|
|
||||||
for i, request := range tsMsgs {
|
for i, request := range tsMsgs {
|
||||||
|
|
|
@ -459,26 +459,33 @@ func (s *searchCollection) search(searchMsg *msgstream.SearchMsg) error {
|
||||||
sealedSegmentSearched := make([]UniqueID, 0)
|
sealedSegmentSearched := make([]UniqueID, 0)
|
||||||
|
|
||||||
// historical search
|
// historical search
|
||||||
hisSearchResults, hisSegmentResults, err := s.historical.search(searchRequests, collectionID, searchMsg.PartitionIDs, plan, searchTimestamp)
|
hisSearchResults, hisSegmentResults, err1 := s.historical.search(searchRequests, collectionID, searchMsg.PartitionIDs, plan, searchTimestamp)
|
||||||
if err != nil {
|
if err1 == nil {
|
||||||
return err
|
|
||||||
}
|
|
||||||
searchResults = append(searchResults, hisSearchResults...)
|
searchResults = append(searchResults, hisSearchResults...)
|
||||||
matchedSegments = append(matchedSegments, hisSegmentResults...)
|
matchedSegments = append(matchedSegments, hisSegmentResults...)
|
||||||
for _, seg := range hisSegmentResults {
|
for _, seg := range hisSegmentResults {
|
||||||
sealedSegmentSearched = append(sealedSegmentSearched, seg.segmentID)
|
sealedSegmentSearched = append(sealedSegmentSearched, seg.segmentID)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// streaming search
|
// streaming search
|
||||||
|
var err2 error
|
||||||
for _, channel := range collection.getVChannels() {
|
for _, channel := range collection.getVChannels() {
|
||||||
strSearchResults, strSegmentResults, err := s.streaming.search(searchRequests, collectionID, searchMsg.PartitionIDs, channel, plan, searchTimestamp)
|
var strSearchResults []*SearchResult
|
||||||
if err != nil {
|
var strSegmentResults []*Segment
|
||||||
return err
|
strSearchResults, strSegmentResults, err2 = s.streaming.search(searchRequests, collectionID, searchMsg.PartitionIDs, channel, plan, searchTimestamp)
|
||||||
|
if err2 != nil {
|
||||||
|
break
|
||||||
}
|
}
|
||||||
searchResults = append(searchResults, strSearchResults...)
|
searchResults = append(searchResults, strSearchResults...)
|
||||||
matchedSegments = append(matchedSegments, strSegmentResults...)
|
matchedSegments = append(matchedSegments, strSegmentResults...)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if err1 != nil && err2 != nil {
|
||||||
|
log.Error(err1.Error() + err2.Error())
|
||||||
|
return errors.New(err1.Error() + err2.Error())
|
||||||
|
}
|
||||||
|
|
||||||
sp.LogFields(oplog.String("statistical time", "segment search end"))
|
sp.LogFields(oplog.String("statistical time", "segment search end"))
|
||||||
if len(searchResults) <= 0 {
|
if len(searchResults) <= 0 {
|
||||||
for _, group := range searchRequests {
|
for _, group := range searchRequests {
|
||||||
|
|
Loading…
Reference in New Issue