mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 17:48:29 +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
|
||||
SearchResultReceiveBufSize int64
|
||||
|
||||
// Retrieve
|
||||
RetrieveChannelNames []string
|
||||
RetrieveResultChannelNames []string
|
||||
RetrieveReceiveBufSize int64
|
||||
retrievePulsarBufSize int64
|
||||
RetrieveResultReceiveBufSize int64
|
||||
|
||||
// stats
|
||||
StatsPublishInterval int
|
||||
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
|
||||
|
||||
msgBuffer chan *msgstream.SearchMsg
|
||||
unsolvedMSgMu sync.Mutex // guards unsolvedMsg
|
||||
unsolvedMsgMu sync.Mutex // guards unsolvedMsg
|
||||
unsolvedMsg []*msgstream.SearchMsg
|
||||
|
||||
tSafeMutex sync.Mutex
|
||||
@ -99,14 +99,14 @@ func (s *searchCollection) register(collectionID UniqueID) {
|
||||
}
|
||||
|
||||
func (s *searchCollection) addToUnsolvedMsg(msg *msgstream.SearchMsg) {
|
||||
s.unsolvedMSgMu.Lock()
|
||||
defer s.unsolvedMSgMu.Unlock()
|
||||
s.unsolvedMsgMu.Lock()
|
||||
defer s.unsolvedMsgMu.Unlock()
|
||||
s.unsolvedMsg = append(s.unsolvedMsg, msg)
|
||||
}
|
||||
|
||||
func (s *searchCollection) popAllUnsolvedMsg() []*msgstream.SearchMsg {
|
||||
s.unsolvedMSgMu.Lock()
|
||||
defer s.unsolvedMSgMu.Unlock()
|
||||
s.unsolvedMsgMu.Lock()
|
||||
defer s.unsolvedMsgMu.Unlock()
|
||||
tmp := s.unsolvedMsg
|
||||
s.unsolvedMsg = s.unsolvedMsg[:0]
|
||||
return tmp
|
||||
|
||||
@ -15,11 +15,12 @@ import "C"
|
||||
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"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
type searchService struct {
|
||||
@ -48,7 +49,7 @@ func newSearchService(ctx context.Context,
|
||||
searchResultStream, _ := factory.NewQueryMsgStream(ctx)
|
||||
|
||||
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
|
||||
consumeSubName := Params.MsgChannelSubName
|
||||
searchStream.AsConsumer(consumeChannels, consumeSubName)
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user