diff --git a/Makefile b/Makefile index 7f70aeb586..b9ba209c75 100644 --- a/Makefile +++ b/Makefile @@ -146,6 +146,10 @@ test-proxy: @echo "Running go unittests..." go test -race -coverpkg=./... -coverprofile=profile.out -covermode=atomic -timeout 5m github.com/milvus-io/milvus/internal/proxy -v +test-datacoord: + @echo "Running go unittests..." + go test -race -coverpkg=./... -coverprofile=profile.out -covermode=atomic -timeout 5m github.com/milvus-io/milvus/internal/datacoord -v + test-datanode: @echo "Running go unittests..." go test -race -coverpkg=./... -coverprofile=profile.out -covermode=atomic -timeout 5m github.com/milvus-io/milvus/internal/datanode -v diff --git a/internal/datacoord/cluster.go b/internal/datacoord/cluster.go index 366565ab03..050cdcd2c7 100644 --- a/internal/datacoord/cluster.go +++ b/internal/datacoord/cluster.go @@ -19,10 +19,12 @@ package datacoord import ( "context" + "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" - "go.uber.org/zap" ) // Cluster provides interfaces to interact with datanode cluster @@ -56,12 +58,20 @@ func (c *Cluster) Startup(nodes []*NodeInfo) error { // Register registers a new node in cluster func (c *Cluster) Register(node *NodeInfo) error { c.sessionManager.AddSession(node) - return c.channelManager.AddNode(node.NodeID) + err := c.channelManager.AddNode(node.NodeID) + if err == nil { + metrics.DataCoordNumDataNodes.WithLabelValues().Inc() + } + return err } // UnRegister removes a node from cluster func (c *Cluster) UnRegister(node *NodeInfo) error { c.sessionManager.DeleteSession(node) + err := c.channelManager.DeleteNode(node.NodeID) + if err == nil { + metrics.DataCoordNumDataNodes.WithLabelValues().Dec() + } return c.channelManager.DeleteNode(node.NodeID) } diff --git a/internal/datacoord/handler.go b/internal/datacoord/handler.go index ee71e64c01..74a483fe71 100644 --- a/internal/datacoord/handler.go +++ b/internal/datacoord/handler.go @@ -23,7 +23,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/rootcoord" + "github.com/milvus-io/milvus/internal/util/funcutil" "go.uber.org/zap" ) @@ -108,7 +108,7 @@ func (h *ServerHandler) GetVChanPositions(channel string, collectionID UniqueID, func getCollectionStartPosition(channel string, collectionInfo *datapb.CollectionInfo) *internalpb.MsgPosition { for _, sp := range collectionInfo.GetStartPositions() { - if sp.GetKey() != rootcoord.ToPhysicalChannel(channel) { + if sp.GetKey() != funcutil.ToPhysicalChannel(channel) { continue } return &internalpb.MsgPosition{ diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index a7bc1fae45..d07e1ceb61 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -25,6 +25,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" @@ -68,16 +69,28 @@ func (m *meta) reloadFromKV() error { if err != nil { return err } - + metrics.DataCoordNumCollections.WithLabelValues().Set(0) + metrics.DataCoordNumSegments.WithLabelValues(metrics.SealedSegmentLabel).Set(0) + metrics.DataCoordNumSegments.WithLabelValues(metrics.GrowingSegmentLabel).Set(0) + metrics.DataCoordNumSegments.WithLabelValues(metrics.FlushedSegmentLabel).Set(0) + metrics.DataCoordNumSegments.WithLabelValues(metrics.FlushingSegmentLabel).Set(0) + metrics.DataCoordNumSegments.WithLabelValues(metrics.DropedSegmentLabel).Set(0) + metrics.DataCoordNumStoredRows.WithLabelValues().Set(0) + numStoredRows := int64(0) for _, value := range values { segmentInfo := &datapb.SegmentInfo{} err = proto.Unmarshal([]byte(value), segmentInfo) if err != nil { return fmt.Errorf("DataCoord reloadFromKV UnMarshal datapb.SegmentInfo err:%w", err) } + state := segmentInfo.GetState() m.segments.SetSegment(segmentInfo.GetID(), NewSegmentInfo(segmentInfo)) + metrics.DataCoordNumSegments.WithLabelValues(string(state)).Inc() + if state == commonpb.SegmentState_Flushed { + numStoredRows += segmentInfo.GetNumOfRows() + } } - + metrics.DataCoordNumStoredRows.WithLabelValues().Set(float64(numStoredRows)) return nil } @@ -87,6 +100,7 @@ func (m *meta) AddCollection(collection *datapb.CollectionInfo) { m.Lock() defer m.Unlock() m.collections[collection.ID] = collection + metrics.DataCoordNumCollections.WithLabelValues().Set(float64(len(m.collections))) } // GetCollection returns collection info with provided collection id from local cache @@ -160,6 +174,7 @@ func (m *meta) AddSegment(segment *SegmentInfo) error { if err := m.saveSegmentInfo(segment); err != nil { return err } + metrics.DataCoordNumSegments.WithLabelValues(string(segment.GetState())).Inc() return nil } @@ -174,6 +189,7 @@ func (m *meta) DropSegment(segmentID UniqueID) error { if err := m.removeSegmentInfo(segment); err != nil { return err } + metrics.DataCoordNumSegments.WithLabelValues(metrics.DropedSegmentLabel).Inc() m.segments.DropSegment(segmentID) return nil } @@ -194,9 +210,25 @@ func (m *meta) GetSegment(segID UniqueID) *SegmentInfo { func (m *meta) SetState(segmentID UniqueID, state commonpb.SegmentState) error { m.Lock() defer m.Unlock() + curSegInfo := m.segments.GetSegment(segmentID) + if curSegInfo == nil { + return nil + } + oldState := curSegInfo.GetState() m.segments.SetState(segmentID, state) - if segInfo := m.segments.GetSegment(segmentID); segInfo != nil && isSegmentHealthy(segInfo) { - return m.saveSegmentInfo(segInfo) + curSegInfo = m.segments.GetSegment(segmentID) + if curSegInfo != nil && isSegmentHealthy(curSegInfo) { + err := m.saveSegmentInfo(curSegInfo) + if err == nil { + metrics.DataCoordNumSegments.WithLabelValues(string(oldState)).Dec() + metrics.DataCoordNumSegments.WithLabelValues(string(state)).Inc() + if state == commonpb.SegmentState_Flushed { + metrics.DataCoordNumStoredRows.WithLabelValues().Add(float64(curSegInfo.GetNumOfRows())) + } else if oldState == commonpb.SegmentState_Flushed { + metrics.DataCoordNumStoredRows.WithLabelValues().Sub(float64(curSegInfo.GetNumOfRows())) + } + } + return err } return nil } @@ -336,7 +368,15 @@ func (m *meta) UpdateFlushSegmentsInfo( if err := m.saveKvTxn(kv); err != nil { return err } - + oldSegmentState := segment.GetState() + newSegmentState := clonedSegment.GetState() + metrics.DataCoordNumSegments.WithLabelValues(string(oldSegmentState)).Dec() + metrics.DataCoordNumSegments.WithLabelValues(string(newSegmentState)).Inc() + if newSegmentState == commonpb.SegmentState_Flushed { + metrics.DataCoordNumStoredRows.WithLabelValues().Add(float64(clonedSegment.GetNumOfRows())) + } else if oldSegmentState == commonpb.SegmentState_Flushed { + metrics.DataCoordNumStoredRows.WithLabelValues().Sub(float64(segment.GetNumOfRows())) + } // update memory status for id, s := range modSegments { m.segments.SetSegment(id, s) @@ -351,10 +391,12 @@ func (m *meta) UpdateDropChannelSegmentInfo(channel string, segments []*SegmentI defer m.Unlock() modSegments := make(map[UniqueID]*SegmentInfo) + originSegments := make(map[UniqueID]*SegmentInfo) for _, seg2Drop := range segments { segment := m.mergeDropSegment(seg2Drop) if segment != nil { + originSegments[seg2Drop.GetID()] = seg2Drop modSegments[seg2Drop.GetID()] = segment } } @@ -369,10 +411,21 @@ func (m *meta) UpdateDropChannelSegmentInfo(channel string, segments []*SegmentI clonedSeg := seg.Clone() clonedSeg.State = commonpb.SegmentState_Dropped modSegments[seg.ID] = clonedSeg + originSegments[seg.GetID()] = seg } } - - return m.batchSaveDropSegments(channel, modSegments) + err := m.batchSaveDropSegments(channel, modSegments) + if err == nil { + for _, seg := range originSegments { + state := seg.GetState() + metrics.DataCoordNumSegments.WithLabelValues( + string(state)).Dec() + if state == commonpb.SegmentState_Flushed { + metrics.DataCoordNumStoredRows.WithLabelValues().Sub(float64(seg.GetNumOfRows())) + } + } + } + return err } // mergeDropSegment merges drop segment information with meta segments @@ -495,6 +548,7 @@ func (m *meta) saveDropSegmentAndRemove(channel string, modSegments map[int64]*S for _, s := range update { m.segments.SetSegment(s.GetID(), s) } + metrics.DataCoordNumSegments.WithLabelValues(metrics.DropedSegmentLabel).Add(float64(len(update))) return nil } diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 3c9ebf2a1f..58acae3e9f 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -30,6 +30,7 @@ import ( rootcoordclient "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper" "github.com/milvus-io/milvus/internal/proto/commonpb" @@ -518,6 +519,9 @@ func (s *Server) handleTimetickMessage(ctx context.Context, ttMsg *msgstream.Dat log.RatedWarn(60.0, "time tick lag behind for more than 1 minutes", zap.String("channel", ch), zap.Time("timetick", physical)) } + utcT, _ := tsoutil.ParseHybridTs(ts) + metrics.DataCoordSyncUTC.WithLabelValues().Set(float64(utcT)) + s.updateSegmentStatistics(ttMsg.GetSegmentsStats()) if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil { diff --git a/internal/datanode/compactor.go b/internal/datanode/compactor.go index 58a09fccd9..97d734555f 100644 --- a/internal/datanode/compactor.go +++ b/internal/datanode/compactor.go @@ -545,7 +545,7 @@ func (t *compactionTask) compact() error { ) log.Info("overall elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Any("elapse", nano2Milli(time.Since(compactStart)))) - metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Observe(float64(t.tr.ElapseSpan().Milliseconds())) + metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Observe(float64(t.tr.ElapseSpan().Milliseconds())) return nil } diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 95ed3deda0..51498ba271 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -564,7 +564,9 @@ func (node *DataNode) ReadyToFlush() error { // // One precondition: The segmentID in req is in ascending order. func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) { - metrics.DataNodeFlushSegmentsCounter.WithLabelValues(MetricRequestsTotal).Inc() + metrics.DataNodeFlushSegmentsReqCounter.WithLabelValues( + fmt.Sprint(Params.DataNodeCfg.NodeID), + MetricRequestsTotal).Inc() status := &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -625,7 +627,9 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen } status.ErrorCode = commonpb.ErrorCode_Success - metrics.DataNodeFlushSegmentsCounter.WithLabelValues(MetricRequestsSuccess).Inc() + metrics.DataNodeFlushSegmentsReqCounter.WithLabelValues( + fmt.Sprint(Params.DataNodeCfg.NodeID), + MetricRequestsSuccess).Inc() return status, nil } diff --git a/internal/datanode/data_sync_service.go b/internal/datanode/data_sync_service.go index 7ffb0f426d..8369095eff 100644 --- a/internal/datanode/data_sync_service.go +++ b/internal/datanode/data_sync_service.go @@ -130,8 +130,8 @@ func (dsService *dataSyncService) close() { log.Info("dataSyncService closing flowgraph", zap.Int64("collectionID", dsService.collectionID), zap.String("vChanName", dsService.vchannelName)) dsService.fg.Close() - metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(dsService.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() - metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(dsService.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Sub(2) // timeTickChannel + deltaChannel + metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() + metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Sub(2) // timeTickChannel + deltaChannel } dsService.cancelFn() diff --git a/internal/datanode/flow_graph_dd_node.go b/internal/datanode/flow_graph_dd_node.go index 59b1429591..c46d4e1d60 100644 --- a/internal/datanode/flow_graph_dd_node.go +++ b/internal/datanode/flow_graph_dd_node.go @@ -28,8 +28,8 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/util/flowgraph" + "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/trace" "github.com/opentracing/opentracing-go" "go.uber.org/zap" @@ -282,8 +282,8 @@ func newDDNode(ctx context.Context, collID UniqueID, vchanInfo *datapb.VchannelI log.Error(err.Error()) return nil } - pChannelName := rootcoord.ToPhysicalChannel(vchanInfo.ChannelName) - deltaChannelName, err := rootcoord.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) + pChannelName := funcutil.ToPhysicalChannel(vchanInfo.ChannelName) + deltaChannelName, err := funcutil.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) if err != nil { log.Error(err.Error()) return nil @@ -291,7 +291,7 @@ func newDDNode(ctx context.Context, collID UniqueID, vchanInfo *datapb.VchannelI deltaStream.SetRepackFunc(msgstream.DefaultRepackFunc) deltaStream.AsProducer([]string{deltaChannelName}) - metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() log.Debug("datanode AsProducer", zap.String("DeltaChannelName", deltaChannelName)) var deltaMsgStream msgstream.MsgStream = deltaStream deltaMsgStream.Start() diff --git a/internal/datanode/flow_graph_dmstream_input_node.go b/internal/datanode/flow_graph_dmstream_input_node.go index 6ed2c0844f..d779205ac3 100644 --- a/internal/datanode/flow_graph_dmstream_input_node.go +++ b/internal/datanode/flow_graph_dmstream_input_node.go @@ -24,8 +24,8 @@ import ( "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/util/flowgraph" + "github.com/milvus-io/milvus/internal/util/funcutil" "go.uber.org/zap" ) @@ -43,9 +43,9 @@ func newDmInputNode(ctx context.Context, seekPos *internalpb.MsgPosition, dmNode // MsgStream needs a physical channel name, but the channel name in seek position from DataCoord // is virtual channel name, so we need to convert vchannel name into pchannel neme here. - pchannelName := rootcoord.ToPhysicalChannel(dmNodeConfig.vChannelName) + pchannelName := funcutil.ToPhysicalChannel(dmNodeConfig.vChannelName) insertStream.AsConsumer([]string{pchannelName}, consumeSubName) - metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(dmNodeConfig.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() log.Info("datanode AsConsumer", zap.String("physical channel", pchannelName), zap.String("subName", consumeSubName), zap.Int64("collection ID", dmNodeConfig.collectionID)) if seekPos != nil { diff --git a/internal/datanode/flow_graph_insert_buffer_node.go b/internal/datanode/flow_graph_insert_buffer_node.go index 2a185ac71c..a9d385e8fc 100644 --- a/internal/datanode/flow_graph_insert_buffer_node.go +++ b/internal/datanode/flow_graph_insert_buffer_node.go @@ -35,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/tsoutil" ) @@ -296,7 +297,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { dropped: false, }) - metrics.DataNodeAutoFlushSegmentCount.WithLabelValues(ibNode.channelName, fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeAutoFlushSegmentCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() } } @@ -341,13 +342,13 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { err := ibNode.flushManager.flushBufferData(task.buffer, task.segmentID, task.flushed, task.dropped, endPositions[0]) if err != nil { log.Warn("failed to invoke flushBufferData", zap.Error(err)) - metrics.DataNodeFlushSegmentCount.WithLabelValues(metrics.FailLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeFlushSegmentCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.FailLabel).Inc() } else { segmentsToFlush = append(segmentsToFlush, task.segmentID) ibNode.insertBuffer.Delete(task.segmentID) - metrics.DataNodeFlushSegmentCount.WithLabelValues(metrics.SuccessLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeFlushSegmentCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.SuccessLabel).Inc() } - metrics.DataNodeFlushSegmentCount.WithLabelValues(metrics.TotalLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeFlushSegmentCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.TotalLabel).Inc() } if err := ibNode.writeHardTimeTick(fgMsg.timeRange.timestampMax, seg2Upload); err != nil { @@ -471,7 +472,7 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos // update buffer size buffer.updateSize(int64(msg.NRows())) - metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(metrics.InsertLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Add(float64(len(msg.RowData))) + metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.InsertLabel).Add(float64(len(msg.RowData))) // store in buffer ibNode.insertBuffer.Store(currentSegID, buffer) @@ -505,7 +506,7 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl return nil, err } wTt.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick}) - metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() log.Debug("datanode AsProducer", zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick)) var wTtMsgStream msgstream.MsgStream = wTt wTtMsgStream.Start() @@ -519,7 +520,7 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl continue } stats = append(stats, stat) - metrics.DataNodeSegmentRowsCount.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Add(float64(stat.NumRows)) + metrics.DataNodeSegmentRowsCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Add(float64(stat.NumRows)) } msgPack := msgstream.MsgPack{} timeTickMsg := msgstream.DataNodeTtMsg{ @@ -541,7 +542,8 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl } msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg) pt, _ := tsoutil.ParseHybridTs(ts) - metrics.DataNodeTimeSync.WithLabelValues(config.vChannelName, fmt.Sprint(Params.DataNodeCfg.NodeID)).Set(float64(pt)) + pChan := funcutil.ToPhysicalChannel(config.vChannelName) + metrics.DataNodeTimeSync.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), pChan).Set(float64(pt)) return wTtMsgStream.Produce(&msgPack) }) diff --git a/internal/datanode/flow_graph_manager.go b/internal/datanode/flow_graph_manager.go index 2c1caa5e54..0e36a3ceeb 100644 --- a/internal/datanode/flow_graph_manager.go +++ b/internal/datanode/flow_graph_manager.go @@ -68,8 +68,6 @@ func (fm *flowgraphManager) addAndStart(dn *DataNode, vchan *datapb.VchannelInfo fm.flowgraphs.Store(vchan.GetChannelName(), dataSyncService) metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() - metrics.DataNodeNumDmlChannels.WithLabelValues(fmt.Sprint(vchan.GetCollectionID()), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() - metrics.DataNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(vchan.GetCollectionID()), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() return nil } @@ -77,11 +75,8 @@ func (fm *flowgraphManager) release(vchanName string) { log.Info("release flowgraph resources begin", zap.String("vChannelName", vchanName)) if fg, loaded := fm.flowgraphs.LoadAndDelete(vchanName); loaded { - collectionID := fg.(*dataSyncService).collectionID fg.(*dataSyncService).close() metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() - metrics.DataNodeNumDmlChannels.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() - metrics.DataNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() } log.Info("release flowgraph resources end", zap.String("Vchannel", vchanName)) } diff --git a/internal/datanode/flush_manager.go b/internal/datanode/flush_manager.go index 35b91f40fc..4c613df45f 100644 --- a/internal/datanode/flush_manager.go +++ b/internal/datanode/flush_manager.go @@ -413,7 +413,7 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segmentID Uni data: kvs, }, field2Insert, field2Stats, flushed, dropped, pos) - metrics.DataNodeFlushSegmentLatency.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.DataNodeFlushSegmentLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) return nil } @@ -555,11 +555,11 @@ type flushBufferInsertTask struct { func (t *flushBufferInsertTask) flushInsertData() error { if t.BaseKV != nil && len(t.data) > 0 { for _, d := range t.data { - metrics.DataNodeFlushedSize.WithLabelValues(metrics.InsertLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Add(float64(len(d))) + metrics.DataNodeFlushedSize.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.InsertLabel).Add(float64(len(d))) } tr := timerecord.NewTimeRecorder("insertData") err := t.MultiSave(t.data) - metrics.DataNodeSave2StorageLatency.WithLabelValues(metrics.InsertLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.DataNodeSave2StorageLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.InsertLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) return err } return nil @@ -574,11 +574,11 @@ type flushBufferDeleteTask struct { func (t *flushBufferDeleteTask) flushDeleteData() error { if len(t.data) > 0 && t.BaseKV != nil { for _, d := range t.data { - metrics.DataNodeFlushedSize.WithLabelValues(metrics.DeleteLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Add(float64(len(d))) + metrics.DataNodeFlushedSize.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.DeleteLabel).Add(float64(len(d))) } tr := timerecord.NewTimeRecorder("deleteData") err := t.MultiSave(t.data) - metrics.DataNodeSave2StorageLatency.WithLabelValues(metrics.DeleteLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.DataNodeSave2StorageLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID), metrics.DeleteLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) return err } return nil diff --git a/internal/datanode/segment_replica.go b/internal/datanode/segment_replica.go index 2d669c6ca9..f89352aa65 100644 --- a/internal/datanode/segment_replica.go +++ b/internal/datanode/segment_replica.go @@ -194,7 +194,7 @@ func (replica *SegmentReplica) new2FlushedSegment(segID UniqueID) { replica.flushedSegments[segID] = &seg delete(replica.newSegments, segID) - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(seg.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() } // normal2FlushedSegment transfers a segment from *normal* to *flushed* by changing *isFlushed* @@ -206,7 +206,7 @@ func (replica *SegmentReplica) normal2FlushedSegment(segID UniqueID) { replica.flushedSegments[segID] = &seg delete(replica.normalSegments, segID) - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(seg.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() } func (replica *SegmentReplica) getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) { @@ -268,7 +268,7 @@ func (replica *SegmentReplica) addNewSegment(segID, collID, partitionID UniqueID replica.segMu.Lock() defer replica.segMu.Unlock() replica.newSegments[segID] = seg - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() return nil } @@ -363,7 +363,7 @@ func (replica *SegmentReplica) addNormalSegment(segID, collID, partitionID Uniqu replica.segMu.Lock() replica.normalSegments[segID] = seg replica.segMu.Unlock() - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(collID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() return nil } @@ -560,15 +560,15 @@ func (replica *SegmentReplica) removeSegments(segIDs ...UniqueID) { defer replica.segMu.Unlock() log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs)) - + cnt := 0 for _, segID := range segIDs { - if seg, ok := replica.newSegments[segID]; ok { - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(seg.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() - } - if seg, ok := replica.normalSegments[segID]; ok { - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(seg.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() + if _, ok := replica.newSegments[segID]; ok { + cnt++ + } else if _, ok := replica.normalSegments[segID]; ok { + cnt++ } } + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Sub(float64(cnt)) for _, segID := range segIDs { delete(replica.newSegments, segID) diff --git a/internal/indexcoord/node_manager.go b/internal/indexcoord/node_manager.go index 3a7439a555..431dcd9c15 100644 --- a/internal/indexcoord/node_manager.go +++ b/internal/indexcoord/node_manager.go @@ -76,7 +76,7 @@ func (nm *NodeManager) RemoveNode(nodeID UniqueID) { delete(nm.nodeClients, nodeID) nm.lock.Unlock() nm.pq.Remove(nodeID) - metrics.IndexCoordIndexNodeNum.WithLabelValues("index_node_num").Dec() + metrics.IndexCoordIndexNodeNum.WithLabelValues().Dec() } // AddNode adds the client of IndexNode. @@ -97,7 +97,7 @@ func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error { log.Error("IndexCoord NodeManager", zap.Any("Add node err", err)) return err } - metrics.IndexCoordIndexNodeNum.WithLabelValues("index_node_num").Inc() + metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc() return nm.setClient(nodeID, nodeClient) } diff --git a/internal/indexnode/task.go b/internal/indexnode/task.go index 91a5958c1e..b3571abb07 100644 --- a/internal/indexnode/task.go +++ b/internal/indexnode/task.go @@ -380,8 +380,8 @@ func (it *IndexBuildTask) loadVector(ctx context.Context) (storage.FieldID, stor } // TODO: @xiaocai2333 metrics.IndexNodeLoadBinlogLatency should be added above, put here to get segmentID. - metrics.IndexNodeLoadBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), strconv.FormatInt(it.segmentID, 10)).Observe(float64(loadVectorDuration)) - metrics.IndexNodeDecodeBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), strconv.FormatInt(it.segmentID, 10)).Observe(float64(it.tr.RecordSpan())) + metrics.IndexNodeLoadBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(loadVectorDuration)) + metrics.IndexNodeDecodeBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(it.tr.RecordSpan())) if len(insertData.Data) != 1 { return storage.InvalidUniqueID, nil, errors.New("we expect only one field in deserialized insert data") @@ -438,7 +438,7 @@ func (it *IndexBuildTask) buildIndex(ctx context.Context) ([]*storage.Blob, erro } } - metrics.IndexNodeKnowhereBuildIndexLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), strconv.FormatInt(it.segmentID, 10)).Observe(float64(it.tr.RecordSpan())) + metrics.IndexNodeKnowhereBuildIndexLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(it.tr.RecordSpan())) if !fOk && !bOk { return nil, errors.New("we expect FloatVectorFieldData or BinaryVectorFieldData") @@ -478,8 +478,7 @@ func (it *IndexBuildTask) buildIndex(ctx context.Context) ([]*storage.Blob, erro return nil, err } encodeIndexFileDur := it.tr.Record("index codec serialize done") - metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), - strconv.FormatInt(it.segmentID, 10)).Observe(float64(encodeIndexFileDur.Milliseconds())) + metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(encodeIndexFileDur.Milliseconds())) return serializedIndexBlobs, nil } @@ -593,8 +592,7 @@ func (it *IndexBuildTask) Execute(ctx context.Context) error { return err } saveIndexFileDur := it.tr.Record("index file save done") - metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), - strconv.FormatInt(it.segmentID, 10)).Observe(float64(saveIndexFileDur.Milliseconds())) + metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(saveIndexFileDur.Milliseconds())) it.tr.Elapse("index building all done") log.Info("IndexNode CreateIndex successfully ", zap.Int64("collect", it.collectionID), zap.Int64("partition", it.partitionID), zap.Int64("segment", it.segmentID)) diff --git a/internal/metrics/datacoord.go b/internal/metrics/datacoord.go new file mode 100644 index 0000000000..3125bda979 --- /dev/null +++ b/internal/metrics/datacoord.go @@ -0,0 +1,138 @@ +// 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 metrics + +import ( + "github.com/milvus-io/milvus/internal/util/typeutil" + "github.com/prometheus/client_golang/prometheus" +) + +const ( + CompactTypeI = "compactTypeI" + CompactTypeII = "compactTypeII" + CompactInputLabel = "input" + CompactInput2Label = "input2" + CompactOutputLabel = "output" + compactIOLabelName = "IO" + compactTypeLabelName = "compactType" +) + +var ( + //DataCoordNumDataNodes records the num of data nodes managed by DataCoord. + DataCoordNumDataNodes = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "num_datanodes", + Help: "Number of data nodes managed by DataCoord", + }, []string{}) + + DataCoordNumSegments = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "num_segments", + Help: "number of segments", + }, []string{ + segmentTypeLabelName, + }) + + //DataCoordCollectionNum records the num of collections managed by DataCoord. + DataCoordNumCollections = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "num_collections", + Help: "Number of collections", + }, []string{}) + + DataCoordNumStoredRows = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "num_stored_rows", + Help: "number of stored rows", + }, []string{}) + + DataCoordSyncUTC = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "sync_utc_time", + Help: "sync timestamp", + }, []string{}) + + /* hard to implement, commented now + DataCoordSegmentSizeRatio = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "segment_size_ratio", + Help: "size ratio compared to the configuration size", + Buckets: prometheus.LinearBuckets(0.0, 0.1, 15), + }, []string{}) + + DataCoordSegmentFlushDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "segment_flush_duration", + Help: "time spent on each segment flush", + Buckets: []float64{0.1, 0.5, 1, 5, 10, 20, 50, 100, 250, 500, 1000, 3600, 5000, 10000}, // unit seconds + }, []string{}) + + DataCoordCompactDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "segment_compact_duration", + Help: "time spent on each segment flush", + Buckets: []float64{0.1, 0.5, 1, 5, 10, 20, 50, 100, 250, 500, 1000, 3600, 5000, 10000}, // unit seconds + }, []string{compactTypeLabelName}) + + DataCoordCompactLoad = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "compaction_load", + Help: "Information on the input and output of compaction", + }, []string{compactTypeLabelName, compactIOLabelName}) + + DataCoordNumCompactionTask = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataCoordRole, + Name: "num_compaction_tasks", + Help: "Number of compaction tasks currently", + }, []string{statusLabelName}) + */ + +) + +//RegisterDataCoord registers DataCoord metrics +func RegisterDataCoord() { + prometheus.MustRegister(DataCoordNumDataNodes) + prometheus.MustRegister(DataCoordNumSegments) + prometheus.MustRegister(DataCoordNumCollections) + prometheus.MustRegister(DataCoordNumStoredRows) + prometheus.MustRegister(DataCoordSyncUTC) + // prometheus.MustRegister(DataCoordSegmentSizeRatio) + // prometheus.MustRegister(DataCoordSegmentFlushDuration) + // prometheus.MustRegister(DataCoordCompactDuration) + // prometheus.MustRegister(DataCoordCompactLoad) + // prometheus.MustRegister(DataCoordNumCompactionTask) +} diff --git a/internal/metrics/datanode.go b/internal/metrics/datanode.go index 488b635beb..1a849f43cf 100644 --- a/internal/metrics/datanode.go +++ b/internal/metrics/datanode.go @@ -39,8 +39,8 @@ var ( Name: "message_rows_count", Help: "Messages rows size count consumed from msgStream in DataNode.", }, []string{ - msgTypeLabelName, nodeIDLabelName, + msgTypeLabelName, }) DataNodeFlushedSize = prometheus.NewCounterVec( @@ -50,31 +50,31 @@ var ( Name: "flushed_size", Help: "Data size flushed to storage in DataNode.", }, []string{ + nodeIDLabelName, msgTypeLabelName, - nodeIDLabelName, }) - DataNodeNumDmlChannels = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.DataNodeRole, - Name: "num_dml_channels", - Help: "Number of dmlChannels per collection in DataNode.", - }, []string{ - collectionIDLabelName, - nodeIDLabelName, - }) - - DataNodeNumDeltaChannels = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.DataNodeRole, - Name: "num_delta_channels", - Help: "Number of deltaChannels per collection in DataNode.", - }, []string{ - collectionIDLabelName, - nodeIDLabelName, - }) + //DataNodeNumDmlChannels = prometheus.NewGaugeVec( + // prometheus.GaugeOpts{ + // Namespace: milvusNamespace, + // Subsystem: typeutil.DataNodeRole, + // Name: "num_dml_channels", + // Help: "Number of dmlChannels per collection in DataNode.", + // }, []string{ + // collectionIDLabelName, + // nodeIDLabelName, + // }) + // + //DataNodeNumDeltaChannels = prometheus.NewGaugeVec( + // prometheus.GaugeOpts{ + // Namespace: milvusNamespace, + // Subsystem: typeutil.DataNodeRole, + // Name: "num_delta_channels", + // Help: "Number of deltaChannels per collection in DataNode.", + // }, []string{ + // collectionIDLabelName, + // nodeIDLabelName, + // }) DataNodeNumConsumers = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -83,7 +83,6 @@ var ( Name: "num_consumers", Help: "Number of consumers per collection in DataNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -94,7 +93,6 @@ var ( Name: "num_producers", Help: "Number of producers per collection in DataNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -105,8 +103,8 @@ var ( Name: "time_sync", Help: "Synchronized timestamps per channel in DataNode.", }, []string{ - channelNameLabelName, nodeIDLabelName, + channelNameLabelName, }) DataNodeSegmentRowsCount = prometheus.NewCounterVec( @@ -116,7 +114,6 @@ var ( Name: "seg_rows_count", Help: "Rows count of segments which sent to DataCoord from DataNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -127,7 +124,6 @@ var ( Name: "num_unflushed_segments", Help: "Number of unflushed segments in DataNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -139,7 +135,6 @@ var ( Help: "The flush segment latency in DataNode.", Buckets: buckets, }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -151,8 +146,8 @@ var ( Help: "The latency saving flush data to storage in DataNode.", Buckets: []float64{0, 10, 100, 200, 400, 1000, 10000}, }, []string{ - msgTypeLabelName, nodeIDLabelName, + msgTypeLabelName, }) DataNodeFlushSegmentCount = prometheus.NewCounterVec( // TODO: arguably @@ -162,8 +157,8 @@ var ( Name: "flush_segment_count", Help: "Flush segment statistics in DataNode.", }, []string{ - statusLabelName, nodeIDLabelName, + statusLabelName, }) DataNodeAutoFlushSegmentCount = prometheus.NewCounterVec( // TODO: arguably @@ -173,7 +168,6 @@ var ( Name: "auto_flush_segment_count", Help: "Auto flush segment statistics in DataNode.", }, []string{ - channelNameLabelName, nodeIDLabelName, }) @@ -185,9 +179,20 @@ var ( Help: "Compaction latency in DataNode.", Buckets: buckets, }, []string{ - collectionIDLabelName, nodeIDLabelName, }) + + // DataNodeFlushSegmentsReqCounter counts the num of calls of FlushSegments + DataNodeFlushSegmentsReqCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.DataNodeRole, + Name: "flush_segments_total", + Help: "Counter of flush segments", + }, []string{ + nodeIDLabelName, + statusLabelName, + }) ) //RegisterDataNode registers DataNode metrics @@ -195,8 +200,8 @@ func RegisterDataNode() { prometheus.MustRegister(DataNodeNumFlowGraphs) prometheus.MustRegister(DataNodeConsumeMsgRowsCount) prometheus.MustRegister(DataNodeFlushedSize) - prometheus.MustRegister(DataNodeNumDmlChannels) - prometheus.MustRegister(DataNodeNumDeltaChannels) + //prometheus.MustRegister(DataNodeNumDmlChannels) + //prometheus.MustRegister(DataNodeNumDeltaChannels) prometheus.MustRegister(DataNodeNumConsumers) prometheus.MustRegister(DataNodeNumProducers) prometheus.MustRegister(DataNodeTimeSync) @@ -207,4 +212,5 @@ func RegisterDataNode() { prometheus.MustRegister(DataNodeFlushSegmentCount) prometheus.MustRegister(DataNodeAutoFlushSegmentCount) prometheus.MustRegister(DataNodeCompactionLatency) + prometheus.MustRegister(DataNodeFlushSegmentsReqCounter) } diff --git a/internal/metrics/indexcoord_metrics.go b/internal/metrics/indexcoord_metrics.go index 52a3effa5d..3f2372a6a0 100644 --- a/internal/metrics/indexcoord_metrics.go +++ b/internal/metrics/indexcoord_metrics.go @@ -38,7 +38,7 @@ var ( Subsystem: typeutil.IndexCoordRole, Name: "index_task_counter", Help: "The number of index tasks of each type", - }, []string{"index_task_status"}) + }, []string{indexTaskStatusLabelName}) // IndexCoordIndexNodeNum records the number of IndexNodes managed by IndexCoord. IndexCoordIndexNodeNum = prometheus.NewGaugeVec( @@ -47,7 +47,7 @@ var ( Subsystem: typeutil.IndexCoordRole, Name: "index_node_num", Help: "The number of IndexNodes managed by IndexCoord", - }, []string{"type"}) + }, []string{}) ) //RegisterIndexCoord registers IndexCoord metrics diff --git a/internal/metrics/indexnode_metrics.go b/internal/metrics/indexnode_metrics.go index 75f75e7723..e5f3413263 100644 --- a/internal/metrics/indexnode_metrics.go +++ b/internal/metrics/indexnode_metrics.go @@ -37,7 +37,7 @@ var ( Name: "load_segment_latency", Help: "The latency of loading the segment", Buckets: buckets, - }, []string{nodeIDLabelName, segmentIDLabelName}) + }, []string{nodeIDLabelName}) IndexNodeDecodeBinlogLatency = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -46,7 +46,7 @@ var ( Name: "decode_binlog_latency", Help: "The latency of decode the binlog", Buckets: buckets, - }, []string{nodeIDLabelName, segmentIDLabelName}) + }, []string{nodeIDLabelName}) IndexNodeKnowhereBuildIndexLatency = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -55,7 +55,7 @@ var ( Name: "knowhere_build_index_latency", Help: "The latency of knowhere building the index", Buckets: buckets, - }, []string{nodeIDLabelName, segmentIDLabelName}) + }, []string{nodeIDLabelName}) IndexNodeEncodeIndexFileLatency = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -64,7 +64,7 @@ var ( Name: "encode_index_file_latency", Help: "The latency of encoding the index file", Buckets: buckets, - }, []string{nodeIDLabelName, segmentIDLabelName}) + }, []string{nodeIDLabelName}) IndexNodeSaveIndexFileLatency = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -73,7 +73,7 @@ var ( Name: "save_index_file_latency", Help: "The latency of saving the index file", Buckets: buckets, - }, []string{nodeIDLabelName, segmentIDLabelName}) + }, []string{nodeIDLabelName}) ) //RegisterIndexNode registers IndexNode metrics diff --git a/internal/metrics/metrics.go b/internal/metrics/metrics.go index 61d3ab23fe..77dbc08811 100644 --- a/internal/metrics/metrics.go +++ b/internal/metrics/metrics.go @@ -23,8 +23,6 @@ import ( _ "net/http/pprof" "github.com/milvus-io/milvus/internal/log" - "github.com/milvus-io/milvus/internal/util/typeutil" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "go.uber.org/zap" @@ -52,18 +50,21 @@ const ( FailedIndexTaskLabel = "failed" RecycledIndexTaskLabel = "recycled" - SealedSegmentLabel = "sealed" - GrowingSegmentLabel = "growing" + SealedSegmentLabel = "Sealed" + GrowingSegmentLabel = "Growing" + FlushedSegmentLabel = "Flushed" + FlushingSegmentLabel = "Flushing" + DropedSegmentLabel = "Dropped" - nodeIDLabelName = "node_id" - statusLabelName = "status" - msgTypeLabelName = "msg_type" - collectionIDLabelName = "collection_id" - channelNameLabelName = "channel_name" - segmentIDLabelName = "segment_id" - functionLabelName = "function_name" - queryTypeLabelName = "query_type" - segmentTypeLabelName = "segment_type" + nodeIDLabelName = "node_id" + statusLabelName = "status" + indexTaskStatusLabelName = "index_task_status" + msgTypeLabelName = "msg_type" + collectionIDLabelName = "collection_id" + channelNameLabelName = "channel_name" + functionLabelName = "function_name" + queryTypeLabelName = "query_type" + segmentTypeLabelName = "segment_type" ) var ( @@ -72,43 +73,6 @@ var ( buckets = prometheus.ExponentialBuckets(1, 2, 18) ) -var ( - //DataCoordDataNodeList records the num of regsitered data nodes - DataCoordDataNodeList = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.DataCoordRole, - Name: "list_of_data_node", - Help: "List of data nodes registered within etcd", - }, []string{"status"}, - ) -) - -//RegisterDataCoord registers DataCoord metrics -func RegisterDataCoord() { - prometheus.MustRegister(DataCoordDataNodeList) -} - -var ( - // DataNodeFlushSegmentsCounter counts the num of calls of FlushSegments - DataNodeFlushSegmentsCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.DataNodeRole, - Name: "flush_segments_total", - Help: "Counter of flush segments", - }, []string{"type"}) - - // DataNodeWatchDmChannelsCounter counts the num of calls of WatchDmChannels - DataNodeWatchDmChannelsCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.DataNodeRole, - Name: "watch_dm_channels_total", - Help: "Counter of watch dm channel", - }, []string{"type"}) -) - //ServeHTTP serves prometheus http service func ServeHTTP() { http.Handle("/metrics", promhttp.Handler()) diff --git a/internal/metrics/proxy_metrics.go b/internal/metrics/proxy_metrics.go index 12a3dcffb3..d3672e456d 100644 --- a/internal/metrics/proxy_metrics.go +++ b/internal/metrics/proxy_metrics.go @@ -22,15 +22,6 @@ import ( ) var ( - // ProxyDmlChannelTimeTick counts the time tick value of dml channels - ProxyDmlChannelTimeTick = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.ProxyRole, - Name: "dml_channels_time_tick", - Help: "Time tick of dml channels", - }, []string{nodeIDLabelName, "pchan"}) - // ProxySearchCount record the number of times search succeeded or failed. ProxySearchCount = prometheus.NewCounterVec( prometheus.CounterOpts{ @@ -38,7 +29,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "search_counter", Help: "The number of times search succeeded or failed", - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName, statusLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName, statusLabelName}) // ProxyInsertCount record the number of times insert succeeded or failed. ProxyInsertCount = prometheus.NewCounterVec( @@ -47,7 +38,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "insert_counter", Help: "The number of times insert succeeded or failed", - }, []string{nodeIDLabelName, collectionIDLabelName, statusLabelName}) + }, []string{nodeIDLabelName, statusLabelName}) // ProxySearchVectors record the number of vectors search successfully. ProxySearchVectors = prometheus.NewGaugeVec( @@ -56,7 +47,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "search_vectors", Help: "The number of vectors search successfully", - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxyInsertVectors record the number of vectors insert successfully. ProxyInsertVectors = prometheus.NewGaugeVec( @@ -65,17 +56,17 @@ var ( Subsystem: typeutil.ProxyRole, Name: "insert_vectors", Help: "The number of vectors insert successfully", - }, []string{nodeIDLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName}) // ProxyLinkedSDKs record The number of SDK linked proxy. // TODO: how to know when sdk disconnect? - ProxyLinkedSDKs = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.ProxyRole, - Name: "linked_sdk_numbers", - Help: "The number of SDK linked proxy", - }, []string{nodeIDLabelName}) + //ProxyLinkedSDKs = prometheus.NewGaugeVec( + // prometheus.GaugeOpts{ + // Namespace: milvusNamespace, + // Subsystem: typeutil.ProxyRole, + // Name: "linked_sdk_numbers", + // Help: "The number of SDK linked proxy", + // }, []string{nodeIDLabelName}) // ProxySearchLatency record the latency of search successfully. ProxySearchLatency = prometheus.NewHistogramVec( @@ -85,7 +76,7 @@ var ( Name: "search_latency", Help: "The latency of search successfully", Buckets: buckets, - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxySendMessageLatency record the latency that the proxy sent the search request to the message stream. ProxySendMessageLatency = prometheus.NewHistogramVec( @@ -95,7 +86,7 @@ var ( Name: "send_search_msg_time", Help: "The latency that the proxy sent the search request to the message stream", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxyWaitForSearchResultLatency record the time that the proxy waits for the search result. ProxyWaitForSearchResultLatency = prometheus.NewHistogramVec( @@ -105,7 +96,7 @@ var ( Name: "wait_for_search_result_time", Help: "The time that the proxy waits for the search result", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxyReduceSearchResultLatency record the time that the proxy reduces search result. ProxyReduceSearchResultLatency = prometheus.NewHistogramVec( @@ -115,7 +106,7 @@ var ( Name: "reduce_search_result_time", Help: "The time that the proxy reduces search result", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxyDecodeSearchResultLatency record the time that the proxy decodes the search result. ProxyDecodeSearchResultLatency = prometheus.NewHistogramVec( @@ -125,7 +116,7 @@ var ( Name: "decode_search_result_time", Help: "The time that the proxy decodes the search result", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxyMsgStreamObjectsForPChan record the number of MsgStream objects per PChannel on each collection_id on Proxy. ProxyMsgStreamObjectsForPChan = prometheus.NewGaugeVec( @@ -134,7 +125,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "msg_stream_obj_for_PChan", Help: "The number of MsgStream objects per PChannel on each collection on Proxy", - }, []string{nodeIDLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName, channelNameLabelName}) // ProxyMsgStreamObjectsForSearch record the number of MsgStream objects for search per collection_id. ProxyMsgStreamObjectsForSearch = prometheus.NewGaugeVec( @@ -143,7 +134,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "msg_stream_obj_for_search", Help: "The number of MsgStream objects for search per collection", - }, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) + }, []string{nodeIDLabelName, queryTypeLabelName}) // ProxyInsertLatency record the latency that insert successfully. ProxyInsertLatency = prometheus.NewHistogramVec( @@ -153,17 +144,7 @@ var ( Name: "insert_latency", Help: "The latency that insert successfully.", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName}) - - // ProxyInsertColToRowLatency record the latency that column to row for inserting in Proxy. - ProxyInsertColToRowLatency = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.ProxyRole, - Name: "col_to_row_latency", - Help: "The time that column to row for inserting in Proxy", - Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName}) // ProxySendInsertReqLatency record the latency that Proxy send insert request to MsgStream. ProxySendInsertReqLatency = prometheus.NewHistogramVec( @@ -173,10 +154,9 @@ var ( Name: "send_insert_req_latency", Help: "The latency that Proxy send insert request to MsgStream", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName}) // ProxyCacheHitCounter record the number of Proxy cache hits or miss. - // TODO: @xiaocai2333 add more cache type ProxyCacheHitCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: milvusNamespace, @@ -240,7 +220,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "DQL_call_counter", Help: "", - }, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName, statusLabelName}) + }, []string{nodeIDLabelName, functionLabelName, statusLabelName}) // ProxyDMLFunctionCall records the number of times the function of the DML operation was executed, like `LoadCollection`. ProxyDMLFunctionCall = prometheus.NewCounterVec( @@ -249,7 +229,7 @@ var ( Subsystem: typeutil.ProxyRole, Name: "DML_call_counter", Help: "", - }, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName, statusLabelName}) + }, []string{nodeIDLabelName, functionLabelName, statusLabelName}) // ProxyDDLReqLatency records the latency that for DML request, like "CreateCollection". ProxyDDLReqLatency = prometheus.NewHistogramVec( @@ -269,7 +249,7 @@ var ( Name: "DML_call_latency", Help: "The latency that for DML request", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName, functionLabelName}) // ProxyDQLReqLatency record the latency that for DQL request, like "HasCollection". ProxyDQLReqLatency = prometheus.NewHistogramVec( @@ -279,7 +259,7 @@ var ( Name: "DQL_call_latency", Help: "The latency that for DQL request", Buckets: buckets, // unit: ms - }, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName, functionLabelName}) // ProxySearchLatencyPerNQ records the latency for searching. ProxySearchLatencyPerNQ = prometheus.NewHistogramVec( @@ -289,19 +269,17 @@ var ( Name: "proxy_search_latency_count", Help: "The latency for searching", Buckets: buckets, - }, []string{nodeIDLabelName, collectionIDLabelName}) + }, []string{nodeIDLabelName}) ) //RegisterProxy registers Proxy metrics func RegisterProxy() { - prometheus.MustRegister(ProxyDmlChannelTimeTick) - prometheus.MustRegister(ProxySearchCount) prometheus.MustRegister(ProxyInsertCount) prometheus.MustRegister(ProxySearchVectors) prometheus.MustRegister(ProxyInsertVectors) - prometheus.MustRegister(ProxyLinkedSDKs) + //prometheus.MustRegister(ProxyLinkedSDKs) prometheus.MustRegister(ProxySearchLatency) prometheus.MustRegister(ProxySearchLatencyPerNQ) @@ -314,7 +292,6 @@ func RegisterProxy() { prometheus.MustRegister(ProxyMsgStreamObjectsForSearch) prometheus.MustRegister(ProxyInsertLatency) - prometheus.MustRegister(ProxyInsertColToRowLatency) prometheus.MustRegister(ProxySendInsertReqLatency) prometheus.MustRegister(ProxyCacheHitCounter) diff --git a/internal/metrics/querycoord.go b/internal/metrics/querycoord.go index 3b903312c9..3e8124eceb 100644 --- a/internal/metrics/querycoord.go +++ b/internal/metrics/querycoord.go @@ -37,9 +37,7 @@ var ( Subsystem: typeutil.QueryCoordRole, Name: "num_entities", Help: "Number of entities in collection.", - }, []string{ - collectionIDLabelName, - }) + }, []string{}) QueryCoordLoadCount = prometheus.NewCounterVec( prometheus.CounterOpts{ diff --git a/internal/metrics/querynode.go b/internal/metrics/querynode.go index 8df658a894..925860c395 100644 --- a/internal/metrics/querynode.go +++ b/internal/metrics/querynode.go @@ -40,7 +40,6 @@ var ( Name: "num_partitions", Help: "Number of partitions per collection in QueryNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -51,7 +50,6 @@ var ( Name: "num_segments", Help: "Number of segments per collection in QueryNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -62,7 +60,6 @@ var ( Name: "num_dml_channels", Help: "Number of dmlChannels per collection in QueryNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -73,7 +70,6 @@ var ( Name: "num_delta_channels", Help: "Number of deltaChannels per collection in QueryNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -84,18 +80,6 @@ var ( Name: "num_consumers", Help: "Number of consumers per collection in QueryNode.", }, []string{ - collectionIDLabelName, - nodeIDLabelName, - }) - - QueryNodeNumReaders = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: milvusNamespace, - Subsystem: typeutil.QueryNodeRole, - Name: "num_readers", - Help: "Number of readers per collection in QueryNode.", - }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -106,9 +90,9 @@ var ( Name: "sq_count", Help: "Search and query requests statistic in QueryNode.", }, []string{ - statusLabelName, - queryTypeLabelName, nodeIDLabelName, + queryTypeLabelName, + statusLabelName, }) QueryNodeSQReqLatency = prometheus.NewHistogramVec( @@ -119,8 +103,8 @@ var ( Help: "Search and query requests latency in QueryNode.", Buckets: buckets, }, []string{ - queryTypeLabelName, nodeIDLabelName, + queryTypeLabelName, }) QueryNodeSQLatencyInQueue = prometheus.NewHistogramVec( @@ -131,8 +115,8 @@ var ( Help: "The search and query latency in queue(unsolved buffer) in QueryNode.", Buckets: buckets, }, []string{ - queryTypeLabelName, nodeIDLabelName, + queryTypeLabelName, }) QueryNodeSQSegmentLatency = prometheus.NewHistogramVec( @@ -143,9 +127,9 @@ var ( Help: "The search and query on segments(sealed/growing segments).", Buckets: buckets, }, []string{ + nodeIDLabelName, queryTypeLabelName, segmentTypeLabelName, - nodeIDLabelName, }) QueryNodeSQSegmentLatencyInCore = prometheus.NewHistogramVec( @@ -156,8 +140,8 @@ var ( Help: "The search and query latency in core.", Buckets: buckets, }, []string{ - queryTypeLabelName, nodeIDLabelName, + queryTypeLabelName, }) QueryNodeTranslateHitsLatency = prometheus.NewHistogramVec( @@ -179,8 +163,8 @@ var ( Help: "The search and query latency in reduce(local reduce) in QueryNode.", Buckets: buckets, }, []string{ - segmentTypeLabelName, nodeIDLabelName, + segmentTypeLabelName, }) QueryNodeLoadSegmentLatency = prometheus.NewHistogramVec( @@ -201,7 +185,6 @@ var ( Name: "service_time", Help: "ServiceTimes of collections in QueryNode.", }, []string{ - collectionIDLabelName, nodeIDLabelName, }) @@ -224,7 +207,6 @@ func RegisterQueryNode() { prometheus.MustRegister(QueryNodeNumDmlChannels) prometheus.MustRegister(QueryNodeNumDeltaChannels) prometheus.MustRegister(QueryNodeNumConsumers) - prometheus.MustRegister(QueryNodeNumReaders) prometheus.MustRegister(QueryNodeSQCount) prometheus.MustRegister(QueryNodeSQReqLatency) prometheus.MustRegister(QueryNodeSQLatencyInQueue) diff --git a/internal/metrics/rootcoord_metrics.go b/internal/metrics/rootcoord_metrics.go index b720cd8592..86c70c4c80 100644 --- a/internal/metrics/rootcoord_metrics.go +++ b/internal/metrics/rootcoord_metrics.go @@ -163,7 +163,7 @@ var ( Subsystem: typeutil.RootCoordRole, Name: "ddl_read_type_latency", Help: "The latency for read type of DDL operations", - }, []string{functionLabelName, collectionIDLabelName}) + }, []string{functionLabelName}) // RootCoordDDLWriteTypeLatency records the latency for write type of DDL operations. RootCoordDDLWriteTypeLatency = prometheus.NewHistogramVec( @@ -172,7 +172,7 @@ var ( Subsystem: typeutil.RootCoordRole, Name: "ddl_write_type_latency", Help: "The latency for write type of DDL operations", - }, []string{functionLabelName, collectionIDLabelName}) + }, []string{functionLabelName}) // RootCoordSyncTimeTickLatency records the latency of sync time tick. RootCoordSyncTimeTickLatency = prometheus.NewHistogram( diff --git a/internal/proxy/channels_mgr.go b/internal/proxy/channels_mgr.go index 7cd1ee4a3f..d824c77a42 100644 --- a/internal/proxy/channels_mgr.go +++ b/internal/proxy/channels_mgr.go @@ -399,9 +399,9 @@ func (mgr *singleTypeChannelsMgr) createMsgStream(collectionID UniqueID) error { mgr.addStream(id, stream) mgr.updateCollection(collectionID, id) - - metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(collectionID, 10), "PChan").Inc() - + for _, pc := range pchans { + metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), pc).Inc() + } return nil } @@ -423,16 +423,20 @@ func (mgr *singleTypeChannelsMgr) getStream(collectionID UniqueID) (msgstream.Ms } func (mgr *singleTypeChannelsMgr) removeStream(collectionID UniqueID) error { - ids, err := mgr.getAllVIDs(collectionID) + channels, err := mgr.getChannels(collectionID) if err != nil { return err } + ids, err2 := mgr.getAllVIDs(collectionID) + if err2 != nil { + return err2 + } mgr.deleteVChansByVIDs(ids) mgr.deleteStreamByVIDs(ids) - - metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(collectionID, 10), "PChan").Dec() - + for _, pc := range channels { + metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), pc).Dec() + } return nil } @@ -482,7 +486,7 @@ func (mgr *channelsMgrImpl) getVChannels(collectionID UniqueID) ([]vChan, error) } func (mgr *channelsMgrImpl) createDQLStream(collectionID UniqueID) error { - metrics.ProxyMsgStreamObjectsForSearch.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), strconv.FormatInt(collectionID, 10), "query").Inc() + metrics.ProxyMsgStreamObjectsForSearch.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "query").Inc() return mgr.dqlChannelsMgr.createMsgStream(collectionID) } @@ -491,7 +495,7 @@ func (mgr *channelsMgrImpl) getDQLStream(collectionID UniqueID) (msgstream.MsgSt } func (mgr *channelsMgrImpl) removeDQLStream(collectionID UniqueID) error { - metrics.ProxyMsgStreamObjectsForSearch.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), strconv.FormatInt(collectionID, 10), "query").Dec() + metrics.ProxyMsgStreamObjectsForSearch.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "query").Dec() return mgr.dqlChannelsMgr.removeStream(collectionID) } diff --git a/internal/proxy/channels_time_ticker.go b/internal/proxy/channels_time_ticker.go index e57a1bbf4b..e95c2fab39 100644 --- a/internal/proxy/channels_time_ticker.go +++ b/internal/proxy/channels_time_ticker.go @@ -18,14 +18,9 @@ package proxy import ( "context" - "strconv" "sync" "time" - "github.com/milvus-io/milvus/internal/util/timerecord" - - "github.com/milvus-io/milvus/internal/metrics" - "go.uber.org/zap" "github.com/milvus-io/milvus/internal/log" @@ -95,17 +90,15 @@ func (ticker *channelsTimeTickerImpl) initCurrents(current Timestamp) { } func (ticker *channelsTimeTickerImpl) tick() error { - tr := timerecord.NewTimeRecorder("applyTimestamp") now, err := ticker.tso.AllocOne() if err != nil { log.Warn("Proxy channelsTimeTickerImpl failed to get ts from tso", zap.Error(err)) return err } - metrics.ProxyApplyTimestampLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) - stats, err := ticker.getStatisticsFunc() - if err != nil { - log.Debug("Proxy channelsTimeTickerImpl failed to getStatistics", zap.Error(err)) + stats, err2 := ticker.getStatisticsFunc() + if err2 != nil { + log.Debug("Proxy channelsTimeTickerImpl failed to getStatistics", zap.Error(err2)) return nil } diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index c9f5b6247f..313d2dd6b3 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -348,7 +348,7 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle method := "HasCollection" tr := timerecord.NewTimeRecorder(method) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() log.Debug("HasCollection received", zap.String("traceID", traceID), @@ -372,7 +372,7 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.BoolResponse{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -402,7 +402,7 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.BoolResponse{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -421,9 +421,8 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return hct.result, nil } @@ -461,7 +460,7 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol zap.String("collection", request.CollectionName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: err.Error(), @@ -489,9 +488,9 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol zap.String("collection", request.CollectionName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lct.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lct.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: err.Error(), @@ -508,11 +507,10 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol zap.String("collection", request.CollectionName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lct.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lct.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lct.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return lct.result, nil } @@ -553,7 +551,7 @@ func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.Rele zap.String("collection", request.CollectionName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: err.Error(), @@ -583,9 +581,9 @@ func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.Rele zap.String("collection", request.CollectionName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rct.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rct.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: err.Error(), @@ -603,11 +601,10 @@ func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.Rele zap.String("collection", request.CollectionName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rct.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rct.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rct.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return rct.result, nil } @@ -647,7 +644,7 @@ func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.Des zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.DescribeCollectionResponse{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -677,9 +674,9 @@ func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.Des zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dct.CollectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dct.CollectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.DescribeCollectionResponse{ Status: &commonpb.Status{ @@ -699,11 +696,10 @@ func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.Des zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dct.CollectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dct.CollectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dct.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return dct.result, nil } @@ -743,7 +739,7 @@ func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvusp zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.GetCollectionStatisticsResponse{ Status: &commonpb.Status{ @@ -774,9 +770,9 @@ func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvusp zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.GetCollectionStatisticsResponse{ Status: &commonpb.Status{ @@ -796,11 +792,10 @@ func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvusp zap.String("collection", request.CollectionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return g.result, nil } @@ -1102,7 +1097,7 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit tr := timerecord.NewTimeRecorder(method) //TODO: use collectionID instead of collectionName metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() hpt := &hasPartitionTask{ ctx: ctx, @@ -1131,7 +1126,7 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit zap.String("partition", request.PartitionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.BoolResponse{ Status: &commonpb.Status{ @@ -1167,7 +1162,7 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit zap.String("partition", request.PartitionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.BoolResponse{ Status: &commonpb.Status{ @@ -1190,9 +1185,8 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit zap.String("partition", request.PartitionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return hpt.result, nil } @@ -1234,7 +1228,7 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar zap.Any("partitions", request.PartitionNames)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1267,9 +1261,9 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar zap.Any("partitions", request.PartitionNames)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lpt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lpt.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1289,11 +1283,10 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar zap.Any("partitions", request.PartitionNames)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lpt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lpt.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(lpt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return lpt.result, nil } @@ -1336,7 +1329,7 @@ func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.Rele zap.Any("partitions", request.PartitionNames)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1369,9 +1362,9 @@ func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.Rele zap.Any("partitions", request.PartitionNames)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rpt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rpt.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1391,11 +1384,10 @@ func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.Rele zap.Any("partitions", request.PartitionNames)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rpt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rpt.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(rpt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return rpt.result, nil } @@ -1440,7 +1432,7 @@ func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb zap.String("partition", request.PartitionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.GetPartitionStatisticsResponse{ Status: &commonpb.Status{ @@ -1475,9 +1467,9 @@ func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb zap.String("partition", request.PartitionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.GetPartitionStatisticsResponse{ Status: &commonpb.Status{ @@ -1499,11 +1491,10 @@ func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb zap.String("partition", request.PartitionName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(g.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return g.result, nil } @@ -1532,7 +1523,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar tr := timerecord.NewTimeRecorder(method) //TODO: use collectionID instead of collectionName metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() log.Debug( rpcReceived(method), @@ -1549,7 +1540,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar zap.Any("request", request)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.ShowPartitionsResponse{ Status: &commonpb.Status{ @@ -1584,7 +1575,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.ShowPartitionsResponse{ Status: &commonpb.Status{ @@ -1606,9 +1597,8 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return spt.result, nil } @@ -1653,7 +1643,7 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde zap.Any("extra_params", request.ExtraParams)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1688,9 +1678,9 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde zap.Any("extra_params", request.ExtraParams)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(cit.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(cit.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1711,11 +1701,10 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde zap.Any("extra_params", request.ExtraParams)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(cit.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(cit.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(cit.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return cit.result, nil } @@ -1764,7 +1753,7 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe zap.String("index name", indexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.DescribeIndexResponse{ Status: &commonpb.Status{ @@ -1805,9 +1794,9 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe errCode = dit.result.Status.GetErrorCode() } metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.DescribeIndexResponse{ Status: &commonpb.Status{ @@ -1830,11 +1819,10 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe zap.String("index name", indexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return dit.result, nil } @@ -1878,7 +1866,7 @@ func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexReq zap.String("field", request.FieldName), zap.String("index name", request.IndexName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1913,9 +1901,9 @@ func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexReq zap.String("index name", request.IndexName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1936,11 +1924,10 @@ func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexReq zap.String("index name", request.IndexName)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dit.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return dit.result, nil } @@ -1989,7 +1976,7 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb. zap.String("field", request.FieldName), zap.String("index name", request.IndexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.GetIndexBuildProgressResponse{ Status: &commonpb.Status{ @@ -2025,9 +2012,9 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb. zap.String("field", request.FieldName), zap.String("index name", request.IndexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(gibpt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(gibpt.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.GetIndexBuildProgressResponse{ Status: &commonpb.Status{ @@ -2051,11 +2038,10 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb. zap.Any("result", gibpt.result)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(gibpt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(gibpt.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(gibpt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return gibpt.result, nil } @@ -2103,7 +2089,7 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex zap.String("index name", request.IndexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.AbandonLabel).Inc() + metrics.AbandonLabel).Inc() return &milvuspb.GetIndexStateResponse{ Status: &commonpb.Status{ @@ -2140,9 +2126,9 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex zap.String("index name", request.IndexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dipt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dipt.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.GetIndexStateResponse{ Status: &commonpb.Status{ @@ -2165,11 +2151,10 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex zap.String("index name", request.IndexName)) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dipt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dipt.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dipt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return dipt.result, nil } @@ -2245,7 +2230,7 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest) if err := node.sched.dmQueue.Enqueue(it); err != nil { log.Debug("Failed to enqueue insert task: " + err.Error()) - metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), request.CollectionName, metrics.AbandonLabel).Inc() + metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.AbandonLabel).Inc() return constructFailedResponse(err), nil } @@ -2263,9 +2248,9 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest) if err := it.WaitToFinish(); err != nil { log.Debug("Failed to execute insert task in task scheduler: "+err.Error(), zap.String("traceID", traceID)) metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(it.CollectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(it.CollectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return constructFailedResponse(err), nil } @@ -2286,11 +2271,9 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest) it.result.InsertCnt = int64(it.req.NumRows) metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(it.CollectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyInsertVectors.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(it.CollectionID, 10)).Add(float64(it.result.InsertCnt)) - metrics.ProxyInsertLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(it.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyInsertVectors.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Add(float64(it.result.InsertCnt)) + metrics.ProxyInsertLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) return it.result, nil } @@ -2351,7 +2334,7 @@ func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest) if err := node.sched.dmQueue.Enqueue(dt); err != nil { log.Error("Failed to enqueue delete task: "+err.Error(), zap.String("traceID", traceID)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - request.CollectionName, metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.MutationResult{ Status: &commonpb.Status{ @@ -2374,9 +2357,9 @@ func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest) if err := dt.WaitToFinish(); err != nil { log.Error("Failed to execute delete task in task scheduler: "+err.Error(), zap.String("traceID", traceID)) metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dt.collectionID, 10), metrics.FailLabel).Inc() + metrics.FailLabel).Inc() return &milvuspb.MutationResult{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -2386,11 +2369,10 @@ func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest) } metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dt.collectionID, 10), metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dt.collectionID, 10), metrics.SuccessLabel).Inc() - metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - strconv.FormatInt(dt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return dt.result, nil } @@ -2459,7 +2441,7 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest) zap.Uint64("guarantee_timestamp", guaranteeTs)) metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - request.CollectionName, metrics.SearchLabel, metrics.AbandonLabel).Inc() + metrics.SearchLabel, metrics.AbandonLabel).Inc() return &milvuspb.SearchResults{ Status: &commonpb.Status{ @@ -2502,9 +2484,10 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest) zap.Uint64("travel_timestamp", travelTs), zap.Uint64("guarantee_timestamp", guaranteeTs)) metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.SearchLabel, metrics.TotalLabel).Inc() + metrics.SearchLabel, metrics.TotalLabel).Inc() - metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), request.CollectionName, metrics.SearchLabel, metrics.FailLabel).Inc() + metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), + metrics.SearchLabel, metrics.FailLabel).Inc() return &milvuspb.SearchResults{ Status: &commonpb.Status{ @@ -2530,16 +2513,15 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest) zap.Uint64("guarantee_timestamp", guaranteeTs)) metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.SearchLabel, metrics.TotalLabel).Inc() + metrics.SearchLabel, metrics.TotalLabel).Inc() metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.SearchLabel, metrics.SuccessLabel).Inc() + metrics.SearchLabel, metrics.SuccessLabel).Inc() metrics.ProxySearchVectors.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.SearchLabel).Set(float64(qt.result.Results.NumQueries)) + metrics.SearchLabel).Set(float64(qt.result.Results.NumQueries)) searchDur := tr.ElapseSpan().Milliseconds() metrics.ProxySearchLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.SearchLabel).Observe(float64(searchDur)) - metrics.ProxySearchLatencyPerNQ.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10)).Observe(float64(searchDur) / float64(qt.result.Results.NumQueries)) + metrics.SearchLabel).Observe(float64(searchDur)) + metrics.ProxySearchLatencyPerNQ.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(searchDur) / float64(qt.result.Results.NumQueries)) return qt.result, nil } @@ -2687,7 +2669,7 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (* zap.Any("partitions", request.PartitionNames)) metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - request.CollectionName, metrics.QueryLabel, metrics.FailLabel).Inc() + metrics.QueryLabel, metrics.FailLabel).Inc() return &milvuspb.QueryResults{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -2721,9 +2703,9 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (* zap.Any("partitions", request.PartitionNames)) metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.QueryLabel, metrics.TotalLabel).Inc() + metrics.QueryLabel, metrics.TotalLabel).Inc() metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.QueryLabel, metrics.FailLabel).Inc() + metrics.QueryLabel, metrics.FailLabel).Inc() return &milvuspb.QueryResults{ Status: &commonpb.Status{ @@ -2745,13 +2727,13 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (* zap.Any("partitions", request.PartitionNames)) metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.QueryLabel, metrics.TotalLabel).Inc() + metrics.QueryLabel, metrics.TotalLabel).Inc() metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.QueryLabel, metrics.SuccessLabel).Inc() + metrics.QueryLabel, metrics.SuccessLabel).Inc() metrics.ProxySearchVectors.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.QueryLabel).Set(float64(len(qt.result.FieldsData))) + metrics.QueryLabel).Set(float64(len(qt.result.FieldsData))) metrics.ProxySendMessageLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), - strconv.FormatInt(qt.CollectionID, 10), metrics.QueryLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.QueryLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) return &milvuspb.QueryResults{ Status: qt.result.Status, FieldsData: qt.result.FieldsData, @@ -3497,7 +3479,7 @@ func (node *Proxy) GetPersistentSegmentInfo(ctx context.Context, req *milvuspb.G method := "GetPersistentSegmentInfo" tr := timerecord.NewTimeRecorder(method) metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - req.CollectionName, metrics.TotalLabel).Inc() + metrics.TotalLabel).Inc() segments, err := node.getSegmentsOfCollection(ctx, req.DbName, req.CollectionName) if err != nil { resp.Status.Reason = fmt.Errorf("getSegmentsOfCollection, err:%w", err).Error() @@ -3533,9 +3515,8 @@ func (node *Proxy) GetPersistentSegmentInfo(ctx context.Context, req *milvuspb.G } } metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - req.CollectionName, metrics.SuccessLabel).Inc() - metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, - req.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.SuccessLabel).Inc() + metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds())) resp.Status.ErrorCode = commonpb.ErrorCode_Success resp.Infos = persistentInfos return resp, nil @@ -3725,7 +3706,7 @@ func (node *Proxy) RegisterLink(ctx context.Context, req *milvuspb.RegisterLinkR }, }, nil } - metrics.ProxyLinkedSDKs.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Inc() + //metrics.ProxyLinkedSDKs.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Inc() return &milvuspb.RegisterLinkResponse{ Address: nil, Status: &commonpb.Status{ diff --git a/internal/proxy/meta_cache.go b/internal/proxy/meta_cache.go index 80a267435d..c9b3f2b0bf 100644 --- a/internal/proxy/meta_cache.go +++ b/internal/proxy/meta_cache.go @@ -22,21 +22,17 @@ import ( "fmt" "strconv" "sync" - "time" - - "github.com/milvus-io/milvus/internal/util/timerecord" - - "github.com/milvus-io/milvus/internal/metrics" - - "github.com/milvus-io/milvus/internal/common" "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/timerecord" "github.com/milvus-io/milvus/internal/util/typeutil" ) @@ -110,7 +106,8 @@ func (m *MetaCache) GetCollectionID(ctx context.Context, collectionName string) collInfo, ok := m.collInfo[collectionName] if !ok { - metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollection", metrics.CacheMissLabel).Inc() + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GeCollectionID", metrics.CacheMissLabel).Inc() + tr := timerecord.NewTimeRecorder("UpdateCache") m.mu.RUnlock() coll, err := m.describeCollection(ctx, collectionName) if err != nil { @@ -118,14 +115,13 @@ func (m *MetaCache) GetCollectionID(ctx context.Context, collectionName string) } m.mu.Lock() defer m.mu.Unlock() - tr := timerecord.NewTimeRecorder("UpdateCache") m.updateCollection(coll, collectionName) metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) collInfo = m.collInfo[collectionName] return collInfo.collID, nil } defer m.mu.RUnlock() - metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollection", metrics.CacheHitLabel).Inc() + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollectionID", metrics.CacheHitLabel).Inc() return collInfo.collID, nil } @@ -139,6 +135,8 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, collectionName string m.mu.RUnlock() if !ok { + tr := timerecord.NewTimeRecorder("UpdateCache") + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollectionInfo", metrics.CacheMissLabel).Inc() coll, err := m.describeCollection(ctx, collectionName) if err != nil { return nil, err @@ -147,8 +145,9 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, collectionName string defer m.mu.Unlock() m.updateCollection(coll, collectionName) collInfo = m.collInfo[collectionName] + metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) } - + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollectionInfo", metrics.CacheHitLabel).Inc() return &collectionInfo{ collID: collInfo.collID, schema: collInfo.schema, @@ -163,7 +162,8 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, collectionName stri collInfo, ok := m.collInfo[collectionName] if !ok { - t0 := time.Now() + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollectionSchema", metrics.CacheMissLabel).Inc() + tr := timerecord.NewTimeRecorder("UpdateCache") m.mu.RUnlock() coll, err := m.describeCollection(ctx, collectionName) if err != nil { @@ -176,12 +176,14 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, collectionName stri defer m.mu.Unlock() m.updateCollection(coll, collectionName) collInfo = m.collInfo[collectionName] - log.Debug("Reload collection from rootcoord ", + metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + log.Debug("Reload collection from root coordinator ", zap.String("collection name ", collectionName), - zap.Any("time take ", time.Since(t0))) + zap.Any("time (milliseconds) take ", tr.ElapseSpan().Milliseconds())) return collInfo.schema, nil } defer m.mu.RUnlock() + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollectionSchema", metrics.CacheHitLabel).Inc() return collInfo.schema, nil } @@ -220,6 +222,8 @@ func (m *MetaCache) GetPartitions(ctx context.Context, collectionName string) (m } if collInfo.partInfo == nil || len(collInfo.partInfo) == 0 { + tr := timerecord.NewTimeRecorder("UpdateCache") + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetPartitions", metrics.CacheMissLabel).Inc() m.mu.RUnlock() partitions, err := m.showPartitions(ctx, collectionName) @@ -234,6 +238,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, collectionName string) (m if err != nil { return nil, err } + metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) log.Debug("proxy", zap.Any("GetPartitions:partitions after update", partitions), zap.Any("collectionName", collectionName)) ret := make(map[string]typeutil.UniqueID) partInfo := m.collInfo[collectionName].partInfo @@ -244,6 +249,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, collectionName string) (m } defer m.mu.RUnlock() + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetPartitions", metrics.CacheHitLabel).Inc() ret := make(map[string]typeutil.UniqueID) partInfo := m.collInfo[collectionName].partInfo @@ -273,6 +279,8 @@ func (m *MetaCache) GetPartitionInfo(ctx context.Context, collectionName string, m.mu.RUnlock() if !ok { + tr := timerecord.NewTimeRecorder("UpdateCache") + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetPartitionInfo", metrics.CacheMissLabel).Inc() partitions, err := m.showPartitions(ctx, collectionName) if err != nil { return nil, err @@ -284,13 +292,14 @@ func (m *MetaCache) GetPartitionInfo(ctx context.Context, collectionName string, if err != nil { return nil, err } + metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) log.Debug("proxy", zap.Any("GetPartitionID:partitions after update", partitions), zap.Any("collectionName", collectionName)) - partInfo, ok = m.collInfo[collectionName].partInfo[partitionName] if !ok { return nil, fmt.Errorf("partitionID of partitionName:%s can not be find", partitionName) } } + metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetPartitionInfo", metrics.CacheHitLabel).Inc() return &partitionInfo{ partitionID: partInfo.partitionID, createdTimestamp: partInfo.createdTimestamp, diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index 5b0584a832..7083cc4721 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -313,11 +313,7 @@ func (node *Proxy) sendChannelsTimeTickLoop() { DefaultTimestamp: maxTs, } - for idx, channel := range channels { - ts := tss[idx] - metrics.ProxyDmlChannelTimeTick.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), channel).Set(float64(ts)) - } - metrics.ProxyDmlChannelTimeTick.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "DefaultTimestamp").Set(float64(maxTs)) + metrics.ProxySyncTimeTick.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "DefaultTimestamp").Set(float64(maxTs)) status, err := node.rootCoord.UpdateChannelTimeTick(node.ctx, req) if err != nil { diff --git a/internal/proxy/task.go b/internal/proxy/task.go index c91f111d05..a60ade6b4f 100644 --- a/internal/proxy/task.go +++ b/internal/proxy/task.go @@ -833,7 +833,7 @@ func (it *insertTask) Execute(ctx context.Context) error { return err } sendMsgDur := tr.Record("send insert request to message stream") - metrics.ProxySendInsertReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), collectionName).Observe(float64(sendMsgDur.Milliseconds())) + metrics.ProxySendInsertReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(sendMsgDur.Milliseconds())) return nil } @@ -1495,7 +1495,8 @@ func (st *searchTask) Execute(ctx context.Context) error { zap.Int("length of search msg", len(msgPack.Msgs)), zap.Uint64("timeoutTs", st.SearchRequest.TimeoutTimestamp)) sendMsgDur := tr.Record("send search msg to message stream") - metrics.ProxySendMessageLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), collectionName, metrics.SearchLabel).Observe(float64(sendMsgDur.Milliseconds())) + metrics.ProxySendMessageLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), + metrics.SearchLabel).Observe(float64(sendMsgDur.Milliseconds())) return err } @@ -1712,7 +1713,7 @@ func (st *searchTask) PostExecute(ctx context.Context) error { log.Debug("Proxy Search PostExecute stage1", zap.Any("len(filterSearchResults)", len(filterSearchResults))) - metrics.ProxyWaitForSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), st.collectionName, metrics.SearchLabel).Observe(float64(st.tr.RecordSpan().Milliseconds())) + metrics.ProxyWaitForSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.SearchLabel).Observe(float64(st.tr.RecordSpan().Milliseconds())) tr.Record("Proxy Search PostExecute stage1 done") if len(filterSearchResults) <= 0 || errNum > 0 { st.result = &milvuspb.SearchResults{ @@ -1729,7 +1730,7 @@ func (st *searchTask) PostExecute(ctx context.Context) error { if err != nil { return err } - metrics.ProxyDecodeSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), st.collectionName, metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds())) + metrics.ProxyDecodeSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds())) log.Debug("Proxy Search PostExecute stage2", zap.Any("len(validSearchResults)", len(validSearchResults))) if len(validSearchResults) <= 0 { filterReason += "empty search result\n" @@ -1754,7 +1755,7 @@ func (st *searchTask) PostExecute(ctx context.Context) error { if err != nil { return err } - metrics.ProxyReduceSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), st.collectionName, metrics.SuccessLabel).Observe(float64(tr.RecordSpan().Milliseconds())) + metrics.ProxyReduceSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.SuccessLabel).Observe(float64(tr.RecordSpan().Milliseconds())) st.result.CollectionName = st.collectionName schema, err := globalMetaCache.GetCollectionSchema(ctx, st.query.CollectionName) diff --git a/internal/proxy/timestamp.go b/internal/proxy/timestamp.go index 754c5b6d68..a9a477c555 100644 --- a/internal/proxy/timestamp.go +++ b/internal/proxy/timestamp.go @@ -19,10 +19,13 @@ package proxy import ( "context" "fmt" + "strconv" "time" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/internal/util/timerecord" ) // timestampAllocator implements tsoAllocator. @@ -43,6 +46,7 @@ func newTimestampAllocator(ctx context.Context, tso timestampAllocatorInterface, } func (ta *timestampAllocator) alloc(count uint32) ([]Timestamp, error) { + tr := timerecord.NewTimeRecorder("applyTimestamp") ctx, cancel := context.WithTimeout(ta.ctx, 5*time.Second) req := &rootcoordpb.AllocTimestampRequest{ Base: &commonpb.MsgBase{ @@ -55,7 +59,10 @@ func (ta *timestampAllocator) alloc(count uint32) ([]Timestamp, error) { } resp, err := ta.tso.AllocTimestamp(ctx, req) - defer cancel() + defer func() { + cancel() + metrics.ProxyApplyTimestampLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + }() if err != nil { return nil, fmt.Errorf("syncTimestamp Failed:%w", err) diff --git a/internal/querycoord/meta.go b/internal/querycoord/meta.go index 2732cf439c..b2e37feecd 100644 --- a/internal/querycoord/meta.go +++ b/internal/querycoord/meta.go @@ -37,8 +37,8 @@ import ( "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/schemapb" - "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/util" + "github.com/milvus-io/milvus/internal/util/funcutil" ) const ( @@ -954,7 +954,7 @@ func (m *MetaReplica) getWatchedChannelsByNodeID(nodeID int64) *querypb.Unsubscr // get dmChannel/search channel the node has watched for _, channelInfo := range dmChannelInfos { collectionID := channelInfo.CollectionID - dmChannel := rootcoord.ToPhysicalChannel(channelInfo.DmChannel) + dmChannel := funcutil.ToPhysicalChannel(channelInfo.DmChannel) if _, ok := colID2DmChannels[collectionID]; !ok { colID2DmChannels[collectionID] = []string{} } @@ -977,7 +977,7 @@ func (m *MetaReplica) getWatchedChannelsByNodeID(nodeID int64) *querypb.Unsubscr } deltaChannels := make([]string, len(deltaChanelInfos)) for offset, channelInfo := range deltaChanelInfos { - deltaChannels[offset] = rootcoord.ToPhysicalChannel(channelInfo.ChannelName) + deltaChannels[offset] = funcutil.ToPhysicalChannel(channelInfo.ChannelName) } colID2DeltaChannels[collectionID] = deltaChannels } diff --git a/internal/querycoord/segments_info.go b/internal/querycoord/segments_info.go index ead6702b7c..f693264272 100644 --- a/internal/querycoord/segments_info.go +++ b/internal/querycoord/segments_info.go @@ -52,14 +52,17 @@ func (s *segmentsInfo) loadSegments() error { if err != nil { return } + numRowsCnt := float64(0) for _, v := range values { segment := &querypb.SegmentInfo{} if err = proto.Unmarshal([]byte(v), segment); err != nil { return } s.segmentIDMap[segment.GetSegmentID()] = segment - metrics.QueryCoordNumEntities.WithLabelValues(fmt.Sprint(segment.CollectionID)).Add(float64(segment.NumRows)) + numRowsCnt += float64(segment.NumRows) } + metrics.QueryCoordNumEntities.WithLabelValues().Add(numRowsCnt) + }) return err } @@ -77,7 +80,7 @@ func (s *segmentsInfo) saveSegment(segment *querypb.SegmentInfo) error { return err } s.segmentIDMap[segment.GetSegmentID()] = segment - metrics.QueryCoordNumEntities.WithLabelValues(fmt.Sprint(segment.CollectionID)).Add(float64(segment.NumRows)) + metrics.QueryCoordNumEntities.WithLabelValues().Add(float64(segment.NumRows)) return nil } @@ -89,7 +92,7 @@ func (s *segmentsInfo) removeSegment(segment *querypb.SegmentInfo) error { return err } delete(s.segmentIDMap, segment.GetSegmentID()) - metrics.QueryCoordNumEntities.WithLabelValues(fmt.Sprint(segment.CollectionID)).Sub(float64(segment.NumRows)) + metrics.QueryCoordNumEntities.WithLabelValues().Sub(float64(segment.NumRows)) return nil } diff --git a/internal/querycoord/task.go b/internal/querycoord/task.go index 5ca1b1cf0b..a2bf775ab8 100644 --- a/internal/querycoord/task.go +++ b/internal/querycoord/task.go @@ -32,7 +32,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/rootcoord" + "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/timerecord" ) @@ -2025,7 +2025,7 @@ func assignInternalTask(ctx context.Context, } func generateWatchDeltaChannelInfo(info *datapb.VchannelInfo) (*datapb.VchannelInfo, error) { - deltaChannelName, err := rootcoord.ConvertChannelName(info.ChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) + deltaChannelName, err := funcutil.ConvertChannelName(info.ChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) if err != nil { return nil, err } diff --git a/internal/querynode/collection.go b/internal/querynode/collection.go index 7ca238c042..48c5199b79 100644 --- a/internal/querynode/collection.go +++ b/internal/querynode/collection.go @@ -117,7 +117,7 @@ OUTER: c.vChannels = append(c.vChannels, dstChan) } - metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(c.ID()), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(c.vChannels))) + metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Add(float64(len(c.vChannels))) } // getVChannels get virtual channels of collection @@ -145,7 +145,7 @@ func (c *Collection) removeVChannel(channel Channel) { zap.Any("channel", channel), ) - metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(c.ID()), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(c.vChannels))) + metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Sub(float64(len(c.vChannels))) } // addPChannels add physical channels to physical channels of collection @@ -242,7 +242,7 @@ OUTER: c.vDeltaChannels = append(c.vDeltaChannels, dstChan) } - metrics.QueryNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(c.ID()), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(c.vDeltaChannels))) + metrics.QueryNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Add(float64(len(c.vDeltaChannels))) } func (c *Collection) removeVDeltaChannel(channel Channel) { @@ -260,7 +260,7 @@ func (c *Collection) removeVDeltaChannel(channel Channel) { zap.Any("channel", channel), ) - metrics.QueryNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(c.ID()), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(c.vDeltaChannels))) + metrics.QueryNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Sub(float64(len(c.vDeltaChannels))) } // setReleaseTime records when collection is released diff --git a/internal/querynode/collection_replica.go b/internal/querynode/collection_replica.go index 28f064578b..c4be487c43 100644 --- a/internal/querynode/collection_replica.go +++ b/internal/querynode/collection_replica.go @@ -242,14 +242,7 @@ func (colReplica *collectionReplica) removeCollectionPrivate(collectionID Unique delete(colReplica.collections, collectionID) metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(colReplica.collections))) - metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(0) - metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(0) - - metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(0) - metrics.QueryNodeNumDeltaChannels.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(0) - - metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(0) - metrics.QueryNodeNumReaders.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(0) + metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Sub(float64(len(collection.partitionIDs))) return nil } @@ -409,7 +402,7 @@ func (colReplica *collectionReplica) addPartitionPrivate(collectionID UniqueID, colReplica.partitions[partitionID] = newPartition } - metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(colReplica.partitions))) + metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(colReplica.partitions))) return nil } @@ -443,7 +436,7 @@ func (colReplica *collectionReplica) removePartitionPrivate(partitionID UniqueID collection.removePartitionID(partitionID) delete(colReplica.partitions, partitionID) - metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(collection.ID()), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(colReplica.partitions))) + metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(colReplica.partitions))) return nil } @@ -551,7 +544,7 @@ func (colReplica *collectionReplica) addSegmentPrivate(segmentID UniqueID, parti partition.addSegmentID(segmentID) colReplica.segments[segmentID] = segment - metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(segment.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() return nil } @@ -589,7 +582,7 @@ func (colReplica *collectionReplica) removeSegmentPrivate(segmentID UniqueID) er delete(colReplica.segments, segmentID) deleteSegment(segment) - metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(segment.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Dec() + metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Dec() return nil } diff --git a/internal/querynode/flow_graph_query_node.go b/internal/querynode/flow_graph_query_node.go index 788ca69457..d1def5309c 100644 --- a/internal/querynode/flow_graph_query_node.go +++ b/internal/querynode/flow_graph_query_node.go @@ -39,6 +39,7 @@ type queryNodeFlowGraph struct { channel Channel flowGraph *flowgraph.TimeTickedFlowGraph dmlStream msgstream.MsgStream + consumerCnt int } // newQueryNodeFlowGraph returns a new queryNodeFlowGraph @@ -208,8 +209,8 @@ func (q *queryNodeFlowGraph) consumeFlowGraph(channel Channel, subName ConsumeSu zap.Any("channel", channel), zap.Any("subName", subName), ) - - metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + q.consumerCnt++ + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() return nil } @@ -224,8 +225,8 @@ func (q *queryNodeFlowGraph) consumeFlowGraphFromLatest(channel Channel, subName zap.Any("channel", channel), zap.Any("subName", subName), ) - - metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + q.consumerCnt++ + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() return nil } @@ -237,8 +238,8 @@ func (q *queryNodeFlowGraph) seekQueryNodeFlowGraph(position *internalpb.MsgPosi zap.Any("collectionID", q.collectionID), zap.Any("channel", position.ChannelName), ) - - metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + q.consumerCnt++ + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() return err } @@ -246,6 +247,9 @@ func (q *queryNodeFlowGraph) seekQueryNodeFlowGraph(position *internalpb.MsgPosi func (q *queryNodeFlowGraph) close() { q.cancel() q.flowGraph.Close() + if q.dmlStream != nil && q.consumerCnt > 0 { + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Sub(float64(q.consumerCnt)) + } log.Debug("stop query node flow graph", zap.Any("collectionID", q.collectionID), zap.Any("channel", q.channel), diff --git a/internal/querynode/query_collection.go b/internal/querynode/query_collection.go index 4e24f27419..fe82147978 100644 --- a/internal/querynode/query_collection.go +++ b/internal/querynode/query_collection.go @@ -340,7 +340,7 @@ func (q *queryCollection) setServiceableTime(t Timestamp) { } q.serviceableTime = t ps, _ := tsoutil.ParseHybridTs(t) - metrics.QueryNodeServiceTime.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(ps)) + metrics.QueryNodeServiceTime.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(ps)) } func (q *queryCollection) checkTimeout(msg queryMsg) bool { @@ -674,12 +674,12 @@ func (q *queryCollection) doUnsolvedQueryMsg() { ) switch msgType { case commonpb.MsgType_Retrieve: - metrics.QueryNodeSQLatencyInQueue.WithLabelValues(metrics.QueryLabel, - fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(m.RecordSpan().Milliseconds())) + metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), + metrics.QueryLabel).Observe(float64(m.RecordSpan().Milliseconds())) err = q.retrieve(m) case commonpb.MsgType_Search: - metrics.QueryNodeSQLatencyInQueue.WithLabelValues(metrics.SearchLabel, - fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(m.RecordSpan().Milliseconds())) + metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), + metrics.SearchLabel).Observe(float64(m.RecordSpan().Milliseconds())) err = q.search(m) default: err := fmt.Errorf("receive invalid msgType = %d", msgType) @@ -1132,8 +1132,8 @@ func (q *queryCollection) search(msg queryMsg) error { if err != nil { return err } - metrics.QueryNodeSQReqLatency.WithLabelValues(metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(msg.ElapseSpan().Milliseconds())) - metrics.QueryNodeSQCount.WithLabelValues(metrics.SuccessLabel, metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel).Observe(float64(msg.ElapseSpan().Milliseconds())) + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel, metrics.SuccessLabel).Inc() tr.Record(fmt.Sprintf("publish empty search result done, msgID = %d", searchMsg.ID())) tr.Elapse(fmt.Sprintf("all done, msgID = %d", searchMsg.ID())) @@ -1164,7 +1164,7 @@ func (q *queryCollection) search(msg queryMsg) error { if err != nil { return err } - metrics.QueryNodeReduceLatency.WithLabelValues(metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(tr.RecordSpan().Milliseconds())) + metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds())) var offset int64 for index := range searchRequests { @@ -1244,11 +1244,11 @@ func (q *queryCollection) search(msg queryMsg) error { if err != nil { return err } - metrics.QueryNodeSQReqLatency.WithLabelValues(metrics.SearchLabel, - fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(msg.ElapseSpan().Milliseconds())) - metrics.QueryNodeSQCount.WithLabelValues(metrics.SuccessLabel, + metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), + metrics.SearchLabel).Observe(float64(msg.ElapseSpan().Milliseconds())) + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel, - fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.SuccessLabel).Inc() tr.Record(fmt.Sprintf("publish search result, msgID = %d", searchMsg.ID())) } sp.LogFields(oplog.String("statistical time", "stats done")) @@ -1332,7 +1332,7 @@ func (q *queryCollection) retrieve(msg queryMsg) error { return err } reduceDuration := tr.Record(fmt.Sprintf("merge result done, msgID = %d", retrieveMsg.ID())) - metrics.QueryNodeReduceLatency.WithLabelValues(metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(reduceDuration.Milliseconds())) + metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel).Observe(float64(reduceDuration.Milliseconds())) resultChannelInt := 0 retrieveResultMsg := &msgstream.RetrieveResultMsg{ @@ -1357,8 +1357,8 @@ func (q *queryCollection) retrieve(msg queryMsg) error { if err != nil { return err } - metrics.QueryNodeSQCount.WithLabelValues(metrics.SuccessLabel, metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() - metrics.QueryNodeSQReqLatency.WithLabelValues(metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(msg.ElapseSpan().Milliseconds())) + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel, metrics.SuccessLabel).Inc() + metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel).Observe(float64(msg.ElapseSpan().Milliseconds())) log.Debug("QueryNode publish RetrieveResultMsg", zap.Int64("msgID", retrieveMsg.ID()), @@ -1429,7 +1429,7 @@ func (q *queryCollection) publishSearchResultWithCtx(ctx context.Context, result } func (q *queryCollection) publishSearchResult(result *internalpb.SearchResults, nodeID UniqueID) error { - metrics.QueryNodeSQCount.WithLabelValues(metrics.TotalLabel, metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel, metrics.TotalLabel).Inc() return q.publishSearchResultWithCtx(q.releaseCtx, result, nodeID) } @@ -1438,7 +1438,7 @@ func (q *queryCollection) publishRetrieveResultWithCtx(ctx context.Context, resu } func (q *queryCollection) publishRetrieveResult(result *internalpb.RetrieveResults, nodeID UniqueID) error { - metrics.QueryNodeSQCount.WithLabelValues(metrics.TotalLabel, metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel, metrics.TotalLabel).Inc() return q.publishRetrieveResultWithCtx(q.releaseCtx, result, nodeID) } @@ -1458,7 +1458,7 @@ func (q *queryCollection) publishFailedQueryResultWithCtx(ctx context.Context, m case commonpb.MsgType_Retrieve: retrieveMsg := msg.(*msgstream.RetrieveMsg) baseResult.MsgType = commonpb.MsgType_RetrieveResult - metrics.QueryNodeSQCount.WithLabelValues(metrics.FailLabel, metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel, metrics.FailLabel).Inc() return q.publishRetrieveResult(&internalpb.RetrieveResults{ Base: baseResult, Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg}, @@ -1469,7 +1469,7 @@ func (q *queryCollection) publishFailedQueryResultWithCtx(ctx context.Context, m case commonpb.MsgType_Search: searchMsg := msg.(*msgstream.SearchMsg) baseResult.MsgType = commonpb.MsgType_SearchResult - metrics.QueryNodeSQCount.WithLabelValues(metrics.FailLabel, metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel, metrics.FailLabel).Inc() return q.publishSearchResultWithCtx(ctx, &internalpb.SearchResults{ Base: baseResult, Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg}, diff --git a/internal/querynode/segment.go b/internal/querynode/segment.go index 0eb76a0650..5800efae9b 100644 --- a/internal/querynode/segment.go +++ b/internal/querynode/segment.go @@ -313,7 +313,7 @@ func (s *Segment) search(plan *SearchPlan, log.Debug("do search on segment", zap.Int64("segmentID", s.segmentID), zap.Int32("segmentType", int32(s.segmentType))) tr := timerecord.NewTimeRecorder("cgoSearch") status := C.Search(s.segmentPtr, plan.cSearchPlan, cPlaceHolderGroup, ts, &searchResult.cSearchResult, C.int64_t(s.segmentID)) - metrics.QueryNodeSQSegmentLatencyInCore.WithLabelValues(metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.QueryNodeSQSegmentLatencyInCore.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) if err := HandleCStatus(&status, "Search failed"); err != nil { return nil, err } @@ -343,8 +343,8 @@ func (s *Segment) retrieve(plan *RetrievePlan) (*segcorepb.RetrieveResults, erro ts := C.uint64_t(plan.Timestamp) tr := timerecord.NewTimeRecorder("cgoRetrieve") status := C.Retrieve(s.segmentPtr, plan.cRetrievePlan, ts, &retrieveResult.cRetrieveResult) - metrics.QueryNodeSQSegmentLatencyInCore.WithLabelValues(metrics.QueryLabel, - fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.QueryNodeSQSegmentLatencyInCore.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), + metrics.QueryLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) if err := HandleCStatus(&status, "Retrieve failed"); err != nil { return nil, err } diff --git a/internal/querynode/segment_loader.go b/internal/querynode/segment_loader.go index 82c27817b5..ca51db3d00 100644 --- a/internal/querynode/segment_loader.go +++ b/internal/querynode/segment_loader.go @@ -39,7 +39,6 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/funcutil" @@ -514,8 +513,13 @@ func (loader *segmentLoader) FromDmlCPLoadDelete(ctx context.Context, collection if err != nil { return err } - defer stream.Close() - pChannelName := rootcoord.ToPhysicalChannel(position.ChannelName) + + defer func() { + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Dec() + stream.Close() + }() + + pChannelName := funcutil.ToPhysicalChannel(position.ChannelName) position.ChannelName = pChannelName stream.AsConsumer([]string{pChannelName}, fmt.Sprintf("querynode-%d-%d", Params.QueryNodeCfg.QueryNodeID, collectionID)) @@ -529,7 +533,7 @@ func (loader *segmentLoader) FromDmlCPLoadDelete(ctx context.Context, collection return nil } - metrics.QueryNodeNumReaders.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() err = stream.Seek([]*internalpb.MsgPosition{position}) if err != nil { return err diff --git a/internal/querynode/streaming.go b/internal/querynode/streaming.go index 99c55d77ad..79ce6c1ac6 100644 --- a/internal/querynode/streaming.go +++ b/internal/querynode/streaming.go @@ -211,9 +211,9 @@ func (s *streaming) search(searchReqs []*searchRequest, collID UniqueID, partIDs err2 = err return } - metrics.QueryNodeSQSegmentLatency.WithLabelValues(metrics.SearchLabel, - metrics.GrowingSegmentLabel, - fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), + metrics.SearchLabel, + metrics.GrowingSegmentLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) segmentLock.Lock() searchResults = append(searchResults, searchResult) searchSegmentIDs = append(searchSegmentIDs, seg.segmentID) diff --git a/internal/querynode/task.go b/internal/querynode/task.go index d755fda60a..e168992638 100644 --- a/internal/querynode/task.go +++ b/internal/querynode/task.go @@ -32,7 +32,6 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" queryPb "github.com/milvus-io/milvus/internal/proto/querypb" - "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/util/funcutil" ) @@ -162,7 +161,7 @@ func (r *addQueryChannelTask) Execute(ctx context.Context) error { consumeSubName := funcutil.GenChannelSubName(Params.CommonCfg.QueryNodeSubName, collectionID, Params.QueryNodeCfg.QueryNodeID) sc.queryMsgStream.AsConsumer(consumeChannels, consumeSubName) - metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() + metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() if r.req.SeekPosition == nil || len(r.req.SeekPosition.MsgID) == 0 { // as consumer log.Debug("QueryNode AsConsumer", zap.Strings("channels", consumeChannels), zap.String("sub name", consumeSubName)) @@ -242,7 +241,7 @@ func (w *watchDmChannelsTask) Execute(ctx context.Context) error { VPChannels := make(map[string]string) // map[vChannel]pChannel for _, info := range w.req.Infos { v := info.ChannelName - p := rootcoord.ToPhysicalChannel(info.ChannelName) + p := funcutil.ToPhysicalChannel(info.ChannelName) vChannels = append(vChannels, v) pChannels = append(pChannels, p) VPChannels[v] = p @@ -511,7 +510,7 @@ func (w *watchDeltaChannelsTask) Execute(ctx context.Context) error { vChannel2SeekPosition := make(map[string]*internalpb.MsgPosition) for _, info := range w.req.Infos { v := info.ChannelName - p := rootcoord.ToPhysicalChannel(info.ChannelName) + p := funcutil.ToPhysicalChannel(info.ChannelName) vDeltaChannels = append(vDeltaChannels, v) pDeltaChannels = append(pDeltaChannels, p) VPDeltaChannels[v] = p diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 2fb16bcb1f..5f9bcb5928 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -1304,7 +1304,7 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordCreateCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateCollection", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateCollection").Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.RootCoordNumOfCollections.Inc() return succStatus(), nil } @@ -1335,7 +1335,7 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordDropCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropCollection", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropCollection").Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.RootCoordNumOfCollections.Dec() return succStatus(), nil } @@ -1374,8 +1374,7 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordHasCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasCollection", - in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasCollection").Observe(float64(tr.ElapseSpan().Milliseconds())) return &milvuspb.BoolResponse{ Status: succStatus(), Value: t.HasCollection, @@ -1414,8 +1413,7 @@ func (c *Core) DescribeCollection(ctx context.Context, in *milvuspb.DescribeColl zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordDescribeCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeCollection", - strconv.FormatInt(in.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeCollection").Observe(float64(tr.ElapseSpan().Milliseconds())) t.Rsp.Status = succStatus() return t.Rsp, nil } @@ -1454,7 +1452,7 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections metrics.RootCoordShowCollectionsCounter.WithLabelValues(MetricRequestsSuccess).Inc() t.Rsp.Status = succStatus() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowCollections", "ALL").Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowCollections").Observe(float64(tr.ElapseSpan().Milliseconds())) return t.Rsp, nil } @@ -1487,7 +1485,7 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordCreatePartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreatePartition", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreatePartition").Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.RootCoordNumOfPartitions.WithLabelValues(in.CollectionName).Inc() return succStatus(), nil } @@ -1521,7 +1519,7 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordDropPartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropPartition", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropPartition").Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.RootCoordNumOfPartitions.WithLabelValues(in.CollectionName).Dec() return succStatus(), nil } @@ -1563,7 +1561,7 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordHasPartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasPartition", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasPartition").Observe(float64(tr.ElapseSpan().Milliseconds())) return &milvuspb.BoolResponse{ Status: succStatus(), Value: t.HasPartition, @@ -1604,7 +1602,7 @@ func (c *Core) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRe metrics.RootCoordShowPartitionsCounter.WithLabelValues(metrics.SuccessLabel).Inc() t.Rsp.Status = succStatus() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowPartitions", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowPartitions").Observe(float64(tr.ElapseSpan().Milliseconds())) return t.Rsp, nil } @@ -1637,7 +1635,7 @@ func (c *Core) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest) zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordCreateIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateIndex", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateIndex").Observe(float64(tr.ElapseSpan().Milliseconds())) return succStatus(), nil } @@ -1684,7 +1682,7 @@ func (c *Core) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequ } else { t.Rsp.Status = succStatus() } - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DescribeIndex", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DescribeIndex").Observe(float64(tr.ElapseSpan().Milliseconds())) return t.Rsp, nil } @@ -1717,7 +1715,7 @@ func (c *Core) DropIndex(ctx context.Context, in *milvuspb.DropIndexRequest) (*c zap.String("index name", in.IndexName), zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordDropIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropIndex", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropIndex").Observe(float64(tr.ElapseSpan().Milliseconds())) return succStatus(), nil } @@ -1755,8 +1753,7 @@ func (c *Core) DescribeSegment(ctx context.Context, in *milvuspb.DescribeSegment zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeSegment", - strconv.FormatInt(in.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeSegment").Observe(float64(tr.ElapseSpan().Milliseconds())) t.Rsp.Status = succStatus() return t.Rsp, nil } @@ -1797,7 +1794,7 @@ func (c *Core) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsReques zap.Int64("msgID", in.Base.MsgID)) metrics.RootCoordShowSegmentsCounter.WithLabelValues(metrics.SuccessLabel).Inc() - metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowSegments", strconv.FormatInt(in.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowSegments").Observe(float64(tr.ElapseSpan().Milliseconds())) t.Rsp.Status = succStatus() return t.Rsp, nil } @@ -2033,7 +2030,7 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest) zap.String("alias", in.Alias), zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateAlias", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateAlias").Observe(float64(tr.ElapseSpan().Milliseconds())) return succStatus(), nil } @@ -2061,7 +2058,7 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c log.Debug("DropAlias success", zap.String("role", typeutil.RootCoordRole), zap.String("alias", in.Alias), zap.Int64("msgID", in.Base.MsgID)) - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropAlias", in.Alias).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropAlias").Observe(float64(tr.ElapseSpan().Milliseconds())) return succStatus(), nil } @@ -2092,7 +2089,7 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) ( zap.String("alias", in.Alias), zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) - metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("AlterAlias", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) + metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("AlterAlias").Observe(float64(tr.ElapseSpan().Milliseconds())) return succStatus(), nil } diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index a557861734..4e6ec6d4fe 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -35,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/retry" "github.com/stretchr/testify/require" @@ -338,7 +339,7 @@ func createCollectionInMeta(dbName, collName string, core *Core, shardsNum int32 chanNames := make([]string, t.ShardsNum) for i := int32(0); i < t.ShardsNum; i++ { vchanNames[i] = fmt.Sprintf("%s_%dv%d", core.chanTimeTick.getDmlChannelName(), collID, i) - chanNames[i] = ToPhysicalChannel(vchanNames[i]) + chanNames[i] = funcutil.ToPhysicalChannel(vchanNames[i]) } collInfo := etcdpb.CollectionInfo{ @@ -746,7 +747,7 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, shardsNum, createMeta.ShardsNum) vChanName := createMeta.VirtualChannelNames[0] - assert.Equal(t, createMeta.PhysicalChannelNames[0], ToPhysicalChannel(vChanName)) + assert.Equal(t, createMeta.PhysicalChannelNames[0], funcutil.ToPhysicalChannel(vChanName)) // get TimeTickMsg //msgPack, ok = <-dmlStream.Chan() @@ -1360,7 +1361,7 @@ func TestRootCoord(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) vChanName := collMeta.VirtualChannelNames[0] - assert.Equal(t, collMeta.PhysicalChannelNames[0], ToPhysicalChannel(vChanName)) + assert.Equal(t, collMeta.PhysicalChannelNames[0], funcutil.ToPhysicalChannel(vChanName)) msgs := getNotTtMsg(ctx, 1, dmlStream.Chan()) assert.Equal(t, 1, len(msgs)) diff --git a/internal/rootcoord/task.go b/internal/rootcoord/task.go index 80838614cc..8760ba403b 100644 --- a/internal/rootcoord/task.go +++ b/internal/rootcoord/task.go @@ -21,16 +21,16 @@ import ( "fmt" "strconv" - "github.com/milvus-io/milvus/internal/metrics" - "github.com/golang/protobuf/proto" "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/schemapb" + "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/typeutil" "go.uber.org/zap" @@ -146,10 +146,10 @@ func (t *CreateCollectionReqTask) Execute(ctx context.Context) error { deltaChanNames := make([]string, t.Req.ShardsNum) for i := int32(0); i < t.Req.ShardsNum; i++ { vchanNames[i] = fmt.Sprintf("%s_%dv%d", t.core.chanTimeTick.getDmlChannelName(), collID, i) - chanNames[i] = ToPhysicalChannel(vchanNames[i]) + chanNames[i] = funcutil.ToPhysicalChannel(vchanNames[i]) deltaChanNames[i] = t.core.chanTimeTick.getDeltaChannelName() - deltaChanName, err1 := ConvertChannelName(chanNames[i], Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) + deltaChanName, err1 := funcutil.ConvertChannelName(chanNames[i], Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) if err1 != nil || deltaChanName != deltaChanNames[i] { return fmt.Errorf("dmlChanName %s and deltaChanName %s mis-match", chanNames[i], deltaChanNames[i]) } @@ -363,7 +363,7 @@ func (t *DropCollectionReqTask) Execute(ctx context.Context) error { // remove delta channels deltaChanNames := make([]string, len(collMeta.PhysicalChannelNames)) for i, chanName := range collMeta.PhysicalChannelNames { - if deltaChanNames[i], err = ConvertChannelName(chanName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta); err != nil { + if deltaChanNames[i], err = funcutil.ConvertChannelName(chanName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta); err != nil { return err } } diff --git a/internal/rootcoord/timeticksync.go b/internal/rootcoord/timeticksync.go index 81a6e48faf..072fe68a8b 100644 --- a/internal/rootcoord/timeticksync.go +++ b/internal/rootcoord/timeticksync.go @@ -28,6 +28,7 @@ import ( "github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/timerecord" "github.com/milvus-io/milvus/internal/util/tsoutil" @@ -99,7 +100,7 @@ func newTimeTickSync(ctx context.Context, sourceID int64, factory msgstream.Fact var err error deltaChanNames := make([]string, len(chanNames)) for i, chanName := range chanNames { - deltaChanNames[i], err = ConvertChannelName(chanName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) + deltaChanNames[i], err = funcutil.ConvertChannelName(chanName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) if err != nil { log.Error("failed to convert dml channel name to delta channel name", zap.String("chanName", chanName)) panic("invalid dml channel name " + chanName) diff --git a/internal/rootcoord/util.go b/internal/rootcoord/util.go index 3c49de1ab8..79019074f2 100644 --- a/internal/rootcoord/util.go +++ b/internal/rootcoord/util.go @@ -127,33 +127,3 @@ func DecodeMsgPositions(str string, msgPositions *[]*msgstream.MsgPosition) erro } return json.Unmarshal([]byte(str), msgPositions) } - -// ToPhysicalChannel get physical channel name from virtual channel name -func ToPhysicalChannel(vchannel string) string { - var idx int - for idx = len(vchannel) - 1; idx >= 0; idx-- { - if vchannel[idx] == '_' { - break - } - } - if idx < 0 { - return vchannel - } - return vchannel[:idx] -} - -// ConvertChannelName assembles channel name according to parameters. -func ConvertChannelName(chanName string, tokenFrom string, tokenTo string) (string, error) { - chanNameLen := len(chanName) - tokenFromLen := len(tokenFrom) - if chanNameLen < tokenFromLen { - return "", fmt.Errorf("cannot find token '%s' in '%s'", tokenFrom, chanName) - } - - for i := 0; i < (chanNameLen - tokenFromLen); i++ { - if chanName[i:i+tokenFromLen] == tokenFrom { - return chanName[0:i] + tokenTo + chanName[i+tokenFromLen:], nil - } - } - return "", fmt.Errorf("cannot find token '%s' in '%s'", tokenFrom, chanName) -} diff --git a/internal/rootcoord/util_test.go b/internal/rootcoord/util_test.go index 1dd4d1fa33..82d98b1cf5 100644 --- a/internal/rootcoord/util_test.go +++ b/internal/rootcoord/util_test.go @@ -97,14 +97,6 @@ func Test_GetFieldSchemaByIndexID(t *testing.T) { assert.NotNil(t, err) } -func Test_ToPhysicalChannel(t *testing.T) { - assert.Equal(t, "abc", ToPhysicalChannel("abc_")) - assert.Equal(t, "abc", ToPhysicalChannel("abc_123")) - assert.Equal(t, "abc", ToPhysicalChannel("abc_defgsg")) - assert.Equal(t, "abc__", ToPhysicalChannel("abc___defgsg")) - assert.Equal(t, "abcdef", ToPhysicalChannel("abcdef")) -} - func Test_EncodeMsgPositions(t *testing.T) { mp := &msgstream.MsgPosition{ ChannelName: "test", @@ -140,19 +132,3 @@ func Test_DecodeMsgPositions(t *testing.T) { err = DecodeMsgPositions("null", &mpOut) assert.Nil(t, err) } - -func Test_ConvertChannelName(t *testing.T) { - const ( - chanName = "by-dev_rootcoord-dml_123v0" - deltaChanName = "by-dev_rootcoord-delta_123v0" - tFrom = "rootcoord-dml" - tTo = "rootcoord-delta" - ) - _, err := ConvertChannelName("by-dev", tFrom, tTo) - assert.NotNil(t, err) - _, err = ConvertChannelName("by-dev_rootcoord-delta_123v0", tFrom, tTo) - assert.NotNil(t, err) - str, err := ConvertChannelName(chanName, tFrom, tTo) - assert.Nil(t, err) - assert.Equal(t, deltaChanName, str) -} diff --git a/internal/util/funcutil/func.go b/internal/util/funcutil/func.go index c62a7b6469..cbb1e918c3 100644 --- a/internal/util/funcutil/func.go +++ b/internal/util/funcutil/func.go @@ -235,3 +235,33 @@ func GetAvailablePort() int { return listener.Addr().(*net.TCPAddr).Port } + +// ToPhysicalChannel get physical channel name from virtual channel name +func ToPhysicalChannel(vchannel string) string { + var idx int + for idx = len(vchannel) - 1; idx >= 0; idx-- { + if vchannel[idx] == '_' { + break + } + } + if idx < 0 { + return vchannel + } + return vchannel[:idx] +} + +// ConvertChannelName assembles channel name according to parameters. +func ConvertChannelName(chanName string, tokenFrom string, tokenTo string) (string, error) { + chanNameLen := len(chanName) + tokenFromLen := len(tokenFrom) + if chanNameLen < tokenFromLen { + return "", fmt.Errorf("cannot find token '%s' in '%s'", tokenFrom, chanName) + } + + for i := 0; i < (chanNameLen - tokenFromLen); i++ { + if chanName[i:i+tokenFromLen] == tokenFrom { + return chanName[0:i] + tokenTo + chanName[i+tokenFromLen:], nil + } + } + return "", fmt.Errorf("cannot find token '%s' in '%s'", tokenFrom, chanName) +} diff --git a/internal/util/funcutil/func_test.go b/internal/util/funcutil/func_test.go index 9f1e5ce5d7..002d95de69 100644 --- a/internal/util/funcutil/func_test.go +++ b/internal/util/funcutil/func_test.go @@ -290,3 +290,27 @@ func TestCheckPortAvailable(t *testing.T) { assert.Equal(t, CheckPortAvailable(port), true) } } + +func Test_ToPhysicalChannel(t *testing.T) { + assert.Equal(t, "abc", ToPhysicalChannel("abc_")) + assert.Equal(t, "abc", ToPhysicalChannel("abc_123")) + assert.Equal(t, "abc", ToPhysicalChannel("abc_defgsg")) + assert.Equal(t, "abc__", ToPhysicalChannel("abc___defgsg")) + assert.Equal(t, "abcdef", ToPhysicalChannel("abcdef")) +} + +func Test_ConvertChannelName(t *testing.T) { + const ( + chanName = "by-dev_rootcoord-dml_123v0" + deltaChanName = "by-dev_rootcoord-delta_123v0" + tFrom = "rootcoord-dml" + tTo = "rootcoord-delta" + ) + _, err := ConvertChannelName("by-dev", tFrom, tTo) + assert.NotNil(t, err) + _, err = ConvertChannelName("by-dev_rootcoord-delta_123v0", tFrom, tTo) + assert.NotNil(t, err) + str, err := ConvertChannelName(chanName, tFrom, tTo) + assert.Nil(t, err) + assert.Equal(t, deltaChanName, str) +}