2021-01-16 10:12:14 +08:00
|
|
|
package querynode
|
2020-08-25 15:45:19 +08:00
|
|
|
|
2020-09-02 10:38:08 +08:00
|
|
|
/*
|
|
|
|
|
2020-10-23 18:01:24 +08:00
|
|
|
#cgo CFLAGS: -I${SRCDIR}/../core/output/include
|
2020-09-02 10:38:08 +08:00
|
|
|
|
2020-10-31 15:11:47 +08:00
|
|
|
#cgo LDFLAGS: -L${SRCDIR}/../core/output/lib -lmilvus_segcore -Wl,-rpath=${SRCDIR}/../core/output/lib
|
2020-09-02 10:38:08 +08:00
|
|
|
|
2020-11-25 10:31:51 +08:00
|
|
|
#include "segcore/collection_c.h"
|
|
|
|
#include "segcore/segment_c.h"
|
2020-09-02 10:38:08 +08:00
|
|
|
|
|
|
|
*/
|
2020-08-25 15:45:19 +08:00
|
|
|
import "C"
|
2020-09-02 10:38:08 +08:00
|
|
|
|
2020-08-25 15:45:19 +08:00
|
|
|
import (
|
2020-10-15 21:31:50 +08:00
|
|
|
"context"
|
2021-01-19 11:37:16 +08:00
|
|
|
"errors"
|
2021-01-11 18:35:54 +08:00
|
|
|
"fmt"
|
|
|
|
"io"
|
2021-01-26 13:41:41 +08:00
|
|
|
"log"
|
2021-01-21 15:20:23 +08:00
|
|
|
"sync/atomic"
|
2021-01-11 18:35:54 +08:00
|
|
|
|
|
|
|
"github.com/opentracing/opentracing-go"
|
|
|
|
"github.com/uber/jaeger-client-go/config"
|
2021-01-21 15:20:23 +08:00
|
|
|
|
2021-01-20 10:02:59 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
|
|
|
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
2021-01-15 15:28:54 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
2021-01-27 14:41:56 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
2021-01-21 15:20:23 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
2021-01-15 15:28:54 +08:00
|
|
|
queryPb "github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
2021-01-22 11:17:18 +08:00
|
|
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
2020-08-25 15:45:19 +08:00
|
|
|
)
|
|
|
|
|
2021-01-27 09:50:52 +08:00
|
|
|
type Node interface {
|
|
|
|
typeutil.Component
|
|
|
|
|
|
|
|
AddQueryChannel(in *queryPb.AddQueryChannelsRequest) (*commonpb.Status, error)
|
|
|
|
RemoveQueryChannel(in *queryPb.RemoveQueryChannelsRequest) (*commonpb.Status, error)
|
|
|
|
WatchDmChannels(in *queryPb.WatchDmChannelsRequest) (*commonpb.Status, error)
|
|
|
|
LoadSegments(in *queryPb.LoadSegmentRequest) (*commonpb.Status, error)
|
|
|
|
ReleaseSegments(in *queryPb.ReleaseSegmentRequest) (*commonpb.Status, error)
|
|
|
|
}
|
|
|
|
|
2021-01-22 14:28:06 +08:00
|
|
|
type QueryService = typeutil.QueryServiceInterface
|
2021-01-15 15:28:54 +08:00
|
|
|
|
2020-08-25 15:45:19 +08:00
|
|
|
type QueryNode struct {
|
2021-01-22 11:17:18 +08:00
|
|
|
typeutil.Service
|
|
|
|
|
2020-12-08 14:41:04 +08:00
|
|
|
queryNodeLoopCtx context.Context
|
2020-12-10 16:31:09 +08:00
|
|
|
queryNodeLoopCancel context.CancelFunc
|
2020-10-15 21:31:50 +08:00
|
|
|
|
2020-11-05 10:52:50 +08:00
|
|
|
QueryNodeID uint64
|
2021-01-21 15:20:23 +08:00
|
|
|
stateCode atomic.Value
|
2020-08-25 15:45:19 +08:00
|
|
|
|
2020-12-08 14:41:04 +08:00
|
|
|
replica collectionReplica
|
2020-08-25 15:45:19 +08:00
|
|
|
|
2021-01-15 15:28:54 +08:00
|
|
|
// internal services
|
2020-12-24 20:55:40 +08:00
|
|
|
dataSyncService *dataSyncService
|
|
|
|
metaService *metaService
|
|
|
|
searchService *searchService
|
|
|
|
loadIndexService *loadIndexService
|
|
|
|
statsService *statsService
|
2021-01-11 18:35:54 +08:00
|
|
|
|
2021-01-18 10:09:17 +08:00
|
|
|
segManager *segmentManager
|
|
|
|
|
2021-01-11 18:35:54 +08:00
|
|
|
//opentracing
|
|
|
|
tracer opentracing.Tracer
|
|
|
|
closer io.Closer
|
2021-01-26 13:41:41 +08:00
|
|
|
|
|
|
|
// clients
|
2021-01-27 14:41:56 +08:00
|
|
|
masterClient MasterServiceInterface
|
|
|
|
queryClient QueryServiceInterface
|
|
|
|
indexClient IndexServiceInterface
|
|
|
|
dataClient DataServiceInterface
|
2020-11-05 10:52:50 +08:00
|
|
|
}
|
2020-09-07 17:01:46 +08:00
|
|
|
|
2021-01-22 11:17:18 +08:00
|
|
|
func NewQueryNode(ctx context.Context, queryNodeID uint64) *QueryNode {
|
2020-12-08 14:41:04 +08:00
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
2021-01-21 15:20:23 +08:00
|
|
|
node := &QueryNode{
|
2021-01-11 18:35:54 +08:00
|
|
|
queryNodeLoopCtx: ctx1,
|
|
|
|
queryNodeLoopCancel: cancel,
|
|
|
|
QueryNodeID: queryNodeID,
|
|
|
|
|
|
|
|
dataSyncService: nil,
|
|
|
|
metaService: nil,
|
|
|
|
searchService: nil,
|
|
|
|
statsService: nil,
|
2021-01-19 11:37:16 +08:00
|
|
|
segManager: nil,
|
2021-01-11 18:35:54 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
var err error
|
|
|
|
cfg := &config.Configuration{
|
|
|
|
ServiceName: "query_node",
|
|
|
|
Sampler: &config.SamplerConfig{
|
|
|
|
Type: "const",
|
|
|
|
Param: 1,
|
|
|
|
},
|
|
|
|
}
|
2021-01-21 15:20:23 +08:00
|
|
|
node.tracer, node.closer, err = cfg.NewTracer()
|
2021-01-11 18:35:54 +08:00
|
|
|
if err != nil {
|
|
|
|
panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
|
|
|
|
}
|
2021-01-21 15:20:23 +08:00
|
|
|
opentracing.SetGlobalTracer(node.tracer)
|
2020-12-08 14:41:04 +08:00
|
|
|
|
2020-11-05 10:52:50 +08:00
|
|
|
segmentsMap := make(map[int64]*Segment)
|
2020-11-09 16:27:11 +08:00
|
|
|
collections := make([]*Collection, 0)
|
2020-09-23 17:38:15 +08:00
|
|
|
|
2020-11-18 16:38:28 +08:00
|
|
|
tSafe := newTSafe()
|
|
|
|
|
2021-01-21 15:20:23 +08:00
|
|
|
node.replica = &collectionReplicaImpl{
|
2020-11-13 17:20:13 +08:00
|
|
|
collections: collections,
|
|
|
|
segments: segmentsMap,
|
2020-11-18 16:38:28 +08:00
|
|
|
|
|
|
|
tSafe: tSafe,
|
2020-11-13 17:20:13 +08:00
|
|
|
}
|
2021-01-21 15:20:23 +08:00
|
|
|
node.stateCode.Store(internalpb2.StateCode_INITIALIZING)
|
|
|
|
return node
|
|
|
|
}
|
2020-11-13 17:20:13 +08:00
|
|
|
|
2021-01-27 09:50:52 +08:00
|
|
|
func NewQueryNodeWithoutID(ctx context.Context) *QueryNode {
|
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
|
|
|
node := &QueryNode{
|
|
|
|
queryNodeLoopCtx: ctx1,
|
|
|
|
queryNodeLoopCancel: cancel,
|
|
|
|
|
|
|
|
dataSyncService: nil,
|
|
|
|
metaService: nil,
|
|
|
|
searchService: nil,
|
|
|
|
statsService: nil,
|
|
|
|
segManager: nil,
|
|
|
|
}
|
|
|
|
|
|
|
|
var err error
|
|
|
|
cfg := &config.Configuration{
|
|
|
|
ServiceName: "query_node",
|
|
|
|
Sampler: &config.SamplerConfig{
|
|
|
|
Type: "const",
|
|
|
|
Param: 1,
|
|
|
|
},
|
|
|
|
}
|
|
|
|
node.tracer, node.closer, err = cfg.NewTracer()
|
|
|
|
if err != nil {
|
|
|
|
panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
|
|
|
|
}
|
|
|
|
opentracing.SetGlobalTracer(node.tracer)
|
|
|
|
|
|
|
|
segmentsMap := make(map[int64]*Segment)
|
|
|
|
collections := make([]*Collection, 0)
|
|
|
|
|
|
|
|
tSafe := newTSafe()
|
|
|
|
|
|
|
|
node.replica = &collectionReplicaImpl{
|
|
|
|
collections: collections,
|
|
|
|
segments: segmentsMap,
|
|
|
|
|
|
|
|
tSafe: tSafe,
|
|
|
|
}
|
|
|
|
node.stateCode.Store(internalpb2.StateCode_INITIALIZING)
|
|
|
|
return node
|
|
|
|
}
|
|
|
|
|
2021-01-21 15:20:23 +08:00
|
|
|
// TODO: delete this and call node.Init()
|
|
|
|
func Init() {
|
|
|
|
Params.Init()
|
2020-09-15 15:53:10 +08:00
|
|
|
}
|
|
|
|
|
2021-01-21 10:01:29 +08:00
|
|
|
func (node *QueryNode) Init() error {
|
2021-01-27 09:50:52 +08:00
|
|
|
Params.Init()
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) Start() error {
|
2021-01-22 14:28:06 +08:00
|
|
|
registerReq := &queryPb.RegisterNodeRequest{
|
2021-01-21 15:20:23 +08:00
|
|
|
Address: &commonpb.Address{
|
|
|
|
Ip: Params.QueryNodeIP,
|
|
|
|
Port: Params.QueryNodePort,
|
|
|
|
},
|
|
|
|
}
|
2021-01-27 09:50:52 +08:00
|
|
|
|
|
|
|
response, err := node.queryClient.RegisterNode(registerReq)
|
2021-01-21 15:20:23 +08:00
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
if response.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
|
|
|
panic(response.Status.Reason)
|
|
|
|
}
|
|
|
|
|
2021-01-27 09:50:52 +08:00
|
|
|
Params.QueryNodeID = response.InitParams.NodeID
|
|
|
|
fmt.Println("QueryNodeID is", Params.QueryNodeID)
|
2021-01-21 15:20:23 +08:00
|
|
|
|
2021-01-26 13:41:41 +08:00
|
|
|
if node.indexClient == nil {
|
|
|
|
log.Println("WARN: null index service detected")
|
|
|
|
}
|
|
|
|
|
|
|
|
if node.dataClient == nil {
|
|
|
|
log.Println("WARN: null data service detected")
|
|
|
|
}
|
|
|
|
|
2020-12-08 14:41:04 +08:00
|
|
|
// todo add connectMaster logic
|
2021-01-19 11:37:16 +08:00
|
|
|
// init services and manager
|
2020-12-08 14:41:04 +08:00
|
|
|
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, node.replica)
|
|
|
|
node.searchService = newSearchService(node.queryNodeLoopCtx, node.replica)
|
|
|
|
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
|
2020-12-24 20:55:40 +08:00
|
|
|
node.loadIndexService = newLoadIndexService(node.queryNodeLoopCtx, node.replica)
|
|
|
|
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, node.loadIndexService.fieldStatsChan)
|
2021-01-27 14:41:56 +08:00
|
|
|
node.segManager = newSegmentManager(node.queryNodeLoopCtx, node.masterClient, node.dataClient, node.indexClient, node.replica, node.dataSyncService.dmStream, node.loadIndexService.loadIndexReqChan)
|
2020-09-15 15:53:10 +08:00
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
// start services
|
2020-11-09 16:27:11 +08:00
|
|
|
go node.dataSyncService.start()
|
2020-11-19 17:09:22 +08:00
|
|
|
go node.searchService.start()
|
2020-11-05 10:52:50 +08:00
|
|
|
go node.metaService.start()
|
2020-12-24 20:55:40 +08:00
|
|
|
go node.loadIndexService.start()
|
2020-12-08 14:41:04 +08:00
|
|
|
go node.statsService.start()
|
2020-12-10 16:31:09 +08:00
|
|
|
|
2021-01-21 15:20:23 +08:00
|
|
|
node.stateCode.Store(internalpb2.StateCode_HEALTHY)
|
2020-12-10 16:31:09 +08:00
|
|
|
<-node.queryNodeLoopCtx.Done()
|
2021-01-21 10:01:29 +08:00
|
|
|
return nil
|
2020-11-05 10:52:50 +08:00
|
|
|
}
|
2020-09-15 15:53:10 +08:00
|
|
|
|
2021-01-21 10:01:29 +08:00
|
|
|
func (node *QueryNode) Stop() error {
|
2021-01-21 15:20:23 +08:00
|
|
|
node.stateCode.Store(internalpb2.StateCode_ABNORMAL)
|
2020-12-08 14:41:04 +08:00
|
|
|
node.queryNodeLoopCancel()
|
|
|
|
|
2020-11-24 16:12:39 +08:00
|
|
|
// free collectionReplica
|
2020-12-08 14:41:04 +08:00
|
|
|
node.replica.freeAll()
|
2020-11-24 16:12:39 +08:00
|
|
|
|
|
|
|
// close services
|
|
|
|
if node.dataSyncService != nil {
|
2020-12-08 14:41:04 +08:00
|
|
|
node.dataSyncService.close()
|
2020-11-24 16:12:39 +08:00
|
|
|
}
|
|
|
|
if node.searchService != nil {
|
2020-12-08 14:41:04 +08:00
|
|
|
node.searchService.close()
|
2020-11-24 16:12:39 +08:00
|
|
|
}
|
2021-01-12 18:03:24 +08:00
|
|
|
if node.loadIndexService != nil {
|
|
|
|
node.loadIndexService.close()
|
|
|
|
}
|
2020-11-24 16:12:39 +08:00
|
|
|
if node.statsService != nil {
|
2020-12-08 14:41:04 +08:00
|
|
|
node.statsService.close()
|
2020-11-24 16:12:39 +08:00
|
|
|
}
|
2021-01-11 18:35:54 +08:00
|
|
|
if node.closer != nil {
|
|
|
|
node.closer.Close()
|
|
|
|
}
|
2021-01-21 10:01:29 +08:00
|
|
|
return nil
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-01-27 14:41:56 +08:00
|
|
|
func (node *QueryNode) SetMasterService(master MasterServiceInterface) error {
|
|
|
|
if master == nil {
|
|
|
|
return errors.New("null master service interface")
|
|
|
|
}
|
|
|
|
node.masterClient = master
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-01-27 09:50:52 +08:00
|
|
|
func (node *QueryNode) SetQueryService(query QueryServiceInterface) error {
|
|
|
|
if query == nil {
|
2021-01-27 14:41:56 +08:00
|
|
|
return errors.New("null query service interface")
|
2021-01-27 09:50:52 +08:00
|
|
|
}
|
|
|
|
node.queryClient = query
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-01-26 13:41:41 +08:00
|
|
|
func (node *QueryNode) SetIndexService(index IndexServiceInterface) error {
|
|
|
|
if index == nil {
|
|
|
|
return errors.New("null index service interface")
|
|
|
|
}
|
|
|
|
node.indexClient = index
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) SetDataService(data DataServiceInterface) error {
|
|
|
|
if data == nil {
|
|
|
|
return errors.New("null data service interface")
|
|
|
|
}
|
|
|
|
node.dataClient = data
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-01-21 15:20:23 +08:00
|
|
|
func (node *QueryNode) GetComponentStates() (*internalpb2.ComponentStates, error) {
|
|
|
|
code, ok := node.stateCode.Load().(internalpb2.StateCode)
|
|
|
|
if !ok {
|
|
|
|
return nil, errors.New("unexpected error in type assertion")
|
|
|
|
}
|
|
|
|
info := &internalpb2.ComponentInfo{
|
|
|
|
NodeID: Params.QueryNodeID,
|
2021-01-23 18:56:08 +08:00
|
|
|
Role: typeutil.QueryNodeRole,
|
2021-01-21 15:20:23 +08:00
|
|
|
StateCode: code,
|
|
|
|
}
|
|
|
|
stats := &internalpb2.ComponentStates{
|
|
|
|
State: info,
|
|
|
|
}
|
|
|
|
return stats, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) GetTimeTickChannel() (string, error) {
|
2021-01-21 10:01:29 +08:00
|
|
|
return Params.QueryTimeTickChannelName, nil
|
2021-01-21 15:20:23 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) GetStatisticsChannel() (string, error) {
|
|
|
|
return Params.StatsChannelName, nil
|
|
|
|
}
|
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
func (node *QueryNode) AddQueryChannel(in *queryPb.AddQueryChannelsRequest) (*commonpb.Status, error) {
|
|
|
|
if node.searchService == nil || node.searchService.searchMsgStream == nil {
|
|
|
|
errMsg := "null search service or null search message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
2021-01-20 10:02:59 +08:00
|
|
|
searchStream, ok := node.searchService.searchMsgStream.(*pulsarms.PulsarMsgStream)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
|
|
|
errMsg := "type assertion failed for search message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
2021-01-20 10:02:59 +08:00
|
|
|
resultStream, ok := node.searchService.searchResultMsgStream.(*pulsarms.PulsarMsgStream)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
|
|
|
errMsg := "type assertion failed for search result message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
|
|
|
// add request channel
|
|
|
|
pulsarBufSize := Params.SearchPulsarBufSize
|
|
|
|
consumeChannels := []string{in.RequestChannelID}
|
|
|
|
consumeSubName := Params.MsgChannelSubName
|
2021-01-20 10:02:59 +08:00
|
|
|
unmarshalDispatcher := util.NewUnmarshalDispatcher()
|
2021-01-19 11:37:16 +08:00
|
|
|
searchStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
|
|
|
|
|
|
|
|
// add result channel
|
|
|
|
producerChannels := []string{in.ResultChannelID}
|
|
|
|
resultStream.CreatePulsarProducers(producerChannels)
|
|
|
|
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
|
|
|
}
|
|
|
|
return status, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) RemoveQueryChannel(in *queryPb.RemoveQueryChannelsRequest) (*commonpb.Status, error) {
|
|
|
|
if node.searchService == nil || node.searchService.searchMsgStream == nil {
|
|
|
|
errMsg := "null search service or null search result message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
2021-01-20 10:02:59 +08:00
|
|
|
searchStream, ok := node.searchService.searchMsgStream.(*pulsarms.PulsarMsgStream)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
|
|
|
errMsg := "type assertion failed for search message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
2021-01-20 10:02:59 +08:00
|
|
|
resultStream, ok := node.searchService.searchResultMsgStream.(*pulsarms.PulsarMsgStream)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
|
|
|
errMsg := "type assertion failed for search result message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
|
|
|
// remove request channel
|
|
|
|
pulsarBufSize := Params.SearchPulsarBufSize
|
|
|
|
consumeChannels := []string{in.RequestChannelID}
|
|
|
|
consumeSubName := Params.MsgChannelSubName
|
2021-01-20 10:02:59 +08:00
|
|
|
unmarshalDispatcher := util.NewUnmarshalDispatcher()
|
2021-01-19 11:37:16 +08:00
|
|
|
// TODO: searchStream.RemovePulsarConsumers(producerChannels)
|
|
|
|
searchStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
|
|
|
|
|
|
|
|
// remove result channel
|
|
|
|
producerChannels := []string{in.ResultChannelID}
|
|
|
|
// TODO: resultStream.RemovePulsarProducer(producerChannels)
|
|
|
|
resultStream.CreatePulsarProducers(producerChannels)
|
|
|
|
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
|
|
|
}
|
|
|
|
return status, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) WatchDmChannels(in *queryPb.WatchDmChannelsRequest) (*commonpb.Status, error) {
|
|
|
|
if node.dataSyncService == nil || node.dataSyncService.dmStream == nil {
|
|
|
|
errMsg := "null data sync service or null data manipulation stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
2021-01-20 10:02:59 +08:00
|
|
|
fgDMMsgStream, ok := node.dataSyncService.dmStream.(*pulsarms.PulsarMsgStream)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
|
|
|
errMsg := "type assertion failed for dm message stream"
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: errMsg,
|
|
|
|
}
|
|
|
|
|
|
|
|
return status, errors.New(errMsg)
|
|
|
|
}
|
|
|
|
|
|
|
|
// add request channel
|
|
|
|
pulsarBufSize := Params.SearchPulsarBufSize
|
|
|
|
consumeChannels := in.ChannelIDs
|
|
|
|
consumeSubName := Params.MsgChannelSubName
|
2021-01-20 10:02:59 +08:00
|
|
|
unmarshalDispatcher := util.NewUnmarshalDispatcher()
|
2021-01-19 11:37:16 +08:00
|
|
|
fgDMMsgStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
|
|
|
|
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
|
|
|
}
|
|
|
|
return status, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) LoadSegments(in *queryPb.LoadSegmentRequest) (*commonpb.Status, error) {
|
|
|
|
// TODO: support db
|
2021-01-20 09:36:50 +08:00
|
|
|
collectionID := in.CollectionID
|
2021-01-21 15:20:23 +08:00
|
|
|
partitionID := in.PartitionID
|
|
|
|
segmentIDs := in.SegmentIDs
|
2021-01-19 11:37:16 +08:00
|
|
|
fieldIDs := in.FieldIDs
|
2021-01-24 18:02:08 +08:00
|
|
|
|
2021-01-26 09:38:40 +08:00
|
|
|
err := node.replica.enablePartitionDM(collectionID, partitionID)
|
|
|
|
if err != nil {
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: err.Error(),
|
|
|
|
}
|
|
|
|
return status, err
|
|
|
|
}
|
|
|
|
|
2021-01-24 18:02:08 +08:00
|
|
|
// segments are ordered before LoadSegments calling
|
|
|
|
if in.LastSegmentState.State == datapb.SegmentState_SegmentGrowing {
|
|
|
|
segmentNum := len(segmentIDs)
|
2021-01-26 09:38:40 +08:00
|
|
|
positions := in.LastSegmentState.StartPositions
|
|
|
|
err = node.segManager.seekSegment(positions)
|
|
|
|
if err != nil {
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: err.Error(),
|
|
|
|
}
|
|
|
|
return status, err
|
|
|
|
}
|
2021-01-24 18:02:08 +08:00
|
|
|
segmentIDs = segmentIDs[:segmentNum-1]
|
|
|
|
}
|
|
|
|
|
2021-01-26 09:38:40 +08:00
|
|
|
err = node.segManager.loadSegment(collectionID, partitionID, segmentIDs, fieldIDs)
|
2021-01-21 15:20:23 +08:00
|
|
|
if err != nil {
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: err.Error(),
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
2021-01-21 15:20:23 +08:00
|
|
|
return status, err
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
2021-01-21 15:20:23 +08:00
|
|
|
return nil, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) ReleaseSegments(in *queryPb.ReleaseSegmentRequest) (*commonpb.Status, error) {
|
2021-01-26 09:38:40 +08:00
|
|
|
for _, id := range in.PartitionIDs {
|
|
|
|
err := node.replica.enablePartitionDM(in.CollectionID, id)
|
|
|
|
if err != nil {
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: err.Error(),
|
|
|
|
}
|
|
|
|
return status, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-21 15:20:23 +08:00
|
|
|
// release all fields in the segments
|
|
|
|
for _, id := range in.SegmentIDs {
|
|
|
|
err := node.segManager.releaseSegment(id)
|
2021-01-19 11:37:16 +08:00
|
|
|
if err != nil {
|
|
|
|
status := &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
|
|
|
Reason: err.Error(),
|
|
|
|
}
|
|
|
|
return status, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil, nil
|
|
|
|
}
|