2021-11-17 19:47:36 +08:00
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
2021-04-19 13:47:10 +08:00
// with the License. You may obtain a copy of the License at
//
2021-11-17 19:47:36 +08:00
// http://www.apache.org/licenses/LICENSE-2.0
2021-04-19 13:47:10 +08:00
//
2021-11-17 19:47:36 +08:00
// 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-04-19 13:47:10 +08:00
2021-06-22 16:44:09 +08:00
package querycoord
2021-01-15 15:28:54 +08:00
2021-01-16 15:31:10 +08:00
import (
2021-01-22 14:28:06 +08:00
"context"
2021-10-11 19:00:46 +08:00
"errors"
2021-11-12 18:49:10 +08:00
"math"
"sort"
2021-11-22 16:23:17 +08:00
"syscall"
2021-11-12 18:49:10 +08:00
2021-10-24 22:39:09 +08:00
"fmt"
2021-03-08 15:25:55 +08:00
"math/rand"
2021-06-03 14:58:34 +08:00
"strconv"
2021-06-19 11:45:09 +08:00
"sync"
2021-01-22 14:28:06 +08:00
"sync/atomic"
2021-02-20 09:20:51 +08:00
"time"
2021-01-22 14:28:06 +08:00
2021-10-24 22:39:09 +08:00
"github.com/golang/protobuf/proto"
"go.etcd.io/etcd/api/v3/mvccpb"
2021-06-04 16:29:35 +08:00
"go.uber.org/zap"
2021-10-22 19:07:15 +08:00
"github.com/milvus-io/milvus/internal/allocator"
2021-06-15 12:41:40 +08:00
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
2021-06-19 11:45:09 +08:00
"github.com/milvus-io/milvus/internal/proto/commonpb"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/proto/internalpb"
2021-06-19 11:45:09 +08:00
"github.com/milvus-io/milvus/internal/proto/querypb"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/types"
2021-09-15 20:40:07 +08:00
"github.com/milvus-io/milvus/internal/util/metricsinfo"
2021-06-19 11:45:09 +08:00
"github.com/milvus-io/milvus/internal/util/retry"
2021-05-21 19:28:52 +08:00
"github.com/milvus-io/milvus/internal/util/sessionutil"
2021-10-22 19:07:15 +08:00
"github.com/milvus-io/milvus/internal/util/tsoutil"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/util/typeutil"
2021-01-16 15:31:10 +08:00
)
2021-01-15 15:28:54 +08:00
2021-10-24 22:39:09 +08:00
const (
handoffSegmentPrefix = "querycoord-handoff"
)
2021-10-04 17:14:07 +08:00
// Timestamp is an alias for the Int64 type
2021-04-15 15:15:46 +08:00
type Timestamp = typeutil . Timestamp
2021-02-18 16:26:02 +08:00
type queryChannelInfo struct {
requestChannel string
responseChannel string
}
2021-10-05 15:34:26 +08:00
// QueryCoord is the coordinator of queryNodes
2021-06-22 16:44:09 +08:00
type QueryCoord struct {
2021-01-22 14:28:06 +08:00
loopCtx context . Context
loopCancel context . CancelFunc
2021-06-19 11:45:09 +08:00
loopWg sync . WaitGroup
kvClient * etcdkv . EtcdKV
2021-01-22 14:28:06 +08:00
2021-09-23 21:56:02 +08:00
initOnce sync . Once
2021-06-22 16:44:09 +08:00
queryCoordID uint64
2021-08-02 22:39:25 +08:00
meta Meta
2021-09-15 20:40:07 +08:00
cluster Cluster
2021-08-26 14:17:54 +08:00
newNodeFn newQueryNodeFn
2021-06-22 16:44:09 +08:00
scheduler * TaskScheduler
2021-10-22 19:07:15 +08:00
idAllocator func ( ) ( UniqueID , error )
2021-11-17 09:47:12 +08:00
indexChecker * IndexChecker
2021-01-22 14:28:06 +08:00
2021-09-03 17:15:26 +08:00
metricsCacheManager * metricsinfo . MetricsCacheManager
2021-11-12 16:49:10 +08:00
dataCoordClient types . DataCoord
rootCoordClient types . RootCoord
indexCoordClient types . IndexCoord
2021-01-22 14:28:06 +08:00
2021-06-19 11:45:09 +08:00
session * sessionutil . Session
eventChan <- chan * sessionutil . SessionEvent
2021-05-21 19:28:52 +08:00
2021-01-26 15:13:20 +08:00
stateCode atomic . Value
enableGrpc bool
2021-02-08 14:30:54 +08:00
msFactory msgstream . Factory
2021-01-22 14:28:06 +08:00
}
2021-05-25 15:06:05 +08:00
// Register register query service at etcd
2021-06-22 16:44:09 +08:00
func ( qc * QueryCoord ) Register ( ) error {
2021-07-13 14:16:00 +08: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 16:44:09 +08:00
qc . session = sessionutil . NewSession ( qc . loopCtx , Params . MetaRootPath , Params . EtcdEndpoints )
2021-10-14 16:40:35 +08:00
qc . session . Init ( typeutil . QueryCoordRole , Params . Address , true )
2021-06-22 16:44:09 +08:00
Params . NodeID = uint64 ( qc . session . ServerID )
2021-10-01 08:52:50 +08:00
Params . SetLogger ( typeutil . UniqueID ( - 1 ) )
2021-05-25 15:06:05 +08:00
return nil
}
2021-10-05 15:36:17 +08:00
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
2021-06-22 16:44:09 +08:00
func ( qc * QueryCoord ) Init ( ) error {
2021-10-11 09:54:37 +08:00
log . Debug ( "query coordinator start init" )
//connect etcd
2021-06-19 11:45:09 +08:00
connectEtcdFn := func ( ) error {
2021-08-13 11:04:09 +08:00
etcdKV , err := etcdkv . NewEtcdKV ( Params . EtcdEndpoints , Params . MetaRootPath )
2021-06-19 11:45:09 +08:00
if err != nil {
return err
}
2021-06-22 16:44:09 +08:00
qc . kvClient = etcdKV
2021-08-03 22:03:25 +08:00
return nil
2021-06-19 11:45:09 +08:00
}
2021-09-23 21:56:02 +08: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 19:07:15 +08: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 21:56:02 +08:00
if initError != nil {
log . Error ( "query coordinator init meta failed" , zap . Error ( initError ) )
return
}
2021-08-03 22:03:25 +08:00
2021-10-22 19:07:15 +08:00
// init cluster
2021-09-23 21:56:02 +08: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 22:03:25 +08:00
2021-10-22 19:07:15 +08:00
// init task scheduler
2021-11-17 09:47:12 +08:00
qc . scheduler , initError = NewTaskScheduler ( qc . loopCtx , qc . meta , qc . cluster , qc . kvClient , qc . rootCoordClient , qc . dataCoordClient , qc . indexCoordClient , qc . idAllocator )
2021-09-23 21:56:02 +08:00
if initError != nil {
log . Error ( "query coordinator init task scheduler failed" , zap . Error ( initError ) )
return
}
2021-08-03 22:03:25 +08:00
2021-11-17 09:47:12 +08:00
// init index checker
qc . indexChecker , initError = newIndexChecker ( qc . loopCtx , qc . kvClient , qc . meta , qc . cluster , qc . scheduler , qc . rootCoordClient , qc . indexCoordClient , qc . dataCoordClient )
if initError != nil {
log . Error ( "query coordinator init index checker failed" , zap . Error ( initError ) )
return
}
2021-09-23 21:56:02 +08:00
qc . metricsCacheManager = metricsinfo . NewMetricsCacheManager ( )
} )
2021-11-12 21:25:08 +08:00
log . Debug ( "query coordinator init success" )
2021-09-23 21:56:02 +08:00
return initError
2021-01-22 14:28:06 +08:00
}
2021-10-05 15:38:03 +08:00
// Start function starts the goroutines to watch the meta and node updates
2021-06-22 16:44:09 +08:00
func ( qc * QueryCoord ) Start ( ) error {
2021-10-22 19:07:15 +08: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 16:44:09 +08:00
qc . scheduler . Start ( )
2021-04-15 15:15:46 +08:00
log . Debug ( "start scheduler ..." )
2021-10-12 19:39:24 +08:00
2021-11-17 09:47:12 +08:00
qc . indexChecker . start ( )
log . Debug ( "start index checker ..." )
2021-10-12 19:39:24 +08:00
Params . CreatedTime = time . Now ( )
Params . UpdatedTime = time . Now ( )
2021-06-22 16:44:09 +08:00
qc . UpdateStateCode ( internalpb . StateCode_Healthy )
2021-06-19 11:45:09 +08:00
2021-06-22 16:44:09 +08:00
qc . loopWg . Add ( 1 )
go qc . watchNodeLoop ( )
2021-06-19 11:45:09 +08:00
2021-10-24 22:39:09 +08:00
qc . loopWg . Add ( 1 )
go qc . watchHandoffSegmentLoop ( )
2021-12-06 10:19:41 +08:00
if Params . AutoBalance {
qc . loopWg . Add ( 1 )
go qc . loadBalanceSegmentLoop ( )
}
2021-11-12 18:49:10 +08:00
2021-10-14 16:40:35 +08:00
go qc . session . LivenessCheck ( qc . loopCtx , func ( ) {
2021-10-30 10:24:38 +08: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-11-22 16:23:17 +08:00
// manually send signal to starter goroutine
syscall . Kill ( syscall . Getpid ( ) , syscall . SIGINT )
2021-09-23 18:29:55 +08:00
} )
2021-01-22 14:28:06 +08:00
return nil
2021-01-16 15:31:10 +08:00
}
2021-10-06 14:00:47 +08:00
// Stop function stops watching the meta and node updates
2021-06-22 16:44:09 +08:00
func ( qc * QueryCoord ) Stop ( ) error {
qc . scheduler . Close ( )
2021-04-15 15:15:46 +08:00
log . Debug ( "close scheduler ..." )
2021-11-17 09:47:12 +08:00
qc . indexChecker . close ( )
log . Debug ( "close index checker ..." )
2021-06-22 16:44:09 +08:00
qc . loopCancel ( )
qc . UpdateStateCode ( internalpb . StateCode_Abnormal )
2021-06-19 11:45:09 +08:00
2021-06-22 16:44:09 +08:00
qc . loopWg . Wait ( )
2021-11-16 22:31:14 +08:00
qc . session . Revoke ( time . Second )
2021-01-22 14:28:06 +08:00
return nil
2021-01-16 15:31:10 +08:00
}
2021-10-06 14:02:34 +08:00
// UpdateStateCode updates the status of the coord, including healthy, unhealthy
2021-06-22 16:44:09 +08:00
func ( qc * QueryCoord ) UpdateStateCode ( code internalpb . StateCode ) {
qc . stateCode . Store ( code )
2021-02-23 11:40:30 +08:00
}
2021-10-09 20:53:10 +08:00
// NewQueryCoord creates a QueryCoord object.
2021-06-22 16:44:09 +08:00
func NewQueryCoord ( ctx context . Context , factory msgstream . Factory ) ( * QueryCoord , error ) {
2021-03-08 15:25:55 +08:00
rand . Seed ( time . Now ( ) . UnixNano ( ) )
2021-02-18 16:26:02 +08:00
queryChannels := make ( [ ] * queryChannelInfo , 0 )
2021-06-03 14:58:34 +08: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 14:28:06 +08:00
ctx1 , cancel := context . WithCancel ( ctx )
2021-06-22 16:44:09 +08:00
service := & QueryCoord {
2021-06-15 12:41:40 +08:00
loopCtx : ctx1 ,
loopCancel : cancel ,
msFactory : factory ,
2021-08-26 14:17:54 +08:00
newNodeFn : newQueryNode ,
2021-01-22 14:28:06 +08:00
}
2021-02-24 09:48:17 +08:00
2021-03-12 14:22:09 +08:00
service . UpdateStateCode ( internalpb . StateCode_Abnormal )
2021-06-22 16:44:09 +08:00
log . Debug ( "query coordinator" , zap . Any ( "queryChannels" , queryChannels ) )
2021-01-22 14:28:06 +08:00
return service , nil
2021-01-15 15:28:54 +08:00
}
2021-01-26 15:13:20 +08:00
2021-10-06 14:04:25 +08:00
// SetRootCoord sets root coordinator's client
2021-10-11 19:00:46 +08:00
func ( qc * QueryCoord ) SetRootCoord ( rootCoord types . RootCoord ) error {
if rootCoord == nil {
return errors . New ( "null root coordinator interface" )
}
2021-06-22 16:44:09 +08:00
qc . rootCoordClient = rootCoord
2021-10-11 19:00:46 +08:00
return nil
2021-01-26 15:13:20 +08:00
}
2021-10-07 19:52:46 +08:00
// SetDataCoord sets data coordinator's client
2021-10-11 19:00:46 +08:00
func ( qc * QueryCoord ) SetDataCoord ( dataCoord types . DataCoord ) error {
if dataCoord == nil {
return errors . New ( "null data coordinator interface" )
}
2021-06-22 16:44:09 +08:00
qc . dataCoordClient = dataCoord
2021-10-11 19:00:46 +08:00
return nil
2021-01-26 15:13:20 +08:00
}
2021-06-19 11:45:09 +08:00
2021-11-12 16:49:10 +08: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 16:44:09 +08:00
func ( qc * QueryCoord ) watchNodeLoop ( ) {
ctx , cancel := context . WithCancel ( qc . loopCtx )
2021-06-19 11:45:09 +08:00
defer cancel ( )
2021-06-22 16:44:09 +08:00
defer qc . loopWg . Done ( )
log . Debug ( "query coordinator start watch node loop" )
2021-06-19 11:45:09 +08:00
2021-09-15 20:40:07 +08: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 11:45:09 +08:00
}
2021-10-11 09:54:37 +08:00
baseTask := newBaseTask ( qc . loopCtx , querypb . TriggerCondition_nodeDown )
2021-10-18 21:34:47 +08:00
loadBalanceTask := & loadBalanceTask {
baseTask : baseTask ,
2021-09-15 20:40:07 +08:00
LoadBalanceRequest : loadBalanceSegment ,
rootCoord : qc . rootCoordClient ,
dataCoord : qc . dataCoordClient ,
2021-11-17 09:47:12 +08:00
indexCoord : qc . indexCoordClient ,
2021-09-15 20:40:07 +08:00
cluster : qc . cluster ,
meta : qc . meta ,
2021-06-19 11:45:09 +08:00
}
2021-10-11 09:54:37 +08:00
//TODO::deal enqueue error
qc . scheduler . Enqueue ( loadBalanceTask )
2021-09-15 20:40:07 +08:00
log . Debug ( "start a loadBalance task" , zap . Any ( "task" , loadBalanceTask ) )
2021-06-19 11:45:09 +08:00
}
2021-09-15 20:40:07 +08:00
qc . eventChan = qc . session . WatchServices ( typeutil . QueryNodeRole , qc . cluster . getSessionVersion ( ) + 1 )
2021-06-19 11:45:09 +08:00
for {
select {
case <- ctx . Done ( ) :
return
2021-10-14 19:20:35 +08:00
case event , ok := <- qc . eventChan :
if ! ok {
return
}
2021-06-19 11:45:09 +08:00
switch event . EventType {
case sessionutil . SessionAddEvent :
serverID := event . Session . ServerID
2021-07-13 14:16:00 +08:00
log . Debug ( "start add a queryNode to cluster" , zap . Any ( "nodeID" , serverID ) )
2021-09-15 20:40:07 +08:00
err := qc . cluster . registerNode ( ctx , event . Session , serverID , disConnect )
2021-07-13 14:16:00 +08:00
if err != nil {
log . Error ( "query node failed to register" , zap . Int64 ( "nodeID" , serverID ) , zap . String ( "error info" , err . Error ( ) ) )
}
2021-09-03 17:15:26 +08:00
qc . metricsCacheManager . InvalidateSystemInfoMetrics ( )
2021-06-19 11:45:09 +08:00
case sessionutil . SessionDelEvent :
serverID := event . Session . ServerID
2021-07-13 14:16:00 +08:00
log . Debug ( "get a del event after queryNode down" , zap . Int64 ( "nodeID" , serverID ) )
2021-11-05 16:00:55 +08:00
nodeExist := qc . cluster . hasNode ( serverID )
if ! nodeExist {
2021-07-13 14:16:00 +08: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 09:54:37 +08:00
baseTask := newBaseTask ( qc . loopCtx , querypb . TriggerCondition_nodeDown )
2021-10-18 21:34:47 +08:00
loadBalanceTask := & loadBalanceTask {
baseTask : baseTask ,
2021-07-13 14:16:00 +08:00
LoadBalanceRequest : loadBalanceSegment ,
rootCoord : qc . rootCoordClient ,
dataCoord : qc . dataCoordClient ,
2021-11-17 09:47:12 +08:00
indexCoord : qc . indexCoordClient ,
2021-07-13 14:16:00 +08:00
cluster : qc . cluster ,
meta : qc . meta ,
2021-06-27 12:16:09 +08:00
}
2021-09-03 17:15:26 +08:00
qc . metricsCacheManager . InvalidateSystemInfoMetrics ( )
2021-10-11 09:54:37 +08:00
//TODO:: deal enqueue error
qc . scheduler . Enqueue ( loadBalanceTask )
log . Debug ( "start a loadBalance task" , zap . Any ( "task" , loadBalanceTask ) )
2021-06-19 11:45:09 +08:00
}
}
}
}
2021-10-24 22:39:09 +08: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" )
2021-11-17 09:47:12 +08:00
watchChan := qc . kvClient . WatchWithRevision ( handoffSegmentPrefix , qc . indexChecker . revision + 1 )
2021-10-24 22:39:09 +08: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-17 09:47:12 +08:00
if Params . AutoHandoff && qc . indexChecker . verifyHandoffReqValid ( segmentInfo ) {
qc . indexChecker . enqueueHandoffReq ( segmentInfo )
log . Debug ( "watchHandoffSegmentLoop: enqueue a handoff request to index checker" , zap . Any ( "segment info" , segmentInfo ) )
2021-11-12 16:49:10 +08:00
} else {
2021-11-17 09:47:12 +08:00
log . Debug ( "watchHandoffSegmentLoop: collection/partition has not been loaded or autoHandoff equal to false, remove req from etcd" , zap . Any ( "segmentInfo" , segmentInfo ) )
2021-11-12 16:49:10 +08:00
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-11-17 09:47:12 +08:00
panic ( err )
2021-11-12 16:49:10 +08:00
}
2021-10-24 22:39:09 +08:00
}
default :
// do nothing
}
}
}
}
2021-11-12 16:49:10 +08:00
}
2021-11-12 18:49:10 +08: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
}
}
2021-12-06 10:07:35 +08:00
log . Debug ( "loadBalanceSegmentLoop: memory usage rate of all online query node" , zap . Any ( "mem rate" , nodeID2MemUsageRate ) )
2021-11-12 18:49:10 +08:00
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 ,
2021-11-17 09:47:12 +08:00
indexCoord : qc . indexCoordClient ,
2021-11-12 18:49:10 +08:00
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
}