mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-08 10:08:42 +08:00
Add retrieve_service (#5481)
* Add retrieve_service Signed-off-by: fishpenguin <kun.yu@zilliz.com> * Fix for tSafeReplica Signed-off-by: fishpenguin <kun.yu@zilliz.com>
This commit is contained in:
parent
03f90ff28c
commit
cd549f812d
@ -52,6 +52,13 @@ type ParamTable struct {
|
|||||||
SearchPulsarBufSize int64
|
SearchPulsarBufSize int64
|
||||||
SearchResultReceiveBufSize int64
|
SearchResultReceiveBufSize int64
|
||||||
|
|
||||||
|
// Retrieve
|
||||||
|
RetrieveChannelNames []string
|
||||||
|
RetrieveResultChannelNames []string
|
||||||
|
RetrieveReceiveBufSize int64
|
||||||
|
retrievePulsarBufSize int64
|
||||||
|
RetrieveResultReceiveBufSize int64
|
||||||
|
|
||||||
// stats
|
// stats
|
||||||
StatsPublishInterval int
|
StatsPublishInterval int
|
||||||
StatsChannelName string
|
StatsChannelName string
|
||||||
|
|||||||
273
internal/querynode/retrieve_collection.go
Normal file
273
internal/querynode/retrieve_collection.go
Normal file
@ -0,0 +1,273 @@
|
|||||||
|
// 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 querynode
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"sync"
|
||||||
|
|
||||||
|
"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/util/trace"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
|
oplog "github.com/opentracing/opentracing-go/log"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
)
|
||||||
|
|
||||||
|
type retrieveCollection struct {
|
||||||
|
releaseCtx context.Context
|
||||||
|
cancel context.CancelFunc
|
||||||
|
|
||||||
|
collectionID UniqueID
|
||||||
|
historicalReplica ReplicaInterface
|
||||||
|
streamingReplica ReplicaInterface
|
||||||
|
tSafeReplica TSafeReplicaInterface
|
||||||
|
|
||||||
|
msgBuffer chan *msgstream.RetrieveMsg
|
||||||
|
unsolvedMsgMu sync.Mutex
|
||||||
|
unsolvedMsg []*msgstream.RetrieveMsg
|
||||||
|
|
||||||
|
tSafeMutex sync.Mutex
|
||||||
|
tSafeWatcher *tSafeWatcher
|
||||||
|
|
||||||
|
serviceableTimeMutex sync.Mutex
|
||||||
|
serviceableTime Timestamp
|
||||||
|
|
||||||
|
retrieveResultMsgStream msgstream.MsgStream
|
||||||
|
}
|
||||||
|
|
||||||
|
func newRetrieveCollection(releaseCtx context.Context,
|
||||||
|
cancel context.CancelFunc,
|
||||||
|
collectionID UniqueID,
|
||||||
|
historicalReplica ReplicaInterface,
|
||||||
|
streamingReplica ReplicaInterface,
|
||||||
|
tSafeReplica TSafeReplicaInterface,
|
||||||
|
retrieveResultStream msgstream.MsgStream) *retrieveCollection {
|
||||||
|
receiveBufSize := Params.RetrieveReceiveBufSize
|
||||||
|
msgBuffer := make(chan *msgstream.RetrieveMsg, receiveBufSize)
|
||||||
|
unsolvedMsg := make([]*msgstream.RetrieveMsg, 0)
|
||||||
|
|
||||||
|
rc := &retrieveCollection{
|
||||||
|
releaseCtx: releaseCtx,
|
||||||
|
cancel: cancel,
|
||||||
|
|
||||||
|
collectionID: collectionID,
|
||||||
|
historicalReplica: historicalReplica,
|
||||||
|
streamingReplica: streamingReplica,
|
||||||
|
tSafeReplica: tSafeReplica,
|
||||||
|
|
||||||
|
msgBuffer: msgBuffer,
|
||||||
|
unsolvedMsg: unsolvedMsg,
|
||||||
|
}
|
||||||
|
|
||||||
|
rc.register(collectionID)
|
||||||
|
return rc
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) getServiceableTime() Timestamp {
|
||||||
|
rc.serviceableTimeMutex.Lock()
|
||||||
|
defer rc.serviceableTimeMutex.Unlock()
|
||||||
|
return rc.serviceableTime
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) setServiceableTime(t Timestamp) {
|
||||||
|
rc.serviceableTimeMutex.Lock()
|
||||||
|
gracefulTimeInMilliSecond := Params.GracefulTime
|
||||||
|
if gracefulTimeInMilliSecond > 0 {
|
||||||
|
gracefulTime := tsoutil.ComposeTS(gracefulTimeInMilliSecond, 0)
|
||||||
|
rc.serviceableTime = t + gracefulTime
|
||||||
|
} else {
|
||||||
|
rc.serviceableTime = t
|
||||||
|
}
|
||||||
|
rc.serviceableTimeMutex.Unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) waitNewTSafe() Timestamp {
|
||||||
|
// block until dataSyncService updating tSafe
|
||||||
|
// TODO: remove and use vChannel
|
||||||
|
vChannel := collectionIDToChannel(rc.collectionID)
|
||||||
|
// block until dataSyncService updating tSafe
|
||||||
|
rc.tSafeWatcher.hasUpdate()
|
||||||
|
ts := rc.tSafeReplica.getTSafe(vChannel)
|
||||||
|
return ts
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) register(collectionID UniqueID) {
|
||||||
|
vChannel := collectionIDToChannel(collectionID)
|
||||||
|
rc.tSafeReplica.addTSafe(vChannel)
|
||||||
|
rc.tSafeMutex.Lock()
|
||||||
|
rc.tSafeWatcher = newTSafeWatcher()
|
||||||
|
rc.tSafeMutex.Unlock()
|
||||||
|
rc.tSafeReplica.registerTSafeWatcher(vChannel, rc.tSafeWatcher)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) addToUnsolvedMsg(msg *msgstream.RetrieveMsg) {
|
||||||
|
rc.unsolvedMsgMu.Lock()
|
||||||
|
defer rc.unsolvedMsgMu.Unlock()
|
||||||
|
rc.unsolvedMsg = append(rc.unsolvedMsg, msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) receiveRetrieveMsg() {
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-rc.releaseCtx.Done():
|
||||||
|
log.Debug("stop retrieveCollection's receiveRetrieveMsg", zap.Int64("collectionID", rc.collectionID))
|
||||||
|
return
|
||||||
|
case rm := <-rc.msgBuffer:
|
||||||
|
sp, ctx := trace.StartSpanFromContext(rm.TraceCtx())
|
||||||
|
rm.SetTraceCtx(ctx)
|
||||||
|
log.Debug("get retrieve message from msgBuffer",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
serviceTime := rc.getServiceableTime()
|
||||||
|
if rm.BeginTs() > serviceTime {
|
||||||
|
bt, _ := tsoutil.ParseTS(rm.BeginTs())
|
||||||
|
st, _ := tsoutil.ParseTS(serviceTime)
|
||||||
|
log.Debug("querynode::receiveRetrieveMsg: add to unsolvedMsgs",
|
||||||
|
zap.Any("sm.BeginTs", bt),
|
||||||
|
zap.Any("serviceTime", st),
|
||||||
|
zap.Any("delta seconds", (rm.BeginTs()-serviceTime)/(1000*1000*1000)),
|
||||||
|
zap.Any("collectionID", rc.collectionID),
|
||||||
|
)
|
||||||
|
rc.addToUnsolvedMsg(rm)
|
||||||
|
sp.LogFields(
|
||||||
|
oplog.String("send to unsolved retrieve buffer", "send to unsolved buffer"),
|
||||||
|
oplog.Object("begin ts", bt),
|
||||||
|
oplog.Object("serviceTime", st),
|
||||||
|
oplog.Float64("delta seconds", float64(rm.BeginTs()-serviceTime)/(1000.0*1000.0*1000.0)),
|
||||||
|
)
|
||||||
|
sp.Finish()
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
log.Debug("doing retrieve in receiveRetrieveMsg...",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
err := rc.retrieve(rm)
|
||||||
|
if err != nil {
|
||||||
|
log.Error(err.Error())
|
||||||
|
log.Debug("do retrieve failed in receiveRetrieveMsg, prepare to publish failed retrieve result",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
err2 := rc.publishFailedRetrieveResult(rm, err.Error())
|
||||||
|
if err2 != nil {
|
||||||
|
log.Error("publish FailedRetrieveResult failed", zap.Error(err2))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
log.Debug("do retrieve done in retrieveRetrieveMsg",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) doUnsolvedMsgRetrieve() {
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-rc.releaseCtx.Done():
|
||||||
|
log.Debug("stop retrieveCollection's doUnsolvedMsgRetrieve", zap.Int64("collectionID", rc.collectionID))
|
||||||
|
return
|
||||||
|
default:
|
||||||
|
serviceTime := rc.waitNewTSafe()
|
||||||
|
rc.setServiceableTime(serviceTime)
|
||||||
|
log.Debug("querynode::doUnsolvedMsgRetrieve: setServiceableTime",
|
||||||
|
zap.Any("serviceTime", serviceTime),
|
||||||
|
)
|
||||||
|
log.Debug("get tSafe from flow graph",
|
||||||
|
zap.Int64("collectionID", rc.collectionID),
|
||||||
|
zap.Uint64("tSafe", serviceTime))
|
||||||
|
|
||||||
|
retrieveMsg := make([]*msgstream.RetrieveMsg, 0)
|
||||||
|
rc.unsolvedMsgMu.Lock()
|
||||||
|
tmpMsg := rc.unsolvedMsg
|
||||||
|
rc.unsolvedMsg = rc.unsolvedMsg[:0]
|
||||||
|
rc.unsolvedMsgMu.Unlock()
|
||||||
|
|
||||||
|
for _, rm := range tmpMsg {
|
||||||
|
log.Debug("get retrieve message from unsolvedMsg",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
|
||||||
|
if rm.EndTs() <= serviceTime {
|
||||||
|
retrieveMsg = append(retrieveMsg, rm)
|
||||||
|
}
|
||||||
|
rc.addToUnsolvedMsg(rm)
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(retrieveMsg) <= 0 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
for _, rm := range retrieveMsg {
|
||||||
|
sp, ctx := trace.StartSpanFromContext(rm.TraceCtx())
|
||||||
|
rm.SetTraceCtx(ctx)
|
||||||
|
log.Debug("doing retrieve in doUnsolvedMsgRetrieve...",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
err := rc.retrieve(rm)
|
||||||
|
if err != nil {
|
||||||
|
log.Error(err.Error())
|
||||||
|
log.Debug("do retrieve failed in doUnsolvedMsgRetrieve, prepare to publish failed retrieve result",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
err2 := rc.publishFailedRetrieveResult(rm, err.Error())
|
||||||
|
if err2 != nil {
|
||||||
|
log.Error("publish FailedRetrieveResult failed", zap.Error(err2))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
sp.Finish()
|
||||||
|
log.Debug("do retrieve done in doUnsolvedMsgRetrieve",
|
||||||
|
zap.Int64("msgID", rm.ID()),
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
}
|
||||||
|
log.Debug("doUnsolvedMsgRetrieve, do retrieve done", zap.Int("num of retrieveMsg", len(retrieveMsg)))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) retrieve(retrieveMsg *msgstream.RetrieveMsg) error {
|
||||||
|
// TODO(yukun)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rc *retrieveCollection) publishFailedRetrieveResult(retrieveMsg *msgstream.RetrieveMsg, errMsg string) error {
|
||||||
|
span, ctx := trace.StartSpanFromContext(retrieveMsg.TraceCtx())
|
||||||
|
defer span.Finish()
|
||||||
|
retrieveMsg.SetTraceCtx(ctx)
|
||||||
|
msgPack := msgstream.MsgPack{}
|
||||||
|
|
||||||
|
resultChannelInt := 0
|
||||||
|
retrieveResultMsg := &msgstream.RetrieveResultMsg{
|
||||||
|
BaseMsg: msgstream.BaseMsg{HashValues: []uint32{uint32(resultChannelInt)}},
|
||||||
|
RetrieveResults: internalpb.RetrieveResults{
|
||||||
|
Base: &commonpb.MsgBase{
|
||||||
|
MsgType: commonpb.MsgType_RetrieveResult,
|
||||||
|
MsgID: retrieveMsg.Base.MsgID,
|
||||||
|
Timestamp: retrieveMsg.Base.Timestamp,
|
||||||
|
SourceID: retrieveMsg.Base.SourceID,
|
||||||
|
},
|
||||||
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg},
|
||||||
|
ResultChannelID: retrieveMsg.ResultChannelID,
|
||||||
|
Ids: nil,
|
||||||
|
FieldsData: nil,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
msgPack.Msgs = append(msgPack.Msgs, retrieveResultMsg)
|
||||||
|
err := rc.retrieveResultMsgStream.Produce(&msgPack)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
165
internal/querynode/retrieve_service.go
Normal file
165
internal/querynode/retrieve_service.go
Normal file
@ -0,0 +1,165 @@
|
|||||||
|
// 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 querynode
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"errors"
|
||||||
|
"strconv"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
)
|
||||||
|
|
||||||
|
type retrieveService struct {
|
||||||
|
ctx context.Context
|
||||||
|
cancel context.CancelFunc
|
||||||
|
|
||||||
|
historicalReplica ReplicaInterface
|
||||||
|
streamingReplica ReplicaInterface
|
||||||
|
tSafeReplica TSafeReplicaInterface
|
||||||
|
|
||||||
|
retrieveMsgStream msgstream.MsgStream
|
||||||
|
retrieveResultMsgStream msgstream.MsgStream
|
||||||
|
|
||||||
|
queryNodeID UniqueID
|
||||||
|
retrieveCollections map[UniqueID]*retrieveCollection
|
||||||
|
}
|
||||||
|
|
||||||
|
func newRetrieveService(ctx context.Context,
|
||||||
|
historicalReplica ReplicaInterface,
|
||||||
|
streamingReplica ReplicaInterface,
|
||||||
|
tSafeReplica TSafeReplicaInterface,
|
||||||
|
factory msgstream.Factory) *retrieveService {
|
||||||
|
retrieveStream, _ := factory.NewQueryMsgStream(ctx)
|
||||||
|
retrieveResultStream, _ := factory.NewQueryMsgStream(ctx)
|
||||||
|
|
||||||
|
if len(Params.RetrieveChannelNames) > 0 && len(Params.RetrieveResultChannelNames) > 0 {
|
||||||
|
consumeChannels := Params.RetrieveChannelNames
|
||||||
|
consumeSubName := Params.MsgChannelSubName
|
||||||
|
retrieveStream.AsConsumer(consumeChannels, consumeSubName)
|
||||||
|
log.Debug("query node AdConsumer", zap.Any("retrieveChannels", consumeChannels), zap.Any("consumeSubName", consumeSubName))
|
||||||
|
producerChannels := Params.RetrieveChannelNames
|
||||||
|
retrieveResultStream.AsProducer(producerChannels)
|
||||||
|
log.Debug("query node AsProducer", zap.Any("retrieveResultChannels", producerChannels))
|
||||||
|
}
|
||||||
|
|
||||||
|
retrieveServiceCtx, retrieveServiceCancel := context.WithCancel(ctx)
|
||||||
|
return &retrieveService{
|
||||||
|
ctx: retrieveServiceCtx,
|
||||||
|
cancel: retrieveServiceCancel,
|
||||||
|
|
||||||
|
historicalReplica: historicalReplica,
|
||||||
|
streamingReplica: streamingReplica,
|
||||||
|
tSafeReplica: tSafeReplica,
|
||||||
|
|
||||||
|
retrieveMsgStream: retrieveStream,
|
||||||
|
retrieveResultMsgStream: retrieveResultStream,
|
||||||
|
|
||||||
|
queryNodeID: Params.QueryNodeID,
|
||||||
|
retrieveCollections: make(map[UniqueID]*retrieveCollection),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) start() {
|
||||||
|
rs.retrieveMsgStream.Start()
|
||||||
|
rs.retrieveResultMsgStream.Start()
|
||||||
|
rs.consumeRetrieve()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) collectionCheck(collectionID UniqueID) error {
|
||||||
|
if ok := rs.historicalReplica.hasCollection(collectionID); !ok {
|
||||||
|
err := errors.New("no collection found, collectionID = " + strconv.FormatInt(collectionID, 10))
|
||||||
|
log.Error(err.Error())
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) consumeRetrieve() {
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-rs.ctx.Done():
|
||||||
|
return
|
||||||
|
default:
|
||||||
|
msgPack := rs.retrieveMsgStream.Consume()
|
||||||
|
if msgPack == nil || len(msgPack.Msgs) <= 0 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
for _, msg := range msgPack.Msgs {
|
||||||
|
log.Debug("consume retrieve message", zap.Int64("msgID", msg.ID()))
|
||||||
|
rm, ok := msg.(*msgstream.RetrieveMsg)
|
||||||
|
if !ok {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
sp, ctx := trace.StartSpanFromContext(rm.TraceCtx())
|
||||||
|
rm.SetTraceCtx(ctx)
|
||||||
|
err := rs.collectionCheck(rm.CollectionID)
|
||||||
|
if err != nil {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
_, ok = rs.retrieveCollections[rm.CollectionID]
|
||||||
|
if !ok {
|
||||||
|
rs.startRetrieveCollection(rm.CollectionID)
|
||||||
|
log.Debug("new retrieve collection, start retrieve collection service",
|
||||||
|
zap.Int64("collectionID", rm.CollectionID))
|
||||||
|
}
|
||||||
|
rs.retrieveCollections[rm.CollectionID].msgBuffer <- rm
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) close() {
|
||||||
|
if rs.retrieveMsgStream != nil {
|
||||||
|
rs.retrieveMsgStream.Close()
|
||||||
|
}
|
||||||
|
if rs.retrieveResultMsgStream != nil {
|
||||||
|
rs.retrieveResultMsgStream.Close()
|
||||||
|
}
|
||||||
|
for collectionID := range rs.retrieveCollections {
|
||||||
|
rs.stopRetrieveCollection(collectionID)
|
||||||
|
}
|
||||||
|
rs.retrieveCollections = make(map[UniqueID]*retrieveCollection)
|
||||||
|
rs.cancel()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) startRetrieveCollection(collectionID UniqueID) {
|
||||||
|
ctx1, cancel := context.WithCancel(rs.ctx)
|
||||||
|
rc := newRetrieveCollection(ctx1,
|
||||||
|
cancel,
|
||||||
|
collectionID,
|
||||||
|
rs.historicalReplica,
|
||||||
|
rs.streamingReplica,
|
||||||
|
rs.tSafeReplica,
|
||||||
|
rs.retrieveResultMsgStream)
|
||||||
|
rs.retrieveCollections[collectionID] = rc
|
||||||
|
rs.start()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) hasRetrieveCollection(collectionID UniqueID) bool {
|
||||||
|
_, ok := rs.retrieveCollections[collectionID]
|
||||||
|
return ok
|
||||||
|
}
|
||||||
|
|
||||||
|
func (rs *retrieveService) stopRetrieveCollection(collectionID UniqueID) {
|
||||||
|
rc, ok := rs.retrieveCollections[collectionID]
|
||||||
|
if !ok {
|
||||||
|
log.Error("stopRetrieveCollection failed, collection doesn't exist", zap.Int64("collectionID", collectionID))
|
||||||
|
}
|
||||||
|
rc.cancel()
|
||||||
|
delete(rs.retrieveCollections, collectionID)
|
||||||
|
}
|
||||||
@ -39,7 +39,7 @@ type searchCollection struct {
|
|||||||
tSafeReplica TSafeReplicaInterface
|
tSafeReplica TSafeReplicaInterface
|
||||||
|
|
||||||
msgBuffer chan *msgstream.SearchMsg
|
msgBuffer chan *msgstream.SearchMsg
|
||||||
unsolvedMSgMu sync.Mutex // guards unsolvedMsg
|
unsolvedMsgMu sync.Mutex // guards unsolvedMsg
|
||||||
unsolvedMsg []*msgstream.SearchMsg
|
unsolvedMsg []*msgstream.SearchMsg
|
||||||
|
|
||||||
tSafeMutex sync.Mutex
|
tSafeMutex sync.Mutex
|
||||||
@ -99,14 +99,14 @@ func (s *searchCollection) register(collectionID UniqueID) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (s *searchCollection) addToUnsolvedMsg(msg *msgstream.SearchMsg) {
|
func (s *searchCollection) addToUnsolvedMsg(msg *msgstream.SearchMsg) {
|
||||||
s.unsolvedMSgMu.Lock()
|
s.unsolvedMsgMu.Lock()
|
||||||
defer s.unsolvedMSgMu.Unlock()
|
defer s.unsolvedMsgMu.Unlock()
|
||||||
s.unsolvedMsg = append(s.unsolvedMsg, msg)
|
s.unsolvedMsg = append(s.unsolvedMsg, msg)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *searchCollection) popAllUnsolvedMsg() []*msgstream.SearchMsg {
|
func (s *searchCollection) popAllUnsolvedMsg() []*msgstream.SearchMsg {
|
||||||
s.unsolvedMSgMu.Lock()
|
s.unsolvedMsgMu.Lock()
|
||||||
defer s.unsolvedMSgMu.Unlock()
|
defer s.unsolvedMsgMu.Unlock()
|
||||||
tmp := s.unsolvedMsg
|
tmp := s.unsolvedMsg
|
||||||
s.unsolvedMsg = s.unsolvedMsg[:0]
|
s.unsolvedMsg = s.unsolvedMsg[:0]
|
||||||
return tmp
|
return tmp
|
||||||
|
|||||||
@ -15,11 +15,12 @@ import "C"
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
|
"strconv"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
"github.com/milvus-io/milvus/internal/util/trace"
|
"github.com/milvus-io/milvus/internal/util/trace"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
"strconv"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type searchService struct {
|
type searchService struct {
|
||||||
@ -48,7 +49,7 @@ func newSearchService(ctx context.Context,
|
|||||||
searchResultStream, _ := factory.NewQueryMsgStream(ctx)
|
searchResultStream, _ := factory.NewQueryMsgStream(ctx)
|
||||||
|
|
||||||
if len(Params.SearchChannelNames) > 0 && len(Params.SearchResultChannelNames) > 0 {
|
if len(Params.SearchChannelNames) > 0 && len(Params.SearchResultChannelNames) > 0 {
|
||||||
// query node need to consumer search channels and produce search result channels when init.
|
// query node need to consume search channels and produce search result channels when init.
|
||||||
consumeChannels := Params.SearchChannelNames
|
consumeChannels := Params.SearchChannelNames
|
||||||
consumeSubName := Params.MsgChannelSubName
|
consumeSubName := Params.MsgChannelSubName
|
||||||
searchStream.AsConsumer(consumeChannels, consumeSubName)
|
searchStream.AsConsumer(consumeChannels, consumeSubName)
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user