2021-04-19 13:47:10 +08:00
|
|
|
// 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.
|
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
package querynode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"errors"
|
2021-06-15 12:41:40 +08:00
|
|
|
"fmt"
|
2021-06-09 11:37:55 +08:00
|
|
|
"math"
|
|
|
|
"reflect"
|
2021-03-23 05:59:46 +08:00
|
|
|
"sync"
|
|
|
|
|
|
|
|
"github.com/golang/protobuf/proto"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/msgstream"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/trace"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
2021-06-15 12:41:40 +08:00
|
|
|
oplog "github.com/opentracing/opentracing-go/log"
|
|
|
|
"go.uber.org/zap"
|
2021-03-23 05:59:46 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
type searchCollection struct {
|
|
|
|
releaseCtx context.Context
|
|
|
|
cancel context.CancelFunc
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
collectionID UniqueID
|
|
|
|
historical *historical
|
|
|
|
streaming *streaming
|
2021-03-23 05:59:46 +08:00
|
|
|
|
2021-03-25 04:40:59 +08:00
|
|
|
msgBuffer chan *msgstream.SearchMsg
|
2021-05-29 11:16:29 +08:00
|
|
|
unsolvedMsgMu sync.Mutex // guards unsolvedMsg
|
2021-03-25 04:40:59 +08:00
|
|
|
unsolvedMsg []*msgstream.SearchMsg
|
2021-03-23 05:59:46 +08:00
|
|
|
|
2021-06-15 20:06:10 +08:00
|
|
|
tSafeWatchers map[Channel]*tSafeWatcher
|
2021-06-09 11:37:55 +08:00
|
|
|
watcherSelectCase []reflect.SelectCase
|
2021-03-23 05:59:46 +08:00
|
|
|
|
|
|
|
serviceableTimeMutex sync.Mutex // guards serviceableTime
|
|
|
|
serviceableTime Timestamp
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
searchMsgStream msgstream.MsgStream
|
2021-03-23 05:59:46 +08:00
|
|
|
searchResultMsgStream msgstream.MsgStream
|
|
|
|
}
|
|
|
|
|
|
|
|
type ResultEntityIds []UniqueID
|
|
|
|
|
2021-05-28 10:26:30 +08:00
|
|
|
func newSearchCollection(releaseCtx context.Context,
|
|
|
|
cancel context.CancelFunc,
|
|
|
|
collectionID UniqueID,
|
2021-06-15 12:41:40 +08:00
|
|
|
historical *historical,
|
|
|
|
streaming *streaming,
|
|
|
|
factory msgstream.Factory) *searchCollection {
|
2021-06-09 11:37:55 +08:00
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
receiveBufSize := Params.SearchReceiveBufSize
|
|
|
|
msgBuffer := make(chan *msgstream.SearchMsg, receiveBufSize)
|
|
|
|
unsolvedMsg := make([]*msgstream.SearchMsg, 0)
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
searchStream, _ := factory.NewQueryMsgStream(releaseCtx)
|
|
|
|
searchResultStream, _ := factory.NewQueryMsgStream(releaseCtx)
|
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
sc := &searchCollection{
|
|
|
|
releaseCtx: releaseCtx,
|
|
|
|
cancel: cancel,
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
collectionID: collectionID,
|
|
|
|
historical: historical,
|
|
|
|
streaming: streaming,
|
2021-03-23 05:59:46 +08:00
|
|
|
|
2021-06-15 20:06:10 +08:00
|
|
|
tSafeWatchers: make(map[Channel]*tSafeWatcher),
|
2021-06-09 11:37:55 +08:00
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
msgBuffer: msgBuffer,
|
|
|
|
unsolvedMsg: unsolvedMsg,
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
searchMsgStream: searchStream,
|
2021-03-23 05:59:46 +08:00
|
|
|
searchResultMsgStream: searchResultStream,
|
|
|
|
}
|
|
|
|
|
2021-06-09 11:37:55 +08:00
|
|
|
sc.register()
|
2021-03-23 05:59:46 +08:00
|
|
|
return sc
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *searchCollection) start() {
|
2021-06-15 12:41:40 +08:00
|
|
|
go s.searchMsgStream.Start()
|
|
|
|
go s.searchResultMsgStream.Start()
|
|
|
|
go s.consumeSearch()
|
2021-03-23 05:59:46 +08:00
|
|
|
go s.doUnsolvedMsgSearch()
|
|
|
|
}
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
func (s *searchCollection) close() {
|
|
|
|
if s.searchMsgStream != nil {
|
|
|
|
s.searchMsgStream.Close()
|
|
|
|
}
|
|
|
|
if s.searchResultMsgStream != nil {
|
|
|
|
s.searchResultMsgStream.Close()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-09 11:37:55 +08:00
|
|
|
func (s *searchCollection) register() {
|
2021-06-15 12:41:40 +08:00
|
|
|
collection, err := s.streaming.replica.getCollectionByID(s.collectionID)
|
2021-06-09 11:37:55 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Error(err.Error())
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
s.watcherSelectCase = make([]reflect.SelectCase, 0)
|
|
|
|
log.Debug("register tSafe watcher and init watcher select case",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Any("collectionID", collection.ID()),
|
2021-06-15 20:06:10 +08:00
|
|
|
zap.Any("dml channels", collection.getVChannels()),
|
2021-06-15 12:41:40 +08:00
|
|
|
)
|
2021-06-15 20:06:10 +08:00
|
|
|
for _, channel := range collection.getVChannels() {
|
2021-06-09 11:37:55 +08:00
|
|
|
s.tSafeWatchers[channel] = newTSafeWatcher()
|
2021-06-15 12:41:40 +08:00
|
|
|
s.streaming.tSafeReplica.registerTSafeWatcher(channel, s.tSafeWatchers[channel])
|
2021-06-09 11:37:55 +08:00
|
|
|
s.watcherSelectCase = append(s.watcherSelectCase, reflect.SelectCase{
|
|
|
|
Dir: reflect.SelectRecv,
|
|
|
|
Chan: reflect.ValueOf(s.tSafeWatchers[channel].watcherChan()),
|
|
|
|
})
|
|
|
|
}
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
|
2021-03-25 04:40:59 +08:00
|
|
|
func (s *searchCollection) addToUnsolvedMsg(msg *msgstream.SearchMsg) {
|
2021-05-29 11:16:29 +08:00
|
|
|
s.unsolvedMsgMu.Lock()
|
|
|
|
defer s.unsolvedMsgMu.Unlock()
|
2021-03-25 04:40:59 +08:00
|
|
|
s.unsolvedMsg = append(s.unsolvedMsg, msg)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *searchCollection) popAllUnsolvedMsg() []*msgstream.SearchMsg {
|
2021-05-29 11:16:29 +08:00
|
|
|
s.unsolvedMsgMu.Lock()
|
|
|
|
defer s.unsolvedMsgMu.Unlock()
|
2021-03-25 04:40:59 +08:00
|
|
|
tmp := s.unsolvedMsg
|
|
|
|
s.unsolvedMsg = s.unsolvedMsg[:0]
|
|
|
|
return tmp
|
|
|
|
}
|
|
|
|
|
2021-05-28 15:40:32 +08:00
|
|
|
func (s *searchCollection) waitNewTSafe() Timestamp {
|
2021-06-09 11:37:55 +08:00
|
|
|
// block until any vChannel updating tSafe
|
|
|
|
_, _, recvOK := reflect.Select(s.watcherSelectCase)
|
|
|
|
if !recvOK {
|
|
|
|
log.Error("tSafe has been closed", zap.Any("collectionID", s.collectionID))
|
|
|
|
return invalidTimestamp
|
|
|
|
}
|
2021-06-15 12:41:40 +08:00
|
|
|
//log.Debug("wait new tSafe", zap.Any("collectionID", s.collectionID))
|
2021-06-09 11:37:55 +08:00
|
|
|
t := Timestamp(math.MaxInt64)
|
|
|
|
for channel := range s.tSafeWatchers {
|
2021-06-15 12:41:40 +08:00
|
|
|
ts := s.streaming.tSafeReplica.getTSafe(channel)
|
2021-06-09 11:37:55 +08:00
|
|
|
if ts <= t {
|
|
|
|
t = ts
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return t
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *searchCollection) getServiceableTime() Timestamp {
|
|
|
|
s.serviceableTimeMutex.Lock()
|
|
|
|
defer s.serviceableTimeMutex.Unlock()
|
|
|
|
return s.serviceableTime
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *searchCollection) setServiceableTime(t Timestamp) {
|
|
|
|
s.serviceableTimeMutex.Lock()
|
2021-06-09 11:37:55 +08:00
|
|
|
defer s.serviceableTimeMutex.Unlock()
|
|
|
|
|
|
|
|
if t < s.serviceableTime {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2021-04-24 15:53:37 +08:00
|
|
|
gracefulTimeInMilliSecond := Params.GracefulTime
|
|
|
|
if gracefulTimeInMilliSecond > 0 {
|
|
|
|
gracefulTime := tsoutil.ComposeTS(gracefulTimeInMilliSecond, 0)
|
|
|
|
s.serviceableTime = t + gracefulTime
|
|
|
|
} else {
|
|
|
|
s.serviceableTime = t
|
|
|
|
}
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *searchCollection) emptySearch(searchMsg *msgstream.SearchMsg) {
|
2021-03-25 14:41:46 +08:00
|
|
|
sp, ctx := trace.StartSpanFromContext(searchMsg.TraceCtx())
|
|
|
|
defer sp.Finish()
|
|
|
|
searchMsg.SetTraceCtx(ctx)
|
2021-03-23 05:59:46 +08:00
|
|
|
err := s.search(searchMsg)
|
|
|
|
if err != nil {
|
|
|
|
log.Error(err.Error())
|
2021-06-15 12:41:40 +08:00
|
|
|
s.publishFailedSearchResult(searchMsg, err.Error())
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
func (s *searchCollection) consumeSearch() {
|
2021-03-23 05:59:46 +08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-s.releaseCtx.Done():
|
2021-03-26 07:09:47 +08:00
|
|
|
log.Debug("stop searchCollection's receiveSearchMsg", zap.Int64("collectionID", s.collectionID))
|
2021-03-23 05:59:46 +08:00
|
|
|
return
|
2021-06-15 12:41:40 +08:00
|
|
|
default:
|
|
|
|
msgPack := s.searchMsgStream.Consume()
|
|
|
|
if msgPack == nil || len(msgPack.Msgs) <= 0 {
|
|
|
|
msgPackNil := msgPack == nil
|
|
|
|
msgPackEmpty := true
|
|
|
|
if msgPack != nil {
|
|
|
|
msgPackEmpty = len(msgPack.Msgs) <= 0
|
|
|
|
}
|
|
|
|
log.Debug("consume search message failed", zap.Any("msgPack is Nil", msgPackNil),
|
|
|
|
zap.Any("msgPackEmpty", msgPackEmpty))
|
2021-03-23 05:59:46 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-06-15 12:41:40 +08:00
|
|
|
for _, msg := range msgPack.Msgs {
|
|
|
|
switch sm := msg.(type) {
|
|
|
|
case *msgstream.SearchMsg:
|
|
|
|
s.receiveSearch(sm)
|
|
|
|
case *msgstream.LoadBalanceSegmentsMsg:
|
|
|
|
s.loadBalance(sm)
|
|
|
|
default:
|
|
|
|
log.Warn("unsupported msg type in search channel", zap.Any("msg", sm))
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
func (s *searchCollection) loadBalance(msg *msgstream.LoadBalanceSegmentsMsg) {
|
|
|
|
log.Debug("consume load balance message",
|
|
|
|
zap.Int64("msgID", msg.ID()))
|
|
|
|
nodeID := Params.QueryNodeID
|
|
|
|
for _, info := range msg.Infos {
|
|
|
|
segmentID := info.SegmentID
|
|
|
|
if nodeID == info.SourceNodeID {
|
|
|
|
err := s.historical.replica.removeSegment(segmentID)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("loadBalance failed when remove segment",
|
|
|
|
zap.Error(err),
|
|
|
|
zap.Any("segmentID", segmentID))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if nodeID == info.DstNodeID {
|
|
|
|
segment, err := s.historical.replica.getSegmentByID(segmentID)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("loadBalance failed when making segment on service",
|
|
|
|
zap.Error(err),
|
|
|
|
zap.Any("segmentID", segmentID))
|
|
|
|
continue // not return, try to load balance all segment
|
|
|
|
}
|
|
|
|
segment.setOnService(true)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
log.Debug("load balance done",
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
zap.Int("num of segment", len(msg.Infos)))
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *searchCollection) receiveSearch(msg *msgstream.SearchMsg) {
|
|
|
|
if msg.CollectionID != s.collectionID {
|
|
|
|
log.Debug("not target collection search request",
|
|
|
|
zap.Any("collectionID", msg.CollectionID),
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("consume search message",
|
|
|
|
zap.Any("collectionID", msg.CollectionID),
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
)
|
|
|
|
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
|
|
|
msg.SetTraceCtx(ctx)
|
|
|
|
|
|
|
|
// check if collection has been released
|
|
|
|
collection, err := s.historical.replica.getCollectionByID(msg.CollectionID)
|
|
|
|
if err != nil {
|
|
|
|
log.Error(err.Error())
|
|
|
|
s.publishFailedSearchResult(msg, err.Error())
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if msg.BeginTs() >= collection.getReleaseTime() {
|
|
|
|
err := errors.New("search failed, collection has been released, msgID = " +
|
|
|
|
fmt.Sprintln(msg.ID()) +
|
|
|
|
", collectionID = " +
|
|
|
|
fmt.Sprintln(msg.CollectionID))
|
|
|
|
log.Error(err.Error())
|
|
|
|
s.publishFailedSearchResult(msg, err.Error())
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
serviceTime := s.getServiceableTime()
|
|
|
|
if msg.BeginTs() > serviceTime {
|
|
|
|
bt, _ := tsoutil.ParseTS(msg.BeginTs())
|
|
|
|
st, _ := tsoutil.ParseTS(serviceTime)
|
|
|
|
log.Debug("query node::receiveSearchMsg: add to unsolvedMsg",
|
|
|
|
zap.Any("collectionID", s.collectionID),
|
|
|
|
zap.Any("sm.BeginTs", bt),
|
|
|
|
zap.Any("serviceTime", st),
|
|
|
|
zap.Any("delta seconds", (msg.BeginTs()-serviceTime)/(1000*1000*1000)),
|
|
|
|
zap.Any("msgID", msg.ID()),
|
|
|
|
)
|
|
|
|
s.addToUnsolvedMsg(msg)
|
|
|
|
sp.LogFields(
|
|
|
|
oplog.String("send to unsolved buffer", "send to unsolved buffer"),
|
|
|
|
oplog.Object("begin ts", bt),
|
|
|
|
oplog.Object("serviceTime", st),
|
|
|
|
oplog.Float64("delta seconds", float64(msg.BeginTs()-serviceTime)/(1000.0*1000.0*1000.0)),
|
|
|
|
)
|
|
|
|
sp.Finish()
|
|
|
|
return
|
|
|
|
}
|
|
|
|
log.Debug("doing search in receiveSearchMsg...",
|
|
|
|
zap.Int64("collectionID", msg.CollectionID),
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
)
|
|
|
|
err = s.search(msg)
|
|
|
|
if err != nil {
|
|
|
|
log.Error(err.Error())
|
|
|
|
log.Debug("do search failed in receiveSearchMsg, prepare to publish failed search result",
|
|
|
|
zap.Int64("collectionID", msg.CollectionID),
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
)
|
|
|
|
s.publishFailedSearchResult(msg, err.Error())
|
|
|
|
}
|
|
|
|
log.Debug("do search done in receiveSearch",
|
|
|
|
zap.Int64("collectionID", msg.CollectionID),
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
)
|
|
|
|
sp.Finish()
|
|
|
|
}
|
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
func (s *searchCollection) doUnsolvedMsgSearch() {
|
2021-06-15 12:41:40 +08:00
|
|
|
log.Debug("starting doUnsolvedMsgSearch...", zap.Any("collectionID", s.collectionID))
|
2021-03-23 05:59:46 +08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-s.releaseCtx.Done():
|
2021-03-26 07:09:47 +08:00
|
|
|
log.Debug("stop searchCollection's doUnsolvedMsgSearch", zap.Int64("collectionID", s.collectionID))
|
2021-03-23 05:59:46 +08:00
|
|
|
return
|
|
|
|
default:
|
2021-06-15 12:41:40 +08:00
|
|
|
//time.Sleep(10 * time.Millisecond)
|
2021-05-28 15:40:32 +08:00
|
|
|
serviceTime := s.waitNewTSafe()
|
2021-06-15 12:41:40 +08:00
|
|
|
st, _ := tsoutil.ParseTS(serviceTime)
|
|
|
|
log.Debug("get tSafe from flow graph",
|
|
|
|
zap.Int64("collectionID", s.collectionID),
|
|
|
|
zap.Any("tSafe", st))
|
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
s.setServiceableTime(serviceTime)
|
2021-06-09 11:37:55 +08:00
|
|
|
log.Debug("query node::doUnsolvedMsgSearch: setServiceableTime",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Any("serviceTime", st),
|
2021-03-30 15:00:14 +08:00
|
|
|
)
|
2021-03-23 05:59:46 +08:00
|
|
|
|
|
|
|
searchMsg := make([]*msgstream.SearchMsg, 0)
|
2021-03-25 04:40:59 +08:00
|
|
|
tempMsg := s.popAllUnsolvedMsg()
|
2021-03-23 05:59:46 +08:00
|
|
|
|
|
|
|
for _, sm := range tempMsg {
|
2021-06-15 12:41:40 +08:00
|
|
|
bt, _ := tsoutil.ParseTS(sm.EndTs())
|
|
|
|
st, _ = tsoutil.ParseTS(serviceTime)
|
2021-03-26 07:09:47 +08:00
|
|
|
log.Debug("get search message from unsolvedMsg",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Int64("collectionID", sm.CollectionID),
|
2021-03-26 07:09:47 +08:00
|
|
|
zap.Int64("msgID", sm.ID()),
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Any("reqTime_p", bt),
|
|
|
|
zap.Any("serviceTime_p", st),
|
|
|
|
zap.Any("reqTime_l", sm.EndTs()),
|
|
|
|
zap.Any("serviceTime_l", serviceTime),
|
|
|
|
)
|
2021-03-23 05:59:46 +08:00
|
|
|
if sm.EndTs() <= serviceTime {
|
|
|
|
searchMsg = append(searchMsg, sm)
|
|
|
|
continue
|
|
|
|
}
|
2021-06-15 12:41:40 +08:00
|
|
|
log.Debug("query node::doUnsolvedMsgSearch: add to unsolvedMsg",
|
|
|
|
zap.Any("collectionID", s.collectionID),
|
|
|
|
zap.Any("sm.BeginTs", bt),
|
|
|
|
zap.Any("serviceTime", st),
|
|
|
|
zap.Any("delta seconds", (sm.BeginTs()-serviceTime)/(1000*1000*1000)),
|
|
|
|
zap.Any("msgID", sm.ID()),
|
|
|
|
)
|
2021-03-25 04:40:59 +08:00
|
|
|
s.addToUnsolvedMsg(sm)
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
if len(searchMsg) <= 0 {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
for _, sm := range searchMsg {
|
2021-03-25 14:41:46 +08:00
|
|
|
sp, ctx := trace.StartSpanFromContext(sm.TraceCtx())
|
|
|
|
sm.SetTraceCtx(ctx)
|
2021-03-26 07:09:47 +08:00
|
|
|
log.Debug("doing search in doUnsolvedMsgSearch...",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Int64("collectionID", sm.CollectionID),
|
2021-03-26 07:09:47 +08:00
|
|
|
zap.Int64("msgID", sm.ID()),
|
2021-06-15 12:41:40 +08:00
|
|
|
)
|
2021-03-23 05:59:46 +08:00
|
|
|
err := s.search(sm)
|
|
|
|
if err != nil {
|
|
|
|
log.Error(err.Error())
|
2021-03-26 07:09:47 +08:00
|
|
|
log.Debug("do search failed in doUnsolvedMsgSearch, prepare to publish failed search result",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Int64("collectionID", sm.CollectionID),
|
2021-03-26 07:09:47 +08:00
|
|
|
zap.Int64("msgID", sm.ID()),
|
2021-06-15 12:41:40 +08:00
|
|
|
)
|
|
|
|
s.publishFailedSearchResult(sm, err.Error())
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
2021-03-25 14:41:46 +08:00
|
|
|
sp.Finish()
|
2021-03-26 07:09:47 +08:00
|
|
|
log.Debug("do search done in doUnsolvedMsgSearch",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Int64("collectionID", sm.CollectionID),
|
2021-03-26 07:09:47 +08:00
|
|
|
zap.Int64("msgID", sm.ID()),
|
2021-06-15 12:41:40 +08:00
|
|
|
)
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
log.Debug("doUnsolvedMsgSearch, do search done", zap.Int("num of searchMsg", len(searchMsg)))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// TODO:: cache map[dsl]plan
|
|
|
|
// TODO: reBatched search requests
|
|
|
|
func (s *searchCollection) search(searchMsg *msgstream.SearchMsg) error {
|
2021-03-25 14:41:46 +08:00
|
|
|
sp, ctx := trace.StartSpanFromContext(searchMsg.TraceCtx())
|
|
|
|
defer sp.Finish()
|
|
|
|
searchMsg.SetTraceCtx(ctx)
|
2021-03-23 05:59:46 +08:00
|
|
|
searchTimestamp := searchMsg.Base.Timestamp
|
2021-04-25 14:08:28 +08:00
|
|
|
|
2021-03-23 05:59:46 +08:00
|
|
|
collectionID := searchMsg.CollectionID
|
2021-06-11 09:50:34 +08:00
|
|
|
collection, err := s.streaming.replica.getCollectionByID(collectionID)
|
2021-03-23 05:59:46 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-05-07 15:20:47 +08:00
|
|
|
var plan *Plan
|
|
|
|
if searchMsg.GetDslType() == commonpb.DslType_BoolExprV1 {
|
|
|
|
expr := searchMsg.SerializedExprPlan
|
2021-06-15 12:41:40 +08:00
|
|
|
plan, err = createPlanByExpr(collection, expr)
|
2021-05-07 15:20:47 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
dsl := searchMsg.Dsl
|
2021-06-15 12:41:40 +08:00
|
|
|
plan, err = createPlan(collection, dsl)
|
2021-05-07 15:20:47 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
2021-04-25 14:08:28 +08:00
|
|
|
searchRequestBlob := searchMsg.PlaceholderGroup
|
2021-03-30 22:16:58 +08:00
|
|
|
searchReq, err := parseSearchRequest(plan, searchRequestBlob)
|
2021-03-23 05:59:46 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-04-19 19:30:36 +08:00
|
|
|
queryNum := searchReq.getNumOfQuery()
|
2021-03-30 22:16:58 +08:00
|
|
|
searchRequests := make([]*searchRequest, 0)
|
|
|
|
searchRequests = append(searchRequests, searchReq)
|
2021-03-23 05:59:46 +08:00
|
|
|
|
2021-05-07 15:20:47 +08:00
|
|
|
if searchMsg.GetDslType() == commonpb.DslType_BoolExprV1 {
|
|
|
|
sp.LogFields(oplog.String("statistical time", "stats start"),
|
|
|
|
oplog.Object("nq", queryNum),
|
|
|
|
oplog.Object("expr", searchMsg.SerializedExprPlan))
|
|
|
|
} else {
|
|
|
|
sp.LogFields(oplog.String("statistical time", "stats start"),
|
|
|
|
oplog.Object("nq", queryNum),
|
|
|
|
oplog.Object("dsl", searchMsg.Dsl))
|
|
|
|
}
|
2021-05-28 10:26:30 +08:00
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
searchResults := make([]*SearchResult, 0)
|
|
|
|
matchedSegments := make([]*Segment, 0)
|
2021-05-28 10:26:30 +08:00
|
|
|
sealedSegmentSearched := make([]UniqueID, 0)
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
// historical search
|
2021-06-18 16:02:05 +08:00
|
|
|
hisSearchResults, hisSegmentResults, err1 := s.historical.search(searchRequests, collectionID, searchMsg.PartitionIDs, plan, searchTimestamp)
|
|
|
|
if err1 == nil {
|
|
|
|
searchResults = append(searchResults, hisSearchResults...)
|
|
|
|
matchedSegments = append(matchedSegments, hisSegmentResults...)
|
|
|
|
for _, seg := range hisSegmentResults {
|
|
|
|
sealedSegmentSearched = append(sealedSegmentSearched, seg.segmentID)
|
|
|
|
}
|
2021-05-28 10:26:30 +08:00
|
|
|
}
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
// streaming search
|
2021-06-18 16:02:05 +08:00
|
|
|
var err2 error
|
2021-06-15 20:06:10 +08:00
|
|
|
for _, channel := range collection.getVChannels() {
|
2021-06-18 16:02:05 +08:00
|
|
|
var strSearchResults []*SearchResult
|
|
|
|
var strSegmentResults []*Segment
|
|
|
|
strSearchResults, strSegmentResults, err2 = s.streaming.search(searchRequests, collectionID, searchMsg.PartitionIDs, channel, plan, searchTimestamp)
|
|
|
|
if err2 != nil {
|
|
|
|
break
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
2021-06-15 12:41:40 +08:00
|
|
|
searchResults = append(searchResults, strSearchResults...)
|
|
|
|
matchedSegments = append(matchedSegments, strSegmentResults...)
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
|
2021-06-18 16:02:05 +08:00
|
|
|
if err1 != nil && err2 != nil {
|
|
|
|
log.Error(err1.Error() + err2.Error())
|
|
|
|
return errors.New(err1.Error() + err2.Error())
|
|
|
|
}
|
|
|
|
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "segment search end"))
|
2021-03-23 05:59:46 +08:00
|
|
|
if len(searchResults) <= 0 {
|
2021-03-30 22:16:58 +08:00
|
|
|
for _, group := range searchRequests {
|
2021-03-23 05:59:46 +08:00
|
|
|
nq := group.getNumOfQuery()
|
|
|
|
nilHits := make([][]byte, nq)
|
|
|
|
hit := &milvuspb.Hits{}
|
|
|
|
for i := 0; i < int(nq); i++ {
|
|
|
|
bs, err := proto.Marshal(hit)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
nilHits[i] = bs
|
|
|
|
}
|
2021-05-24 16:30:09 +08:00
|
|
|
resultChannelInt := 0
|
2021-03-23 05:59:46 +08:00
|
|
|
searchResultMsg := &msgstream.SearchResultMsg{
|
2021-03-25 14:41:46 +08:00
|
|
|
BaseMsg: msgstream.BaseMsg{Ctx: searchMsg.Ctx, HashValues: []uint32{uint32(resultChannelInt)}},
|
2021-03-23 05:59:46 +08:00
|
|
|
SearchResults: internalpb.SearchResults{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_SearchResult,
|
|
|
|
MsgID: searchMsg.Base.MsgID,
|
|
|
|
Timestamp: searchTimestamp,
|
|
|
|
SourceID: searchMsg.Base.SourceID,
|
|
|
|
},
|
2021-06-15 10:19:38 +08:00
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
|
|
|
ResultChannelID: searchMsg.ResultChannelID,
|
|
|
|
Hits: nilHits,
|
|
|
|
MetricType: plan.getMetricType(),
|
|
|
|
SealedSegmentIDsSearched: sealedSegmentSearched,
|
2021-06-15 20:06:10 +08:00
|
|
|
ChannelIDsSearched: collection.getPChannels(),
|
2021-06-15 10:19:38 +08:00
|
|
|
//TODO:: get global sealed segment from etcd
|
|
|
|
GlobalSealedSegmentIDs: sealedSegmentSearched,
|
2021-03-23 05:59:46 +08:00
|
|
|
},
|
|
|
|
}
|
2021-06-16 12:05:56 +08:00
|
|
|
log.Debug("QueryNode Empty SearchResultMsg",
|
2021-06-15 20:06:10 +08:00
|
|
|
zap.Any("collectionID", collection.ID()),
|
2021-06-16 12:05:56 +08:00
|
|
|
zap.Any("msgID", searchMsg.ID()),
|
|
|
|
zap.Any("pChannels", collection.getPChannels()),
|
2021-06-15 20:06:10 +08:00
|
|
|
zap.Any("sealedSegmentSearched", sealedSegmentSearched),
|
|
|
|
)
|
2021-03-26 07:09:47 +08:00
|
|
|
err = s.publishSearchResult(searchResultMsg, searchMsg.CollectionID)
|
2021-03-23 05:59:46 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
inReduced := make([]bool, len(searchResults))
|
|
|
|
numSegment := int64(len(searchResults))
|
2021-03-30 22:16:58 +08:00
|
|
|
var marshaledHits *MarshaledHits = nil
|
|
|
|
if numSegment == 1 {
|
|
|
|
inReduced[0] = true
|
|
|
|
err = fillTargetEntry(plan, searchResults, matchedSegments, inReduced)
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "fillTargetEntry end"))
|
2021-03-30 22:16:58 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
marshaledHits, err = reorganizeSingleQueryResult(plan, searchRequests, searchResults[0])
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "reorganizeSingleQueryResult end"))
|
2021-03-30 22:16:58 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
err = reduceSearchResults(searchResults, numSegment, inReduced)
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "reduceSearchResults end"))
|
2021-03-30 22:16:58 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
err = fillTargetEntry(plan, searchResults, matchedSegments, inReduced)
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "fillTargetEntry end"))
|
2021-03-30 22:16:58 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
marshaledHits, err = reorganizeQueryResults(plan, searchRequests, searchResults, numSegment, inReduced)
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "reorganizeQueryResults end"))
|
2021-03-30 22:16:58 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
hitsBlob, err := marshaledHits.getHitsBlob()
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "getHitsBlob end"))
|
2021-03-23 05:59:46 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var offset int64 = 0
|
2021-03-30 22:16:58 +08:00
|
|
|
for index := range searchRequests {
|
2021-03-23 05:59:46 +08:00
|
|
|
hitBlobSizePeerQuery, err := marshaledHits.hitBlobSizeInGroup(int64(index))
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-03-30 22:16:58 +08:00
|
|
|
hits := make([][]byte, len(hitBlobSizePeerQuery))
|
|
|
|
for i, len := range hitBlobSizePeerQuery {
|
|
|
|
hits[i] = hitsBlob[offset : offset+len]
|
2021-03-23 05:59:46 +08:00
|
|
|
//test code to checkout marshaled hits
|
|
|
|
//marshaledHit := hitsBlob[offset:offset+len]
|
|
|
|
//unMarshaledHit := milvuspb.Hits{}
|
|
|
|
//err = proto.Unmarshal(marshaledHit, &unMarshaledHit)
|
|
|
|
//if err != nil {
|
|
|
|
// return err
|
|
|
|
//}
|
|
|
|
//log.Debug("hits msg = ", unMarshaledHit)
|
|
|
|
offset += len
|
|
|
|
}
|
2021-05-24 16:30:09 +08:00
|
|
|
resultChannelInt := 0
|
2021-03-23 05:59:46 +08:00
|
|
|
searchResultMsg := &msgstream.SearchResultMsg{
|
2021-03-25 14:41:46 +08:00
|
|
|
BaseMsg: msgstream.BaseMsg{Ctx: searchMsg.Ctx, HashValues: []uint32{uint32(resultChannelInt)}},
|
2021-03-23 05:59:46 +08:00
|
|
|
SearchResults: internalpb.SearchResults{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_SearchResult,
|
|
|
|
MsgID: searchMsg.Base.MsgID,
|
|
|
|
Timestamp: searchTimestamp,
|
|
|
|
SourceID: searchMsg.Base.SourceID,
|
|
|
|
},
|
2021-05-28 10:26:30 +08:00
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
|
|
|
ResultChannelID: searchMsg.ResultChannelID,
|
|
|
|
Hits: hits,
|
|
|
|
MetricType: plan.getMetricType(),
|
|
|
|
SealedSegmentIDsSearched: sealedSegmentSearched,
|
2021-06-15 20:06:10 +08:00
|
|
|
ChannelIDsSearched: collection.getPChannels(),
|
2021-05-28 10:26:30 +08:00
|
|
|
//TODO:: get global sealed segment from etcd
|
|
|
|
GlobalSealedSegmentIDs: sealedSegmentSearched,
|
2021-03-23 05:59:46 +08:00
|
|
|
},
|
|
|
|
}
|
2021-06-15 20:06:10 +08:00
|
|
|
log.Debug("QueryNode SearchResultMsg",
|
|
|
|
zap.Any("collectionID", collection.ID()),
|
2021-06-16 12:05:56 +08:00
|
|
|
zap.Any("msgID", searchMsg.ID()),
|
|
|
|
zap.Any("pChannels", collection.getPChannels()),
|
2021-06-15 20:06:10 +08:00
|
|
|
zap.Any("sealedSegmentSearched", sealedSegmentSearched),
|
|
|
|
)
|
2021-03-23 05:59:46 +08:00
|
|
|
|
|
|
|
// For debugging, please don't delete.
|
|
|
|
//fmt.Println("==================== search result ======================")
|
|
|
|
//for i := 0; i < len(hits); i++ {
|
|
|
|
// testHits := milvuspb.Hits{}
|
|
|
|
// err := proto.Unmarshal(hits[i], &testHits)
|
|
|
|
// if err != nil {
|
|
|
|
// panic(err)
|
|
|
|
// }
|
|
|
|
// fmt.Println(testHits.IDs)
|
|
|
|
// fmt.Println(testHits.Scores)
|
|
|
|
//}
|
2021-03-26 07:09:47 +08:00
|
|
|
err = s.publishSearchResult(searchResultMsg, searchMsg.CollectionID)
|
2021-03-23 05:59:46 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "before free c++ memory"))
|
2021-03-23 05:59:46 +08:00
|
|
|
deleteSearchResults(searchResults)
|
|
|
|
deleteMarshaledHits(marshaledHits)
|
2021-04-19 19:30:36 +08:00
|
|
|
sp.LogFields(oplog.String("statistical time", "stats done"))
|
2021-03-23 05:59:46 +08:00
|
|
|
plan.delete()
|
2021-03-30 22:16:58 +08:00
|
|
|
searchReq.delete()
|
2021-03-23 05:59:46 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-03-26 07:09:47 +08:00
|
|
|
func (s *searchCollection) publishSearchResult(msg msgstream.TsMsg, collectionID UniqueID) error {
|
|
|
|
log.Debug("publishing search result...",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Int64("collectionID", collectionID),
|
2021-03-26 07:09:47 +08:00
|
|
|
zap.Int64("msgID", msg.ID()),
|
2021-06-15 12:41:40 +08:00
|
|
|
)
|
2021-03-25 14:41:46 +08:00
|
|
|
span, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
|
|
|
defer span.Finish()
|
|
|
|
msg.SetTraceCtx(ctx)
|
2021-03-23 05:59:46 +08:00
|
|
|
msgPack := msgstream.MsgPack{}
|
|
|
|
msgPack.Msgs = append(msgPack.Msgs, msg)
|
2021-03-25 14:41:46 +08:00
|
|
|
err := s.searchResultMsgStream.Produce(&msgPack)
|
2021-03-26 07:09:47 +08:00
|
|
|
if err != nil {
|
2021-06-15 12:41:40 +08:00
|
|
|
log.Error("publishing search result failed, err = "+err.Error(),
|
|
|
|
zap.Int64("collectionID", collectionID),
|
|
|
|
zap.Int64("msgID", msg.ID()),
|
|
|
|
)
|
2021-03-26 07:09:47 +08:00
|
|
|
} else {
|
|
|
|
log.Debug("publish search result done",
|
2021-06-15 12:41:40 +08:00
|
|
|
zap.Int64("collectionID", collectionID),
|
2021-03-26 07:09:47 +08:00
|
|
|
zap.Int64("msgID", msg.ID()),
|
2021-06-15 12:41:40 +08:00
|
|
|
)
|
2021-03-26 07:09:47 +08:00
|
|
|
}
|
2021-03-23 05:59:46 +08:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
func (s *searchCollection) publishFailedSearchResult(searchMsg *msgstream.SearchMsg, errMsg string) {
|
2021-03-25 14:41:46 +08:00
|
|
|
span, ctx := trace.StartSpanFromContext(searchMsg.TraceCtx())
|
|
|
|
defer span.Finish()
|
|
|
|
searchMsg.SetTraceCtx(ctx)
|
2021-03-23 05:59:46 +08:00
|
|
|
//log.Debug("Public fail SearchResult!")
|
|
|
|
msgPack := msgstream.MsgPack{}
|
|
|
|
|
2021-05-24 16:30:09 +08:00
|
|
|
resultChannelInt := 0
|
2021-03-23 05:59:46 +08:00
|
|
|
searchResultMsg := &msgstream.SearchResultMsg{
|
|
|
|
BaseMsg: msgstream.BaseMsg{HashValues: []uint32{uint32(resultChannelInt)}},
|
|
|
|
SearchResults: internalpb.SearchResults{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_SearchResult,
|
|
|
|
MsgID: searchMsg.Base.MsgID,
|
|
|
|
Timestamp: searchMsg.Base.Timestamp,
|
|
|
|
SourceID: searchMsg.Base.SourceID,
|
|
|
|
},
|
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg},
|
|
|
|
ResultChannelID: searchMsg.ResultChannelID,
|
|
|
|
Hits: [][]byte{},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
msgPack.Msgs = append(msgPack.Msgs, searchResultMsg)
|
2021-03-25 14:41:46 +08:00
|
|
|
err := s.searchResultMsgStream.Produce(&msgPack)
|
2021-03-23 05:59:46 +08:00
|
|
|
if err != nil {
|
2021-06-15 12:41:40 +08:00
|
|
|
log.Error("publish FailedSearchResult failed" + err.Error())
|
2021-03-23 05:59:46 +08:00
|
|
|
}
|
|
|
|
}
|