Add more metrics (#21975)

Signed-off-by: xige-16 <xi.ge@zilliz.com>
pull/22094/head
xige-16 2023-02-09 16:06:33 +08:00 committed by GitHub
parent a1b10a80e4
commit 50739e6c6a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
86 changed files with 1227 additions and 432 deletions

View File

@ -59,7 +59,9 @@ func init() {
Registry = prometheus.NewRegistry()
Registry.MustRegister(prometheus.NewProcessCollector(prometheus.ProcessCollectorOpts{}))
Registry.MustRegister(prometheus.NewGoCollector())
metrics.RegisterEtcdMetrics(Registry)
metrics.RegisterMetaMetrics(Registry)
metrics.RegisterStorageMetrics(Registry)
metrics.RegisterMsgStreamMetrics(Registry)
}
func stopRocksmq() {

View File

@ -25,7 +25,9 @@ import (
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"go.uber.org/zap/zapcore"
)
@ -156,6 +158,7 @@ func (c *ChannelStore) Reload() error {
Schema: cw.GetSchema(),
}
c.channelsInfo[nodeID].Channels = append(c.channelsInfo[nodeID].Channels, channel)
metrics.DataCoordDmlChannelNum.WithLabelValues(strconv.FormatInt(nodeID, 10)).Set(float64(len(c.channelsInfo[nodeID].Channels)))
}
return nil
}
@ -264,6 +267,7 @@ func (c *ChannelStore) update(opSet ChannelOpSet) error {
default:
return errUnknownOpType
}
metrics.DataCoordDmlChannelNum.WithLabelValues(strconv.FormatInt(op.NodeID, 10)).Set(float64(len(c.channelsInfo[op.NodeID].Channels)))
}
return nil
}

View File

@ -25,8 +25,10 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"go.uber.org/zap"
)
@ -249,6 +251,8 @@ func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResu
nodeID := c.plans[planID].dataNodeID
c.releaseQueue(nodeID)
metrics.DataCoordCompactedSegmentSize.WithLabelValues().Observe(float64(getCompactedSegmentSize(result)))
return nil
}

View File

@ -103,6 +103,24 @@ func (m *meta) reloadFromKV() error {
metrics.DataCoordNumSegments.WithLabelValues(segment.State.String()).Inc()
if segment.State == commonpb.SegmentState_Flushed {
numStoredRows += segment.NumOfRows
insertFileNum := 0
for _, fieldBinlog := range segment.GetBinlogs() {
insertFileNum += len(fieldBinlog.GetBinlogs())
}
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.InsertFileLabel).Observe(float64(insertFileNum))
statFileNum := 0
for _, fieldBinlog := range segment.GetStatslogs() {
statFileNum += len(fieldBinlog.GetBinlogs())
}
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.StatFileLabel).Observe(float64(statFileNum))
deleteFileNum := 0
for _, filedBinlog := range segment.GetDeltalogs() {
deleteFileNum += len(filedBinlog.GetBinlogs())
}
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.DeleteFileLabel).Observe(float64(deleteFileNum))
}
}
metrics.DataCoordNumStoredRows.WithLabelValues().Set(float64(numStoredRows))

View File

@ -22,6 +22,7 @@ import (
"fmt"
"math/rand"
"os"
"strconv"
"sync"
"sync/atomic"
"syscall"
@ -222,11 +223,14 @@ func (s *Server) Register() error {
if s.enableActiveStandBy {
s.session.ProcessActiveStandBy(s.activateFunc)
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(s.session.ServerID, 10), typeutil.DataCoordRole).Inc()
log.Info("DataCoord Register Finished")
go s.session.LivenessCheck(s.serverLoopCtx, func() {
logutil.Logger(s.ctx).Error("disconnected from etcd and exited", zap.Int64("serverID", s.session.ServerID))
if err := s.Stop(); err != nil {
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(s.session.ServerID, 10), typeutil.DataCoordRole).Dec()
// manually send signal to starter goroutine
if s.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
@ -798,6 +802,25 @@ func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
log.Error("flush segment complete failed", zap.Error(err))
return err
}
insertFileNum := 0
for _, fieldBinlog := range segment.GetBinlogs() {
insertFileNum += len(fieldBinlog.GetBinlogs())
}
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.InsertFileLabel).Observe(float64(insertFileNum))
statFileNum := 0
for _, fieldBinlog := range segment.GetStatslogs() {
statFileNum += len(fieldBinlog.GetBinlogs())
}
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.StatFileLabel).Observe(float64(statFileNum))
deleteFileNum := 0
for _, filedBinlog := range segment.GetDeltalogs() {
deleteFileNum += len(filedBinlog.GetBinlogs())
}
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.DeleteFileLabel).Observe(float64(deleteFileNum))
log.Info("flush segment complete", zap.Int64("id", segmentID))
return nil
}

View File

@ -27,10 +27,12 @@ import (
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap"
)
@ -206,3 +208,29 @@ func getCollectionTTL(properties map[string]string) (time.Duration, error) {
return Params.CommonCfg.EntityExpirationTTL, nil
}
func getCompactedSegmentSize(s *datapb.CompactionResult) int64 {
var segmentSize int64
if s != nil {
for _, binlogs := range s.GetInsertLogs() {
for _, l := range binlogs.GetBinlogs() {
segmentSize += l.GetLogSize()
}
}
for _, deltaLogs := range s.GetDeltalogs() {
for _, l := range deltaLogs.GetBinlogs() {
segmentSize += l.GetLogSize()
}
}
for _, statsLogs := range s.GetDeltalogs() {
for _, l := range statsLogs.GetBinlogs() {
segmentSize += l.GetLogSize()
}
}
}
return segmentSize
}

View File

@ -693,7 +693,7 @@ func (c *ChannelMeta) setCurInsertBuffer(segmentID UniqueID, buf *BufferData) {
seg, ok := c.segments[segmentID]
if ok {
seg.curInsertBuf = buf
seg.setInsertBuffer(buf)
return
}
log.Warn("cannot find segment when setCurInsertBuffer", zap.Int64("segmentID", segmentID))

View File

@ -422,6 +422,8 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
return nil, errContext
}
durInQueue := t.tr.Record("compact task start to process")
ctxTimeout, cancelAll := context.WithTimeout(t.ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
defer cancelAll()
@ -682,6 +684,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
log.Info("overall elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(time.Since(compactStart))))
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Observe(float64(durInQueue.Milliseconds()))
return pack, nil
}

View File

@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
@ -569,6 +570,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
emptyTask := &compactionTask{
ctx: ctx,
cancel: cancel,
tr: timerecord.NewTimeRecorder("test"),
}
plan := &datapb.CompactionPlan{

View File

@ -71,12 +71,6 @@ const (
// RPCConnectionTimeout is used to set the timeout for rpc request
RPCConnectionTimeout = 30 * time.Second
// MetricRequestsTotal is used to count the num of total requests
MetricRequestsTotal = "total"
// MetricRequestsSuccess is used to count the num of successful requests
MetricRequestsSuccess = "success"
// ConnectEtcdMaxRetryTime is used to limit the max retry time for connection etcd
ConnectEtcdMaxRetryTime = 100
@ -188,13 +182,15 @@ func (node *DataNode) SetDataCoord(ds types.DataCoord) error {
// Register register datanode to etcd
func (node *DataNode) Register() error {
node.session.Register()
metrics.NumNodes.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), typeutil.DataNodeRole).Inc()
log.Info("DataNode Register Finished")
// Start liveness check
go node.session.LivenessCheck(node.ctx, func() {
log.Error("Data Node disconnected from etcd, process will exit", zap.Int64("Server Id", node.session.ServerID))
if err := node.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), typeutil.DataNodeRole).Dec()
// manually send signal to starter goroutine
if node.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
@ -584,7 +580,7 @@ func (node *DataNode) ReadyToFlush() error {
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
metrics.DataNodeFlushReqCounter.WithLabelValues(
fmt.Sprint(Params.DataNodeCfg.GetNodeID()),
MetricRequestsTotal).Inc()
metrics.TotalLabel).Inc()
errStatus := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -670,7 +666,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
metrics.DataNodeFlushReqCounter.WithLabelValues(
fmt.Sprint(Params.DataNodeCfg.GetNodeID()),
MetricRequestsSuccess).Inc()
metrics.SuccessLabel).Inc()
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil

View File

@ -494,7 +494,7 @@ func (ibNode *insertBufferNode) Sync(fgMsg *flowGraphMsg, seg2Upload []UniqueID,
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.TotalLabel).Inc()
if task.auto {
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.TotalLabel).Inc()
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.FailLabel).Inc()
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID()), metrics.SuccessLabel).Inc()
}
}
return segmentsToSync

View File

@ -17,6 +17,7 @@
package datanode
import (
"strconv"
"sync"
"sync/atomic"
@ -24,6 +25,7 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
@ -114,11 +116,35 @@ func (s *Segment) isPKExist(pk primaryKey) bool {
return false
}
// setInsertBuffer set curInsertBuf.
func (s *Segment) setInsertBuffer(buf *BufferData) {
s.curInsertBuf = buf
if buf != nil && buf.buffer != nil {
dataSize := 0
for _, data := range buf.buffer.Data {
dataSize += data.GetMemorySize()
}
metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(strconv.FormatInt(Params.QueryNodeCfg.GetNodeID(), 10),
strconv.FormatInt(s.collectionID, 10)).Add(float64(dataSize))
}
}
// rollInsertBuffer moves curInsertBuf to historyInsertBuf, and then sets curInsertBuf to nil.
func (s *Segment) rollInsertBuffer() {
if s.curInsertBuf == nil {
return
}
if s.curInsertBuf.buffer != nil {
dataSize := 0
for _, data := range s.curInsertBuf.buffer.Data {
dataSize += data.GetMemorySize()
}
metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(strconv.FormatInt(Params.QueryNodeCfg.GetNodeID(), 10),
strconv.FormatInt(s.collectionID, 10)).Sub(float64(dataSize))
}
s.curInsertBuf.buffer = nil // free buffer memory, only keep meta infos in historyInsertBuf
s.historyInsertBuf = append(s.historyInsertBuf, s.curInsertBuf)
s.curInsertBuf = nil

View File

@ -30,16 +30,11 @@ import (
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/errorutil"
"golang.org/x/sync/errgroup"
"go.etcd.io/etcd/api/v3/mvccpb"
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
@ -56,6 +51,8 @@ import (
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/errorutil"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
@ -142,11 +139,14 @@ func (i *IndexCoord) Register() error {
if i.enableActiveStandBy {
i.session.ProcessActiveStandBy(i.activateFunc)
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(i.session.ServerID, 10), typeutil.IndexCoordRole).Inc()
log.Info("IndexCoord Register Finished")
go i.session.LivenessCheck(i.loopCtx, func() {
log.Error("Index Coord disconnected from etcd, process will exit", zap.Int64("Server Id", i.session.ServerID))
if err := i.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(i.session.ServerID, 10), typeutil.IndexCoordRole).Dec()
// manually send signal to starter goroutine
if i.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {

View File

@ -112,6 +112,7 @@ func (mt *metaTable) reloadFromKV() error {
}
for _, segIdx := range segmentIndxes {
mt.updateSegmentIndex(segIdx)
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.IndexFileLabel).Observe(float64(len(segIdx.IndexFileKeys)))
}
log.Info("IndexCoord metaTable reloadFromKV success")
@ -1038,6 +1039,7 @@ func (mt *metaTable) FinishTask(taskInfo *indexpb.IndexTaskInfo) error {
}
mt.updateIndexTasksMetrics()
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.IndexFileLabel).Observe(float64(len(taskInfo.IndexFileKeys)))
log.Info("finish index task success", zap.Int64("buildID", taskInfo.BuildID),
zap.String("state", taskInfo.GetState().String()), zap.String("fail reason", taskInfo.GetFailReason()))
return nil

View File

@ -43,6 +43,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
@ -53,6 +54,7 @@ import (
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
)
@ -125,6 +127,8 @@ func NewIndexNode(ctx context.Context, factory dependency.Factory) (*IndexNode,
// Register register index node at etcd.
func (i *IndexNode) Register() error {
i.session.Register()
metrics.NumNodes.WithLabelValues(strconv.FormatInt(i.session.ServerID, 10), typeutil.IndexNodeRole).Inc()
log.Info("IndexNode Register Finished")
//start liveness check
go i.session.LivenessCheck(i.loopCtx, func() {
@ -132,6 +136,7 @@ func (i *IndexNode) Register() error {
if err := i.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(i.session.ServerID, 10), typeutil.IndexNodeRole).Dec()
// manually send signal to starter goroutine
if i.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {

View File

@ -66,6 +66,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
cancel: taskCancel,
state: commonpb.IndexState_InProgress}); oldInfo != nil {
log.Ctx(ctx).Warn("duplicated index build task", zap.String("ClusterID", req.ClusterID), zap.Int64("BuildID", req.BuildID))
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10), metrics.FailLabel).Inc()
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BuildIndexError,
Reason: "duplicated index build task",
@ -76,6 +77,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
log.Ctx(ctx).Error("create chunk manager failed", zap.String("Bucket", req.StorageConfig.BucketName),
zap.String("AccessKey", req.StorageConfig.AccessKeyID),
zap.String("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10), metrics.FailLabel).Inc()
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BuildIndexError,
Reason: "create chunk manager failed",
@ -105,6 +107,8 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10), metrics.FailLabel).Inc()
return ret, nil
}
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10), metrics.SuccessLabel).Inc()
log.Ctx(ctx).Info("IndexNode successfully scheduled", zap.Int64("IndexBuildID", req.BuildID), zap.String("ClusterID", req.ClusterID), zap.String("indexName", req.IndexName))
return ret, nil
}

View File

@ -98,6 +98,7 @@ type indexBuildTask struct {
newIndexParams map[string]string
serializedSize uint64
tr *timerecord.TimeRecorder
queueDur time.Duration
statistic indexpb.JobInfo
node *IndexNode
}
@ -138,6 +139,8 @@ func (it *indexBuildTask) GetState() commonpb.IndexState {
// OnEnqueue enqueues indexing tasks.
func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
it.queueDur = 0
it.tr.Record("enqueue done")
it.statistic.StartTime = time.Now().UnixMicro()
it.statistic.PodID = it.node.GetNodeID()
log.Ctx(ctx).Debug("IndexNode IndexBuilderTask Enqueue")
@ -145,6 +148,9 @@ func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
}
func (it *indexBuildTask) Prepare(ctx context.Context) error {
// cal time duration in task queue
it.queueDur = it.tr.Record("start to process")
logutil.Logger(ctx).Info("Begin to prepare indexBuildTask", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID), zap.Int64("SegmentIf", it.segmentID))
typeParams := make(map[string]string)
indexParams := make(map[string]string)

View File

@ -21,12 +21,14 @@ import (
"context"
"errors"
"runtime/debug"
"strconv"
"sync"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
)
// TaskQueue is a queue used to store tasks.
@ -228,6 +230,10 @@ func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
}
}
t.SetState(commonpb.IndexState_Finished, "")
if indexBuildTask, ok := t.(*indexBuildTask); ok {
metrics.IndexNodeBuildIndexLatency.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10)).Observe(float64(indexBuildTask.tr.ElapseSpan().Milliseconds()))
metrics.IndexNodeIndexTaskLatencyInQueue.WithLabelValues(strconv.FormatInt(Params.IndexNodeCfg.GetNodeID(), 10)).Observe(float64(indexBuildTask.queueDur.Milliseconds()))
}
}
func (sched *TaskScheduler) indexBuildLoop() {

View File

@ -23,15 +23,13 @@ import (
"path"
"time"
"github.com/milvus-io/milvus/internal/metrics"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/common"
clientv3 "go.etcd.io/etcd/client/v3"
"github.com/milvus-io/milvus/internal/log"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
const (
@ -67,8 +65,6 @@ func (kv *EtcdKV) GetPath(key string) string {
func (kv *EtcdKV) WalkWithPrefix(prefix string, paginationSize int, fn func([]byte, []byte) error) error {
start := time.Now()
prefix = path.Join(kv.rootPath, prefix)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
batch := int64(paginationSize)
opts := []clientv3.OpOption{
@ -79,7 +75,7 @@ func (kv *EtcdKV) WalkWithPrefix(prefix string, paginationSize int, fn func([]by
key := prefix
for {
resp, err := kv.client.Get(ctx, key, opts...)
resp, err := kv.getEtcdMeta(context.TODO(), key, opts...)
if err != nil {
return err
}
@ -105,9 +101,7 @@ func (kv *EtcdKV) WalkWithPrefix(prefix string, paginationSize int, fn func([]by
func (kv *EtcdKV) LoadWithPrefix(key string) ([]string, []string, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, err
@ -126,9 +120,7 @@ func (kv *EtcdKV) LoadWithPrefix(key string) ([]string, []string, error) {
func (kv *EtcdKV) LoadBytesWithPrefix(key string) ([]string, [][]byte, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, err
@ -147,9 +139,7 @@ func (kv *EtcdKV) LoadBytesWithPrefix(key string) ([]string, [][]byte, error) {
func (kv *EtcdKV) LoadWithPrefix2(key string) ([]string, []string, []int64, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, nil, err
@ -169,9 +159,7 @@ func (kv *EtcdKV) LoadWithPrefix2(key string) ([]string, []string, []int64, erro
func (kv *EtcdKV) LoadWithRevisionAndVersions(key string) ([]string, []string, []int64, int64, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, nil, 0, err
@ -192,9 +180,7 @@ func (kv *EtcdKV) LoadWithRevisionAndVersions(key string) ([]string, []string, [
func (kv *EtcdKV) LoadBytesWithPrefix2(key string) ([]string, [][]byte, []int64, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, nil, err
@ -215,9 +201,7 @@ func (kv *EtcdKV) LoadBytesWithPrefix2(key string) ([]string, [][]byte, []int64,
func (kv *EtcdKV) Load(key string) (string, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key)
resp, err := kv.getEtcdMeta(context.TODO(), key)
if err != nil {
return "", err
}
@ -232,9 +216,7 @@ func (kv *EtcdKV) Load(key string) (string, error) {
func (kv *EtcdKV) LoadBytes(key string) ([]byte, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key)
resp, err := kv.getEtcdMeta(context.TODO(), key)
if err != nil {
return []byte{}, err
}
@ -255,7 +237,7 @@ func (kv *EtcdKV) MultiLoad(keys []string) ([]string, error) {
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
return []string{}, err
}
@ -290,7 +272,7 @@ func (kv *EtcdKV) MultiLoadBytes(keys []string) ([][]byte, error) {
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
return [][]byte{}, err
}
@ -319,9 +301,7 @@ func (kv *EtcdKV) MultiLoadBytes(keys []string) ([][]byte, error) {
func (kv *EtcdKV) LoadWithRevision(key string) ([]string, []string, int64, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByCreateRevision, clientv3.SortAscend))
if err != nil {
return nil, nil, 0, err
@ -340,9 +320,7 @@ func (kv *EtcdKV) LoadWithRevision(key string) ([]string, []string, int64, error
func (kv *EtcdKV) LoadBytesWithRevision(key string) ([]string, [][]byte, int64, error) {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
resp, err := kv.getEtcdMeta(context.TODO(), key, clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, 0, err
@ -361,10 +339,8 @@ func (kv *EtcdKV) LoadBytesWithRevision(key string) ([]string, [][]byte, int64,
func (kv *EtcdKV) Save(key, value string) error {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
CheckValueSizeAndWarn(key, value)
_, err := kv.client.Put(ctx, key, value)
_, err := kv.putEtcdMeta(context.TODO(), key, value)
CheckElapseAndWarn(start, "Slow etcd operation save", zap.String("key", key))
return err
}
@ -373,10 +349,8 @@ func (kv *EtcdKV) Save(key, value string) error {
func (kv *EtcdKV) SaveBytes(key string, value []byte) error {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
CheckValueSizeAndWarn(key, value)
_, err := kv.client.Put(ctx, key, string(value))
_, err := kv.putEtcdMeta(context.TODO(), key, string(value))
CheckElapseAndWarn(start, "Slow etcd operation save", zap.String("key", key))
return err
}
@ -386,10 +360,8 @@ func (kv *EtcdKV) SaveWithLease(key, value string, id clientv3.LeaseID) error {
log.Debug("Etcd saving with lease", zap.String("etcd_key", key))
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
CheckValueSizeAndWarn(key, value)
_, err := kv.client.Put(ctx, key, value, clientv3.WithLease(id))
_, err := kv.putEtcdMeta(context.TODO(), key, value, clientv3.WithLease(id))
CheckElapseAndWarn(start, "Slow etcd operation save with lease", zap.String("key", key))
return err
}
@ -399,10 +371,8 @@ func (kv *EtcdKV) SaveWithIgnoreLease(key, value string) error {
log.Debug("Etcd saving with ignore lease", zap.String("etcd_key", key))
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
CheckValueSizeAndWarn(key, value)
_, err := kv.client.Put(ctx, key, value, clientv3.WithIgnoreLease())
_, err := kv.putEtcdMeta(context.TODO(), key, value, clientv3.WithIgnoreLease())
CheckElapseAndWarn(start, "Slow etcd operation save with lease", zap.String("key", key))
return err
}
@ -411,10 +381,8 @@ func (kv *EtcdKV) SaveWithIgnoreLease(key, value string) error {
func (kv *EtcdKV) SaveBytesWithLease(key string, value []byte, id clientv3.LeaseID) error {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
CheckValueSizeAndWarn(key, value)
_, err := kv.client.Put(ctx, key, string(value), clientv3.WithLease(id))
_, err := kv.putEtcdMeta(context.TODO(), key, string(value), clientv3.WithLease(id))
CheckElapseAndWarn(start, "Slow etcd operation save with lease", zap.String("key", key))
return err
}
@ -433,7 +401,7 @@ func (kv *EtcdKV) MultiSave(kvs map[string]string) error {
defer cancel()
CheckTxnStringValueSizeAndWarn(kvs)
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiSave error", zap.Any("kvs", kvs), zap.Int("len", len(kvs)), zap.Error(err))
}
@ -455,7 +423,7 @@ func (kv *EtcdKV) MultiSaveBytes(kvs map[string][]byte) error {
defer cancel()
CheckTxnBytesValueSizeAndWarn(kvs)
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiSaveBytes err", zap.Any("kvs", kvs), zap.Int("len", len(kvs)), zap.Error(err))
}
@ -467,10 +435,7 @@ func (kv *EtcdKV) MultiSaveBytes(kvs map[string][]byte) error {
func (kv *EtcdKV) RemoveWithPrefix(prefix string) error {
start := time.Now()
key := path.Join(kv.rootPath, prefix)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Delete(ctx, key, clientv3.WithPrefix())
_, err := kv.removeEtcdMeta(context.TODO(), key, clientv3.WithPrefix())
CheckElapseAndWarn(start, "Slow etcd operation remove with prefix", zap.String("prefix", prefix))
return err
}
@ -479,10 +444,7 @@ func (kv *EtcdKV) RemoveWithPrefix(prefix string) error {
func (kv *EtcdKV) Remove(key string) error {
start := time.Now()
key = path.Join(kv.rootPath, key)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Delete(ctx, key)
_, err := kv.removeEtcdMeta(context.TODO(), key)
CheckElapseAndWarn(start, "Slow etcd operation remove", zap.String("key", key))
return err
}
@ -498,7 +460,7 @@ func (kv *EtcdKV) MultiRemove(keys []string) error {
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiRemove error", zap.Strings("keys", keys), zap.Int("len", len(keys)), zap.Error(err))
}
@ -523,7 +485,7 @@ func (kv *EtcdKV) MultiSaveAndRemove(saves map[string]string, removals []string)
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiSaveAndRemove error",
zap.Any("saves", saves),
@ -553,7 +515,7 @@ func (kv *EtcdKV) MultiSaveBytesAndRemove(saves map[string][]byte, removals []st
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiSaveBytesAndRemove error",
zap.Any("saves", saves),
@ -605,7 +567,7 @@ func (kv *EtcdKV) MultiRemoveWithPrefix(keys []string) error {
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiRemoveWithPrefix error", zap.Strings("keys", keys), zap.Int("len", len(keys)), zap.Error(err))
}
@ -630,7 +592,7 @@ func (kv *EtcdKV) MultiSaveAndRemoveWithPrefix(saves map[string]string, removals
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiSaveAndRemoveWithPrefix error",
zap.Any("saves", saves),
@ -660,7 +622,7 @@ func (kv *EtcdKV) MultiSaveBytesAndRemoveWithPrefix(saves map[string][]byte, rem
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
if err != nil {
log.Warn("Etcd MultiSaveBytesAndRemoveWithPrefix error",
zap.Any("saves", saves),
@ -702,12 +664,9 @@ func (kv *EtcdKV) CompareValueAndSwap(key, value, target string, opts ...clientv
start := time.Now()
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If(
clientv3.Compare(
clientv3.Value(path.Join(kv.rootPath, key)),
"=",
value)).
Then(clientv3.OpPut(path.Join(kv.rootPath, key), target, opts...)).Commit()
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx,
clientv3.Compare(clientv3.Value(path.Join(kv.rootPath, key)), "=", value)),
clientv3.OpPut(path.Join(kv.rootPath, key), target, opts...))
if err != nil {
return false, err
}
@ -721,12 +680,9 @@ func (kv *EtcdKV) CompareValueAndSwapBytes(key string, value, target []byte, opt
start := time.Now()
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If(
clientv3.Compare(
clientv3.Value(path.Join(kv.rootPath, key)),
"=",
string(value))).
Then(clientv3.OpPut(path.Join(kv.rootPath, key), string(target), opts...)).Commit()
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx,
clientv3.Compare(clientv3.Value(path.Join(kv.rootPath, key)), "=", string(value))),
clientv3.OpPut(path.Join(kv.rootPath, key), string(target), opts...))
if err != nil {
return false, err
}
@ -740,12 +696,9 @@ func (kv *EtcdKV) CompareVersionAndSwap(key string, source int64, target string,
start := time.Now()
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If(
clientv3.Compare(
clientv3.Version(path.Join(kv.rootPath, key)),
"=",
source)).
Then(clientv3.OpPut(path.Join(kv.rootPath, key), target, opts...)).Commit()
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx,
clientv3.Compare(clientv3.Version(path.Join(kv.rootPath, key)), "=", source)),
clientv3.OpPut(path.Join(kv.rootPath, key), target, opts...))
if err != nil {
return false, err
}
@ -759,12 +712,9 @@ func (kv *EtcdKV) CompareVersionAndSwapBytes(key string, source int64, target []
start := time.Now()
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If(
clientv3.Compare(
clientv3.Version(path.Join(kv.rootPath, key)),
"=",
source)).
Then(clientv3.OpPut(path.Join(kv.rootPath, key), string(target), opts...)).Commit()
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx,
clientv3.Compare(clientv3.Version(path.Join(kv.rootPath, key)), "=", source)),
clientv3.OpPut(path.Join(kv.rootPath, key), string(target), opts...))
if err != nil {
return false, err
}
@ -775,7 +725,6 @@ func (kv *EtcdKV) CompareVersionAndSwapBytes(key string, source int64, target []
// CheckElapseAndWarn checks the elapsed time and warns if it is too long.
func CheckElapseAndWarn(start time.Time, message string, fields ...zap.Field) bool {
elapsed := time.Since(start)
metrics.EtcdRequestLatency.Observe(float64(elapsed))
if elapsed.Milliseconds() > 2000 {
log.Warn(message, append([]zap.Field{zap.String("time spent", elapsed.String())}, fields...)...)
return true
@ -785,7 +734,6 @@ func CheckElapseAndWarn(start time.Time, message string, fields ...zap.Field) bo
func CheckValueSizeAndWarn(key string, value interface{}) bool {
size := binary.Size(value)
metrics.EtcdPutKvSize.Observe(float64(size))
if size > 102400 {
log.Warn("value size large than 100kb", zap.String("key", key), zap.Int("value_size(kb)", size/1024))
return true
@ -811,3 +759,106 @@ func CheckTxnStringValueSizeAndWarn(kvs map[string]string) bool {
return CheckTxnBytesValueSizeAndWarn(newKvs)
}
func (kv *EtcdKV) getEtcdMeta(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.GetResponse, error) {
ctx1, cancel := context.WithTimeout(ctx, RequestTimeout)
defer cancel()
start := timerecord.NewTimeRecorder("getEtcdMeta")
resp, err := kv.client.Get(ctx1, key, opts...)
elapsed := start.Elapse("getEtcdMeta done")
metrics.MetaOpCounter.WithLabelValues(metrics.MetaGetLabel, metrics.TotalLabel).Inc()
// cal meta kv size
if err == nil && resp != nil {
totalSize := 0
for _, v := range resp.Kvs {
totalSize += binary.Size(v)
}
metrics.MetaKvSize.WithLabelValues(metrics.MetaGetLabel).Observe(float64(totalSize))
metrics.MetaRequestLatency.WithLabelValues(metrics.MetaGetLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MetaOpCounter.WithLabelValues(metrics.MetaGetLabel, metrics.SuccessLabel).Inc()
} else {
metrics.MetaOpCounter.WithLabelValues(metrics.MetaGetLabel, metrics.FailLabel).Inc()
}
return resp, err
}
func (kv *EtcdKV) putEtcdMeta(ctx context.Context, key, val string, opts ...clientv3.OpOption) (*clientv3.PutResponse, error) {
ctx1, cancel := context.WithTimeout(ctx, RequestTimeout)
defer cancel()
start := timerecord.NewTimeRecorder("putEtcdMeta")
resp, err := kv.client.Put(ctx1, key, val, opts...)
elapsed := start.Elapse("putEtcdMeta done")
metrics.MetaOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.TotalLabel).Inc()
if err == nil {
metrics.MetaKvSize.WithLabelValues(metrics.MetaPutLabel).Observe(float64(len(val)))
metrics.MetaRequestLatency.WithLabelValues(metrics.MetaPutLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MetaOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.SuccessLabel).Inc()
} else {
metrics.MetaOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.FailLabel).Inc()
}
return resp, err
}
func (kv *EtcdKV) removeEtcdMeta(ctx context.Context, key string, opts ...clientv3.OpOption) (*clientv3.DeleteResponse, error) {
ctx1, cancel := context.WithTimeout(ctx, RequestTimeout)
defer cancel()
start := timerecord.NewTimeRecorder("removeEtcdMeta")
resp, err := kv.client.Delete(ctx1, key, opts...)
elapsed := start.Elapse("removeEtcdMeta done")
metrics.MetaOpCounter.WithLabelValues(metrics.MetaRemoveLabel, metrics.TotalLabel).Inc()
if err == nil {
metrics.MetaRequestLatency.WithLabelValues(metrics.MetaRemoveLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MetaOpCounter.WithLabelValues(metrics.MetaRemoveLabel, metrics.SuccessLabel).Inc()
} else {
metrics.MetaOpCounter.WithLabelValues(metrics.MetaRemoveLabel, metrics.FailLabel).Inc()
}
return resp, err
}
func (kv *EtcdKV) getTxnWithCmp(ctx context.Context, cmp ...clientv3.Cmp) clientv3.Txn {
return kv.client.Txn(ctx).If(cmp...)
}
func (kv *EtcdKV) executeTxn(txn clientv3.Txn, ops ...clientv3.Op) (*clientv3.TxnResponse, error) {
start := timerecord.NewTimeRecorder("executeTxn")
resp, err := txn.Then(ops...).Commit()
elapsed := start.Elapse("executeTxn done")
metrics.MetaOpCounter.WithLabelValues(metrics.MetaTxnLabel, metrics.TotalLabel).Inc()
if err == nil && resp.Succeeded {
// cal put meta kv size
totalPutSize := 0
for _, op := range ops {
if op.IsPut() {
totalPutSize += binary.Size(op.ValueBytes())
}
}
metrics.MetaKvSize.WithLabelValues(metrics.MetaPutLabel).Observe(float64(totalPutSize))
// cal get meta kv size
totalGetSize := 0
for _, rp := range resp.Responses {
if rp.GetResponseRange() != nil {
for _, v := range rp.GetResponseRange().Kvs {
totalGetSize += binary.Size(v)
}
}
}
metrics.MetaKvSize.WithLabelValues(metrics.MetaGetLabel).Observe(float64(totalGetSize))
metrics.MetaRequestLatency.WithLabelValues(metrics.MetaTxnLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MetaOpCounter.WithLabelValues(metrics.MetaTxnLabel, metrics.SuccessLabel).Inc()
} else {
metrics.MetaOpCounter.WithLabelValues(metrics.MetaTxnLabel, metrics.FailLabel).Inc()
}
return resp, err
}

View File

@ -95,6 +95,25 @@ var (
Help: "binlog size of segments",
}, []string{segmentStateLabelName})
DataCoordDmlChannelNum = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataCoordRole,
Name: "watched_dml_chanel_num",
Help: "the num of dml channel watched by datanode",
}, []string{
nodeIDLabelName,
})
DataCoordCompactedSegmentSize = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataCoordRole,
Name: "compacted_segment_size",
Help: "the segment size of compacted segment",
Buckets: buckets,
}, []string{})
/* hard to implement, commented now
DataCoordSegmentSizeRatio = prometheus.NewHistogramVec(
prometheus.HistogramOpts{

View File

@ -168,6 +168,17 @@ var (
nodeIDLabelName,
})
DataNodeCompactionLatencyInQueue = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataNodeRole,
Name: "compaction_latency_in_queue",
Help: "latency of compaction operation in queue",
Buckets: buckets,
}, []string{
nodeIDLabelName,
})
// DataNodeFlushReqCounter counts the num of calls of FlushSegments
DataNodeFlushReqCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
@ -197,6 +208,17 @@ var (
Help: "forward delete message time taken",
Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName})
DataNodeFlowGraphBufferDataSize = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataNodeRole,
Name: "fg_buffer_size",
Help: "the buffered data size of flow graph",
}, []string{
nodeIDLabelName,
collectionIDLabelName,
})
)
// RegisterDataNode registers DataNode metrics

View File

@ -1,31 +0,0 @@
package metrics
import (
"github.com/prometheus/client_golang/prometheus"
)
var (
EtcdPutKvSize = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "etcd",
Name: "etcd_kv_size",
Help: "kv size stats",
Buckets: buckets,
})
EtcdRequestLatency = prometheus.NewHistogram(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "etcd",
Name: "client_request_latency",
Help: "request latency on the client side ",
Buckets: buckets,
})
)
//RegisterEtcdMetrics registers etcd metrics
func RegisterEtcdMetrics(registry *prometheus.Registry) {
registry.MustRegister(EtcdPutKvSize)
registry.MustRegister(EtcdRequestLatency)
}

View File

@ -52,7 +52,7 @@ var (
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.IndexNodeRole,
Name: "build_index_latency",
Name: "knowhere_build_index_latency",
Help: "latency of building the index by knowhere",
Buckets: buckets,
}, []string{nodeIDLabelName})
@ -74,6 +74,24 @@ var (
Help: "latency of saving the index file",
Buckets: buckets,
}, []string{nodeIDLabelName})
IndexNodeIndexTaskLatencyInQueue = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.IndexNodeRole,
Name: "index_task_latency_in_queue",
Help: "latency of index task in queue",
Buckets: buckets,
}, []string{nodeIDLabelName})
IndexNodeBuildIndexLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.IndexNodeRole,
Name: "build_index_latency",
Help: "latency of build index for segment",
Buckets: buckets,
}, []string{nodeIDLabelName})
)
//RegisterIndexNode registers IndexNode metrics

View File

@ -0,0 +1,65 @@
// 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/prometheus/client_golang/prometheus"
)
const (
MetaGetLabel = "get"
MetaPutLabel = "put"
MetaRemoveLabel = "remove"
MetaTxnLabel = "txn"
metaOpType = "meta_op_type"
)
var (
MetaKvSize = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "meta",
Name: "kv_size",
Help: "kv size stats",
Buckets: buckets,
}, []string{metaOpType})
MetaRequestLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "meta",
Name: "request_latency",
Help: "request latency on the client side ",
Buckets: buckets,
}, []string{metaOpType})
MetaOpCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: "meta",
Name: "op_count",
Help: "count of meta operation",
}, []string{metaOpType, statusLabelName})
)
//RegisterMetaMetrics registers etcd metrics
func RegisterMetaMetrics(registry *prometheus.Registry) {
registry.MustRegister(MetaKvSize)
registry.MustRegister(MetaRequestLatency)
registry.MustRegister(MetaOpCounter)
}

View File

@ -42,6 +42,11 @@ const (
TimetickLabel = "timetick"
AllLabel = "all"
InsertFileLabel = "insert_file"
DeleteFileLabel = "delete_file"
StatFileLabel = "stat_file"
IndexFileLabel = "index_file"
UnissuedIndexTaskLabel = "unissued"
InProgressIndexTaskLabel = "in-progress"
FinishedIndexTaskLabel = "finished"
@ -69,6 +74,7 @@ const (
queryTypeLabelName = "query_type"
collectionName = "collection_name"
segmentStateLabelName = "segment_state"
segmentFileTypeLabelName = "segment_file_type"
usernameLabelName = "username"
roleNameLabelName = "role_name"
cacheNameLabelName = "cache_name"
@ -81,6 +87,21 @@ var (
// buckets involves durations in milliseconds,
// [1 2 4 8 16 32 64 128 256 512 1024 2048 4096 8192 16384 32768 65536 1.31072e+05]
buckets = prometheus.ExponentialBuckets(1, 2, 18)
NumNodes = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Name: "num_node",
Help: "number of nodes and coordinates",
}, []string{nodeIDLabelName, roleNameLabelName})
FlushedSegmentFileNum = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Name: "flushed_segment_file_num",
Help: "the num of files for flushed segment",
Buckets: buckets,
}, []string{segmentFileTypeLabelName})
)
// Register serves prometheus http service
@ -93,4 +114,5 @@ func Register(r *prometheus.Registry) {
Path: "/metrics_default",
Handler: promhttp.Handler(),
})
r.MustRegister(NumNodes)
}

View File

@ -33,6 +33,8 @@ func TestRegisterMetrics(t *testing.T) {
RegisterProxy(r)
RegisterQueryNode(r)
RegisterQueryCoord(r)
RegisterEtcdMetrics(r)
RegisterMetaMetrics(r)
RegisterStorageMetrics(r)
RegisterMsgStreamMetrics(r)
Register(r)
}

View File

@ -0,0 +1,54 @@
// 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/prometheus/client_golang/prometheus"
const (
SendMsgLabel = "produce"
ReceiveMsgLabel = "consume" // not used
CreateProducerLabel = "create_producer"
CreateConsumerLabel = "create_consumer"
msgStreamOpType = "message_op_type"
)
var (
MsgStreamRequestLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "msgstream",
Name: "request_latency",
Help: "request latency on the client side ",
Buckets: buckets,
}, []string{msgStreamOpType})
MsgStreamOpCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: "msgstream",
Name: "op_count",
Help: "count of stream message operation",
}, []string{msgStreamOpType, statusLabelName})
)
//RegisterMsgStreamMetrics registers storage metrics
func RegisterMsgStreamMetrics(registry *prometheus.Registry) {
registry.MustRegister(MsgStreamRequestLatency)
registry.MustRegister(MsgStreamOpCounter)
}

View File

@ -0,0 +1,64 @@
// 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/prometheus/client_golang/prometheus"
const (
DataGetLabel = "get"
DataPutLabel = "put"
DataRemoveLabel = "remove"
DataListLabel = "list"
DataStatLabel = "stat"
persistentDataOpType = "persistent_data_op_type"
)
var (
PersistentDataKvSize = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "storage",
Name: "kv_size",
Help: "kv size stats",
Buckets: buckets,
}, []string{persistentDataOpType})
PersistentDataRequestLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: "storage",
Name: "request_latency",
Help: "request latency on the client side ",
Buckets: buckets,
}, []string{persistentDataOpType})
PersistentDataOpCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: "storage",
Name: "op_count",
Help: "count of persistent data operation",
}, []string{persistentDataOpType, statusLabelName})
)
//RegisterStorageMetrics registers storage metrics
func RegisterStorageMetrics(registry *prometheus.Registry) {
registry.MustRegister(PersistentDataKvSize)
registry.MustRegister(PersistentDataRequestLatency)
registry.MustRegister(PersistentDataOpCounter)
}

View File

@ -131,7 +131,7 @@ var (
Buckets: buckets, // unit: ms
}, []string{nodeIDLabelName, msgTypeLabelName})
// ProxyCacheHitCounter record the number of Proxy cache hits or miss.
// ProxyCacheStatsCounter record the number of Proxy cache hits or miss.
ProxyCacheStatsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,

View File

@ -22,6 +22,13 @@ import (
"github.com/milvus-io/milvus/internal/util/typeutil"
)
const (
LoadSegmentLabel = "load_segment"
WatchDmlChannelLabel = "watch_dml_channel"
QueryCoordTaskType = "querycoord_task_type"
)
var (
QueryCoordNumCollections = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
@ -83,7 +90,7 @@ var (
Subsystem: typeutil.QueryCoordRole,
Name: "task_num",
Help: "the number of tasks in QueryCoord's scheduler",
}, []string{})
}, []string{QueryCoordTaskType})
QueryCoordNumQueryNodes = prometheus.NewGaugeVec(
prometheus.GaugeOpts{

View File

@ -162,6 +162,19 @@ var (
segmentStateLabelName,
})
QueryNodeSegmentSearchLatencyPerVector = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.QueryNodeRole,
Name: "segment_latency_per_vector",
Help: "one vector's search latency per segment",
Buckets: buckets,
}, []string{
nodeIDLabelName,
queryTypeLabelName,
segmentStateLabelName,
})
QueryNodeSQSegmentLatencyInCore = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
@ -197,6 +210,17 @@ var (
nodeIDLabelName,
})
QueryNodeWatchDmlChannelLatency = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.QueryNodeRole,
Name: "watch_dml_channel_latency",
Help: "latency of watch dml channel",
Buckets: buckets,
}, []string{
nodeIDLabelName,
})
QueryNodeReadTaskUnsolveLen = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,

View File

@ -44,6 +44,14 @@ var (
Help: "latency of each DDL operations",
}, []string{functionLabelName})
RootCoordDDLReqLatencyInQueue = prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.RootCoordRole,
Name: "ddl_req_latency_in_queue",
Help: "latency of each DDL operations in queue",
}, []string{functionLabelName})
// RootCoordSyncTimeTickLatency records the latency of sync time tick.
RootCoordSyncTimeTickLatency = prometheus.NewHistogram(
prometheus.HistogramOpts{

View File

@ -1844,6 +1844,10 @@ func (t *MarshalFailTsMsg) Unmarshal(_ MarshalType) (TsMsg, error) {
return nil, errors.New("mocked error")
}
func (t *MarshalFailTsMsg) Size() int {
return 0
}
var _ mqwrapper.Producer = (*mockSendFailProducer)(nil)
type mockSendFailProducer struct {

View File

@ -1,3 +1,19 @@
// 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 kafka
import (
@ -5,12 +21,14 @@ import (
"strconv"
"sync"
"github.com/milvus-io/milvus/internal/util/paramtable"
"go.uber.org/zap"
"github.com/confluentinc/confluent-kafka-go/kafka"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
var Producer *kafka.Producer
@ -143,22 +161,38 @@ func (kc *kafkaClient) newConsumerConfig(group string, offset mqwrapper.Subscrip
}
func (kc *kafkaClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
start := timerecord.NewTimeRecorder("create producer")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.TotalLabel).Inc()
pp, err := kc.getKafkaProducer()
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
return nil, err
}
elapsed := start.Elapse("create producer done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
deliveryChan := make(chan kafka.Event, 128)
producer := &kafkaProducer{p: pp, deliveryChan: deliveryChan, topic: options.Topic}
return producer, nil
}
func (kc *kafkaClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
start := timerecord.NewTimeRecorder("create consumer")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc()
config := kc.newConsumerConfig(options.SubscriptionName, options.SubscriptionInitialPosition)
consumer, err := newKafkaConsumer(config, options.Topic, options.SubscriptionName, options.SubscriptionInitialPosition)
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
return nil, err
}
elapsed := start.Elapse("create consumer done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
return consumer, nil
}

View File

@ -1,3 +1,19 @@
// 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 kafka
import (
@ -11,7 +27,9 @@ import (
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
type kafkaProducer struct {
@ -26,26 +44,36 @@ func (kp *kafkaProducer) Topic() string {
}
func (kp *kafkaProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
start := timerecord.NewTimeRecorder("send msg to stream")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.TotalLabel).Inc()
err := kp.p.Produce(&kafka.Message{
TopicPartition: kafka.TopicPartition{Topic: &kp.topic, Partition: mqwrapper.DefaultPartitionIdx},
Value: message.Payload,
}, kp.deliveryChan)
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return nil, err
}
e, ok := <-kp.deliveryChan
if !ok {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
log.Error("kafka produce message fail because of delivery chan is closed", zap.String("topic", kp.topic))
return nil, common.NewIgnorableError(fmt.Errorf("delivery chan of kafka producer is closed"))
}
m := e.(*kafka.Message)
if m.TopicPartition.Error != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return nil, m.TopicPartition.Error
}
elapsed := start.Elapse("send msg to stream done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.SendMsgLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.SuccessLabel).Inc()
return &kafkaID{messageID: int64(m.TopicPartition.Offset)}, nil
}

View File

@ -24,12 +24,15 @@ import (
"time"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/util/retry"
pulsarctl "github.com/streamnative/pulsarctl/pkg/pulsar"
"github.com/streamnative/pulsarctl/pkg/pulsar/common"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
type pulsarClient struct {
@ -62,8 +65,12 @@ func NewClient(tenant string, namespace string, opts pulsar.ClientOptions) (*pul
// CreateProducer create a pulsar producer from options
func (pc *pulsarClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
start := timerecord.NewTimeRecorder("create producer")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.TotalLabel).Inc()
fullTopicName, err := GetFullTopicName(pc.tenant, pc.namespace, options.Topic)
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
return nil, err
}
opts := pulsar.ProducerOptions{Topic: fullTopicName}
@ -78,20 +85,30 @@ func (pc *pulsarClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwra
pp, err := pc.client.CreateProducer(opts)
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
return nil, err
}
if pp == nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
return nil, errors.New("pulsar is not ready, producer is nil")
}
elapsed := start.Elapse("create producer done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
producer := &pulsarProducer{p: pp}
return producer, nil
}
// Subscribe creates a pulsar consumer instance and subscribe a topic
func (pc *pulsarClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
start := timerecord.NewTimeRecorder("create consumer")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc()
receiveChannel := make(chan pulsar.ConsumerMessage, options.BufSize)
fullTopicName, err := GetFullTopicName(pc.tenant, pc.namespace, options.Topic)
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
return nil, err
}
consumer, err := pc.client.Subscribe(pulsar.ConsumerOptions{
@ -102,6 +119,7 @@ func (pc *pulsarClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.
MessageChannel: receiveChannel,
})
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
if strings.Contains(err.Error(), "ConsumerBusy") {
return nil, retry.Unrecoverable(err)
}
@ -114,6 +132,9 @@ func (pc *pulsarClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.
pConsumer.AtLatest = true
}
elapsed := start.Elapse("create consumer done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
return pConsumer, nil
}

View File

@ -19,9 +19,11 @@ package pulsar
import (
"context"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
// implementation assertion
@ -37,9 +39,20 @@ func (pp *pulsarProducer) Topic() string {
}
func (pp *pulsarProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
start := timerecord.NewTimeRecorder("send msg to stream")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.TotalLabel).Inc()
ppm := &pulsar.ProducerMessage{Payload: message.Payload, Properties: message.Properties}
pmID, err := pp.p.Send(ctx, ppm)
return &pulsarID{messageID: pmID}, err
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return &pulsarID{messageID: pmID}, err
}
elapsed := start.Elapse("send msg to stream")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.SendMsgLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.SuccessLabel).Inc()
return &pulsarID{messageID: pmID}, nil
}
func (pp *pulsarProducer) Close() {

View File

@ -19,13 +19,14 @@ package rmq
import (
"strconv"
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/client"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/client"
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
// rmqClient contains a rocksmq client
@ -50,17 +51,28 @@ func NewClient(opts client.Options) (*rmqClient, error) {
// CreateProducer creates a producer for rocksmq client
func (rc *rmqClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
start := timerecord.NewTimeRecorder("create producer")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.TotalLabel).Inc()
rmqOpts := client.ProducerOptions{Topic: options.Topic}
pp, err := rc.client.CreateProducer(rmqOpts)
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
return nil, err
}
rp := rmqProducer{p: pp}
elapsed := start.Elapse("create producer done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
return &rp, nil
}
// Subscribe subscribes a consumer in rmq client
func (rc *rmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
start := timerecord.NewTimeRecorder("create consumer")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc()
receiveChannel := make(chan client.Message, options.BufSize)
cli, err := rc.client.Subscribe(client.ConsumerOptions{
@ -70,11 +82,15 @@ func (rc *rmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Con
SubscriptionInitialPosition: options.SubscriptionInitialPosition,
})
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
return nil, err
}
rConsumer := &Consumer{c: cli, closeCh: make(chan struct{})}
elapsed := start.Elapse("create consumer done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
return rConsumer, nil
}

View File

@ -14,9 +14,10 @@ package rmq
import (
"context"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/client"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
var _ mqwrapper.Producer = (*rmqProducer)(nil)
@ -33,9 +34,20 @@ func (rp *rmqProducer) Topic() string {
// Send send the producer messages to rocksmq
func (rp *rmqProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
start := timerecord.NewTimeRecorder("send msg to stream")
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.TotalLabel).Inc()
pm := &client.ProducerMessage{Payload: message.Payload}
id, err := rp.p.Send(pm)
return &rmqID{messageID: id}, err
if err != nil {
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
return &rmqID{messageID: id}, err
}
elapsed := start.Elapse("send msg to stream done")
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.SendMsgLabel).Observe(float64(elapsed.Milliseconds()))
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.SuccessLabel).Inc()
return &rmqID{messageID: id}, nil
}
// Close does nothing currently

View File

@ -53,6 +53,7 @@ type TsMsg interface {
Unmarshal(MarshalType) (TsMsg, error)
Position() *MsgPosition
SetPosition(*MsgPosition)
Size() int
}
// BaseMsg is a basic structure that contains begin timestamp, end timestamp and the position of msgstream
@ -285,6 +286,10 @@ func (it *InsertMsg) IndexMsg(index int) *InsertMsg {
}
}
func (it *InsertMsg) Size() int {
return proto.Size(&it.InsertRequest)
}
/////////////////////////////////////////Delete//////////////////////////////////////////
// DeleteMsg is a message pack that contains delete request
@ -380,6 +385,10 @@ func (dt *DeleteMsg) CheckAligned() error {
return nil
}
func (dt *DeleteMsg) Size() int {
return proto.Size(&dt.DeleteRequest)
}
/////////////////////////////////////////TimeTick//////////////////////////////////////////
// TimeTickMsg is a message pack that contains time tick only
@ -435,6 +444,10 @@ func (tst *TimeTickMsg) Unmarshal(input MarshalType) (TsMsg, error) {
return timeTick, nil
}
func (tst *TimeTickMsg) Size() int {
return proto.Size(&tst.TimeTickMsg)
}
/////////////////////////////////////////CreateCollection//////////////////////////////////////////
// CreateCollectionMsg is a message pack that contains create collection request
@ -490,6 +503,10 @@ func (cc *CreateCollectionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
return createCollectionMsg, nil
}
func (cc *CreateCollectionMsg) Size() int {
return proto.Size(&cc.CreateCollectionRequest)
}
/////////////////////////////////////////DropCollection//////////////////////////////////////////
// DropCollectionMsg is a message pack that contains drop collection request
@ -545,6 +562,10 @@ func (dc *DropCollectionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
return dropCollectionMsg, nil
}
func (dc *DropCollectionMsg) Size() int {
return proto.Size(&dc.DropCollectionRequest)
}
/////////////////////////////////////////CreatePartition//////////////////////////////////////////
// CreatePartitionMsg is a message pack that contains create partition request
@ -600,6 +621,10 @@ func (cp *CreatePartitionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
return createPartitionMsg, nil
}
func (cp *CreatePartitionMsg) Size() int {
return proto.Size(&cp.CreatePartitionRequest)
}
/////////////////////////////////////////DropPartition//////////////////////////////////////////
// DropPartitionMsg is a message pack that contains drop partition request
@ -655,6 +680,10 @@ func (dp *DropPartitionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
return dropPartitionMsg, nil
}
func (dp *DropPartitionMsg) Size() int {
return proto.Size(&dp.DropPartitionRequest)
}
/////////////////////////////////////////DataNodeTtMsg//////////////////////////////////////////
// DataNodeTtMsg is a message pack that contains datanode time tick
@ -706,3 +735,7 @@ func (m *DataNodeTtMsg) Unmarshal(input MarshalType) (TsMsg, error) {
DataNodeTtMsg: msg,
}, nil
}
func (m *DataNodeTtMsg) Size() int {
return proto.Size(&m.DataNodeTtMsg)
}

View File

@ -130,11 +130,14 @@ func NewProxy(ctx context.Context, factory dependency.Factory) (*Proxy, error) {
// Register registers proxy at etcd
func (node *Proxy) Register() error {
node.session.Register()
metrics.NumNodes.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), typeutil.ProxyRole).Inc()
log.Info("Proxy Register Finished")
go node.session.LivenessCheck(node.ctx, func() {
log.Error("Proxy disconnected from etcd, process will exit", zap.Int64("Server Id", node.session.ServerID))
if err := node.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), typeutil.ProxyRole).Dec()
if node.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
p.Signal(syscall.SIGINT)

View File

@ -22,6 +22,7 @@ import (
"fmt"
"os"
"sort"
"strconv"
"sync"
"sync/atomic"
"syscall"
@ -129,11 +130,14 @@ func (s *Server) Register() error {
if s.enableActiveStandBy {
s.session.ProcessActiveStandBy(s.activateFunc)
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(s.session.ServerID, 10), typeutil.QueryCoordRole).Inc()
log.Info("QueryCoord Register Finished")
go s.session.LivenessCheck(s.ctx, func() {
log.Error("QueryCoord disconnected from etcd, process will exit", zap.Int64("serverID", s.session.ServerID))
if err := s.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(s.session.ServerID, 10), typeutil.QueryCoordRole).Dec()
// manually send signal to starter goroutine
if s.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {

View File

@ -251,13 +251,14 @@ func (scheduler *taskScheduler) Add(task Task) error {
case *SegmentTask:
index := NewReplicaSegmentIndex(task)
scheduler.segmentTasks[index] = task
metrics.QueryCoordTaskNum.WithLabelValues(metrics.LoadSegmentLabel).Set(float64(len(scheduler.segmentTasks)))
case *ChannelTask:
index := replicaChannelIndex{task.ReplicaID(), task.Channel()}
scheduler.channelTasks[index] = task
metrics.QueryCoordTaskNum.WithLabelValues(metrics.WatchDmlChannelLabel).Set(float64(len(scheduler.channelTasks)))
}
metrics.QueryCoordTaskNum.WithLabelValues().Set(float64(scheduler.tasks.Len()))
log.Info("task added", zap.String("task", task.String()))
return nil
}
@ -676,15 +677,16 @@ func (scheduler *taskScheduler) remove(task Task) {
if task.Err() != nil {
log.Warn("task scheduler recordSegmentTaskError", zap.Error(task.err))
scheduler.recordSegmentTaskError(task)
metrics.QueryCoordTaskNum.WithLabelValues(metrics.LoadSegmentLabel).Set(float64(len(scheduler.segmentTasks)))
}
case *ChannelTask:
index := replicaChannelIndex{task.ReplicaID(), task.Channel()}
delete(scheduler.channelTasks, index)
log = log.With(zap.String("channel", task.Channel()))
metrics.QueryCoordTaskNum.WithLabelValues(metrics.WatchDmlChannelLabel).Set(float64(len(scheduler.channelTasks)))
}
metrics.QueryCoordTaskNum.WithLabelValues().Set(float64(scheduler.tasks.Len()))
log.Info("task removed")
}

View File

@ -30,6 +30,7 @@ import "C"
import (
"context"
"fmt"
"github.com/milvus-io/milvus/internal/metrics"
"os"
"path"
"runtime"
@ -166,12 +167,16 @@ func (node *QueryNode) initSession() error {
// Register register query node at etcd
func (node *QueryNode) Register() error {
node.session.Register()
metrics.NumNodes.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), typeutil.QueryNodeRole).Inc()
log.Info("QueryNode Register Finished")
// start liveness check
go node.session.LivenessCheck(node.queryNodeLoopCtx, func() {
log.Error("Query Node disconnected from etcd, process will exit", zap.Int64("Server Id", node.session.ServerID))
if err := node.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), typeutil.QueryNodeRole).Dec()
// manually send signal to starter goroutine
if node.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {

View File

@ -19,9 +19,13 @@ package querynode
import (
"context"
"errors"
"fmt"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/segcorepb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
// retrieveOnSegments performs retrieve on listed segments
@ -29,6 +33,11 @@ import (
func retrieveOnSegments(ctx context.Context, replica ReplicaInterface, segType segmentType, collID UniqueID, plan *RetrievePlan, segIDs []UniqueID, vcm storage.ChunkManager) ([]*segcorepb.RetrieveResults, error) {
var retrieveResults []*segcorepb.RetrieveResults
queryLabel := metrics.SealedSegmentLabel
if segType == commonpb.SegmentState_Growing {
queryLabel = metrics.GrowingSegmentLabel
}
for _, segID := range segIDs {
seg, err := replica.getSegmentByID(segID, segType)
if err != nil {
@ -37,6 +46,8 @@ func retrieveOnSegments(ctx context.Context, replica ReplicaInterface, segType s
}
return nil, err
}
// record retrieve time
tr := timerecord.NewTimeRecorder("retrieveOnSegments")
result, err := seg.retrieve(plan)
if err != nil {
return nil, err
@ -45,6 +56,8 @@ func retrieveOnSegments(ctx context.Context, replica ReplicaInterface, segType s
return nil, err
}
retrieveResults = append(retrieveResults, result)
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
metrics.QueryLabel, queryLabel).Observe(float64(tr.ElapseSpan().Milliseconds()))
}
return retrieveResults, nil
}

View File

@ -74,8 +74,11 @@ func searchSegments(ctx context.Context, replica ReplicaInterface, segType segme
errs[i] = err
resultCh <- searchResult
// update metrics
elapsed := tr.ElapseSpan().Milliseconds()
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
metrics.SearchLabel, searchLabel).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.SearchLabel, searchLabel).Observe(float64(elapsed))
metrics.QueryNodeSegmentSearchLatencyPerVector.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
metrics.SearchLabel, searchLabel).Observe(float64(elapsed) / float64(searchReq.getNumOfQuery()))
}(segID, i)
}
wg.Wait()

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
var _ readTask = (*queryTask)(nil)
@ -47,7 +48,7 @@ func (q *queryTask) PreExecute(ctx context.Context) error {
if !funcutil.CheckCtxValid(q.Ctx()) {
return errors.New("search context timeout1$")
}
q.SetStep(TaskStepPreExecute)
q.SetStep(typeutil.TaskStepPreExecute)
rateCol.rtCounter.increaseQueueTime(q)
return nil
}

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap"
)
@ -44,7 +45,7 @@ type readTask interface {
TimeoutError() error
SetMaxCPUUsage(int32)
SetStep(step TaskStep)
SetStep(step typeutil.TaskStep)
}
var _ readTask = (*baseReadTask)(nil)
@ -62,7 +63,7 @@ type baseReadTask struct {
TravelTimestamp uint64
GuaranteeTimestamp uint64
TimeoutTimestamp uint64
step TaskStep
step typeutil.TaskStep
queueDur time.Duration
reduceDur time.Duration
waitTsDur time.Duration
@ -70,19 +71,19 @@ type baseReadTask struct {
tr *timerecord.TimeRecorder
}
func (b *baseReadTask) SetStep(step TaskStep) {
func (b *baseReadTask) SetStep(step typeutil.TaskStep) {
b.step = step
switch step {
case TaskStepEnqueue:
case typeutil.TaskStepEnqueue:
b.queueDur = 0
b.tr.Record("enqueue done")
case TaskStepPreExecute:
case typeutil.TaskStepPreExecute:
b.queueDur = b.tr.Record("start to process")
}
}
func (b *baseReadTask) OnEnqueue() error {
b.SetStep(TaskStepEnqueue)
b.SetStep(typeutil.TaskStepEnqueue)
return nil
}
@ -91,25 +92,25 @@ func (b *baseReadTask) SetMaxCPUUsage(cpu int32) {
}
func (b *baseReadTask) PreExecute(ctx context.Context) error {
b.SetStep(TaskStepPreExecute)
b.SetStep(typeutil.TaskStepPreExecute)
return nil
}
func (b *baseReadTask) Execute(ctx context.Context) error {
b.SetStep(TaskStepExecute)
b.SetStep(typeutil.TaskStepExecute)
return nil
}
func (b *baseReadTask) PostExecute(ctx context.Context) error {
b.SetStep(TaskStepPostExecute)
b.SetStep(typeutil.TaskStepPostExecute)
return nil
}
func (b *baseReadTask) Notify(err error) {
switch b.step {
case TaskStepEnqueue:
case typeutil.TaskStepEnqueue:
b.queueDur = b.tr.Record("enqueueEnd")
case TaskStepPostExecute:
case typeutil.TaskStepPostExecute:
b.tr.Record("execute task done")
}
b.baseTask.Notify(err)

View File

@ -9,6 +9,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
)
@ -57,14 +58,14 @@ func (s *baseReadTaskSuite) TestPreExecute() {
ctx := context.Background()
err := s.task.PreExecute(ctx)
s.Assert().NoError(err)
s.Assert().Equal(TaskStepPreExecute, s.task.step)
s.Assert().Equal(typeutil.TaskStepPreExecute, s.task.step)
}
func (s *baseReadTaskSuite) TestExecute() {
ctx := context.Background()
err := s.task.Execute(ctx)
s.Assert().NoError(err)
s.Assert().Equal(TaskStepExecute, s.task.step)
s.Assert().Equal(typeutil.TaskStepExecute, s.task.step)
}
func (s *baseReadTaskSuite) TestTimeout() {

View File

@ -21,6 +21,7 @@ import (
"errors"
"testing"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert"
)
@ -68,7 +69,7 @@ type mockReadTask struct {
canMerge bool
timeout bool
timeoutError error
step TaskStep
step typeutil.TaskStep
readyError error
}
@ -100,7 +101,7 @@ func (m *mockReadTask) SetMaxCPUUsage(cpu int32) {
m.maxCPU = cpu
}
func (m *mockReadTask) SetStep(step TaskStep) {
func (m *mockReadTask) SetStep(step typeutil.TaskStep) {
m.step = step
}

View File

@ -36,6 +36,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
var _ readTask = (*searchTask)(nil)
@ -59,10 +60,10 @@ type searchTask struct {
}
func (s *searchTask) PreExecute(ctx context.Context) error {
s.SetStep(TaskStepPreExecute)
s.SetStep(typeutil.TaskStepPreExecute)
rateCol.rtCounter.increaseQueueTime(s)
for _, t := range s.otherTasks {
t.SetStep(TaskStepPreExecute)
t.SetStep(typeutil.TaskStepPreExecute)
rateCol.rtCounter.increaseQueueTime(t)
}
s.combinePlaceHolderGroups()

View File

@ -26,11 +26,13 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
type watchDmChannelsTask struct {
@ -43,6 +45,7 @@ type watchDmChannelsTask struct {
func (w *watchDmChannelsTask) Execute(ctx context.Context) (err error) {
collectionID := w.req.CollectionID
partitionIDs := w.req.GetPartitionIDs()
tr := timerecord.NewTimeRecorder("watchDmChannels")
lType := w.req.GetLoadMeta().GetLoadType()
if lType == queryPb.LoadType_UnKnownType {
@ -145,6 +148,7 @@ func (w *watchDmChannelsTask) Execute(ctx context.Context) (err error) {
fg.flowGraph.Start()
}
metrics.QueryNodeWatchDmlChannelLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Observe(float64(tr.ElapseSpan().Milliseconds()))
log.Info("WatchDmChannels done")
return nil
}

View File

@ -5,6 +5,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type alterAliasTask struct {
@ -13,6 +14,7 @@ type alterAliasTask struct {
}
func (t *alterAliasTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.GetBase().GetMsgType(), commonpb.MsgType_AlterAlias); err != nil {
return err
}
@ -20,6 +22,7 @@ func (t *alterAliasTask) Prepare(ctx context.Context) error {
}
func (t *alterAliasTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
if err := t.core.ExpireMetaCache(ctx, []string{t.Req.GetAlias()}, InvalidCollectionID, t.GetTs()); err != nil {
return err
}

View File

@ -13,13 +13,17 @@ import (
func Test_alterAliasTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &alterAliasTask{Req: &milvuspb.AlterAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}}}
task := &alterAliasTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.AlterAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}}}
err := task.Prepare(context.Background())
assert.Error(t, err)
})
t.Run("normal case", func(t *testing.T) {
task := &alterAliasTask{Req: &milvuspb.AlterAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias}}}
task := &alterAliasTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.AlterAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias}}}
err := task.Prepare(context.Background())
assert.NoError(t, err)
})
@ -29,7 +33,7 @@ func Test_alterAliasTask_Execute(t *testing.T) {
t.Run("failed to expire cache", func(t *testing.T) {
core := newTestCore(withInvalidProxyManager())
task := &alterAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.AlterAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
Alias: "test",
@ -42,7 +46,7 @@ func Test_alterAliasTask_Execute(t *testing.T) {
t.Run("failed to alter alias", func(t *testing.T) {
core := newTestCore(withValidProxyManager(), withInvalidMeta())
task := &alterAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.AlterAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
Alias: "test",

View File

@ -5,10 +5,11 @@ import (
"errors"
"fmt"
"github.com/milvus-io/milvus/internal/log"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type alterCollectionTask struct {
@ -17,6 +18,7 @@ type alterCollectionTask struct {
}
func (a *alterCollectionTask) Prepare(ctx context.Context) error {
a.SetStep(typeutil.TaskStepExecute)
if a.Req.GetCollectionName() == "" {
return fmt.Errorf("alter collection failed, collection name does not exists")
}
@ -25,6 +27,7 @@ func (a *alterCollectionTask) Prepare(ctx context.Context) error {
}
func (a *alterCollectionTask) Execute(ctx context.Context) error {
a.SetStep(typeutil.TaskStepExecute)
// Now we only support alter properties of collection
if a.Req.GetProperties() == nil {
return errors.New("only support alter collection properties, but collection properties is empty")

View File

@ -5,14 +5,12 @@ import (
"errors"
"testing"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/metastore/model"
)
func Test_alterCollectionTask_Prepare(t *testing.T) {
@ -51,7 +49,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
t.Run("failed to create alias", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &alterCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.AlterCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
CollectionName: "cn",
@ -73,7 +71,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
core := newTestCore(withMeta(meta))
task := &alterCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.AlterCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
CollectionName: "cn",
@ -101,7 +99,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker))
task := &alterCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.AlterCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
CollectionName: "cn",
@ -129,7 +127,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker))
task := &alterCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.AlterCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
CollectionName: "cn",

View File

@ -4,8 +4,8 @@ import (
"context"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type createAliasTask struct {
@ -14,6 +14,7 @@ type createAliasTask struct {
}
func (t *createAliasTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.GetBase().GetMsgType(), commonpb.MsgType_CreateAlias); err != nil {
return err
}
@ -21,6 +22,7 @@ func (t *createAliasTask) Prepare(ctx context.Context) error {
}
func (t *createAliasTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
if err := t.core.ExpireMetaCache(ctx, []string{t.Req.GetAlias(), t.Req.GetCollectionName()}, InvalidCollectionID, t.GetTs()); err != nil {
return err
}

View File

@ -12,13 +12,17 @@ import (
func Test_createAliasTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &createAliasTask{Req: &milvuspb.CreateAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}}}
task := &createAliasTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.CreateAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}}}
err := task.Prepare(context.Background())
assert.Error(t, err)
})
t.Run("normal case", func(t *testing.T) {
task := &createAliasTask{Req: &milvuspb.CreateAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias}}}
task := &createAliasTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.CreateAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias}}}
err := task.Prepare(context.Background())
assert.NoError(t, err)
})
@ -28,7 +32,7 @@ func Test_createAliasTask_Execute(t *testing.T) {
t.Run("failed to expire cache", func(t *testing.T) {
core := newTestCore(withInvalidProxyManager())
task := &createAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias},
Alias: "test",
@ -41,7 +45,7 @@ func Test_createAliasTask_Execute(t *testing.T) {
t.Run("failed to create alias", func(t *testing.T) {
core := newTestCore(withInvalidMeta(), withValidProxyManager())
task := &createAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias},
Alias: "test",

View File

@ -173,6 +173,7 @@ func (t *createCollectionTask) assignChannels() error {
}
func (t *createCollectionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := t.validate(); err != nil {
return err
}
@ -235,6 +236,7 @@ func (t *createCollectionTask) addChannelsAndGetStartPositions(ctx context.Conte
}
func (t *createCollectionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
collID := t.collID
partID := t.partID
ts := t.GetTs()

View File

@ -207,6 +207,7 @@ func Test_createCollectionTask_prepareSchema(t *testing.T) {
func Test_createCollectionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &createCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
},
@ -218,6 +219,7 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
t.Run("invalid schema", func(t *testing.T) {
collectionName := funcutil.GenRandomStr()
task := &createCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,
@ -245,7 +247,7 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
core := newTestCore(withInvalidIDAllocator())
task := createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,
@ -278,7 +280,7 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
assert.NoError(t, err)
task := createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,
@ -311,7 +313,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
core := newTestCore(withMeta(meta), withTtSynchronizer(ticker))
task := &createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,
@ -356,7 +358,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
core := newTestCore(withMeta(meta), withTtSynchronizer(ticker))
task := &createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,
@ -376,7 +378,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
pchans := ticker.getDmlChannelNames(shardNum)
core := newTestCore(withTtSynchronizer(ticker))
task := &createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
channels: collectionChannels{
physicalChannels: pchans,
virtualChannels: []string{funcutil.GenRandomStr(), funcutil.GenRandomStr()},
@ -441,7 +443,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
assert.NoError(t, err)
task := createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,
@ -524,7 +526,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
assert.NoError(t, err)
task := createCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
CollectionName: collectionName,

View File

@ -3,15 +3,15 @@ package rootcoord
import (
"context"
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type createPartitionTask struct {
@ -21,6 +21,7 @@ type createPartitionTask struct {
}
func (t *createPartitionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.GetBase().GetMsgType(), commonpb.MsgType_CreatePartition); err != nil {
return err
}
@ -33,6 +34,7 @@ func (t *createPartitionTask) Prepare(ctx context.Context) error {
}
func (t *createPartitionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
for _, partition := range t.collMeta.Partitions {
if partition.PartitionName == t.Req.GetPartitionName() {
log.Warn("add duplicate partition", zap.String("collection", t.Req.GetCollectionName()), zap.String("partition", t.Req.GetPartitionName()), zap.Uint64("ts", t.GetTs()))

View File

@ -17,7 +17,8 @@ import (
func Test_createPartitionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &createPartitionTask{
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}},
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}},
}
err := task.Prepare(context.Background())
assert.Error(t, err)
@ -26,7 +27,7 @@ func Test_createPartitionTask_Prepare(t *testing.T) {
t.Run("failed to get collection meta", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &createPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreatePartition}},
}
err := task.Prepare(context.Background())
@ -42,7 +43,7 @@ func Test_createPartitionTask_Prepare(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &createPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreatePartition}},
}
err := task.Prepare(context.Background())
@ -70,7 +71,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
core := newTestCore(withInvalidIDAllocator())
task := &createPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
collMeta: coll,
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
}
@ -84,7 +85,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
core := newTestCore(withValidIDAllocator(), withInvalidProxyManager())
task := &createPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
collMeta: coll,
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
}
@ -98,7 +99,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
core := newTestCore(withValidIDAllocator(), withValidProxyManager(), withInvalidMeta())
task := &createPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
collMeta: coll,
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
}
@ -116,7 +117,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
}
core := newTestCore(withValidIDAllocator(), withValidProxyManager(), withMeta(meta))
task := &createPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
collMeta: coll,
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
}

View File

@ -5,6 +5,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
// describeCollectionTask describe collection request task
@ -16,6 +17,7 @@ type describeCollectionTask struct {
}
func (t *describeCollectionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.Base.MsgType, commonpb.MsgType_DescribeCollection); err != nil {
return err
}
@ -24,6 +26,7 @@ func (t *describeCollectionTask) Prepare(ctx context.Context) error {
// Execute task execution
func (t *describeCollectionTask) Execute(ctx context.Context) (err error) {
t.SetStep(typeutil.TaskStepExecute)
coll, err := t.core.describeCollection(ctx, t.Req, t.allowUnavailable)
if err != nil {
return err

View File

@ -14,6 +14,7 @@ import (
func Test_describeCollectionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &describeCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropCollection,
@ -26,6 +27,7 @@ func Test_describeCollectionTask_Prepare(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
task := &describeCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
@ -41,10 +43,7 @@ func Test_describeCollectionTask_Execute(t *testing.T) {
t.Run("failed to get collection by name", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &describeCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
@ -60,10 +59,7 @@ func Test_describeCollectionTask_Execute(t *testing.T) {
t.Run("failed to get collection by id", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &describeCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
@ -91,10 +87,7 @@ func Test_describeCollectionTask_Execute(t *testing.T) {
core := newTestCore(withMeta(meta))
task := &describeCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,

View File

@ -4,8 +4,8 @@ import (
"context"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type dropAliasTask struct {
@ -14,6 +14,7 @@ type dropAliasTask struct {
}
func (t *dropAliasTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.GetBase().GetMsgType(), commonpb.MsgType_DropAlias); err != nil {
return err
}
@ -21,6 +22,7 @@ func (t *dropAliasTask) Prepare(ctx context.Context) error {
}
func (t *dropAliasTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
// drop alias is atomic enough.
if err := t.core.ExpireMetaCache(ctx, []string{t.Req.GetAlias()}, InvalidCollectionID, t.GetTs()); err != nil {
return err

View File

@ -15,7 +15,8 @@ import (
func Test_dropAliasTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &dropAliasTask{
Req: &milvuspb.DropAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}},
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DropAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}},
}
err := task.Prepare(context.Background())
assert.Error(t, err)
@ -23,7 +24,8 @@ func Test_dropAliasTask_Prepare(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
task := &dropAliasTask{
Req: &milvuspb.DropAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias}},
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DropAliasRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias}},
}
err := task.Prepare(context.Background())
assert.NoError(t, err)
@ -35,7 +37,7 @@ func Test_dropAliasTask_Execute(t *testing.T) {
core := newTestCore(withInvalidProxyManager())
alias := funcutil.GenRandomStr()
task := &dropAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
@ -50,7 +52,7 @@ func Test_dropAliasTask_Execute(t *testing.T) {
core := newTestCore(withValidProxyManager(), withInvalidMeta())
alias := funcutil.GenRandomStr()
task := &dropAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
@ -69,7 +71,7 @@ func Test_dropAliasTask_Execute(t *testing.T) {
core := newTestCore(withValidProxyManager(), withMeta(meta))
alias := funcutil.GenRandomStr()
task := &dropAliasTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropAliasRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},

View File

@ -34,10 +34,12 @@ func (t *dropCollectionTask) validate() error {
}
func (t *dropCollectionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
return t.validate()
}
func (t *dropCollectionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
// use max ts to check if latest collection exists.
// we cannot handle case that
// dropping collection with `ts1` but a collection exists in catalog with newer ts which is bigger than `ts1`.

View File

@ -21,6 +21,7 @@ import (
func Test_dropCollectionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &dropCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DescribeCollection},
},
@ -37,7 +38,7 @@ func Test_dropCollectionTask_Prepare(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &dropCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
CollectionName: collectionName,
@ -55,7 +56,7 @@ func Test_dropCollectionTask_Prepare(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &dropCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
CollectionName: collectionName,
@ -82,7 +83,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
})
core := newTestCore(withMeta(meta))
task := &dropCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
CollectionName: collectionName,
@ -111,7 +112,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
core := newTestCore(withInvalidProxyManager(), withMeta(meta))
task := &dropCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
CollectionName: collectionName,
@ -136,7 +137,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
}
core := newTestCore(withValidProxyManager(), withMeta(meta))
task := &dropCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
CollectionName: collectionName,
@ -215,7 +216,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
withTtSynchronizer(ticker))
task := &dropCollectionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
CollectionName: collectionName,

View File

@ -4,17 +4,16 @@ import (
"context"
"fmt"
"github.com/milvus-io/milvus/internal/log"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type dropPartitionTask struct {
@ -24,6 +23,7 @@ type dropPartitionTask struct {
}
func (t *dropPartitionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.GetBase().GetMsgType(), commonpb.MsgType_DropPartition); err != nil {
return err
}
@ -40,6 +40,7 @@ func (t *dropPartitionTask) Prepare(ctx context.Context) error {
}
func (t *dropPartitionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
partID := common.InvalidPartitionID
for _, partition := range t.collMeta.Partitions {
if partition.PartitionName == t.Req.GetPartitionName() {

View File

@ -18,6 +18,7 @@ import (
func Test_dropPartitionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &dropPartitionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
},
@ -28,6 +29,7 @@ func Test_dropPartitionTask_Prepare(t *testing.T) {
t.Run("drop default partition", func(t *testing.T) {
task := &dropPartitionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
PartitionName: Params.CommonCfg.DefaultPartitionName,
@ -40,7 +42,7 @@ func Test_dropPartitionTask_Prepare(t *testing.T) {
t.Run("failed to get collection meta", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &dropPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
},
@ -60,7 +62,7 @@ func Test_dropPartitionTask_Prepare(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &dropPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
CollectionName: collectionName,
@ -95,7 +97,7 @@ func Test_dropPartitionTask_Execute(t *testing.T) {
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{{PartitionName: partitionName}}}
core := newTestCore(withInvalidProxyManager())
task := &dropPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
CollectionName: collectionName,
@ -113,7 +115,7 @@ func Test_dropPartitionTask_Execute(t *testing.T) {
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{{PartitionName: partitionName}}}
core := newTestCore(withValidProxyManager(), withInvalidMeta())
task := &dropPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
CollectionName: collectionName,
@ -169,7 +171,7 @@ func Test_dropPartitionTask_Execute(t *testing.T) {
withBroker(broker))
task := &dropPartitionTask{
baseTask: baseTask{core: core},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.DropPartitionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
CollectionName: collectionName,

View File

@ -5,6 +5,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
// hasCollectionTask has collection request task
@ -15,6 +16,7 @@ type hasCollectionTask struct {
}
func (t *hasCollectionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.Base.MsgType, commonpb.MsgType_HasCollection); err != nil {
return err
}
@ -23,6 +25,7 @@ func (t *hasCollectionTask) Prepare(ctx context.Context) error {
// Execute task execution
func (t *hasCollectionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
t.Rsp.Status = succStatus()
ts := getTravelTs(t.Req)
// TODO: what if err != nil && common.IsCollectionNotExistError == false, should we consider this RPC as failure?

View File

@ -13,6 +13,7 @@ import (
func Test_hasCollectionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &hasCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined,
@ -26,6 +27,7 @@ func Test_hasCollectionTask_Prepare(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
task := &hasCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasCollection,
@ -41,10 +43,7 @@ func Test_hasCollectionTask_Execute(t *testing.T) {
t.Run("failed", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &hasCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasCollection,
@ -65,10 +64,7 @@ func Test_hasCollectionTask_Execute(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &hasCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.HasCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasCollection,

View File

@ -16,6 +16,7 @@ type hasPartitionTask struct {
}
func (t *hasPartitionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.Base.MsgType, commonpb.MsgType_HasPartition); err != nil {
return err
}
@ -24,6 +25,7 @@ func (t *hasPartitionTask) Prepare(ctx context.Context) error {
// Execute task execution
func (t *hasPartitionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
t.Rsp.Status = succStatus()
t.Rsp.Value = false
// TODO: why HasPartitionRequest doesn't contain Timestamp but other requests do.

View File

@ -13,6 +13,7 @@ import (
func Test_hasPartitionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &hasPartitionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined,
@ -25,6 +26,7 @@ func Test_hasPartitionTask_Prepare(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
task := &hasPartitionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasPartition,
@ -40,10 +42,7 @@ func Test_hasPartitionTask_Execute(t *testing.T) {
t.Run("fail to get collection", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &hasPartitionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasPartition,
@ -71,10 +70,7 @@ func Test_hasPartitionTask_Execute(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &hasPartitionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasCollection,
@ -106,10 +102,7 @@ func Test_hasPartitionTask_Execute(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &hasPartitionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.HasPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_HasCollection,

View File

@ -22,6 +22,7 @@ import (
"fmt"
"math/rand"
"os"
"strconv"
"sync"
"sync/atomic"
"syscall"
@ -299,12 +300,14 @@ func (c *Core) Register() error {
if c.enableActiveStandBy {
c.session.ProcessActiveStandBy(c.activateFunc)
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(c.session.ServerID, 10), typeutil.RootCoordRole).Inc()
log.Info("RootCoord Register Finished")
go c.session.LivenessCheck(c.ctx, func() {
log.Error("Root Coord disconnected from etcd, process will exit", zap.Int64("Server Id", c.session.ServerID))
if err := c.Stop(); err != nil {
log.Fatal("failed to stop server", zap.Error(err))
}
metrics.NumNodes.WithLabelValues(strconv.FormatInt(c.session.ServerID, 10), typeutil.RootCoordRole).Dec()
// manually send signal to starter goroutine
if c.session.TriggerKill {
if p, err := os.FindProcess(os.Getpid()); err == nil {
@ -770,19 +773,16 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.TotalLabel).Inc()
method := "CreateCollection"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("CreateCollection")
log.Ctx(ctx).Info("received request to create collection", zap.String("role", typeutil.RootCoordRole),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &createCollectionTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -790,7 +790,7 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti
zap.Error(err),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -800,13 +800,14 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti
zap.String("name", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("CreateCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfCollections.Inc()
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Ctx(ctx).Info("done to create collection", zap.String("role", typeutil.RootCoordRole),
zap.String("name", in.GetCollectionName()),
@ -820,19 +821,16 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.TotalLabel).Inc()
method := "DropCollection"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("DropCollection")
log.Ctx(ctx).Info("received request to drop collection", zap.String("role", typeutil.RootCoordRole),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &dropCollectionTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -840,7 +838,7 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe
zap.Error(err),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -850,13 +848,14 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe
zap.String("name", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("DropCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfCollections.Dec()
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Ctx(ctx).Info("done to drop collection", zap.String("role", typeutil.RootCoordRole),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()),
@ -873,7 +872,8 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.TotalLabel).Inc()
method := "HasCollection"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("HasCollection")
ts := getTravelTs(in)
@ -889,7 +889,7 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ
if err := c.scheduler.AddTask(t); err != nil {
log.Warn("failed to enqueue request to has collection", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.BoolResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasCollection failed: "+err.Error()),
Value: false,
@ -898,15 +898,16 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ
if err := t.WaitToFinish(); err != nil {
log.Warn("failed to has collection", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.BoolResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasCollection failed: "+err.Error()),
Value: false,
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("HasCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to has collection", zap.Bool("exist", t.Rsp.GetValue()))
@ -956,7 +957,8 @@ func (c *Core) describeCollectionImpl(ctx context.Context, in *milvuspb.Describe
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.TotalLabel).Inc()
method := "DescribeCollection"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("DescribeCollection")
ts := getTravelTs(in)
@ -997,8 +999,9 @@ func (c *Core) describeCollectionImpl(ctx context.Context, in *milvuspb.Describe
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("DescribeCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to describe collection", zap.Int64("collection_id", t.Rsp.GetCollectionID()))
@ -1027,8 +1030,8 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]),
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.TotalLabel).Inc()
method := "ShowCollections"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("ShowCollections")
ts := getTravelTs(in)
@ -1044,7 +1047,7 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections
if err := c.scheduler.AddTask(t); err != nil {
log.Warn("failed to enqueue request to show collections", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.ShowCollectionsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowCollections failed: "+err.Error()),
}, nil
@ -1052,14 +1055,15 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections
if err := t.WaitToFinish(); err != nil {
log.Warn("failed to show collections", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.ShowCollectionsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowCollections failed: "+err.Error()),
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("ShowCollections").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to show collections", zap.Int("num of collections", len(t.Rsp.GetCollectionNames()))) // maybe very large, print number instead.
@ -1071,19 +1075,16 @@ func (c *Core) AlterCollection(ctx context.Context, in *milvuspb.AlterCollection
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterCollection", metrics.TotalLabel).Inc()
method := "AlterCollection"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("AlterCollection")
log.Info("received request to alter collection", zap.String("role", typeutil.RootCoordRole),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &alterCollectionTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -1092,7 +1093,7 @@ func (c *Core) AlterCollection(ctx context.Context, in *milvuspb.AlterCollection
zap.Error(err),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -1103,13 +1104,14 @@ func (c *Core) AlterCollection(ctx context.Context, in *milvuspb.AlterCollection
zap.String("name", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterCollection", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterCollection", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("AlterCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordNumOfCollections.Dec()
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to alter collection", zap.String("role", typeutil.RootCoordRole),
zap.String("name", in.GetCollectionName()), zap.Int64("msgID", in.GetBase().GetMsgID()),
@ -1123,7 +1125,8 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.TotalLabel).Inc()
method := "CreatePartition"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("CreatePartition")
log.Ctx(ctx).Info("received request to create partition", zap.String("role", typeutil.RootCoordRole),
@ -1131,12 +1134,8 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &createPartitionTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -1145,7 +1144,7 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -1155,12 +1154,13 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("CreatePartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Ctx(ctx).Info("done to create partition", zap.String("role", typeutil.RootCoordRole),
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
@ -1174,7 +1174,8 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.TotalLabel).Inc()
method := "DropPartition"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("DropPartition")
log.Ctx(ctx).Info("received request to drop partition", zap.String("role", typeutil.RootCoordRole),
@ -1182,12 +1183,8 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &dropPartitionTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -1196,7 +1193,7 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
if err := t.WaitToFinish(); err != nil {
@ -1205,12 +1202,13 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("DropPartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Ctx(ctx).Info("done to drop partition", zap.String("role", typeutil.RootCoordRole),
zap.String("collection", in.GetCollectionName()), zap.String("partition", in.GetPartitionName()),
@ -1227,7 +1225,8 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.TotalLabel).Inc()
method := "HasPartition"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("HasPartition")
// TODO(longjiquan): why HasPartitionRequest doesn't contain Timestamp but other requests do.
@ -1244,7 +1243,7 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
if err := c.scheduler.AddTask(t); err != nil {
log.Warn("failed to enqueue request to has partition", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.BoolResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasPartition failed: "+err.Error()),
Value: false,
@ -1253,15 +1252,16 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
if err := t.WaitToFinish(); err != nil {
log.Warn("failed to has partition", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.BoolResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "HasPartition failed: "+err.Error()),
Value: false,
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("HasPartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to has partition", zap.Bool("exist", t.Rsp.GetValue()))
@ -1275,7 +1275,8 @@ func (c *Core) showPartitionsImpl(ctx context.Context, in *milvuspb.ShowPartitio
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.TotalLabel).Inc()
method := "ShowPartitions"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("ShowPartitions")
log := log.Ctx(ctx).With(
@ -1294,7 +1295,7 @@ func (c *Core) showPartitionsImpl(ctx context.Context, in *milvuspb.ShowPartitio
if err := c.scheduler.AddTask(t); err != nil {
log.Warn("failed to enqueue request to show partitions", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.ShowPartitionsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowPartitions failed: "+err.Error()),
// Status: common.StatusFromError(err),
@ -1303,15 +1304,16 @@ func (c *Core) showPartitionsImpl(ctx context.Context, in *milvuspb.ShowPartitio
if err := t.WaitToFinish(); err != nil {
log.Warn("failed to show partitions", zap.Error(err))
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return &milvuspb.ShowPartitionsResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "ShowPartitions failed: "+err.Error()),
// Status: common.StatusFromError(err),
}, nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("ShowPartitions").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to show partitions", zap.Strings("partitions", t.Rsp.GetPartitionNames()))
@ -1491,7 +1493,8 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.TotalLabel).Inc()
method := "CreateAlias"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("CreateAlias")
log.Info("received request to create alias", zap.String("role", typeutil.RootCoordRole),
@ -1499,12 +1502,8 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &createAliasTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -1514,7 +1513,7 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -1525,12 +1524,13 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("CreateAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to create alias", zap.String("role", typeutil.RootCoordRole),
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
@ -1544,19 +1544,16 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.TotalLabel).Inc()
method := "DropAlias"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("DropAlias")
log.Info("received request to drop alias", zap.String("role", typeutil.RootCoordRole),
zap.String("alias", in.GetAlias()), zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &dropAliasTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -1565,7 +1562,7 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c
zap.Error(err),
zap.String("alias", in.GetAlias()), zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -1576,12 +1573,13 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c
zap.String("alias", in.GetAlias()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("DropAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to drop alias", zap.String("role", typeutil.RootCoordRole),
zap.String("alias", in.GetAlias()),
@ -1595,7 +1593,8 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+commonpb.StateCode_name[int32(code)]), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.TotalLabel).Inc()
method := "AlterAlias"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()
tr := timerecord.NewTimeRecorder("AlterAlias")
log.Info("received request to alter alias", zap.String("role", typeutil.RootCoordRole),
@ -1603,12 +1602,8 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
zap.Int64("msgID", in.GetBase().GetMsgID()))
t := &alterAliasTask{
baseTask: baseTask{
ctx: ctx,
core: c,
done: make(chan error, 1),
},
Req: in,
baseTask: newBaseTask(ctx, c),
Req: in,
}
if err := c.scheduler.AddTask(t); err != nil {
@ -1618,7 +1613,7 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()))
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterAlias", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
@ -1629,12 +1624,13 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),
zap.Int64("msgID", in.GetBase().GetMsgID()), zap.Uint64("ts", t.GetTs()))
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterAlias", metrics.FailLabel).Inc()
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc()
return failStatus(commonpb.ErrorCode_UnexpectedError, err.Error()), nil
}
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterAlias", metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues("AlterAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc()
metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds()))
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues(method).Observe(float64(t.queueDur.Milliseconds()))
log.Info("done to alter alias", zap.String("role", typeutil.RootCoordRole),
zap.String("alias", in.GetAlias()), zap.String("collection", in.GetCollectionName()),

View File

@ -119,6 +119,7 @@ func (s *scheduler) setTs(task task) error {
}
func (s *scheduler) enqueue(task task) {
task.OnEnqueue()
s.taskChan <- task
}

View File

@ -11,6 +11,8 @@ import (
"go.uber.org/atomic"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type mockFailTask struct {
@ -21,10 +23,7 @@ type mockFailTask struct {
func newMockFailTask() *mockFailTask {
task := &mockFailTask{
baseTask: baseTask{
ctx: context.Background(),
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), nil),
}
task.SetCtx(context.Background())
return task
@ -43,10 +42,12 @@ func newMockExecuteFailTask() *mockFailTask {
}
func (m mockFailTask) Prepare(context.Context) error {
m.SetStep(typeutil.TaskStepPreExecute)
return m.prepareErr
}
func (m mockFailTask) Execute(context.Context) error {
m.SetStep(typeutil.TaskStepExecute)
return m.executeErr
}
@ -56,10 +57,7 @@ type mockNormalTask struct {
func newMockNormalTask() *mockNormalTask {
task := &mockNormalTask{
baseTask: baseTask{
ctx: context.Background(),
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), nil),
}
task.SetCtx(context.Background())
return task

View File

@ -17,6 +17,7 @@ type showCollectionTask struct {
}
func (t *showCollectionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.Base.MsgType, commonpb.MsgType_ShowCollections); err != nil {
return err
}
@ -25,6 +26,7 @@ func (t *showCollectionTask) Prepare(ctx context.Context) error {
// Execute task execution
func (t *showCollectionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
t.Rsp.Status = succStatus()
ts := t.Req.GetTimeStamp()
if ts == 0 {

View File

@ -13,6 +13,7 @@ import (
func Test_showCollectionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &showCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.ShowCollectionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined,
@ -25,6 +26,7 @@ func Test_showCollectionTask_Prepare(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
task := &showCollectionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.ShowCollectionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowCollections,
@ -40,10 +42,7 @@ func Test_showCollectionTask_Execute(t *testing.T) {
t.Run("failed to list collections", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &showCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.ShowCollectionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowCollections,
@ -69,10 +68,7 @@ func Test_showCollectionTask_Execute(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &showCollectionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.ShowCollectionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowCollections,

View File

@ -19,6 +19,7 @@ type showPartitionTask struct {
}
func (t *showPartitionTask) Prepare(ctx context.Context) error {
t.SetStep(typeutil.TaskStepPreExecute)
if err := CheckMsgType(t.Req.Base.MsgType, commonpb.MsgType_ShowPartitions); err != nil {
return err
}
@ -27,6 +28,7 @@ func (t *showPartitionTask) Prepare(ctx context.Context) error {
// Execute task execution
func (t *showPartitionTask) Execute(ctx context.Context) error {
t.SetStep(typeutil.TaskStepExecute)
var coll *model.Collection
var err error
t.Rsp.Status = succStatus()

View File

@ -14,6 +14,7 @@ import (
func Test_showPartitionTask_Prepare(t *testing.T) {
t.Run("invalid msg type", func(t *testing.T) {
task := &showPartitionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined,
@ -26,6 +27,7 @@ func Test_showPartitionTask_Prepare(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
task := &showPartitionTask{
baseTask: newBaseTask(context.TODO(), nil),
Req: &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,
@ -41,10 +43,7 @@ func Test_showPartitionTask_Execute(t *testing.T) {
t.Run("failed to list collections by name", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &showPartitionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,
@ -61,10 +60,7 @@ func Test_showPartitionTask_Execute(t *testing.T) {
t.Run("failed to list collections by id", func(t *testing.T) {
core := newTestCore(withInvalidMeta())
task := &showPartitionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,
@ -98,10 +94,7 @@ func Test_showPartitionTask_Execute(t *testing.T) {
}
core := newTestCore(withMeta(meta))
task := &showPartitionTask{
baseTask: baseTask{
core: core,
done: make(chan error, 1),
},
baseTask: newBaseTask(context.TODO(), core),
Req: &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,

View File

@ -2,6 +2,10 @@ package rootcoord
import (
"context"
"time"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type task interface {
@ -15,6 +19,7 @@ type task interface {
Execute(ctx context.Context) error
WaitToFinish() error
NotifyDone(err error)
OnEnqueue() error
}
type baseTask struct {
@ -23,17 +28,33 @@ type baseTask struct {
done chan error
ts Timestamp
id UniqueID
tr *timerecord.TimeRecorder
step typeutil.TaskStep
queueDur time.Duration
}
func newBaseTask(ctx context.Context, core *Core) baseTask {
b := baseTask{
core: core,
done: make(chan error, 1),
tr: timerecord.NewTimeRecorder("ddl request"),
}
b.SetCtx(ctx)
return b
}
func (b *baseTask) SetStep(step typeutil.TaskStep) {
b.step = step
switch step {
case typeutil.TaskStepEnqueue:
b.queueDur = 0
b.tr.Record("enqueue done")
case typeutil.TaskStepPreExecute:
b.queueDur = b.tr.Record("start to process")
}
}
func (b *baseTask) SetCtx(ctx context.Context) {
b.ctx = ctx
}
@ -59,10 +80,12 @@ func (b *baseTask) GetID() UniqueID {
}
func (b *baseTask) Prepare(ctx context.Context) error {
b.SetStep(typeutil.TaskStepPreExecute)
return nil
}
func (b *baseTask) Execute(ctx context.Context) error {
b.SetStep(typeutil.TaskStepExecute)
return nil
}
@ -73,3 +96,8 @@ func (b *baseTask) WaitToFinish() error {
func (b *baseTask) NotifyDone(err error) {
b.done <- err
}
func (b *baseTask) OnEnqueue() error {
b.SetStep(typeutil.TaskStepEnqueue)
return nil
}

View File

@ -26,15 +26,18 @@ import (
"strings"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/storage/gcp"
"github.com/milvus-io/milvus/internal/util/errorutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
"go.uber.org/zap"
"golang.org/x/exp/mmap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/storage/gcp"
"github.com/milvus-io/milvus/internal/util/errorutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
var (
@ -162,7 +165,7 @@ func (mcm *MinioChunkManager) Path(ctx context.Context, filePath string) (string
// Reader returns the path of minio data if exists.
func (mcm *MinioChunkManager) Reader(ctx context.Context, filePath string) (FileReader, error) {
reader, err := mcm.Client.GetObject(ctx, mcm.bucketName, filePath, minio.GetObjectOptions{})
reader, err := mcm.getMinioObject(ctx, mcm.bucketName, filePath, minio.GetObjectOptions{})
if err != nil {
log.Warn("failed to get object", zap.String("path", filePath), zap.Error(err))
return nil, err
@ -171,7 +174,7 @@ func (mcm *MinioChunkManager) Reader(ctx context.Context, filePath string) (File
}
func (mcm *MinioChunkManager) Size(ctx context.Context, filePath string) (int64, error) {
objectInfo, err := mcm.Client.StatObject(ctx, mcm.bucketName, filePath, minio.StatObjectOptions{})
objectInfo, err := mcm.statMinioObject(ctx, mcm.bucketName, filePath, minio.StatObjectOptions{})
if err != nil {
log.Warn("failed to stat object", zap.String("path", filePath), zap.Error(err))
return 0, err
@ -182,13 +185,14 @@ func (mcm *MinioChunkManager) Size(ctx context.Context, filePath string) (int64,
// Write writes the data to minio storage.
func (mcm *MinioChunkManager) Write(ctx context.Context, filePath string, content []byte) error {
_, err := mcm.Client.PutObject(ctx, mcm.bucketName, filePath, bytes.NewReader(content), int64(len(content)), minio.PutObjectOptions{})
_, err := mcm.putMinioObject(ctx, mcm.bucketName, filePath, bytes.NewReader(content), int64(len(content)), minio.PutObjectOptions{})
if err != nil {
log.Warn("failed to put object", zap.String("path", filePath), zap.Error(err))
return err
}
metrics.PersistentDataKvSize.WithLabelValues(metrics.DataPutLabel).Observe(float64(len(content)))
return nil
}
@ -210,7 +214,7 @@ func (mcm *MinioChunkManager) MultiWrite(ctx context.Context, kvs map[string][]b
// Exist checks whether chunk is saved to minio storage.
func (mcm *MinioChunkManager) Exist(ctx context.Context, filePath string) (bool, error) {
_, err := mcm.Client.StatObject(ctx, mcm.bucketName, filePath, minio.StatObjectOptions{})
_, err := mcm.statMinioObject(ctx, mcm.bucketName, filePath, minio.StatObjectOptions{})
if err != nil {
errResponse := minio.ToErrorResponse(err)
if errResponse.Code == "NoSuchKey" {
@ -224,7 +228,7 @@ func (mcm *MinioChunkManager) Exist(ctx context.Context, filePath string) (bool,
// Read reads the minio storage data if exists.
func (mcm *MinioChunkManager) Read(ctx context.Context, filePath string) ([]byte, error) {
object, err := mcm.Client.GetObject(ctx, mcm.bucketName, filePath, minio.GetObjectOptions{})
object, err := mcm.getMinioObject(ctx, mcm.bucketName, filePath, minio.GetObjectOptions{})
if err != nil {
log.Warn("failed to get object", zap.String("path", filePath), zap.Error(err))
return nil, err
@ -250,6 +254,7 @@ func (mcm *MinioChunkManager) Read(ctx context.Context, filePath string) ([]byte
log.Warn("failed to read object", zap.String("path", filePath), zap.Error(err))
return nil, err
}
metrics.PersistentDataKvSize.WithLabelValues(metrics.DataGetLabel).Observe(float64(objectInfo.Size))
return data, nil
}
@ -300,7 +305,7 @@ func (mcm *MinioChunkManager) ReadAt(ctx context.Context, filePath string, off i
return nil, err
}
object, err := mcm.Client.GetObject(ctx, mcm.bucketName, filePath, opts)
object, err := mcm.getMinioObject(ctx, mcm.bucketName, filePath, opts)
if err != nil {
log.Warn("failed to get object", zap.String("path", filePath), zap.Error(err))
return nil, err
@ -316,12 +321,13 @@ func (mcm *MinioChunkManager) ReadAt(ctx context.Context, filePath string, off i
log.Warn("failed to read object", zap.String("path", filePath), zap.Error(err))
return nil, err
}
metrics.PersistentDataKvSize.WithLabelValues(metrics.DataGetLabel).Observe(float64(length))
return data, nil
}
// Remove deletes an object with @key.
func (mcm *MinioChunkManager) Remove(ctx context.Context, filePath string) error {
err := mcm.Client.RemoveObject(ctx, mcm.bucketName, filePath, minio.RemoveObjectOptions{})
err := mcm.removeMinioObject(ctx, mcm.bucketName, filePath, minio.RemoveObjectOptions{})
if err != nil {
log.Warn("failed to remove object", zap.String("path", filePath), zap.Error(err))
return err
@ -346,8 +352,8 @@ func (mcm *MinioChunkManager) MultiRemove(ctx context.Context, keys []string) er
// RemoveWithPrefix removes all objects with the same prefix @prefix from minio.
func (mcm *MinioChunkManager) RemoveWithPrefix(ctx context.Context, prefix string) error {
objects := mcm.Client.ListObjects(ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: prefix, Recursive: true})
for rErr := range mcm.Client.RemoveObjects(ctx, mcm.bucketName, objects, minio.RemoveObjectsOptions{GovernanceBypass: false}) {
objects := mcm.listMinioObjects(ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: prefix, Recursive: true})
for rErr := range mcm.removeMinioObjects(ctx, mcm.bucketName, objects, minio.RemoveObjectsOptions{GovernanceBypass: false}) {
if rErr.Err != nil {
log.Warn("failed to remove objects", zap.String("prefix", prefix), zap.Error(rErr.Err))
return rErr.Err
@ -380,7 +386,7 @@ func (mcm *MinioChunkManager) ListWithPrefix(ctx context.Context, prefix string,
// TODO add concurrent call if performance matters
// only return current level per call
objects := mcm.Client.ListObjects(ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: pre, Recursive: false})
objects := mcm.listMinioObjects(ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: pre, Recursive: false})
for object := range objects {
if object.Err != nil {
@ -421,3 +427,97 @@ func Read(r io.Reader, size int64) ([]byte, error) {
}
}
}
func (mcm *MinioChunkManager) getMinioObject(ctx context.Context, bucketName, objectName string,
opts minio.GetObjectOptions) (*minio.Object, error) {
start := timerecord.NewTimeRecorder("getMinioObject")
reader, err := mcm.Client.GetObject(ctx, bucketName, objectName, opts)
elapsed := start.Elapse("getMinioObject done")
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.TotalLabel).Inc()
if err == nil && reader != nil {
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataGetLabel).Observe(float64(elapsed.Milliseconds()))
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.SuccessLabel).Inc()
} else {
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.FailLabel).Inc()
}
return reader, err
}
func (mcm *MinioChunkManager) putMinioObject(ctx context.Context, bucketName, objectName string, reader io.Reader, objectSize int64,
opts minio.PutObjectOptions) (minio.UploadInfo, error) {
start := timerecord.NewTimeRecorder("putMinioObject")
info, err := mcm.Client.PutObject(ctx, bucketName, objectName, reader, objectSize, opts)
elapsed := start.Elapse("putMinioObject done")
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataPutLabel, metrics.TotalLabel).Inc()
if err == nil {
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataPutLabel).Observe(float64(elapsed.Milliseconds()))
metrics.PersistentDataOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.SuccessLabel).Inc()
} else {
metrics.PersistentDataOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.FailLabel).Inc()
}
return info, err
}
func (mcm *MinioChunkManager) statMinioObject(ctx context.Context, bucketName, objectName string,
opts minio.StatObjectOptions) (minio.ObjectInfo, error) {
start := timerecord.NewTimeRecorder("statMinioObject")
info, err := mcm.Client.StatObject(ctx, bucketName, objectName, opts)
elapsed := start.Elapse("statMinioObject")
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.TotalLabel).Inc()
if err == nil {
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataStatLabel).Observe(float64(elapsed.Milliseconds()))
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.SuccessLabel).Inc()
} else {
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.FailLabel).Inc()
}
return info, err
}
func (mcm *MinioChunkManager) listMinioObjects(ctx context.Context, bucketName string,
opts minio.ListObjectsOptions) <-chan minio.ObjectInfo {
start := timerecord.NewTimeRecorder("listMinioObjects")
res := mcm.Client.ListObjects(ctx, bucketName, opts)
elapsed := start.Elapse("listMinioObjects done")
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataListLabel).Observe(float64(elapsed.Milliseconds()))
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataListLabel, metrics.TotalLabel).Inc()
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataListLabel, metrics.SuccessLabel).Inc()
return res
}
func (mcm *MinioChunkManager) removeMinioObjects(ctx context.Context, bucketName string, objectsCh <-chan minio.ObjectInfo,
opts minio.RemoveObjectsOptions) <-chan minio.RemoveObjectError {
start := timerecord.NewTimeRecorder("removeMinioObjects")
res := mcm.Client.RemoveObjects(ctx, bucketName, objectsCh, opts)
elapsed := start.Elapse("removeMinioObjects done")
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataRemoveLabel).Observe(float64(elapsed.Milliseconds()))
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.TotalLabel).Inc()
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.SuccessLabel).Inc()
return res
}
func (mcm *MinioChunkManager) removeMinioObject(ctx context.Context, bucketName, objectName string,
opts minio.RemoveObjectOptions) error {
start := timerecord.NewTimeRecorder("removeMinioObject")
err := mcm.Client.RemoveObject(ctx, bucketName, objectName, opts)
elapsed := start.Elapse("removeMinioObject done")
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.TotalLabel).Inc()
if err == nil {
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataRemoveLabel).Observe(float64(elapsed.Milliseconds()))
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.SuccessLabel).Inc()
} else {
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.FailLabel).Inc()
}
return err
}

View File

@ -76,6 +76,10 @@ func (bm *MockMsg) SetPosition(position *MsgPosition) {
}
func (bm *MockMsg) Size() int {
return 0
}
func Test_GenerateMsgStreamMsg(t *testing.T) {
messages := make([]msgstream.TsMsg, 1)
messages[0] = &MockMsg{

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package querynode
package typeutil
type TaskStep int32

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package querynode
package typeutil
import (
"testing"