mirror of https://github.com/milvus-io/milvus.git
parent
a1b10a80e4
commit
50739e6c6a
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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,
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()))
|
||||
|
|
|
@ -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},
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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},
|
||||
|
|
|
@ -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`.
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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?
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -119,6 +119,7 @@ func (s *scheduler) setTs(task task) error {
|
|||
}
|
||||
|
||||
func (s *scheduler) enqueue(task task) {
|
||||
task.OnEnqueue()
|
||||
s.taskChan <- task
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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
|
||||
|
|
@ -14,7 +14,7 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package querynode
|
||||
package typeutil
|
||||
|
||||
import (
|
||||
"testing"
|
Loading…
Reference in New Issue