milvus/internal/proxynode/proxy_node.go
sunby 189ac881f3 Fix bugs (#5676)
* Remove redundant session startup

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Register datanode after start success

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* fix meta snap shot

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* fix datanode message stream channel

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

* Fix bugs when drop empty collection

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Fix bug of getting pchan statistics from task scheduler

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* Fix i/dist/dataservice test code

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* Fix epoch lifetime not applied

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* fix datanode flowgraph dd node

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

* Fix handle datanode timetick bug

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Remove repack function of dml stream

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* fix proxynode

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Apply extended seal policy

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* add check for time tick

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* fix check

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Fix the repack function of dml stream

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* Fix the bug when send statistics of pchan

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* Fix the repack function when craete dml stream

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* fix bugs

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* fix describe collection

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Fix bug when send timestamp statistics

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* fix data node

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Add length check before flush request

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* add log for data node

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Fix SaveBinlog bugs

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Add more log in datanode

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

* Put SegmentState.Flushing as the last one in enum to fit the client

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Fix params in GetInsertBinlogPaths

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Rename policy

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Remove unused ddl functions and fields

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* Remove pchan when drop collection

Signed-off-by: dragondriver <jiquan.long@zilliz.com>

* Add balanced assignment policy

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* fix master ut

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Add lock in session manager

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* add log for debug

Signed-off-by: yefu.chen <yefu.chen@zilliz.com>

* Fix some logic bug and typo

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>

* Fix recover bugs

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Get collection scheme of a specific timestamp

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

* Change CheckPoint to SegmentInfo in VchannelInfo

Signed-off-by: sunby <bingyi.sun@zilliz.com>

* Recover Unflushed segment numOfRows

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

* Fix dataservice unit tests

Signed-off-by: sunby <bingyi.sun@zilliz.com>

Co-authored-by: yefu.chen <yefu.chen@zilliz.com>
Co-authored-by: yangxuan <xuan.yang@zilliz.com>
Co-authored-by: dragondriver <jiquan.long@zilliz.com>
Co-authored-by: Congqi Xia <congqi.xia@zilliz.com>
2021-06-15 16:06:11 +08:00

454 lines
13 KiB
Go

// 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 proxynode
import (
"context"
"errors"
"fmt"
"math/rand"
"sync"
"sync/atomic"
"time"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/allocator"
"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/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type ProxyNode struct {
ctx context.Context
cancel func()
wg sync.WaitGroup
initParams *internalpb.InitParams
ip string
port int
stateCode atomic.Value
masterService types.MasterService
indexService types.IndexService
dataService types.DataService
proxyService types.ProxyService
queryService types.QueryService
chMgr channelsMgr
sched *TaskScheduler
tick *timeTick
chTicker channelsTimeTicker
idAllocator *allocator.IDAllocator
tsoAllocator *TimestampAllocator
segAssigner *SegIDAssigner
session *sessionutil.Session
queryMsgStream msgstream.MsgStream
msFactory msgstream.Factory
// Add callback functions at different stages
startCallbacks []func()
closeCallbacks []func()
}
func NewProxyNode(ctx context.Context, factory msgstream.Factory) (*ProxyNode, error) {
rand.Seed(time.Now().UnixNano())
ctx1, cancel := context.WithCancel(ctx)
node := &ProxyNode{
ctx: ctx1,
cancel: cancel,
msFactory: factory,
}
node.UpdateStateCode(internalpb.StateCode_Abnormal)
log.Debug("ProxyNode", zap.Any("State", node.stateCode.Load()))
return node, nil
}
// Register register proxy node at etcd
func (node *ProxyNode) Register() error {
node.session = sessionutil.NewSession(node.ctx, Params.MetaRootPath, []string{Params.EtcdAddress})
node.session.Init(typeutil.ProxyNodeRole, Params.NetworkAddress, false)
Params.ProxyID = node.session.ServerID
return nil
}
func (node *ProxyNode) Init() error {
// todo wait for proxyservice state changed to Healthy
ctx := context.Background()
err := funcutil.WaitForComponentHealthy(ctx, node.proxyService, "ProxyService", 1000000, time.Millisecond*200)
if err != nil {
return err
}
log.Debug("ProxyService is ready ...")
request := &proxypb.RegisterNodeRequest{
Address: &commonpb.Address{
Ip: Params.IP,
Port: int64(Params.NetworkPort),
},
}
response, err := node.proxyService.RegisterNode(ctx, request)
if err != nil {
log.Debug("ProxyNode RegisterNode failed", zap.Error(err))
return err
}
if response.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Debug("ProxyNode RegisterNode failed", zap.String("Reason", response.Status.Reason))
return errors.New(response.Status.Reason)
}
err = Params.LoadConfigFromInitParams(response.InitParams)
if err != nil {
log.Debug("ProxyNode LoadConfigFromInitParams failed", zap.Error(err))
return err
}
// wait for dataservice state changed to Healthy
if node.dataService != nil {
log.Debug("ProxyNode wait for dataService ready")
err := funcutil.WaitForComponentHealthy(ctx, node.dataService, "DataService", 1000000, time.Millisecond*200)
if err != nil {
log.Debug("ProxyNode wait for dataService ready failed", zap.Error(err))
return err
}
log.Debug("ProxyNode dataService is ready")
}
// wait for queryService state changed to Healthy
if node.queryService != nil {
log.Debug("ProxyNode wait for queryService ready")
err := funcutil.WaitForComponentHealthy(ctx, node.queryService, "QueryService", 1000000, time.Millisecond*200)
if err != nil {
log.Debug("ProxyNode wait for queryService ready failed", zap.Error(err))
return err
}
log.Debug("ProxyNode queryService is ready")
}
// wait for indexservice state changed to Healthy
if node.indexService != nil {
log.Debug("ProxyNode wait for indexService ready")
err := funcutil.WaitForComponentHealthy(ctx, node.indexService, "IndexService", 1000000, time.Millisecond*200)
if err != nil {
log.Debug("ProxyNode wait for indexService ready failed", zap.Error(err))
return err
}
log.Debug("ProxyNode indexService is ready")
}
if node.queryService != nil {
resp, err := node.queryService.CreateQueryChannel(ctx, &querypb.CreateQueryChannelRequest{})
if err != nil {
log.Debug("ProxyNode CreateQueryChannel failed", zap.Error(err))
return err
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Debug("ProxyNode CreateQueryChannel failed", zap.String("reason", resp.Status.Reason))
return errors.New(resp.Status.Reason)
}
log.Debug("ProxyNode CreateQueryChannel success")
Params.SearchChannelNames = []string{resp.RequestChannel}
Params.SearchResultChannelNames = []string{resp.ResultChannel}
Params.RetrieveChannelNames = []string{resp.RequestChannel}
Params.RetrieveResultChannelNames = []string{resp.ResultChannel}
log.Debug("ProxyNode CreateQueryChannel success", zap.Any("SearchChannelNames", Params.SearchChannelNames))
log.Debug("ProxyNode CreateQueryChannel success", zap.Any("SearchResultChannelNames", Params.SearchResultChannelNames))
log.Debug("ProxyNode CreateQueryChannel success", zap.Any("RetrieveChannelNames", Params.RetrieveChannelNames))
log.Debug("ProxyNode CreateQueryChannel success", zap.Any("RetrieveResultChannelNames", Params.RetrieveResultChannelNames))
}
// todo
//Params.InsertChannelNames, err = node.dataService.GetInsertChannels()
//if err != nil {
// return err
//}
m := map[string]interface{}{
"PulsarAddress": Params.PulsarAddress,
"PulsarBufSize": 1024}
err = node.msFactory.SetParams(m)
if err != nil {
return err
}
node.queryMsgStream, _ = node.msFactory.NewQueryMsgStream(node.ctx)
node.queryMsgStream.AsProducer(Params.SearchChannelNames)
// FIXME(wxyu): use log.Debug instead
log.Debug("proxynode", zap.Strings("proxynode AsProducer:", Params.SearchChannelNames))
log.Debug("create query message stream ...")
idAllocator, err := allocator.NewIDAllocator(node.ctx, Params.MetaRootPath, []string{Params.EtcdAddress})
if err != nil {
return err
}
node.idAllocator = idAllocator
node.idAllocator.PeerID = Params.ProxyID
tsoAllocator, err := NewTimestampAllocator(node.masterService, Params.ProxyID)
if err != nil {
return err
}
node.tsoAllocator = tsoAllocator
segAssigner, err := NewSegIDAssigner(node.ctx, node.dataService, node.lastTick)
if err != nil {
panic(err)
}
node.segAssigner = segAssigner
node.segAssigner.PeerID = Params.ProxyID
getDmlChannelsFunc := func(collectionID UniqueID) (map[vChan]pChan, error) {
req := &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
MsgID: 0, // todo
Timestamp: 0, // todo
SourceID: 0, // todo
},
DbName: "", // todo
CollectionName: "", // todo
CollectionID: collectionID,
TimeStamp: 0, // todo
}
resp, err := node.masterService.DescribeCollection(node.ctx, req)
if err != nil {
log.Warn("DescribeCollection", zap.Error(err))
return nil, err
}
if resp.Status.ErrorCode != 0 {
log.Warn("DescribeCollection",
zap.Any("ErrorCode", resp.Status.ErrorCode),
zap.Any("Reason", resp.Status.Reason))
return nil, err
}
if len(resp.VirtualChannelNames) != len(resp.PhysicalChannelNames) {
err := fmt.Errorf(
"len(VirtualChannelNames): %v, len(PhysicalChannelNames): %v",
len(resp.VirtualChannelNames),
len(resp.PhysicalChannelNames))
log.Warn("GetDmlChannels", zap.Error(err))
return nil, err
}
ret := make(map[vChan]pChan)
for idx, name := range resp.VirtualChannelNames {
if _, ok := ret[name]; ok {
err := fmt.Errorf(
"duplicated virtual channel found, vchan: %v, pchan: %v",
name,
resp.PhysicalChannelNames[idx])
return nil, err
}
ret[name] = resp.PhysicalChannelNames[idx]
}
return ret, nil
}
mockQueryService := newMockGetChannelsService()
chMgr := newChannelsMgr(getDmlChannelsFunc, mockQueryService.GetChannels, node.msFactory)
node.chMgr = chMgr
node.sched, err = NewTaskScheduler(node.ctx, node.idAllocator, node.tsoAllocator, node.msFactory)
if err != nil {
return err
}
node.tick = newTimeTick(node.ctx, node.tsoAllocator, time.Millisecond*200, node.sched.TaskDoneTest, node.msFactory)
// TODO(dragondriver): read this from config
interval := time.Millisecond * 200
node.chTicker = newChannelsTimeTicker(node.ctx, interval, []string{}, node.sched.getPChanStatistics, tsoAllocator)
return nil
}
func (node *ProxyNode) sendChannelsTimeTickLoop() {
node.wg.Add(1)
go func() {
defer node.wg.Done()
// TODO(dragondriver): read this from config
interval := time.Millisecond * 200
timer := time.NewTicker(interval)
for {
select {
case <-node.ctx.Done():
return
case <-timer.C:
stats, err := node.chTicker.getMinTsStatistics()
if err != nil {
log.Warn("sendChannelsTimeTickLoop.getMinTsStatistics", zap.Error(err))
continue
}
log.Debug("send timestamp statistics of pchan", zap.Any("statistics", stats))
channels := make([]pChan, 0, len(stats))
tss := make([]Timestamp, 0, len(stats))
for channel, ts := range stats {
channels = append(channels, channel)
tss = append(tss, ts)
}
req := &internalpb.ChannelTimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick, // todo
MsgID: 0, // todo
Timestamp: 0, // todo
SourceID: node.session.ServerID,
},
ChannelNames: channels,
Timestamps: tss,
}
status, err := node.masterService.UpdateChannelTimeTick(node.ctx, req)
if err != nil {
log.Warn("sendChannelsTimeTickLoop.UpdateChannelTimeTick", zap.Error(err))
continue
}
if status.ErrorCode != 0 {
log.Warn("sendChannelsTimeTickLoop.UpdateChannelTimeTick",
zap.Any("ErrorCode", status.ErrorCode),
zap.Any("Reason", status.Reason))
continue
}
}
}
}()
}
func (node *ProxyNode) Start() error {
err := InitMetaCache(node.masterService)
if err != nil {
return err
}
log.Debug("init global meta cache ...")
initGlobalInsertChannelsMap(node)
log.Debug("init global insert channels map ...")
node.queryMsgStream.Start()
log.Debug("start query message stream ...")
node.sched.Start()
log.Debug("start scheduler ...")
node.idAllocator.Start()
log.Debug("start id allocator ...")
node.segAssigner.Start()
log.Debug("start seg assigner ...")
node.tick.Start()
log.Debug("start time tick ...")
err = node.chTicker.start()
if err != nil {
return err
}
log.Debug("start channelsTimeTicker")
node.sendChannelsTimeTickLoop()
// Start callbacks
for _, cb := range node.startCallbacks {
cb()
}
node.UpdateStateCode(internalpb.StateCode_Healthy)
log.Debug("ProxyNode", zap.Any("State", node.stateCode.Load()))
return nil
}
func (node *ProxyNode) Stop() error {
node.cancel()
globalInsertChannelsMap.CloseAllMsgStream()
node.idAllocator.Close()
node.segAssigner.Close()
node.sched.Close()
node.queryMsgStream.Close()
node.tick.Close()
err := node.chTicker.close()
if err != nil {
return err
}
node.wg.Wait()
for _, cb := range node.closeCallbacks {
cb()
}
return nil
}
// AddStartCallback adds a callback in the startServer phase.
func (node *ProxyNode) AddStartCallback(callbacks ...func()) {
node.startCallbacks = append(node.startCallbacks, callbacks...)
}
func (node *ProxyNode) lastTick() Timestamp {
return node.tick.LastTick()
}
// AddCloseCallback adds a callback in the Close phase.
func (node *ProxyNode) AddCloseCallback(callbacks ...func()) {
node.closeCallbacks = append(node.closeCallbacks, callbacks...)
}
func (node *ProxyNode) SetMasterClient(cli types.MasterService) {
node.masterService = cli
}
func (node *ProxyNode) SetIndexServiceClient(cli types.IndexService) {
node.indexService = cli
}
func (node *ProxyNode) SetDataServiceClient(cli types.DataService) {
node.dataService = cli
}
func (node *ProxyNode) SetProxyServiceClient(cli types.ProxyService) {
node.proxyService = cli
}
func (node *ProxyNode) SetQueryServiceClient(cli types.QueryService) {
node.queryService = cli
}