Add node ctx mananger logic for IndexCoord (#13862)

Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
This commit is contained in:
zhenshan.cao 2021-12-23 21:35:52 +08:00 committed by GitHub
parent f9e6709839
commit 3b7dad4486
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 72 additions and 35 deletions

View File

@ -178,7 +178,7 @@ func (i *IndexCoord) Init() error {
return return
} }
log.Debug("IndexCoord try to connect etcd success") log.Debug("IndexCoord try to connect etcd success")
i.nodeManager = NewNodeManager() i.nodeManager = NewNodeManager(i.loopCtx)
sessions, revision, err := i.session.GetSessions(typeutil.IndexNodeRole) sessions, revision, err := i.session.GetSessions(typeutil.IndexNodeRole)
log.Debug("IndexCoord", zap.Int("session number", len(sessions)), zap.Int64("revision", revision)) log.Debug("IndexCoord", zap.Int("session number", len(sessions)), zap.Int64("revision", revision))

View File

@ -19,10 +19,10 @@ package indexcoord
import ( import (
"context" "context"
"sync" "sync"
"time"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
grpcindexnodeclient "github.com/milvus-io/milvus/internal/distributed/indexnode/client" grpcindexnodeclient "github.com/milvus-io/milvus/internal/distributed/indexnode/client"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
@ -34,26 +34,24 @@ import (
type NodeManager struct { type NodeManager struct {
nodeClients map[UniqueID]types.IndexNode nodeClients map[UniqueID]types.IndexNode
pq *PriorityQueue pq *PriorityQueue
lock sync.RWMutex
lock sync.RWMutex ctx context.Context
} }
// NewNodeManager is used to create a new NodeManager. // NewNodeManager is used to create a new NodeManager.
func NewNodeManager() *NodeManager { func NewNodeManager(ctx context.Context) *NodeManager {
return &NodeManager{ return &NodeManager{
nodeClients: make(map[UniqueID]types.IndexNode), nodeClients: make(map[UniqueID]types.IndexNode),
pq: &PriorityQueue{ pq: &PriorityQueue{
policy: PeekClientV1, policy: PeekClientV1,
}, },
lock: sync.RWMutex{}, lock: sync.RWMutex{},
ctx: ctx,
} }
} }
// setClient sets IndexNode client to node manager. // setClient sets IndexNode client to node manager.
func (nm *NodeManager) setClient(nodeID UniqueID, client types.IndexNode) error { func (nm *NodeManager) setClient(nodeID UniqueID, client types.IndexNode) error {
nm.lock.Lock()
defer nm.lock.Unlock()
log.Debug("IndexCoord NodeManager setClient", zap.Int64("nodeID", nodeID)) log.Debug("IndexCoord NodeManager setClient", zap.Int64("nodeID", nodeID))
defer log.Debug("IndexNode NodeManager setclient success", zap.Any("nodeID", nodeID)) defer log.Debug("IndexNode NodeManager setclient success", zap.Any("nodeID", nodeID))
item := &PQItem{ item := &PQItem{
@ -62,18 +60,19 @@ func (nm *NodeManager) setClient(nodeID UniqueID, client types.IndexNode) error
weight: 0, weight: 0,
totalMem: 0, totalMem: 0,
} }
nm.lock.Lock()
nm.nodeClients[nodeID] = client nm.nodeClients[nodeID] = client
nm.lock.Unlock()
nm.pq.Push(item) nm.pq.Push(item)
return nil return nil
} }
// RemoveNode removes the unused client of IndexNode. // RemoveNode removes the unused client of IndexNode.
func (nm *NodeManager) RemoveNode(nodeID UniqueID) { func (nm *NodeManager) RemoveNode(nodeID UniqueID) {
nm.lock.Lock()
defer nm.lock.Unlock()
log.Debug("IndexCoord", zap.Any("Remove node with ID", nodeID)) log.Debug("IndexCoord", zap.Any("Remove node with ID", nodeID))
nm.lock.Lock()
delete(nm.nodeClients, nodeID) delete(nm.nodeClients, nodeID)
nm.lock.Unlock()
nm.pq.Remove(nodeID) nm.pq.Remove(nodeID)
} }
@ -100,9 +99,6 @@ func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error {
// PeekClient peeks the client with the least load. // PeekClient peeks the client with the least load.
func (nm *NodeManager) PeekClient(meta Meta) (UniqueID, types.IndexNode) { func (nm *NodeManager) PeekClient(meta Meta) (UniqueID, types.IndexNode) {
nm.lock.Lock()
defer nm.lock.Unlock()
log.Debug("IndexCoord NodeManager PeekClient") log.Debug("IndexCoord NodeManager PeekClient")
dim, err := getDimension(meta.indexMeta.Req) dim, err := getDimension(meta.indexMeta.Req)
@ -120,6 +116,8 @@ func (nm *NodeManager) PeekClient(meta Meta) (UniqueID, types.IndexNode) {
log.Error("No IndexNode available", zap.Uint64("data size", dataSize), log.Error("No IndexNode available", zap.Uint64("data size", dataSize),
zap.Uint64("IndexNode must have memory size", dataSize*indexSizeFactor)) zap.Uint64("IndexNode must have memory size", dataSize*indexSizeFactor))
} }
nm.lock.Lock()
defer nm.lock.Unlock()
client, ok := nm.nodeClients[nodeID] client, ok := nm.nodeClients[nodeID]
if !ok { if !ok {
log.Error("IndexCoord NodeManager PeekClient", zap.Int64("There is no IndexNode client corresponding to NodeID", nodeID)) log.Error("IndexCoord NodeManager PeekClient", zap.Int64("There is no IndexNode client corresponding to NodeID", nodeID))
@ -131,34 +129,57 @@ func (nm *NodeManager) PeekClient(meta Meta) (UniqueID, types.IndexNode) {
// ListNode list all IndexNodes in node manager. // ListNode list all IndexNodes in node manager.
func (nm *NodeManager) ListNode() []UniqueID { func (nm *NodeManager) ListNode() []UniqueID {
nm.lock.Lock() //nm.lock.Lock()
defer nm.lock.Unlock() //defer nm.lock.Unlock()
clients := []UniqueID{} var clientIDs []UniqueID
nm.lock.RLock()
for id := range nm.nodeClients { for id := range nm.nodeClients {
clients = append(clients, id) clientIDs = append(clientIDs, id)
if item := (nm.pq.getItemByKey(id)).(*PQItem); item.totalMem == 0 { }
nm.lock.RUnlock()
var wg sync.WaitGroup
for _, id := range clientIDs {
memory := nm.pq.GetMemory(id)
if memory == 0 {
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
if err != nil { if err != nil {
log.Error("create metrics request failed", zap.Error(err)) log.Error("create metrics request failed", zap.Error(err))
continue continue
} }
metrics, err := nm.nodeClients[id].GetMetrics(context.Background(), req)
if err != nil {
log.Error("get indexnode metrics failed", zap.Error(err))
continue
}
infos := &metricsinfo.IndexNodeInfos{} nm.lock.RLock()
err = metricsinfo.UnmarshalComponentInfos(metrics.Response, infos) client, ok := nm.nodeClients[id]
if err != nil { if !ok {
log.Error("get indexnode metrics info failed", zap.Error(err)) nm.lock.RUnlock()
log.Debug("NodeManager ListNode find client not exist")
continue continue
} }
nm.pq.SetMemory(id, infos.HardwareInfos.Memory) nm.lock.RUnlock()
wg.Add(1)
go func(group *sync.WaitGroup, id UniqueID) {
defer group.Done()
ctx, cancel := context.WithTimeout(nm.ctx, time.Second*5)
defer cancel()
metrics, err := client.GetMetrics(ctx, req)
if err != nil {
log.Error("get IndexNode metrics failed", zap.Error(err))
return
}
infos := &metricsinfo.IndexNodeInfos{}
err = metricsinfo.UnmarshalComponentInfos(metrics.Response, infos)
if err != nil {
log.Error("get IndexNode metrics info failed", zap.Error(err))
return
}
nm.pq.SetMemory(id, infos.HardwareInfos.Memory)
}(&wg, id)
} }
} }
return clients wg.Wait()
return clientIDs
} }
// indexNodeGetMetricsResponse record the metrics information of IndexNode. // indexNodeGetMetricsResponse record the metrics information of IndexNode.
@ -169,17 +190,20 @@ type indexNodeGetMetricsResponse struct {
// getMetrics get metrics information of all IndexNode. // getMetrics get metrics information of all IndexNode.
func (nm *NodeManager) getMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) []indexNodeGetMetricsResponse { func (nm *NodeManager) getMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) []indexNodeGetMetricsResponse {
var clients []types.IndexNode
nm.lock.RLock() nm.lock.RLock()
defer nm.lock.RUnlock() for _, node := range nm.nodeClients {
clients = append(clients, node)
}
nm.lock.RUnlock()
ret := make([]indexNodeGetMetricsResponse, 0, len(nm.nodeClients)) ret := make([]indexNodeGetMetricsResponse, 0, len(nm.nodeClients))
for _, node := range nm.nodeClients { for _, node := range clients {
resp, err := node.GetMetrics(ctx, req) resp, err := node.GetMetrics(ctx, req)
ret = append(ret, indexNodeGetMetricsResponse{ ret = append(ret, indexNodeGetMetricsResponse{
resp: resp, resp: resp,
err: err, err: err,
}) })
} }
return ret return ret
} }

View File

@ -167,6 +167,19 @@ func (pq *PriorityQueue) PeekAll() []UniqueID {
return ret return ret
} }
// GetMemory get the memory info for the speicied key.
func (pq *PriorityQueue) GetMemory(key UniqueID) uint64 {
pq.lock.RLock()
defer pq.lock.RUnlock()
for i := range pq.items {
if pq.items[i].key == key {
return pq.items[i].totalMem
}
}
return 0
}
// SetMemory sets the memory info for IndexNode. // SetMemory sets the memory info for IndexNode.
func (pq *PriorityQueue) SetMemory(key UniqueID, memorySize uint64) { func (pq *PriorityQueue) SetMemory(key UniqueID, memorySize uint64) {
pq.lock.Lock() pq.lock.Lock()