Reduce the number of metrics (#16030)

Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
This commit is contained in:
zhenshan.cao 2022-03-15 21:51:21 +08:00 committed by GitHub
parent c984c4ea73
commit f3eeecf146
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
51 changed files with 689 additions and 560 deletions

View File

@ -146,6 +146,10 @@ test-proxy:
@echo "Running go unittests..." @echo "Running go unittests..."
go test -race -coverpkg=./... -coverprofile=profile.out -covermode=atomic -timeout 5m github.com/milvus-io/milvus/internal/proxy -v 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: test-datanode:
@echo "Running go unittests..." @echo "Running go unittests..."
go test -race -coverpkg=./... -coverprofile=profile.out -covermode=atomic -timeout 5m github.com/milvus-io/milvus/internal/datanode -v go test -race -coverpkg=./... -coverprofile=profile.out -covermode=atomic -timeout 5m github.com/milvus-io/milvus/internal/datanode -v

View File

@ -19,10 +19,12 @@ package datacoord
import ( import (
"context" "context"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"go.uber.org/zap"
) )
// Cluster provides interfaces to interact with datanode cluster // 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 // Register registers a new node in cluster
func (c *Cluster) Register(node *NodeInfo) error { func (c *Cluster) Register(node *NodeInfo) error {
c.sessionManager.AddSession(node) 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 // UnRegister removes a node from cluster
func (c *Cluster) UnRegister(node *NodeInfo) error { func (c *Cluster) UnRegister(node *NodeInfo) error {
c.sessionManager.DeleteSession(node) c.sessionManager.DeleteSession(node)
err := c.channelManager.DeleteNode(node.NodeID)
if err == nil {
metrics.DataCoordNumDataNodes.WithLabelValues().Dec()
}
return c.channelManager.DeleteNode(node.NodeID) return c.channelManager.DeleteNode(node.NodeID)
} }

View File

@ -23,7 +23,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "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" "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 { func getCollectionStartPosition(channel string, collectionInfo *datapb.CollectionInfo) *internalpb.MsgPosition {
for _, sp := range collectionInfo.GetStartPositions() { for _, sp := range collectionInfo.GetStartPositions() {
if sp.GetKey() != rootcoord.ToPhysicalChannel(channel) { if sp.GetKey() != funcutil.ToPhysicalChannel(channel) {
continue continue
} }
return &internalpb.MsgPosition{ return &internalpb.MsgPosition{

View File

@ -25,6 +25,7 @@ import (
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
@ -68,16 +69,28 @@ func (m *meta) reloadFromKV() error {
if err != nil { if err != nil {
return err 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 { for _, value := range values {
segmentInfo := &datapb.SegmentInfo{} segmentInfo := &datapb.SegmentInfo{}
err = proto.Unmarshal([]byte(value), segmentInfo) err = proto.Unmarshal([]byte(value), segmentInfo)
if err != nil { if err != nil {
return fmt.Errorf("DataCoord reloadFromKV UnMarshal datapb.SegmentInfo err:%w", err) return fmt.Errorf("DataCoord reloadFromKV UnMarshal datapb.SegmentInfo err:%w", err)
} }
state := segmentInfo.GetState()
m.segments.SetSegment(segmentInfo.GetID(), NewSegmentInfo(segmentInfo)) 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 return nil
} }
@ -87,6 +100,7 @@ func (m *meta) AddCollection(collection *datapb.CollectionInfo) {
m.Lock() m.Lock()
defer m.Unlock() defer m.Unlock()
m.collections[collection.ID] = collection m.collections[collection.ID] = collection
metrics.DataCoordNumCollections.WithLabelValues().Set(float64(len(m.collections)))
} }
// GetCollection returns collection info with provided collection id from local cache // 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 { if err := m.saveSegmentInfo(segment); err != nil {
return err return err
} }
metrics.DataCoordNumSegments.WithLabelValues(string(segment.GetState())).Inc()
return nil return nil
} }
@ -174,6 +189,7 @@ func (m *meta) DropSegment(segmentID UniqueID) error {
if err := m.removeSegmentInfo(segment); err != nil { if err := m.removeSegmentInfo(segment); err != nil {
return err return err
} }
metrics.DataCoordNumSegments.WithLabelValues(metrics.DropedSegmentLabel).Inc()
m.segments.DropSegment(segmentID) m.segments.DropSegment(segmentID)
return nil return nil
} }
@ -194,9 +210,25 @@ func (m *meta) GetSegment(segID UniqueID) *SegmentInfo {
func (m *meta) SetState(segmentID UniqueID, state commonpb.SegmentState) error { func (m *meta) SetState(segmentID UniqueID, state commonpb.SegmentState) error {
m.Lock() m.Lock()
defer m.Unlock() defer m.Unlock()
curSegInfo := m.segments.GetSegment(segmentID)
if curSegInfo == nil {
return nil
}
oldState := curSegInfo.GetState()
m.segments.SetState(segmentID, state) m.segments.SetState(segmentID, state)
if segInfo := m.segments.GetSegment(segmentID); segInfo != nil && isSegmentHealthy(segInfo) { curSegInfo = m.segments.GetSegment(segmentID)
return m.saveSegmentInfo(segInfo) 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 return nil
} }
@ -336,7 +368,15 @@ func (m *meta) UpdateFlushSegmentsInfo(
if err := m.saveKvTxn(kv); err != nil { if err := m.saveKvTxn(kv); err != nil {
return err 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 // update memory status
for id, s := range modSegments { for id, s := range modSegments {
m.segments.SetSegment(id, s) m.segments.SetSegment(id, s)
@ -351,10 +391,12 @@ func (m *meta) UpdateDropChannelSegmentInfo(channel string, segments []*SegmentI
defer m.Unlock() defer m.Unlock()
modSegments := make(map[UniqueID]*SegmentInfo) modSegments := make(map[UniqueID]*SegmentInfo)
originSegments := make(map[UniqueID]*SegmentInfo)
for _, seg2Drop := range segments { for _, seg2Drop := range segments {
segment := m.mergeDropSegment(seg2Drop) segment := m.mergeDropSegment(seg2Drop)
if segment != nil { if segment != nil {
originSegments[seg2Drop.GetID()] = seg2Drop
modSegments[seg2Drop.GetID()] = segment modSegments[seg2Drop.GetID()] = segment
} }
} }
@ -369,10 +411,21 @@ func (m *meta) UpdateDropChannelSegmentInfo(channel string, segments []*SegmentI
clonedSeg := seg.Clone() clonedSeg := seg.Clone()
clonedSeg.State = commonpb.SegmentState_Dropped clonedSeg.State = commonpb.SegmentState_Dropped
modSegments[seg.ID] = clonedSeg modSegments[seg.ID] = clonedSeg
originSegments[seg.GetID()] = seg
} }
} }
err := m.batchSaveDropSegments(channel, modSegments)
return 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 // 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 { for _, s := range update {
m.segments.SetSegment(s.GetID(), s) m.segments.SetSegment(s.GetID(), s)
} }
metrics.DataCoordNumSegments.WithLabelValues(metrics.DropedSegmentLabel).Add(float64(len(update)))
return nil return nil
} }

View File

@ -30,6 +30,7 @@ import (
rootcoordclient "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" rootcoordclient "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log" "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"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper" "github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/proto/commonpb" "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)) 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()) s.updateSegmentStatistics(ttMsg.GetSegmentsStats())
if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil { if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil {

View File

@ -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)))) 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 return nil
} }

View File

@ -564,7 +564,9 @@ func (node *DataNode) ReadyToFlush() error {
// //
// One precondition: The segmentID in req is in ascending order. // One precondition: The segmentID in req is in ascending order.
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) { 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{ status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -625,7 +627,9 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
} }
status.ErrorCode = commonpb.ErrorCode_Success status.ErrorCode = commonpb.ErrorCode_Success
metrics.DataNodeFlushSegmentsCounter.WithLabelValues(MetricRequestsSuccess).Inc() metrics.DataNodeFlushSegmentsReqCounter.WithLabelValues(
fmt.Sprint(Params.DataNodeCfg.NodeID),
MetricRequestsSuccess).Inc()
return status, nil return status, nil
} }

View File

@ -130,8 +130,8 @@ func (dsService *dataSyncService) close() {
log.Info("dataSyncService closing flowgraph", zap.Int64("collectionID", dsService.collectionID), log.Info("dataSyncService closing flowgraph", zap.Int64("collectionID", dsService.collectionID),
zap.String("vChanName", dsService.vchannelName)) zap.String("vChanName", dsService.vchannelName))
dsService.fg.Close() dsService.fg.Close()
metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(dsService.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec()
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(dsService.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Sub(2) // timeTickChannel + deltaChannel metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Sub(2) // timeTickChannel + deltaChannel
} }
dsService.cancelFn() dsService.cancelFn()

View File

@ -28,8 +28,8 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "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/flowgraph"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
"go.uber.org/zap" "go.uber.org/zap"
@ -282,8 +282,8 @@ func newDDNode(ctx context.Context, collID UniqueID, vchanInfo *datapb.VchannelI
log.Error(err.Error()) log.Error(err.Error())
return nil return nil
} }
pChannelName := rootcoord.ToPhysicalChannel(vchanInfo.ChannelName) pChannelName := funcutil.ToPhysicalChannel(vchanInfo.ChannelName)
deltaChannelName, err := rootcoord.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) deltaChannelName, err := funcutil.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta)
if err != nil { if err != nil {
log.Error(err.Error()) log.Error(err.Error())
return nil return nil
@ -291,7 +291,7 @@ func newDDNode(ctx context.Context, collID UniqueID, vchanInfo *datapb.VchannelI
deltaStream.SetRepackFunc(msgstream.DefaultRepackFunc) deltaStream.SetRepackFunc(msgstream.DefaultRepackFunc)
deltaStream.AsProducer([]string{deltaChannelName}) 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)) log.Debug("datanode AsProducer", zap.String("DeltaChannelName", deltaChannelName))
var deltaMsgStream msgstream.MsgStream = deltaStream var deltaMsgStream msgstream.MsgStream = deltaStream
deltaMsgStream.Start() deltaMsgStream.Start()

View File

@ -24,8 +24,8 @@ import (
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/internalpb" "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/flowgraph"
"github.com/milvus-io/milvus/internal/util/funcutil"
"go.uber.org/zap" "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 // 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. // 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) 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)) log.Info("datanode AsConsumer", zap.String("physical channel", pchannelName), zap.String("subName", consumeSubName), zap.Int64("collection ID", dmNodeConfig.collectionID))
if seekPos != nil { if seekPos != nil {

View File

@ -35,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/storage" "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/trace"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
) )
@ -296,7 +297,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
dropped: false, 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]) err := ibNode.flushManager.flushBufferData(task.buffer, task.segmentID, task.flushed, task.dropped, endPositions[0])
if err != nil { if err != nil {
log.Warn("failed to invoke flushBufferData", zap.Error(err)) 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 { } else {
segmentsToFlush = append(segmentsToFlush, task.segmentID) segmentsToFlush = append(segmentsToFlush, task.segmentID)
ibNode.insertBuffer.Delete(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 { 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 // update buffer size
buffer.updateSize(int64(msg.NRows())) 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 // store in buffer
ibNode.insertBuffer.Store(currentSegID, buffer) ibNode.insertBuffer.Store(currentSegID, buffer)
@ -505,7 +506,7 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
return nil, err return nil, err
} }
wTt.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick}) 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)) log.Debug("datanode AsProducer", zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick))
var wTtMsgStream msgstream.MsgStream = wTt var wTtMsgStream msgstream.MsgStream = wTt
wTtMsgStream.Start() wTtMsgStream.Start()
@ -519,7 +520,7 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
continue continue
} }
stats = append(stats, stat) 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{} msgPack := msgstream.MsgPack{}
timeTickMsg := msgstream.DataNodeTtMsg{ timeTickMsg := msgstream.DataNodeTtMsg{
@ -541,7 +542,8 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
} }
msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg) msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg)
pt, _ := tsoutil.ParseHybridTs(ts) 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) return wTtMsgStream.Produce(&msgPack)
}) })

View File

@ -68,8 +68,6 @@ func (fm *flowgraphManager) addAndStart(dn *DataNode, vchan *datapb.VchannelInfo
fm.flowgraphs.Store(vchan.GetChannelName(), dataSyncService) fm.flowgraphs.Store(vchan.GetChannelName(), dataSyncService)
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Inc() 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 return nil
} }
@ -77,11 +75,8 @@ func (fm *flowgraphManager) release(vchanName string) {
log.Info("release flowgraph resources begin", zap.String("vChannelName", vchanName)) log.Info("release flowgraph resources begin", zap.String("vChannelName", vchanName))
if fg, loaded := fm.flowgraphs.LoadAndDelete(vchanName); loaded { if fg, loaded := fm.flowgraphs.LoadAndDelete(vchanName); loaded {
collectionID := fg.(*dataSyncService).collectionID
fg.(*dataSyncService).close() fg.(*dataSyncService).close()
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() 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)) log.Info("release flowgraph resources end", zap.String("Vchannel", vchanName))
} }

View File

@ -413,7 +413,7 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segmentID Uni
data: kvs, data: kvs,
}, field2Insert, field2Stats, flushed, dropped, pos) }, 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 return nil
} }
@ -555,11 +555,11 @@ type flushBufferInsertTask struct {
func (t *flushBufferInsertTask) flushInsertData() error { func (t *flushBufferInsertTask) flushInsertData() error {
if t.BaseKV != nil && len(t.data) > 0 { if t.BaseKV != nil && len(t.data) > 0 {
for _, d := range t.data { 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") tr := timerecord.NewTimeRecorder("insertData")
err := t.MultiSave(t.data) 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 err
} }
return nil return nil
@ -574,11 +574,11 @@ type flushBufferDeleteTask struct {
func (t *flushBufferDeleteTask) flushDeleteData() error { func (t *flushBufferDeleteTask) flushDeleteData() error {
if len(t.data) > 0 && t.BaseKV != nil { if len(t.data) > 0 && t.BaseKV != nil {
for _, d := range t.data { 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") tr := timerecord.NewTimeRecorder("deleteData")
err := t.MultiSave(t.data) 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 err
} }
return nil return nil

View File

@ -194,7 +194,7 @@ func (replica *SegmentReplica) new2FlushedSegment(segID UniqueID) {
replica.flushedSegments[segID] = &seg replica.flushedSegments[segID] = &seg
delete(replica.newSegments, segID) 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* // 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 replica.flushedSegments[segID] = &seg
delete(replica.normalSegments, segID) 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) { 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() replica.segMu.Lock()
defer replica.segMu.Unlock() defer replica.segMu.Unlock()
replica.newSegments[segID] = seg 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 return nil
} }
@ -363,7 +363,7 @@ func (replica *SegmentReplica) addNormalSegment(segID, collID, partitionID Uniqu
replica.segMu.Lock() replica.segMu.Lock()
replica.normalSegments[segID] = seg replica.normalSegments[segID] = seg
replica.segMu.Unlock() 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 return nil
} }
@ -560,15 +560,15 @@ func (replica *SegmentReplica) removeSegments(segIDs ...UniqueID) {
defer replica.segMu.Unlock() defer replica.segMu.Unlock()
log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs)) log.Info("remove segments if exist", zap.Int64s("segmentIDs", segIDs))
cnt := 0
for _, segID := range segIDs { for _, segID := range segIDs {
if seg, ok := replica.newSegments[segID]; ok { if _, ok := replica.newSegments[segID]; ok {
metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(seg.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() cnt++
} } else if _, ok := replica.normalSegments[segID]; ok {
if seg, ok := replica.normalSegments[segID]; ok { cnt++
metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(seg.collectionID), fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec()
} }
} }
metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Sub(float64(cnt))
for _, segID := range segIDs { for _, segID := range segIDs {
delete(replica.newSegments, segID) delete(replica.newSegments, segID)

View File

@ -76,7 +76,7 @@ func (nm *NodeManager) RemoveNode(nodeID UniqueID) {
delete(nm.nodeClients, nodeID) delete(nm.nodeClients, nodeID)
nm.lock.Unlock() nm.lock.Unlock()
nm.pq.Remove(nodeID) nm.pq.Remove(nodeID)
metrics.IndexCoordIndexNodeNum.WithLabelValues("index_node_num").Dec() metrics.IndexCoordIndexNodeNum.WithLabelValues().Dec()
} }
// AddNode adds the client of IndexNode. // 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)) log.Error("IndexCoord NodeManager", zap.Any("Add node err", err))
return err return err
} }
metrics.IndexCoordIndexNodeNum.WithLabelValues("index_node_num").Inc() metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc()
return nm.setClient(nodeID, nodeClient) return nm.setClient(nodeID, nodeClient)
} }

View File

@ -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. // 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.IndexNodeLoadBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(loadVectorDuration))
metrics.IndexNodeDecodeBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), strconv.FormatInt(it.segmentID, 10)).Observe(float64(it.tr.RecordSpan())) metrics.IndexNodeDecodeBinlogLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(it.tr.RecordSpan()))
if len(insertData.Data) != 1 { if len(insertData.Data) != 1 {
return storage.InvalidUniqueID, nil, errors.New("we expect only one field in deserialized insert data") 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 { if !fOk && !bOk {
return nil, errors.New("we expect FloatVectorFieldData or BinaryVectorFieldData") 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 return nil, err
} }
encodeIndexFileDur := it.tr.Record("index codec serialize done") encodeIndexFileDur := it.tr.Record("index codec serialize done")
metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(encodeIndexFileDur.Milliseconds()))
strconv.FormatInt(it.segmentID, 10)).Observe(float64(encodeIndexFileDur.Milliseconds()))
return serializedIndexBlobs, nil return serializedIndexBlobs, nil
} }
@ -593,8 +592,7 @@ func (it *IndexBuildTask) Execute(ctx context.Context) error {
return err return err
} }
saveIndexFileDur := it.tr.Record("index file save done") saveIndexFileDur := it.tr.Record("index file save done")
metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10), metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.NodeID, 10)).Observe(float64(saveIndexFileDur.Milliseconds()))
strconv.FormatInt(it.segmentID, 10)).Observe(float64(saveIndexFileDur.Milliseconds()))
it.tr.Elapse("index building all done") it.tr.Elapse("index building all done")
log.Info("IndexNode CreateIndex successfully ", zap.Int64("collect", it.collectionID), log.Info("IndexNode CreateIndex successfully ", zap.Int64("collect", it.collectionID),
zap.Int64("partition", it.partitionID), zap.Int64("segment", it.segmentID)) zap.Int64("partition", it.partitionID), zap.Int64("segment", it.segmentID))

View File

@ -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)
}

View File

@ -39,8 +39,8 @@ var (
Name: "message_rows_count", Name: "message_rows_count",
Help: "Messages rows size count consumed from msgStream in DataNode.", Help: "Messages rows size count consumed from msgStream in DataNode.",
}, []string{ }, []string{
msgTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
msgTypeLabelName,
}) })
DataNodeFlushedSize = prometheus.NewCounterVec( DataNodeFlushedSize = prometheus.NewCounterVec(
@ -50,31 +50,31 @@ var (
Name: "flushed_size", Name: "flushed_size",
Help: "Data size flushed to storage in DataNode.", Help: "Data size flushed to storage in DataNode.",
}, []string{ }, []string{
nodeIDLabelName,
msgTypeLabelName, msgTypeLabelName,
nodeIDLabelName,
}) })
DataNodeNumDmlChannels = prometheus.NewGaugeVec( //DataNodeNumDmlChannels = prometheus.NewGaugeVec(
prometheus.GaugeOpts{ // prometheus.GaugeOpts{
Namespace: milvusNamespace, // Namespace: milvusNamespace,
Subsystem: typeutil.DataNodeRole, // Subsystem: typeutil.DataNodeRole,
Name: "num_dml_channels", // Name: "num_dml_channels",
Help: "Number of dmlChannels per collection in DataNode.", // Help: "Number of dmlChannels per collection in DataNode.",
}, []string{ // }, []string{
collectionIDLabelName, // collectionIDLabelName,
nodeIDLabelName, // nodeIDLabelName,
}) // })
//
DataNodeNumDeltaChannels = prometheus.NewGaugeVec( //DataNodeNumDeltaChannels = prometheus.NewGaugeVec(
prometheus.GaugeOpts{ // prometheus.GaugeOpts{
Namespace: milvusNamespace, // Namespace: milvusNamespace,
Subsystem: typeutil.DataNodeRole, // Subsystem: typeutil.DataNodeRole,
Name: "num_delta_channels", // Name: "num_delta_channels",
Help: "Number of deltaChannels per collection in DataNode.", // Help: "Number of deltaChannels per collection in DataNode.",
}, []string{ // }, []string{
collectionIDLabelName, // collectionIDLabelName,
nodeIDLabelName, // nodeIDLabelName,
}) // })
DataNodeNumConsumers = prometheus.NewGaugeVec( DataNodeNumConsumers = prometheus.NewGaugeVec(
prometheus.GaugeOpts{ prometheus.GaugeOpts{
@ -83,7 +83,6 @@ var (
Name: "num_consumers", Name: "num_consumers",
Help: "Number of consumers per collection in DataNode.", Help: "Number of consumers per collection in DataNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -94,7 +93,6 @@ var (
Name: "num_producers", Name: "num_producers",
Help: "Number of producers per collection in DataNode.", Help: "Number of producers per collection in DataNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -105,8 +103,8 @@ var (
Name: "time_sync", Name: "time_sync",
Help: "Synchronized timestamps per channel in DataNode.", Help: "Synchronized timestamps per channel in DataNode.",
}, []string{ }, []string{
channelNameLabelName,
nodeIDLabelName, nodeIDLabelName,
channelNameLabelName,
}) })
DataNodeSegmentRowsCount = prometheus.NewCounterVec( DataNodeSegmentRowsCount = prometheus.NewCounterVec(
@ -116,7 +114,6 @@ var (
Name: "seg_rows_count", Name: "seg_rows_count",
Help: "Rows count of segments which sent to DataCoord from DataNode.", Help: "Rows count of segments which sent to DataCoord from DataNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -127,7 +124,6 @@ var (
Name: "num_unflushed_segments", Name: "num_unflushed_segments",
Help: "Number of unflushed segments in DataNode.", Help: "Number of unflushed segments in DataNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -139,7 +135,6 @@ var (
Help: "The flush segment latency in DataNode.", Help: "The flush segment latency in DataNode.",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -151,8 +146,8 @@ var (
Help: "The latency saving flush data to storage in DataNode.", Help: "The latency saving flush data to storage in DataNode.",
Buckets: []float64{0, 10, 100, 200, 400, 1000, 10000}, Buckets: []float64{0, 10, 100, 200, 400, 1000, 10000},
}, []string{ }, []string{
msgTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
msgTypeLabelName,
}) })
DataNodeFlushSegmentCount = prometheus.NewCounterVec( // TODO: arguably DataNodeFlushSegmentCount = prometheus.NewCounterVec( // TODO: arguably
@ -162,8 +157,8 @@ var (
Name: "flush_segment_count", Name: "flush_segment_count",
Help: "Flush segment statistics in DataNode.", Help: "Flush segment statistics in DataNode.",
}, []string{ }, []string{
statusLabelName,
nodeIDLabelName, nodeIDLabelName,
statusLabelName,
}) })
DataNodeAutoFlushSegmentCount = prometheus.NewCounterVec( // TODO: arguably DataNodeAutoFlushSegmentCount = prometheus.NewCounterVec( // TODO: arguably
@ -173,7 +168,6 @@ var (
Name: "auto_flush_segment_count", Name: "auto_flush_segment_count",
Help: "Auto flush segment statistics in DataNode.", Help: "Auto flush segment statistics in DataNode.",
}, []string{ }, []string{
channelNameLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -185,9 +179,20 @@ var (
Help: "Compaction latency in DataNode.", Help: "Compaction latency in DataNode.",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, 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 //RegisterDataNode registers DataNode metrics
@ -195,8 +200,8 @@ func RegisterDataNode() {
prometheus.MustRegister(DataNodeNumFlowGraphs) prometheus.MustRegister(DataNodeNumFlowGraphs)
prometheus.MustRegister(DataNodeConsumeMsgRowsCount) prometheus.MustRegister(DataNodeConsumeMsgRowsCount)
prometheus.MustRegister(DataNodeFlushedSize) prometheus.MustRegister(DataNodeFlushedSize)
prometheus.MustRegister(DataNodeNumDmlChannels) //prometheus.MustRegister(DataNodeNumDmlChannels)
prometheus.MustRegister(DataNodeNumDeltaChannels) //prometheus.MustRegister(DataNodeNumDeltaChannels)
prometheus.MustRegister(DataNodeNumConsumers) prometheus.MustRegister(DataNodeNumConsumers)
prometheus.MustRegister(DataNodeNumProducers) prometheus.MustRegister(DataNodeNumProducers)
prometheus.MustRegister(DataNodeTimeSync) prometheus.MustRegister(DataNodeTimeSync)
@ -207,4 +212,5 @@ func RegisterDataNode() {
prometheus.MustRegister(DataNodeFlushSegmentCount) prometheus.MustRegister(DataNodeFlushSegmentCount)
prometheus.MustRegister(DataNodeAutoFlushSegmentCount) prometheus.MustRegister(DataNodeAutoFlushSegmentCount)
prometheus.MustRegister(DataNodeCompactionLatency) prometheus.MustRegister(DataNodeCompactionLatency)
prometheus.MustRegister(DataNodeFlushSegmentsReqCounter)
} }

View File

@ -38,7 +38,7 @@ var (
Subsystem: typeutil.IndexCoordRole, Subsystem: typeutil.IndexCoordRole,
Name: "index_task_counter", Name: "index_task_counter",
Help: "The number of index tasks of each type", 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 records the number of IndexNodes managed by IndexCoord.
IndexCoordIndexNodeNum = prometheus.NewGaugeVec( IndexCoordIndexNodeNum = prometheus.NewGaugeVec(
@ -47,7 +47,7 @@ var (
Subsystem: typeutil.IndexCoordRole, Subsystem: typeutil.IndexCoordRole,
Name: "index_node_num", Name: "index_node_num",
Help: "The number of IndexNodes managed by IndexCoord", Help: "The number of IndexNodes managed by IndexCoord",
}, []string{"type"}) }, []string{})
) )
//RegisterIndexCoord registers IndexCoord metrics //RegisterIndexCoord registers IndexCoord metrics

View File

@ -37,7 +37,7 @@ var (
Name: "load_segment_latency", Name: "load_segment_latency",
Help: "The latency of loading the segment", Help: "The latency of loading the segment",
Buckets: buckets, Buckets: buckets,
}, []string{nodeIDLabelName, segmentIDLabelName}) }, []string{nodeIDLabelName})
IndexNodeDecodeBinlogLatency = prometheus.NewHistogramVec( IndexNodeDecodeBinlogLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{ prometheus.HistogramOpts{
@ -46,7 +46,7 @@ var (
Name: "decode_binlog_latency", Name: "decode_binlog_latency",
Help: "The latency of decode the binlog", Help: "The latency of decode the binlog",
Buckets: buckets, Buckets: buckets,
}, []string{nodeIDLabelName, segmentIDLabelName}) }, []string{nodeIDLabelName})
IndexNodeKnowhereBuildIndexLatency = prometheus.NewHistogramVec( IndexNodeKnowhereBuildIndexLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{ prometheus.HistogramOpts{
@ -55,7 +55,7 @@ var (
Name: "knowhere_build_index_latency", Name: "knowhere_build_index_latency",
Help: "The latency of knowhere building the index", Help: "The latency of knowhere building the index",
Buckets: buckets, Buckets: buckets,
}, []string{nodeIDLabelName, segmentIDLabelName}) }, []string{nodeIDLabelName})
IndexNodeEncodeIndexFileLatency = prometheus.NewHistogramVec( IndexNodeEncodeIndexFileLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{ prometheus.HistogramOpts{
@ -64,7 +64,7 @@ var (
Name: "encode_index_file_latency", Name: "encode_index_file_latency",
Help: "The latency of encoding the index file", Help: "The latency of encoding the index file",
Buckets: buckets, Buckets: buckets,
}, []string{nodeIDLabelName, segmentIDLabelName}) }, []string{nodeIDLabelName})
IndexNodeSaveIndexFileLatency = prometheus.NewHistogramVec( IndexNodeSaveIndexFileLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{ prometheus.HistogramOpts{
@ -73,7 +73,7 @@ var (
Name: "save_index_file_latency", Name: "save_index_file_latency",
Help: "The latency of saving the index file", Help: "The latency of saving the index file",
Buckets: buckets, Buckets: buckets,
}, []string{nodeIDLabelName, segmentIDLabelName}) }, []string{nodeIDLabelName})
) )
//RegisterIndexNode registers IndexNode metrics //RegisterIndexNode registers IndexNode metrics

View File

@ -23,8 +23,6 @@ import (
_ "net/http/pprof" _ "net/http/pprof"
"github.com/milvus-io/milvus/internal/log" "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"
"github.com/prometheus/client_golang/prometheus/promhttp" "github.com/prometheus/client_golang/prometheus/promhttp"
"go.uber.org/zap" "go.uber.org/zap"
@ -52,15 +50,18 @@ const (
FailedIndexTaskLabel = "failed" FailedIndexTaskLabel = "failed"
RecycledIndexTaskLabel = "recycled" RecycledIndexTaskLabel = "recycled"
SealedSegmentLabel = "sealed" SealedSegmentLabel = "Sealed"
GrowingSegmentLabel = "growing" GrowingSegmentLabel = "Growing"
FlushedSegmentLabel = "Flushed"
FlushingSegmentLabel = "Flushing"
DropedSegmentLabel = "Dropped"
nodeIDLabelName = "node_id" nodeIDLabelName = "node_id"
statusLabelName = "status" statusLabelName = "status"
indexTaskStatusLabelName = "index_task_status"
msgTypeLabelName = "msg_type" msgTypeLabelName = "msg_type"
collectionIDLabelName = "collection_id" collectionIDLabelName = "collection_id"
channelNameLabelName = "channel_name" channelNameLabelName = "channel_name"
segmentIDLabelName = "segment_id"
functionLabelName = "function_name" functionLabelName = "function_name"
queryTypeLabelName = "query_type" queryTypeLabelName = "query_type"
segmentTypeLabelName = "segment_type" segmentTypeLabelName = "segment_type"
@ -72,43 +73,6 @@ var (
buckets = prometheus.ExponentialBuckets(1, 2, 18) 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 //ServeHTTP serves prometheus http service
func ServeHTTP() { func ServeHTTP() {
http.Handle("/metrics", promhttp.Handler()) http.Handle("/metrics", promhttp.Handler())

View File

@ -22,15 +22,6 @@ import (
) )
var ( 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 record the number of times search succeeded or failed.
ProxySearchCount = prometheus.NewCounterVec( ProxySearchCount = prometheus.NewCounterVec(
prometheus.CounterOpts{ prometheus.CounterOpts{
@ -38,7 +29,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "search_counter", Name: "search_counter",
Help: "The number of times search succeeded or failed", 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 record the number of times insert succeeded or failed.
ProxyInsertCount = prometheus.NewCounterVec( ProxyInsertCount = prometheus.NewCounterVec(
@ -47,7 +38,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "insert_counter", Name: "insert_counter",
Help: "The number of times insert succeeded or failed", 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 record the number of vectors search successfully.
ProxySearchVectors = prometheus.NewGaugeVec( ProxySearchVectors = prometheus.NewGaugeVec(
@ -56,7 +47,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "search_vectors", Name: "search_vectors",
Help: "The number of vectors search successfully", Help: "The number of vectors search successfully",
}, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) }, []string{nodeIDLabelName, queryTypeLabelName})
// ProxyInsertVectors record the number of vectors insert successfully. // ProxyInsertVectors record the number of vectors insert successfully.
ProxyInsertVectors = prometheus.NewGaugeVec( ProxyInsertVectors = prometheus.NewGaugeVec(
@ -65,17 +56,17 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "insert_vectors", Name: "insert_vectors",
Help: "The number of vectors insert successfully", Help: "The number of vectors insert successfully",
}, []string{nodeIDLabelName, collectionIDLabelName}) }, []string{nodeIDLabelName})
// ProxyLinkedSDKs record The number of SDK linked proxy. // ProxyLinkedSDKs record The number of SDK linked proxy.
// TODO: how to know when sdk disconnect? // TODO: how to know when sdk disconnect?
ProxyLinkedSDKs = prometheus.NewGaugeVec( //ProxyLinkedSDKs = prometheus.NewGaugeVec(
prometheus.GaugeOpts{ // prometheus.GaugeOpts{
Namespace: milvusNamespace, // Namespace: milvusNamespace,
Subsystem: typeutil.ProxyRole, // Subsystem: typeutil.ProxyRole,
Name: "linked_sdk_numbers", // Name: "linked_sdk_numbers",
Help: "The number of SDK linked proxy", // Help: "The number of SDK linked proxy",
}, []string{nodeIDLabelName}) // }, []string{nodeIDLabelName})
// ProxySearchLatency record the latency of search successfully. // ProxySearchLatency record the latency of search successfully.
ProxySearchLatency = prometheus.NewHistogramVec( ProxySearchLatency = prometheus.NewHistogramVec(
@ -85,7 +76,7 @@ var (
Name: "search_latency", Name: "search_latency",
Help: "The latency of search successfully", Help: "The latency of search successfully",
Buckets: buckets, 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 record the latency that the proxy sent the search request to the message stream.
ProxySendMessageLatency = prometheus.NewHistogramVec( ProxySendMessageLatency = prometheus.NewHistogramVec(
@ -95,7 +86,7 @@ var (
Name: "send_search_msg_time", Name: "send_search_msg_time",
Help: "The latency that the proxy sent the search request to the message stream", Help: "The latency that the proxy sent the search request to the message stream",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) }, []string{nodeIDLabelName, queryTypeLabelName})
// ProxyWaitForSearchResultLatency record the time that the proxy waits for the search result. // ProxyWaitForSearchResultLatency record the time that the proxy waits for the search result.
ProxyWaitForSearchResultLatency = prometheus.NewHistogramVec( ProxyWaitForSearchResultLatency = prometheus.NewHistogramVec(
@ -105,7 +96,7 @@ var (
Name: "wait_for_search_result_time", Name: "wait_for_search_result_time",
Help: "The time that the proxy waits for the search result", Help: "The time that the proxy waits for the search result",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) }, []string{nodeIDLabelName, queryTypeLabelName})
// ProxyReduceSearchResultLatency record the time that the proxy reduces search result. // ProxyReduceSearchResultLatency record the time that the proxy reduces search result.
ProxyReduceSearchResultLatency = prometheus.NewHistogramVec( ProxyReduceSearchResultLatency = prometheus.NewHistogramVec(
@ -115,7 +106,7 @@ var (
Name: "reduce_search_result_time", Name: "reduce_search_result_time",
Help: "The time that the proxy reduces search result", Help: "The time that the proxy reduces search result",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, collectionIDLabelName, queryTypeLabelName}) }, []string{nodeIDLabelName, queryTypeLabelName})
// ProxyDecodeSearchResultLatency record the time that the proxy decodes the search result. // ProxyDecodeSearchResultLatency record the time that the proxy decodes the search result.
ProxyDecodeSearchResultLatency = prometheus.NewHistogramVec( ProxyDecodeSearchResultLatency = prometheus.NewHistogramVec(
@ -125,7 +116,7 @@ var (
Name: "decode_search_result_time", Name: "decode_search_result_time",
Help: "The time that the proxy decodes the search result", Help: "The time that the proxy decodes the search result",
Buckets: buckets, // unit: ms 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 record the number of MsgStream objects per PChannel on each collection_id on Proxy.
ProxyMsgStreamObjectsForPChan = prometheus.NewGaugeVec( ProxyMsgStreamObjectsForPChan = prometheus.NewGaugeVec(
@ -134,7 +125,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "msg_stream_obj_for_PChan", Name: "msg_stream_obj_for_PChan",
Help: "The number of MsgStream objects per PChannel on each collection on Proxy", 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 record the number of MsgStream objects for search per collection_id.
ProxyMsgStreamObjectsForSearch = prometheus.NewGaugeVec( ProxyMsgStreamObjectsForSearch = prometheus.NewGaugeVec(
@ -143,7 +134,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "msg_stream_obj_for_search", Name: "msg_stream_obj_for_search",
Help: "The number of MsgStream objects for search per collection", 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 record the latency that insert successfully.
ProxyInsertLatency = prometheus.NewHistogramVec( ProxyInsertLatency = prometheus.NewHistogramVec(
@ -153,17 +144,7 @@ var (
Name: "insert_latency", Name: "insert_latency",
Help: "The latency that insert successfully.", Help: "The latency that insert successfully.",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, collectionIDLabelName}) }, []string{nodeIDLabelName})
// 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})
// ProxySendInsertReqLatency record the latency that Proxy send insert request to MsgStream. // ProxySendInsertReqLatency record the latency that Proxy send insert request to MsgStream.
ProxySendInsertReqLatency = prometheus.NewHistogramVec( ProxySendInsertReqLatency = prometheus.NewHistogramVec(
@ -173,10 +154,9 @@ var (
Name: "send_insert_req_latency", Name: "send_insert_req_latency",
Help: "The latency that Proxy send insert request to MsgStream", Help: "The latency that Proxy send insert request to MsgStream",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, collectionIDLabelName}) }, []string{nodeIDLabelName})
// ProxyCacheHitCounter record the number of Proxy cache hits or miss. // ProxyCacheHitCounter record the number of Proxy cache hits or miss.
// TODO: @xiaocai2333 add more cache type
ProxyCacheHitCounter = prometheus.NewCounterVec( ProxyCacheHitCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{ prometheus.CounterOpts{
Namespace: milvusNamespace, Namespace: milvusNamespace,
@ -240,7 +220,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "DQL_call_counter", Name: "DQL_call_counter",
Help: "", 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 records the number of times the function of the DML operation was executed, like `LoadCollection`.
ProxyDMLFunctionCall = prometheus.NewCounterVec( ProxyDMLFunctionCall = prometheus.NewCounterVec(
@ -249,7 +229,7 @@ var (
Subsystem: typeutil.ProxyRole, Subsystem: typeutil.ProxyRole,
Name: "DML_call_counter", Name: "DML_call_counter",
Help: "", Help: "",
}, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName, statusLabelName}) }, []string{nodeIDLabelName, functionLabelName, statusLabelName})
// ProxyDDLReqLatency records the latency that for DML request, like "CreateCollection". // ProxyDDLReqLatency records the latency that for DML request, like "CreateCollection".
ProxyDDLReqLatency = prometheus.NewHistogramVec( ProxyDDLReqLatency = prometheus.NewHistogramVec(
@ -269,7 +249,7 @@ var (
Name: "DML_call_latency", Name: "DML_call_latency",
Help: "The latency that for DML request", Help: "The latency that for DML request",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName}) }, []string{nodeIDLabelName, functionLabelName})
// ProxyDQLReqLatency record the latency that for DQL request, like "HasCollection". // ProxyDQLReqLatency record the latency that for DQL request, like "HasCollection".
ProxyDQLReqLatency = prometheus.NewHistogramVec( ProxyDQLReqLatency = prometheus.NewHistogramVec(
@ -279,7 +259,7 @@ var (
Name: "DQL_call_latency", Name: "DQL_call_latency",
Help: "The latency that for DQL request", Help: "The latency that for DQL request",
Buckets: buckets, // unit: ms Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, functionLabelName, collectionIDLabelName}) }, []string{nodeIDLabelName, functionLabelName})
// ProxySearchLatencyPerNQ records the latency for searching. // ProxySearchLatencyPerNQ records the latency for searching.
ProxySearchLatencyPerNQ = prometheus.NewHistogramVec( ProxySearchLatencyPerNQ = prometheus.NewHistogramVec(
@ -289,19 +269,17 @@ var (
Name: "proxy_search_latency_count", Name: "proxy_search_latency_count",
Help: "The latency for searching", Help: "The latency for searching",
Buckets: buckets, Buckets: buckets,
}, []string{nodeIDLabelName, collectionIDLabelName}) }, []string{nodeIDLabelName})
) )
//RegisterProxy registers Proxy metrics //RegisterProxy registers Proxy metrics
func RegisterProxy() { func RegisterProxy() {
prometheus.MustRegister(ProxyDmlChannelTimeTick)
prometheus.MustRegister(ProxySearchCount) prometheus.MustRegister(ProxySearchCount)
prometheus.MustRegister(ProxyInsertCount) prometheus.MustRegister(ProxyInsertCount)
prometheus.MustRegister(ProxySearchVectors) prometheus.MustRegister(ProxySearchVectors)
prometheus.MustRegister(ProxyInsertVectors) prometheus.MustRegister(ProxyInsertVectors)
prometheus.MustRegister(ProxyLinkedSDKs) //prometheus.MustRegister(ProxyLinkedSDKs)
prometheus.MustRegister(ProxySearchLatency) prometheus.MustRegister(ProxySearchLatency)
prometheus.MustRegister(ProxySearchLatencyPerNQ) prometheus.MustRegister(ProxySearchLatencyPerNQ)
@ -314,7 +292,6 @@ func RegisterProxy() {
prometheus.MustRegister(ProxyMsgStreamObjectsForSearch) prometheus.MustRegister(ProxyMsgStreamObjectsForSearch)
prometheus.MustRegister(ProxyInsertLatency) prometheus.MustRegister(ProxyInsertLatency)
prometheus.MustRegister(ProxyInsertColToRowLatency)
prometheus.MustRegister(ProxySendInsertReqLatency) prometheus.MustRegister(ProxySendInsertReqLatency)
prometheus.MustRegister(ProxyCacheHitCounter) prometheus.MustRegister(ProxyCacheHitCounter)

View File

@ -37,9 +37,7 @@ var (
Subsystem: typeutil.QueryCoordRole, Subsystem: typeutil.QueryCoordRole,
Name: "num_entities", Name: "num_entities",
Help: "Number of entities in collection.", Help: "Number of entities in collection.",
}, []string{ }, []string{})
collectionIDLabelName,
})
QueryCoordLoadCount = prometheus.NewCounterVec( QueryCoordLoadCount = prometheus.NewCounterVec(
prometheus.CounterOpts{ prometheus.CounterOpts{

View File

@ -40,7 +40,6 @@ var (
Name: "num_partitions", Name: "num_partitions",
Help: "Number of partitions per collection in QueryNode.", Help: "Number of partitions per collection in QueryNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -51,7 +50,6 @@ var (
Name: "num_segments", Name: "num_segments",
Help: "Number of segments per collection in QueryNode.", Help: "Number of segments per collection in QueryNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -62,7 +60,6 @@ var (
Name: "num_dml_channels", Name: "num_dml_channels",
Help: "Number of dmlChannels per collection in QueryNode.", Help: "Number of dmlChannels per collection in QueryNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -73,7 +70,6 @@ var (
Name: "num_delta_channels", Name: "num_delta_channels",
Help: "Number of deltaChannels per collection in QueryNode.", Help: "Number of deltaChannels per collection in QueryNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -84,18 +80,6 @@ var (
Name: "num_consumers", Name: "num_consumers",
Help: "Number of consumers per collection in QueryNode.", Help: "Number of consumers per collection in QueryNode.",
}, []string{ }, []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, nodeIDLabelName,
}) })
@ -106,9 +90,9 @@ var (
Name: "sq_count", Name: "sq_count",
Help: "Search and query requests statistic in QueryNode.", Help: "Search and query requests statistic in QueryNode.",
}, []string{ }, []string{
statusLabelName,
queryTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
queryTypeLabelName,
statusLabelName,
}) })
QueryNodeSQReqLatency = prometheus.NewHistogramVec( QueryNodeSQReqLatency = prometheus.NewHistogramVec(
@ -119,8 +103,8 @@ var (
Help: "Search and query requests latency in QueryNode.", Help: "Search and query requests latency in QueryNode.",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
queryTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
queryTypeLabelName,
}) })
QueryNodeSQLatencyInQueue = prometheus.NewHistogramVec( QueryNodeSQLatencyInQueue = prometheus.NewHistogramVec(
@ -131,8 +115,8 @@ var (
Help: "The search and query latency in queue(unsolved buffer) in QueryNode.", Help: "The search and query latency in queue(unsolved buffer) in QueryNode.",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
queryTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
queryTypeLabelName,
}) })
QueryNodeSQSegmentLatency = prometheus.NewHistogramVec( QueryNodeSQSegmentLatency = prometheus.NewHistogramVec(
@ -143,9 +127,9 @@ var (
Help: "The search and query on segments(sealed/growing segments).", Help: "The search and query on segments(sealed/growing segments).",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
nodeIDLabelName,
queryTypeLabelName, queryTypeLabelName,
segmentTypeLabelName, segmentTypeLabelName,
nodeIDLabelName,
}) })
QueryNodeSQSegmentLatencyInCore = prometheus.NewHistogramVec( QueryNodeSQSegmentLatencyInCore = prometheus.NewHistogramVec(
@ -156,8 +140,8 @@ var (
Help: "The search and query latency in core.", Help: "The search and query latency in core.",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
queryTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
queryTypeLabelName,
}) })
QueryNodeTranslateHitsLatency = prometheus.NewHistogramVec( QueryNodeTranslateHitsLatency = prometheus.NewHistogramVec(
@ -179,8 +163,8 @@ var (
Help: "The search and query latency in reduce(local reduce) in QueryNode.", Help: "The search and query latency in reduce(local reduce) in QueryNode.",
Buckets: buckets, Buckets: buckets,
}, []string{ }, []string{
segmentTypeLabelName,
nodeIDLabelName, nodeIDLabelName,
segmentTypeLabelName,
}) })
QueryNodeLoadSegmentLatency = prometheus.NewHistogramVec( QueryNodeLoadSegmentLatency = prometheus.NewHistogramVec(
@ -201,7 +185,6 @@ var (
Name: "service_time", Name: "service_time",
Help: "ServiceTimes of collections in QueryNode.", Help: "ServiceTimes of collections in QueryNode.",
}, []string{ }, []string{
collectionIDLabelName,
nodeIDLabelName, nodeIDLabelName,
}) })
@ -224,7 +207,6 @@ func RegisterQueryNode() {
prometheus.MustRegister(QueryNodeNumDmlChannels) prometheus.MustRegister(QueryNodeNumDmlChannels)
prometheus.MustRegister(QueryNodeNumDeltaChannels) prometheus.MustRegister(QueryNodeNumDeltaChannels)
prometheus.MustRegister(QueryNodeNumConsumers) prometheus.MustRegister(QueryNodeNumConsumers)
prometheus.MustRegister(QueryNodeNumReaders)
prometheus.MustRegister(QueryNodeSQCount) prometheus.MustRegister(QueryNodeSQCount)
prometheus.MustRegister(QueryNodeSQReqLatency) prometheus.MustRegister(QueryNodeSQReqLatency)
prometheus.MustRegister(QueryNodeSQLatencyInQueue) prometheus.MustRegister(QueryNodeSQLatencyInQueue)

View File

@ -163,7 +163,7 @@ var (
Subsystem: typeutil.RootCoordRole, Subsystem: typeutil.RootCoordRole,
Name: "ddl_read_type_latency", Name: "ddl_read_type_latency",
Help: "The latency for read type of DDL operations", 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 records the latency for write type of DDL operations.
RootCoordDDLWriteTypeLatency = prometheus.NewHistogramVec( RootCoordDDLWriteTypeLatency = prometheus.NewHistogramVec(
@ -172,7 +172,7 @@ var (
Subsystem: typeutil.RootCoordRole, Subsystem: typeutil.RootCoordRole,
Name: "ddl_write_type_latency", Name: "ddl_write_type_latency",
Help: "The latency for write type of DDL operations", Help: "The latency for write type of DDL operations",
}, []string{functionLabelName, collectionIDLabelName}) }, []string{functionLabelName})
// RootCoordSyncTimeTickLatency records the latency of sync time tick. // RootCoordSyncTimeTickLatency records the latency of sync time tick.
RootCoordSyncTimeTickLatency = prometheus.NewHistogram( RootCoordSyncTimeTickLatency = prometheus.NewHistogram(

View File

@ -399,9 +399,9 @@ func (mgr *singleTypeChannelsMgr) createMsgStream(collectionID UniqueID) error {
mgr.addStream(id, stream) mgr.addStream(id, stream)
mgr.updateCollection(collectionID, id) mgr.updateCollection(collectionID, id)
for _, pc := range pchans {
metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(collectionID, 10), "PChan").Inc() metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), pc).Inc()
}
return nil return nil
} }
@ -423,16 +423,20 @@ func (mgr *singleTypeChannelsMgr) getStream(collectionID UniqueID) (msgstream.Ms
} }
func (mgr *singleTypeChannelsMgr) removeStream(collectionID UniqueID) error { func (mgr *singleTypeChannelsMgr) removeStream(collectionID UniqueID) error {
ids, err := mgr.getAllVIDs(collectionID) channels, err := mgr.getChannels(collectionID)
if err != nil { if err != nil {
return err return err
} }
ids, err2 := mgr.getAllVIDs(collectionID)
if err2 != nil {
return err2
}
mgr.deleteVChansByVIDs(ids) mgr.deleteVChansByVIDs(ids)
mgr.deleteStreamByVIDs(ids) mgr.deleteStreamByVIDs(ids)
for _, pc := range channels {
metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(collectionID, 10), "PChan").Dec() metrics.ProxyMsgStreamObjectsForPChan.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), pc).Dec()
}
return nil return nil
} }
@ -482,7 +486,7 @@ func (mgr *channelsMgrImpl) getVChannels(collectionID UniqueID) ([]vChan, error)
} }
func (mgr *channelsMgrImpl) createDQLStream(collectionID UniqueID) 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) return mgr.dqlChannelsMgr.createMsgStream(collectionID)
} }
@ -491,7 +495,7 @@ func (mgr *channelsMgrImpl) getDQLStream(collectionID UniqueID) (msgstream.MsgSt
} }
func (mgr *channelsMgrImpl) removeDQLStream(collectionID UniqueID) error { 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) return mgr.dqlChannelsMgr.removeStream(collectionID)
} }

View File

@ -18,14 +18,9 @@ package proxy
import ( import (
"context" "context"
"strconv"
"sync" "sync"
"time" "time"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/metrics"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
@ -95,17 +90,15 @@ func (ticker *channelsTimeTickerImpl) initCurrents(current Timestamp) {
} }
func (ticker *channelsTimeTickerImpl) tick() error { func (ticker *channelsTimeTickerImpl) tick() error {
tr := timerecord.NewTimeRecorder("applyTimestamp")
now, err := ticker.tso.AllocOne() now, err := ticker.tso.AllocOne()
if err != nil { if err != nil {
log.Warn("Proxy channelsTimeTickerImpl failed to get ts from tso", zap.Error(err)) log.Warn("Proxy channelsTimeTickerImpl failed to get ts from tso", zap.Error(err))
return err return err
} }
metrics.ProxyApplyTimestampLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
stats, err := ticker.getStatisticsFunc() stats, err2 := ticker.getStatisticsFunc()
if err != nil { if err2 != nil {
log.Debug("Proxy channelsTimeTickerImpl failed to getStatistics", zap.Error(err)) log.Debug("Proxy channelsTimeTickerImpl failed to getStatistics", zap.Error(err2))
return nil return nil
} }

View File

@ -348,7 +348,7 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle
method := "HasCollection" method := "HasCollection"
tr := timerecord.NewTimeRecorder(method) tr := timerecord.NewTimeRecorder(method)
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.TotalLabel).Inc() metrics.TotalLabel).Inc()
log.Debug("HasCollection received", log.Debug("HasCollection received",
zap.String("traceID", traceID), zap.String("traceID", traceID),
@ -372,7 +372,7 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -402,7 +402,7 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -421,9 +421,8 @@ func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasColle
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return hct.result, nil return hct.result, nil
} }
@ -461,7 +460,7 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(), Reason: err.Error(),
@ -489,9 +488,9 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(lct.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(), Reason: err.Error(),
@ -508,11 +507,10 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(lct.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(lct.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return lct.result, nil return lct.result, nil
} }
@ -553,7 +551,7 @@ func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.Rele
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(), Reason: err.Error(),
@ -583,9 +581,9 @@ func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.Rele
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(rct.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(), Reason: err.Error(),
@ -603,11 +601,10 @@ func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.Rele
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(rct.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(rct.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return rct.result, nil return rct.result, nil
} }
@ -647,7 +644,7 @@ func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.Des
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.DescribeCollectionResponse{ return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -677,9 +674,9 @@ func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.Des
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dct.CollectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.DescribeCollectionResponse{ return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -699,11 +696,10 @@ func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.Des
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dct.CollectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(dct.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return dct.result, nil return dct.result, nil
} }
@ -743,7 +739,7 @@ func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvusp
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.GetCollectionStatisticsResponse{ return &milvuspb.GetCollectionStatisticsResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -774,9 +770,9 @@ func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvusp
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(g.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.GetCollectionStatisticsResponse{ return &milvuspb.GetCollectionStatisticsResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -796,11 +792,10 @@ func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvusp
zap.String("collection", request.CollectionName)) zap.String("collection", request.CollectionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(g.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(g.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return g.result, nil return g.result, nil
} }
@ -1102,7 +1097,7 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit
tr := timerecord.NewTimeRecorder(method) tr := timerecord.NewTimeRecorder(method)
//TODO: use collectionID instead of collectionName //TODO: use collectionID instead of collectionName
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.TotalLabel).Inc() metrics.TotalLabel).Inc()
hpt := &hasPartitionTask{ hpt := &hasPartitionTask{
ctx: ctx, ctx: ctx,
@ -1131,7 +1126,7 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit
zap.String("partition", request.PartitionName)) zap.String("partition", request.PartitionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1167,7 +1162,7 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit
zap.String("partition", request.PartitionName)) zap.String("partition", request.PartitionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1190,9 +1185,8 @@ func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartit
zap.String("partition", request.PartitionName)) zap.String("partition", request.PartitionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return hpt.result, nil return hpt.result, nil
} }
@ -1234,7 +1228,7 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1267,9 +1261,9 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(lpt.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1289,11 +1283,10 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(lpt.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(lpt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return lpt.result, nil return lpt.result, nil
} }
@ -1336,7 +1329,7 @@ func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.Rele
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1369,9 +1362,9 @@ func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.Rele
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(rpt.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1391,11 +1384,10 @@ func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.Rele
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(rpt.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(rpt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return rpt.result, nil return rpt.result, nil
} }
@ -1440,7 +1432,7 @@ func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb
zap.String("partition", request.PartitionName)) zap.String("partition", request.PartitionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.GetPartitionStatisticsResponse{ return &milvuspb.GetPartitionStatisticsResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1475,9 +1467,9 @@ func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb
zap.String("partition", request.PartitionName)) zap.String("partition", request.PartitionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(g.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.GetPartitionStatisticsResponse{ return &milvuspb.GetPartitionStatisticsResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1499,11 +1491,10 @@ func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb
zap.String("partition", request.PartitionName)) zap.String("partition", request.PartitionName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(g.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(g.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return g.result, nil return g.result, nil
} }
@ -1532,7 +1523,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar
tr := timerecord.NewTimeRecorder(method) tr := timerecord.NewTimeRecorder(method)
//TODO: use collectionID instead of collectionName //TODO: use collectionID instead of collectionName
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.TotalLabel).Inc() metrics.TotalLabel).Inc()
log.Debug( log.Debug(
rpcReceived(method), rpcReceived(method),
@ -1549,7 +1540,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar
zap.Any("request", request)) zap.Any("request", request))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.ShowPartitionsResponse{ return &milvuspb.ShowPartitionsResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1584,7 +1575,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar
zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames)) zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.ShowPartitionsResponse{ return &milvuspb.ShowPartitionsResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1606,9 +1597,8 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar
zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames)) zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return spt.result, nil return spt.result, nil
} }
@ -1653,7 +1643,7 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde
zap.Any("extra_params", request.ExtraParams)) zap.Any("extra_params", request.ExtraParams))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1688,9 +1678,9 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde
zap.Any("extra_params", request.ExtraParams)) zap.Any("extra_params", request.ExtraParams))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(cit.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1711,11 +1701,10 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde
zap.Any("extra_params", request.ExtraParams)) zap.Any("extra_params", request.ExtraParams))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(cit.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(cit.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return cit.result, nil return cit.result, nil
} }
@ -1764,7 +1753,7 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe
zap.String("index name", indexName)) zap.String("index name", indexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.DescribeIndexResponse{ return &milvuspb.DescribeIndexResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1805,9 +1794,9 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe
errCode = dit.result.Status.GetErrorCode() errCode = dit.result.Status.GetErrorCode()
} }
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dit.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.DescribeIndexResponse{ return &milvuspb.DescribeIndexResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -1830,11 +1819,10 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe
zap.String("index name", indexName)) zap.String("index name", indexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dit.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(dit.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return dit.result, nil 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("field", request.FieldName),
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1913,9 +1901,9 @@ func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexReq
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dit.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &commonpb.Status{ return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -1936,11 +1924,10 @@ func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexReq
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dit.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(dit.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return dit.result, nil return dit.result, nil
} }
@ -1989,7 +1976,7 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb.
zap.String("field", request.FieldName), zap.String("field", request.FieldName),
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.GetIndexBuildProgressResponse{ return &milvuspb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2025,9 +2012,9 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb.
zap.String("field", request.FieldName), zap.String("field", request.FieldName),
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(gibpt.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.GetIndexBuildProgressResponse{ return &milvuspb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2051,11 +2038,10 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb.
zap.Any("result", gibpt.result)) zap.Any("result", gibpt.result))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(gibpt.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(gibpt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return gibpt.result, nil return gibpt.result, nil
} }
@ -2103,7 +2089,7 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.AbandonLabel).Inc() metrics.AbandonLabel).Inc()
return &milvuspb.GetIndexStateResponse{ return &milvuspb.GetIndexStateResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2140,9 +2126,9 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dipt.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.GetIndexStateResponse{ return &milvuspb.GetIndexStateResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2165,11 +2151,10 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex
zap.String("index name", request.IndexName)) zap.String("index name", request.IndexName))
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dipt.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(dipt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return dipt.result, nil 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 { if err := node.sched.dmQueue.Enqueue(it); err != nil {
log.Debug("Failed to enqueue insert task: " + err.Error()) 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 return constructFailedResponse(err), nil
} }
@ -2263,9 +2248,9 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest)
if err := it.WaitToFinish(); err != nil { if err := it.WaitToFinish(); err != nil {
log.Debug("Failed to execute insert task in task scheduler: "+err.Error(), zap.String("traceID", traceID)) 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), 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), metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
strconv.FormatInt(it.CollectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return constructFailedResponse(err), nil 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) it.result.InsertCnt = int64(it.req.NumRows)
metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.ProxyInsertCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
strconv.FormatInt(it.CollectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyInsertVectors.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.ProxyInsertVectors.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Add(float64(it.result.InsertCnt))
strconv.FormatInt(it.CollectionID, 10)).Add(float64(it.result.InsertCnt)) metrics.ProxyInsertLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.ProxyInsertLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
strconv.FormatInt(it.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return it.result, nil 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 { if err := node.sched.dmQueue.Enqueue(dt); err != nil {
log.Error("Failed to enqueue delete task: "+err.Error(), zap.String("traceID", traceID)) log.Error("Failed to enqueue delete task: "+err.Error(), zap.String("traceID", traceID))
metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
request.CollectionName, metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.MutationResult{ return &milvuspb.MutationResult{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2374,9 +2357,9 @@ func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest)
if err := dt.WaitToFinish(); err != nil { if err := dt.WaitToFinish(); err != nil {
log.Error("Failed to execute delete task in task scheduler: "+err.Error(), zap.String("traceID", traceID)) 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, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dt.collectionID, 10), metrics.FailLabel).Inc() metrics.FailLabel).Inc()
return &milvuspb.MutationResult{ return &milvuspb.MutationResult{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, 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, 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, metrics.ProxyDMLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
strconv.FormatInt(dt.collectionID, 10), metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDMLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(dt.collectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
return dt.result, nil return dt.result, nil
} }
@ -2459,7 +2441,7 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest)
zap.Uint64("guarantee_timestamp", guaranteeTs)) zap.Uint64("guarantee_timestamp", guaranteeTs))
metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
request.CollectionName, metrics.SearchLabel, metrics.AbandonLabel).Inc() metrics.SearchLabel, metrics.AbandonLabel).Inc()
return &milvuspb.SearchResults{ return &milvuspb.SearchResults{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2502,9 +2484,10 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest)
zap.Uint64("travel_timestamp", travelTs), zap.Uint64("travel_timestamp", travelTs),
zap.Uint64("guarantee_timestamp", guaranteeTs)) zap.Uint64("guarantee_timestamp", guaranteeTs))
metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), 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{ return &milvuspb.SearchResults{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2530,16 +2513,15 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest)
zap.Uint64("guarantee_timestamp", guaranteeTs)) zap.Uint64("guarantee_timestamp", guaranteeTs))
metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), 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), 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), 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() searchDur := tr.ElapseSpan().Milliseconds()
metrics.ProxySearchLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.ProxySearchLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
strconv.FormatInt(qt.CollectionID, 10), metrics.SearchLabel).Observe(float64(searchDur)) metrics.SearchLabel).Observe(float64(searchDur))
metrics.ProxySearchLatencyPerNQ.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.ProxySearchLatencyPerNQ.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(searchDur) / float64(qt.result.Results.NumQueries))
strconv.FormatInt(qt.CollectionID, 10)).Observe(float64(searchDur) / float64(qt.result.Results.NumQueries))
return qt.result, nil return qt.result, nil
} }
@ -2687,7 +2669,7 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (*
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
request.CollectionName, metrics.QueryLabel, metrics.FailLabel).Inc() metrics.QueryLabel, metrics.FailLabel).Inc()
return &milvuspb.QueryResults{ return &milvuspb.QueryResults{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError, ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -2721,9 +2703,9 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (*
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), 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), 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{ return &milvuspb.QueryResults{
Status: &commonpb.Status{ Status: &commonpb.Status{
@ -2745,13 +2727,13 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (*
zap.Any("partitions", request.PartitionNames)) zap.Any("partitions", request.PartitionNames))
metrics.ProxySearchCount.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), 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), 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), 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), 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{ return &milvuspb.QueryResults{
Status: qt.result.Status, Status: qt.result.Status,
FieldsData: qt.result.FieldsData, FieldsData: qt.result.FieldsData,
@ -3497,7 +3479,7 @@ func (node *Proxy) GetPersistentSegmentInfo(ctx context.Context, req *milvuspb.G
method := "GetPersistentSegmentInfo" method := "GetPersistentSegmentInfo"
tr := timerecord.NewTimeRecorder(method) tr := timerecord.NewTimeRecorder(method)
metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), 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) segments, err := node.getSegmentsOfCollection(ctx, req.DbName, req.CollectionName)
if err != nil { if err != nil {
resp.Status.Reason = fmt.Errorf("getSegmentsOfCollection, err:%w", err).Error() 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, metrics.ProxyDQLFunctionCall.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method,
req.CollectionName, metrics.SuccessLabel).Inc() metrics.SuccessLabel).Inc()
metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method, metrics.ProxyDQLReqLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
req.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
resp.Status.ErrorCode = commonpb.ErrorCode_Success resp.Status.ErrorCode = commonpb.ErrorCode_Success
resp.Infos = persistentInfos resp.Infos = persistentInfos
return resp, nil return resp, nil
@ -3725,7 +3706,7 @@ func (node *Proxy) RegisterLink(ctx context.Context, req *milvuspb.RegisterLinkR
}, },
}, nil }, nil
} }
metrics.ProxyLinkedSDKs.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Inc() //metrics.ProxyLinkedSDKs.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Inc()
return &milvuspb.RegisterLinkResponse{ return &milvuspb.RegisterLinkResponse{
Address: nil, Address: nil,
Status: &commonpb.Status{ Status: &commonpb.Status{

View File

@ -22,21 +22,17 @@ import (
"fmt" "fmt"
"strconv" "strconv"
"sync" "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" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/typeutil" "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] collInfo, ok := m.collInfo[collectionName]
if !ok { 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() m.mu.RUnlock()
coll, err := m.describeCollection(ctx, collectionName) coll, err := m.describeCollection(ctx, collectionName)
if err != nil { if err != nil {
@ -118,14 +115,13 @@ func (m *MetaCache) GetCollectionID(ctx context.Context, collectionName string)
} }
m.mu.Lock() m.mu.Lock()
defer m.mu.Unlock() defer m.mu.Unlock()
tr := timerecord.NewTimeRecorder("UpdateCache")
m.updateCollection(coll, collectionName) m.updateCollection(coll, collectionName)
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
collInfo = m.collInfo[collectionName] collInfo = m.collInfo[collectionName]
return collInfo.collID, nil return collInfo.collID, nil
} }
defer m.mu.RUnlock() 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 return collInfo.collID, nil
} }
@ -139,6 +135,8 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, collectionName string
m.mu.RUnlock() m.mu.RUnlock()
if !ok { 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) coll, err := m.describeCollection(ctx, collectionName)
if err != nil { if err != nil {
return nil, err return nil, err
@ -147,8 +145,9 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, collectionName string
defer m.mu.Unlock() defer m.mu.Unlock()
m.updateCollection(coll, collectionName) m.updateCollection(coll, collectionName)
collInfo = m.collInfo[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{ return &collectionInfo{
collID: collInfo.collID, collID: collInfo.collID,
schema: collInfo.schema, schema: collInfo.schema,
@ -163,7 +162,8 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, collectionName stri
collInfo, ok := m.collInfo[collectionName] collInfo, ok := m.collInfo[collectionName]
if !ok { 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() m.mu.RUnlock()
coll, err := m.describeCollection(ctx, collectionName) coll, err := m.describeCollection(ctx, collectionName)
if err != nil { if err != nil {
@ -176,12 +176,14 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, collectionName stri
defer m.mu.Unlock() defer m.mu.Unlock()
m.updateCollection(coll, collectionName) m.updateCollection(coll, collectionName)
collInfo = m.collInfo[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.String("collection name ", collectionName),
zap.Any("time take ", time.Since(t0))) zap.Any("time (milliseconds) take ", tr.ElapseSpan().Milliseconds()))
return collInfo.schema, nil return collInfo.schema, nil
} }
defer m.mu.RUnlock() defer m.mu.RUnlock()
metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetCollectionSchema", metrics.CacheHitLabel).Inc()
return collInfo.schema, nil 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 { 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() m.mu.RUnlock()
partitions, err := m.showPartitions(ctx, collectionName) partitions, err := m.showPartitions(ctx, collectionName)
@ -234,6 +238,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, collectionName string) (m
if err != nil { if err != nil {
return nil, err 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)) log.Debug("proxy", zap.Any("GetPartitions:partitions after update", partitions), zap.Any("collectionName", collectionName))
ret := make(map[string]typeutil.UniqueID) ret := make(map[string]typeutil.UniqueID)
partInfo := m.collInfo[collectionName].partInfo partInfo := m.collInfo[collectionName].partInfo
@ -244,6 +249,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, collectionName string) (m
} }
defer m.mu.RUnlock() defer m.mu.RUnlock()
metrics.ProxyCacheHitCounter.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "GetPartitions", metrics.CacheHitLabel).Inc()
ret := make(map[string]typeutil.UniqueID) ret := make(map[string]typeutil.UniqueID)
partInfo := m.collInfo[collectionName].partInfo partInfo := m.collInfo[collectionName].partInfo
@ -273,6 +279,8 @@ func (m *MetaCache) GetPartitionInfo(ctx context.Context, collectionName string,
m.mu.RUnlock() m.mu.RUnlock()
if !ok { 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) partitions, err := m.showPartitions(ctx, collectionName)
if err != nil { if err != nil {
return nil, err return nil, err
@ -284,13 +292,14 @@ func (m *MetaCache) GetPartitionInfo(ctx context.Context, collectionName string,
if err != nil { if err != nil {
return nil, err 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)) log.Debug("proxy", zap.Any("GetPartitionID:partitions after update", partitions), zap.Any("collectionName", collectionName))
partInfo, ok = m.collInfo[collectionName].partInfo[partitionName] partInfo, ok = m.collInfo[collectionName].partInfo[partitionName]
if !ok { if !ok {
return nil, fmt.Errorf("partitionID of partitionName:%s can not be find", partitionName) 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{ return &partitionInfo{
partitionID: partInfo.partitionID, partitionID: partInfo.partitionID,
createdTimestamp: partInfo.createdTimestamp, createdTimestamp: partInfo.createdTimestamp,

View File

@ -313,11 +313,7 @@ func (node *Proxy) sendChannelsTimeTickLoop() {
DefaultTimestamp: maxTs, DefaultTimestamp: maxTs,
} }
for idx, channel := range channels { metrics.ProxySyncTimeTick.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), "DefaultTimestamp").Set(float64(maxTs))
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))
status, err := node.rootCoord.UpdateChannelTimeTick(node.ctx, req) status, err := node.rootCoord.UpdateChannelTimeTick(node.ctx, req)
if err != nil { if err != nil {

View File

@ -833,7 +833,7 @@ func (it *insertTask) Execute(ctx context.Context) error {
return err return err
} }
sendMsgDur := tr.Record("send insert request to message stream") 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 return nil
} }
@ -1495,7 +1495,8 @@ func (st *searchTask) Execute(ctx context.Context) error {
zap.Int("length of search msg", len(msgPack.Msgs)), zap.Int("length of search msg", len(msgPack.Msgs)),
zap.Uint64("timeoutTs", st.SearchRequest.TimeoutTimestamp)) zap.Uint64("timeoutTs", st.SearchRequest.TimeoutTimestamp))
sendMsgDur := tr.Record("send search msg to message stream") 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 return err
} }
@ -1712,7 +1713,7 @@ func (st *searchTask) PostExecute(ctx context.Context) error {
log.Debug("Proxy Search PostExecute stage1", log.Debug("Proxy Search PostExecute stage1",
zap.Any("len(filterSearchResults)", len(filterSearchResults))) 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") tr.Record("Proxy Search PostExecute stage1 done")
if len(filterSearchResults) <= 0 || errNum > 0 { if len(filterSearchResults) <= 0 || errNum > 0 {
st.result = &milvuspb.SearchResults{ st.result = &milvuspb.SearchResults{
@ -1729,7 +1730,7 @@ func (st *searchTask) PostExecute(ctx context.Context) error {
if err != nil { if err != nil {
return err 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))) log.Debug("Proxy Search PostExecute stage2", zap.Any("len(validSearchResults)", len(validSearchResults)))
if len(validSearchResults) <= 0 { if len(validSearchResults) <= 0 {
filterReason += "empty search result\n" filterReason += "empty search result\n"
@ -1754,7 +1755,7 @@ func (st *searchTask) PostExecute(ctx context.Context) error {
if err != nil { if err != nil {
return err 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 st.result.CollectionName = st.collectionName
schema, err := globalMetaCache.GetCollectionSchema(ctx, st.query.CollectionName) schema, err := globalMetaCache.GetCollectionSchema(ctx, st.query.CollectionName)

View File

@ -19,10 +19,13 @@ package proxy
import ( import (
"context" "context"
"fmt" "fmt"
"strconv"
"time" "time"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/timerecord"
) )
// timestampAllocator implements tsoAllocator. // timestampAllocator implements tsoAllocator.
@ -43,6 +46,7 @@ func newTimestampAllocator(ctx context.Context, tso timestampAllocatorInterface,
} }
func (ta *timestampAllocator) alloc(count uint32) ([]Timestamp, error) { func (ta *timestampAllocator) alloc(count uint32) ([]Timestamp, error) {
tr := timerecord.NewTimeRecorder("applyTimestamp")
ctx, cancel := context.WithTimeout(ta.ctx, 5*time.Second) ctx, cancel := context.WithTimeout(ta.ctx, 5*time.Second)
req := &rootcoordpb.AllocTimestampRequest{ req := &rootcoordpb.AllocTimestampRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
@ -55,7 +59,10 @@ func (ta *timestampAllocator) alloc(count uint32) ([]Timestamp, error) {
} }
resp, err := ta.tso.AllocTimestamp(ctx, req) 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 { if err != nil {
return nil, fmt.Errorf("syncTimestamp Failed:%w", err) return nil, fmt.Errorf("syncTimestamp Failed:%w", err)

View File

@ -37,8 +37,8 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "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"
"github.com/milvus-io/milvus/internal/util/funcutil"
) )
const ( const (
@ -954,7 +954,7 @@ func (m *MetaReplica) getWatchedChannelsByNodeID(nodeID int64) *querypb.Unsubscr
// get dmChannel/search channel the node has watched // get dmChannel/search channel the node has watched
for _, channelInfo := range dmChannelInfos { for _, channelInfo := range dmChannelInfos {
collectionID := channelInfo.CollectionID collectionID := channelInfo.CollectionID
dmChannel := rootcoord.ToPhysicalChannel(channelInfo.DmChannel) dmChannel := funcutil.ToPhysicalChannel(channelInfo.DmChannel)
if _, ok := colID2DmChannels[collectionID]; !ok { if _, ok := colID2DmChannels[collectionID]; !ok {
colID2DmChannels[collectionID] = []string{} colID2DmChannels[collectionID] = []string{}
} }
@ -977,7 +977,7 @@ func (m *MetaReplica) getWatchedChannelsByNodeID(nodeID int64) *querypb.Unsubscr
} }
deltaChannels := make([]string, len(deltaChanelInfos)) deltaChannels := make([]string, len(deltaChanelInfos))
for offset, channelInfo := range deltaChanelInfos { for offset, channelInfo := range deltaChanelInfos {
deltaChannels[offset] = rootcoord.ToPhysicalChannel(channelInfo.ChannelName) deltaChannels[offset] = funcutil.ToPhysicalChannel(channelInfo.ChannelName)
} }
colID2DeltaChannels[collectionID] = deltaChannels colID2DeltaChannels[collectionID] = deltaChannels
} }

View File

@ -52,14 +52,17 @@ func (s *segmentsInfo) loadSegments() error {
if err != nil { if err != nil {
return return
} }
numRowsCnt := float64(0)
for _, v := range values { for _, v := range values {
segment := &querypb.SegmentInfo{} segment := &querypb.SegmentInfo{}
if err = proto.Unmarshal([]byte(v), segment); err != nil { if err = proto.Unmarshal([]byte(v), segment); err != nil {
return return
} }
s.segmentIDMap[segment.GetSegmentID()] = segment 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 return err
} }
@ -77,7 +80,7 @@ func (s *segmentsInfo) saveSegment(segment *querypb.SegmentInfo) error {
return err return err
} }
s.segmentIDMap[segment.GetSegmentID()] = segment 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 return nil
} }
@ -89,7 +92,7 @@ func (s *segmentsInfo) removeSegment(segment *querypb.SegmentInfo) error {
return err return err
} }
delete(s.segmentIDMap, segment.GetSegmentID()) 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 return nil
} }

View File

@ -32,7 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb" "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" "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) { 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 { if err != nil {
return nil, err return nil, err
} }

View File

@ -117,7 +117,7 @@ OUTER:
c.vChannels = append(c.vChannels, dstChan) 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 // getVChannels get virtual channels of collection
@ -145,7 +145,7 @@ func (c *Collection) removeVChannel(channel Channel) {
zap.Any("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 // addPChannels add physical channels to physical channels of collection
@ -242,7 +242,7 @@ OUTER:
c.vDeltaChannels = append(c.vDeltaChannels, dstChan) 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) { func (c *Collection) removeVDeltaChannel(channel Channel) {
@ -260,7 +260,7 @@ func (c *Collection) removeVDeltaChannel(channel Channel) {
zap.Any("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 // setReleaseTime records when collection is released

View File

@ -242,14 +242,7 @@ func (colReplica *collectionReplica) removeCollectionPrivate(collectionID Unique
delete(colReplica.collections, collectionID) delete(colReplica.collections, collectionID)
metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Set(float64(len(colReplica.collections))) 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.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Sub(float64(len(collection.partitionIDs)))
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)
return nil return nil
} }
@ -409,7 +402,7 @@ func (colReplica *collectionReplica) addPartitionPrivate(collectionID UniqueID,
colReplica.partitions[partitionID] = newPartition 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 return nil
} }
@ -443,7 +436,7 @@ func (colReplica *collectionReplica) removePartitionPrivate(partitionID UniqueID
collection.removePartitionID(partitionID) collection.removePartitionID(partitionID)
delete(colReplica.partitions, 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 return nil
} }
@ -551,7 +544,7 @@ func (colReplica *collectionReplica) addSegmentPrivate(segmentID UniqueID, parti
partition.addSegmentID(segmentID) partition.addSegmentID(segmentID)
colReplica.segments[segmentID] = segment 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 return nil
} }
@ -589,7 +582,7 @@ func (colReplica *collectionReplica) removeSegmentPrivate(segmentID UniqueID) er
delete(colReplica.segments, segmentID) delete(colReplica.segments, segmentID)
deleteSegment(segment) 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 return nil
} }

View File

@ -39,6 +39,7 @@ type queryNodeFlowGraph struct {
channel Channel channel Channel
flowGraph *flowgraph.TimeTickedFlowGraph flowGraph *flowgraph.TimeTickedFlowGraph
dmlStream msgstream.MsgStream dmlStream msgstream.MsgStream
consumerCnt int
} }
// newQueryNodeFlowGraph returns a new queryNodeFlowGraph // newQueryNodeFlowGraph returns a new queryNodeFlowGraph
@ -208,8 +209,8 @@ func (q *queryNodeFlowGraph) consumeFlowGraph(channel Channel, subName ConsumeSu
zap.Any("channel", channel), zap.Any("channel", channel),
zap.Any("subName", subName), zap.Any("subName", subName),
) )
q.consumerCnt++
metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc()
return nil return nil
} }
@ -224,8 +225,8 @@ func (q *queryNodeFlowGraph) consumeFlowGraphFromLatest(channel Channel, subName
zap.Any("channel", channel), zap.Any("channel", channel),
zap.Any("subName", subName), zap.Any("subName", subName),
) )
q.consumerCnt++
metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc()
return nil return nil
} }
@ -237,8 +238,8 @@ func (q *queryNodeFlowGraph) seekQueryNodeFlowGraph(position *internalpb.MsgPosi
zap.Any("collectionID", q.collectionID), zap.Any("collectionID", q.collectionID),
zap.Any("channel", position.ChannelName), zap.Any("channel", position.ChannelName),
) )
q.consumerCnt++
metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(q.collectionID), fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() metrics.QueryNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc()
return err return err
} }
@ -246,6 +247,9 @@ func (q *queryNodeFlowGraph) seekQueryNodeFlowGraph(position *internalpb.MsgPosi
func (q *queryNodeFlowGraph) close() { func (q *queryNodeFlowGraph) close() {
q.cancel() q.cancel()
q.flowGraph.Close() 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", log.Debug("stop query node flow graph",
zap.Any("collectionID", q.collectionID), zap.Any("collectionID", q.collectionID),
zap.Any("channel", q.channel), zap.Any("channel", q.channel),

View File

@ -340,7 +340,7 @@ func (q *queryCollection) setServiceableTime(t Timestamp) {
} }
q.serviceableTime = t q.serviceableTime = t
ps, _ := tsoutil.ParseHybridTs(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 { func (q *queryCollection) checkTimeout(msg queryMsg) bool {
@ -674,12 +674,12 @@ func (q *queryCollection) doUnsolvedQueryMsg() {
) )
switch msgType { switch msgType {
case commonpb.MsgType_Retrieve: case commonpb.MsgType_Retrieve:
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(metrics.QueryLabel, metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID),
fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(m.RecordSpan().Milliseconds())) metrics.QueryLabel).Observe(float64(m.RecordSpan().Milliseconds()))
err = q.retrieve(m) err = q.retrieve(m)
case commonpb.MsgType_Search: case commonpb.MsgType_Search:
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(metrics.SearchLabel, metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID),
fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(m.RecordSpan().Milliseconds())) metrics.SearchLabel).Observe(float64(m.RecordSpan().Milliseconds()))
err = q.search(m) err = q.search(m)
default: default:
err := fmt.Errorf("receive invalid msgType = %d", msgType) err := fmt.Errorf("receive invalid msgType = %d", msgType)
@ -1132,8 +1132,8 @@ func (q *queryCollection) search(msg queryMsg) error {
if err != nil { if err != nil {
return err return err
} }
metrics.QueryNodeSQReqLatency.WithLabelValues(metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(msg.ElapseSpan().Milliseconds())) metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.SearchLabel).Observe(float64(msg.ElapseSpan().Milliseconds()))
metrics.QueryNodeSQCount.WithLabelValues(metrics.SuccessLabel, metrics.SearchLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() 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.Record(fmt.Sprintf("publish empty search result done, msgID = %d", searchMsg.ID()))
tr.Elapse(fmt.Sprintf("all 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 { if err != nil {
return err 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 var offset int64
for index := range searchRequests { for index := range searchRequests {
@ -1244,11 +1244,11 @@ func (q *queryCollection) search(msg queryMsg) error {
if err != nil { if err != nil {
return err return err
} }
metrics.QueryNodeSQReqLatency.WithLabelValues(metrics.SearchLabel, metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID),
fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(msg.ElapseSpan().Milliseconds())) metrics.SearchLabel).Observe(float64(msg.ElapseSpan().Milliseconds()))
metrics.QueryNodeSQCount.WithLabelValues(metrics.SuccessLabel, metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID),
metrics.SearchLabel, metrics.SearchLabel,
fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() metrics.SuccessLabel).Inc()
tr.Record(fmt.Sprintf("publish search result, msgID = %d", searchMsg.ID())) tr.Record(fmt.Sprintf("publish search result, msgID = %d", searchMsg.ID()))
} }
sp.LogFields(oplog.String("statistical time", "stats done")) sp.LogFields(oplog.String("statistical time", "stats done"))
@ -1332,7 +1332,7 @@ func (q *queryCollection) retrieve(msg queryMsg) error {
return err return err
} }
reduceDuration := tr.Record(fmt.Sprintf("merge result done, msgID = %d", retrieveMsg.ID())) 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 resultChannelInt := 0
retrieveResultMsg := &msgstream.RetrieveResultMsg{ retrieveResultMsg := &msgstream.RetrieveResultMsg{
@ -1357,8 +1357,8 @@ func (q *queryCollection) retrieve(msg queryMsg) error {
if err != nil { if err != nil {
return err return err
} }
metrics.QueryNodeSQCount.WithLabelValues(metrics.SuccessLabel, metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Inc() metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel, metrics.SuccessLabel).Inc()
metrics.QueryNodeSQReqLatency.WithLabelValues(metrics.QueryLabel, fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(msg.ElapseSpan().Milliseconds())) metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID), metrics.QueryLabel).Observe(float64(msg.ElapseSpan().Milliseconds()))
log.Debug("QueryNode publish RetrieveResultMsg", log.Debug("QueryNode publish RetrieveResultMsg",
zap.Int64("msgID", retrieveMsg.ID()), 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 { 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) 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 { 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) return q.publishRetrieveResultWithCtx(q.releaseCtx, result, nodeID)
} }
@ -1458,7 +1458,7 @@ func (q *queryCollection) publishFailedQueryResultWithCtx(ctx context.Context, m
case commonpb.MsgType_Retrieve: case commonpb.MsgType_Retrieve:
retrieveMsg := msg.(*msgstream.RetrieveMsg) retrieveMsg := msg.(*msgstream.RetrieveMsg)
baseResult.MsgType = commonpb.MsgType_RetrieveResult 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{ return q.publishRetrieveResult(&internalpb.RetrieveResults{
Base: baseResult, Base: baseResult,
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg}, 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: case commonpb.MsgType_Search:
searchMsg := msg.(*msgstream.SearchMsg) searchMsg := msg.(*msgstream.SearchMsg)
baseResult.MsgType = commonpb.MsgType_SearchResult 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{ return q.publishSearchResultWithCtx(ctx, &internalpb.SearchResults{
Base: baseResult, Base: baseResult,
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg}, Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: errMsg},

View File

@ -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))) log.Debug("do search on segment", zap.Int64("segmentID", s.segmentID), zap.Int32("segmentType", int32(s.segmentType)))
tr := timerecord.NewTimeRecorder("cgoSearch") tr := timerecord.NewTimeRecorder("cgoSearch")
status := C.Search(s.segmentPtr, plan.cSearchPlan, cPlaceHolderGroup, ts, &searchResult.cSearchResult, C.int64_t(s.segmentID)) 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 { if err := HandleCStatus(&status, "Search failed"); err != nil {
return nil, err return nil, err
} }
@ -343,8 +343,8 @@ func (s *Segment) retrieve(plan *RetrievePlan) (*segcorepb.RetrieveResults, erro
ts := C.uint64_t(plan.Timestamp) ts := C.uint64_t(plan.Timestamp)
tr := timerecord.NewTimeRecorder("cgoRetrieve") tr := timerecord.NewTimeRecorder("cgoRetrieve")
status := C.Retrieve(s.segmentPtr, plan.cRetrievePlan, ts, &retrieveResult.cRetrieveResult) status := C.Retrieve(s.segmentPtr, plan.cRetrievePlan, ts, &retrieveResult.cRetrieveResult)
metrics.QueryNodeSQSegmentLatencyInCore.WithLabelValues(metrics.QueryLabel, metrics.QueryNodeSQSegmentLatencyInCore.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID),
fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.QueryLabel).Observe(float64(tr.ElapseSpan().Milliseconds()))
if err := HandleCStatus(&status, "Retrieve failed"); err != nil { if err := HandleCStatus(&status, "Retrieve failed"); err != nil {
return nil, err return nil, err
} }

View File

@ -39,7 +39,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb" "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/storage"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
@ -514,8 +513,13 @@ func (loader *segmentLoader) FromDmlCPLoadDelete(ctx context.Context, collection
if err != nil { if err != nil {
return err 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 position.ChannelName = pChannelName
stream.AsConsumer([]string{pChannelName}, fmt.Sprintf("querynode-%d-%d", Params.QueryNodeCfg.QueryNodeID, collectionID)) 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 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}) err = stream.Seek([]*internalpb.MsgPosition{position})
if err != nil { if err != nil {
return err return err

View File

@ -211,9 +211,9 @@ func (s *streaming) search(searchReqs []*searchRequest, collID UniqueID, partIDs
err2 = err err2 = err
return return
} }
metrics.QueryNodeSQSegmentLatency.WithLabelValues(metrics.SearchLabel, metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.QueryNodeID),
metrics.GrowingSegmentLabel, metrics.SearchLabel,
fmt.Sprint(Params.QueryNodeCfg.QueryNodeID)).Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.GrowingSegmentLabel).Observe(float64(tr.ElapseSpan().Milliseconds()))
segmentLock.Lock() segmentLock.Lock()
searchResults = append(searchResults, searchResult) searchResults = append(searchResults, searchResult)
searchSegmentIDs = append(searchSegmentIDs, seg.segmentID) searchSegmentIDs = append(searchSegmentIDs, seg.segmentID)

View File

@ -32,7 +32,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
queryPb "github.com/milvus-io/milvus/internal/proto/querypb" queryPb "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/funcutil"
) )
@ -162,7 +161,7 @@ func (r *addQueryChannelTask) Execute(ctx context.Context) error {
consumeSubName := funcutil.GenChannelSubName(Params.CommonCfg.QueryNodeSubName, collectionID, Params.QueryNodeCfg.QueryNodeID) consumeSubName := funcutil.GenChannelSubName(Params.CommonCfg.QueryNodeSubName, collectionID, Params.QueryNodeCfg.QueryNodeID)
sc.queryMsgStream.AsConsumer(consumeChannels, consumeSubName) 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 { if r.req.SeekPosition == nil || len(r.req.SeekPosition.MsgID) == 0 {
// as consumer // as consumer
log.Debug("QueryNode AsConsumer", zap.Strings("channels", consumeChannels), zap.String("sub name", consumeSubName)) 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 VPChannels := make(map[string]string) // map[vChannel]pChannel
for _, info := range w.req.Infos { for _, info := range w.req.Infos {
v := info.ChannelName v := info.ChannelName
p := rootcoord.ToPhysicalChannel(info.ChannelName) p := funcutil.ToPhysicalChannel(info.ChannelName)
vChannels = append(vChannels, v) vChannels = append(vChannels, v)
pChannels = append(pChannels, p) pChannels = append(pChannels, p)
VPChannels[v] = p VPChannels[v] = p
@ -511,7 +510,7 @@ func (w *watchDeltaChannelsTask) Execute(ctx context.Context) error {
vChannel2SeekPosition := make(map[string]*internalpb.MsgPosition) vChannel2SeekPosition := make(map[string]*internalpb.MsgPosition)
for _, info := range w.req.Infos { for _, info := range w.req.Infos {
v := info.ChannelName v := info.ChannelName
p := rootcoord.ToPhysicalChannel(info.ChannelName) p := funcutil.ToPhysicalChannel(info.ChannelName)
vDeltaChannels = append(vDeltaChannels, v) vDeltaChannels = append(vDeltaChannels, v)
pDeltaChannels = append(pDeltaChannels, p) pDeltaChannels = append(pDeltaChannels, p)
VPDeltaChannels[v] = p VPDeltaChannels[v] = p

View File

@ -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)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordCreateCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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() metrics.RootCoordNumOfCollections.Inc()
return succStatus(), nil 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)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDropCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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() metrics.RootCoordNumOfCollections.Dec()
return succStatus(), nil 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)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordHasCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() metrics.RootCoordHasCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasCollection", metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: succStatus(), Status: succStatus(),
Value: t.HasCollection, 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)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDescribeCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc() metrics.RootCoordDescribeCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeCollection", metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(in.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
t.Rsp.Status = succStatus() t.Rsp.Status = succStatus()
return t.Rsp, nil return t.Rsp, nil
} }
@ -1454,7 +1452,7 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections
metrics.RootCoordShowCollectionsCounter.WithLabelValues(MetricRequestsSuccess).Inc() metrics.RootCoordShowCollectionsCounter.WithLabelValues(MetricRequestsSuccess).Inc()
t.Rsp.Status = succStatus() 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 return t.Rsp, nil
} }
@ -1487,7 +1485,7 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordCreatePartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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() metrics.RootCoordNumOfPartitions.WithLabelValues(in.CollectionName).Inc()
return succStatus(), nil return succStatus(), nil
} }
@ -1521,7 +1519,7 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDropPartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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() metrics.RootCoordNumOfPartitions.WithLabelValues(in.CollectionName).Dec()
return succStatus(), nil return succStatus(), nil
} }
@ -1563,7 +1561,7 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordHasPartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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{ return &milvuspb.BoolResponse{
Status: succStatus(), Status: succStatus(),
Value: t.HasPartition, Value: t.HasPartition,
@ -1604,7 +1602,7 @@ func (c *Core) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRe
metrics.RootCoordShowPartitionsCounter.WithLabelValues(metrics.SuccessLabel).Inc() metrics.RootCoordShowPartitionsCounter.WithLabelValues(metrics.SuccessLabel).Inc()
t.Rsp.Status = succStatus() 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 return t.Rsp, nil
} }
@ -1637,7 +1635,7 @@ func (c *Core) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest)
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordCreateIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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 return succStatus(), nil
} }
@ -1684,7 +1682,7 @@ func (c *Core) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequ
} else { } else {
t.Rsp.Status = succStatus() 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 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)) zap.String("index name", in.IndexName), zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDropIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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 return succStatus(), nil
} }
@ -1755,8 +1753,7 @@ func (c *Core) DescribeSegment(ctx context.Context, in *milvuspb.DescribeSegment
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metrics.SuccessLabel).Inc() metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeSegment", metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeSegment").Observe(float64(tr.ElapseSpan().Milliseconds()))
strconv.FormatInt(in.CollectionID, 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
t.Rsp.Status = succStatus() t.Rsp.Status = succStatus()
return t.Rsp, nil return t.Rsp, nil
} }
@ -1797,7 +1794,7 @@ func (c *Core) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsReques
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordShowSegmentsCounter.WithLabelValues(metrics.SuccessLabel).Inc() 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() t.Rsp.Status = succStatus()
return t.Rsp, nil 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.String("alias", in.Alias), zap.String("collection name", in.CollectionName),
zap.Int64("msgID", in.Base.MsgID)) 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 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), log.Debug("DropAlias success", zap.String("role", typeutil.RootCoordRole),
zap.String("alias", in.Alias), zap.Int64("msgID", in.Base.MsgID)) 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 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.String("alias", in.Alias), zap.String("collection name", in.CollectionName),
zap.Int64("msgID", in.Base.MsgID)) 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 return succStatus(), nil
} }

View File

@ -35,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "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/metricsinfo"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -338,7 +339,7 @@ func createCollectionInMeta(dbName, collName string, core *Core, shardsNum int32
chanNames := make([]string, t.ShardsNum) chanNames := make([]string, t.ShardsNum)
for i := int32(0); i < t.ShardsNum; i++ { for i := int32(0); i < t.ShardsNum; i++ {
vchanNames[i] = fmt.Sprintf("%s_%dv%d", core.chanTimeTick.getDmlChannelName(), collID, 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{ collInfo := etcdpb.CollectionInfo{
@ -746,7 +747,7 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, shardsNum, createMeta.ShardsNum) assert.Equal(t, shardsNum, createMeta.ShardsNum)
vChanName := createMeta.VirtualChannelNames[0] vChanName := createMeta.VirtualChannelNames[0]
assert.Equal(t, createMeta.PhysicalChannelNames[0], ToPhysicalChannel(vChanName)) assert.Equal(t, createMeta.PhysicalChannelNames[0], funcutil.ToPhysicalChannel(vChanName))
// get TimeTickMsg // get TimeTickMsg
//msgPack, ok = <-dmlStream.Chan() //msgPack, ok = <-dmlStream.Chan()
@ -1360,7 +1361,7 @@ func TestRootCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
vChanName := collMeta.VirtualChannelNames[0] 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()) msgs := getNotTtMsg(ctx, 1, dmlStream.Chan())
assert.Equal(t, 1, len(msgs)) assert.Equal(t, 1, len(msgs))

View File

@ -21,16 +21,16 @@ import (
"fmt" "fmt"
"strconv" "strconv"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "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/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap" "go.uber.org/zap"
@ -146,10 +146,10 @@ func (t *CreateCollectionReqTask) Execute(ctx context.Context) error {
deltaChanNames := make([]string, t.Req.ShardsNum) deltaChanNames := make([]string, t.Req.ShardsNum)
for i := int32(0); i < t.Req.ShardsNum; i++ { for i := int32(0); i < t.Req.ShardsNum; i++ {
vchanNames[i] = fmt.Sprintf("%s_%dv%d", t.core.chanTimeTick.getDmlChannelName(), collID, 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() 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] { if err1 != nil || deltaChanName != deltaChanNames[i] {
return fmt.Errorf("dmlChanName %s and deltaChanName %s mis-match", chanNames[i], 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 // remove delta channels
deltaChanNames := make([]string, len(collMeta.PhysicalChannelNames)) deltaChanNames := make([]string, len(collMeta.PhysicalChannelNames))
for i, chanName := range 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 return err
} }
} }

View File

@ -28,6 +28,7 @@ import (
"github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "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/sessionutil"
"github.com/milvus-io/milvus/internal/util/timerecord" "github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/tsoutil" "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 var err error
deltaChanNames := make([]string, len(chanNames)) deltaChanNames := make([]string, len(chanNames))
for i, chanName := range 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 { if err != nil {
log.Error("failed to convert dml channel name to delta channel name", zap.String("chanName", chanName)) log.Error("failed to convert dml channel name to delta channel name", zap.String("chanName", chanName))
panic("invalid dml channel name " + chanName) panic("invalid dml channel name " + chanName)

View File

@ -127,33 +127,3 @@ func DecodeMsgPositions(str string, msgPositions *[]*msgstream.MsgPosition) erro
} }
return json.Unmarshal([]byte(str), msgPositions) 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)
}

View File

@ -97,14 +97,6 @@ func Test_GetFieldSchemaByIndexID(t *testing.T) {
assert.NotNil(t, err) 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) { func Test_EncodeMsgPositions(t *testing.T) {
mp := &msgstream.MsgPosition{ mp := &msgstream.MsgPosition{
ChannelName: "test", ChannelName: "test",
@ -140,19 +132,3 @@ func Test_DecodeMsgPositions(t *testing.T) {
err = DecodeMsgPositions("null", &mpOut) err = DecodeMsgPositions("null", &mpOut)
assert.Nil(t, err) 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)
}

View File

@ -235,3 +235,33 @@ func GetAvailablePort() int {
return listener.Addr().(*net.TCPAddr).Port 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)
}

View File

@ -290,3 +290,27 @@ func TestCheckPortAvailable(t *testing.T) {
assert.Equal(t, CheckPortAvailable(port), true) 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)
}