Add prometheus metrics for RootCoord (#15684)

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
pull/15784/head
cai.zhang 2022-03-02 21:11:57 +08:00 committed by GitHub
parent 09a27d1620
commit 9f4c59e621
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 421 additions and 231 deletions

View File

@ -42,193 +42,6 @@ const (
CacheMissLabel = "miss" CacheMissLabel = "miss"
) )
var (
// RootCoordProxyLister counts the num of registered proxy nodes
RootCoordProxyLister = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "list_of_proxy",
Help: "List of proxy nodes which have registered with etcd",
}, []string{"client_id"})
////////////////////////////////////////////////////////////////////////////
// for grpc
// RootCoordCreateCollectionCounter counts the num of calls of CreateCollection
RootCoordCreateCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "create_collection_total",
Help: "Counter of create collection",
}, []string{"client_id", "type"})
// RootCoordDropCollectionCounter counts the num of calls of DropCollection
RootCoordDropCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "drop_collection_total",
Help: "Counter of drop collection",
}, []string{"client_id", "type"})
// RootCoordHasCollectionCounter counts the num of calls of HasCollection
RootCoordHasCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "has_collection_total",
Help: "Counter of has collection",
}, []string{"client_id", "type"})
// RootCoordDescribeCollectionCounter counts the num of calls of DescribeCollection
RootCoordDescribeCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "describe_collection_total",
Help: "Counter of describe collection",
}, []string{"client_id", "type"})
// RootCoordShowCollectionsCounter counts the num of calls of ShowCollections
RootCoordShowCollectionsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "show_collections_total",
Help: "Counter of show collections",
}, []string{"client_id", "type"})
// RootCoordCreatePartitionCounter counts the num of calls of CreatePartition
RootCoordCreatePartitionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "create_partition_total",
Help: "Counter of create partition",
}, []string{"client_id", "type"})
// RootCoordDropPartitionCounter counts the num of calls of DropPartition
RootCoordDropPartitionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "drop_partition_total",
Help: "Counter of drop partition",
}, []string{"client_id", "type"})
// RootCoordHasPartitionCounter counts the num of calls of HasPartition
RootCoordHasPartitionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "has_partition_total",
Help: "Counter of has partition",
}, []string{"client_id", "type"})
// RootCoordShowPartitionsCounter counts the num of calls of ShowPartitions
RootCoordShowPartitionsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "show_partitions_total",
Help: "Counter of show partitions",
}, []string{"client_id", "type"})
// RootCoordCreateIndexCounter counts the num of calls of CreateIndex
RootCoordCreateIndexCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "create_index_total",
Help: "Counter of create index",
}, []string{"client_id", "type"})
// RootCoordDropIndexCounter counts the num of calls of DropIndex
RootCoordDropIndexCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "drop_index_total",
Help: "Counter of drop index",
}, []string{"client_id", "type"})
// RootCoordDescribeIndexCounter counts the num of calls of DescribeIndex
RootCoordDescribeIndexCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "describe_index_total",
Help: "Counter of describe index",
}, []string{"client_id", "type"})
// RootCoordDescribeSegmentCounter counts the num of calls of DescribeSegment
RootCoordDescribeSegmentCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "describe_segment_total",
Help: "Counter of describe segment",
}, []string{"client_id", "type"})
// RootCoordShowSegmentsCounter counts the num of calls of ShowSegments
RootCoordShowSegmentsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "show_segments_total",
Help: "Counter of show segments",
}, []string{"client_id", "type"})
////////////////////////////////////////////////////////////////////////////
// for time tick
// RootCoordInsertChannelTimeTick counts the time tick num of insert channel in 24H
RootCoordInsertChannelTimeTick = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "insert_channel_time_tick",
Help: "Time tick of insert Channel in 24H",
}, []string{"vchannel"})
// RootCoordDDChannelTimeTick counts the time tick num of dd channel in 24H
RootCoordDDChannelTimeTick = prometheus.NewGauge(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "dd_channel_time_tick",
Help: "Time tick of dd Channel in 24H",
})
)
//RegisterRootCoord registers RootCoord metrics
func RegisterRootCoord() {
prometheus.MustRegister(RootCoordProxyLister)
// for grpc
prometheus.MustRegister(RootCoordCreateCollectionCounter)
prometheus.MustRegister(RootCoordDropCollectionCounter)
prometheus.MustRegister(RootCoordHasCollectionCounter)
prometheus.MustRegister(RootCoordDescribeCollectionCounter)
prometheus.MustRegister(RootCoordShowCollectionsCounter)
prometheus.MustRegister(RootCoordCreatePartitionCounter)
prometheus.MustRegister(RootCoordDropPartitionCounter)
prometheus.MustRegister(RootCoordHasPartitionCounter)
prometheus.MustRegister(RootCoordShowPartitionsCounter)
prometheus.MustRegister(RootCoordCreateIndexCounter)
prometheus.MustRegister(RootCoordDropIndexCounter)
prometheus.MustRegister(RootCoordDescribeIndexCounter)
prometheus.MustRegister(RootCoordDescribeSegmentCounter)
prometheus.MustRegister(RootCoordShowSegmentsCounter)
// for time tick
prometheus.MustRegister(RootCoordInsertChannelTimeTick)
prometheus.MustRegister(RootCoordDDChannelTimeTick)
//prometheus.MustRegister(PanicCounter)
}
var ( var (
//DataCoordDataNodeList records the num of regsitered data nodes //DataCoordDataNodeList records the num of regsitered data nodes
DataCoordDataNodeList = prometheus.NewGaugeVec( DataCoordDataNodeList = prometheus.NewGaugeVec(

View File

@ -0,0 +1,310 @@
package metrics
import (
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/prometheus/client_golang/prometheus"
)
var (
// RootCoordProxyLister counts the num of registered proxy nodes
RootCoordProxyLister = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "list_of_proxy",
Help: "List of proxy nodes which have registered with etcd",
}, []string{"node_id"})
////////////////////////////////////////////////////////////////////////////
// for grpc
// RootCoordCreateCollectionCounter counts the num of calls of CreateCollection
RootCoordCreateCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "create_collection_total",
Help: "Counter of create collection",
}, []string{"status"})
// RootCoordDropCollectionCounter counts the num of calls of DropCollection
RootCoordDropCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "drop_collection_total",
Help: "Counter of drop collection",
}, []string{"status"})
// RootCoordHasCollectionCounter counts the num of calls of HasCollection
RootCoordHasCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "has_collection_total",
Help: "Counter of has collection",
}, []string{"status"})
// RootCoordDescribeCollectionCounter counts the num of calls of DescribeCollection
RootCoordDescribeCollectionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "describe_collection_total",
Help: "Counter of describe collection",
}, []string{"status"})
// RootCoordShowCollectionsCounter counts the num of calls of ShowCollections
RootCoordShowCollectionsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "show_collections_total",
Help: "Counter of show collections",
}, []string{"type"})
// RootCoordCreatePartitionCounter counts the num of calls of CreatePartition
RootCoordCreatePartitionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "create_partition_total",
Help: "Counter of create partition",
}, []string{"type"})
// RootCoordDropPartitionCounter counts the num of calls of DropPartition
RootCoordDropPartitionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "drop_partition_total",
Help: "Counter of drop partition",
}, []string{"type"})
// RootCoordHasPartitionCounter counts the num of calls of HasPartition
RootCoordHasPartitionCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "has_partition_total",
Help: "Counter of has partition",
}, []string{"type"})
// RootCoordShowPartitionsCounter counts the num of calls of ShowPartitions
RootCoordShowPartitionsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "show_partitions_total",
Help: "Counter of show partitions",
}, []string{"type"})
// RootCoordCreateIndexCounter counts the num of calls of CreateIndex
RootCoordCreateIndexCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "create_index_total",
Help: "Counter of create index",
}, []string{"type"})
// RootCoordDropIndexCounter counts the num of calls of DropIndex
RootCoordDropIndexCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "drop_index_total",
Help: "Counter of drop index",
}, []string{"type"})
// RootCoordDescribeIndexCounter counts the num of calls of DescribeIndex
RootCoordDescribeIndexCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "describe_index_total",
Help: "Counter of describe index",
}, []string{"type"})
// RootCoordDescribeSegmentCounter counts the num of calls of DescribeSegment
RootCoordDescribeSegmentCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "describe_segment_total",
Help: "Counter of describe segment",
}, []string{"type"})
// RootCoordShowSegmentsCounter counts the num of calls of ShowSegments
RootCoordShowSegmentsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "show_segments_total",
Help: "Counter of show segments",
}, []string{"type"})
////////////////////////////////////////////////////////////////////////////
// for time tick
// RootCoordInsertChannelTimeTick counts the time tick num of insert channel in 24H
RootCoordInsertChannelTimeTick = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "insert_channel_time_tick",
Help: "Time tick of insert Channel in 24H",
}, []string{"PChannel"})
// RootCoordDDLReadTypeLatency records the latency for read type of DDL operations.
RootCoordDDLReadTypeLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "ddl_read_type_latency",
Help: "The latency for read type of DDL operations",
}, []string{"function_name", "collection_id"})
// RootCoordDDLWriteTypeLatency records the latency for write type of DDL operations.
RootCoordDDLWriteTypeLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "ddl_write_type_latency",
Help: "The latency for write type of DDL operations",
}, []string{"function_name", "collection_name"})
// RootCoordSyncTimeTickLatency records the latency of sync time tick.
RootCoordSyncTimeTickLatency = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "sync_time_tick_latency",
Help: "The latency of sync time tick",
})
// RootCoordIDAllocCounter records the number of global ID allocations.
RootCoordIDAllocCounter = prometheus.NewCounter(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "id_alloc_count",
Help: "The number of global ID allocations",
})
// RootCoordLocalTimestampAllocCounter records the number of timestamp allocations in RootCoord.
RootCoordTimestampAllocCounter = prometheus.NewGauge(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "timestamp_alloc_count",
Help: "The number of timestamp allocations in RootCoord",
})
// RootCoordETCDTimestampAllocCounter records the number of timestamp allocations in ETCD.
RootCoordETCDTimestampAllocCounter = prometheus.NewGauge(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "etcd_timestamp_alloc_count",
Help: "The number of timestamp allocations in ETCD",
})
// RootCoordNumOfCollections counts the number of collections.
RootCoordNumOfCollections = prometheus.NewGauge(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "num_of_collections",
Help: "The number of collections",
})
// RootCoordNumOfPartitions counts the number of partitions per collection.
RootCoordNumOfPartitions = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "num_of_partitions",
Help: "The number of partitions per collection",
}, []string{"collection_id"})
// RootCoordNumOfSegments counts the number of segments per collections.
RootCoordNumOfSegments = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "num_of_segments",
Help: "The number of segments per collection",
}, []string{"collection_id"})
// RootCoordNumOfIndexedSegments counts the number of indexed segments per collection.
RootCoordNumOfIndexedSegments = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "num_of_indexed_segments",
Help: "The number of indexed segments per collection",
}, []string{"collection_id"})
// RootCoordNumOfDMLChannel counts the number of DML channels.
RootCoordNumOfDMLChannel = prometheus.NewGauge(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "num_of_dml_channel",
Help: "The number of DML channels",
})
// RootCoordNumOfMsgStream counts the number of message streams.
RootCoordNumOfMsgStream = prometheus.NewGauge(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "num_of_msg_stream",
Help: "The number of message streams",
})
)
//RegisterRootCoord registers RootCoord metrics
func RegisterRootCoord() {
prometheus.MustRegister(RootCoordProxyLister)
// for grpc
prometheus.MustRegister(RootCoordCreateCollectionCounter)
prometheus.MustRegister(RootCoordDropCollectionCounter)
prometheus.MustRegister(RootCoordHasCollectionCounter)
prometheus.MustRegister(RootCoordDescribeCollectionCounter)
prometheus.MustRegister(RootCoordShowCollectionsCounter)
prometheus.MustRegister(RootCoordCreatePartitionCounter)
prometheus.MustRegister(RootCoordDropPartitionCounter)
prometheus.MustRegister(RootCoordHasPartitionCounter)
prometheus.MustRegister(RootCoordShowPartitionsCounter)
prometheus.MustRegister(RootCoordCreateIndexCounter)
prometheus.MustRegister(RootCoordDropIndexCounter)
prometheus.MustRegister(RootCoordDescribeIndexCounter)
prometheus.MustRegister(RootCoordDescribeSegmentCounter)
prometheus.MustRegister(RootCoordShowSegmentsCounter)
// for time tick
prometheus.MustRegister(RootCoordInsertChannelTimeTick)
//prometheus.MustRegister(PanicCounter)
prometheus.MustRegister(RootCoordSyncTimeTickLatency)
// for DDL latency
prometheus.MustRegister(RootCoordDDLReadTypeLatency)
prometheus.MustRegister(RootCoordDDLWriteTypeLatency)
// for allocator
prometheus.MustRegister(RootCoordIDAllocCounter)
prometheus.MustRegister(RootCoordTimestampAllocCounter)
prometheus.MustRegister(RootCoordETCDTimestampAllocCounter)
// for collection
prometheus.MustRegister(RootCoordNumOfCollections)
prometheus.MustRegister(RootCoordNumOfPartitions)
prometheus.MustRegister(RootCoordNumOfSegments)
prometheus.MustRegister(RootCoordNumOfIndexedSegments)
prometheus.MustRegister(RootCoordNumOfDMLChannel)
prometheus.MustRegister(RootCoordNumOfMsgStream)
}

View File

@ -21,6 +21,8 @@ import (
"fmt" "fmt"
"sync" "sync"
"github.com/milvus-io/milvus/internal/metrics"
"go.uber.org/atomic" "go.uber.org/atomic"
"go.uber.org/zap" "go.uber.org/zap"
@ -67,6 +69,7 @@ func newDmlChannels(ctx context.Context, factory msgstream.Factory, chanNamePref
}) })
} }
log.Debug("init dml channels", zap.Int64("num", chanNum)) log.Debug("init dml channels", zap.Int64("num", chanNum))
metrics.RootCoordNumOfDMLChannel.Add(float64(chanNum))
return d return d
} }
@ -162,6 +165,7 @@ func (d *dmlChannels) addChannels(names ...string) {
dms.refcnt++ dms.refcnt++
dms.mutex.Unlock() dms.mutex.Unlock()
} }
metrics.RootCoordNumOfDMLChannel.Inc()
} }
func (d *dmlChannels) removeChannels(names ...string) { func (d *dmlChannels) removeChannels(names ...string) {
@ -182,6 +186,7 @@ func (d *dmlChannels) removeChannels(names ...string) {
} }
dms.mutex.Unlock() dms.mutex.Unlock()
} }
metrics.RootCoordNumOfDMLChannel.Dec()
} }
func genChannelName(prefix string, idx int64) string { func genChannelName(prefix string, idx int64) string {

View File

@ -139,7 +139,7 @@ func (p *proxyManager) handlePutEvent(e *clientv3.Event) error {
for _, f := range p.addSessionsFunc { for _, f := range p.addSessionsFunc {
f(session) f(session)
} }
metrics.RootCoordProxyLister.WithLabelValues(metricProxy(session.ServerID)).Set(1) metrics.RootCoordProxyLister.WithLabelValues(metricProxy(session.ServerID)).Inc()
return nil return nil
} }
@ -152,7 +152,7 @@ func (p *proxyManager) handleDeleteEvent(e *clientv3.Event) error {
for _, f := range p.delSessionsFunc { for _, f := range p.delSessionsFunc {
f(session) f(session)
} }
metrics.RootCoordProxyLister.WithLabelValues(metricProxy(session.ServerID)).Set(0) metrics.RootCoordProxyLister.WithLabelValues(metricProxy(session.ServerID)).Dec()
return nil return nil
} }

View File

@ -28,6 +28,8 @@ import (
"syscall" "syscall"
"time" "time"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/common"
@ -89,8 +91,9 @@ type Core struct {
IDAllocatorUpdate func() error IDAllocatorUpdate func() error
//tso allocator //tso allocator
TSOAllocator func(count uint32) (typeutil.Timestamp, error) TSOAllocator func(count uint32) (typeutil.Timestamp, error)
TSOAllocatorUpdate func() error TSOAllocatorUpdate func() error
TSOGetLastSavedTime func() time.Time
//inner members //inner members
ctx context.Context ctx context.Context
@ -305,6 +308,8 @@ func (c *Core) tsLoop() {
log.Warn("failed to update timestamp: ", zap.Error(err)) log.Warn("failed to update timestamp: ", zap.Error(err))
continue continue
} }
ts := c.TSOGetLastSavedTime()
metrics.RootCoordETCDTimestampAllocCounter.Set(float64(ts.Unix()))
if err := c.IDAllocatorUpdate(); err != nil { if err := c.IDAllocatorUpdate(); err != nil {
log.Warn("failed to update id: ", zap.Error(err)) log.Warn("failed to update id: ", zap.Error(err))
continue continue
@ -428,6 +433,7 @@ func (c *Core) getSegments(ctx context.Context, collID typeutil.UniqueID) (map[t
} }
} }
metrics.RootCoordNumOfSegments.WithLabelValues(strconv.FormatInt(collID, 10)).Set(float64(len(segID2PartID)))
return segID2PartID, nil return segID2PartID, nil
} }
@ -444,6 +450,7 @@ func (c *Core) setMsgStreams() error {
return fmt.Errorf("timeTickChannel is empty") return fmt.Errorf("timeTickChannel is empty")
} }
timeTickStream, _ := c.msFactory.NewMsgStream(c.ctx) timeTickStream, _ := c.msFactory.NewMsgStream(c.ctx)
metrics.RootCoordNumOfMsgStream.Inc()
timeTickStream.AsProducer([]string{Params.MsgChannelCfg.RootCoordTimeTick}) timeTickStream.AsProducer([]string{Params.MsgChannelCfg.RootCoordTimeTick})
log.Debug("RootCoord register timetick producer success", zap.String("channel name", Params.MsgChannelCfg.RootCoordTimeTick)) log.Debug("RootCoord register timetick producer success", zap.String("channel name", Params.MsgChannelCfg.RootCoordTimeTick))
@ -470,7 +477,6 @@ func (c *Core) setMsgStreams() error {
if err := timeTickStream.Broadcast(&msgPack); err != nil { if err := timeTickStream.Broadcast(&msgPack); err != nil {
return err return err
} }
metrics.RootCoordDDChannelTimeTick.Set(float64(tsoutil.Mod24H(t)))
pc := c.chanTimeTick.listDmlChannels() pc := c.chanTimeTick.listDmlChannels()
pt := make([]uint64, len(pc)) pt := make([]uint64, len(pc))
@ -1011,6 +1017,9 @@ func (c *Core) Init() error {
c.TSOAllocatorUpdate = func() error { c.TSOAllocatorUpdate = func() error {
return tsoAllocator.UpdateTSO() return tsoAllocator.UpdateTSO()
} }
c.TSOGetLastSavedTime = func() time.Time {
return tsoAllocator.GetLastSavedTime()
}
m := map[string]interface{}{ m := map[string]interface{}{
"PulsarAddress": Params.PulsarCfg.Address, "PulsarAddress": Params.PulsarCfg.Address,
@ -1269,11 +1278,13 @@ func (c *Core) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringRespon
// CreateCollection create collection // CreateCollection create collection
func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) { func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
metrics.RootCoordCreateCollectionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordCreateCollectionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("CreateCollection")
log.Debug("CreateCollection", zap.String("role", typeutil.RootCoordRole), log.Debug("CreateCollection", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
t := &CreateCollectionReqTask{ t := &CreateCollectionReqTask{
@ -1292,17 +1303,19 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti
log.Debug("CreateCollection success", zap.String("role", typeutil.RootCoordRole), log.Debug("CreateCollection success", zap.String("role", typeutil.RootCoordRole),
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(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordCreateCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateCollection", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfCollections.Inc()
return succStatus(), nil return succStatus(), nil
} }
// DropCollection drop collection // DropCollection drop collection
func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) { func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
metrics.RootCoordDropCollectionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordDropCollectionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("DropCollection")
log.Debug("DropCollection", zap.String("role", typeutil.RootCoordRole), log.Debug("DropCollection", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
t := &DropCollectionReqTask{ t := &DropCollectionReqTask{
@ -1321,19 +1334,22 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe
log.Debug("DropCollection success", zap.String("role", typeutil.RootCoordRole), log.Debug("DropCollection success", zap.String("role", typeutil.RootCoordRole),
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(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordDropCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropCollection", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfCollections.Dec()
return succStatus(), nil return succStatus(), nil
} }
// HasCollection check collection existence // HasCollection check collection existence
func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) { func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
metrics.RootCoordHasCollectionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordHasCollectionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
Value: false, Value: false,
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("HasCollection")
log.Debug("HasCollection", zap.String("role", typeutil.RootCoordRole), log.Debug("HasCollection", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
@ -1357,7 +1373,9 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ
log.Debug("HasCollection success", zap.String("role", typeutil.RootCoordRole), log.Debug("HasCollection success", zap.String("role", typeutil.RootCoordRole),
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(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordHasCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasCollection",
in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: succStatus(), Status: succStatus(),
Value: t.HasCollection, Value: t.HasCollection,
@ -1366,12 +1384,13 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ
// DescribeCollection return collection info // DescribeCollection return collection info
func (c *Core) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) { func (c *Core) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
metrics.RootCoordDescribeCollectionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordDescribeCollectionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.DescribeCollectionResponse{ return &milvuspb.DescribeCollectionResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode"+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode"+internalpb.StateCode_name[int32(code)]),
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("DescribeCollection")
log.Debug("DescribeCollection", zap.String("role", typeutil.RootCoordRole), log.Debug("DescribeCollection", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
@ -1394,19 +1413,22 @@ func (c *Core) DescribeCollection(ctx context.Context, in *milvuspb.DescribeColl
log.Debug("DescribeCollection success", zap.String("role", typeutil.RootCoordRole), log.Debug("DescribeCollection success", zap.String("role", typeutil.RootCoordRole),
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(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordDescribeCollectionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeCollection",
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
} }
// ShowCollections list all collection names // ShowCollections list all collection names
func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) { func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
metrics.RootCoordShowCollectionsCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordShowCollectionsCounter.WithLabelValues(MetricRequestsTotal).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.ShowCollectionsResponse{ return &milvuspb.ShowCollectionsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("ShowCollections")
log.Debug("ShowCollections", zap.String("role", typeutil.RootCoordRole), log.Debug("ShowCollections", zap.String("role", typeutil.RootCoordRole),
zap.String("dbname", in.DbName), zap.Int64("msgID", in.Base.MsgID)) zap.String("dbname", in.DbName), zap.Int64("msgID", in.Base.MsgID))
@ -1430,18 +1452,19 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections
zap.String("dbname", in.DbName), zap.Int("num of collections", len(t.Rsp.CollectionNames)), zap.String("dbname", in.DbName), zap.Int("num of collections", len(t.Rsp.CollectionNames)),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordShowCollectionsCounter.WithLabelValues(metricProxy(in.Base.SourceID), 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()))
return t.Rsp, nil return t.Rsp, nil
} }
// CreatePartition create partition // CreatePartition create partition
func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) { func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
metrics.RootCoordCreatePartitionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordCreatePartitionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("CreatePartition")
log.Debug("CreatePartition", zap.String("role", typeutil.RootCoordRole), log.Debug("CreatePartition", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName), zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
@ -1463,17 +1486,19 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName), zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordCreatePartitionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordCreatePartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreatePartition", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfPartitions.WithLabelValues(in.CollectionName).Inc()
return succStatus(), nil return succStatus(), nil
} }
// DropPartition drop partition // DropPartition drop partition
func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) { func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
metrics.RootCoordDropPartitionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordDropPartitionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("DropPartition")
log.Debug("DropPartition", zap.String("role", typeutil.RootCoordRole), log.Debug("DropPartition", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName), zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
@ -1495,19 +1520,22 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName), zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDropPartitionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordDropPartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropPartition", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfPartitions.WithLabelValues(in.CollectionName).Dec()
return succStatus(), nil return succStatus(), nil
} }
// HasPartition check partition existence // HasPartition check partition existence
func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) { func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
metrics.RootCoordHasPartitionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordHasPartitionCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
Value: false, Value: false,
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("HasPartition")
log.Debug("HasPartition", zap.String("role", typeutil.RootCoordRole), log.Debug("HasPartition", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName), zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName),
@ -1534,7 +1562,8 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName), zap.String("collection name", in.CollectionName), zap.String("partition name", in.PartitionName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordHasPartitionCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordHasPartitionCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("HasPartition", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return &milvuspb.BoolResponse{ return &milvuspb.BoolResponse{
Status: succStatus(), Status: succStatus(),
Value: t.HasPartition, Value: t.HasPartition,
@ -1543,13 +1572,14 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
// ShowPartitions list all partition names // ShowPartitions list all partition names
func (c *Core) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) { func (c *Core) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
metrics.RootCoordShowPartitionsCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordShowPartitionsCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.ShowPartitionsResponse{ return &milvuspb.ShowPartitionsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("ShowPartitions")
log.Debug("ShowPartitions", zap.String("role", typeutil.RootCoordRole), log.Debug("ShowPartitions", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID)) zap.String("collection name", in.CollectionName), zap.Int64("msgID", in.Base.MsgID))
t := &ShowPartitionReqTask{ t := &ShowPartitionReqTask{
@ -1572,18 +1602,19 @@ func (c *Core) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRe
zap.String("collection name", in.CollectionName), zap.Int("num of partitions", len(t.Rsp.PartitionNames)), zap.String("collection name", in.CollectionName), zap.Int("num of partitions", len(t.Rsp.PartitionNames)),
zap.Int64("msgID", t.Req.Base.MsgID)) zap.Int64("msgID", t.Req.Base.MsgID))
metrics.RootCoordShowPartitionsCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).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()))
return t.Rsp, nil return t.Rsp, nil
} }
// CreateIndex create index // CreateIndex create index
func (c *Core) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest) (*commonpb.Status, error) { func (c *Core) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
metrics.RootCoordCreateIndexCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordCreateIndexCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("CreateIndex")
log.Debug("CreateIndex", zap.String("role", typeutil.RootCoordRole), log.Debug("CreateIndex", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName), zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
@ -1605,19 +1636,20 @@ func (c *Core) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest)
zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName), zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordCreateIndexCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordCreateIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("CreateIndex", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return succStatus(), nil return succStatus(), nil
} }
// DescribeIndex return index info // DescribeIndex return index info
func (c *Core) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) { func (c *Core) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
metrics.RootCoordDescribeIndexCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordDescribeIndexCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.DescribeIndexResponse{ return &milvuspb.DescribeIndexResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("DescribeIndex")
log.Debug("DescribeIndex", zap.String("role", typeutil.RootCoordRole), log.Debug("DescribeIndex", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName), zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
@ -1646,22 +1678,23 @@ func (c *Core) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequ
zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName), zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName),
zap.Strings("index names", idxNames), zap.Int64("msgID", in.Base.MsgID)) zap.Strings("index names", idxNames), zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDescribeIndexCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordDescribeIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc()
if len(t.Rsp.IndexDescriptions) == 0 { if len(t.Rsp.IndexDescriptions) == 0 {
t.Rsp.Status = failStatus(commonpb.ErrorCode_IndexNotExist, "index not exist") t.Rsp.Status = failStatus(commonpb.ErrorCode_IndexNotExist, "index not exist")
} else { } else {
t.Rsp.Status = succStatus() t.Rsp.Status = succStatus()
} }
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DescribeIndex", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return t.Rsp, nil return t.Rsp, nil
} }
// DropIndex drop index // DropIndex drop index
func (c *Core) DropIndex(ctx context.Context, in *milvuspb.DropIndexRequest) (*commonpb.Status, error) { func (c *Core) DropIndex(ctx context.Context, in *milvuspb.DropIndexRequest) (*commonpb.Status, error) {
metrics.RootCoordDropIndexCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordDropIndexCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("DropIndex")
log.Debug("DropIndex", zap.String("role", typeutil.RootCoordRole), log.Debug("DropIndex", zap.String("role", typeutil.RootCoordRole),
zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName), zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName),
zap.String("index name", in.IndexName), zap.Int64("msgID", in.Base.MsgID)) zap.String("index name", in.IndexName), zap.Int64("msgID", in.Base.MsgID))
@ -1683,19 +1716,20 @@ func (c *Core) DropIndex(ctx context.Context, in *milvuspb.DropIndexRequest) (*c
zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName), zap.String("collection name", in.CollectionName), zap.String("field name", in.FieldName),
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(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordDropIndexCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLWriteTypeLatency.WithLabelValues("DropIndex", in.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
return succStatus(), nil return succStatus(), nil
} }
// DescribeSegment return segment info // DescribeSegment return segment info
func (c *Core) DescribeSegment(ctx context.Context, in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) { func (c *Core) DescribeSegment(ctx context.Context, in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) {
metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.DescribeSegmentResponse{ return &milvuspb.DescribeSegmentResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("DescribeSegment")
log.Debug("DescribeSegment", zap.String("role", typeutil.RootCoordRole), log.Debug("DescribeSegment", zap.String("role", typeutil.RootCoordRole),
zap.Int64("collection id", in.CollectionID), zap.Int64("segment id", in.SegmentID), zap.Int64("collection id", in.CollectionID), zap.Int64("segment id", in.SegmentID),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
@ -1720,19 +1754,22 @@ func (c *Core) DescribeSegment(ctx context.Context, in *milvuspb.DescribeSegment
zap.Int64("collection id", in.CollectionID), zap.Int64("segment id", in.SegmentID), zap.Int64("collection id", in.CollectionID), zap.Int64("segment id", in.SegmentID),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordDescribeSegmentCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("DescribeSegment",
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
} }
// ShowSegments list all segments // ShowSegments list all segments
func (c *Core) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) { func (c *Core) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) {
metrics.RootCoordShowSegmentsCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsTotal).Inc() metrics.RootCoordShowSegmentsCounter.WithLabelValues(metrics.TotalLabel).Inc()
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return &milvuspb.ShowSegmentsResponse{ return &milvuspb.ShowSegmentsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]),
}, nil }, nil
} }
tr := timerecord.NewTimeRecorder("ShowSegments")
log.Debug("ShowSegments", zap.String("role", typeutil.RootCoordRole), log.Debug("ShowSegments", zap.String("role", typeutil.RootCoordRole),
zap.Int64("collection id", in.CollectionID), zap.Int64("partition id", in.PartitionID), zap.Int64("collection id", in.CollectionID), zap.Int64("partition id", in.PartitionID),
@ -1759,7 +1796,8 @@ func (c *Core) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsReques
zap.Int64s("segments ids", t.Rsp.SegmentIDs), zap.Int64s("segments ids", t.Rsp.SegmentIDs),
zap.Int64("msgID", in.Base.MsgID)) zap.Int64("msgID", in.Base.MsgID))
metrics.RootCoordShowSegmentsCounter.WithLabelValues(metricProxy(in.Base.SourceID), MetricRequestsSuccess).Inc() metrics.RootCoordShowSegmentsCounter.WithLabelValues(metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReadTypeLatency.WithLabelValues("ShowSegments", 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
} }
@ -1782,6 +1820,7 @@ func (c *Core) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimestam
//return first available time stamp //return first available time stamp
ts = ts - uint64(in.Count) + 1 ts = ts - uint64(in.Count) + 1
metrics.RootCoordTimestampAllocCounter.Set(float64(ts))
return &rootcoordpb.AllocTimestampResponse{ return &rootcoordpb.AllocTimestampResponse{
Status: succStatus(), Status: succStatus(),
Timestamp: ts, Timestamp: ts,
@ -1805,6 +1844,7 @@ func (c *Core) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*ro
Count: in.Count, Count: in.Count,
}, nil }, nil
} }
metrics.RootCoordIDAllocCounter.Add(float64(in.Count))
return &rootcoordpb.AllocIDResponse{ return &rootcoordpb.AllocIDResponse{
Status: succStatus(), Status: succStatus(),
ID: start, ID: start,
@ -1971,7 +2011,7 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("CreateAlias")
log.Debug("CreateAlias", zap.String("role", typeutil.RootCoordRole), log.Debug("CreateAlias", zap.String("role", typeutil.RootCoordRole),
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))
@ -1993,6 +2033,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()))
return succStatus(), nil return succStatus(), nil
} }
@ -2001,7 +2042,7 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("DropAlias")
log.Debug("DropAlias", zap.String("role", typeutil.RootCoordRole), log.Debug("DropAlias", 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))
t := &DropAliasReqTask{ t := &DropAliasReqTask{
@ -2020,6 +2061,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()))
return succStatus(), nil return succStatus(), nil
} }
@ -2028,7 +2070,7 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
if code, ok := c.checkHealthy(); !ok { if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
} }
tr := timerecord.NewTimeRecorder("AlterAlias")
log.Debug("AlterAlias", zap.String("role", typeutil.RootCoordRole), log.Debug("AlterAlias", zap.String("role", typeutil.RootCoordRole),
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))
@ -2050,5 +2092,6 @@ 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()))
return succStatus(), nil return succStatus(), nil
} }

View File

@ -21,6 +21,8 @@ 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"
@ -876,9 +878,16 @@ func (t *CreateIndexReqTask) Execute(ctx context.Context) error {
return fmt.Errorf("field name = %s, data type = %s", t.Req.FieldName, schemapb.DataType_name[int32(field.DataType)]) return fmt.Errorf("field name = %s, data type = %s", t.Req.FieldName, schemapb.DataType_name[int32(field.DataType)])
} }
collectionID := collMeta.ID
cnt := 0
defer func() {
metrics.RootCoordNumOfIndexedSegments.WithLabelValues(strconv.FormatInt(collectionID, 10)).Add(float64(cnt))
}()
for _, segID := range segIDs { for _, segID := range segIDs {
info := etcdpb.SegmentIndexInfo{ info := etcdpb.SegmentIndexInfo{
CollectionID: collMeta.ID, CollectionID: collectionID,
PartitionID: segID2PartID[segID], PartitionID: segID2PartID[segID],
SegmentID: segID, SegmentID: segID,
FieldID: field.FieldID, FieldID: field.FieldID,
@ -895,6 +904,7 @@ func (t *CreateIndexReqTask) Execute(ctx context.Context) error {
if err := t.core.MetaTable.AddIndex(&info); err != nil { if err := t.core.MetaTable.AddIndex(&info); err != nil {
log.Debug("Add index into meta table failed", zap.Int64("collection_id", collMeta.ID), zap.Int64("index_id", info.IndexID), zap.Int64("build_id", info.BuildID), zap.Error(err)) log.Debug("Add index into meta table failed", zap.Int64("collection_id", collMeta.ID), zap.Int64("index_id", info.IndexID), zap.Int64("build_id", info.BuildID), zap.Error(err))
} }
cnt++
} }
return nil return nil

View File

@ -332,6 +332,7 @@ func (t *timetickSync) startWatch(wg *sync.WaitGroup) {
} }
wg.Wait() wg.Wait()
span := tr.ElapseSpan() span := tr.ElapseSpan()
metrics.RootCoordSyncTimeTickLatency.Observe(float64(span.Milliseconds()))
// rootcoord send tt msg to all channels every 200ms by default // rootcoord send tt msg to all channels every 200ms by default
if span > Params.ProxyCfg.TimeTickInterval { if span > Params.ProxyCfg.TimeTickInterval {
log.Warn("rootcoord send tt to all channels too slowly", log.Warn("rootcoord send tt to all channels too slowly",

View File

@ -58,6 +58,8 @@ type Allocator interface {
GenerateTSO(count uint32) (uint64, error) GenerateTSO(count uint32) (uint64, error)
// Reset is used to reset the TSO allocator. // Reset is used to reset the TSO allocator.
Reset() Reset()
GetLastSavedTime() time.Time
} }
// GlobalTSOAllocator is the global single point TSO allocator. // GlobalTSOAllocator is the global single point TSO allocator.
@ -156,3 +158,9 @@ func (gta *GlobalTSOAllocator) AllocOne() (typeutil.Timestamp, error) {
func (gta *GlobalTSOAllocator) Reset() { func (gta *GlobalTSOAllocator) Reset() {
gta.tso.ResetTimestamp() gta.tso.ResetTimestamp()
} }
// GetLastSavedTime get the last saved time for tso.
func (gta *GlobalTSOAllocator) GetLastSavedTime() time.Time {
ts := gta.tso.lastSavedTime.Load()
return ts.(time.Time)
}