2021-01-16 10:12:14 +08:00
|
|
|
package querynode
|
2020-11-26 16:01:31 +08:00
|
|
|
|
|
|
|
import "C"
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"errors"
|
2021-01-18 19:32:08 +08:00
|
|
|
"strconv"
|
2021-03-05 18:16:50 +08:00
|
|
|
"strings"
|
2020-11-26 16:01:31 +08:00
|
|
|
"sync"
|
|
|
|
|
2021-03-05 09:21:35 +08:00
|
|
|
"go.uber.org/zap"
|
2020-12-08 14:41:04 +08:00
|
|
|
|
2021-03-05 09:21:35 +08:00
|
|
|
"github.com/golang/protobuf/proto"
|
|
|
|
"github.com/zilliztech/milvus-distributed/internal/log"
|
2020-11-26 16:01:31 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
|
|
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
2021-03-12 14:22:09 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
2021-01-22 09:36:18 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
2020-11-26 16:01:31 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
type searchService struct {
|
|
|
|
ctx context.Context
|
|
|
|
wait sync.WaitGroup
|
|
|
|
cancel context.CancelFunc
|
|
|
|
|
2021-03-05 16:52:45 +08:00
|
|
|
replica ReplicaInterface
|
2020-11-26 16:01:31 +08:00
|
|
|
tSafeWatcher *tSafeWatcher
|
|
|
|
|
2021-01-13 10:40:46 +08:00
|
|
|
serviceableTimeMutex sync.Mutex // guards serviceableTime
|
2020-11-26 16:01:31 +08:00
|
|
|
serviceableTime Timestamp
|
|
|
|
|
|
|
|
msgBuffer chan msgstream.TsMsg
|
|
|
|
unsolvedMsg []msgstream.TsMsg
|
2020-12-08 14:41:04 +08:00
|
|
|
searchMsgStream msgstream.MsgStream
|
|
|
|
searchResultMsgStream msgstream.MsgStream
|
|
|
|
queryNodeID UniqueID
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
type ResultEntityIds []UniqueID
|
|
|
|
|
2021-03-05 16:52:45 +08:00
|
|
|
func newSearchService(ctx context.Context, replica ReplicaInterface, factory msgstream.Factory) *searchService {
|
2020-12-10 16:31:09 +08:00
|
|
|
receiveBufSize := Params.SearchReceiveBufSize
|
2021-02-03 17:30:10 +08:00
|
|
|
|
2021-03-19 20:16:04 +08:00
|
|
|
searchStream, _ := factory.NewQueryMsgStream(ctx)
|
|
|
|
searchResultStream, _ := factory.NewQueryMsgStream(ctx)
|
2021-02-09 17:09:26 +08:00
|
|
|
|
|
|
|
// query node doesn't need to consumer any search or search result channel actively.
|
2021-02-18 16:26:02 +08:00
|
|
|
consumeChannels := Params.SearchChannelNames
|
|
|
|
consumeSubName := Params.MsgChannelSubName
|
|
|
|
searchStream.AsConsumer(consumeChannels, consumeSubName)
|
2021-03-05 18:16:50 +08:00
|
|
|
log.Debug("querynode AsConsumer: " + strings.Join(consumeChannels, ", ") + " : " + consumeSubName)
|
2021-02-18 16:26:02 +08:00
|
|
|
producerChannels := Params.SearchResultChannelNames
|
|
|
|
searchResultStream.AsProducer(producerChannels)
|
2021-03-05 18:16:50 +08:00
|
|
|
log.Debug("querynode AsProducer: " + strings.Join(producerChannels, ", "))
|
2020-11-26 16:01:31 +08:00
|
|
|
|
|
|
|
searchServiceCtx, searchServiceCancel := context.WithCancel(ctx)
|
|
|
|
msgBuffer := make(chan msgstream.TsMsg, receiveBufSize)
|
|
|
|
unsolvedMsg := make([]msgstream.TsMsg, 0)
|
|
|
|
return &searchService{
|
|
|
|
ctx: searchServiceCtx,
|
|
|
|
cancel: searchServiceCancel,
|
|
|
|
serviceableTime: Timestamp(0),
|
|
|
|
msgBuffer: msgBuffer,
|
|
|
|
unsolvedMsg: unsolvedMsg,
|
|
|
|
|
|
|
|
replica: replica,
|
|
|
|
tSafeWatcher: newTSafeWatcher(),
|
|
|
|
|
2021-02-09 17:09:26 +08:00
|
|
|
searchMsgStream: searchStream,
|
|
|
|
searchResultMsgStream: searchResultStream,
|
2020-12-10 16:31:09 +08:00
|
|
|
queryNodeID: Params.QueryNodeID,
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) start() {
|
2020-12-08 14:41:04 +08:00
|
|
|
ss.searchMsgStream.Start()
|
|
|
|
ss.searchResultMsgStream.Start()
|
2020-11-26 16:01:31 +08:00
|
|
|
ss.register()
|
|
|
|
ss.wait.Add(2)
|
|
|
|
go ss.receiveSearchMsg()
|
|
|
|
go ss.doUnsolvedMsgSearch()
|
|
|
|
ss.wait.Wait()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) close() {
|
2020-12-08 14:41:04 +08:00
|
|
|
if ss.searchMsgStream != nil {
|
|
|
|
ss.searchMsgStream.Close()
|
|
|
|
}
|
|
|
|
if ss.searchResultMsgStream != nil {
|
|
|
|
ss.searchResultMsgStream.Close()
|
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
ss.cancel()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) register() {
|
2020-12-08 14:41:04 +08:00
|
|
|
tSafe := ss.replica.getTSafe()
|
|
|
|
tSafe.registerTSafeWatcher(ss.tSafeWatcher)
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) waitNewTSafe() Timestamp {
|
|
|
|
// block until dataSyncService updating tSafe
|
|
|
|
ss.tSafeWatcher.hasUpdate()
|
2020-12-08 14:41:04 +08:00
|
|
|
timestamp := ss.replica.getTSafe().get()
|
2020-11-26 16:01:31 +08:00
|
|
|
return timestamp
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) getServiceableTime() Timestamp {
|
|
|
|
ss.serviceableTimeMutex.Lock()
|
|
|
|
defer ss.serviceableTimeMutex.Unlock()
|
|
|
|
return ss.serviceableTime
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) setServiceableTime(t Timestamp) {
|
|
|
|
ss.serviceableTimeMutex.Lock()
|
2021-03-13 14:42:53 +08:00
|
|
|
// hard code gracefultime to 1 second
|
|
|
|
// TODO: use config to set gracefultime
|
|
|
|
ss.serviceableTime = t + 1000*1000*1000
|
2020-11-26 16:01:31 +08:00
|
|
|
ss.serviceableTimeMutex.Unlock()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) receiveSearchMsg() {
|
|
|
|
defer ss.wait.Done()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ss.ctx.Done():
|
|
|
|
return
|
|
|
|
default:
|
2021-02-25 17:35:36 +08:00
|
|
|
msgPack, _ := ss.searchMsgStream.Consume()
|
2020-11-26 16:01:31 +08:00
|
|
|
if msgPack == nil || len(msgPack.Msgs) <= 0 {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
searchMsg := make([]msgstream.TsMsg, 0)
|
|
|
|
serverTime := ss.getServiceableTime()
|
2021-01-11 18:35:54 +08:00
|
|
|
for i, msg := range msgPack.Msgs {
|
|
|
|
if msg.BeginTs() > serverTime {
|
|
|
|
ss.msgBuffer <- msg
|
2020-11-26 16:01:31 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
searchMsg = append(searchMsg, msgPack.Msgs[i])
|
|
|
|
}
|
|
|
|
for _, msg := range searchMsg {
|
|
|
|
err := ss.search(msg)
|
|
|
|
if err != nil {
|
2021-03-05 09:21:35 +08:00
|
|
|
log.Error(err.Error())
|
2020-12-11 11:38:32 +08:00
|
|
|
err2 := ss.publishFailedSearchResult(msg, err.Error())
|
|
|
|
if err2 != nil {
|
2021-03-05 09:21:35 +08:00
|
|
|
log.Error("publish FailedSearchResult failed", zap.Error(err2))
|
2020-11-30 11:22:59 +08:00
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-05 09:21:35 +08:00
|
|
|
log.Debug("ReceiveSearchMsg, do search done", zap.Int("num of searchMsg", len(searchMsg)))
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) doUnsolvedMsgSearch() {
|
|
|
|
defer ss.wait.Done()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ss.ctx.Done():
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
serviceTime := ss.waitNewTSafe()
|
|
|
|
ss.setServiceableTime(serviceTime)
|
|
|
|
searchMsg := make([]msgstream.TsMsg, 0)
|
|
|
|
tempMsg := make([]msgstream.TsMsg, 0)
|
|
|
|
tempMsg = append(tempMsg, ss.unsolvedMsg...)
|
|
|
|
ss.unsolvedMsg = ss.unsolvedMsg[:0]
|
|
|
|
for _, msg := range tempMsg {
|
|
|
|
if msg.EndTs() <= serviceTime {
|
|
|
|
searchMsg = append(searchMsg, msg)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
ss.unsolvedMsg = append(ss.unsolvedMsg, msg)
|
|
|
|
}
|
|
|
|
|
2020-11-26 17:58:08 +08:00
|
|
|
for {
|
2020-11-28 19:06:48 +08:00
|
|
|
msgBufferLength := len(ss.msgBuffer)
|
|
|
|
if msgBufferLength <= 0 {
|
|
|
|
break
|
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
msg := <-ss.msgBuffer
|
|
|
|
if msg.EndTs() <= serviceTime {
|
|
|
|
searchMsg = append(searchMsg, msg)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
ss.unsolvedMsg = append(ss.unsolvedMsg, msg)
|
|
|
|
}
|
2020-11-26 17:58:08 +08:00
|
|
|
|
2020-11-26 16:01:31 +08:00
|
|
|
if len(searchMsg) <= 0 {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
for _, msg := range searchMsg {
|
|
|
|
err := ss.search(msg)
|
|
|
|
if err != nil {
|
2021-03-05 09:21:35 +08:00
|
|
|
log.Error(err.Error())
|
2020-12-11 11:38:32 +08:00
|
|
|
err2 := ss.publishFailedSearchResult(msg, err.Error())
|
|
|
|
if err2 != nil {
|
2021-03-05 09:21:35 +08:00
|
|
|
log.Error("publish FailedSearchResult failed", zap.Error(err2))
|
2020-11-30 11:22:59 +08:00
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-05 09:21:35 +08:00
|
|
|
log.Debug("doUnsolvedMsgSearch, do search done", zap.Int("num of searchMsg", len(searchMsg)))
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// TODO:: cache map[dsl]plan
|
|
|
|
// TODO: reBatched search requests
|
|
|
|
func (ss *searchService) search(msg msgstream.TsMsg) error {
|
|
|
|
searchMsg, ok := msg.(*msgstream.SearchMsg)
|
|
|
|
if !ok {
|
|
|
|
return errors.New("invalid request type = " + string(msg.Type()))
|
|
|
|
}
|
|
|
|
|
2021-01-18 19:32:08 +08:00
|
|
|
searchTimestamp := searchMsg.Base.Timestamp
|
2020-11-26 16:01:31 +08:00
|
|
|
var queryBlob = searchMsg.Query.Value
|
2021-01-22 09:36:18 +08:00
|
|
|
query := milvuspb.SearchRequest{}
|
2020-11-26 16:01:31 +08:00
|
|
|
err := proto.Unmarshal(queryBlob, &query)
|
|
|
|
if err != nil {
|
|
|
|
return errors.New("unmarshal query failed")
|
|
|
|
}
|
2021-02-03 11:52:19 +08:00
|
|
|
collectionID := searchMsg.CollectionID
|
|
|
|
collection, err := ss.replica.getCollectionByID(collectionID)
|
2020-11-26 16:01:31 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
dsl := query.Dsl
|
2020-11-30 17:58:23 +08:00
|
|
|
plan, err := createPlan(*collection, dsl)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
placeHolderGroupBlob := query.PlaceholderGroup
|
2020-11-30 17:58:23 +08:00
|
|
|
placeholderGroup, err := parserPlaceholderGroup(plan, placeHolderGroupBlob)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
placeholderGroups := make([]*PlaceholderGroup, 0)
|
|
|
|
placeholderGroups = append(placeholderGroups, placeholderGroup)
|
|
|
|
|
|
|
|
searchResults := make([]*SearchResult, 0)
|
2020-12-03 19:00:11 +08:00
|
|
|
matchedSegments := make([]*Segment, 0)
|
2020-11-26 16:01:31 +08:00
|
|
|
|
2021-03-05 09:21:35 +08:00
|
|
|
//log.Debug("search msg's partitionID = ", partitionIDsInQuery)
|
2021-02-05 10:53:11 +08:00
|
|
|
partitionIDsInCol, err := ss.replica.getPartitionIDs(collectionID)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2021-01-12 18:03:24 +08:00
|
|
|
}
|
2021-02-03 18:12:48 +08:00
|
|
|
var searchPartitionIDs []UniqueID
|
|
|
|
partitionIDsInQuery := searchMsg.PartitionIDs
|
|
|
|
if len(partitionIDsInQuery) == 0 {
|
2021-03-15 19:58:52 +08:00
|
|
|
if len(partitionIDsInCol) == 0 {
|
|
|
|
return errors.New("can't find any partition in this collection on query node")
|
|
|
|
}
|
2021-02-03 18:12:48 +08:00
|
|
|
searchPartitionIDs = partitionIDsInCol
|
2021-01-12 18:03:24 +08:00
|
|
|
} else {
|
2021-03-15 19:58:52 +08:00
|
|
|
findPartition := false
|
|
|
|
for _, id := range partitionIDsInQuery {
|
|
|
|
_, err := ss.replica.getPartitionByID(id)
|
|
|
|
if err == nil {
|
|
|
|
searchPartitionIDs = append(searchPartitionIDs, id)
|
|
|
|
findPartition = true
|
2021-01-12 18:03:24 +08:00
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
2021-03-15 19:58:52 +08:00
|
|
|
if !findPartition {
|
|
|
|
return errors.New("partition to be searched not exist in query node")
|
|
|
|
}
|
2021-01-09 09:47:22 +08:00
|
|
|
}
|
|
|
|
|
2021-02-03 18:12:48 +08:00
|
|
|
for _, partitionID := range searchPartitionIDs {
|
2021-02-05 10:53:11 +08:00
|
|
|
segmentIDs, err := ss.replica.getSegmentIDs(partitionID)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
for _, segmentID := range segmentIDs {
|
2021-03-05 09:21:35 +08:00
|
|
|
//log.Debug("dsl = ", dsl)
|
2021-02-05 10:53:11 +08:00
|
|
|
segment, err := ss.replica.getSegmentByID(segmentID)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
searchResult, err := segment.segmentSearch(plan, placeholderGroups, []Timestamp{searchTimestamp})
|
2020-11-28 19:06:48 +08:00
|
|
|
|
2020-11-26 16:01:31 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
searchResults = append(searchResults, searchResult)
|
2020-12-03 19:00:11 +08:00
|
|
|
matchedSegments = append(matchedSegments, segment)
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-28 19:06:48 +08:00
|
|
|
if len(searchResults) <= 0 {
|
2021-01-12 18:03:24 +08:00
|
|
|
for _, group := range placeholderGroups {
|
|
|
|
nq := group.getNumOfQuery()
|
|
|
|
nilHits := make([][]byte, nq)
|
2021-01-22 09:36:18 +08:00
|
|
|
hit := &milvuspb.Hits{}
|
2021-01-12 18:03:24 +08:00
|
|
|
for i := 0; i < int(nq); i++ {
|
|
|
|
bs, err := proto.Marshal(hit)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
nilHits[i] = bs
|
|
|
|
}
|
2021-01-18 19:32:08 +08:00
|
|
|
resultChannelInt, _ := strconv.ParseInt(searchMsg.ResultChannelID, 10, 64)
|
2021-01-12 18:03:24 +08:00
|
|
|
searchResultMsg := &msgstream.SearchResultMsg{
|
2021-01-18 19:32:08 +08:00
|
|
|
BaseMsg: msgstream.BaseMsg{HashValues: []uint32{uint32(resultChannelInt)}},
|
2021-03-12 14:22:09 +08:00
|
|
|
SearchResults: internalpb.SearchResults{
|
2021-01-18 19:32:08 +08:00
|
|
|
Base: &commonpb.MsgBase{
|
2021-03-10 14:45:35 +08:00
|
|
|
MsgType: commonpb.MsgType_SearchResult,
|
2021-01-18 19:32:08 +08:00
|
|
|
MsgID: searchMsg.Base.MsgID,
|
|
|
|
Timestamp: searchTimestamp,
|
|
|
|
SourceID: searchMsg.Base.SourceID,
|
|
|
|
},
|
2021-03-10 22:06:22 +08:00
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
2021-01-18 19:32:08 +08:00
|
|
|
ResultChannelID: searchMsg.ResultChannelID,
|
|
|
|
Hits: nilHits,
|
|
|
|
MetricType: plan.getMetricType(),
|
|
|
|
},
|
2021-01-12 18:03:24 +08:00
|
|
|
}
|
|
|
|
err = ss.publishSearchResult(searchResultMsg)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
2020-12-01 02:16:53 +08:00
|
|
|
}
|
2020-11-28 19:06:48 +08:00
|
|
|
}
|
|
|
|
|
2020-12-03 19:00:11 +08:00
|
|
|
inReduced := make([]bool, len(searchResults))
|
|
|
|
numSegment := int64(len(searchResults))
|
2020-12-11 17:20:14 +08:00
|
|
|
err2 := reduceSearchResults(searchResults, numSegment, inReduced)
|
|
|
|
if err2 != nil {
|
|
|
|
return err2
|
2020-12-03 19:00:11 +08:00
|
|
|
}
|
|
|
|
err = fillTargetEntry(plan, searchResults, matchedSegments, inReduced)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
marshaledHits, err := reorganizeQueryResults(plan, placeholderGroups, searchResults, numSegment, inReduced)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-11-26 16:01:31 +08:00
|
|
|
hitsBlob, err := marshaledHits.getHitsBlob()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var offset int64 = 0
|
|
|
|
for index := range placeholderGroups {
|
2020-12-03 19:00:11 +08:00
|
|
|
hitBlobSizePeerQuery, err := marshaledHits.hitBlobSizeInGroup(int64(index))
|
2020-11-26 16:01:31 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
hits := make([][]byte, 0)
|
2020-12-03 19:00:11 +08:00
|
|
|
for _, len := range hitBlobSizePeerQuery {
|
2020-11-26 16:01:31 +08:00
|
|
|
hits = append(hits, hitsBlob[offset:offset+len])
|
|
|
|
//test code to checkout marshaled hits
|
|
|
|
//marshaledHit := hitsBlob[offset:offset+len]
|
2021-01-22 09:36:18 +08:00
|
|
|
//unMarshaledHit := milvuspb.Hits{}
|
2020-11-26 16:01:31 +08:00
|
|
|
//err = proto.Unmarshal(marshaledHit, &unMarshaledHit)
|
|
|
|
//if err != nil {
|
|
|
|
// return err
|
|
|
|
//}
|
2021-03-05 09:21:35 +08:00
|
|
|
//log.Debug("hits msg = ", unMarshaledHit)
|
2020-11-26 16:01:31 +08:00
|
|
|
offset += len
|
|
|
|
}
|
2021-01-18 19:32:08 +08:00
|
|
|
resultChannelInt, _ := strconv.ParseInt(searchMsg.ResultChannelID, 10, 64)
|
2020-11-26 16:01:31 +08:00
|
|
|
searchResultMsg := &msgstream.SearchResultMsg{
|
2021-01-18 19:32:08 +08:00
|
|
|
BaseMsg: msgstream.BaseMsg{HashValues: []uint32{uint32(resultChannelInt)}},
|
2021-03-12 14:22:09 +08:00
|
|
|
SearchResults: internalpb.SearchResults{
|
2021-01-18 19:32:08 +08:00
|
|
|
Base: &commonpb.MsgBase{
|
2021-03-10 14:45:35 +08:00
|
|
|
MsgType: commonpb.MsgType_SearchResult,
|
2021-01-18 19:32:08 +08:00
|
|
|
MsgID: searchMsg.Base.MsgID,
|
|
|
|
Timestamp: searchTimestamp,
|
|
|
|
SourceID: searchMsg.Base.SourceID,
|
|
|
|
},
|
2021-03-10 22:06:22 +08:00
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
2021-01-18 19:32:08 +08:00
|
|
|
ResultChannelID: searchMsg.ResultChannelID,
|
|
|
|
Hits: hits,
|
|
|
|
MetricType: plan.getMetricType(),
|
|
|
|
},
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
2021-01-18 19:32:08 +08:00
|
|
|
|
|
|
|
// For debugging, please don't delete.
|
|
|
|
//for i := 0; i < len(hits); i++ {
|
|
|
|
// testHits := milvuspb.Hits{}
|
|
|
|
// err := proto.Unmarshal(hits[i], &testHits)
|
|
|
|
// if err != nil {
|
|
|
|
// panic(err)
|
|
|
|
// }
|
2021-03-05 09:21:35 +08:00
|
|
|
// log.Debug(testHits.IDs)
|
|
|
|
// log.Debug(testHits.Scores)
|
2021-01-18 19:32:08 +08:00
|
|
|
//}
|
2020-11-26 16:01:31 +08:00
|
|
|
err = ss.publishSearchResult(searchResultMsg)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
deleteSearchResults(searchResults)
|
|
|
|
deleteMarshaledHits(marshaledHits)
|
|
|
|
plan.delete()
|
|
|
|
placeholderGroup.delete()
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ss *searchService) publishSearchResult(msg msgstream.TsMsg) error {
|
2021-01-12 18:03:24 +08:00
|
|
|
// span, ctx := opentracing.StartSpanFromContext(msg.GetMsgContext(), "publish search result")
|
|
|
|
// defer span.Finish()
|
|
|
|
// msg.SetMsgContext(ctx)
|
2020-11-26 16:01:31 +08:00
|
|
|
msgPack := msgstream.MsgPack{}
|
|
|
|
msgPack.Msgs = append(msgPack.Msgs, msg)
|
2021-02-24 09:48:17 +08:00
|
|
|
err := ss.searchResultMsgStream.Produce(context.TODO(), &msgPack)
|
2020-12-11 12:01:20 +08:00
|
|
|
return err
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
|
|
|
|
2020-12-03 19:00:11 +08:00
|
|
|
func (ss *searchService) publishFailedSearchResult(msg msgstream.TsMsg, errMsg string) error {
|
2021-01-12 18:03:24 +08:00
|
|
|
// span, ctx := opentracing.StartSpanFromContext(msg.GetMsgContext(), "receive search msg")
|
|
|
|
// defer span.Finish()
|
|
|
|
// msg.SetMsgContext(ctx)
|
2021-03-05 09:21:35 +08:00
|
|
|
//log.Debug("Public fail SearchResult!")
|
2020-11-26 16:01:31 +08:00
|
|
|
msgPack := msgstream.MsgPack{}
|
|
|
|
searchMsg, ok := msg.(*msgstream.SearchMsg)
|
|
|
|
if !ok {
|
|
|
|
return errors.New("invalid request type = " + string(msg.Type()))
|
|
|
|
}
|
|
|
|
|
2021-01-18 19:32:08 +08:00
|
|
|
resultChannelInt, _ := strconv.ParseInt(searchMsg.ResultChannelID, 10, 64)
|
|
|
|
searchResultMsg := &msgstream.SearchResultMsg{
|
|
|
|
BaseMsg: msgstream.BaseMsg{HashValues: []uint32{uint32(resultChannelInt)}},
|
2021-03-12 14:22:09 +08:00
|
|
|
SearchResults: internalpb.SearchResults{
|
2021-01-18 19:32:08 +08:00
|
|
|
Base: &commonpb.MsgBase{
|
2021-03-10 14:45:35 +08:00
|
|
|
MsgType: commonpb.MsgType_SearchResult,
|
2021-01-18 19:32:08 +08:00
|
|
|
MsgID: searchMsg.Base.MsgID,
|
|
|
|
Timestamp: searchMsg.Base.Timestamp,
|
|
|
|
SourceID: searchMsg.Base.SourceID,
|
|
|
|
},
|
2021-03-10 22:06:22 +08:00
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg},
|
2021-01-18 19:32:08 +08:00
|
|
|
ResultChannelID: searchMsg.ResultChannelID,
|
|
|
|
Hits: [][]byte{},
|
|
|
|
},
|
2020-11-26 16:01:31 +08:00
|
|
|
}
|
2021-01-18 19:32:08 +08:00
|
|
|
|
|
|
|
msgPack.Msgs = append(msgPack.Msgs, searchResultMsg)
|
2021-02-24 09:48:17 +08:00
|
|
|
err := ss.searchResultMsgStream.Produce(context.TODO(), &msgPack)
|
2020-11-26 16:01:31 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|