2021-04-19 05:47:10 +00:00
|
|
|
// 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.
|
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
package querycoord
|
2021-01-15 07:28:54 +00:00
|
|
|
|
2021-01-16 07:31:10 +00:00
|
|
|
import (
|
2021-01-22 06:28:06 +00:00
|
|
|
"context"
|
2021-10-11 11:00:46 +00:00
|
|
|
"errors"
|
2021-11-12 10:49:10 +00:00
|
|
|
"math"
|
|
|
|
"sort"
|
|
|
|
|
2021-10-24 14:39:09 +00:00
|
|
|
"fmt"
|
2021-03-08 07:25:55 +00:00
|
|
|
"math/rand"
|
2021-06-03 06:58:34 +00:00
|
|
|
"strconv"
|
2021-06-19 03:45:09 +00:00
|
|
|
"sync"
|
2021-01-22 06:28:06 +00:00
|
|
|
"sync/atomic"
|
2021-02-20 01:20:51 +00:00
|
|
|
"time"
|
2021-01-22 06:28:06 +00:00
|
|
|
|
2021-10-24 14:39:09 +00:00
|
|
|
"github.com/golang/protobuf/proto"
|
|
|
|
"go.etcd.io/etcd/api/v3/mvccpb"
|
2021-06-04 08:29:35 +00:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2021-10-22 11:07:15 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/allocator"
|
2021-06-15 04:41:40 +00:00
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
2021-04-22 06:45:57 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/msgstream"
|
2021-06-19 03:45:09 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
2021-11-12 08:49:10 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
2021-04-22 06:45:57 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-11-12 08:49:10 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
2021-06-19 03:45:09 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
2021-04-22 06:45:57 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2021-09-15 12:40:07 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
2021-06-19 03:45:09 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/util/retry"
|
2021-05-21 11:28:52 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
2021-10-22 11:07:15 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
2021-04-22 06:45:57 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
2021-01-16 07:31:10 +00:00
|
|
|
)
|
2021-01-15 07:28:54 +00:00
|
|
|
|
2021-10-24 14:39:09 +00:00
|
|
|
const (
|
|
|
|
handoffSegmentPrefix = "querycoord-handoff"
|
|
|
|
)
|
|
|
|
|
2021-10-04 09:14:07 +00:00
|
|
|
// Timestamp is an alias for the Int64 type
|
2021-04-15 07:15:46 +00:00
|
|
|
type Timestamp = typeutil.Timestamp
|
|
|
|
|
2021-02-18 08:26:02 +00:00
|
|
|
type queryChannelInfo struct {
|
|
|
|
requestChannel string
|
|
|
|
responseChannel string
|
|
|
|
}
|
|
|
|
|
2021-10-05 07:34:26 +00:00
|
|
|
// QueryCoord is the coordinator of queryNodes
|
2021-06-22 08:44:09 +00:00
|
|
|
type QueryCoord struct {
|
2021-01-22 06:28:06 +00:00
|
|
|
loopCtx context.Context
|
|
|
|
loopCancel context.CancelFunc
|
2021-06-19 03:45:09 +00:00
|
|
|
loopWg sync.WaitGroup
|
|
|
|
kvClient *etcdkv.EtcdKV
|
2021-01-22 06:28:06 +00:00
|
|
|
|
2021-09-23 13:56:02 +00:00
|
|
|
initOnce sync.Once
|
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
queryCoordID uint64
|
2021-08-02 14:39:25 +00:00
|
|
|
meta Meta
|
2021-09-15 12:40:07 +00:00
|
|
|
cluster Cluster
|
2021-08-26 06:17:54 +00:00
|
|
|
newNodeFn newQueryNodeFn
|
2021-06-22 08:44:09 +00:00
|
|
|
scheduler *TaskScheduler
|
2021-10-22 11:07:15 +00:00
|
|
|
idAllocator func() (UniqueID, error)
|
2021-01-22 06:28:06 +00:00
|
|
|
|
2021-09-03 09:15:26 +00:00
|
|
|
metricsCacheManager *metricsinfo.MetricsCacheManager
|
|
|
|
|
2021-11-12 08:49:10 +00:00
|
|
|
dataCoordClient types.DataCoord
|
|
|
|
rootCoordClient types.RootCoord
|
|
|
|
indexCoordClient types.IndexCoord
|
2021-01-22 06:28:06 +00:00
|
|
|
|
2021-06-19 03:45:09 +00:00
|
|
|
session *sessionutil.Session
|
|
|
|
eventChan <-chan *sessionutil.SessionEvent
|
2021-05-21 11:28:52 +00:00
|
|
|
|
2021-01-26 07:13:20 +00:00
|
|
|
stateCode atomic.Value
|
|
|
|
enableGrpc bool
|
2021-02-08 06:30:54 +00:00
|
|
|
|
|
|
|
msFactory msgstream.Factory
|
2021-01-22 06:28:06 +00:00
|
|
|
}
|
|
|
|
|
2021-05-25 07:06:05 +00:00
|
|
|
// Register register query service at etcd
|
2021-06-22 08:44:09 +00:00
|
|
|
func (qc *QueryCoord) Register() error {
|
2021-07-13 06:16:00 +00:00
|
|
|
log.Debug("query coord session info", zap.String("metaPath", Params.MetaRootPath), zap.Strings("etcdEndPoints", Params.EtcdEndpoints), zap.String("address", Params.Address))
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.session = sessionutil.NewSession(qc.loopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
2021-10-14 08:40:35 +00:00
|
|
|
qc.session.Init(typeutil.QueryCoordRole, Params.Address, true)
|
2021-06-22 08:44:09 +00:00
|
|
|
Params.NodeID = uint64(qc.session.ServerID)
|
2021-10-01 00:52:50 +00:00
|
|
|
Params.SetLogger(typeutil.UniqueID(-1))
|
2021-05-25 07:06:05 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-05 07:36:17 +00:00
|
|
|
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
|
2021-06-22 08:44:09 +00:00
|
|
|
func (qc *QueryCoord) Init() error {
|
2021-10-11 01:54:37 +00:00
|
|
|
log.Debug("query coordinator start init")
|
|
|
|
//connect etcd
|
2021-06-19 03:45:09 +00:00
|
|
|
connectEtcdFn := func() error {
|
2021-08-13 03:04:09 +00:00
|
|
|
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
2021-06-19 03:45:09 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.kvClient = etcdKV
|
2021-08-03 14:03:25 +00:00
|
|
|
return nil
|
2021-06-19 03:45:09 +00:00
|
|
|
}
|
2021-09-23 13:56:02 +00:00
|
|
|
var initError error = nil
|
|
|
|
qc.initOnce.Do(func() {
|
|
|
|
log.Debug("query coordinator try to connect etcd")
|
|
|
|
initError = retry.Do(qc.loopCtx, connectEtcdFn, retry.Attempts(300))
|
|
|
|
if initError != nil {
|
|
|
|
log.Debug("query coordinator try to connect etcd failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
log.Debug("query coordinator try to connect etcd success")
|
2021-10-22 11:07:15 +00:00
|
|
|
|
|
|
|
// init id allocator
|
|
|
|
var idAllocatorKV *etcdkv.EtcdKV
|
|
|
|
idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryCoordTaskID")
|
|
|
|
if initError != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
|
|
|
initError = idAllocator.Initialize()
|
|
|
|
if initError != nil {
|
|
|
|
log.Debug("query coordinator idAllocator initialize failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
qc.idAllocator = func() (UniqueID, error) {
|
|
|
|
return idAllocator.AllocOne()
|
|
|
|
}
|
|
|
|
|
|
|
|
// init meta
|
|
|
|
qc.meta, initError = newMeta(qc.loopCtx, qc.kvClient, qc.msFactory, qc.idAllocator)
|
2021-09-23 13:56:02 +00:00
|
|
|
if initError != nil {
|
|
|
|
log.Error("query coordinator init meta failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
2021-08-03 14:03:25 +00:00
|
|
|
|
2021-10-22 11:07:15 +00:00
|
|
|
// init cluster
|
2021-09-23 13:56:02 +00:00
|
|
|
qc.cluster, initError = newQueryNodeCluster(qc.loopCtx, qc.meta, qc.kvClient, qc.newNodeFn, qc.session)
|
|
|
|
if initError != nil {
|
|
|
|
log.Error("query coordinator init cluster failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
2021-08-03 14:03:25 +00:00
|
|
|
|
2021-10-22 11:07:15 +00:00
|
|
|
// init task scheduler
|
|
|
|
qc.scheduler, initError = NewTaskScheduler(qc.loopCtx, qc.meta, qc.cluster, qc.kvClient, qc.rootCoordClient, qc.dataCoordClient, qc.idAllocator)
|
2021-09-23 13:56:02 +00:00
|
|
|
if initError != nil {
|
|
|
|
log.Error("query coordinator init task scheduler failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
2021-08-03 14:03:25 +00:00
|
|
|
|
2021-09-23 13:56:02 +00:00
|
|
|
qc.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
|
|
|
|
})
|
2021-09-03 09:15:26 +00:00
|
|
|
|
2021-09-23 13:56:02 +00:00
|
|
|
return initError
|
2021-01-22 06:28:06 +00:00
|
|
|
}
|
|
|
|
|
2021-10-05 07:38:03 +00:00
|
|
|
// Start function starts the goroutines to watch the meta and node updates
|
2021-06-22 08:44:09 +00:00
|
|
|
func (qc *QueryCoord) Start() error {
|
2021-10-22 11:07:15 +00:00
|
|
|
m := map[string]interface{}{
|
|
|
|
"PulsarAddress": Params.PulsarAddress,
|
|
|
|
"ReceiveBufSize": 1024,
|
|
|
|
"PulsarBufSize": 1024}
|
|
|
|
err := qc.msFactory.SetParams(m)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.scheduler.Start()
|
2021-04-15 07:15:46 +00:00
|
|
|
log.Debug("start scheduler ...")
|
2021-10-12 11:39:24 +00:00
|
|
|
|
|
|
|
Params.CreatedTime = time.Now()
|
|
|
|
Params.UpdatedTime = time.Now()
|
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.UpdateStateCode(internalpb.StateCode_Healthy)
|
2021-06-19 03:45:09 +00:00
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.loopWg.Add(1)
|
|
|
|
go qc.watchNodeLoop()
|
2021-06-19 03:45:09 +00:00
|
|
|
|
2021-10-24 14:39:09 +00:00
|
|
|
qc.loopWg.Add(1)
|
|
|
|
go qc.watchHandoffSegmentLoop()
|
|
|
|
|
2021-11-12 10:49:10 +00:00
|
|
|
qc.loopWg.Add(1)
|
|
|
|
go qc.loadBalanceSegmentLoop()
|
|
|
|
|
2021-10-14 08:40:35 +00:00
|
|
|
go qc.session.LivenessCheck(qc.loopCtx, func() {
|
2021-10-30 02:24:38 +00:00
|
|
|
log.Error("Query Coord disconnected from etcd, process will exit", zap.Int64("Server Id", qc.session.ServerID))
|
|
|
|
if err := qc.Stop(); err != nil {
|
|
|
|
log.Fatal("failed to stop server", zap.Error(err))
|
|
|
|
}
|
2021-09-23 10:29:55 +00:00
|
|
|
})
|
|
|
|
|
2021-01-22 06:28:06 +00:00
|
|
|
return nil
|
2021-01-16 07:31:10 +00:00
|
|
|
}
|
|
|
|
|
2021-10-06 06:00:47 +00:00
|
|
|
// Stop function stops watching the meta and node updates
|
2021-06-22 08:44:09 +00:00
|
|
|
func (qc *QueryCoord) Stop() error {
|
|
|
|
qc.scheduler.Close()
|
2021-04-15 07:15:46 +00:00
|
|
|
log.Debug("close scheduler ...")
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.loopCancel()
|
|
|
|
qc.UpdateStateCode(internalpb.StateCode_Abnormal)
|
2021-06-19 03:45:09 +00:00
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.loopWg.Wait()
|
2021-01-22 06:28:06 +00:00
|
|
|
return nil
|
2021-01-16 07:31:10 +00:00
|
|
|
}
|
|
|
|
|
2021-10-06 06:02:34 +00:00
|
|
|
// UpdateStateCode updates the status of the coord, including healthy, unhealthy
|
2021-06-22 08:44:09 +00:00
|
|
|
func (qc *QueryCoord) UpdateStateCode(code internalpb.StateCode) {
|
|
|
|
qc.stateCode.Store(code)
|
2021-02-23 03:40:30 +00:00
|
|
|
}
|
|
|
|
|
2021-10-09 12:53:10 +00:00
|
|
|
// NewQueryCoord creates a QueryCoord object.
|
2021-06-22 08:44:09 +00:00
|
|
|
func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord, error) {
|
2021-03-08 07:25:55 +00:00
|
|
|
rand.Seed(time.Now().UnixNano())
|
2021-02-18 08:26:02 +00:00
|
|
|
queryChannels := make([]*queryChannelInfo, 0)
|
2021-06-03 06:58:34 +00:00
|
|
|
channelID := len(queryChannels)
|
|
|
|
searchPrefix := Params.SearchChannelPrefix
|
|
|
|
searchResultPrefix := Params.SearchResultChannelPrefix
|
|
|
|
allocatedQueryChannel := searchPrefix + "-" + strconv.FormatInt(int64(channelID), 10)
|
|
|
|
allocatedQueryResultChannel := searchResultPrefix + "-" + strconv.FormatInt(int64(channelID), 10)
|
|
|
|
|
|
|
|
queryChannels = append(queryChannels, &queryChannelInfo{
|
|
|
|
requestChannel: allocatedQueryChannel,
|
|
|
|
responseChannel: allocatedQueryResultChannel,
|
|
|
|
})
|
|
|
|
|
2021-01-22 06:28:06 +00:00
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
2021-06-22 08:44:09 +00:00
|
|
|
service := &QueryCoord{
|
2021-06-15 04:41:40 +00:00
|
|
|
loopCtx: ctx1,
|
|
|
|
loopCancel: cancel,
|
|
|
|
msFactory: factory,
|
2021-08-26 06:17:54 +00:00
|
|
|
newNodeFn: newQueryNode,
|
2021-01-22 06:28:06 +00:00
|
|
|
}
|
2021-02-24 01:48:17 +00:00
|
|
|
|
2021-03-12 06:22:09 +00:00
|
|
|
service.UpdateStateCode(internalpb.StateCode_Abnormal)
|
2021-06-22 08:44:09 +00:00
|
|
|
log.Debug("query coordinator", zap.Any("queryChannels", queryChannels))
|
2021-01-22 06:28:06 +00:00
|
|
|
return service, nil
|
2021-01-15 07:28:54 +00:00
|
|
|
}
|
2021-01-26 07:13:20 +00:00
|
|
|
|
2021-10-06 06:04:25 +00:00
|
|
|
// SetRootCoord sets root coordinator's client
|
2021-10-11 11:00:46 +00:00
|
|
|
func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error {
|
|
|
|
if rootCoord == nil {
|
|
|
|
return errors.New("null root coordinator interface")
|
|
|
|
}
|
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.rootCoordClient = rootCoord
|
2021-10-11 11:00:46 +00:00
|
|
|
return nil
|
2021-01-26 07:13:20 +00:00
|
|
|
}
|
|
|
|
|
2021-10-07 11:52:46 +00:00
|
|
|
// SetDataCoord sets data coordinator's client
|
2021-10-11 11:00:46 +00:00
|
|
|
func (qc *QueryCoord) SetDataCoord(dataCoord types.DataCoord) error {
|
|
|
|
if dataCoord == nil {
|
|
|
|
return errors.New("null data coordinator interface")
|
|
|
|
}
|
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
qc.dataCoordClient = dataCoord
|
2021-10-11 11:00:46 +00:00
|
|
|
return nil
|
2021-01-26 07:13:20 +00:00
|
|
|
}
|
2021-06-19 03:45:09 +00:00
|
|
|
|
2021-11-12 08:49:10 +00:00
|
|
|
func (qc *QueryCoord) SetIndexCoord(indexCoord types.IndexCoord) error {
|
|
|
|
if indexCoord == nil {
|
|
|
|
return errors.New("null index coordinator interface")
|
|
|
|
}
|
|
|
|
|
|
|
|
qc.indexCoordClient = indexCoord
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-06-22 08:44:09 +00:00
|
|
|
func (qc *QueryCoord) watchNodeLoop() {
|
|
|
|
ctx, cancel := context.WithCancel(qc.loopCtx)
|
2021-06-19 03:45:09 +00:00
|
|
|
defer cancel()
|
2021-06-22 08:44:09 +00:00
|
|
|
defer qc.loopWg.Done()
|
|
|
|
log.Debug("query coordinator start watch node loop")
|
2021-06-19 03:45:09 +00:00
|
|
|
|
2021-09-15 12:40:07 +00:00
|
|
|
offlineNodes, err := qc.cluster.offlineNodes()
|
|
|
|
if err == nil {
|
|
|
|
offlineNodeIDs := make([]int64, 0)
|
|
|
|
for id := range offlineNodes {
|
|
|
|
offlineNodeIDs = append(offlineNodeIDs, id)
|
|
|
|
}
|
|
|
|
loadBalanceSegment := &querypb.LoadBalanceRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_LoadBalanceSegments,
|
|
|
|
SourceID: qc.session.ServerID,
|
|
|
|
},
|
|
|
|
SourceNodeIDs: offlineNodeIDs,
|
2021-06-19 03:45:09 +00:00
|
|
|
}
|
|
|
|
|
2021-10-11 01:54:37 +00:00
|
|
|
baseTask := newBaseTask(qc.loopCtx, querypb.TriggerCondition_nodeDown)
|
2021-10-18 13:34:47 +00:00
|
|
|
loadBalanceTask := &loadBalanceTask{
|
|
|
|
baseTask: baseTask,
|
2021-09-15 12:40:07 +00:00
|
|
|
LoadBalanceRequest: loadBalanceSegment,
|
|
|
|
rootCoord: qc.rootCoordClient,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
2021-06-19 03:45:09 +00:00
|
|
|
}
|
2021-10-11 01:54:37 +00:00
|
|
|
//TODO::deal enqueue error
|
|
|
|
qc.scheduler.Enqueue(loadBalanceTask)
|
2021-09-15 12:40:07 +00:00
|
|
|
log.Debug("start a loadBalance task", zap.Any("task", loadBalanceTask))
|
2021-06-19 03:45:09 +00:00
|
|
|
}
|
|
|
|
|
2021-09-15 12:40:07 +00:00
|
|
|
qc.eventChan = qc.session.WatchServices(typeutil.QueryNodeRole, qc.cluster.getSessionVersion()+1)
|
2021-06-19 03:45:09 +00:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
2021-10-14 11:20:35 +00:00
|
|
|
case event, ok := <-qc.eventChan:
|
|
|
|
if !ok {
|
|
|
|
return
|
|
|
|
}
|
2021-06-19 03:45:09 +00:00
|
|
|
switch event.EventType {
|
|
|
|
case sessionutil.SessionAddEvent:
|
|
|
|
serverID := event.Session.ServerID
|
2021-07-13 06:16:00 +00:00
|
|
|
log.Debug("start add a queryNode to cluster", zap.Any("nodeID", serverID))
|
2021-09-15 12:40:07 +00:00
|
|
|
err := qc.cluster.registerNode(ctx, event.Session, serverID, disConnect)
|
2021-07-13 06:16:00 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Error("query node failed to register", zap.Int64("nodeID", serverID), zap.String("error info", err.Error()))
|
|
|
|
}
|
2021-09-03 09:15:26 +00:00
|
|
|
qc.metricsCacheManager.InvalidateSystemInfoMetrics()
|
2021-06-19 03:45:09 +00:00
|
|
|
case sessionutil.SessionDelEvent:
|
|
|
|
serverID := event.Session.ServerID
|
2021-07-13 06:16:00 +00:00
|
|
|
log.Debug("get a del event after queryNode down", zap.Int64("nodeID", serverID))
|
2021-11-05 08:00:55 +00:00
|
|
|
nodeExist := qc.cluster.hasNode(serverID)
|
|
|
|
if !nodeExist {
|
2021-07-13 06:16:00 +00:00
|
|
|
log.Error("queryNode not exist", zap.Int64("nodeID", serverID))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
qc.cluster.stopNode(serverID)
|
|
|
|
loadBalanceSegment := &querypb.LoadBalanceRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_LoadBalanceSegments,
|
|
|
|
SourceID: qc.session.ServerID,
|
|
|
|
},
|
|
|
|
SourceNodeIDs: []int64{serverID},
|
|
|
|
BalanceReason: querypb.TriggerCondition_nodeDown,
|
|
|
|
}
|
|
|
|
|
2021-10-11 01:54:37 +00:00
|
|
|
baseTask := newBaseTask(qc.loopCtx, querypb.TriggerCondition_nodeDown)
|
2021-10-18 13:34:47 +00:00
|
|
|
loadBalanceTask := &loadBalanceTask{
|
|
|
|
baseTask: baseTask,
|
2021-07-13 06:16:00 +00:00
|
|
|
LoadBalanceRequest: loadBalanceSegment,
|
|
|
|
rootCoord: qc.rootCoordClient,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
2021-06-27 04:16:09 +00:00
|
|
|
}
|
2021-09-03 09:15:26 +00:00
|
|
|
qc.metricsCacheManager.InvalidateSystemInfoMetrics()
|
2021-10-11 01:54:37 +00:00
|
|
|
//TODO:: deal enqueue error
|
|
|
|
qc.scheduler.Enqueue(loadBalanceTask)
|
|
|
|
log.Debug("start a loadBalance task", zap.Any("task", loadBalanceTask))
|
2021-06-19 03:45:09 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-10-24 14:39:09 +00:00
|
|
|
|
|
|
|
func (qc *QueryCoord) watchHandoffSegmentLoop() {
|
|
|
|
ctx, cancel := context.WithCancel(qc.loopCtx)
|
|
|
|
|
|
|
|
defer cancel()
|
|
|
|
defer qc.loopWg.Done()
|
|
|
|
log.Debug("query coordinator start watch segment loop")
|
|
|
|
|
|
|
|
// TODO:: recover handoff task when coord down
|
|
|
|
watchChan := qc.kvClient.WatchWithPrefix(handoffSegmentPrefix)
|
2021-11-12 08:49:10 +00:00
|
|
|
unIndexedSegmentChan := make(chan *querypb.SegmentInfo, 1024)
|
|
|
|
indexSegmentChan := make(chan *querypb.SegmentInfo, 1024)
|
|
|
|
|
|
|
|
go qc.checkIndexLoop(ctx, unIndexedSegmentChan, indexSegmentChan)
|
|
|
|
go qc.processHandoffAfterIndexDone(ctx, indexSegmentChan)
|
2021-10-24 14:39:09 +00:00
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case resp := <-watchChan:
|
|
|
|
for _, event := range resp.Events {
|
|
|
|
segmentInfo := &querypb.SegmentInfo{}
|
|
|
|
err := proto.Unmarshal(event.Kv.Value, segmentInfo)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: unmarshal failed", zap.Any("error", err.Error()))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
switch event.Type {
|
|
|
|
case mvccpb.PUT:
|
2021-11-12 08:49:10 +00:00
|
|
|
processDone := true
|
|
|
|
// if collection has not been loaded, then skip the segment
|
|
|
|
collectionInfo, err := qc.meta.getCollectionInfoByID(segmentInfo.CollectionID)
|
|
|
|
if err != nil {
|
|
|
|
log.Debug("watchHandoffSegmentLoop: collection has not been loaded into memory", zap.Int64("collectionID", segmentInfo.CollectionID))
|
|
|
|
} else {
|
|
|
|
// if partition has not been loaded or released, then skip handoff the segment
|
|
|
|
if collectionInfo.LoadType == querypb.LoadType_LoadPartition {
|
|
|
|
for _, id := range collectionInfo.PartitionIDs {
|
|
|
|
if id == segmentInfo.PartitionID {
|
|
|
|
unIndexedSegmentChan <- segmentInfo
|
|
|
|
processDone = false
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
partitionReleased := false
|
|
|
|
for _, id := range collectionInfo.ReleasedPartitionIDs {
|
|
|
|
if id == segmentInfo.PartitionID {
|
|
|
|
partitionReleased = true
|
|
|
|
}
|
2021-10-24 14:39:09 +00:00
|
|
|
}
|
2021-11-12 08:49:10 +00:00
|
|
|
if !partitionReleased {
|
|
|
|
unIndexedSegmentChan <- segmentInfo
|
|
|
|
processDone = false
|
|
|
|
}
|
|
|
|
}
|
2021-10-24 14:39:09 +00:00
|
|
|
}
|
|
|
|
|
2021-11-12 08:49:10 +00:00
|
|
|
if processDone {
|
|
|
|
buildQuerySegmentPath := fmt.Sprintf("%s/%d/%d/%d", handoffSegmentPrefix, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.SegmentID)
|
|
|
|
err = qc.kvClient.Remove(buildQuerySegmentPath)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: remove handoff segment from etcd failed", zap.Error(err))
|
|
|
|
}
|
2021-10-24 14:39:09 +00:00
|
|
|
}
|
|
|
|
default:
|
|
|
|
// do nothing
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-11-12 08:49:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (qc *QueryCoord) checkIndexLoop(ctx context.Context, unIndexedChan chan *querypb.SegmentInfo, indexedChan chan *querypb.SegmentInfo) {
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case segmentInfo := <-unIndexedChan:
|
|
|
|
processDone := true
|
|
|
|
// TODO:: check whether the index exists in parallel, in case indexCoord cannot create the index normally, and then block the loop
|
|
|
|
for {
|
|
|
|
// if the collection has been released, then skip handoff the segment
|
|
|
|
collectionInfo, err := qc.meta.getCollectionInfoByID(segmentInfo.CollectionID)
|
|
|
|
if err != nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
// if the partition has been released, then skip handoff the segment
|
|
|
|
partitionReleased := false
|
|
|
|
for _, id := range collectionInfo.ReleasedPartitionIDs {
|
|
|
|
if id == segmentInfo.PartitionID {
|
|
|
|
partitionReleased = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if partitionReleased {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
// check the buildID of the segment's index whether exist on rootCoord
|
|
|
|
req := &milvuspb.DescribeSegmentRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_DescribeSegment,
|
|
|
|
},
|
|
|
|
CollectionID: segmentInfo.CollectionID,
|
|
|
|
SegmentID: segmentInfo.SegmentID,
|
|
|
|
}
|
|
|
|
response, err := qc.rootCoordClient.DescribeSegment(ctx, req)
|
|
|
|
if err != nil || response.Status.ErrorCode != commonpb.ErrorCode_Success {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// if the segment.EnableIndex == false, then load the segment immediately
|
|
|
|
// only sealed segment can be balanced, so the handoff is needed
|
|
|
|
if !response.EnableIndex {
|
|
|
|
log.Debug("checkIndexLoop: segment's enableIndex equal to false, ready to handoff", zap.Int64("segmentID", segmentInfo.SegmentID))
|
|
|
|
indexedChan <- segmentInfo
|
|
|
|
processDone = false
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
indexFilePathRequest := &indexpb.GetIndexFilePathsRequest{
|
|
|
|
IndexBuildIDs: []UniqueID{response.BuildID},
|
|
|
|
}
|
|
|
|
// if index created done on indexNode, then handoff start
|
|
|
|
pathResponse, err := qc.indexCoordClient.GetIndexFilePaths(ctx, indexFilePathRequest)
|
|
|
|
if err != nil || pathResponse.Status.ErrorCode != commonpb.ErrorCode_Success {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("checkIndexLoop: create segment's index done, ready to handoff", zap.Int64("segmentID", segmentInfo.SegmentID))
|
|
|
|
indexedChan <- segmentInfo
|
|
|
|
processDone = false
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
if processDone {
|
|
|
|
buildQuerySegmentPath := fmt.Sprintf("%s/%d/%d/%d", handoffSegmentPrefix, segmentInfo.CollectionID, segmentInfo.PartitionID, segmentInfo.SegmentID)
|
|
|
|
err := qc.kvClient.Remove(buildQuerySegmentPath)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: remove handoff segment from etcd failed", zap.Error(err))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-10-24 14:39:09 +00:00
|
|
|
|
2021-11-12 08:49:10 +00:00
|
|
|
func (qc *QueryCoord) processHandoffAfterIndexDone(ctx context.Context, indexedChan chan *querypb.SegmentInfo) {
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case segmentInfo := <-indexedChan:
|
|
|
|
collectionID := segmentInfo.CollectionID
|
|
|
|
partitionID := segmentInfo.PartitionID
|
|
|
|
segmentID := segmentInfo.SegmentID
|
|
|
|
if Params.AutoHandoff {
|
|
|
|
log.Debug("processHandoffAfterIndexDone: handoff segment received",
|
|
|
|
zap.Int64("collectionID", collectionID),
|
|
|
|
zap.Int64("partitionID", partitionID),
|
|
|
|
zap.Int64("segmentID", segmentID),
|
|
|
|
zap.Any("segmentInfo", segmentInfo),
|
|
|
|
)
|
|
|
|
baseTask := newBaseTask(ctx, querypb.TriggerCondition_handoff)
|
|
|
|
handoffReq := &querypb.HandoffSegmentsRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_HandoffSegments,
|
|
|
|
},
|
|
|
|
SegmentInfos: []*querypb.SegmentInfo{segmentInfo},
|
|
|
|
}
|
|
|
|
handoffTask := &handoffTask{
|
|
|
|
baseTask: baseTask,
|
|
|
|
HandoffSegmentsRequest: handoffReq,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
|
|
|
}
|
|
|
|
err := qc.scheduler.Enqueue(handoffTask)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("processHandoffAfterIndexDone: handoffTask enqueue failed", zap.Error(err))
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
err := handoffTask.waitToFinish()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("processHandoffAfterIndexDone: handoffTask failed", zap.Error(err))
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
log.Debug("processHandoffAfterIndexDone: handoffTask completed",
|
|
|
|
zap.Any("collectionID", collectionID),
|
|
|
|
zap.Any("partitionID", partitionID),
|
|
|
|
zap.Any("segmentID", segmentID),
|
|
|
|
zap.Any("channel", segmentInfo.ChannelID),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
|
|
|
|
buildQuerySegmentPath := fmt.Sprintf("%s/%d/%d/%d", handoffSegmentPrefix, collectionID, partitionID, segmentID)
|
|
|
|
err := qc.kvClient.Remove(buildQuerySegmentPath)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("processHandoffAfterIndexDone: remove handoff segment from etcd failed", zap.Error(err))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-10-24 14:39:09 +00:00
|
|
|
}
|
2021-11-12 10:49:10 +00:00
|
|
|
|
|
|
|
func (qc *QueryCoord) loadBalanceSegmentLoop() {
|
|
|
|
ctx, cancel := context.WithCancel(qc.loopCtx)
|
|
|
|
defer cancel()
|
|
|
|
defer qc.loopWg.Done()
|
|
|
|
log.Debug("query coordinator start load balance segment loop")
|
|
|
|
|
|
|
|
timer := time.NewTicker(time.Duration(Params.BalanceIntervalSeconds) * time.Second)
|
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case <-timer.C:
|
|
|
|
onlineNodes, err := qc.cluster.onlineNodes()
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("loadBalanceSegmentLoop: there are no online query node to balance")
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
// get mem info of online nodes from cluster
|
|
|
|
nodeID2MemUsageRate := make(map[int64]float64)
|
|
|
|
nodeID2MemUsage := make(map[int64]uint64)
|
|
|
|
nodeID2TotalMem := make(map[int64]uint64)
|
|
|
|
nodeID2SegmentInfos := make(map[int64]map[UniqueID]*querypb.SegmentInfo)
|
|
|
|
onlineNodeIDs := make([]int64, 0)
|
|
|
|
for nodeID := range onlineNodes {
|
|
|
|
nodeInfo, err := qc.cluster.getNodeInfoByID(nodeID)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("loadBalanceSegmentLoop: get node info from query node failed", zap.Int64("nodeID", nodeID), zap.Error(err))
|
|
|
|
delete(onlineNodes, nodeID)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
updateSegmentInfoDone := true
|
|
|
|
leastSegmentInfos := make(map[UniqueID]*querypb.SegmentInfo)
|
|
|
|
segmentInfos := qc.meta.getSegmentInfosByNode(nodeID)
|
|
|
|
for _, segmentInfo := range segmentInfos {
|
|
|
|
leastInfo, err := qc.cluster.getSegmentInfoByID(ctx, segmentInfo.SegmentID)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("loadBalanceSegmentLoop: get segment info from query node failed", zap.Int64("nodeID", nodeID), zap.Error(err))
|
|
|
|
delete(onlineNodes, nodeID)
|
|
|
|
updateSegmentInfoDone = false
|
|
|
|
break
|
|
|
|
}
|
|
|
|
leastSegmentInfos[segmentInfo.SegmentID] = leastInfo
|
|
|
|
}
|
|
|
|
if updateSegmentInfoDone {
|
|
|
|
nodeID2MemUsageRate[nodeID] = nodeInfo.(*queryNode).memUsageRate
|
|
|
|
nodeID2MemUsage[nodeID] = nodeInfo.(*queryNode).memUsage
|
|
|
|
nodeID2TotalMem[nodeID] = nodeInfo.(*queryNode).totalMem
|
|
|
|
onlineNodeIDs = append(onlineNodeIDs, nodeID)
|
|
|
|
nodeID2SegmentInfos[nodeID] = leastSegmentInfos
|
|
|
|
}
|
|
|
|
}
|
|
|
|
log.Debug("loadBalanceSegmentLoop: memory usage rage of all online query node", zap.Any("mem rate", nodeID2MemUsageRate))
|
|
|
|
if len(onlineNodeIDs) <= 1 {
|
|
|
|
log.Warn("loadBalanceSegmentLoop: there are too few online query nodes to balance", zap.Int64s("onlineNodeIDs", onlineNodeIDs))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// check which nodes need balance and determine which segments on these nodes need to be migrated to other nodes
|
|
|
|
memoryInsufficient := false
|
|
|
|
loadBalanceTasks := make([]*loadBalanceTask, 0)
|
|
|
|
for {
|
|
|
|
var selectedSegmentInfo *querypb.SegmentInfo = nil
|
|
|
|
sort.Slice(onlineNodeIDs, func(i, j int) bool {
|
|
|
|
return nodeID2MemUsageRate[onlineNodeIDs[i]] > nodeID2MemUsageRate[onlineNodeIDs[j]]
|
|
|
|
})
|
|
|
|
|
|
|
|
// the memoryUsageRate of the sourceNode is higher than other query node
|
|
|
|
sourceNodeID := onlineNodeIDs[0]
|
|
|
|
dstNodeID := onlineNodeIDs[len(onlineNodeIDs)-1]
|
|
|
|
memUsageRateDiff := nodeID2MemUsageRate[sourceNodeID] - nodeID2MemUsageRate[dstNodeID]
|
|
|
|
// if memoryUsageRate of source node is greater then 90%, and the max memUsageDiff is greater than 30%
|
|
|
|
// then migrate the segments on source node to other query nodes
|
|
|
|
if nodeID2MemUsageRate[sourceNodeID] > Params.OverloadedMemoryThresholdPercentage ||
|
|
|
|
memUsageRateDiff > Params.MemoryUsageMaxDifferencePercentage {
|
|
|
|
segmentInfos := nodeID2SegmentInfos[sourceNodeID]
|
|
|
|
// select the segment that needs balance on the source node
|
|
|
|
selectedSegmentInfo, err = chooseSegmentToBalance(sourceNodeID, dstNodeID, segmentInfos, nodeID2MemUsage, nodeID2TotalMem, nodeID2MemUsageRate)
|
|
|
|
if err == nil && selectedSegmentInfo != nil {
|
|
|
|
req := &querypb.LoadBalanceRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_LoadBalanceSegments,
|
|
|
|
},
|
|
|
|
BalanceReason: querypb.TriggerCondition_loadBalance,
|
|
|
|
SourceNodeIDs: []UniqueID{sourceNodeID},
|
|
|
|
DstNodeIDs: []UniqueID{dstNodeID},
|
|
|
|
SealedSegmentIDs: []UniqueID{selectedSegmentInfo.SegmentID},
|
|
|
|
}
|
|
|
|
baseTask := newBaseTask(qc.loopCtx, querypb.TriggerCondition_loadBalance)
|
|
|
|
balanceTask := &loadBalanceTask{
|
|
|
|
baseTask: baseTask,
|
|
|
|
LoadBalanceRequest: req,
|
|
|
|
rootCoord: qc.rootCoordClient,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
|
|
|
}
|
|
|
|
loadBalanceTasks = append(loadBalanceTasks, balanceTask)
|
|
|
|
nodeID2MemUsage[sourceNodeID] -= uint64(selectedSegmentInfo.MemSize)
|
|
|
|
nodeID2MemUsage[dstNodeID] += uint64(selectedSegmentInfo.MemSize)
|
|
|
|
nodeID2MemUsageRate[sourceNodeID] = float64(nodeID2MemUsage[sourceNodeID]) / float64(nodeID2TotalMem[sourceNodeID])
|
|
|
|
nodeID2MemUsageRate[dstNodeID] = float64(nodeID2MemUsage[dstNodeID]) / float64(nodeID2TotalMem[dstNodeID])
|
|
|
|
delete(nodeID2SegmentInfos[sourceNodeID], selectedSegmentInfo.SegmentID)
|
|
|
|
nodeID2SegmentInfos[dstNodeID][selectedSegmentInfo.SegmentID] = selectedSegmentInfo
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
// no enough memory on query nodes to balance, then notify proxy to stop insert
|
|
|
|
memoryInsufficient = true
|
|
|
|
}
|
|
|
|
// if memoryInsufficient == false
|
|
|
|
// all query node's memoryUsageRate is less than 90%, and the max memUsageDiff is less than 30%
|
|
|
|
// this balance loop is done
|
|
|
|
break
|
|
|
|
}
|
|
|
|
if !memoryInsufficient {
|
|
|
|
for _, t := range loadBalanceTasks {
|
|
|
|
qc.scheduler.Enqueue(t)
|
|
|
|
log.Debug("loadBalanceSegmentLoop: enqueue a loadBalance task", zap.Any("task", t))
|
|
|
|
err = t.waitToFinish()
|
|
|
|
if err != nil {
|
|
|
|
// if failed, wait for next balance loop
|
|
|
|
// it may be that the collection/partition of the balanced segment has been released
|
|
|
|
// it also may be other abnormal errors
|
|
|
|
log.Error("loadBalanceSegmentLoop: balance task execute failed", zap.Any("task", t))
|
|
|
|
} else {
|
|
|
|
log.Debug("loadBalanceSegmentLoop: balance task execute success", zap.Any("task", t))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
log.Debug("loadBalanceSegmentLoop: load balance Done in this loop", zap.Any("tasks", loadBalanceTasks))
|
|
|
|
} else {
|
|
|
|
// no enough memory on query nodes to balance, then notify proxy to stop insert
|
|
|
|
//TODO:: xige-16
|
|
|
|
log.Error("loadBalanceSegmentLoop: query node has insufficient memory, stop inserting data")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func chooseSegmentToBalance(sourceNodeID int64, dstNodeID int64,
|
|
|
|
segmentInfos map[UniqueID]*querypb.SegmentInfo,
|
|
|
|
nodeID2MemUsage map[int64]uint64,
|
|
|
|
nodeID2TotalMem map[int64]uint64,
|
|
|
|
nodeID2MemUsageRate map[int64]float64) (*querypb.SegmentInfo, error) {
|
|
|
|
memoryInsufficient := true
|
|
|
|
minMemDiffPercentage := 1.0
|
|
|
|
var selectedSegmentInfo *querypb.SegmentInfo = nil
|
|
|
|
for _, info := range segmentInfos {
|
|
|
|
dstNodeMemUsageAfterBalance := nodeID2MemUsage[dstNodeID] + uint64(info.MemSize)
|
|
|
|
dstNodeMemUsageRateAfterBalance := float64(dstNodeMemUsageAfterBalance) / float64(nodeID2TotalMem[dstNodeID])
|
|
|
|
// if memUsageRate of dstNode is greater than OverloadedMemoryThresholdPercentage after balance, than can't balance
|
|
|
|
if dstNodeMemUsageRateAfterBalance < Params.OverloadedMemoryThresholdPercentage {
|
|
|
|
memoryInsufficient = false
|
|
|
|
sourceNodeMemUsageAfterBalance := nodeID2MemUsage[sourceNodeID] - uint64(info.MemSize)
|
|
|
|
sourceNodeMemUsageRateAfterBalance := float64(sourceNodeMemUsageAfterBalance) / float64(nodeID2TotalMem[sourceNodeID])
|
|
|
|
// assume all query node has same memory capacity
|
|
|
|
// if the memUsageRateDiff between the two nodes does not become smaller after balance, there is no need for balance
|
|
|
|
diffBeforBalance := nodeID2MemUsageRate[sourceNodeID] - nodeID2MemUsageRate[dstNodeID]
|
|
|
|
diffAfterBalance := dstNodeMemUsageRateAfterBalance - sourceNodeMemUsageRateAfterBalance
|
|
|
|
if diffAfterBalance < diffBeforBalance {
|
|
|
|
if math.Abs(diffAfterBalance) < minMemDiffPercentage {
|
|
|
|
selectedSegmentInfo = info
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if memoryInsufficient {
|
|
|
|
return nil, errors.New("all query nodes has insufficient memory")
|
|
|
|
}
|
|
|
|
|
|
|
|
return selectedSegmentInfo, nil
|
|
|
|
}
|