diff --git a/configs/milvus.yaml b/configs/milvus.yaml index 23b8e5d644f5b..6782e3bb1b7d8 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -252,6 +252,8 @@ proxy: serverMaxRecvSize: 67108864 clientMaxSendSize: 268435456 clientMaxRecvSize: 67108864 + # query whose executed time exceeds the `slowQuerySpanInSeconds` can be considered slow, in seconds. + slowQuerySpanInSeconds: 5 # Related configuration of queryCoord, used to manage topology and load balancing for the query nodes, and handoff from growing segments to sealed segments. queryCoord: diff --git a/go.mod b/go.mod index ab94b3a88d621..91a624f9a1f95 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,8 @@ module github.com/milvus-io/milvus -go 1.18 +go 1.21 + +toolchain go1.21.4 require ( github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.0 diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 83a1d57bbd8b1..43592a7e52b39 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -247,10 +247,7 @@ func (m *meta) GetSegmentsChanPart(selector SegmentInfoSelector) []*chanPartSegm return result } -// GetNumRowsOfCollection returns total rows count of segments belongs to provided collection -func (m *meta) GetNumRowsOfCollection(collectionID UniqueID) int64 { - m.RLock() - defer m.RUnlock() +func (m *meta) getNumRowsOfCollectionUnsafe(collectionID UniqueID) int64 { var ret int64 segments := m.segments.GetSegments() for _, segment := range segments { @@ -261,6 +258,13 @@ func (m *meta) GetNumRowsOfCollection(collectionID UniqueID) int64 { return ret } +// GetNumRowsOfCollection returns total rows count of segments belongs to provided collection +func (m *meta) GetNumRowsOfCollection(collectionID UniqueID) int64 { + m.RLock() + defer m.RUnlock() + return m.getNumRowsOfCollectionUnsafe(collectionID) +} + // GetCollectionBinlogSize returns the total binlog size and binlog size of collections. func (m *meta) GetCollectionBinlogSize() (int64, map[UniqueID]int64) { m.RLock() @@ -290,6 +294,16 @@ func (m *meta) GetCollectionBinlogSize() (int64, map[UniqueID]int64) { return total, collectionBinlogSize } +func (m *meta) GetAllCollectionNumRows() map[int64]int64 { + m.RLock() + defer m.RUnlock() + ret := make(map[int64]int64, len(m.collections)) + for collectionID := range m.collections { + ret[collectionID] = m.getNumRowsOfCollectionUnsafe(collectionID) + } + return ret +} + // AddSegment records segment info, persisting info into kv store func (m *meta) AddSegment(ctx context.Context, segment *SegmentInfo) error { log := log.Ctx(ctx) diff --git a/internal/datacoord/metrics_info.go b/internal/datacoord/metrics_info.go index c2e8e2fd802f4..78b4890af81e5 100644 --- a/internal/datacoord/metrics_info.go +++ b/internal/datacoord/metrics_info.go @@ -24,6 +24,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/hardware" @@ -43,6 +44,42 @@ func (s *Server) getQuotaMetrics() *metricsinfo.DataCoordQuotaMetrics { } } +func (s *Server) getCollectionMetrics(ctx context.Context) *metricsinfo.DataCoordCollectionMetrics { + totalNumRows := s.meta.GetAllCollectionNumRows() + ret := &metricsinfo.DataCoordCollectionMetrics{ + Collections: make(map[int64]*metricsinfo.DataCoordCollectionInfo, len(totalNumRows)), + } + for collectionID, total := range totalNumRows { + if _, ok := ret.Collections[collectionID]; !ok { + ret.Collections[collectionID] = &metricsinfo.DataCoordCollectionInfo{ + NumEntitiesTotal: 0, + IndexInfo: make([]*metricsinfo.DataCoordIndexInfo, 0), + } + } + ret.Collections[collectionID].NumEntitiesTotal = total + indexInfo, err := s.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{ + CollectionID: collectionID, + IndexName: "", + Timestamp: 0, + }) + if err := merr.CheckRPCCall(indexInfo, err); err != nil { + log.Ctx(ctx).Warn("failed to describe index, ignore to report index metrics", + zap.Int64("collection", collectionID), + zap.Error(err), + ) + continue + } + for _, info := range indexInfo.GetIndexInfos() { + ret.Collections[collectionID].IndexInfo = append(ret.Collections[collectionID].IndexInfo, &metricsinfo.DataCoordIndexInfo{ + NumEntitiesIndexed: info.GetIndexedRows(), + IndexName: info.GetIndexName(), + FieldID: info.GetIndexID(), + }) + } + } + return ret +} + // getSystemInfoMetrics composes data cluster metrics func (s *Server) getSystemInfoMetrics( ctx context.Context, @@ -53,7 +90,7 @@ func (s *Server) getSystemInfoMetrics( // get datacoord info nodes := s.cluster.GetSessions() clusterTopology := metricsinfo.DataClusterTopology{ - Self: s.getDataCoordMetrics(), + Self: s.getDataCoordMetrics(ctx), ConnectedDataNodes: make([]metricsinfo.DataNodeInfos, 0, len(nodes)), ConnectedIndexNodes: make([]metricsinfo.IndexNodeInfos, 0), } @@ -103,7 +140,7 @@ func (s *Server) getSystemInfoMetrics( } // getDataCoordMetrics composes datacoord infos -func (s *Server) getDataCoordMetrics() metricsinfo.DataCoordInfos { +func (s *Server) getDataCoordMetrics(ctx context.Context) metricsinfo.DataCoordInfos { ret := metricsinfo.DataCoordInfos{ BaseComponentInfos: metricsinfo.BaseComponentInfos{ Name: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, paramtable.GetNodeID()), @@ -125,7 +162,8 @@ func (s *Server) getDataCoordMetrics() metricsinfo.DataCoordInfos { SystemConfigurations: metricsinfo.DataCoordConfiguration{ SegmentMaxSize: Params.DataCoordCfg.SegmentMaxSize.GetAsFloat(), }, - QuotaMetrics: s.getQuotaMetrics(), + QuotaMetrics: s.getQuotaMetrics(), + CollectionMetrics: s.getCollectionMetrics(ctx), } metricsinfo.FillDeployMetricsWithEnv(&ret.BaseComponentInfos.SystemInfo) diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index 6450867c26e44..e4adf87bcfed6 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -2370,6 +2370,9 @@ func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest) receiveSize := proto.Size(dr.req) rateCol.Add(internalpb.RateType_DMLDelete.String(), float64(receiveSize)) + successCnt := dr.result.GetDeleteCnt() + metrics.ProxyDeleteVectors.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Add(float64(successCnt)) + metrics.ProxyFunctionCall.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method, metrics.SuccessLabel).Inc() metrics.ProxyMutationLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.DeleteLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) @@ -2576,8 +2579,12 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest) defer func() { span := tr.ElapseSpan() - if span >= SlowReadSpan { + if span >= paramtable.Get().ProxyCfg.SlowQuerySpanInSeconds.GetAsDuration(time.Second) { log.Info(rpcSlow(method), zap.Int64("nq", qt.SearchRequest.GetNq()), zap.Duration("duration", span)) + metrics.ProxySlowQueryCount.WithLabelValues( + strconv.FormatInt(paramtable.GetNodeID(), 10), + metrics.SearchLabel, + ).Inc() } }() @@ -2837,7 +2844,7 @@ func (node *Proxy) query(ctx context.Context, qt *queryTask) (*milvuspb.QueryRes defer func() { span := tr.ElapseSpan() - if span >= SlowReadSpan { + if span >= paramtable.Get().ProxyCfg.SlowQuerySpanInSeconds.GetAsDuration(time.Second) { log.Info( rpcSlow(method), zap.String("expr", request.Expr), @@ -2845,6 +2852,10 @@ func (node *Proxy) query(ctx context.Context, qt *queryTask) (*milvuspb.QueryRes zap.Uint64("travel_timestamp", request.TravelTimestamp), zap.Uint64("guarantee_timestamp", request.GuaranteeTimestamp), zap.Duration("duration", span)) + metrics.ProxySlowQueryCount.WithLabelValues( + strconv.FormatInt(paramtable.GetNodeID(), 10), + metrics.QueryLabel, + ).Inc() } }() diff --git a/internal/querynodev2/metrics_info.go b/internal/querynodev2/metrics_info.go index cb9310dbd11b1..edf24a3deb594 100644 --- a/internal/querynodev2/metrics_info.go +++ b/internal/querynodev2/metrics_info.go @@ -154,6 +154,18 @@ func getQuotaMetrics(node *QueryNode) (*metricsinfo.QueryNodeQuotaMetrics, error }, nil } +func getCollectionMetrics(node *QueryNode) (*metricsinfo.QueryNodeCollectionMetrics, error) { + allSegments := node.manager.Segment.GetBy() + ret := &metricsinfo.QueryNodeCollectionMetrics{ + CollectionRows: make(map[int64]int64), + } + for _, segment := range allSegments { + collectionID := segment.Collection() + ret.CollectionRows[collectionID] += segment.RowNum() + } + return ret, nil +} + // getSystemInfoMetrics returns metrics info of QueryNode func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, node *QueryNode) (*milvuspb.GetMetricsResponse, error) { usedMem := hardware.GetUsedMemoryCount() @@ -163,7 +175,7 @@ func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, if err != nil { return &milvuspb.GetMetricsResponse{ Status: merr.Status(err), - ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()), + ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, paramtable.GetNodeID()), }, nil } hardwareInfos := metricsinfo.HardwareMetrics{ @@ -177,6 +189,14 @@ func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, } quotaMetrics.Hms = hardwareInfos + collectionMetrics, err := getCollectionMetrics(node) + if err != nil { + return &milvuspb.GetMetricsResponse{ + Status: merr.Status(err), + ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, paramtable.GetNodeID()), + }, nil + } + nodeInfos := metricsinfo.QueryNodeInfos{ BaseComponentInfos: metricsinfo.BaseComponentInfos{ Name: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, paramtable.GetNodeID()), @@ -190,7 +210,8 @@ func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, SystemConfigurations: metricsinfo.QueryNodeConfiguration{ SimdType: paramtable.Get().CommonCfg.SimdType.GetValue(), }, - QuotaMetrics: quotaMetrics, + QuotaMetrics: quotaMetrics, + CollectionMetrics: collectionMetrics, } metricsinfo.FillDeployMetricsWithEnv(&nodeInfos.SystemInfo) diff --git a/internal/rootcoord/quota_center.go b/internal/rootcoord/quota_center.go index ce18aad3fd2da..307e68c4ead03 100644 --- a/internal/rootcoord/quota_center.go +++ b/internal/rootcoord/quota_center.go @@ -19,6 +19,7 @@ package rootcoord import ( "context" "fmt" + "github.com/milvus-io/milvus/internal/metastore/model" "math" "strconv" "sync" @@ -175,6 +176,59 @@ func (q *QuotaCenter) clearMetrics() { q.proxyMetrics = make(map[UniqueID]*metricsinfo.ProxyQuotaMetrics, 0) } +func updateNumEntitiesLoaded(current map[int64]int64, qn *metricsinfo.QueryNodeCollectionMetrics) map[int64]int64 { + for collectionID, rowNum := range qn.CollectionRows { + current[collectionID] += rowNum + } + return current +} + +func (q *QuotaCenter) reportNumEntitiesLoaded(numEntitiesLoaded map[int64]int64) { + for collectionID, num := range numEntitiesLoaded { + info, err := q.meta.GetCollectionByID(context.Background(), "", collectionID, typeutil.MaxTimestamp, false) + if err != nil { + log.Warn("failed to get collection info by its id, ignore to report loaded num entities", + zap.Int64("collection", collectionID), + zap.Int64("num_entities_loaded", num), + zap.Error(err), + ) + continue + } + metrics.RootCoordNumEntities.WithLabelValues(info.Name, metrics.LoadedLabel).Set(float64(num)) + } +} + +func (q *QuotaCenter) reportDataCoordCollectionMetrics(dc *metricsinfo.DataCoordCollectionMetrics) { + for collectionID, collection := range dc.Collections { + info, err := q.meta.GetCollectionByID(context.Background(), "", collectionID, typeutil.MaxTimestamp, false) + if err != nil { + log.Warn("failed to get collection info by its id, ignore to report total_num_entities/indexed_entities", + zap.Int64("collection", collectionID), + zap.Int64("num_entities_total", collection.NumEntitiesTotal), + zap.Int("lenOfIndexedInfo", len(collection.IndexInfo)), + zap.Error(err), + ) + continue + } + metrics.RootCoordNumEntities.WithLabelValues(info.Name, metrics.TotalLabel).Set(float64(collection.NumEntitiesTotal)) + fields := lo.KeyBy(info.Fields, func(v *model.Field) int64 { return v.FieldID }) + for _, indexInfo := range collection.IndexInfo { + if _, ok := fields[indexInfo.FieldID]; !ok { + log.Warn("field id not found, ignore to report indexed num entities", + zap.Int64("collection", collectionID), + zap.Int64("field", indexInfo.FieldID), + ) + continue + } + field := fields[indexInfo.FieldID] + metrics.RootCoordIndexedNumEntities.WithLabelValues( + info.Name, + indexInfo.IndexName, + strconv.FormatBool(typeutil.IsVectorType(field.DataType))).Set(float64(indexInfo.NumEntitiesIndexed)) + } + } +} + // syncMetrics sends GetMetrics requests to DataCoord and QueryCoord to sync the metrics in DataNodes and QueryNodes. func (q *QuotaCenter) syncMetrics() error { oldDataNodes := typeutil.NewSet(lo.Keys(q.dataNodeMetrics)...) @@ -189,6 +243,8 @@ func (q *QuotaCenter) syncMetrics() error { return err } + numEntitiesLoaded := make(map[int64]int64) + // get Query cluster metrics group.Go(func() error { rsp, err := q.queryCoord.GetMetrics(ctx, req) @@ -208,8 +264,12 @@ func (q *QuotaCenter) syncMetrics() error { q.queryNodeMetrics[queryNodeMetric.ID] = queryNodeMetric.QuotaMetrics collections.Insert(queryNodeMetric.QuotaMetrics.Effect.CollectionIDs...) } + if queryNodeMetric.CollectionMetrics != nil { + numEntitiesLoaded = updateNumEntitiesLoaded(numEntitiesLoaded, queryNodeMetric.CollectionMetrics) + } } q.readableCollections = collections.Collect() + q.reportNumEntitiesLoaded(numEntitiesLoaded) return nil }) // get Data cluster metrics @@ -224,6 +284,10 @@ func (q *QuotaCenter) syncMetrics() error { return err } + if dataCoordTopology.Cluster.Self.CollectionMetrics != nil { + q.reportDataCoordCollectionMetrics(dataCoordTopology.Cluster.Self.CollectionMetrics) + } + collections := typeutil.NewUniqueSet() for _, dataNodeMetric := range dataCoordTopology.Cluster.ConnectedDataNodes { if dataNodeMetric.QuotaMetrics != nil { @@ -842,14 +906,31 @@ func (q *QuotaCenter) setRates() error { func (q *QuotaCenter) recordMetrics() { record := func(errorCode commonpb.ErrorCode) { var hasException float64 = 0 - for _, states := range q.quotaStates { + for collectionID, states := range q.quotaStates { + info, err := q.meta.GetCollectionByID(context.Background(), "", collectionID, typeutil.MaxTimestamp, false) + if err != nil { + log.Warn("failed to get collection info by its id, ignore to report quota states", + zap.Int64("collection", collectionID), + zap.Error(err), + ) + continue + } + dbm, err := q.meta.GetDatabaseByID(context.Background(), info.DBID, typeutil.MaxTimestamp) + if err != nil { + log.Warn("failed to get database name info by its id, ignore to report quota states", + zap.Int64("collection", collectionID), + zap.Error(err), + ) + continue + } + for _, state := range states { if state == errorCode { hasException = 1 } } + metrics.RootCoordQuotaStates.WithLabelValues(errorCode.String(), dbm.Name).Set(hasException) } - metrics.RootCoordQuotaStates.WithLabelValues(errorCode.String()).Set(hasException) } record(commonpb.ErrorCode_MemoryQuotaExhausted) record(commonpb.ErrorCode_DiskQuotaExhausted) diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index 3ee36221dfb83..eaa1734928091 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -74,6 +74,8 @@ const ( functionLabelName = "function_name" queryTypeLabelName = "query_type" collectionName = "collection_name" + indexName = "index_name" + isVectorIndex = "is_vector_index" segmentStateLabelName = "segment_state" segmentIDLabelName = "segment_id" usernameLabelName = "username" @@ -88,6 +90,10 @@ const ( lockSource = "lock_source" lockType = "lock_type" lockOp = "lock_op" + + // entities label + LoadedLabel = "loaded" + NumEntitiesAllLabel = "all" ) var ( diff --git a/pkg/metrics/proxy_metrics.go b/pkg/metrics/proxy_metrics.go index e2349adff9316..66a8ee01d7d47 100644 --- a/pkg/metrics/proxy_metrics.go +++ b/pkg/metrics/proxy_metrics.go @@ -60,6 +60,14 @@ var ( Help: "counter of vectors successfully upserted", }, []string{nodeIDLabelName}) + ProxyDeleteVectors = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.ProxyRole, + Name: "delete_vectors_count", + Help: "counter of vectors successfully deleted", + }, []string{nodeIDLabelName}) + // ProxySQLatency record the latency of search successfully. ProxySQLatency = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -306,6 +314,14 @@ var ( Name: "rate_limit_req_count", Help: "count of operation executed", }, []string{nodeIDLabelName, msgTypeLabelName, statusLabelName}) + + ProxySlowQueryCount = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.ProxyRole, + Name: "slow_query_count", + Help: "count of slow query executed", + }, []string{nodeIDLabelName, msgTypeLabelName}) ) // RegisterProxy registers Proxy metrics @@ -314,6 +330,7 @@ func RegisterProxy(registry *prometheus.Registry) { registry.MustRegister(ProxySearchVectors) registry.MustRegister(ProxyInsertVectors) registry.MustRegister(ProxyUpsertVectors) + registry.MustRegister(ProxyDeleteVectors) registry.MustRegister(ProxySQLatency) registry.MustRegister(ProxyCollectionSQLatency) @@ -351,6 +368,8 @@ func RegisterProxy(registry *prometheus.Registry) { registry.MustRegister(ProxyWorkLoadScore) registry.MustRegister(ProxyExecutingTotalNq) registry.MustRegister(ProxyRateLimitReqCount) + + registry.MustRegister(ProxySlowQueryCount) } func CleanupCollectionMetrics(nodeID int64, collection string) { diff --git a/pkg/metrics/rootcoord_metrics.go b/pkg/metrics/rootcoord_metrics.go index c73238f470b37..3b7fdd0868169 100644 --- a/pkg/metrics/rootcoord_metrics.go +++ b/pkg/metrics/rootcoord_metrics.go @@ -167,6 +167,7 @@ var ( Help: "The quota states of cluster", }, []string{ "quota_states", + "db_name", }) // RootCoordRateLimitRatio reflects the ratio of rate limit. @@ -185,6 +186,29 @@ var ( Name: "ddl_req_latency_in_queue", Help: "latency of each DDL operations in queue", }, []string{functionLabelName}) + + RootCoordNumEntities = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.RootCoordRole, + Name: "entity_num", + Help: "number of entities, clustered by collection and their status(loaded/total)", + }, []string{ + collectionName, + statusLabelName, + }) + + RootCoordIndexedNumEntities = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.RootCoordRole, + Name: "indexed_entity_num", + Help: "indexed number of entities, clustered by collection, index name and whether it's a vector index", + }, []string{ + collectionName, + indexName, + isVectorIndex, + }) ) // RegisterRootCoord registers RootCoord metrics @@ -219,4 +243,7 @@ func RegisterRootCoord(registry *prometheus.Registry) { registry.MustRegister(RootCoordQuotaStates) registry.MustRegister(RootCoordRateLimitRatio) registry.MustRegister(RootCoordDDLReqLatencyInQueue) + + registry.MustRegister(RootCoordNumEntities) + registry.MustRegister(RootCoordIndexedNumEntities) } diff --git a/pkg/util/metricsinfo/metrics_info.go b/pkg/util/metricsinfo/metrics_info.go index 7673e5d0e8f37..9baaf1ffcd756 100644 --- a/pkg/util/metricsinfo/metrics_info.go +++ b/pkg/util/metricsinfo/metrics_info.go @@ -94,11 +94,16 @@ type QueryNodeConfiguration struct { SimdType string `json:"simd_type"` } +type QueryNodeCollectionMetrics struct { + CollectionRows map[int64]int64 +} + // QueryNodeInfos implements ComponentInfos type QueryNodeInfos struct { BaseComponentInfos - SystemConfigurations QueryNodeConfiguration `json:"system_configurations"` - QuotaMetrics *QueryNodeQuotaMetrics `json:"quota_metrics"` + SystemConfigurations QueryNodeConfiguration `json:"system_configurations"` + QuotaMetrics *QueryNodeQuotaMetrics `json:"quota_metrics"` + CollectionMetrics *QueryNodeCollectionMetrics `json:"collection_metrics"` } // QueryCoordConfiguration records the configuration of QueryCoord. @@ -167,11 +172,27 @@ type DataCoordConfiguration struct { SegmentMaxSize float64 `json:"segment_max_size"` } +type DataCoordIndexInfo struct { + NumEntitiesIndexed int64 + IndexName string + FieldID int64 +} + +type DataCoordCollectionInfo struct { + NumEntitiesTotal int64 + IndexInfo []*DataCoordIndexInfo +} + +type DataCoordCollectionMetrics struct { + Collections map[int64]*DataCoordCollectionInfo +} + // DataCoordInfos implements ComponentInfos type DataCoordInfos struct { BaseComponentInfos - SystemConfigurations DataCoordConfiguration `json:"system_configurations"` - QuotaMetrics *DataCoordQuotaMetrics `json:"quota_metrics"` + SystemConfigurations DataCoordConfiguration `json:"system_configurations"` + QuotaMetrics *DataCoordQuotaMetrics `json:"quota_metrics"` + CollectionMetrics *DataCoordCollectionMetrics `json:"collection_metrics"` } // RootCoordConfiguration records the configuration of RootCoord. diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index 14424be7cff60..6fc62d1233f85 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -978,6 +978,8 @@ type proxyConfig struct { MaxConnectionNum ParamItem `refreshable:"true"` GracefulStopTimeout ParamItem `refreshable:"true"` + + SlowQuerySpanInSeconds ParamItem `refreshable:"true"` } func (p *proxyConfig) init(base *BaseTable) { @@ -1280,6 +1282,15 @@ please adjust in embedded Milvus: false`, } p.GracefulStopTimeout.Init(base.mgr) + p.SlowQuerySpanInSeconds = ParamItem{ + Key: "proxy.slowQuerySpanInSeconds", + Version: "2.3.11", + Doc: "query whose executed time exceeds the `slowQuerySpanInSeconds` can be considered slow, in seconds.", + DefaultValue: "5", + Export: true, + } + p.SlowQuerySpanInSeconds.Init(base.mgr) + p.ConnectionCheckIntervalSeconds = ParamItem{ Key: "proxy.connectionCheckIntervalSeconds", Version: "2.3.11",