mirror of https://github.com/milvus-io/milvus.git
parent
a5734be42b
commit
33c2012675
|
@ -52,8 +52,10 @@ var Registry *metrics.MilvusRegistry
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
Registry = metrics.NewMilvusRegistry()
|
Registry = metrics.NewMilvusRegistry()
|
||||||
metrics.RegisterEtcdMetrics(Registry.GoRegistry)
|
metrics.Register(Registry.GoRegistry)
|
||||||
metrics.RegisterMq(Registry.GoRegistry)
|
metrics.RegisterMetaMetrics(Registry.GoRegistry)
|
||||||
|
metrics.RegisterMsgStreamMetrics(Registry.GoRegistry)
|
||||||
|
metrics.RegisterStorageMetrics(Registry.GoRegistry)
|
||||||
}
|
}
|
||||||
|
|
||||||
func stopRocksmq() {
|
func stopRocksmq() {
|
||||||
|
|
|
@ -30,6 +30,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -165,6 +166,7 @@ func (c *ChannelStore) Reload() error {
|
||||||
c.channelsInfo[nodeID].Channels = append(c.channelsInfo[nodeID].Channels, channel)
|
c.channelsInfo[nodeID].Channels = append(c.channelsInfo[nodeID].Channels, channel)
|
||||||
log.Info("channel store reload channel",
|
log.Info("channel store reload channel",
|
||||||
zap.Int64("nodeID", nodeID), zap.String("channel", channel.Name))
|
zap.Int64("nodeID", nodeID), zap.String("channel", channel.Name))
|
||||||
|
metrics.DataCoordDmlChannelNum.WithLabelValues(strconv.FormatInt(nodeID, 10)).Set(float64(len(c.channelsInfo[nodeID].Channels)))
|
||||||
}
|
}
|
||||||
log.Info("channel store reload done", zap.Duration("duration", record.ElapseSpan()))
|
log.Info("channel store reload done", zap.Duration("duration", record.ElapseSpan()))
|
||||||
return nil
|
return nil
|
||||||
|
@ -274,6 +276,7 @@ func (c *ChannelStore) update(opSet ChannelOpSet) error {
|
||||||
default:
|
default:
|
||||||
return errUnknownOpType
|
return errUnknownOpType
|
||||||
}
|
}
|
||||||
|
metrics.DataCoordDmlChannelNum.WithLabelValues(strconv.FormatInt(op.NodeID, 10)).Set(float64(len(c.channelsInfo[op.NodeID].Channels)))
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
)
|
)
|
||||||
|
|
||||||
// TODO this num should be determined by resources of datanode, for now, we set to a fixed value for simple
|
// TODO this num should be determined by resources of datanode, for now, we set to a fixed value for simple
|
||||||
|
@ -250,6 +251,8 @@ func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResu
|
||||||
|
|
||||||
nodeID := c.plans[planID].dataNodeID
|
nodeID := c.plans[planID].dataNodeID
|
||||||
c.releaseQueue(nodeID)
|
c.releaseQueue(nodeID)
|
||||||
|
|
||||||
|
metrics.DataCoordCompactedSegmentSize.WithLabelValues().Observe(float64(getCompactedSegmentSize(result)))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -541,6 +541,7 @@ func (m *meta) FinishTask(taskInfo *indexpb.IndexTaskInfo) error {
|
||||||
log.Info("finish index task success", zap.Int64("buildID", taskInfo.BuildID),
|
log.Info("finish index task success", zap.Int64("buildID", taskInfo.BuildID),
|
||||||
zap.String("state", taskInfo.GetState().String()), zap.String("fail reason", taskInfo.GetFailReason()))
|
zap.String("state", taskInfo.GetState().String()), zap.String("fail reason", taskInfo.GetFailReason()))
|
||||||
m.updateIndexTasksMetrics()
|
m.updateIndexTasksMetrics()
|
||||||
|
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.IndexFileLabel).Observe(float64(len(taskInfo.IndexFileKeys)))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -118,6 +118,24 @@ func (m *meta) reloadFromKV() error {
|
||||||
metrics.DataCoordNumSegments.WithLabelValues(segment.State.String()).Inc()
|
metrics.DataCoordNumSegments.WithLabelValues(segment.State.String()).Inc()
|
||||||
if segment.State == commonpb.SegmentState_Flushed {
|
if segment.State == commonpb.SegmentState_Flushed {
|
||||||
numStoredRows += segment.NumOfRows
|
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))
|
metrics.DataCoordNumStoredRows.WithLabelValues().Set(float64(numStoredRows))
|
||||||
|
@ -149,6 +167,7 @@ func (m *meta) reloadFromKV() error {
|
||||||
}
|
}
|
||||||
for _, segIdx := range segmentIndexes {
|
for _, segIdx := range segmentIndexes {
|
||||||
m.updateSegmentIndex(segIdx)
|
m.updateSegmentIndex(segIdx)
|
||||||
|
metrics.FlushedSegmentFileNum.WithLabelValues(metrics.IndexFileLabel).Observe(float64(len(segIdx.IndexFileKeys)))
|
||||||
}
|
}
|
||||||
log.Info("DataCoord meta reloadFromKV done", zap.Duration("duration", record.ElapseSpan()))
|
log.Info("DataCoord meta reloadFromKV done", zap.Duration("duration", record.ElapseSpan()))
|
||||||
return nil
|
return nil
|
||||||
|
|
|
@ -253,11 +253,15 @@ func (s *Server) Register() error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.DataCoordRole).Inc()
|
||||||
|
log.Info("DataCoord Register Finished")
|
||||||
|
|
||||||
s.session.LivenessCheck(s.serverLoopCtx, func() {
|
s.session.LivenessCheck(s.serverLoopCtx, func() {
|
||||||
logutil.Logger(s.ctx).Error("disconnected from etcd and exited", zap.Int64("serverID", s.session.ServerID))
|
logutil.Logger(s.ctx).Error("disconnected from etcd and exited", zap.Int64("serverID", s.session.ServerID))
|
||||||
if err := s.Stop(); err != nil {
|
if err := s.Stop(); err != nil {
|
||||||
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
|
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.DataCoordRole).Dec()
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
if s.session.TriggerKill {
|
if s.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
|
@ -898,6 +902,25 @@ func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
s.buildIndexCh <- segmentID
|
s.buildIndexCh <- segmentID
|
||||||
|
|
||||||
|
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))
|
log.Info("flush segment complete", zap.Int64("id", segmentID))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,7 @@ import (
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/pkg/common"
|
"github.com/milvus-io/milvus/pkg/common"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
)
|
)
|
||||||
|
@ -161,6 +162,32 @@ func getCollectionTTL(properties map[string]string) (time.Duration, error) {
|
||||||
return Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second), nil
|
return Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second), 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
|
||||||
|
}
|
||||||
|
|
||||||
// getCollectionAutoCompactionEnabled returns whether auto compaction for collection is enabled.
|
// getCollectionAutoCompactionEnabled returns whether auto compaction for collection is enabled.
|
||||||
// if not set, returns global auto compaction config.
|
// if not set, returns global auto compaction config.
|
||||||
func getCollectionAutoCompactionEnabled(properties map[string]string) (bool, error) {
|
func getCollectionAutoCompactionEnabled(properties map[string]string) (bool, error) {
|
||||||
|
|
|
@ -851,7 +851,7 @@ func (c *ChannelMeta) setCurInsertBuffer(segmentID UniqueID, buf *BufferData) {
|
||||||
|
|
||||||
seg, ok := c.segments[segmentID]
|
seg, ok := c.segments[segmentID]
|
||||||
if ok {
|
if ok {
|
||||||
seg.curInsertBuf = buf
|
seg.setInsertBuffer(buf)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
log.Warn("cannot find segment when setCurInsertBuffer", zap.Int64("segmentID", segmentID))
|
log.Warn("cannot find segment when setCurInsertBuffer", zap.Int64("segmentID", segmentID))
|
||||||
|
|
|
@ -495,6 +495,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
return nil, errContext
|
return nil, errContext
|
||||||
}
|
}
|
||||||
|
|
||||||
|
durInQueue := t.tr.RecordSpan()
|
||||||
ctxTimeout, cancelAll := context.WithTimeout(t.ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
|
ctxTimeout, cancelAll := context.WithTimeout(t.ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
|
||||||
defer cancelAll()
|
defer cancelAll()
|
||||||
|
|
||||||
|
@ -755,6 +756,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))))
|
log.Info("overall elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(time.Since(compactStart))))
|
||||||
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
|
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds()))
|
||||||
|
|
||||||
return pack, nil
|
return pack, nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,6 +41,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/pkg/common"
|
"github.com/milvus-io/milvus/pkg/common"
|
||||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
var compactTestDir = "/tmp/milvus_test/compact"
|
var compactTestDir = "/tmp/milvus_test/compact"
|
||||||
|
@ -673,6 +674,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
done: make(chan struct{}, 1),
|
done: make(chan struct{}, 1),
|
||||||
Channel: &ChannelMeta{},
|
Channel: &ChannelMeta{},
|
||||||
|
tr: timerecord.NewTimeRecorder("test"),
|
||||||
}
|
}
|
||||||
|
|
||||||
plan := &datapb.CompactionPlan{
|
plan := &datapb.CompactionPlan{
|
||||||
|
|
|
@ -49,6 +49,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||||
"github.com/milvus-io/milvus/pkg/util/logutil"
|
"github.com/milvus-io/milvus/pkg/util/logutil"
|
||||||
|
@ -62,12 +63,6 @@ const (
|
||||||
// RPCConnectionTimeout is used to set the timeout for rpc request
|
// RPCConnectionTimeout is used to set the timeout for rpc request
|
||||||
RPCConnectionTimeout = 30 * time.Second
|
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 is used to limit the max retry time for connection etcd
|
||||||
ConnectEtcdMaxRetryTime = 100
|
ConnectEtcdMaxRetryTime = 100
|
||||||
|
|
||||||
|
@ -193,12 +188,15 @@ func (node *DataNode) SetDataCoord(ds types.DataCoord) error {
|
||||||
func (node *DataNode) Register() error {
|
func (node *DataNode) Register() error {
|
||||||
node.session.Register()
|
node.session.Register()
|
||||||
|
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.DataNodeRole).Inc()
|
||||||
|
log.Info("DataNode Register Finished")
|
||||||
// Start liveness check
|
// Start liveness check
|
||||||
node.session.LivenessCheck(node.ctx, func() {
|
node.session.LivenessCheck(node.ctx, func() {
|
||||||
log.Error("Data Node disconnected from etcd, process will exit", zap.Int64("Server Id", node.GetSession().ServerID))
|
log.Error("Data Node disconnected from etcd, process will exit", zap.Int64("Server Id", node.GetSession().ServerID))
|
||||||
if err := node.Stop(); err != nil {
|
if err := node.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.DataNodeRole).Dec()
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
if node.session.TriggerKill {
|
if node.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
|
|
|
@ -519,7 +519,7 @@ func (ibNode *insertBufferNode) Sync(fgMsg *flowGraphMsg, seg2Upload []UniqueID,
|
||||||
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.TotalLabel).Inc()
|
metrics.DataNodeFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.TotalLabel).Inc()
|
||||||
if task.auto {
|
if task.auto {
|
||||||
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.TotalLabel).Inc()
|
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.TotalLabel).Inc()
|
||||||
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.FailLabel).Inc()
|
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.SuccessLabel).Inc()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return segmentsToSync
|
return segmentsToSync
|
||||||
|
|
|
@ -17,6 +17,8 @@
|
||||||
package datanode
|
package datanode
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"fmt"
|
||||||
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
|
|
||||||
|
@ -29,6 +31,8 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Segment contains the latest segment infos from channel.
|
// Segment contains the latest segment infos from channel.
|
||||||
|
@ -176,11 +180,35 @@ func (s *Segment) isPKExist(pk primaryKey) bool {
|
||||||
return false
|
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(fmt.Sprint(paramtable.GetNodeID()),
|
||||||
|
strconv.FormatInt(s.collectionID, 10)).Add(float64(dataSize))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// rollInsertBuffer moves curInsertBuf to historyInsertBuf, and then sets curInsertBuf to nil.
|
// rollInsertBuffer moves curInsertBuf to historyInsertBuf, and then sets curInsertBuf to nil.
|
||||||
func (s *Segment) rollInsertBuffer() {
|
func (s *Segment) rollInsertBuffer() {
|
||||||
if s.curInsertBuf == nil {
|
if s.curInsertBuf == nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if s.curInsertBuf.buffer != nil {
|
||||||
|
dataSize := 0
|
||||||
|
for _, data := range s.curInsertBuf.buffer.Data {
|
||||||
|
dataSize += data.GetMemorySize()
|
||||||
|
}
|
||||||
|
metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()),
|
||||||
|
strconv.FormatInt(s.collectionID, 10)).Sub(float64(dataSize))
|
||||||
|
}
|
||||||
|
|
||||||
s.curInsertBuf.buffer = nil // free buffer memory, only keep meta infos in historyInsertBuf
|
s.curInsertBuf.buffer = nil // free buffer memory, only keep meta infos in historyInsertBuf
|
||||||
s.historyInsertBuf = append(s.historyInsertBuf, s.curInsertBuf)
|
s.historyInsertBuf = append(s.historyInsertBuf, s.curInsertBuf)
|
||||||
s.curInsertBuf = nil
|
s.curInsertBuf = nil
|
||||||
|
|
|
@ -91,7 +91,7 @@ func (node *DataNode) GetComponentStates(ctx context.Context) (*milvuspb.Compone
|
||||||
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
|
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
|
||||||
metrics.DataNodeFlushReqCounter.WithLabelValues(
|
metrics.DataNodeFlushReqCounter.WithLabelValues(
|
||||||
fmt.Sprint(paramtable.GetNodeID()),
|
fmt.Sprint(paramtable.GetNodeID()),
|
||||||
MetricRequestsTotal).Inc()
|
metrics.TotalLabel).Inc()
|
||||||
|
|
||||||
if !node.isHealthy() {
|
if !node.isHealthy() {
|
||||||
err := merr.WrapErrServiceNotReady(node.GetStateCode().String())
|
err := merr.WrapErrServiceNotReady(node.GetStateCode().String())
|
||||||
|
@ -158,7 +158,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
||||||
|
|
||||||
metrics.DataNodeFlushReqCounter.WithLabelValues(
|
metrics.DataNodeFlushReqCounter.WithLabelValues(
|
||||||
fmt.Sprint(paramtable.GetNodeID()),
|
fmt.Sprint(paramtable.GetNodeID()),
|
||||||
MetricRequestsSuccess).Inc()
|
metrics.SuccessLabel).Inc()
|
||||||
return merr.Status(nil), nil
|
return merr.Status(nil), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,6 +28,7 @@ import "C"
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"fmt"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"os"
|
"os"
|
||||||
"path"
|
"path"
|
||||||
|
@ -50,6 +51,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/util/initcore"
|
"github.com/milvus-io/milvus/internal/util/initcore"
|
||||||
"github.com/milvus-io/milvus/pkg/common"
|
"github.com/milvus-io/milvus/pkg/common"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||||
"github.com/milvus-io/milvus/pkg/util/hardware"
|
"github.com/milvus-io/milvus/pkg/util/hardware"
|
||||||
"github.com/milvus-io/milvus/pkg/util/lifetime"
|
"github.com/milvus-io/milvus/pkg/util/lifetime"
|
||||||
|
@ -122,12 +124,14 @@ func NewIndexNode(ctx context.Context, factory dependency.Factory) *IndexNode {
|
||||||
func (i *IndexNode) Register() error {
|
func (i *IndexNode) Register() error {
|
||||||
i.session.Register()
|
i.session.Register()
|
||||||
|
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.IndexNodeRole).Inc()
|
||||||
//start liveness check
|
//start liveness check
|
||||||
i.session.LivenessCheck(i.loopCtx, func() {
|
i.session.LivenessCheck(i.loopCtx, func() {
|
||||||
log.Error("Index Node disconnected from etcd, process will exit", zap.Int64("Server Id", i.session.ServerID))
|
log.Error("Index Node disconnected from etcd, process will exit", zap.Int64("Server Id", i.session.ServerID))
|
||||||
if err := i.Stop(); err != nil {
|
if err := i.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.IndexNodeRole).Dec()
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
if i.session.TriggerKill {
|
if i.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
|
|
|
@ -73,6 +73,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
|
||||||
cancel: taskCancel,
|
cancel: taskCancel,
|
||||||
state: commonpb.IndexState_InProgress}); oldInfo != nil {
|
state: commonpb.IndexState_InProgress}); oldInfo != nil {
|
||||||
log.Ctx(ctx).Warn("duplicated index build task", zap.String("ClusterID", req.ClusterID), zap.Int64("BuildID", req.BuildID))
|
log.Ctx(ctx).Warn("duplicated index build task", zap.String("ClusterID", req.ClusterID), zap.Int64("BuildID", req.BuildID))
|
||||||
|
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.FailLabel).Inc()
|
||||||
return &commonpb.Status{
|
return &commonpb.Status{
|
||||||
ErrorCode: commonpb.ErrorCode_BuildIndexError,
|
ErrorCode: commonpb.ErrorCode_BuildIndexError,
|
||||||
Reason: "duplicated index build task",
|
Reason: "duplicated index build task",
|
||||||
|
@ -83,6 +84,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),
|
log.Ctx(ctx).Error("create chunk manager failed", zap.String("Bucket", req.StorageConfig.BucketName),
|
||||||
zap.String("AccessKey", req.StorageConfig.AccessKeyID),
|
zap.String("AccessKey", req.StorageConfig.AccessKeyID),
|
||||||
zap.String("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
|
zap.String("ClusterID", req.ClusterID), zap.Int64("IndexBuildID", req.BuildID))
|
||||||
|
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.FailLabel).Inc()
|
||||||
return &commonpb.Status{
|
return &commonpb.Status{
|
||||||
ErrorCode: commonpb.ErrorCode_BuildIndexError,
|
ErrorCode: commonpb.ErrorCode_BuildIndexError,
|
||||||
Reason: "create chunk manager failed",
|
Reason: "create chunk manager failed",
|
||||||
|
@ -112,6 +114,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
|
||||||
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.FailLabel).Inc()
|
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.FailLabel).Inc()
|
||||||
return ret, nil
|
return ret, nil
|
||||||
}
|
}
|
||||||
|
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), 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))
|
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
|
return ret, nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -99,6 +99,7 @@ type indexBuildTask struct {
|
||||||
newIndexParams map[string]string
|
newIndexParams map[string]string
|
||||||
serializedSize uint64
|
serializedSize uint64
|
||||||
tr *timerecord.TimeRecorder
|
tr *timerecord.TimeRecorder
|
||||||
|
queueDur time.Duration
|
||||||
statistic indexpb.JobInfo
|
statistic indexpb.JobInfo
|
||||||
node *IndexNode
|
node *IndexNode
|
||||||
}
|
}
|
||||||
|
@ -139,6 +140,8 @@ func (it *indexBuildTask) GetState() commonpb.IndexState {
|
||||||
|
|
||||||
// OnEnqueue enqueues indexing tasks.
|
// OnEnqueue enqueues indexing tasks.
|
||||||
func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
|
func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
|
||||||
|
it.queueDur = 0
|
||||||
|
it.tr.RecordSpan()
|
||||||
it.statistic.StartTime = time.Now().UnixMicro()
|
it.statistic.StartTime = time.Now().UnixMicro()
|
||||||
it.statistic.PodID = it.node.GetNodeID()
|
it.statistic.PodID = it.node.GetNodeID()
|
||||||
log.Ctx(ctx).Info("IndexNode IndexBuilderTask Enqueue", zap.Int64("buildID", it.BuildID), zap.Int64("segID", it.segmentID))
|
log.Ctx(ctx).Info("IndexNode IndexBuilderTask Enqueue", zap.Int64("buildID", it.BuildID), zap.Int64("segID", it.segmentID))
|
||||||
|
@ -146,6 +149,7 @@ func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *indexBuildTask) Prepare(ctx context.Context) error {
|
func (it *indexBuildTask) Prepare(ctx context.Context) error {
|
||||||
|
it.queueDur = it.tr.RecordSpan()
|
||||||
log.Ctx(ctx).Info("Begin to prepare indexBuildTask", zap.Int64("buildID", it.BuildID),
|
log.Ctx(ctx).Info("Begin to prepare indexBuildTask", zap.Int64("buildID", it.BuildID),
|
||||||
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
|
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
|
||||||
typeParams := make(map[string]string)
|
typeParams := make(map[string]string)
|
||||||
|
|
|
@ -19,6 +19,7 @@ package indexnode
|
||||||
import (
|
import (
|
||||||
"container/list"
|
"container/list"
|
||||||
"context"
|
"context"
|
||||||
|
"fmt"
|
||||||
"runtime/debug"
|
"runtime/debug"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
|
@ -28,6 +29,8 @@ import (
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||||
)
|
)
|
||||||
|
|
||||||
// TaskQueue is a queue used to store tasks.
|
// TaskQueue is a queue used to store tasks.
|
||||||
|
@ -229,6 +232,10 @@ func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
t.SetState(commonpb.IndexState_Finished, "")
|
t.SetState(commonpb.IndexState_Finished, "")
|
||||||
|
if indexBuildTask, ok := t.(*indexBuildTask); ok {
|
||||||
|
metrics.IndexNodeBuildIndexLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(indexBuildTask.tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.IndexNodeIndexTaskLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(indexBuildTask.queueDur.Milliseconds()))
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (sched *TaskScheduler) indexBuildLoop() {
|
func (sched *TaskScheduler) indexBuildLoop() {
|
||||||
|
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/pkg/common"
|
"github.com/milvus-io/milvus/pkg/common"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
"github.com/milvus-io/milvus/pkg/metrics"
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
@ -76,7 +77,7 @@ func (kv *etcdKV) WalkWithPrefix(prefix string, paginationSize int, fn func([]by
|
||||||
|
|
||||||
key := prefix
|
key := prefix
|
||||||
for {
|
for {
|
||||||
resp, err := kv.client.Get(ctx, key, opts...)
|
resp, err := kv.getEtcdMeta(ctx, key, opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -104,7 +105,7 @@ func (kv *etcdKV) LoadWithPrefix(key string) ([]string, []string, error) {
|
||||||
key = path.Join(kv.rootPath, key)
|
key = path.Join(kv.rootPath, key)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
|
resp, err := kv.getEtcdMeta(ctx, key, clientv3.WithPrefix(),
|
||||||
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
|
@ -125,7 +126,7 @@ func (kv *etcdKV) LoadBytesWithPrefix(key string) ([]string, [][]byte, error) {
|
||||||
key = path.Join(kv.rootPath, key)
|
key = path.Join(kv.rootPath, key)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
|
resp, err := kv.getEtcdMeta(ctx, key, clientv3.WithPrefix(),
|
||||||
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, err
|
||||||
|
@ -146,7 +147,7 @@ func (kv *etcdKV) LoadBytesWithPrefix2(key string) ([]string, [][]byte, []int64,
|
||||||
key = path.Join(kv.rootPath, key)
|
key = path.Join(kv.rootPath, key)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
|
resp, err := kv.getEtcdMeta(ctx, key, clientv3.WithPrefix(),
|
||||||
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, nil, err
|
return nil, nil, nil, err
|
||||||
|
@ -169,7 +170,7 @@ func (kv *etcdKV) Load(key string) (string, error) {
|
||||||
key = path.Join(kv.rootPath, key)
|
key = path.Join(kv.rootPath, key)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Get(ctx, key)
|
resp, err := kv.getEtcdMeta(ctx, key)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return "", err
|
return "", err
|
||||||
}
|
}
|
||||||
|
@ -186,7 +187,7 @@ func (kv *etcdKV) LoadBytes(key string) ([]byte, error) {
|
||||||
key = path.Join(kv.rootPath, key)
|
key = path.Join(kv.rootPath, key)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Get(ctx, key)
|
resp, err := kv.getEtcdMeta(ctx, key)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return []byte{}, err
|
return []byte{}, err
|
||||||
}
|
}
|
||||||
|
@ -207,7 +208,7 @@ func (kv *etcdKV) MultiLoad(keys []string) ([]string, error) {
|
||||||
|
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return []string{}, err
|
return []string{}, err
|
||||||
}
|
}
|
||||||
|
@ -242,7 +243,7 @@ func (kv *etcdKV) MultiLoadBytes(keys []string) ([][]byte, error) {
|
||||||
|
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return [][]byte{}, err
|
return [][]byte{}, err
|
||||||
}
|
}
|
||||||
|
@ -273,7 +274,7 @@ func (kv *etcdKV) LoadBytesWithRevision(key string) ([]string, [][]byte, int64,
|
||||||
key = path.Join(kv.rootPath, key)
|
key = path.Join(kv.rootPath, key)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(),
|
resp, err := kv.getEtcdMeta(ctx, key, clientv3.WithPrefix(),
|
||||||
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, 0, err
|
return nil, nil, 0, err
|
||||||
|
@ -295,7 +296,7 @@ func (kv *etcdKV) Save(key, value string) error {
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
CheckValueSizeAndWarn(key, value)
|
CheckValueSizeAndWarn(key, value)
|
||||||
_, err := kv.client.Put(ctx, key, value)
|
_, err := kv.putEtcdMeta(ctx, key, value)
|
||||||
CheckElapseAndWarn(start, "Slow etcd operation save", zap.String("key", key))
|
CheckElapseAndWarn(start, "Slow etcd operation save", zap.String("key", key))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -307,7 +308,7 @@ func (kv *etcdKV) SaveBytes(key string, value []byte) error {
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
CheckValueSizeAndWarn(key, value)
|
CheckValueSizeAndWarn(key, value)
|
||||||
_, err := kv.client.Put(ctx, key, string(value))
|
_, err := kv.putEtcdMeta(ctx, key, string(value))
|
||||||
CheckElapseAndWarn(start, "Slow etcd operation save", zap.String("key", key))
|
CheckElapseAndWarn(start, "Slow etcd operation save", zap.String("key", key))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -319,7 +320,7 @@ func (kv *etcdKV) SaveBytesWithLease(key string, value []byte, id clientv3.Lease
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
CheckValueSizeAndWarn(key, value)
|
CheckValueSizeAndWarn(key, value)
|
||||||
_, err := kv.client.Put(ctx, key, string(value), clientv3.WithLease(id))
|
_, err := kv.putEtcdMeta(ctx, key, string(value), clientv3.WithLease(id))
|
||||||
CheckElapseAndWarn(start, "Slow etcd operation save with lease", zap.String("key", key))
|
CheckElapseAndWarn(start, "Slow etcd operation save with lease", zap.String("key", key))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -338,7 +339,7 @@ func (kv *etcdKV) MultiSave(kvs map[string]string) error {
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
CheckTnxStringValueSizeAndWarn(kvs)
|
CheckTnxStringValueSizeAndWarn(kvs)
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiSave error", zap.Any("kvs", kvs), zap.Int("len", len(kvs)), zap.Error(err))
|
log.Warn("Etcd MultiSave error", zap.Any("kvs", kvs), zap.Int("len", len(kvs)), zap.Error(err))
|
||||||
}
|
}
|
||||||
|
@ -360,7 +361,7 @@ func (kv *etcdKV) MultiSaveBytes(kvs map[string][]byte) error {
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
CheckTnxBytesValueSizeAndWarn(kvs)
|
CheckTnxBytesValueSizeAndWarn(kvs)
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiSaveBytes err", zap.Any("kvs", kvs), zap.Int("len", len(kvs)), zap.Error(err))
|
log.Warn("Etcd MultiSaveBytes err", zap.Any("kvs", kvs), zap.Int("len", len(kvs)), zap.Error(err))
|
||||||
}
|
}
|
||||||
|
@ -375,7 +376,7 @@ func (kv *etcdKV) RemoveWithPrefix(prefix string) error {
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Delete(ctx, key, clientv3.WithPrefix())
|
_, err := kv.removeEtcdMeta(ctx, key, clientv3.WithPrefix())
|
||||||
CheckElapseAndWarn(start, "Slow etcd operation remove with prefix", zap.String("prefix", prefix))
|
CheckElapseAndWarn(start, "Slow etcd operation remove with prefix", zap.String("prefix", prefix))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -387,7 +388,7 @@ func (kv *etcdKV) Remove(key string) error {
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Delete(ctx, key)
|
_, err := kv.removeEtcdMeta(ctx, key)
|
||||||
CheckElapseAndWarn(start, "Slow etcd operation remove", zap.String("key", key))
|
CheckElapseAndWarn(start, "Slow etcd operation remove", zap.String("key", key))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -403,7 +404,7 @@ func (kv *etcdKV) MultiRemove(keys []string) error {
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiRemove error", zap.Strings("keys", keys), zap.Int("len", len(keys)), zap.Error(err))
|
log.Warn("Etcd MultiRemove error", zap.Strings("keys", keys), zap.Int("len", len(keys)), zap.Error(err))
|
||||||
}
|
}
|
||||||
|
@ -428,7 +429,7 @@ func (kv *etcdKV) MultiSaveAndRemove(saves map[string]string, removals []string)
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiSaveAndRemove error",
|
log.Warn("Etcd MultiSaveAndRemove error",
|
||||||
zap.Any("saves", saves),
|
zap.Any("saves", saves),
|
||||||
|
@ -458,7 +459,7 @@ func (kv *etcdKV) MultiSaveBytesAndRemove(saves map[string][]byte, removals []st
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiSaveBytesAndRemove error",
|
log.Warn("Etcd MultiSaveBytesAndRemove error",
|
||||||
zap.Any("saves", saves),
|
zap.Any("saves", saves),
|
||||||
|
@ -509,7 +510,7 @@ func (kv *etcdKV) MultiRemoveWithPrefix(keys []string) error {
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiRemoveWithPrefix error", zap.Strings("keys", keys), zap.Int("len", len(keys)), zap.Error(err))
|
log.Warn("Etcd MultiRemoveWithPrefix error", zap.Strings("keys", keys), zap.Int("len", len(keys)), zap.Error(err))
|
||||||
}
|
}
|
||||||
|
@ -534,7 +535,7 @@ func (kv *etcdKV) MultiSaveAndRemoveWithPrefix(saves map[string]string, removals
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiSaveAndRemoveWithPrefix error",
|
log.Warn("Etcd MultiSaveAndRemoveWithPrefix error",
|
||||||
zap.Any("saves", saves),
|
zap.Any("saves", saves),
|
||||||
|
@ -564,7 +565,7 @@ func (kv *etcdKV) MultiSaveBytesAndRemoveWithPrefix(saves map[string][]byte, rem
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
|
_, err := kv.executeTxn(kv.getTxnWithCmp(ctx), ops...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("Etcd MultiSaveBytesAndRemoveWithPrefix error",
|
log.Warn("Etcd MultiSaveBytesAndRemoveWithPrefix error",
|
||||||
zap.Any("saves", saves),
|
zap.Any("saves", saves),
|
||||||
|
@ -583,12 +584,9 @@ func (kv *etcdKV) CompareVersionAndSwap(key string, source int64, target string)
|
||||||
start := time.Now()
|
start := time.Now()
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Txn(ctx).If(
|
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx,
|
||||||
clientv3.Compare(
|
clientv3.Compare(clientv3.Version(path.Join(kv.rootPath, key)), "=", source)),
|
||||||
clientv3.Version(path.Join(kv.rootPath, key)),
|
clientv3.OpPut(path.Join(kv.rootPath, key), target))
|
||||||
"=",
|
|
||||||
source)).
|
|
||||||
Then(clientv3.OpPut(path.Join(kv.rootPath, key), target)).Commit()
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return false, err
|
||||||
}
|
}
|
||||||
|
@ -602,12 +600,9 @@ func (kv *etcdKV) CompareVersionAndSwapBytes(key string, source int64, target []
|
||||||
start := time.Now()
|
start := time.Now()
|
||||||
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
ctx, cancel := context.WithTimeout(context.TODO(), RequestTimeout)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
resp, err := kv.client.Txn(ctx).If(
|
resp, err := kv.executeTxn(kv.getTxnWithCmp(ctx,
|
||||||
clientv3.Compare(
|
clientv3.Compare(clientv3.Version(path.Join(kv.rootPath, key)), "=", source)),
|
||||||
clientv3.Version(path.Join(kv.rootPath, key)),
|
clientv3.OpPut(path.Join(kv.rootPath, key), string(target), opts...))
|
||||||
"=",
|
|
||||||
source)).
|
|
||||||
Then(clientv3.OpPut(path.Join(kv.rootPath, key), string(target), opts...)).Commit()
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, err
|
return false, err
|
||||||
}
|
}
|
||||||
|
@ -618,7 +613,6 @@ func (kv *etcdKV) CompareVersionAndSwapBytes(key string, source int64, target []
|
||||||
// CheckElapseAndWarn checks the elapsed time and warns if it is too long.
|
// CheckElapseAndWarn checks the elapsed time and warns if it is too long.
|
||||||
func CheckElapseAndWarn(start time.Time, message string, fields ...zap.Field) bool {
|
func CheckElapseAndWarn(start time.Time, message string, fields ...zap.Field) bool {
|
||||||
elapsed := time.Since(start)
|
elapsed := time.Since(start)
|
||||||
metrics.EtcdRequestLatency.Observe(float64(elapsed))
|
|
||||||
if elapsed.Milliseconds() > 2000 {
|
if elapsed.Milliseconds() > 2000 {
|
||||||
log.Warn(message, append([]zap.Field{zap.String("time spent", elapsed.String())}, fields...)...)
|
log.Warn(message, append([]zap.Field{zap.String("time spent", elapsed.String())}, fields...)...)
|
||||||
return true
|
return true
|
||||||
|
@ -628,7 +622,6 @@ func CheckElapseAndWarn(start time.Time, message string, fields ...zap.Field) bo
|
||||||
|
|
||||||
func CheckValueSizeAndWarn(key string, value interface{}) bool {
|
func CheckValueSizeAndWarn(key string, value interface{}) bool {
|
||||||
size := binary.Size(value)
|
size := binary.Size(value)
|
||||||
metrics.EtcdPutKvSize.Observe(float64(size))
|
|
||||||
if size > 102400 {
|
if size > 102400 {
|
||||||
log.Warn("value size large than 100kb", zap.String("key", key), zap.Int("value_size(kb)", size/1024))
|
log.Warn("value size large than 100kb", zap.String("key", key), zap.Int("value_size(kb)", size/1024))
|
||||||
return true
|
return true
|
||||||
|
@ -654,3 +647,105 @@ func CheckTnxStringValueSizeAndWarn(kvs map[string]string) bool {
|
||||||
|
|
||||||
return CheckTnxBytesValueSizeAndWarn(newKvs)
|
return CheckTnxBytesValueSizeAndWarn(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.ElapseSpan()
|
||||||
|
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.ElapseSpan()
|
||||||
|
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.ElapseSpan()
|
||||||
|
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.ElapseSpan()
|
||||||
|
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
|
||||||
|
}
|
||||||
|
|
|
@ -25,7 +25,9 @@ import (
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
// nmqClient implements mqwrapper.Client.
|
// nmqClient implements mqwrapper.Client.
|
||||||
|
@ -53,18 +55,30 @@ func NewClient(opts client.Options) (*rmqClient, error) {
|
||||||
|
|
||||||
// CreateProducer creates a producer for rocksmq client
|
// CreateProducer creates a producer for rocksmq client
|
||||||
func (rc *rmqClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
|
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}
|
rmqOpts := client.ProducerOptions{Topic: options.Topic}
|
||||||
pp, err := rc.client.CreateProducer(rmqOpts)
|
pp, err := rc.client.CreateProducer(rmqOpts)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
rp := rmqProducer{p: pp}
|
rp := rmqProducer{p: pp}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
|
||||||
return &rp, nil
|
return &rp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Subscribe subscribes a consumer in rmq client
|
// Subscribe subscribes a consumer in rmq client
|
||||||
func (rc *rmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
|
func (rc *rmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
|
||||||
|
start := timerecord.NewTimeRecorder("create consumer")
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc()
|
||||||
|
|
||||||
if options.BufSize == 0 {
|
if options.BufSize == 0 {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
err := errors.New("subscription bufSize of rmq should never be zero")
|
err := errors.New("subscription bufSize of rmq should never be zero")
|
||||||
log.Warn("unexpected subscription consumer options", zap.Error(err))
|
log.Warn("unexpected subscription consumer options", zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -78,11 +92,15 @@ func (rc *rmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Con
|
||||||
SubscriptionInitialPosition: options.SubscriptionInitialPosition,
|
SubscriptionInitialPosition: options.SubscriptionInitialPosition,
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
rConsumer := &Consumer{c: cli, closeCh: make(chan struct{})}
|
rConsumer := &Consumer{c: cli, closeCh: make(chan struct{})}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
|
||||||
return rConsumer, nil
|
return rConsumer, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -15,7 +15,9 @@ import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/client"
|
"github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/client"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
var _ mqwrapper.Producer = (*rmqProducer)(nil)
|
var _ mqwrapper.Producer = (*rmqProducer)(nil)
|
||||||
|
@ -32,9 +34,20 @@ func (rp *rmqProducer) Topic() string {
|
||||||
|
|
||||||
// Send send the producer messages to rocksmq
|
// Send send the producer messages to rocksmq
|
||||||
func (rp *rmqProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
|
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, Properties: message.Properties}
|
pm := &client.ProducerMessage{Payload: message.Payload, Properties: message.Properties}
|
||||||
id, err := rp.p.Send(pm)
|
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.ElapseSpan()
|
||||||
|
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
|
// Close does nothing currently
|
||||||
|
|
|
@ -20,7 +20,6 @@ import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"strconv"
|
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
@ -230,8 +229,9 @@ func (m *MetaCache) GetCollectionID(ctx context.Context, database, collectionNam
|
||||||
collInfo, ok = db[collectionName]
|
collInfo, ok = db[collectionName]
|
||||||
}
|
}
|
||||||
|
|
||||||
|
method := "GeCollectionID"
|
||||||
if !ok || !collInfo.isCollectionCached() {
|
if !ok || !collInfo.isCollectionCached() {
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GeCollectionID", metrics.CacheMissLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
tr := timerecord.NewTimeRecorder("UpdateCache")
|
tr := timerecord.NewTimeRecorder("UpdateCache")
|
||||||
m.mu.RUnlock()
|
m.mu.RUnlock()
|
||||||
coll, err := m.describeCollection(ctx, database, collectionName, 0)
|
coll, err := m.describeCollection(ctx, database, collectionName, 0)
|
||||||
|
@ -240,13 +240,14 @@ func (m *MetaCache) GetCollectionID(ctx context.Context, database, collectionNam
|
||||||
}
|
}
|
||||||
m.mu.Lock()
|
m.mu.Lock()
|
||||||
defer m.mu.Unlock()
|
defer m.mu.Unlock()
|
||||||
|
|
||||||
m.updateCollection(coll, database, collectionName)
|
m.updateCollection(coll, database, collectionName)
|
||||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
collInfo = m.collInfo[database][collectionName]
|
collInfo = m.collInfo[database][collectionName]
|
||||||
return collInfo.collID, nil
|
return collInfo.collID, nil
|
||||||
}
|
}
|
||||||
defer m.mu.RUnlock()
|
defer m.mu.RUnlock()
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetCollectionID", metrics.CacheHitLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
|
|
||||||
return collInfo.collID, nil
|
return collInfo.collID, nil
|
||||||
}
|
}
|
||||||
|
@ -264,8 +265,9 @@ func (m *MetaCache) GetDatabaseAndCollectionName(ctx context.Context, collection
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
method := "GeCollectionName"
|
||||||
if collInfo == nil || !collInfo.isCollectionCached() {
|
if collInfo == nil || !collInfo.isCollectionCached() {
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GeCollectionName", metrics.CacheMissLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
tr := timerecord.NewTimeRecorder("UpdateCache")
|
tr := timerecord.NewTimeRecorder("UpdateCache")
|
||||||
m.mu.RUnlock()
|
m.mu.RUnlock()
|
||||||
coll, err := m.describeCollection(ctx, "", "", collectionID)
|
coll, err := m.describeCollection(ctx, "", "", collectionID)
|
||||||
|
@ -276,11 +278,11 @@ func (m *MetaCache) GetDatabaseAndCollectionName(ctx context.Context, collection
|
||||||
defer m.mu.Unlock()
|
defer m.mu.Unlock()
|
||||||
|
|
||||||
m.updateCollection(coll, coll.GetDbName(), coll.Schema.Name)
|
m.updateCollection(coll, coll.GetDbName(), coll.Schema.Name)
|
||||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
return coll.GetDbName(), coll.Schema.Name, nil
|
return coll.GetDbName(), coll.Schema.Name, nil
|
||||||
}
|
}
|
||||||
defer m.mu.RUnlock()
|
defer m.mu.RUnlock()
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GeCollectionName", metrics.CacheHitLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
|
|
||||||
return collInfo.database, collInfo.schema.Name, nil
|
return collInfo.database, collInfo.schema.Name, nil
|
||||||
}
|
}
|
||||||
|
@ -298,9 +300,10 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, database, collectionN
|
||||||
}
|
}
|
||||||
m.mu.RUnlock()
|
m.mu.RUnlock()
|
||||||
|
|
||||||
|
method := "GetCollectionInfo"
|
||||||
if !ok || !collInfo.isCollectionCached() {
|
if !ok || !collInfo.isCollectionCached() {
|
||||||
tr := timerecord.NewTimeRecorder("UpdateCache")
|
tr := timerecord.NewTimeRecorder("UpdateCache")
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetCollectionInfo", metrics.CacheMissLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
coll, err := m.describeCollection(ctx, database, collectionName, 0)
|
coll, err := m.describeCollection(ctx, database, collectionName, 0)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -309,10 +312,10 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, database, collectionN
|
||||||
m.updateCollection(coll, database, collectionName)
|
m.updateCollection(coll, database, collectionName)
|
||||||
collInfo = m.collInfo[database][collectionName]
|
collInfo = m.collInfo[database][collectionName]
|
||||||
m.mu.Unlock()
|
m.mu.Unlock()
|
||||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
}
|
}
|
||||||
|
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetCollectionInfo", metrics.CacheHitLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
return collInfo, nil
|
return collInfo, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -326,8 +329,9 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, database, collectio
|
||||||
collInfo, ok = db[collectionName]
|
collInfo, ok = db[collectionName]
|
||||||
}
|
}
|
||||||
|
|
||||||
|
method := "GetCollectionSchema"
|
||||||
if !ok || !collInfo.isCollectionCached() {
|
if !ok || !collInfo.isCollectionCached() {
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetCollectionSchema", metrics.CacheMissLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
tr := timerecord.NewTimeRecorder("UpdateCache")
|
tr := timerecord.NewTimeRecorder("UpdateCache")
|
||||||
m.mu.RUnlock()
|
m.mu.RUnlock()
|
||||||
coll, err := m.describeCollection(ctx, database, collectionName, 0)
|
coll, err := m.describeCollection(ctx, database, collectionName, 0)
|
||||||
|
@ -339,16 +343,17 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, database, collectio
|
||||||
}
|
}
|
||||||
m.mu.Lock()
|
m.mu.Lock()
|
||||||
defer m.mu.Unlock()
|
defer m.mu.Unlock()
|
||||||
|
|
||||||
m.updateCollection(coll, database, collectionName)
|
m.updateCollection(coll, database, collectionName)
|
||||||
collInfo = m.collInfo[database][collectionName]
|
collInfo = m.collInfo[database][collectionName]
|
||||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
log.Debug("Reload collection from root coordinator ",
|
log.Debug("Reload collection from root coordinator ",
|
||||||
zap.String("collection name", collectionName),
|
zap.String("collection name", collectionName),
|
||||||
zap.Any("time (milliseconds) take ", tr.ElapseSpan().Milliseconds()))
|
zap.Any("time (milliseconds) take ", tr.ElapseSpan().Milliseconds()))
|
||||||
return collInfo.schema, nil
|
return collInfo.schema, nil
|
||||||
}
|
}
|
||||||
defer m.mu.RUnlock()
|
defer m.mu.RUnlock()
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetCollectionSchema", metrics.CacheHitLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
|
|
||||||
return collInfo.schema, nil
|
return collInfo.schema, nil
|
||||||
}
|
}
|
||||||
|
@ -383,6 +388,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, database, collectionName
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
method := "GetPartitions"
|
||||||
m.mu.RLock()
|
m.mu.RLock()
|
||||||
|
|
||||||
var collInfo *collectionInfo
|
var collInfo *collectionInfo
|
||||||
|
@ -399,7 +405,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, database, collectionName
|
||||||
|
|
||||||
if collInfo.partInfo == nil || len(collInfo.partInfo) == 0 {
|
if collInfo.partInfo == nil || len(collInfo.partInfo) == 0 {
|
||||||
tr := timerecord.NewTimeRecorder("UpdateCache")
|
tr := timerecord.NewTimeRecorder("UpdateCache")
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetPartitions", metrics.CacheMissLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
m.mu.RUnlock()
|
m.mu.RUnlock()
|
||||||
|
|
||||||
partitions, err := m.showPartitions(ctx, database, collectionName)
|
partitions, err := m.showPartitions(ctx, database, collectionName)
|
||||||
|
@ -414,7 +420,7 @@ func (m *MetaCache) GetPartitions(ctx context.Context, database, collectionName
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
log.Debug("proxy", zap.Any("GetPartitions:partitions after update", partitions), zap.String("collectionName", collectionName))
|
log.Debug("proxy", zap.Any("GetPartitions:partitions after update", partitions), zap.String("collectionName", collectionName))
|
||||||
ret := make(map[string]typeutil.UniqueID)
|
ret := make(map[string]typeutil.UniqueID)
|
||||||
partInfo := m.collInfo[database][collectionName].partInfo
|
partInfo := m.collInfo[database][collectionName].partInfo
|
||||||
|
@ -424,8 +430,9 @@ func (m *MetaCache) GetPartitions(ctx context.Context, database, collectionName
|
||||||
return ret, nil
|
return ret, nil
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
defer m.mu.RUnlock()
|
defer m.mu.RUnlock()
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetPartitions", metrics.CacheHitLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
|
|
||||||
ret := make(map[string]typeutil.UniqueID)
|
ret := make(map[string]typeutil.UniqueID)
|
||||||
partInfo := collInfo.partInfo
|
partInfo := collInfo.partInfo
|
||||||
|
@ -459,9 +466,10 @@ func (m *MetaCache) GetPartitionInfo(ctx context.Context, database, collectionNa
|
||||||
partInfo, ok = collInfo.partInfo[partitionName]
|
partInfo, ok = collInfo.partInfo[partitionName]
|
||||||
m.mu.RUnlock()
|
m.mu.RUnlock()
|
||||||
|
|
||||||
|
method := "GetPartitionInfo"
|
||||||
if !ok {
|
if !ok {
|
||||||
tr := timerecord.NewTimeRecorder("UpdateCache")
|
tr := timerecord.NewTimeRecorder("UpdateCache")
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetPartitionInfo", metrics.CacheMissLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
partitions, err := m.showPartitions(ctx, database, collectionName)
|
partitions, err := m.showPartitions(ctx, database, collectionName)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -473,14 +481,14 @@ func (m *MetaCache) GetPartitionInfo(ctx context.Context, database, collectionNa
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
log.Debug("proxy", zap.Any("GetPartitionID:partitions after update", partitions), zap.String("collectionName", collectionName))
|
log.Debug("proxy", zap.Any("GetPartitionID:partitions after update", partitions), zap.String("collectionName", collectionName))
|
||||||
partInfo, ok = m.collInfo[database][collectionName].partInfo[partitionName]
|
partInfo, ok = m.collInfo[database][collectionName].partInfo[partitionName]
|
||||||
if !ok {
|
if !ok {
|
||||||
return nil, merr.WrapErrPartitionNotFound(partitionName)
|
return nil, merr.WrapErrPartitionNotFound(partitionName)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
metrics.ProxyCacheStatsCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), "GetPartitionInfo", metrics.CacheHitLabel).Inc()
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
return &partitionInfo{
|
return &partitionInfo{
|
||||||
partitionID: partInfo.partitionID,
|
partitionID: partInfo.partitionID,
|
||||||
createdTimestamp: partInfo.createdTimestamp,
|
createdTimestamp: partInfo.createdTimestamp,
|
||||||
|
@ -692,6 +700,7 @@ func (m *MetaCache) GetShards(ctx context.Context, withCache bool, database, col
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
method := "GetShards"
|
||||||
if withCache {
|
if withCache {
|
||||||
var shardLeaders *shardLeaders
|
var shardLeaders *shardLeaders
|
||||||
info.leaderMutex.RLock()
|
info.leaderMutex.RLock()
|
||||||
|
@ -699,10 +708,12 @@ func (m *MetaCache) GetShards(ctx context.Context, withCache bool, database, col
|
||||||
info.leaderMutex.RUnlock()
|
info.leaderMutex.RUnlock()
|
||||||
|
|
||||||
if shardLeaders != nil && !shardLeaders.deprecated.Load() {
|
if shardLeaders != nil && !shardLeaders.deprecated.Load() {
|
||||||
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheHitLabel).Inc()
|
||||||
iterator := shardLeaders.GetReader()
|
iterator := shardLeaders.GetReader()
|
||||||
return iterator.Shuffle(), nil
|
return iterator.Shuffle(), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
metrics.ProxyCacheStatsCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method, metrics.CacheMissLabel).Inc()
|
||||||
log.Info("no shard cache for collection, try to get shard leaders from QueryCoord",
|
log.Info("no shard cache for collection, try to get shard leaders from QueryCoord",
|
||||||
zap.String("collectionName", collectionName))
|
zap.String("collectionName", collectionName))
|
||||||
}
|
}
|
||||||
|
@ -718,6 +729,7 @@ func (m *MetaCache) GetShards(ctx context.Context, withCache bool, database, col
|
||||||
var resp *querypb.GetShardLeadersResponse
|
var resp *querypb.GetShardLeadersResponse
|
||||||
childCtx, cancel := context.WithTimeout(ctx, time.Second*10)
|
childCtx, cancel := context.WithTimeout(ctx, time.Second*10)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
tr := timerecord.NewTimeRecorder("UpdateShardCache")
|
||||||
err = retry.Do(childCtx, func() error {
|
err = retry.Do(childCtx, func() error {
|
||||||
resp, err = m.queryCoord.GetShardLeaders(ctx, req)
|
resp, err = m.queryCoord.GetShardLeaders(ctx, req)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -765,6 +777,7 @@ func (m *MetaCache) GetShards(ctx context.Context, withCache bool, database, col
|
||||||
// and create new client for new leaders
|
// and create new client for new leaders
|
||||||
_ = m.shardMgr.UpdateShardLeaders(oldLeaders, ret)
|
_ = m.shardMgr.UpdateShardLeaders(oldLeaders, ret)
|
||||||
|
|
||||||
|
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
return ret, nil
|
return ret, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -137,11 +137,14 @@ func NewProxy(ctx context.Context, factory dependency.Factory) (*Proxy, error) {
|
||||||
// Register registers proxy at etcd
|
// Register registers proxy at etcd
|
||||||
func (node *Proxy) Register() error {
|
func (node *Proxy) Register() error {
|
||||||
node.session.Register()
|
node.session.Register()
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.ProxyRole).Inc()
|
||||||
|
log.Info("Proxy Register Finished")
|
||||||
node.session.LivenessCheck(node.ctx, func() {
|
node.session.LivenessCheck(node.ctx, func() {
|
||||||
log.Error("Proxy disconnected from etcd, process will exit", zap.Int64("Server Id", node.session.ServerID))
|
log.Error("Proxy disconnected from etcd, process will exit", zap.Int64("Server Id", node.session.ServerID))
|
||||||
if err := node.Stop(); err != nil {
|
if err := node.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.ProxyRole).Dec()
|
||||||
if node.session.TriggerKill {
|
if node.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
p.Signal(syscall.SIGINT)
|
p.Signal(syscall.SIGINT)
|
||||||
|
|
|
@ -139,12 +139,13 @@ func (s *Server) Register() error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.QueryCoordRole).Inc()
|
||||||
s.session.LivenessCheck(s.ctx, func() {
|
s.session.LivenessCheck(s.ctx, func() {
|
||||||
log.Error("QueryCoord disconnected from etcd, process will exit", zap.Int64("serverID", s.session.ServerID))
|
log.Error("QueryCoord disconnected from etcd, process will exit", zap.Int64("serverID", s.session.ServerID))
|
||||||
if err := s.Stop(); err != nil {
|
if err := s.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.QueryCoordRole).Dec()
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
if s.session.TriggerKill {
|
if s.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
|
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -67,6 +68,7 @@ func (m *manager) Add(collectionID UniqueID, channel string) (Pipeline, error) {
|
||||||
zap.Int64("collectionID", collectionID),
|
zap.Int64("collectionID", collectionID),
|
||||||
zap.String("channel", channel),
|
zap.String("channel", channel),
|
||||||
)
|
)
|
||||||
|
tr := timerecord.NewTimeRecorder("add dmChannel")
|
||||||
collection := m.dataManager.Collection.Get(collectionID)
|
collection := m.dataManager.Collection.Get(collectionID)
|
||||||
if collection == nil {
|
if collection == nil {
|
||||||
return nil, segments.WrapCollectionNotFound(collectionID)
|
return nil, segments.WrapCollectionNotFound(collectionID)
|
||||||
|
@ -90,6 +92,7 @@ func (m *manager) Add(collectionID UniqueID, channel string) (Pipeline, error) {
|
||||||
m.channel2Pipeline[channel] = newPipeLine
|
m.channel2Pipeline[channel] = newPipeLine
|
||||||
metrics.QueryNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
metrics.QueryNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
||||||
metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
||||||
|
metrics.QueryNodeWatchDmlChannelLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
return newPipeLine, nil
|
return newPipeLine, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -71,8 +71,11 @@ func searchSegments(ctx context.Context, manager *Manager, segType SegmentType,
|
||||||
errs[i] = err
|
errs[i] = err
|
||||||
resultCh <- searchResult
|
resultCh <- searchResult
|
||||||
// update metrics
|
// update metrics
|
||||||
|
elapsed := tr.ElapseSpan().Milliseconds()
|
||||||
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()),
|
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()),
|
||||||
metrics.SearchLabel, searchLabel).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.SearchLabel, searchLabel).Observe(float64(elapsed))
|
||||||
|
metrics.QueryNodeSegmentSearchLatencyPerVector.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()),
|
||||||
|
metrics.SearchLabel, searchLabel).Observe(float64(elapsed) / float64(searchReq.getNumOfQuery()))
|
||||||
}(segID, i)
|
}(segID, i)
|
||||||
}
|
}
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
|
@ -58,6 +58,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/pkg/config"
|
"github.com/milvus-io/milvus/pkg/config"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||||
"github.com/milvus-io/milvus/pkg/util/gc"
|
"github.com/milvus-io/milvus/pkg/util/gc"
|
||||||
"github.com/milvus-io/milvus/pkg/util/hardware"
|
"github.com/milvus-io/milvus/pkg/util/hardware"
|
||||||
|
@ -155,11 +156,13 @@ func (node *QueryNode) initSession() error {
|
||||||
func (node *QueryNode) Register() error {
|
func (node *QueryNode) Register() error {
|
||||||
node.session.Register()
|
node.session.Register()
|
||||||
// start liveness check
|
// start liveness check
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.QueryNodeRole).Inc()
|
||||||
node.session.LivenessCheck(node.ctx, func() {
|
node.session.LivenessCheck(node.ctx, func() {
|
||||||
log.Error("Query Node disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID()))
|
log.Error("Query Node disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID()))
|
||||||
if err := node.Stop(); err != nil {
|
if err := node.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.QueryNodeRole).Dec()
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
if node.session.TriggerKill {
|
if node.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
|
|
|
@ -45,7 +45,7 @@ func Test_alterAliasTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to expire cache", func(t *testing.T) {
|
t.Run("failed to expire cache", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidProxyManager())
|
core := newTestCore(withInvalidProxyManager())
|
||||||
task := &alterAliasTask{
|
task := &alterAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AlterAliasRequest{
|
Req: &milvuspb.AlterAliasRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
|
||||||
Alias: "test",
|
Alias: "test",
|
||||||
|
@ -58,7 +58,7 @@ func Test_alterAliasTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to alter alias", func(t *testing.T) {
|
t.Run("failed to alter alias", func(t *testing.T) {
|
||||||
core := newTestCore(withValidProxyManager(), withInvalidMeta())
|
core := newTestCore(withValidProxyManager(), withInvalidMeta())
|
||||||
task := &alterAliasTask{
|
task := &alterAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AlterAliasRequest{
|
Req: &milvuspb.AlterAliasRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias},
|
||||||
Alias: "test",
|
Alias: "test",
|
||||||
|
|
|
@ -67,7 +67,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to create alias", func(t *testing.T) {
|
t.Run("failed to create alias", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &alterCollectionTask{
|
task := &alterCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AlterCollectionRequest{
|
Req: &milvuspb.AlterCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
||||||
CollectionName: "cn",
|
CollectionName: "cn",
|
||||||
|
@ -95,7 +95,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &alterCollectionTask{
|
task := &alterCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AlterCollectionRequest{
|
Req: &milvuspb.AlterCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
||||||
CollectionName: "cn",
|
CollectionName: "cn",
|
||||||
|
@ -129,7 +129,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker))
|
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker))
|
||||||
task := &alterCollectionTask{
|
task := &alterCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AlterCollectionRequest{
|
Req: &milvuspb.AlterCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
||||||
CollectionName: "cn",
|
CollectionName: "cn",
|
||||||
|
@ -163,7 +163,7 @@ func Test_alterCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker))
|
core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker))
|
||||||
task := &alterCollectionTask{
|
task := &alterCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.AlterCollectionRequest{
|
Req: &milvuspb.AlterCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterCollection},
|
||||||
CollectionName: "cn",
|
CollectionName: "cn",
|
||||||
|
|
|
@ -44,7 +44,7 @@ func Test_createAliasTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to expire cache", func(t *testing.T) {
|
t.Run("failed to expire cache", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidProxyManager())
|
core := newTestCore(withInvalidProxyManager())
|
||||||
task := &createAliasTask{
|
task := &createAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateAliasRequest{
|
Req: &milvuspb.CreateAliasRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias},
|
||||||
Alias: "test",
|
Alias: "test",
|
||||||
|
@ -57,7 +57,7 @@ func Test_createAliasTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to create alias", func(t *testing.T) {
|
t.Run("failed to create alias", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta(), withValidProxyManager())
|
core := newTestCore(withInvalidMeta(), withValidProxyManager())
|
||||||
task := &createAliasTask{
|
task := &createAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateAliasRequest{
|
Req: &milvuspb.CreateAliasRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateAlias},
|
||||||
Alias: "test",
|
Alias: "test",
|
||||||
|
|
|
@ -524,7 +524,7 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
|
||||||
core := newTestCore(withInvalidIDAllocator(), withMeta(meta))
|
core := newTestCore(withInvalidIDAllocator(), withMeta(meta))
|
||||||
|
|
||||||
task := createCollectionTask{
|
task := createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateCollectionRequest{
|
Req: &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -558,7 +558,7 @@ func Test_createCollectionTask_Prepare(t *testing.T) {
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
task := createCollectionTask{
|
task := createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateCollectionRequest{
|
Req: &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -595,7 +595,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
|
||||||
core := newTestCore(withMeta(meta), withTtSynchronizer(ticker))
|
core := newTestCore(withMeta(meta), withTtSynchronizer(ticker))
|
||||||
|
|
||||||
task := &createCollectionTask{
|
task := &createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateCollectionRequest{
|
Req: &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -642,7 +642,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
|
||||||
core := newTestCore(withMeta(meta), withTtSynchronizer(ticker))
|
core := newTestCore(withMeta(meta), withTtSynchronizer(ticker))
|
||||||
|
|
||||||
task := &createCollectionTask{
|
task := &createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateCollectionRequest{
|
Req: &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -662,7 +662,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
|
||||||
pchans := ticker.getDmlChannelNames(shardNum)
|
pchans := ticker.getDmlChannelNames(shardNum)
|
||||||
core := newTestCore(withTtSynchronizer(ticker))
|
core := newTestCore(withTtSynchronizer(ticker))
|
||||||
task := &createCollectionTask{
|
task := &createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
channels: collectionChannels{
|
channels: collectionChannels{
|
||||||
physicalChannels: pchans,
|
physicalChannels: pchans,
|
||||||
virtualChannels: []string{funcutil.GenRandomStr(), funcutil.GenRandomStr()},
|
virtualChannels: []string{funcutil.GenRandomStr(), funcutil.GenRandomStr()},
|
||||||
|
@ -734,7 +734,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
task := createCollectionTask{
|
task := createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateCollectionRequest{
|
Req: &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||||
DbName: "mock-db",
|
DbName: "mock-db",
|
||||||
|
@ -829,7 +829,7 @@ func Test_createCollectionTask_Execute(t *testing.T) {
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
task := createCollectionTask{
|
task := createCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreateCollectionRequest{
|
Req: &milvuspb.CreateCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
|
|
@ -43,7 +43,7 @@ func Test_createPartitionTask_Prepare(t *testing.T) {
|
||||||
t.Run("failed to get collection meta", func(t *testing.T) {
|
t.Run("failed to get collection meta", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &createPartitionTask{
|
task := &createPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreatePartition}},
|
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreatePartition}},
|
||||||
}
|
}
|
||||||
err := task.Prepare(context.Background())
|
err := task.Prepare(context.Background())
|
||||||
|
@ -64,7 +64,7 @@ func Test_createPartitionTask_Prepare(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &createPartitionTask{
|
task := &createPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreatePartition}},
|
Req: &milvuspb.CreatePartitionRequest{Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreatePartition}},
|
||||||
}
|
}
|
||||||
err := task.Prepare(context.Background())
|
err := task.Prepare(context.Background())
|
||||||
|
@ -109,7 +109,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
|
||||||
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
|
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
|
||||||
core := newTestCore(withInvalidIDAllocator())
|
core := newTestCore(withInvalidIDAllocator())
|
||||||
task := &createPartitionTask{
|
task := &createPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
collMeta: coll,
|
collMeta: coll,
|
||||||
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
||||||
}
|
}
|
||||||
|
@ -123,7 +123,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
|
||||||
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
|
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
|
||||||
core := newTestCore(withValidIDAllocator(), withInvalidProxyManager())
|
core := newTestCore(withValidIDAllocator(), withInvalidProxyManager())
|
||||||
task := &createPartitionTask{
|
task := &createPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
collMeta: coll,
|
collMeta: coll,
|
||||||
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
||||||
}
|
}
|
||||||
|
@ -137,7 +137,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
|
||||||
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
|
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{}}
|
||||||
core := newTestCore(withValidIDAllocator(), withValidProxyManager(), withInvalidMeta())
|
core := newTestCore(withValidIDAllocator(), withValidProxyManager(), withInvalidMeta())
|
||||||
task := &createPartitionTask{
|
task := &createPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
collMeta: coll,
|
collMeta: coll,
|
||||||
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
||||||
}
|
}
|
||||||
|
@ -162,7 +162,7 @@ func Test_createPartitionTask_Execute(t *testing.T) {
|
||||||
}
|
}
|
||||||
core := newTestCore(withValidIDAllocator(), withValidProxyManager(), withMeta(meta), withBroker(b))
|
core := newTestCore(withValidIDAllocator(), withValidProxyManager(), withMeta(meta), withBroker(b))
|
||||||
task := &createPartitionTask{
|
task := &createPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
collMeta: coll,
|
collMeta: coll,
|
||||||
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
Req: &milvuspb.CreatePartitionRequest{CollectionName: collectionName, PartitionName: partitionName},
|
||||||
}
|
}
|
||||||
|
|
|
@ -60,10 +60,7 @@ func Test_describeCollectionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to get collection by name", func(t *testing.T) {
|
t.Run("failed to get collection by name", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &describeCollectionTask{
|
task := &describeCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.DescribeCollectionRequest{
|
Req: &milvuspb.DescribeCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DescribeCollection,
|
MsgType: commonpb.MsgType_DescribeCollection,
|
||||||
|
@ -79,10 +76,7 @@ func Test_describeCollectionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to get collection by id", func(t *testing.T) {
|
t.Run("failed to get collection by id", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &describeCollectionTask{
|
task := &describeCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.DescribeCollectionRequest{
|
Req: &milvuspb.DescribeCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DescribeCollection,
|
MsgType: commonpb.MsgType_DescribeCollection,
|
||||||
|
@ -114,10 +108,7 @@ func Test_describeCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &describeCollectionTask{
|
task := &describeCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.DescribeCollectionRequest{
|
Req: &milvuspb.DescribeCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_DescribeCollection,
|
MsgType: commonpb.MsgType_DescribeCollection,
|
||||||
|
|
|
@ -52,7 +52,7 @@ func Test_dropAliasTask_Execute(t *testing.T) {
|
||||||
core := newTestCore(withInvalidProxyManager())
|
core := newTestCore(withInvalidProxyManager())
|
||||||
alias := funcutil.GenRandomStr()
|
alias := funcutil.GenRandomStr()
|
||||||
task := &dropAliasTask{
|
task := &dropAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropAliasRequest{
|
Req: &milvuspb.DropAliasRequest{
|
||||||
|
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
|
||||||
|
@ -67,7 +67,7 @@ func Test_dropAliasTask_Execute(t *testing.T) {
|
||||||
core := newTestCore(withValidProxyManager(), withInvalidMeta())
|
core := newTestCore(withValidProxyManager(), withInvalidMeta())
|
||||||
alias := funcutil.GenRandomStr()
|
alias := funcutil.GenRandomStr()
|
||||||
task := &dropAliasTask{
|
task := &dropAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropAliasRequest{
|
Req: &milvuspb.DropAliasRequest{
|
||||||
|
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
|
||||||
|
@ -90,7 +90,7 @@ func Test_dropAliasTask_Execute(t *testing.T) {
|
||||||
core := newTestCore(withValidProxyManager(), withMeta(meta))
|
core := newTestCore(withValidProxyManager(), withMeta(meta))
|
||||||
alias := funcutil.GenRandomStr()
|
alias := funcutil.GenRandomStr()
|
||||||
task := &dropAliasTask{
|
task := &dropAliasTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropAliasRequest{
|
Req: &milvuspb.DropAliasRequest{
|
||||||
|
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropAlias},
|
||||||
|
|
|
@ -55,7 +55,7 @@ func Test_dropCollectionTask_Prepare(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &dropCollectionTask{
|
task := &dropCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropCollectionRequest{
|
Req: &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -76,7 +76,7 @@ func Test_dropCollectionTask_Prepare(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &dropCollectionTask{
|
task := &dropCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropCollectionRequest{
|
Req: &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -104,7 +104,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
|
||||||
})
|
})
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &dropCollectionTask{
|
task := &dropCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropCollectionRequest{
|
Req: &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -134,7 +134,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withInvalidProxyManager(), withMeta(meta))
|
core := newTestCore(withInvalidProxyManager(), withMeta(meta))
|
||||||
task := &dropCollectionTask{
|
task := &dropCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropCollectionRequest{
|
Req: &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -167,7 +167,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withValidProxyManager(), withMeta(meta))
|
core := newTestCore(withValidProxyManager(), withMeta(meta))
|
||||||
task := &dropCollectionTask{
|
task := &dropCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropCollectionRequest{
|
Req: &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -257,7 +257,7 @@ func Test_dropCollectionTask_Execute(t *testing.T) {
|
||||||
withTtSynchronizer(ticker))
|
withTtSynchronizer(ticker))
|
||||||
|
|
||||||
task := &dropCollectionTask{
|
task := &dropCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropCollectionRequest{
|
Req: &milvuspb.DropCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
|
|
@ -56,7 +56,7 @@ func Test_dropPartitionTask_Prepare(t *testing.T) {
|
||||||
t.Run("failed to get collection meta", func(t *testing.T) {
|
t.Run("failed to get collection meta", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &dropPartitionTask{
|
task := &dropPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropPartitionRequest{
|
Req: &milvuspb.DropPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
||||||
},
|
},
|
||||||
|
@ -81,7 +81,7 @@ func Test_dropPartitionTask_Prepare(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &dropPartitionTask{
|
task := &dropPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropPartitionRequest{
|
Req: &milvuspb.DropPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -116,7 +116,7 @@ func Test_dropPartitionTask_Execute(t *testing.T) {
|
||||||
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{{PartitionName: partitionName}}}
|
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{{PartitionName: partitionName}}}
|
||||||
core := newTestCore(withInvalidProxyManager())
|
core := newTestCore(withInvalidProxyManager())
|
||||||
task := &dropPartitionTask{
|
task := &dropPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropPartitionRequest{
|
Req: &milvuspb.DropPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -134,7 +134,7 @@ func Test_dropPartitionTask_Execute(t *testing.T) {
|
||||||
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{{PartitionName: partitionName}}}
|
coll := &model.Collection{Name: collectionName, Partitions: []*model.Partition{{PartitionName: partitionName}}}
|
||||||
core := newTestCore(withValidProxyManager(), withInvalidMeta())
|
core := newTestCore(withValidProxyManager(), withInvalidMeta())
|
||||||
task := &dropPartitionTask{
|
task := &dropPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropPartitionRequest{
|
Req: &milvuspb.DropPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
@ -204,7 +204,7 @@ func Test_dropPartitionTask_Execute(t *testing.T) {
|
||||||
withBroker(broker))
|
withBroker(broker))
|
||||||
|
|
||||||
task := &dropPartitionTask{
|
task := &dropPartitionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.DropPartitionRequest{
|
Req: &milvuspb.DropPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropPartition},
|
||||||
CollectionName: collectionName,
|
CollectionName: collectionName,
|
||||||
|
|
|
@ -59,10 +59,7 @@ func Test_hasCollectionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed", func(t *testing.T) {
|
t.Run("failed", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &hasCollectionTask{
|
task := &hasCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.HasCollectionRequest{
|
Req: &milvuspb.HasCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasCollection,
|
MsgType: commonpb.MsgType_HasCollection,
|
||||||
|
@ -87,10 +84,7 @@ func Test_hasCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &hasCollectionTask{
|
task := &hasCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.HasCollectionRequest{
|
Req: &milvuspb.HasCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasCollection,
|
MsgType: commonpb.MsgType_HasCollection,
|
||||||
|
|
|
@ -59,10 +59,7 @@ func Test_hasPartitionTask_Execute(t *testing.T) {
|
||||||
t.Run("fail to get collection", func(t *testing.T) {
|
t.Run("fail to get collection", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &hasPartitionTask{
|
task := &hasPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.HasPartitionRequest{
|
Req: &milvuspb.HasPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasPartition,
|
MsgType: commonpb.MsgType_HasPartition,
|
||||||
|
@ -94,10 +91,7 @@ func Test_hasPartitionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &hasPartitionTask{
|
task := &hasPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.HasPartitionRequest{
|
Req: &milvuspb.HasPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasCollection,
|
MsgType: commonpb.MsgType_HasCollection,
|
||||||
|
@ -133,10 +127,7 @@ func Test_hasPartitionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &hasPartitionTask{
|
task := &hasPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.HasPartitionRequest{
|
Req: &milvuspb.HasPartitionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_HasCollection,
|
MsgType: commonpb.MsgType_HasCollection,
|
||||||
|
|
|
@ -59,7 +59,7 @@ func Test_renameCollectionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to expire cache", func(t *testing.T) {
|
t.Run("failed to expire cache", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidProxyManager())
|
core := newTestCore(withInvalidProxyManager())
|
||||||
task := &renameCollectionTask{
|
task := &renameCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.RenameCollectionRequest{
|
Req: &milvuspb.RenameCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_RenameCollection,
|
MsgType: commonpb.MsgType_RenameCollection,
|
||||||
|
@ -78,7 +78,7 @@ func Test_renameCollectionTask_Execute(t *testing.T) {
|
||||||
|
|
||||||
core := newTestCore(withValidProxyManager(), withMeta(meta))
|
core := newTestCore(withValidProxyManager(), withMeta(meta))
|
||||||
task := &renameCollectionTask{
|
task := &renameCollectionTask{
|
||||||
baseTask: baseTask{core: core},
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
Req: &milvuspb.RenameCollectionRequest{
|
Req: &milvuspb.RenameCollectionRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_RenameCollection,
|
MsgType: commonpb.MsgType_RenameCollection,
|
||||||
|
|
|
@ -280,12 +280,14 @@ func (c *Core) Register() error {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.RootCoordRole).Inc()
|
||||||
log.Info("RootCoord Register Finished")
|
log.Info("RootCoord Register Finished")
|
||||||
c.session.LivenessCheck(c.ctx, func() {
|
c.session.LivenessCheck(c.ctx, func() {
|
||||||
log.Error("Root Coord disconnected from etcd, process will exit", zap.Int64("Server Id", c.session.ServerID))
|
log.Error("Root Coord disconnected from etcd, process will exit", zap.Int64("Server Id", c.session.ServerID))
|
||||||
if err := c.Stop(); err != nil {
|
if err := c.Stop(); err != nil {
|
||||||
log.Fatal("failed to stop server", zap.Error(err))
|
log.Fatal("failed to stop server", zap.Error(err))
|
||||||
}
|
}
|
||||||
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.RootCoordRole).Dec()
|
||||||
// manually send signal to starter goroutine
|
// manually send signal to starter goroutine
|
||||||
if c.session.TriggerKill {
|
if c.session.TriggerKill {
|
||||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||||
|
@ -900,12 +902,8 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti
|
||||||
zap.String("role", typeutil.RootCoordRole))
|
zap.String("role", typeutil.RootCoordRole))
|
||||||
|
|
||||||
t := &createCollectionTask{
|
t := &createCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -932,6 +930,7 @@ func (c *Core) CreateCollection(ctx context.Context, in *milvuspb.CreateCollecti
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateCollection", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("CreateCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("CreateCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
metrics.RootCoordNumOfCollections.Inc()
|
metrics.RootCoordNumOfCollections.Inc()
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("CreateCollection").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Ctx(ctx).Info("done to create collection",
|
log.Ctx(ctx).Info("done to create collection",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -955,12 +954,8 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe
|
||||||
zap.String("name", in.GetCollectionName()))
|
zap.String("name", in.GetCollectionName()))
|
||||||
|
|
||||||
t := &dropCollectionTask{
|
t := &dropCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -985,6 +980,7 @@ func (c *Core) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRe
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DropCollection", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("DropCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("DropCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
metrics.RootCoordNumOfCollections.Dec()
|
metrics.RootCoordNumOfCollections.Dec()
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("DropCollection").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Ctx(ctx).Info("done to drop collection", zap.String("role", typeutil.RootCoordRole),
|
log.Ctx(ctx).Info("done to drop collection", zap.String("role", typeutil.RootCoordRole),
|
||||||
zap.String("name", in.GetCollectionName()),
|
zap.String("name", in.GetCollectionName()),
|
||||||
|
@ -1033,6 +1029,7 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("HasCollection", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("HasCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("HasCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("HasCollection").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Info("done to has collection", zap.Bool("exist", t.Rsp.GetValue()))
|
log.Info("done to has collection", zap.Bool("exist", t.Rsp.GetValue()))
|
||||||
|
|
||||||
|
@ -1127,6 +1124,7 @@ func (c *Core) describeCollectionImpl(ctx context.Context, in *milvuspb.Describe
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DescribeCollection", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("DescribeCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("DescribeCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("DescribeCollection").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Info("done to describe collection", zap.Int64("collection_id", t.Rsp.GetCollectionID()))
|
log.Info("done to describe collection", zap.Int64("collection_id", t.Rsp.GetCollectionID()))
|
||||||
|
|
||||||
|
@ -1189,6 +1187,7 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollections", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("ShowCollections").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("ShowCollections").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("ShowCollections").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.
|
log.Info("done to show collections", zap.Int("num of collections", len(t.Rsp.GetCollectionNames()))) // maybe very large, print number instead.
|
||||||
|
|
||||||
|
@ -1208,12 +1207,8 @@ func (c *Core) AlterCollection(ctx context.Context, in *milvuspb.AlterCollection
|
||||||
zap.String("name", in.GetCollectionName()))
|
zap.String("name", in.GetCollectionName()))
|
||||||
|
|
||||||
t := &alterCollectionTask{
|
t := &alterCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -1240,6 +1235,7 @@ func (c *Core) AlterCollection(ctx context.Context, in *milvuspb.AlterCollection
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterCollection", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterCollection", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("AlterCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("AlterCollection").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
metrics.RootCoordNumOfCollections.Dec()
|
metrics.RootCoordNumOfCollections.Dec()
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("AlterCollection").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Info("done to alter collection",
|
log.Info("done to alter collection",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -1263,12 +1259,8 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
|
||||||
zap.String("partition", in.GetPartitionName()))
|
zap.String("partition", in.GetPartitionName()))
|
||||||
|
|
||||||
t := &createPartitionTask{
|
t := &createPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -1296,6 +1288,7 @@ func (c *Core) CreatePartition(ctx context.Context, in *milvuspb.CreatePartition
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("CreatePartition", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("CreatePartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("CreatePartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("CreatePartition").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Ctx(ctx).Info("done to create partition",
|
log.Ctx(ctx).Info("done to create partition",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -1320,12 +1313,8 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
|
||||||
zap.String("partition", in.GetPartitionName()))
|
zap.String("partition", in.GetPartitionName()))
|
||||||
|
|
||||||
t := &dropPartitionTask{
|
t := &dropPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -1352,6 +1341,7 @@ func (c *Core) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequ
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DropPartition", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("DropPartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("DropPartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("DropPartition").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Ctx(ctx).Info("done to drop partition",
|
log.Ctx(ctx).Info("done to drop partition",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -1404,6 +1394,7 @@ func (c *Core) HasPartition(ctx context.Context, in *milvuspb.HasPartitionReques
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("HasPartition", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("HasPartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("HasPartition").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("HasPartition").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Info("done to has partition", zap.Bool("exist", t.Rsp.GetValue()))
|
log.Info("done to has partition", zap.Bool("exist", t.Rsp.GetValue()))
|
||||||
|
|
||||||
|
@ -1454,6 +1445,7 @@ func (c *Core) showPartitionsImpl(ctx context.Context, in *milvuspb.ShowPartitio
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("ShowPartitions", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("ShowPartitions").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("ShowPartitions").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("ShowPartitions").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Info("done to show partitions", zap.Strings("partitions", t.Rsp.GetPartitionNames()))
|
log.Info("done to show partitions", zap.Strings("partitions", t.Rsp.GetPartitionNames()))
|
||||||
|
|
||||||
|
@ -1656,12 +1648,8 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
|
||||||
zap.String("collection", in.GetCollectionName()))
|
zap.String("collection", in.GetCollectionName()))
|
||||||
|
|
||||||
t := &createAliasTask{
|
t := &createAliasTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -1689,6 +1677,7 @@ func (c *Core) CreateAlias(ctx context.Context, in *milvuspb.CreateAliasRequest)
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("CreateAlias", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("CreateAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("CreateAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("CreateAlias").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Ctx(ctx).Info("done to create alias",
|
log.Ctx(ctx).Info("done to create alias",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -1712,12 +1701,8 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c
|
||||||
zap.String("alias", in.GetAlias()))
|
zap.String("alias", in.GetAlias()))
|
||||||
|
|
||||||
t := &dropAliasTask{
|
t := &dropAliasTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -1743,6 +1728,7 @@ func (c *Core) DropAlias(ctx context.Context, in *milvuspb.DropAliasRequest) (*c
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("DropAlias", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("DropAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("DropAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("DropAlias").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Ctx(ctx).Info("done to drop alias",
|
log.Ctx(ctx).Info("done to drop alias",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -1766,12 +1752,8 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
|
||||||
zap.String("collection", in.GetCollectionName()))
|
zap.String("collection", in.GetCollectionName()))
|
||||||
|
|
||||||
t := &alterAliasTask{
|
t := &alterAliasTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: in,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: in,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
@ -1799,6 +1781,7 @@ func (c *Core) AlterAlias(ctx context.Context, in *milvuspb.AlterAliasRequest) (
|
||||||
|
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterAlias", metrics.SuccessLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("AlterAlias", metrics.SuccessLabel).Inc()
|
||||||
metrics.RootCoordDDLReqLatency.WithLabelValues("AlterAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
metrics.RootCoordDDLReqLatency.WithLabelValues("AlterAlias").Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.RootCoordDDLReqLatencyInQueue.WithLabelValues("AlterAlias").Observe(float64(t.queueDur.Milliseconds()))
|
||||||
|
|
||||||
log.Info("done to alter alias",
|
log.Info("done to alter alias",
|
||||||
zap.String("role", typeutil.RootCoordRole),
|
zap.String("role", typeutil.RootCoordRole),
|
||||||
|
@ -2713,12 +2696,8 @@ func (c *Core) RenameCollection(ctx context.Context, req *milvuspb.RenameCollect
|
||||||
metrics.RootCoordDDLReqCounter.WithLabelValues("RenameCollection", metrics.TotalLabel).Inc()
|
metrics.RootCoordDDLReqCounter.WithLabelValues("RenameCollection", metrics.TotalLabel).Inc()
|
||||||
tr := timerecord.NewTimeRecorder("RenameCollection")
|
tr := timerecord.NewTimeRecorder("RenameCollection")
|
||||||
t := &renameCollectionTask{
|
t := &renameCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(ctx, c),
|
||||||
ctx: ctx,
|
Req: req,
|
||||||
core: c,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: req,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.scheduler.AddTask(t); err != nil {
|
if err := c.scheduler.AddTask(t); err != nil {
|
||||||
|
|
|
@ -80,6 +80,7 @@ func (s *scheduler) Stop() {
|
||||||
|
|
||||||
func (s *scheduler) execute(task task) {
|
func (s *scheduler) execute(task task) {
|
||||||
defer s.setMinDdlTs(task.GetTs()) // we should update ts, whatever task succeeds or not.
|
defer s.setMinDdlTs(task.GetTs()) // we should update ts, whatever task succeeds or not.
|
||||||
|
task.SetInQueueDuration()
|
||||||
if err := task.Prepare(task.GetCtx()); err != nil {
|
if err := task.Prepare(task.GetCtx()); err != nil {
|
||||||
task.NotifyDone(err)
|
task.NotifyDone(err)
|
||||||
return
|
return
|
||||||
|
|
|
@ -38,10 +38,7 @@ type mockFailTask struct {
|
||||||
|
|
||||||
func newMockFailTask() *mockFailTask {
|
func newMockFailTask() *mockFailTask {
|
||||||
task := &mockFailTask{
|
task := &mockFailTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), nil),
|
||||||
ctx: context.Background(),
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
}
|
}
|
||||||
task.SetCtx(context.Background())
|
task.SetCtx(context.Background())
|
||||||
return task
|
return task
|
||||||
|
@ -73,10 +70,7 @@ type mockNormalTask struct {
|
||||||
|
|
||||||
func newMockNormalTask() *mockNormalTask {
|
func newMockNormalTask() *mockNormalTask {
|
||||||
task := &mockNormalTask{
|
task := &mockNormalTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), nil),
|
||||||
ctx: context.Background(),
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
}
|
}
|
||||||
task.SetCtx(context.Background())
|
task.SetCtx(context.Background())
|
||||||
return task
|
return task
|
||||||
|
|
|
@ -56,10 +56,7 @@ func Test_showCollectionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to list collections", func(t *testing.T) {
|
t.Run("failed to list collections", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &showCollectionTask{
|
task := &showCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.ShowCollectionsRequest{
|
Req: &milvuspb.ShowCollectionsRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_ShowCollections,
|
MsgType: commonpb.MsgType_ShowCollections,
|
||||||
|
@ -85,10 +82,7 @@ func Test_showCollectionTask_Execute(t *testing.T) {
|
||||||
}
|
}
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &showCollectionTask{
|
task := &showCollectionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.ShowCollectionsRequest{
|
Req: &milvuspb.ShowCollectionsRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_ShowCollections,
|
MsgType: commonpb.MsgType_ShowCollections,
|
||||||
|
|
|
@ -57,10 +57,7 @@ func Test_showPartitionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to list collections by name", func(t *testing.T) {
|
t.Run("failed to list collections by name", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &showPartitionTask{
|
task := &showPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.ShowPartitionsRequest{
|
Req: &milvuspb.ShowPartitionsRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_ShowPartitions,
|
MsgType: commonpb.MsgType_ShowPartitions,
|
||||||
|
@ -77,10 +74,7 @@ func Test_showPartitionTask_Execute(t *testing.T) {
|
||||||
t.Run("failed to list collections by id", func(t *testing.T) {
|
t.Run("failed to list collections by id", func(t *testing.T) {
|
||||||
core := newTestCore(withInvalidMeta())
|
core := newTestCore(withInvalidMeta())
|
||||||
task := &showPartitionTask{
|
task := &showPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.ShowPartitionsRequest{
|
Req: &milvuspb.ShowPartitionsRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_ShowPartitions,
|
MsgType: commonpb.MsgType_ShowPartitions,
|
||||||
|
@ -114,10 +108,7 @@ func Test_showPartitionTask_Execute(t *testing.T) {
|
||||||
}
|
}
|
||||||
core := newTestCore(withMeta(meta))
|
core := newTestCore(withMeta(meta))
|
||||||
task := &showPartitionTask{
|
task := &showPartitionTask{
|
||||||
baseTask: baseTask{
|
baseTask: newBaseTask(context.Background(), core),
|
||||||
core: core,
|
|
||||||
done: make(chan error, 1),
|
|
||||||
},
|
|
||||||
Req: &milvuspb.ShowPartitionsRequest{
|
Req: &milvuspb.ShowPartitionsRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_ShowPartitions,
|
MsgType: commonpb.MsgType_ShowPartitions,
|
||||||
|
|
|
@ -18,6 +18,9 @@ package rootcoord
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
type task interface {
|
type task interface {
|
||||||
|
@ -31,6 +34,7 @@ type task interface {
|
||||||
Execute(ctx context.Context) error
|
Execute(ctx context.Context) error
|
||||||
WaitToFinish() error
|
WaitToFinish() error
|
||||||
NotifyDone(err error)
|
NotifyDone(err error)
|
||||||
|
SetInQueueDuration()
|
||||||
}
|
}
|
||||||
|
|
||||||
type baseTask struct {
|
type baseTask struct {
|
||||||
|
@ -39,12 +43,16 @@ type baseTask struct {
|
||||||
done chan error
|
done chan error
|
||||||
ts Timestamp
|
ts Timestamp
|
||||||
id UniqueID
|
id UniqueID
|
||||||
|
|
||||||
|
tr *timerecord.TimeRecorder
|
||||||
|
queueDur time.Duration
|
||||||
}
|
}
|
||||||
|
|
||||||
func newBaseTask(ctx context.Context, core *Core) baseTask {
|
func newBaseTask(ctx context.Context, core *Core) baseTask {
|
||||||
b := baseTask{
|
b := baseTask{
|
||||||
core: core,
|
core: core,
|
||||||
done: make(chan error, 1),
|
done: make(chan error, 1),
|
||||||
|
tr: timerecord.NewTimeRecorderWithTrace(ctx, "new task"),
|
||||||
}
|
}
|
||||||
b.SetCtx(ctx)
|
b.SetCtx(ctx)
|
||||||
return b
|
return b
|
||||||
|
@ -89,3 +97,7 @@ func (b *baseTask) WaitToFinish() error {
|
||||||
func (b *baseTask) NotifyDone(err error) {
|
func (b *baseTask) NotifyDone(err error) {
|
||||||
b.done <- err
|
b.done <- err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (b *baseTask) SetInQueueDuration() {
|
||||||
|
b.queueDur = b.tr.ElapseSpan()
|
||||||
|
}
|
||||||
|
|
|
@ -29,8 +29,10 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/storage/aliyun"
|
"github.com/milvus-io/milvus/internal/storage/aliyun"
|
||||||
"github.com/milvus-io/milvus/internal/storage/gcp"
|
"github.com/milvus-io/milvus/internal/storage/gcp"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
minio "github.com/minio/minio-go/v7"
|
minio "github.com/minio/minio-go/v7"
|
||||||
"github.com/minio/minio-go/v7/pkg/credentials"
|
"github.com/minio/minio-go/v7/pkg/credentials"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
@ -180,7 +182,7 @@ func (mcm *MinioChunkManager) Path(ctx context.Context, filePath string) (string
|
||||||
|
|
||||||
// Reader returns the path of minio data if exists.
|
// Reader returns the path of minio data if exists.
|
||||||
func (mcm *MinioChunkManager) Reader(ctx context.Context, filePath string) (FileReader, error) {
|
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 {
|
if err != nil {
|
||||||
log.Warn("failed to get object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to get object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -189,7 +191,7 @@ func (mcm *MinioChunkManager) Reader(ctx context.Context, filePath string) (File
|
||||||
}
|
}
|
||||||
|
|
||||||
func (mcm *MinioChunkManager) Size(ctx context.Context, filePath string) (int64, error) {
|
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 {
|
if err != nil {
|
||||||
log.Warn("failed to stat object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to stat object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return 0, err
|
return 0, err
|
||||||
|
@ -200,13 +202,14 @@ func (mcm *MinioChunkManager) Size(ctx context.Context, filePath string) (int64,
|
||||||
|
|
||||||
// Write writes the data to minio storage.
|
// Write writes the data to minio storage.
|
||||||
func (mcm *MinioChunkManager) Write(ctx context.Context, filePath string, content []byte) error {
|
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 {
|
if err != nil {
|
||||||
log.Warn("failed to put object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to put object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
metrics.PersistentDataKvSize.WithLabelValues(metrics.DataPutLabel).Observe(float64(len(content)))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -225,7 +228,7 @@ func (mcm *MinioChunkManager) MultiWrite(ctx context.Context, kvs map[string][]b
|
||||||
|
|
||||||
// Exist checks whether chunk is saved to minio storage.
|
// Exist checks whether chunk is saved to minio storage.
|
||||||
func (mcm *MinioChunkManager) Exist(ctx context.Context, filePath string) (bool, error) {
|
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 {
|
if err != nil {
|
||||||
errResponse := minio.ToErrorResponse(err)
|
errResponse := minio.ToErrorResponse(err)
|
||||||
if errResponse.Code == "NoSuchKey" {
|
if errResponse.Code == "NoSuchKey" {
|
||||||
|
@ -239,7 +242,7 @@ func (mcm *MinioChunkManager) Exist(ctx context.Context, filePath string) (bool,
|
||||||
|
|
||||||
// Read reads the minio storage data if exists.
|
// Read reads the minio storage data if exists.
|
||||||
func (mcm *MinioChunkManager) Read(ctx context.Context, filePath string) ([]byte, error) {
|
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 {
|
if err != nil {
|
||||||
log.Warn("failed to get object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to get object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -277,6 +280,7 @@ func (mcm *MinioChunkManager) Read(ctx context.Context, filePath string) ([]byte
|
||||||
log.Warn("failed to read object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to read object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
metrics.PersistentDataKvSize.WithLabelValues(metrics.DataGetLabel).Observe(float64(objectInfo.Size))
|
||||||
return data, nil
|
return data, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -324,7 +328,7 @@ func (mcm *MinioChunkManager) ReadAt(ctx context.Context, filePath string, off i
|
||||||
return nil, err
|
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 {
|
if err != nil {
|
||||||
log.Warn("failed to get object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to get object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -340,12 +344,13 @@ func (mcm *MinioChunkManager) ReadAt(ctx context.Context, filePath string, off i
|
||||||
log.Warn("failed to read object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to read object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
metrics.PersistentDataKvSize.WithLabelValues(metrics.DataGetLabel).Observe(float64(length))
|
||||||
return data, nil
|
return data, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Remove deletes an object with @key.
|
// Remove deletes an object with @key.
|
||||||
func (mcm *MinioChunkManager) Remove(ctx context.Context, filePath string) error {
|
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 {
|
if err != nil {
|
||||||
log.Warn("failed to remove object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
log.Warn("failed to remove object", zap.String("bucket", mcm.bucketName), zap.String("path", filePath), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
|
@ -367,7 +372,7 @@ func (mcm *MinioChunkManager) MultiRemove(ctx context.Context, keys []string) er
|
||||||
|
|
||||||
// RemoveWithPrefix removes all objects with the same prefix @prefix from minio.
|
// RemoveWithPrefix removes all objects with the same prefix @prefix from minio.
|
||||||
func (mcm *MinioChunkManager) RemoveWithPrefix(ctx context.Context, prefix string) error {
|
func (mcm *MinioChunkManager) RemoveWithPrefix(ctx context.Context, prefix string) error {
|
||||||
objects := mcm.Client.ListObjects(ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: prefix, Recursive: true})
|
objects := mcm.listMinioObjects(ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: prefix, Recursive: true})
|
||||||
i := 0
|
i := 0
|
||||||
maxGoroutine := 10
|
maxGoroutine := 10
|
||||||
removeKeys := make([]string, 0, len(objects))
|
removeKeys := make([]string, 0, len(objects))
|
||||||
|
@ -382,7 +387,7 @@ func (mcm *MinioChunkManager) RemoveWithPrefix(ctx context.Context, prefix strin
|
||||||
for j := 0; j < maxGoroutine && i < len(removeKeys); j++ {
|
for j := 0; j < maxGoroutine && i < len(removeKeys); j++ {
|
||||||
key := removeKeys[i]
|
key := removeKeys[i]
|
||||||
runningGroup.Go(func() error {
|
runningGroup.Go(func() error {
|
||||||
err := mcm.Client.RemoveObject(groupCtx, mcm.bucketName, key, minio.RemoveObjectOptions{})
|
err := mcm.removeMinioObject(groupCtx, mcm.bucketName, key, minio.RemoveObjectOptions{})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("failed to remove object", zap.String("path", key), zap.Error(err))
|
log.Warn("failed to remove object", zap.String("path", key), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
|
@ -422,7 +427,7 @@ func (mcm *MinioChunkManager) ListWithPrefix(ctx context.Context, prefix string,
|
||||||
|
|
||||||
// TODO add concurrent call if performance matters
|
// TODO add concurrent call if performance matters
|
||||||
// only return current level per call
|
// 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 {
|
for object := range objects {
|
||||||
if object.Err != nil {
|
if object.Err != nil {
|
||||||
|
@ -463,3 +468,79 @@ 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)
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.TotalLabel).Inc()
|
||||||
|
if err == nil && reader != nil {
|
||||||
|
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataGetLabel).Observe(float64(start.ElapseSpan().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)
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataPutLabel, metrics.TotalLabel).Inc()
|
||||||
|
if err == nil {
|
||||||
|
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataPutLabel).Observe(float64(start.ElapseSpan().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)
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.TotalLabel).Inc()
|
||||||
|
if err == nil {
|
||||||
|
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataStatLabel).Observe(float64(start.ElapseSpan().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)
|
||||||
|
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataListLabel).Observe(float64(start.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataListLabel, metrics.TotalLabel).Inc()
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataListLabel, 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)
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.TotalLabel).Inc()
|
||||||
|
if err == nil {
|
||||||
|
metrics.PersistentDataRequestLatency.WithLabelValues(metrics.DataRemoveLabel).Observe(float64(start.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.SuccessLabel).Inc()
|
||||||
|
} else {
|
||||||
|
metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.FailLabel).Inc()
|
||||||
|
}
|
||||||
|
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
|
@ -80,6 +80,10 @@ func (bm *MockMsg) SetPosition(position *MsgPosition) {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (bm *MockMsg) Size() int {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
func Test_GenerateMsgStreamMsg(t *testing.T) {
|
func Test_GenerateMsgStreamMsg(t *testing.T) {
|
||||||
messages := make([]msgstream.TsMsg, 1)
|
messages := make([]msgstream.TsMsg, 1)
|
||||||
messages[0] = &MockMsg{
|
messages[0] = &MockMsg{
|
||||||
|
|
|
@ -32,6 +32,12 @@ const (
|
||||||
CompactOutputLabel = "output"
|
CompactOutputLabel = "output"
|
||||||
compactIOLabelName = "IO"
|
compactIOLabelName = "IO"
|
||||||
compactTypeLabelName = "compactType"
|
compactTypeLabelName = "compactType"
|
||||||
|
|
||||||
|
InsertFileLabel = "insert_file"
|
||||||
|
DeleteFileLabel = "delete_file"
|
||||||
|
StatFileLabel = "stat_file"
|
||||||
|
IndexFileLabel = "index_file"
|
||||||
|
segmentFileTypeLabelName = "segment_file_type"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
@ -112,6 +118,33 @@ var (
|
||||||
segmentIDLabelName,
|
segmentIDLabelName,
|
||||||
})
|
})
|
||||||
|
|
||||||
|
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{})
|
||||||
|
|
||||||
|
FlushedSegmentFileNum = prometheus.NewHistogramVec(
|
||||||
|
prometheus.HistogramOpts{
|
||||||
|
Namespace: milvusNamespace,
|
||||||
|
Name: "flushed_segment_file_num",
|
||||||
|
Help: "the num of files for flushed segment",
|
||||||
|
Buckets: buckets,
|
||||||
|
}, []string{segmentFileTypeLabelName})
|
||||||
|
|
||||||
/* hard to implement, commented now
|
/* hard to implement, commented now
|
||||||
DataCoordSegmentSizeRatio = prometheus.NewHistogramVec(
|
DataCoordSegmentSizeRatio = prometheus.NewHistogramVec(
|
||||||
prometheus.HistogramOpts{
|
prometheus.HistogramOpts{
|
||||||
|
@ -195,6 +228,9 @@ func RegisterDataCoord(registry *prometheus.Registry) {
|
||||||
registry.MustRegister(DataCoordConsumeDataNodeTimeTickLag)
|
registry.MustRegister(DataCoordConsumeDataNodeTimeTickLag)
|
||||||
registry.MustRegister(DataCoordStoredBinlogSize)
|
registry.MustRegister(DataCoordStoredBinlogSize)
|
||||||
registry.MustRegister(DataCoordSegmentBinLogFileCount)
|
registry.MustRegister(DataCoordSegmentBinLogFileCount)
|
||||||
|
registry.MustRegister(DataCoordDmlChannelNum)
|
||||||
|
registry.MustRegister(DataCoordCompactedSegmentSize)
|
||||||
|
registry.MustRegister(FlushedSegmentFileNum)
|
||||||
registry.MustRegister(IndexRequestCounter)
|
registry.MustRegister(IndexRequestCounter)
|
||||||
registry.MustRegister(IndexTaskNum)
|
registry.MustRegister(IndexTaskNum)
|
||||||
registry.MustRegister(IndexNodeNum)
|
registry.MustRegister(IndexNodeNum)
|
||||||
|
|
|
@ -159,6 +159,17 @@ var (
|
||||||
nodeIDLabelName,
|
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 counts the num of calls of FlushSegments
|
||||||
DataNodeFlushReqCounter = prometheus.NewCounterVec(
|
DataNodeFlushReqCounter = prometheus.NewCounterVec(
|
||||||
prometheus.CounterOpts{
|
prometheus.CounterOpts{
|
||||||
|
@ -189,6 +200,17 @@ var (
|
||||||
Buckets: buckets, // unit: ms
|
Buckets: buckets, // unit: ms
|
||||||
}, []string{nodeIDLabelName})
|
}, []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,
|
||||||
|
})
|
||||||
|
|
||||||
DataNodeMsgDispatcherTtLag = prometheus.NewGaugeVec(
|
DataNodeMsgDispatcherTtLag = prometheus.NewGaugeVec(
|
||||||
prometheus.GaugeOpts{
|
prometheus.GaugeOpts{
|
||||||
Namespace: milvusNamespace,
|
Namespace: milvusNamespace,
|
||||||
|
@ -219,6 +241,8 @@ func RegisterDataNode(registry *prometheus.Registry) {
|
||||||
registry.MustRegister(DataNodeConsumeBytesCount)
|
registry.MustRegister(DataNodeConsumeBytesCount)
|
||||||
registry.MustRegister(DataNodeForwardDeleteMsgTimeTaken)
|
registry.MustRegister(DataNodeForwardDeleteMsgTimeTaken)
|
||||||
registry.MustRegister(DataNodeMsgDispatcherTtLag)
|
registry.MustRegister(DataNodeMsgDispatcherTtLag)
|
||||||
|
registry.MustRegister(DataNodeCompactionLatencyInQueue)
|
||||||
|
registry.MustRegister(DataNodeFlowGraphBufferDataSize)
|
||||||
}
|
}
|
||||||
|
|
||||||
func CleanupDataNodeCollectionMetrics(nodeID int64, collectionID int64, channel string) {
|
func CleanupDataNodeCollectionMetrics(nodeID int64, collectionID int64, channel string) {
|
||||||
|
|
|
@ -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)
|
|
||||||
}
|
|
|
@ -53,7 +53,7 @@ var (
|
||||||
prometheus.HistogramOpts{
|
prometheus.HistogramOpts{
|
||||||
Namespace: milvusNamespace,
|
Namespace: milvusNamespace,
|
||||||
Subsystem: typeutil.IndexNodeRole,
|
Subsystem: typeutil.IndexNodeRole,
|
||||||
Name: "build_index_latency",
|
Name: "knowhere_build_index_latency",
|
||||||
Help: "latency of building the index by knowhere",
|
Help: "latency of building the index by knowhere",
|
||||||
Buckets: buckets,
|
Buckets: buckets,
|
||||||
}, []string{nodeIDLabelName})
|
}, []string{nodeIDLabelName})
|
||||||
|
@ -75,6 +75,24 @@ var (
|
||||||
Help: "latency of saving the index file",
|
Help: "latency of saving the index file",
|
||||||
Buckets: buckets,
|
Buckets: buckets,
|
||||||
}, []string{nodeIDLabelName})
|
}, []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
|
// RegisterIndexNode registers IndexNode metrics
|
||||||
|
@ -85,4 +103,6 @@ func RegisterIndexNode(registry *prometheus.Registry) {
|
||||||
registry.MustRegister(IndexNodeKnowhereBuildIndexLatency)
|
registry.MustRegister(IndexNodeKnowhereBuildIndexLatency)
|
||||||
registry.MustRegister(IndexNodeEncodeIndexFileLatency)
|
registry.MustRegister(IndexNodeEncodeIndexFileLatency)
|
||||||
registry.MustRegister(IndexNodeSaveIndexFileLatency)
|
registry.MustRegister(IndexNodeSaveIndexFileLatency)
|
||||||
|
registry.MustRegister(IndexNodeIndexTaskLatencyInQueue)
|
||||||
|
registry.MustRegister(IndexNodeBuildIndexLatency)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 meta metrics
|
||||||
|
func RegisterMetaMetrics(registry *prometheus.Registry) {
|
||||||
|
registry.MustRegister(MetaKvSize)
|
||||||
|
registry.MustRegister(MetaRequestLatency)
|
||||||
|
registry.MustRegister(MetaOpCounter)
|
||||||
|
}
|
|
@ -86,4 +86,16 @@ var (
|
||||||
// buckets involves durations in milliseconds,
|
// 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]
|
// [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)
|
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})
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// Register serves prometheus http service
|
||||||
|
func Register(r *prometheus.Registry) {
|
||||||
|
r.MustRegister(NumNodes)
|
||||||
|
}
|
||||||
|
|
|
@ -30,6 +30,7 @@ func TestRegisterMetrics(t *testing.T) {
|
||||||
RegisterProxy(r.GoRegistry)
|
RegisterProxy(r.GoRegistry)
|
||||||
RegisterQueryNode(r.GoRegistry)
|
RegisterQueryNode(r.GoRegistry)
|
||||||
RegisterQueryCoord(r.GoRegistry)
|
RegisterQueryCoord(r.GoRegistry)
|
||||||
RegisterEtcdMetrics(r.GoRegistry)
|
RegisterMetaMetrics(r.GoRegistry)
|
||||||
RegisterMq(r.GoRegistry)
|
RegisterStorageMetrics(r.GoRegistry)
|
||||||
|
RegisterMsgStreamMetrics(r.GoRegistry)
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,8 +16,16 @@
|
||||||
|
|
||||||
package metrics
|
package metrics
|
||||||
|
|
||||||
import (
|
import "github.com/prometheus/client_golang/prometheus"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
|
||||||
|
const (
|
||||||
|
SendMsgLabel = "produce"
|
||||||
|
ReceiveMsgLabel = "consume" // not used
|
||||||
|
|
||||||
|
CreateProducerLabel = "create_producer"
|
||||||
|
CreateConsumerLabel = "create_consumer"
|
||||||
|
|
||||||
|
msgStreamOpType = "message_op_type"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
@ -31,8 +39,28 @@ var (
|
||||||
roleNameLabelName,
|
roleNameLabelName,
|
||||||
nodeIDLabelName,
|
nodeIDLabelName,
|
||||||
})
|
})
|
||||||
|
|
||||||
|
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})
|
||||||
)
|
)
|
||||||
|
|
||||||
func RegisterMq(registry *prometheus.Registry) {
|
// RegisterMsgStreamMetrics registers msg stream metrics
|
||||||
|
func RegisterMsgStreamMetrics(registry *prometheus.Registry) {
|
||||||
registry.MustRegister(NumConsumers)
|
registry.MustRegister(NumConsumers)
|
||||||
|
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)
|
||||||
|
}
|
|
@ -138,7 +138,7 @@ var (
|
||||||
Buckets: buckets, // unit: ms
|
Buckets: buckets, // unit: ms
|
||||||
}, []string{nodeIDLabelName, msgTypeLabelName})
|
}, []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(
|
ProxyCacheStatsCounter = prometheus.NewCounterVec(
|
||||||
prometheus.CounterOpts{
|
prometheus.CounterOpts{
|
||||||
Namespace: milvusNamespace,
|
Namespace: milvusNamespace,
|
||||||
|
@ -155,7 +155,7 @@ var (
|
||||||
Name: "cache_update_latency",
|
Name: "cache_update_latency",
|
||||||
Help: "latency that proxy update cache when cache miss",
|
Help: "latency that proxy update cache when cache miss",
|
||||||
Buckets: buckets, // unit: ms
|
Buckets: buckets, // unit: ms
|
||||||
}, []string{nodeIDLabelName})
|
}, []string{nodeIDLabelName, cacheNameLabelName})
|
||||||
|
|
||||||
// ProxySyncTimeTickLag record Proxy synchronization timestamp statistics, differentiated by Channel.
|
// ProxySyncTimeTickLag record Proxy synchronization timestamp statistics, differentiated by Channel.
|
||||||
ProxySyncTimeTickLag = prometheus.NewGaugeVec(
|
ProxySyncTimeTickLag = prometheus.NewGaugeVec(
|
||||||
|
|
|
@ -357,6 +357,30 @@ var (
|
||||||
nodeIDLabelName,
|
nodeIDLabelName,
|
||||||
channelNameLabelName,
|
channelNameLabelName,
|
||||||
})
|
})
|
||||||
|
|
||||||
|
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,
|
||||||
|
})
|
||||||
|
|
||||||
|
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,
|
||||||
|
})
|
||||||
)
|
)
|
||||||
|
|
||||||
// RegisterQueryNode registers QueryNode metrics
|
// RegisterQueryNode registers QueryNode metrics
|
||||||
|
@ -391,6 +415,8 @@ func RegisterQueryNode(registry *prometheus.Registry) {
|
||||||
registry.MustRegister(QueryNodeConsumerMsgCount)
|
registry.MustRegister(QueryNodeConsumerMsgCount)
|
||||||
registry.MustRegister(QueryNodeConsumeTimeTickLag)
|
registry.MustRegister(QueryNodeConsumeTimeTickLag)
|
||||||
registry.MustRegister(QueryNodeMsgDispatcherTtLag)
|
registry.MustRegister(QueryNodeMsgDispatcherTtLag)
|
||||||
|
registry.MustRegister(QueryNodeSegmentSearchLatencyPerVector)
|
||||||
|
registry.MustRegister(QueryNodeWatchDmlChannelLatency)
|
||||||
}
|
}
|
||||||
|
|
||||||
func CleanupQueryNodeCollectionMetrics(nodeID int64, collectionID int64) {
|
func CleanupQueryNodeCollectionMetrics(nodeID int64, collectionID int64) {
|
||||||
|
|
|
@ -168,6 +168,14 @@ var (
|
||||||
}, []string{
|
}, []string{
|
||||||
"quota_states",
|
"quota_states",
|
||||||
})
|
})
|
||||||
|
|
||||||
|
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})
|
||||||
)
|
)
|
||||||
|
|
||||||
// RegisterRootCoord registers RootCoord metrics
|
// RegisterRootCoord registers RootCoord metrics
|
||||||
|
@ -200,4 +208,5 @@ func RegisterRootCoord(registry *prometheus.Registry) {
|
||||||
registry.MustRegister(RootCoordNumOfRoles)
|
registry.MustRegister(RootCoordNumOfRoles)
|
||||||
registry.MustRegister(RootCoordTtDelay)
|
registry.MustRegister(RootCoordTtDelay)
|
||||||
registry.MustRegister(RootCoordQuotaStates)
|
registry.MustRegister(RootCoordQuotaStates)
|
||||||
|
registry.MustRegister(RootCoordDDLReqLatencyInQueue)
|
||||||
}
|
}
|
||||||
|
|
|
@ -1091,6 +1091,10 @@ func (t *MarshalFailTsMsg) Unmarshal(_ MarshalType) (TsMsg, error) {
|
||||||
return nil, errors.New("mocked error")
|
return nil, errors.New("mocked error")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (t *MarshalFailTsMsg) Size() int {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
var _ mqwrapper.Producer = (*mockSendFailProducer)(nil)
|
var _ mqwrapper.Producer = (*mockSendFailProducer)(nil)
|
||||||
|
|
||||||
type mockSendFailProducer struct {
|
type mockSendFailProducer struct {
|
||||||
|
|
|
@ -9,8 +9,10 @@ import (
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
var Producer *kafka.Producer
|
var Producer *kafka.Producer
|
||||||
|
@ -144,22 +146,37 @@ func (kc *kafkaClient) newConsumerConfig(group string, offset mqwrapper.Subscrip
|
||||||
}
|
}
|
||||||
|
|
||||||
func (kc *kafkaClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
|
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()
|
pp, err := kc.getKafkaProducer()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
|
||||||
|
|
||||||
deliveryChan := make(chan kafka.Event, 128)
|
deliveryChan := make(chan kafka.Event, 128)
|
||||||
producer := &kafkaProducer{p: pp, deliveryChan: deliveryChan, topic: options.Topic}
|
producer := &kafkaProducer{p: pp, deliveryChan: deliveryChan, topic: options.Topic}
|
||||||
return producer, nil
|
return producer, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (kc *kafkaClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
|
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)
|
config := kc.newConsumerConfig(options.SubscriptionName, options.SubscriptionInitialPosition)
|
||||||
consumer, err := newKafkaConsumer(config, options.Topic, options.SubscriptionName, options.SubscriptionInitialPosition)
|
consumer, err := newKafkaConsumer(config, options.Topic, options.SubscriptionName, options.SubscriptionInitialPosition)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
|
||||||
return consumer, nil
|
return consumer, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -11,7 +11,9 @@ import (
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/pkg/common"
|
"github.com/milvus-io/milvus/pkg/common"
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
type kafkaProducer struct {
|
type kafkaProducer struct {
|
||||||
|
@ -26,6 +28,9 @@ func (kp *kafkaProducer) Topic() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (kp *kafkaProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
|
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()
|
||||||
|
|
||||||
headers := make([]kafka.Header, 0, len(message.Properties))
|
headers := make([]kafka.Header, 0, len(message.Properties))
|
||||||
for key, value := range message.Properties {
|
for key, value := range message.Properties {
|
||||||
header := kafka.Header{Key: key, Value: []byte(value)}
|
header := kafka.Header{Key: key, Value: []byte(value)}
|
||||||
|
@ -38,20 +43,27 @@ func (kp *kafkaProducer) Send(ctx context.Context, message *mqwrapper.ProducerMe
|
||||||
}, kp.deliveryChan)
|
}, kp.deliveryChan)
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
e, ok := <-kp.deliveryChan
|
e, ok := <-kp.deliveryChan
|
||||||
if !ok {
|
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))
|
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"))
|
return nil, common.NewIgnorableError(fmt.Errorf("delivery chan of kafka producer is closed"))
|
||||||
}
|
}
|
||||||
|
|
||||||
m := e.(*kafka.Message)
|
m := e.(*kafka.Message)
|
||||||
if m.TopicPartition.Error != nil {
|
if m.TopicPartition.Error != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
|
||||||
return nil, m.TopicPartition.Error
|
return nil, m.TopicPartition.Error
|
||||||
}
|
}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
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
|
return &kafkaID{messageID: int64(m.TopicPartition.Offset)}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -25,7 +25,9 @@ import (
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
// nmqClient implements mqwrapper.Client.
|
// nmqClient implements mqwrapper.Client.
|
||||||
|
@ -55,9 +57,13 @@ func NewClient(url string, options ...nats.Option) (*nmqClient, error) {
|
||||||
|
|
||||||
// CreateProducer creates a producer for natsmq client
|
// CreateProducer creates a producer for natsmq client
|
||||||
func (nc *nmqClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
|
func (nc *nmqClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
|
||||||
|
start := timerecord.NewTimeRecorder("create producer")
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.TotalLabel).Inc()
|
||||||
|
|
||||||
// TODO: inject jetstream options.
|
// TODO: inject jetstream options.
|
||||||
js, err := nc.conn.JetStream()
|
js, err := nc.conn.JetStream()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, errors.Wrap(err, "failed to create jetstream context")
|
return nil, errors.Wrap(err, "failed to create jetstream context")
|
||||||
}
|
}
|
||||||
// TODO: (1) investigate on performance of multiple streams vs multiple topics.
|
// TODO: (1) investigate on performance of multiple streams vs multiple topics.
|
||||||
|
@ -67,23 +73,34 @@ func (nc *nmqClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrappe
|
||||||
Subjects: []string{options.Topic},
|
Subjects: []string{options.Topic},
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, errors.Wrap(err, "failed to add/connect to jetstream for producer")
|
return nil, errors.Wrap(err, "failed to add/connect to jetstream for producer")
|
||||||
}
|
}
|
||||||
rp := nmqProducer{js: js, topic: options.Topic}
|
rp := nmqProducer{js: js, topic: options.Topic}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
|
||||||
return &rp, nil
|
return &rp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (nc *nmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
|
func (nc *nmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
|
||||||
|
start := timerecord.NewTimeRecorder("create consumer")
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc()
|
||||||
|
|
||||||
if options.Topic == "" {
|
if options.Topic == "" {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, fmt.Errorf("invalid consumer config: empty topic")
|
return nil, fmt.Errorf("invalid consumer config: empty topic")
|
||||||
}
|
}
|
||||||
|
|
||||||
if options.SubscriptionName == "" {
|
if options.SubscriptionName == "" {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, fmt.Errorf("invalid consumer config: empty subscription name")
|
return nil, fmt.Errorf("invalid consumer config: empty subscription name")
|
||||||
}
|
}
|
||||||
// TODO: inject jetstream options.
|
// TODO: inject jetstream options.
|
||||||
js, err := nc.conn.JetStream()
|
js, err := nc.conn.JetStream()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, errors.Wrap(err, "failed to create jetstream context")
|
return nil, errors.Wrap(err, "failed to create jetstream context")
|
||||||
}
|
}
|
||||||
// TODO: do we allow passing in an existing natsChan from options?
|
// TODO: do we allow passing in an existing natsChan from options?
|
||||||
|
@ -95,6 +112,7 @@ func (nc *nmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Con
|
||||||
Subjects: []string{options.Topic},
|
Subjects: []string{options.Topic},
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, errors.Wrap(err, "failed to add/connect to jetstream for consumer")
|
return nil, errors.Wrap(err, "failed to add/connect to jetstream for consumer")
|
||||||
}
|
}
|
||||||
closeChan := make(chan struct{})
|
closeChan := make(chan struct{})
|
||||||
|
@ -109,9 +127,13 @@ func (nc *nmqClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Con
|
||||||
sub, err = js.ChanSubscribe(options.Topic, natsChan, nats.DeliverAll())
|
sub, err = js.ChanSubscribe(options.Topic, natsChan, nats.DeliverAll())
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, errors.Wrap(err, fmt.Sprintf("failed to get consumer info, subscribe position: %d", position))
|
return nil, errors.Wrap(err, fmt.Sprintf("failed to get consumer info, subscribe position: %d", position))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
|
||||||
return &Consumer{
|
return &Consumer{
|
||||||
js: js,
|
js: js,
|
||||||
sub: sub,
|
sub: sub,
|
||||||
|
|
|
@ -20,7 +20,9 @@ import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
"github.com/nats-io/nats.go"
|
"github.com/nats-io/nats.go"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
@ -40,6 +42,9 @@ func (np *nmqProducer) Topic() string {
|
||||||
|
|
||||||
// Send send the producer messages to natsmq
|
// Send send the producer messages to natsmq
|
||||||
func (np *nmqProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
|
func (np *nmqProducer) 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()
|
||||||
|
|
||||||
// Encode message
|
// Encode message
|
||||||
msg := &nats.Msg{
|
msg := &nats.Msg{
|
||||||
Subject: np.topic,
|
Subject: np.topic,
|
||||||
|
@ -53,9 +58,14 @@ func (np *nmqProducer) Send(ctx context.Context, message *mqwrapper.ProducerMess
|
||||||
// publish to nats-server
|
// publish to nats-server
|
||||||
pa, err := np.js.PublishMsg(msg)
|
pa, err := np.js.PublishMsg(msg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.FailLabel).Inc()
|
||||||
log.Warn("failed to publish message by nmq", zap.String("topic", np.topic), zap.Error(err), zap.Int("payload_size", len(message.Payload)))
|
log.Warn("failed to publish message by nmq", zap.String("topic", np.topic), zap.Error(err), zap.Int("payload_size", len(message.Payload)))
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.SendMsgLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.SuccessLabel).Inc()
|
||||||
return &nmqID{messageID: pa.Sequence}, err
|
return &nmqID{messageID: pa.Sequence}, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -29,8 +29,10 @@ import (
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
type pulsarClient struct {
|
type pulsarClient struct {
|
||||||
|
@ -64,8 +66,12 @@ func NewClient(tenant string, namespace string, opts pulsar.ClientOptions) (*pul
|
||||||
|
|
||||||
// CreateProducer create a pulsar producer from options
|
// CreateProducer create a pulsar producer from options
|
||||||
func (pc *pulsarClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwrapper.Producer, error) {
|
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)
|
fullTopicName, err := GetFullTopicName(pc.tenant, pc.namespace, options.Topic)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
opts := pulsar.ProducerOptions{Topic: fullTopicName}
|
opts := pulsar.ProducerOptions{Topic: fullTopicName}
|
||||||
|
@ -80,20 +86,29 @@ func (pc *pulsarClient) CreateProducer(options mqwrapper.ProducerOptions) (mqwra
|
||||||
|
|
||||||
pp, err := pc.client.CreateProducer(opts)
|
pp, err := pc.client.CreateProducer(opts)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
if pp == nil {
|
if pp == nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, errors.New("pulsar is not ready, producer is nil")
|
return nil, errors.New("pulsar is not ready, producer is nil")
|
||||||
}
|
}
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateProducerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.SuccessLabel).Inc()
|
||||||
producer := &pulsarProducer{p: pp}
|
producer := &pulsarProducer{p: pp}
|
||||||
return producer, nil
|
return producer, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Subscribe creates a pulsar consumer instance and subscribe a topic
|
// Subscribe creates a pulsar consumer instance and subscribe a topic
|
||||||
func (pc *pulsarClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) {
|
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)
|
receiveChannel := make(chan pulsar.ConsumerMessage, options.BufSize)
|
||||||
fullTopicName, err := GetFullTopicName(pc.tenant, pc.namespace, options.Topic)
|
fullTopicName, err := GetFullTopicName(pc.tenant, pc.namespace, options.Topic)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
consumer, err := pc.client.Subscribe(pulsar.ConsumerOptions{
|
consumer, err := pc.client.Subscribe(pulsar.ConsumerOptions{
|
||||||
|
@ -104,6 +119,7 @@ func (pc *pulsarClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.
|
||||||
MessageChannel: receiveChannel,
|
MessageChannel: receiveChannel,
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc()
|
||||||
if strings.Contains(err.Error(), "ConsumerBusy") {
|
if strings.Contains(err.Error(), "ConsumerBusy") {
|
||||||
return nil, retry.Unrecoverable(err)
|
return nil, retry.Unrecoverable(err)
|
||||||
}
|
}
|
||||||
|
@ -116,6 +132,9 @@ func (pc *pulsarClient) Subscribe(options mqwrapper.ConsumerOptions) (mqwrapper.
|
||||||
pConsumer.AtLatest = true
|
pConsumer.AtLatest = true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
elapsed := start.ElapseSpan()
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.CreateConsumerLabel).Observe(float64(elapsed.Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.SuccessLabel).Inc()
|
||||||
return pConsumer, nil
|
return pConsumer, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,9 @@ import (
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
"github.com/milvus-io/milvus/pkg/mq/msgstream/mqwrapper"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||||
)
|
)
|
||||||
|
|
||||||
// implementation assertion
|
// implementation assertion
|
||||||
|
@ -37,9 +39,19 @@ func (pp *pulsarProducer) Topic() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pp *pulsarProducer) Send(ctx context.Context, message *mqwrapper.ProducerMessage) (mqwrapper.MessageID, error) {
|
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}
|
ppm := &pulsar.ProducerMessage{Payload: message.Payload, Properties: message.Properties}
|
||||||
pmID, err := pp.p.Send(ctx, ppm)
|
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
|
||||||
|
}
|
||||||
|
|
||||||
|
metrics.MsgStreamRequestLatency.WithLabelValues(metrics.SendMsgLabel).Observe(float64(start.ElapseSpan().Milliseconds()))
|
||||||
|
metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.SuccessLabel).Inc()
|
||||||
|
return &pulsarID{messageID: pmID}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pp *pulsarProducer) Close() {
|
func (pp *pulsarProducer) Close() {
|
||||||
|
|
|
@ -52,6 +52,7 @@ type TsMsg interface {
|
||||||
Unmarshal(MarshalType) (TsMsg, error)
|
Unmarshal(MarshalType) (TsMsg, error)
|
||||||
Position() *MsgPosition
|
Position() *MsgPosition
|
||||||
SetPosition(*MsgPosition)
|
SetPosition(*MsgPosition)
|
||||||
|
Size() int
|
||||||
}
|
}
|
||||||
|
|
||||||
// BaseMsg is a basic structure that contains begin timestamp, end timestamp and the position of msgstream
|
// BaseMsg is a basic structure that contains begin timestamp, end timestamp and the position of msgstream
|
||||||
|
@ -289,6 +290,10 @@ func (it *InsertMsg) IndexMsg(index int) *InsertMsg {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (it *InsertMsg) Size() int {
|
||||||
|
return proto.Size(&it.InsertRequest)
|
||||||
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////Delete//////////////////////////////////////////
|
/////////////////////////////////////////Delete//////////////////////////////////////////
|
||||||
|
|
||||||
// DeleteMsg is a message pack that contains delete request
|
// DeleteMsg is a message pack that contains delete request
|
||||||
|
@ -388,6 +393,10 @@ func (dt *DeleteMsg) CheckAligned() error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (dt *DeleteMsg) Size() int {
|
||||||
|
return proto.Size(&dt.DeleteRequest)
|
||||||
|
}
|
||||||
|
|
||||||
// ///////////////////////////////////////Upsert//////////////////////////////////////////
|
// ///////////////////////////////////////Upsert//////////////////////////////////////////
|
||||||
type UpsertMsg struct {
|
type UpsertMsg struct {
|
||||||
InsertMsg *InsertMsg
|
InsertMsg *InsertMsg
|
||||||
|
@ -454,6 +463,10 @@ func (tst *TimeTickMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
||||||
return timeTick, nil
|
return timeTick, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (tst *TimeTickMsg) Size() int {
|
||||||
|
return proto.Size(&tst.TimeTickMsg)
|
||||||
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////CreateCollection//////////////////////////////////////////
|
/////////////////////////////////////////CreateCollection//////////////////////////////////////////
|
||||||
|
|
||||||
// CreateCollectionMsg is a message pack that contains create collection request
|
// CreateCollectionMsg is a message pack that contains create collection request
|
||||||
|
@ -514,6 +527,10 @@ func (cc *CreateCollectionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
||||||
return createCollectionMsg, nil
|
return createCollectionMsg, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (cc *CreateCollectionMsg) Size() int {
|
||||||
|
return proto.Size(&cc.CreateCollectionRequest)
|
||||||
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////DropCollection//////////////////////////////////////////
|
/////////////////////////////////////////DropCollection//////////////////////////////////////////
|
||||||
|
|
||||||
// DropCollectionMsg is a message pack that contains drop collection request
|
// DropCollectionMsg is a message pack that contains drop collection request
|
||||||
|
@ -574,6 +591,10 @@ func (dc *DropCollectionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
||||||
return dropCollectionMsg, nil
|
return dropCollectionMsg, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (dc *DropCollectionMsg) Size() int {
|
||||||
|
return proto.Size(&dc.DropCollectionRequest)
|
||||||
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////CreatePartition//////////////////////////////////////////
|
/////////////////////////////////////////CreatePartition//////////////////////////////////////////
|
||||||
|
|
||||||
// CreatePartitionMsg is a message pack that contains create partition request
|
// CreatePartitionMsg is a message pack that contains create partition request
|
||||||
|
@ -634,6 +655,10 @@ func (cp *CreatePartitionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
||||||
return createPartitionMsg, nil
|
return createPartitionMsg, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (cp *CreatePartitionMsg) Size() int {
|
||||||
|
return proto.Size(&cp.CreatePartitionRequest)
|
||||||
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////DropPartition//////////////////////////////////////////
|
/////////////////////////////////////////DropPartition//////////////////////////////////////////
|
||||||
|
|
||||||
// DropPartitionMsg is a message pack that contains drop partition request
|
// DropPartitionMsg is a message pack that contains drop partition request
|
||||||
|
@ -694,6 +719,10 @@ func (dp *DropPartitionMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
||||||
return dropPartitionMsg, nil
|
return dropPartitionMsg, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (dp *DropPartitionMsg) Size() int {
|
||||||
|
return proto.Size(&dp.DropPartitionRequest)
|
||||||
|
}
|
||||||
|
|
||||||
/////////////////////////////////////////DataNodeTtMsg//////////////////////////////////////////
|
/////////////////////////////////////////DataNodeTtMsg//////////////////////////////////////////
|
||||||
|
|
||||||
// DataNodeTtMsg is a message pack that contains datanode time tick
|
// DataNodeTtMsg is a message pack that contains datanode time tick
|
||||||
|
@ -750,3 +779,7 @@ func (m *DataNodeTtMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
||||||
DataNodeTtMsg: msg,
|
DataNodeTtMsg: msg,
|
||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *DataNodeTtMsg) Size() int {
|
||||||
|
return proto.Size(&m.DataNodeTtMsg)
|
||||||
|
}
|
||||||
|
|
Loading…
Reference in New Issue