mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-02 11:59:00 +08:00
3e862ecdfb
Signed-off-by: yudong.cai <yudong.cai@zilliz.com>
617 lines
20 KiB
Go
617 lines
20 KiB
Go
// 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
|
||
// 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 querynode
|
||
|
||
import (
|
||
"context"
|
||
"fmt"
|
||
|
||
"go.uber.org/zap"
|
||
|
||
"github.com/milvus-io/milvus/internal/common"
|
||
"github.com/milvus-io/milvus/internal/log"
|
||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||
queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
|
||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||
)
|
||
|
||
// GetComponentStates returns information about whether the node is healthy
|
||
func (node *QueryNode) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||
log.Debug("Get QueryNode component states")
|
||
stats := &internalpb.ComponentStates{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
},
|
||
}
|
||
code, ok := node.stateCode.Load().(internalpb.StateCode)
|
||
if !ok {
|
||
errMsg := "unexpected error in type assertion"
|
||
stats.Status = &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: errMsg,
|
||
}
|
||
return stats, nil
|
||
}
|
||
nodeID := common.NotRegisteredID
|
||
if node.session != nil && node.session.Registered() {
|
||
nodeID = node.session.ServerID
|
||
}
|
||
info := &internalpb.ComponentInfo{
|
||
NodeID: nodeID,
|
||
Role: typeutil.QueryNodeRole,
|
||
StateCode: code,
|
||
}
|
||
stats.State = info
|
||
log.Debug("Get QueryNode component state done", zap.Any("stateCode", info.StateCode))
|
||
return stats, nil
|
||
}
|
||
|
||
// GetTimeTickChannel returns the time tick channel
|
||
// TimeTickChannel contains many time tick messages, which will be sent by query nodes
|
||
func (node *QueryNode) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
|
||
return &milvuspb.StringResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
Reason: "",
|
||
},
|
||
Value: Params.CommonCfg.QueryCoordTimeTick,
|
||
}, nil
|
||
}
|
||
|
||
// GetStatisticsChannel returns the statistics channel
|
||
// Statistics channel contains statistics infos of query nodes, such as segment infos, memory infos
|
||
func (node *QueryNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
|
||
return &milvuspb.StringResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
Reason: "",
|
||
},
|
||
Value: Params.CommonCfg.QueryNodeStats,
|
||
}, nil
|
||
}
|
||
|
||
// AddQueryChannel watch queryChannel of the collection to receive query message
|
||
func (node *QueryNode) AddQueryChannel(ctx context.Context, in *queryPb.AddQueryChannelRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
dct := &addQueryChannelTask{
|
||
baseTask: baseTask{
|
||
ctx: ctx,
|
||
done: make(chan error),
|
||
},
|
||
req: in,
|
||
node: node,
|
||
}
|
||
|
||
err := node.scheduler.queue.Enqueue(dct)
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("addQueryChannelTask Enqueue done",
|
||
zap.Int64("collectionID", in.CollectionID),
|
||
zap.String("queryChannel", in.QueryChannel),
|
||
zap.String("queryResultChannel", in.QueryResultChannel),
|
||
)
|
||
|
||
waitFunc := func() (*commonpb.Status, error) {
|
||
err = dct.WaitToFinish()
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("addQueryChannelTask WaitToFinish done",
|
||
zap.Int64("collectionID", in.CollectionID),
|
||
zap.String("queryChannel", in.QueryChannel),
|
||
zap.String("queryResultChannel", in.QueryResultChannel),
|
||
)
|
||
|
||
return &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}, nil
|
||
}
|
||
|
||
return waitFunc()
|
||
}
|
||
|
||
// RemoveQueryChannel remove queryChannel of the collection to stop receiving query message
|
||
func (node *QueryNode) RemoveQueryChannel(ctx context.Context, in *queryPb.RemoveQueryChannelRequest) (*commonpb.Status, error) {
|
||
// if node.searchService == nil || node.searchService.searchMsgStream == nil {
|
||
// errMsg := "null search service or null search result message stream"
|
||
// status := &commonpb.Status{
|
||
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
// Reason: errMsg,
|
||
// }
|
||
|
||
// return status, errors.New(errMsg)
|
||
// }
|
||
|
||
// searchStream, ok := node.searchService.searchMsgStream.(*pulsarms.PulsarMsgStream)
|
||
// if !ok {
|
||
// errMsg := "type assertion failed for search message stream"
|
||
// status := &commonpb.Status{
|
||
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
// Reason: errMsg,
|
||
// }
|
||
|
||
// return status, errors.New(errMsg)
|
||
// }
|
||
|
||
// resultStream, ok := node.searchService.searchResultMsgStream.(*pulsarms.PulsarMsgStream)
|
||
// if !ok {
|
||
// errMsg := "type assertion failed for search result message stream"
|
||
// status := &commonpb.Status{
|
||
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
// Reason: errMsg,
|
||
// }
|
||
|
||
// return status, errors.New(errMsg)
|
||
// }
|
||
|
||
// // remove request channel
|
||
// consumeChannels := []string{in.RequestChannelID}
|
||
// consumeSubName := Params.MsgChannelSubName
|
||
// // TODO: searchStream.RemovePulsarConsumers(producerChannels)
|
||
// searchStream.AsConsumer(consumeChannels, consumeSubName)
|
||
|
||
// // remove result channel
|
||
// producerChannels := []string{in.ResultChannelID}
|
||
// // TODO: resultStream.RemovePulsarProducer(producerChannels)
|
||
// resultStream.AsProducer(producerChannels)
|
||
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}
|
||
return status, nil
|
||
}
|
||
|
||
// WatchDmChannels create consumers on dmChannels to receive Incremental data,which is the important part of real-time query
|
||
func (node *QueryNode) WatchDmChannels(ctx context.Context, in *queryPb.WatchDmChannelsRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
dct := &watchDmChannelsTask{
|
||
baseTask: baseTask{
|
||
ctx: ctx,
|
||
done: make(chan error),
|
||
},
|
||
req: in,
|
||
node: node,
|
||
}
|
||
|
||
err := node.scheduler.queue.Enqueue(dct)
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("watchDmChannelsTask Enqueue done", zap.Int64("collectionID", in.CollectionID), zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID))
|
||
|
||
waitFunc := func() (*commonpb.Status, error) {
|
||
err = dct.WaitToFinish()
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("watchDmChannelsTask WaitToFinish done", zap.Int64("collectionID", in.CollectionID), zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID))
|
||
return &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}, nil
|
||
}
|
||
|
||
return waitFunc()
|
||
}
|
||
|
||
// WatchDeltaChannels create consumers on dmChannels to receive Incremental data,which is the important part of real-time query
|
||
func (node *QueryNode) WatchDeltaChannels(ctx context.Context, in *queryPb.WatchDeltaChannelsRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
dct := &watchDeltaChannelsTask{
|
||
baseTask: baseTask{
|
||
ctx: ctx,
|
||
done: make(chan error),
|
||
},
|
||
req: in,
|
||
node: node,
|
||
}
|
||
|
||
err := node.scheduler.queue.Enqueue(dct)
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("watchDeltaChannelsTask Enqueue done", zap.Int64("collectionID", in.CollectionID), zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID))
|
||
|
||
waitFunc := func() (*commonpb.Status, error) {
|
||
err = dct.WaitToFinish()
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("watchDeltaChannelsTask WaitToFinish done", zap.Int64("collectionID", in.CollectionID), zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID))
|
||
return &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}, nil
|
||
}
|
||
|
||
return waitFunc()
|
||
}
|
||
|
||
// LoadSegments load historical data into query node, historical data can be vector data or index
|
||
func (node *QueryNode) LoadSegments(ctx context.Context, in *queryPb.LoadSegmentsRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
dct := &loadSegmentsTask{
|
||
baseTask: baseTask{
|
||
ctx: ctx,
|
||
done: make(chan error),
|
||
},
|
||
req: in,
|
||
node: node,
|
||
}
|
||
|
||
err := node.scheduler.queue.Enqueue(dct)
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
segmentIDs := make([]UniqueID, 0)
|
||
for _, info := range in.Infos {
|
||
segmentIDs = append(segmentIDs, info.SegmentID)
|
||
}
|
||
log.Debug("loadSegmentsTask Enqueue done", zap.Int64("collectionID", in.CollectionID), zap.Int64s("segmentIDs", segmentIDs), zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID))
|
||
|
||
waitFunc := func() (*commonpb.Status, error) {
|
||
err = dct.WaitToFinish()
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("loadSegmentsTask WaitToFinish done", zap.Int64("collectionID", in.CollectionID), zap.Int64s("segmentIDs", segmentIDs), zap.Int64("nodeID", Params.QueryNodeCfg.QueryNodeID))
|
||
return &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}, nil
|
||
}
|
||
|
||
return waitFunc()
|
||
}
|
||
|
||
// ReleaseCollection clears all data related to this collection on the querynode
|
||
func (node *QueryNode) ReleaseCollection(ctx context.Context, in *queryPb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
dct := &releaseCollectionTask{
|
||
baseTask: baseTask{
|
||
ctx: ctx,
|
||
done: make(chan error),
|
||
},
|
||
req: in,
|
||
node: node,
|
||
}
|
||
|
||
err := node.scheduler.queue.Enqueue(dct)
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("releaseCollectionTask Enqueue done", zap.Int64("collectionID", in.CollectionID))
|
||
|
||
func() {
|
||
err = dct.WaitToFinish()
|
||
if err != nil {
|
||
log.Error(err.Error())
|
||
return
|
||
}
|
||
log.Debug("releaseCollectionTask WaitToFinish done", zap.Int64("collectionID", in.CollectionID))
|
||
}()
|
||
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}
|
||
return status, nil
|
||
}
|
||
|
||
// ReleasePartitions clears all data related to this partition on the querynode
|
||
func (node *QueryNode) ReleasePartitions(ctx context.Context, in *queryPb.ReleasePartitionsRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
dct := &releasePartitionsTask{
|
||
baseTask: baseTask{
|
||
ctx: ctx,
|
||
done: make(chan error),
|
||
},
|
||
req: in,
|
||
node: node,
|
||
}
|
||
|
||
err := node.scheduler.queue.Enqueue(dct)
|
||
if err != nil {
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
log.Error(err.Error())
|
||
return status, nil
|
||
}
|
||
log.Debug("releasePartitionsTask Enqueue done", zap.Int64("collectionID", in.CollectionID), zap.Int64s("partitionIDs", in.PartitionIDs))
|
||
|
||
func() {
|
||
err = dct.WaitToFinish()
|
||
if err != nil {
|
||
log.Error(err.Error())
|
||
return
|
||
}
|
||
log.Debug("releasePartitionsTask WaitToFinish done", zap.Int64("collectionID", in.CollectionID), zap.Int64s("partitionIDs", in.PartitionIDs))
|
||
}()
|
||
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}
|
||
return status, nil
|
||
}
|
||
|
||
// ReleaseSegments remove the specified segments from query node according segmentIDs, partitionIDs, and collectionID
|
||
func (node *QueryNode) ReleaseSegments(ctx context.Context, in *queryPb.ReleaseSegmentsRequest) (*commonpb.Status, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
}
|
||
return status, nil
|
||
}
|
||
status := &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
}
|
||
for _, id := range in.SegmentIDs {
|
||
err := node.historical.replica.removeSegment(id)
|
||
if err != nil {
|
||
// not return, try to release all segments
|
||
status.ErrorCode = commonpb.ErrorCode_UnexpectedError
|
||
status.Reason = err.Error()
|
||
}
|
||
err = node.streaming.replica.removeSegment(id)
|
||
if err != nil {
|
||
// not return, try to release all segments
|
||
status.ErrorCode = commonpb.ErrorCode_UnexpectedError
|
||
status.Reason = err.Error()
|
||
}
|
||
}
|
||
|
||
log.Debug("release segments done", zap.Int64("collectionID", in.CollectionID), zap.Int64s("segmentIDs", in.SegmentIDs))
|
||
return status, nil
|
||
}
|
||
|
||
// GetSegmentInfo returns segment information of the collection on the queryNode, and the information includes memSize, numRow, indexName, indexID ...
|
||
func (node *QueryNode) GetSegmentInfo(ctx context.Context, in *queryPb.GetSegmentInfoRequest) (*queryPb.GetSegmentInfoResponse, error) {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
if code != internalpb.StateCode_Healthy {
|
||
err := fmt.Errorf("query node %d is not ready", Params.QueryNodeCfg.QueryNodeID)
|
||
res := &queryPb.GetSegmentInfoResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
},
|
||
}
|
||
return res, nil
|
||
}
|
||
var segmentInfos []*queryPb.SegmentInfo
|
||
|
||
segmentIDs := make(map[int64]struct{})
|
||
for _, segmentID := range in.GetSegmentIDs() {
|
||
segmentIDs[segmentID] = struct{}{}
|
||
}
|
||
|
||
// get info from historical
|
||
historicalSegmentInfos, err := node.historical.replica.getSegmentInfosByColID(in.CollectionID)
|
||
if err != nil {
|
||
log.Debug("GetSegmentInfo: get historical segmentInfo failed", zap.Int64("collectionID", in.CollectionID), zap.Error(err))
|
||
res := &queryPb.GetSegmentInfoResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
},
|
||
}
|
||
return res, nil
|
||
}
|
||
segmentInfos = append(segmentInfos, filterSegmentInfo(historicalSegmentInfos, segmentIDs)...)
|
||
|
||
// get info from streaming
|
||
streamingSegmentInfos, err := node.streaming.replica.getSegmentInfosByColID(in.CollectionID)
|
||
if err != nil {
|
||
log.Debug("GetSegmentInfo: get streaming segmentInfo failed", zap.Int64("collectionID", in.CollectionID), zap.Error(err))
|
||
res := &queryPb.GetSegmentInfoResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
},
|
||
}
|
||
return res, nil
|
||
}
|
||
segmentInfos = append(segmentInfos, filterSegmentInfo(streamingSegmentInfos, segmentIDs)...)
|
||
|
||
return &queryPb.GetSegmentInfoResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_Success,
|
||
},
|
||
Infos: segmentInfos,
|
||
}, nil
|
||
}
|
||
|
||
// filterSegmentInfo returns segment info which segment id in segmentIDs map
|
||
func filterSegmentInfo(segmentInfos []*queryPb.SegmentInfo, segmentIDs map[int64]struct{}) []*queryPb.SegmentInfo {
|
||
if len(segmentIDs) == 0 {
|
||
return segmentInfos
|
||
}
|
||
filtered := make([]*queryPb.SegmentInfo, 0, len(segmentIDs))
|
||
for _, info := range segmentInfos {
|
||
_, ok := segmentIDs[info.GetSegmentID()]
|
||
if !ok {
|
||
continue
|
||
}
|
||
filtered = append(filtered, info)
|
||
}
|
||
return filtered
|
||
}
|
||
|
||
// isHealthy checks if QueryNode is healthy
|
||
func (node *QueryNode) isHealthy() bool {
|
||
code := node.stateCode.Load().(internalpb.StateCode)
|
||
return code == internalpb.StateCode_Healthy
|
||
}
|
||
|
||
// GetMetrics return system infos of the query node, such as total memory, memory usage, cpu usage ...
|
||
// TODO(dragondriver): cache the Metrics and set a retention to the cache
|
||
func (node *QueryNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||
if !node.isHealthy() {
|
||
log.Warn("QueryNode.GetMetrics failed",
|
||
zap.Int64("node_id", Params.QueryNodeCfg.QueryNodeID),
|
||
zap.String("req", req.Request),
|
||
zap.Error(errQueryNodeIsUnhealthy(Params.QueryNodeCfg.QueryNodeID)))
|
||
|
||
return &milvuspb.GetMetricsResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: msgQueryNodeIsUnhealthy(Params.QueryNodeCfg.QueryNodeID),
|
||
},
|
||
Response: "",
|
||
}, nil
|
||
}
|
||
|
||
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
||
if err != nil {
|
||
log.Warn("QueryNode.GetMetrics failed to parse metric type",
|
||
zap.Int64("node_id", Params.QueryNodeCfg.QueryNodeID),
|
||
zap.String("req", req.Request),
|
||
zap.Error(err))
|
||
|
||
return &milvuspb.GetMetricsResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: err.Error(),
|
||
},
|
||
Response: "",
|
||
}, nil
|
||
}
|
||
|
||
if metricType == metricsinfo.SystemInfoMetrics {
|
||
metrics, err := getSystemInfoMetrics(ctx, req, node)
|
||
if err != nil {
|
||
log.Warn("QueryNode.GetMetrics failed",
|
||
zap.Int64("node_id", Params.QueryNodeCfg.QueryNodeID),
|
||
zap.String("req", req.Request),
|
||
zap.String("metric_type", metricType),
|
||
zap.Error(err))
|
||
}
|
||
|
||
return metrics, nil
|
||
}
|
||
|
||
log.Debug("QueryNode.GetMetrics failed, request metric type is not implemented yet",
|
||
zap.Int64("node_id", Params.QueryNodeCfg.QueryNodeID),
|
||
zap.String("req", req.Request),
|
||
zap.String("metric_type", metricType))
|
||
|
||
return &milvuspb.GetMetricsResponse{
|
||
Status: &commonpb.Status{
|
||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||
Reason: metricsinfo.MsgUnimplementedMetric,
|
||
},
|
||
Response: "",
|
||
}, nil
|
||
}
|