mirror of https://github.com/milvus-io/milvus.git
Fix queryservice start
Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>pull/4973/head^2
parent
c231f6da79
commit
dd72dfcbd1
|
@ -30,7 +30,10 @@ func NewQueryService(ctx context.Context) (*QueryService, error) {
|
||||||
const interval = 200
|
const interval = 200
|
||||||
|
|
||||||
queryservice.Params.Init()
|
queryservice.Params.Init()
|
||||||
svr := qs.NewServer(ctx)
|
svr, err := qs.NewServer(ctx)
|
||||||
|
if err != nil {
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
log.Println("Queryservice id is", queryservice.Params.QueryServiceID)
|
log.Println("Queryservice id is", queryservice.Params.QueryServiceID)
|
||||||
|
|
||||||
// --- Master Service Client ---
|
// --- Master Service Client ---
|
||||||
|
@ -66,7 +69,7 @@ func NewQueryService(ctx context.Context) (*QueryService, error) {
|
||||||
if msStates.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
if msStates.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
if msStates.State.StateCode != internalpb2.StateCode_HEALTHY {
|
if msStates.State.StateCode != internalpb2.StateCode_INITIALIZING && msStates.State.StateCode != internalpb2.StateCode_HEALTHY {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
break
|
break
|
||||||
|
|
|
@ -6,30 +6,40 @@ import (
|
||||||
"net"
|
"net"
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
|
||||||
|
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/distributed/dataservice"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/distributed/masterservice"
|
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/queryservice"
|
qs "github.com/zilliztech/milvus-distributed/internal/queryservice"
|
||||||
)
|
)
|
||||||
|
|
||||||
type QueryService = queryservice.QueryService
|
|
||||||
|
|
||||||
type Server struct {
|
type Server struct {
|
||||||
grpcServer *grpc.Server
|
grpcServer *grpc.Server
|
||||||
queryService *QueryService
|
grpcError error
|
||||||
|
grpcErrMux sync.Mutex
|
||||||
|
|
||||||
loopCtx context.Context
|
loopCtx context.Context
|
||||||
loopCancel func()
|
loopCancel context.CancelFunc
|
||||||
loopWg sync.WaitGroup
|
|
||||||
|
queryService *qs.QueryService
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewServer(ctx context.Context) (*Server, error) {
|
||||||
|
ctx1, cancel := context.WithCancel(ctx)
|
||||||
|
service, err := qs.NewQueryService(ctx1)
|
||||||
|
if err != nil {
|
||||||
|
cancel()
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
return &Server{
|
||||||
|
queryService: service,
|
||||||
|
loopCtx: ctx1,
|
||||||
|
loopCancel: cancel,
|
||||||
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) Init() error {
|
func (s *Server) Init() error {
|
||||||
|
@ -42,28 +52,43 @@ func (s *Server) Init() error {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) Start() error {
|
func (s *Server) Start() error {
|
||||||
masterServiceClient, err := masterservice.NewGrpcClient(queryservice.Params.MasterServiceAddress, 30*time.Second)
|
log.Println("start query service ...")
|
||||||
|
|
||||||
|
s.grpcServer = grpc.NewServer()
|
||||||
|
querypb.RegisterQueryServiceServer(s.grpcServer, s)
|
||||||
|
log.Println("Starting start query service Server")
|
||||||
|
lis, err := net.Listen("tcp", ":"+strconv.Itoa(qs.Params.Port))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
s.queryService.SetMasterService(masterServiceClient)
|
|
||||||
dataServiceClient := dataservice.NewClient(queryservice.Params.DataServiceAddress)
|
go func() {
|
||||||
s.queryService.SetDataService(dataServiceClient)
|
if err := s.grpcServer.Serve(lis); err != nil {
|
||||||
log.Println("start query service ...")
|
s.grpcErrMux.Lock()
|
||||||
s.loopWg.Add(1)
|
defer s.grpcErrMux.Unlock()
|
||||||
go s.grpcLoop()
|
s.grpcError = err
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
s.grpcErrMux.Lock()
|
||||||
|
err = s.grpcError
|
||||||
|
s.grpcErrMux.Unlock()
|
||||||
|
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
s.queryService.Start()
|
s.queryService.Start()
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) Stop() error {
|
func (s *Server) Stop() error {
|
||||||
s.queryService.Stop()
|
err := s.queryService.Stop()
|
||||||
s.loopCancel()
|
s.loopCancel()
|
||||||
if s.grpcServer != nil {
|
if s.grpcServer != nil {
|
||||||
s.grpcServer.GracefulStop()
|
s.grpcServer.GracefulStop()
|
||||||
}
|
}
|
||||||
s.loopWg.Wait()
|
return err
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) GetComponentStates(ctx context.Context, req *commonpb.Empty) (*internalpb2.ComponentStates, error) {
|
func (s *Server) GetComponentStates(ctx context.Context, req *commonpb.Empty) (*internalpb2.ComponentStates, error) {
|
||||||
|
@ -120,12 +145,12 @@ func (s *Server) GetStatisticsChannel(ctx context.Context, req *commonpb.Empty)
|
||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) SetMasterService(m queryservice.MasterServiceInterface) error {
|
func (s *Server) SetMasterService(m qs.MasterServiceInterface) error {
|
||||||
s.queryService.SetMasterService(m)
|
s.queryService.SetMasterService(m)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) SetDataService(d queryservice.DataServiceInterface) error {
|
func (s *Server) SetDataService(d qs.DataServiceInterface) error {
|
||||||
s.queryService.SetDataService(d)
|
s.queryService.SetDataService(d)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -169,36 +194,3 @@ func (s *Server) CreateQueryChannel(ctx context.Context, req *commonpb.Empty) (*
|
||||||
func (s *Server) GetSegmentInfo(ctx context.Context, req *querypb.SegmentInfoRequest) (*querypb.SegmentInfoResponse, error) {
|
func (s *Server) GetSegmentInfo(ctx context.Context, req *querypb.SegmentInfoRequest) (*querypb.SegmentInfoResponse, error) {
|
||||||
return s.queryService.GetSegmentInfo(req)
|
return s.queryService.GetSegmentInfo(req)
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewServer(ctx context.Context) *Server {
|
|
||||||
ctx1, cancel := context.WithCancel(ctx)
|
|
||||||
service, err := queryservice.NewQueryService(ctx1)
|
|
||||||
if err != nil {
|
|
||||||
log.Fatal(errors.New("create QueryService failed"))
|
|
||||||
}
|
|
||||||
|
|
||||||
return &Server{
|
|
||||||
queryService: service,
|
|
||||||
loopCtx: ctx1,
|
|
||||||
loopCancel: cancel,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (s *Server) grpcLoop() {
|
|
||||||
defer s.loopWg.Done()
|
|
||||||
|
|
||||||
log.Println("Starting start query service Server")
|
|
||||||
lis, err := net.Listen("tcp", ":"+strconv.Itoa(queryservice.Params.Port))
|
|
||||||
if err != nil {
|
|
||||||
log.Fatalf("query service grpc server fatal error=%v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
s.grpcServer = grpc.NewServer()
|
|
||||||
querypb.RegisterQueryServiceServer(s.grpcServer, s)
|
|
||||||
|
|
||||||
log.Println("queryService's server register finished")
|
|
||||||
if err = s.grpcServer.Serve(lis); err != nil {
|
|
||||||
log.Fatalf("queryService grpc server fatal error=%v", err)
|
|
||||||
}
|
|
||||||
log.Println("query service grpc server starting...")
|
|
||||||
}
|
|
||||||
|
|
|
@ -32,6 +32,7 @@ type QueryNodeStatsMsg = msgstream.QueryNodeStatsMsg
|
||||||
type RepackFunc = msgstream.RepackFunc
|
type RepackFunc = msgstream.RepackFunc
|
||||||
type Consumer = pulsar.Consumer
|
type Consumer = pulsar.Consumer
|
||||||
type Producer = pulsar.Producer
|
type Producer = pulsar.Producer
|
||||||
|
type MessageID = pulsar.MessageID
|
||||||
type UnmarshalDispatcher = msgstream.UnmarshalDispatcher
|
type UnmarshalDispatcher = msgstream.UnmarshalDispatcher
|
||||||
|
|
||||||
type PulsarMsgStream struct {
|
type PulsarMsgStream struct {
|
||||||
|
@ -46,6 +47,8 @@ type PulsarMsgStream struct {
|
||||||
wait *sync.WaitGroup
|
wait *sync.WaitGroup
|
||||||
streamCancel func()
|
streamCancel func()
|
||||||
pulsarBufSize int64
|
pulsarBufSize int64
|
||||||
|
consumerLock *sync.Mutex
|
||||||
|
consumerReflects []reflect.SelectCase
|
||||||
}
|
}
|
||||||
|
|
||||||
func newPulsarMsgStream(ctx context.Context,
|
func newPulsarMsgStream(ctx context.Context,
|
||||||
|
@ -58,22 +61,30 @@ func newPulsarMsgStream(ctx context.Context,
|
||||||
producers := make([]Producer, 0)
|
producers := make([]Producer, 0)
|
||||||
consumers := make([]Consumer, 0)
|
consumers := make([]Consumer, 0)
|
||||||
consumerChannels := make([]string, 0)
|
consumerChannels := make([]string, 0)
|
||||||
|
consumerReflects := make([]reflect.SelectCase, 0)
|
||||||
|
receiveBuf := make(chan *MsgPack, receiveBufSize)
|
||||||
|
|
||||||
|
client, err := pulsar.NewClient(pulsar.ClientOptions{URL: address})
|
||||||
|
if err != nil {
|
||||||
|
defer streamCancel()
|
||||||
|
log.Printf("Set pulsar client failed, error = %v", err)
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
stream := &PulsarMsgStream{
|
stream := &PulsarMsgStream{
|
||||||
ctx: streamCtx,
|
ctx: streamCtx,
|
||||||
streamCancel: streamCancel,
|
client: client,
|
||||||
producers: producers,
|
producers: producers,
|
||||||
consumers: consumers,
|
consumers: consumers,
|
||||||
consumerChannels: consumerChannels,
|
consumerChannels: consumerChannels,
|
||||||
unmarshal: unmarshal,
|
unmarshal: unmarshal,
|
||||||
pulsarBufSize: pulsarBufSize,
|
pulsarBufSize: pulsarBufSize,
|
||||||
|
receiveBuf: receiveBuf,
|
||||||
|
streamCancel: streamCancel,
|
||||||
|
consumerReflects: consumerReflects,
|
||||||
|
consumerLock: &sync.Mutex{},
|
||||||
}
|
}
|
||||||
client, err := pulsar.NewClient(pulsar.ClientOptions{URL: address})
|
|
||||||
if err != nil {
|
|
||||||
log.Printf("Set pulsar client failed, error = %v", err)
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
stream.client = client
|
|
||||||
stream.receiveBuf = make(chan *MsgPack, receiveBufSize)
|
|
||||||
return stream, nil
|
return stream, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -118,7 +129,14 @@ func (ms *PulsarMsgStream) AsConsumer(channels []string,
|
||||||
return errors.New("pulsar is not ready, consumer is nil")
|
return errors.New("pulsar is not ready, consumer is nil")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ms.consumerLock.Lock()
|
||||||
ms.consumers = append(ms.consumers, pc)
|
ms.consumers = append(ms.consumers, pc)
|
||||||
|
ms.consumerChannels = append(ms.consumerChannels, channels[i])
|
||||||
|
ms.consumerReflects = append(ms.consumerReflects, reflect.SelectCase{
|
||||||
|
Dir: reflect.SelectRecv,
|
||||||
|
Chan: reflect.ValueOf(pc.Chan()),
|
||||||
|
})
|
||||||
|
ms.consumerLock.Unlock()
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
err := util.Retry(10, time.Millisecond*200, fn)
|
err := util.Retry(10, time.Millisecond*200, fn)
|
||||||
|
@ -299,12 +317,6 @@ func (ms *PulsarMsgStream) Consume() *MsgPack {
|
||||||
func (ms *PulsarMsgStream) bufMsgPackToChannel() {
|
func (ms *PulsarMsgStream) bufMsgPackToChannel() {
|
||||||
defer ms.wait.Done()
|
defer ms.wait.Done()
|
||||||
|
|
||||||
cases := make([]reflect.SelectCase, len(ms.consumers))
|
|
||||||
for i := 0; i < len(ms.consumers); i++ {
|
|
||||||
ch := ms.consumers[i].Chan()
|
|
||||||
cases[i] = reflect.SelectCase{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(ch)}
|
|
||||||
}
|
|
||||||
|
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-ms.ctx.Done():
|
case <-ms.ctx.Done():
|
||||||
|
@ -314,7 +326,9 @@ func (ms *PulsarMsgStream) bufMsgPackToChannel() {
|
||||||
tsMsgList := make([]TsMsg, 0)
|
tsMsgList := make([]TsMsg, 0)
|
||||||
|
|
||||||
for {
|
for {
|
||||||
chosen, value, ok := reflect.Select(cases)
|
ms.consumerLock.Lock()
|
||||||
|
chosen, value, ok := reflect.Select(ms.consumerReflects)
|
||||||
|
ms.consumerLock.Unlock()
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Printf("channel closed")
|
log.Printf("channel closed")
|
||||||
return
|
return
|
||||||
|
@ -339,6 +353,11 @@ func (ms *PulsarMsgStream) bufMsgPackToChannel() {
|
||||||
log.Printf("Failed to unmarshal tsMsg, error = %v", err)
|
log.Printf("Failed to unmarshal tsMsg, error = %v", err)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
|
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||||
|
ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||||
|
MsgID: typeutil.PulsarMsgIDToString(pulsarMsg.ID()),
|
||||||
|
})
|
||||||
tsMsgList = append(tsMsgList, tsMsg)
|
tsMsgList = append(tsMsgList, tsMsg)
|
||||||
|
|
||||||
noMoreMessage := true
|
noMoreMessage := true
|
||||||
|
@ -386,6 +405,7 @@ func (ms *PulsarMsgStream) Seek(mp *internalpb2.MsgPosition) error {
|
||||||
type PulsarTtMsgStream struct {
|
type PulsarTtMsgStream struct {
|
||||||
PulsarMsgStream
|
PulsarMsgStream
|
||||||
unsolvedBuf map[Consumer][]TsMsg
|
unsolvedBuf map[Consumer][]TsMsg
|
||||||
|
unsolvedMutex *sync.Mutex
|
||||||
lastTimeStamp Timestamp
|
lastTimeStamp Timestamp
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -394,27 +414,53 @@ func NewPulsarTtMsgStream(ctx context.Context,
|
||||||
receiveBufSize int64,
|
receiveBufSize int64,
|
||||||
pulsarBufSize int64,
|
pulsarBufSize int64,
|
||||||
unmarshal msgstream.UnmarshalDispatcher) (*PulsarTtMsgStream, error) {
|
unmarshal msgstream.UnmarshalDispatcher) (*PulsarTtMsgStream, error) {
|
||||||
|
pulsarMsgStream, err := newPulsarMsgStream(ctx, address, receiveBufSize, pulsarBufSize, unmarshal)
|
||||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
|
||||||
pulsarMsgStream := PulsarMsgStream{
|
|
||||||
ctx: streamCtx,
|
|
||||||
streamCancel: streamCancel,
|
|
||||||
pulsarBufSize: pulsarBufSize,
|
|
||||||
unmarshal: unmarshal,
|
|
||||||
}
|
|
||||||
|
|
||||||
client, err := pulsar.NewClient(pulsar.ClientOptions{URL: address})
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("Set pulsar client failed, error = %v", err)
|
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
pulsarMsgStream.client = client
|
unsolvedBuf := make(map[Consumer][]TsMsg)
|
||||||
pulsarMsgStream.receiveBuf = make(chan *MsgPack, receiveBufSize)
|
|
||||||
return &PulsarTtMsgStream{
|
return &PulsarTtMsgStream{
|
||||||
PulsarMsgStream: pulsarMsgStream,
|
PulsarMsgStream: *pulsarMsgStream,
|
||||||
|
unsolvedBuf: unsolvedBuf,
|
||||||
|
unsolvedMutex: &sync.Mutex{},
|
||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ms *PulsarTtMsgStream) AsConsumer(channels []string,
|
||||||
|
subName string) {
|
||||||
|
for i := 0; i < len(channels); i++ {
|
||||||
|
fn := func() error {
|
||||||
|
receiveChannel := make(chan pulsar.ConsumerMessage, ms.pulsarBufSize)
|
||||||
|
pc, err := ms.client.Subscribe(pulsar.ConsumerOptions{
|
||||||
|
Topic: channels[i],
|
||||||
|
SubscriptionName: subName,
|
||||||
|
Type: pulsar.KeyShared,
|
||||||
|
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
|
||||||
|
MessageChannel: receiveChannel,
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
if pc == nil {
|
||||||
|
return errors.New("pulsar is not ready, consumer is nil")
|
||||||
|
}
|
||||||
|
|
||||||
|
ms.consumerLock.Lock()
|
||||||
|
ms.consumers = append(ms.consumers, pc)
|
||||||
|
ms.unsolvedBuf[pc] = make([]TsMsg, 0)
|
||||||
|
ms.consumerChannels = append(ms.consumerChannels, channels[i])
|
||||||
|
ms.consumerLock.Unlock()
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
err := util.Retry(10, time.Millisecond*200, fn)
|
||||||
|
if err != nil {
|
||||||
|
errMsg := "Failed to create consumer " + channels[i] + ", error = " + err.Error()
|
||||||
|
panic(errMsg)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func (ms *PulsarTtMsgStream) Start() {
|
func (ms *PulsarTtMsgStream) Start() {
|
||||||
ms.wait = &sync.WaitGroup{}
|
ms.wait = &sync.WaitGroup{}
|
||||||
if ms.consumers != nil {
|
if ms.consumers != nil {
|
||||||
|
@ -428,33 +474,32 @@ func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
|
||||||
ms.unsolvedBuf = make(map[Consumer][]TsMsg)
|
ms.unsolvedBuf = make(map[Consumer][]TsMsg)
|
||||||
isChannelReady := make(map[Consumer]bool)
|
isChannelReady := make(map[Consumer]bool)
|
||||||
eofMsgTimeStamp := make(map[Consumer]Timestamp)
|
eofMsgTimeStamp := make(map[Consumer]Timestamp)
|
||||||
for _, consumer := range ms.consumers {
|
|
||||||
ms.unsolvedBuf[consumer] = make([]TsMsg, 0)
|
|
||||||
}
|
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-ms.ctx.Done():
|
case <-ms.ctx.Done():
|
||||||
return
|
return
|
||||||
default:
|
default:
|
||||||
wg := sync.WaitGroup{}
|
wg := sync.WaitGroup{}
|
||||||
mu := sync.Mutex{}
|
|
||||||
findMapMutex := sync.RWMutex{}
|
findMapMutex := sync.RWMutex{}
|
||||||
|
ms.consumerLock.Lock()
|
||||||
for _, consumer := range ms.consumers {
|
for _, consumer := range ms.consumers {
|
||||||
if isChannelReady[consumer] {
|
if isChannelReady[consumer] {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
go ms.findTimeTick(consumer, eofMsgTimeStamp, &wg, &mu, &findMapMutex)
|
go ms.findTimeTick(consumer, eofMsgTimeStamp, &wg, &findMapMutex)
|
||||||
}
|
}
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
timeStamp, ok := checkTimeTickMsg(eofMsgTimeStamp, isChannelReady, &findMapMutex)
|
timeStamp, ok := checkTimeTickMsg(eofMsgTimeStamp, isChannelReady, &findMapMutex)
|
||||||
|
ms.consumerLock.Unlock()
|
||||||
if !ok || timeStamp <= ms.lastTimeStamp {
|
if !ok || timeStamp <= ms.lastTimeStamp {
|
||||||
//log.Printf("All timeTick's timestamps are inconsistent")
|
//log.Printf("All timeTick's timestamps are inconsistent")
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
timeTickBuf := make([]TsMsg, 0)
|
timeTickBuf := make([]TsMsg, 0)
|
||||||
msgPositions := make([]*internalpb2.MsgPosition, 0)
|
msgPositions := make([]*internalpb2.MsgPosition, 0)
|
||||||
|
ms.unsolvedMutex.Lock()
|
||||||
for consumer, msgs := range ms.unsolvedBuf {
|
for consumer, msgs := range ms.unsolvedBuf {
|
||||||
tempBuffer := make([]TsMsg, 0)
|
tempBuffer := make([]TsMsg, 0)
|
||||||
var timeTickMsg TsMsg
|
var timeTickMsg TsMsg
|
||||||
|
@ -485,6 +530,7 @@ func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
ms.unsolvedMutex.Unlock()
|
||||||
|
|
||||||
msgPack := MsgPack{
|
msgPack := MsgPack{
|
||||||
BeginTs: ms.lastTimeStamp,
|
BeginTs: ms.lastTimeStamp,
|
||||||
|
@ -502,7 +548,6 @@ func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
|
||||||
func (ms *PulsarTtMsgStream) findTimeTick(consumer Consumer,
|
func (ms *PulsarTtMsgStream) findTimeTick(consumer Consumer,
|
||||||
eofMsgMap map[Consumer]Timestamp,
|
eofMsgMap map[Consumer]Timestamp,
|
||||||
wg *sync.WaitGroup,
|
wg *sync.WaitGroup,
|
||||||
mu *sync.Mutex,
|
|
||||||
findMapMutex *sync.RWMutex) {
|
findMapMutex *sync.RWMutex) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
for {
|
for {
|
||||||
|
@ -519,14 +564,13 @@ func (ms *PulsarTtMsgStream) findTimeTick(consumer Consumer,
|
||||||
headerMsg := commonpb.MsgHeader{}
|
headerMsg := commonpb.MsgHeader{}
|
||||||
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("Failed to unmarshal, error = %v", err)
|
log.Printf("Failed to unmarshal message header, error = %v", err)
|
||||||
|
continue
|
||||||
}
|
}
|
||||||
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
||||||
if tsMsg == nil && err != nil {
|
|
||||||
panic("null unMarshalFunc for " + headerMsg.Base.MsgType.String() + " msg type")
|
|
||||||
}
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("Failed to unmarshal, error = %v", err)
|
log.Printf("Failed to unmarshal tsMsg, error = %v", err)
|
||||||
|
continue
|
||||||
}
|
}
|
||||||
// set pulsar info to tsMsg
|
// set pulsar info to tsMsg
|
||||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||||
|
@ -534,9 +578,9 @@ func (ms *PulsarTtMsgStream) findTimeTick(consumer Consumer,
|
||||||
MsgID: typeutil.PulsarMsgIDToString(pulsarMsg.ID()),
|
MsgID: typeutil.PulsarMsgIDToString(pulsarMsg.ID()),
|
||||||
})
|
})
|
||||||
|
|
||||||
mu.Lock()
|
ms.unsolvedMutex.Lock()
|
||||||
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
||||||
mu.Unlock()
|
ms.unsolvedMutex.Unlock()
|
||||||
|
|
||||||
if headerMsg.Base.MsgType == commonpb.MsgType_kTimeTick {
|
if headerMsg.Base.MsgType == commonpb.MsgType_kTimeTick {
|
||||||
findMapMutex.Lock()
|
findMapMutex.Lock()
|
||||||
|
@ -549,50 +593,60 @@ func (ms *PulsarTtMsgStream) findTimeTick(consumer Consumer,
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ms *PulsarTtMsgStream) Seek(mp *internalpb2.MsgPosition) error {
|
func (ms *PulsarTtMsgStream) Seek(mp *internalpb2.MsgPosition) error {
|
||||||
|
var consumer Consumer
|
||||||
|
var messageID MessageID
|
||||||
for index, channel := range ms.consumerChannels {
|
for index, channel := range ms.consumerChannels {
|
||||||
if filepath.Base(channel) == filepath.Base(mp.ChannelName) {
|
if filepath.Base(channel) == filepath.Base(mp.ChannelName) {
|
||||||
messageID, err := typeutil.StringToPulsarMsgID(mp.MsgID)
|
seekMsgID, err := typeutil.StringToPulsarMsgID(mp.MsgID)
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
consumer := ms.consumers[index]
|
|
||||||
err = (consumer).Seek(messageID)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
consumer = ms.consumers[index]
|
||||||
|
messageID = seekMsgID
|
||||||
|
break
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
for {
|
if consumer != nil {
|
||||||
select {
|
err := (consumer).Seek(messageID)
|
||||||
case <-ms.ctx.Done():
|
if err != nil {
|
||||||
return nil
|
return err
|
||||||
case pulsarMsg, ok := <-consumer.Chan():
|
}
|
||||||
if !ok {
|
|
||||||
return errors.New("consumer closed")
|
|
||||||
}
|
|
||||||
consumer.Ack(pulsarMsg)
|
|
||||||
|
|
||||||
headerMsg := commonpb.MsgHeader{}
|
ms.unsolvedMutex.Lock()
|
||||||
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
ms.unsolvedBuf[consumer] = make([]TsMsg, 0)
|
||||||
if err != nil {
|
for {
|
||||||
log.Printf("Failed to unmarshal msgHeader, error = %v", err)
|
select {
|
||||||
}
|
case <-ms.ctx.Done():
|
||||||
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
return nil
|
||||||
if tsMsg == nil && err != nil {
|
case pulsarMsg, ok := <-consumer.Chan():
|
||||||
panic("null unMarshalFunc for " + headerMsg.Base.MsgType.String() + " msg type")
|
if !ok {
|
||||||
|
return errors.New("consumer closed")
|
||||||
|
}
|
||||||
|
consumer.Ack(pulsarMsg)
|
||||||
|
|
||||||
|
headerMsg := commonpb.MsgHeader{}
|
||||||
|
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||||
|
if err != nil {
|
||||||
|
log.Printf("Failed to unmarshal message header, error = %v", err)
|
||||||
|
}
|
||||||
|
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
||||||
|
if err != nil {
|
||||||
|
log.Printf("Failed to unmarshal tsMsg, error = %v", err)
|
||||||
|
}
|
||||||
|
if tsMsg.Type() == commonpb.MsgType_kTimeTick {
|
||||||
|
if tsMsg.BeginTs() >= mp.Timestamp {
|
||||||
|
ms.unsolvedMutex.Unlock()
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
if err != nil {
|
continue
|
||||||
log.Printf("Failed to unmarshal pulsarMsg, error = %v", err)
|
}
|
||||||
}
|
if tsMsg.BeginTs() > mp.Timestamp {
|
||||||
if tsMsg.Type() == commonpb.MsgType_kTimeTick {
|
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||||
if tsMsg.BeginTs() >= mp.Timestamp {
|
ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||||
return nil
|
MsgID: typeutil.PulsarMsgIDToString(pulsarMsg.ID()),
|
||||||
}
|
})
|
||||||
continue
|
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
||||||
}
|
|
||||||
if tsMsg.BeginTs() > mp.Timestamp {
|
|
||||||
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -568,7 +568,7 @@ func TestStream_PulsarTtMsgStream_Seek(t *testing.T) {
|
||||||
msgPack3.Msgs = append(msgPack3.Msgs, getTsMsg(commonpb.MsgType_kInsert, 9, 9))
|
msgPack3.Msgs = append(msgPack3.Msgs, getTsMsg(commonpb.MsgType_kInsert, 9, 9))
|
||||||
|
|
||||||
msgPack4 := MsgPack{}
|
msgPack4 := MsgPack{}
|
||||||
msgPack4.Msgs = append(msgPack2.Msgs, getTimeTickMsg(11, 11, 11))
|
msgPack4.Msgs = append(msgPack4.Msgs, getTimeTickMsg(11, 11, 11))
|
||||||
|
|
||||||
msgPack5 := MsgPack{}
|
msgPack5 := MsgPack{}
|
||||||
msgPack5.Msgs = append(msgPack5.Msgs, getTimeTickMsg(15, 15, 15))
|
msgPack5.Msgs = append(msgPack5.Msgs, getTimeTickMsg(15, 15, 15))
|
||||||
|
|
|
@ -64,9 +64,16 @@ func (qs *QueryService) Init() error {
|
||||||
|
|
||||||
func (qs *QueryService) Start() error {
|
func (qs *QueryService) Start() error {
|
||||||
isInit := qs.isInit.Load().(bool)
|
isInit := qs.isInit.Load().(bool)
|
||||||
if !isInit {
|
|
||||||
|
switch {
|
||||||
|
case !isInit:
|
||||||
return errors.New("call start before init")
|
return errors.New("call start before init")
|
||||||
|
case qs.dataServiceClient == nil:
|
||||||
|
return errors.New("dataService Client not set")
|
||||||
|
case qs.masterServiceClient == nil:
|
||||||
|
return errors.New("masterService Client not set")
|
||||||
}
|
}
|
||||||
|
|
||||||
qs.stateCode.Store(internalpb2.StateCode_HEALTHY)
|
qs.stateCode.Store(internalpb2.StateCode_HEALTHY)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -120,6 +127,17 @@ func (qs *QueryService) RegisterNode(req *querypb.RegisterNodeRequest) (*querypb
|
||||||
var node *queryNodeInfo
|
var node *queryNodeInfo
|
||||||
if qs.enableGrpc {
|
if qs.enableGrpc {
|
||||||
client := nodeclient.NewClient(registerNodeAddress)
|
client := nodeclient.NewClient(registerNodeAddress)
|
||||||
|
if err := client.Init(); err != nil {
|
||||||
|
return &querypb.RegisterNodeResponse{
|
||||||
|
Status: &commonpb.Status{
|
||||||
|
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||||
|
},
|
||||||
|
InitParams: new(internalpb2.InitParams),
|
||||||
|
}, err
|
||||||
|
}
|
||||||
|
if err := client.Start(); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
node = newQueryNodeInfo(client)
|
node = newQueryNodeInfo(client)
|
||||||
} else {
|
} else {
|
||||||
client := querynode.NewQueryNode(qs.loopCtx, uint64(allocatedID))
|
client := querynode.NewQueryNode(qs.loopCtx, uint64(allocatedID))
|
||||||
|
|
|
@ -2,7 +2,7 @@ grpcio==1.26.0
|
||||||
grpcio-tools==1.26.0
|
grpcio-tools==1.26.0
|
||||||
numpy==1.18.1
|
numpy==1.18.1
|
||||||
pytest-cov==2.8.1
|
pytest-cov==2.8.1
|
||||||
pymilvus-distributed==0.0.19
|
pymilvus-distributed==0.0.20
|
||||||
sklearn==0.0
|
sklearn==0.0
|
||||||
pytest==4.5.0
|
pytest==4.5.0
|
||||||
pytest-timeout==1.3.3
|
pytest-timeout==1.3.3
|
||||||
|
|
Loading…
Reference in New Issue