feat: add tasks page into management WebUI (#37002)

issue: #36621

1. Add API to access task runtime metrics, including:
  - build index task
  - compaction task
  - import task
- balance (including load/release of segments/channels and some leader
tasks on querycoord)
  - sync task
2. Add a debug model to the webpage by using debug=true or debug=false
in the URL query parameters to enable or disable debug mode.

Signed-off-by: jaime <yun.zhang@zilliz.com>
This commit is contained in:
jaime 2024-10-28 10:13:29 +08:00 committed by GitHub
parent d7b2ffe5aa
commit 9d16b972ea
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
84 changed files with 2887 additions and 1159 deletions

2
go.mod
View File

@ -63,6 +63,7 @@ require (
github.com/bytedance/sonic v1.12.2
github.com/cenkalti/backoff/v4 v4.2.1
github.com/cockroachdb/redact v1.1.3
github.com/goccy/go-json v0.10.3
github.com/greatroar/blobloom v0.0.0-00010101000000-000000000000
github.com/hashicorp/golang-lru/v2 v2.0.7
github.com/jolestar/go-commons-pool/v2 v2.1.2
@ -131,7 +132,6 @@ require (
github.com/go-ole/go-ole v1.2.6 // indirect
github.com/go-playground/locales v0.14.1 // indirect
github.com/go-playground/universal-translator v0.18.1 // indirect
github.com/goccy/go-json v0.10.3 // indirect
github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect
github.com/godbus/dbus/v5 v5.0.4 // indirect
github.com/gogo/protobuf v1.3.2 // indirect

View File

@ -50,7 +50,7 @@ func (s *L0CompactionTaskSuite) SetupTest() {
s.mockMeta = NewMockCompactionMeta(s.T())
s.mockSessMgr = session.NewMockDataNodeManager(s.T())
s.mockAlloc = allocator.NewMockAllocator(s.T())
//s.mockMeta.EXPECT().SaveCompactionTask(mock.Anything).Return(nil)
// s.mockMeta.EXPECT().SaveCompactionTask(mock.Anything).Return(nil)
}
func (s *L0CompactionTaskSuite) SetupSubTest() {

View File

@ -18,23 +18,43 @@ package datacoord
import (
"context"
"encoding/json"
"sync"
"time"
"github.com/hashicorp/golang-lru/v2/expirable"
"go.uber.org/zap"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
func newCompactionTaskStats(task *datapb.CompactionTask) *metricsinfo.CompactionTask {
return &metricsinfo.CompactionTask{
PlanID: task.PlanID,
CollectionID: task.CollectionID,
Type: task.Type.String(),
State: task.State.String(),
FailReason: task.FailReason,
StartTime: task.StartTime,
EndTime: task.EndTime,
TotalRows: task.TotalRows,
InputSegments: task.InputSegments,
ResultSegments: task.ResultSegments,
}
}
type compactionTaskMeta struct {
sync.RWMutex
ctx context.Context
catalog metastore.DataCoordCatalog
// currently only clustering compaction task is stored in persist meta
compactionTasks map[int64]map[int64]*datapb.CompactionTask // triggerID -> planID
taskStats *expirable.LRU[UniqueID, *metricsinfo.CompactionTask]
}
func newCompactionTaskMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (*compactionTaskMeta, error) {
@ -43,6 +63,7 @@ func newCompactionTaskMeta(ctx context.Context, catalog metastore.DataCoordCatal
ctx: ctx,
catalog: catalog,
compactionTasks: make(map[int64]map[int64]*datapb.CompactionTask, 0),
taskStats: expirable.NewLRU[UniqueID, *metricsinfo.CompactionTask](1024, nil, time.Minute*60),
}
if err := csm.reloadFromKV(); err != nil {
return nil, err
@ -125,16 +146,17 @@ func (csm *compactionTaskMeta) SaveCompactionTask(task *datapb.CompactionTask) e
log.Error("meta update: update compaction task fail", zap.Error(err))
return err
}
return csm.saveCompactionTaskMemory(task)
csm.saveCompactionTaskMemory(task)
return nil
}
func (csm *compactionTaskMeta) saveCompactionTaskMemory(task *datapb.CompactionTask) error {
func (csm *compactionTaskMeta) saveCompactionTaskMemory(task *datapb.CompactionTask) {
_, triggerIDExist := csm.compactionTasks[task.TriggerID]
if !triggerIDExist {
csm.compactionTasks[task.TriggerID] = make(map[int64]*datapb.CompactionTask, 0)
}
csm.compactionTasks[task.TriggerID][task.PlanID] = task
return nil
csm.taskStats.Add(task.PlanID, newCompactionTaskStats(task))
}
func (csm *compactionTaskMeta) DropCompactionTask(task *datapb.CompactionTask) error {
@ -153,3 +175,16 @@ func (csm *compactionTaskMeta) DropCompactionTask(task *datapb.CompactionTask) e
}
return nil
}
func (csm *compactionTaskMeta) TaskStatsJSON() string {
tasks := csm.taskStats.Values()
if len(tasks) == 0 {
return ""
}
ret, err := json.Marshal(tasks)
if err != nil {
return ""
}
return string(ret)
}

View File

@ -18,13 +18,16 @@ package datacoord
import (
"context"
"encoding/json"
"testing"
"time"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
)
func TestCompactionTaskMetaSuite(t *testing.T) {
@ -79,3 +82,49 @@ func (suite *CompactionTaskMetaSuite) TestGetCompactionTasksByCollectionAbnormal
res := suite.meta.GetCompactionTasksByCollection(101)
suite.Equal(1, len(res))
}
func (suite *CompactionTaskMetaSuite) TestTaskStatsJSON() {
task1 := &datapb.CompactionTask{
PlanID: 1,
CollectionID: 100,
Type: datapb.CompactionType_MergeCompaction,
State: datapb.CompactionTaskState_completed,
FailReason: "",
StartTime: time.Now().Unix(),
EndTime: time.Now().Add(time.Hour).Unix(),
TotalRows: 1000,
InputSegments: []int64{1, 2},
ResultSegments: []int64{3},
}
task2 := &datapb.CompactionTask{
PlanID: 2,
CollectionID: 101,
Type: datapb.CompactionType_MergeCompaction,
State: datapb.CompactionTaskState_completed,
FailReason: "",
StartTime: time.Now().Unix(),
EndTime: time.Now().Add(time.Hour).Unix(),
TotalRows: 2000,
InputSegments: []int64{4, 5},
ResultSegments: []int64{6},
}
// testing return empty string
actualJSON := suite.meta.TaskStatsJSON()
suite.Equal("", actualJSON)
err := suite.meta.SaveCompactionTask(task1)
suite.NoError(err)
err = suite.meta.SaveCompactionTask(task2)
suite.NoError(err)
expectedTasks := []*metricsinfo.CompactionTask{
newCompactionTaskStats(task1),
newCompactionTaskStats(task2),
}
expectedJSON, err := json.Marshal(expectedTasks)
suite.NoError(err)
actualJSON = suite.meta.TaskStatsJSON()
suite.JSONEq(string(expectedJSON), actualJSON)
}

View File

@ -133,7 +133,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
plans []*datapb.CompactionPlan
expectedOut []UniqueID // planID
}{
{"with L0 tasks diff channel",
{
"with L0 tasks diff channel",
[]CompactionTask{
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 10,
@ -156,7 +157,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
},
[]UniqueID{10, 11},
},
{"with L0 tasks same channel",
{
"with L0 tasks same channel",
[]CompactionTask{
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 11,
@ -179,7 +181,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
},
[]UniqueID{10},
},
{"without L0 tasks",
{
"without L0 tasks",
[]CompactionTask{
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 14,
@ -202,10 +205,12 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
},
[]UniqueID{13, 14},
},
{"empty tasks",
{
"empty tasks",
[]CompactionTask{},
[]*datapb.CompactionPlan{},
[]UniqueID{}},
[]UniqueID{},
},
}
for _, test := range tests {
@ -235,7 +240,8 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
plans []*datapb.CompactionPlan
expectedOut []UniqueID // planID
}{
{"with L0 tasks diff channel",
{
"with L0 tasks diff channel",
[]CompactionTask{
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 10,
@ -255,29 +261,31 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
[]*datapb.CompactionPlan{{}, {}},
[]UniqueID{10, 11},
},
{"with L0 tasks same channel", []CompactionTask{
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 10,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 11,
Type: datapb.CompactionType_MixCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 13,
Type: datapb.CompactionType_MixCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-3",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
},
{
"with L0 tasks same channel",
[]CompactionTask{
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 10,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 11,
Type: datapb.CompactionType_MixCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 13,
Type: datapb.CompactionType_MixCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-3",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
},
[]*datapb.CompactionPlan{
{PlanID: 10, Channel: "ch-3", Type: datapb.CompactionType_Level0DeleteCompaction},
{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction},
@ -285,29 +293,31 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
},
[]UniqueID{10, 13},
},
{"with multiple L0 tasks same channel", []CompactionTask{
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 10,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 11,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 12,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
},
{
"with multiple L0 tasks same channel",
[]CompactionTask{
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 10,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 11,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newL0CompactionTask(&datapb.CompactionTask{
PlanID: 12,
Type: datapb.CompactionType_Level0DeleteCompaction,
State: datapb.CompactionTaskState_pipelining,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
},
[]*datapb.CompactionPlan{
{PlanID: 10, Channel: "ch-3", Type: datapb.CompactionType_Level0DeleteCompaction},
{PlanID: 11, Channel: "ch-3", Type: datapb.CompactionType_Level0DeleteCompaction},
@ -315,25 +325,28 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
},
[]UniqueID{10, 11, 12},
},
{"without L0 tasks", []CompactionTask{
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 14,
Type: datapb.CompactionType_MixCompaction,
Channel: "ch-3",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 13,
Type: datapb.CompactionType_MixCompaction,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
},
{
"without L0 tasks",
[]CompactionTask{
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 14,
Type: datapb.CompactionType_MixCompaction,
Channel: "ch-3",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
newMixCompactionTask(&datapb.CompactionTask{
PlanID: 13,
Type: datapb.CompactionType_MixCompaction,
Channel: "ch-11",
NodeID: 102,
}, nil, s.mockMeta, s.mockSessMgr),
},
[]*datapb.CompactionPlan{
{PlanID: 14, Channel: "ch-3", Type: datapb.CompactionType_MixCompaction},
{},
},
[]UniqueID{13, 14}},
[]UniqueID{13, 14},
},
{"empty tasks", []CompactionTask{}, []*datapb.CompactionPlan{}, []UniqueID{}},
}

View File

@ -416,7 +416,7 @@ func createMetaForRecycleUnusedIndexes(catalog metastore.DataCoordCatalog) *meta
},
},
},
buildID2SegmentIndex: nil,
segmentBuildInfo: newSegmentIndexBuildInfo(),
},
}
}
@ -524,47 +524,47 @@ func createMetaForRecycleUnusedSegIndexes(catalog metastore.DataCoordCatalog) *m
},
},
},
indexes: map[UniqueID]map[UniqueID]*model.Index{},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
},
indexes: map[UniqueID]map[UniqueID]*model.Index{},
segmentBuildInfo: newSegmentIndexBuildInfo(),
},
channelCPs: nil,
chunkManager: nil,
}
meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
})
meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
})
for id, segment := range segments {
meta.segments.SetSegment(id, segment)
}
@ -706,45 +706,43 @@ func createMetaTableForRecycleUnusedIndexFiles(catalog *datacoord.Catalog) *meta
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
segmentBuildInfo: newSegmentIndexBuildInfo(),
},
}
meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
})
meta.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
})
for id, segment := range segments {
meta.segments.SetSegment(id, segment)
}
@ -1091,43 +1089,7 @@ func TestGarbageCollector_clearETCD(t *testing.T) {
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 1024,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file3", "file4"},
IndexSize: 1024,
WriteHandoff: false,
},
},
segmentBuildInfo: newSegmentIndexBuildInfo(),
indexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
@ -1175,6 +1137,42 @@ func TestGarbageCollector_clearETCD(t *testing.T) {
},
}
m.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 1024,
WriteHandoff: false,
})
m.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file3", "file4"},
IndexSize: 1024,
WriteHandoff: false,
})
for id, segment := range segments {
m.segments.SetSegment(id, segment)
}

View File

@ -17,6 +17,12 @@
package datacoord
import (
"encoding/json"
"time"
"github.com/hashicorp/golang-lru/v2/expirable"
"golang.org/x/exp/maps"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/pkg/util/lock"
"github.com/milvus-io/milvus/pkg/util/timerecord"
@ -35,13 +41,54 @@ type ImportMeta interface {
GetTask(taskID int64) ImportTask
GetTaskBy(filters ...ImportTaskFilter) []ImportTask
RemoveTask(taskID int64) error
TaskStatsJSON() string
}
type importTasks struct {
tasks map[int64]ImportTask
taskStats *expirable.LRU[int64, ImportTask]
}
func newImportTasks() *importTasks {
return &importTasks{
tasks: make(map[int64]ImportTask),
taskStats: expirable.NewLRU[UniqueID, ImportTask](4096, nil, time.Minute*60),
}
}
func (t *importTasks) get(taskID int64) ImportTask {
ret, ok := t.tasks[taskID]
if !ok {
return nil
}
return ret
}
func (t *importTasks) add(task ImportTask) {
t.tasks[task.GetTaskID()] = task
t.taskStats.Add(task.GetTaskID(), task)
}
func (t *importTasks) remove(taskID int64) {
task, ok := t.tasks[taskID]
if ok {
delete(t.tasks, taskID)
t.taskStats.Add(task.GetTaskID(), task)
}
}
func (t *importTasks) listTasks() []ImportTask {
return maps.Values(t.tasks)
}
func (t *importTasks) listTaskStats() []ImportTask {
return t.taskStats.Values()
}
type importMeta struct {
mu lock.RWMutex // guards jobs and tasks
jobs map[int64]ImportJob
tasks map[int64]ImportTask
mu lock.RWMutex // guards jobs and tasks
jobs map[int64]ImportJob
tasks *importTasks
catalog metastore.DataCoordCatalog
}
@ -59,18 +106,19 @@ func NewImportMeta(catalog metastore.DataCoordCatalog) (ImportMeta, error) {
return nil, err
}
tasks := make(map[int64]ImportTask)
tasks := newImportTasks()
for _, task := range restoredPreImportTasks {
tasks[task.GetTaskID()] = &preImportTask{
tasks.add(&preImportTask{
PreImportTask: task,
tr: timerecord.NewTimeRecorder("preimport task"),
}
})
}
for _, task := range restoredImportTasks {
tasks[task.GetTaskID()] = &importTask{
tasks.add(&importTask{
ImportTaskV2: task,
tr: timerecord.NewTimeRecorder("import task"),
}
})
}
jobs := make(map[int64]ImportJob)
@ -170,13 +218,13 @@ func (m *importMeta) AddTask(task ImportTask) error {
if err != nil {
return err
}
m.tasks[task.GetTaskID()] = task
m.tasks.add(task)
case ImportTaskType:
err := m.catalog.SaveImportTask(task.(*importTask).ImportTaskV2)
if err != nil {
return err
}
m.tasks[task.GetTaskID()] = task
m.tasks.add(task)
}
return nil
}
@ -184,7 +232,7 @@ func (m *importMeta) AddTask(task ImportTask) error {
func (m *importMeta) UpdateTask(taskID int64, actions ...UpdateAction) error {
m.mu.Lock()
defer m.mu.Unlock()
if task, ok := m.tasks[taskID]; ok {
if task := m.tasks.get(taskID); task != nil {
updatedTask := task.Clone()
for _, action := range actions {
action(updatedTask)
@ -195,13 +243,13 @@ func (m *importMeta) UpdateTask(taskID int64, actions ...UpdateAction) error {
if err != nil {
return err
}
m.tasks[updatedTask.GetTaskID()] = updatedTask
m.tasks.add(updatedTask)
case ImportTaskType:
err := m.catalog.SaveImportTask(updatedTask.(*importTask).ImportTaskV2)
if err != nil {
return err
}
m.tasks[updatedTask.GetTaskID()] = updatedTask
m.tasks.add(updatedTask)
}
}
@ -211,7 +259,7 @@ func (m *importMeta) UpdateTask(taskID int64, actions ...UpdateAction) error {
func (m *importMeta) GetTask(taskID int64) ImportTask {
m.mu.RLock()
defer m.mu.RUnlock()
return m.tasks[taskID]
return m.tasks.get(taskID)
}
func (m *importMeta) GetTaskBy(filters ...ImportTaskFilter) []ImportTask {
@ -219,7 +267,7 @@ func (m *importMeta) GetTaskBy(filters ...ImportTaskFilter) []ImportTask {
defer m.mu.RUnlock()
ret := make([]ImportTask, 0)
OUTER:
for _, task := range m.tasks {
for _, task := range m.tasks.listTasks() {
for _, f := range filters {
if !f(task) {
continue OUTER
@ -233,7 +281,7 @@ OUTER:
func (m *importMeta) RemoveTask(taskID int64) error {
m.mu.Lock()
defer m.mu.Unlock()
if task, ok := m.tasks[taskID]; ok {
if task := m.tasks.get(taskID); task != nil {
switch task.GetType() {
case PreImportTaskType:
err := m.catalog.DropPreImportTask(taskID)
@ -246,7 +294,20 @@ func (m *importMeta) RemoveTask(taskID int64) error {
return err
}
}
delete(m.tasks, taskID)
m.tasks.remove(taskID)
}
return nil
}
func (m *importMeta) TaskStatsJSON() string {
tasks := m.tasks.listTaskStats()
if len(tasks) == 0 {
return ""
}
ret, err := json.Marshal(tasks)
if err != nil {
return ""
}
return string(ret)
}

View File

@ -17,6 +17,7 @@
package datacoord
import (
"encoding/json"
"fmt"
"math/rand"
"testing"
@ -28,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
)
func TestImportMeta_Restore(t *testing.T) {
@ -231,9 +233,54 @@ func TestImportMeta_Task_Failed(t *testing.T) {
err = im.AddTask(task)
assert.Error(t, err)
im.(*importMeta).tasks[task.GetTaskID()] = task
im.(*importMeta).tasks.add(task)
err = im.UpdateTask(task.GetTaskID(), UpdateNodeID(9))
assert.Error(t, err)
err = im.RemoveTask(task.GetTaskID())
assert.Error(t, err)
}
func TestTaskStatsJSON(t *testing.T) {
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListImportJobs().Return(nil, nil)
catalog.EXPECT().ListPreImportTasks().Return(nil, nil)
catalog.EXPECT().ListImportTasks().Return(nil, nil)
catalog.EXPECT().SaveImportTask(mock.Anything).Return(nil)
im, err := NewImportMeta(catalog)
assert.NoError(t, err)
statsJSON := im.TaskStatsJSON()
assert.Equal(t, "", statsJSON)
task1 := &importTask{
ImportTaskV2: &datapb.ImportTaskV2{
TaskID: 1,
},
}
err = im.AddTask(task1)
assert.NoError(t, err)
task2 := &importTask{
ImportTaskV2: &datapb.ImportTaskV2{
TaskID: 2,
},
}
err = im.AddTask(task2)
assert.NoError(t, err)
err = im.UpdateTask(1, UpdateState(datapb.ImportTaskStateV2_Completed))
assert.NoError(t, err)
statsJSON = im.TaskStatsJSON()
var tasks []*metricsinfo.ImportTask
err = json.Unmarshal([]byte(statsJSON), &tasks)
assert.NoError(t, err)
assert.Equal(t, 2, len(tasks))
taskMeta := im.(*importMeta).tasks
taskMeta.remove(1)
assert.Nil(t, taskMeta.get(1))
assert.NotNil(t, taskMeta.get(2))
assert.Equal(t, 2, len(taskMeta.listTaskStats()))
}

View File

@ -17,10 +17,13 @@
package datacoord
import (
"encoding/json"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
@ -92,8 +95,11 @@ func UpdateReason(reason string) UpdateAction {
func UpdateCompleteTime(completeTime string) UpdateAction {
return func(t ImportTask) {
if task, ok := t.(*importTask); ok {
task.ImportTaskV2.CompleteTime = completeTime
switch t.GetType() {
case PreImportTaskType:
t.(*preImportTask).PreImportTask.CompleteTime = completeTime
case ImportTaskType:
t.(*importTask).ImportTaskV2.CompleteTime = completeTime
}
}
}
@ -171,6 +177,21 @@ func (p *preImportTask) Clone() ImportTask {
}
}
func (p *preImportTask) MarshalJSON() ([]byte, error) {
importTask := metricsinfo.ImportTask{
JobID: p.GetJobID(),
TaskID: p.GetTaskID(),
CollectionID: p.GetCollectionID(),
NodeID: p.GetNodeID(),
State: p.GetState().String(),
Reason: p.GetReason(),
TaskType: "PreImportTask",
CreatedTime: p.GetCreatedTime(),
CompleteTime: p.GetCompleteTime(),
}
return json.Marshal(importTask)
}
type importTask struct {
*datapb.ImportTaskV2
tr *timerecord.TimeRecorder
@ -201,3 +222,18 @@ func (t *importTask) Clone() ImportTask {
tr: t.tr,
}
}
func (t *importTask) MarshalJSON() ([]byte, error) {
importTask := metricsinfo.ImportTask{
JobID: t.GetJobID(),
TaskID: t.GetTaskID(),
CollectionID: t.GetCollectionID(),
NodeID: t.GetNodeID(),
State: t.GetState().String(),
Reason: t.GetReason(),
TaskType: "ImportTask",
CreatedTime: t.GetCreatedTime(),
CompleteTime: t.GetCompleteTime(),
}
return json.Marshal(importTask)
}

View File

@ -76,6 +76,7 @@ func NewPreImportTasks(fileGroups [][]*internalpb.ImportFile,
CollectionID: job.GetCollectionID(),
State: datapb.ImportTaskStateV2_Pending,
FileStats: fileStats,
CreatedTime: time.Now().Format("2006-01-02T15:04:05Z07:00"),
},
tr: timerecord.NewTimeRecorder("preimport task"),
}
@ -101,6 +102,7 @@ func NewImportTasks(fileGroups [][]*datapb.ImportFileStats,
NodeID: NullNodeID,
State: datapb.ImportTaskStateV2_Pending,
FileStats: group,
CreatedTime: time.Now().Format("2006-01-02T15:04:05Z07:00"),
},
tr: timerecord.NewTimeRecorder("import task"),
}

View File

@ -18,10 +18,12 @@ package datacoord
import (
"context"
"encoding/json"
"fmt"
"math/rand"
"path"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
@ -39,7 +41,9 @@ import (
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/importutilv2"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
func TestImportUtil_NewPreImportTasks(t *testing.T) {
@ -645,3 +649,69 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
assert.Equal(t, internalpb.ImportJobState_Completed, state)
assert.Equal(t, "", reason)
}
func TestPreImportTask_MarshalJSON(t *testing.T) {
task := &preImportTask{
PreImportTask: &datapb.PreImportTask{
JobID: 1,
TaskID: 2,
CollectionID: 3,
NodeID: 4,
State: datapb.ImportTaskStateV2_Pending,
Reason: "test reason",
CreatedTime: time.Now().Format(time.RFC3339),
CompleteTime: time.Now().Add(time.Hour).Format(time.RFC3339),
},
tr: timerecord.NewTimeRecorder("test"),
}
jsonData, err := task.MarshalJSON()
assert.NoError(t, err)
var importTask metricsinfo.ImportTask
err = json.Unmarshal(jsonData, &importTask)
assert.NoError(t, err)
assert.Equal(t, task.GetJobID(), importTask.JobID)
assert.Equal(t, task.GetTaskID(), importTask.TaskID)
assert.Equal(t, task.GetCollectionID(), importTask.CollectionID)
assert.Equal(t, task.GetNodeID(), importTask.NodeID)
assert.Equal(t, task.GetState().String(), importTask.State)
assert.Equal(t, task.GetReason(), importTask.Reason)
assert.Equal(t, "PreImportTask", importTask.TaskType)
assert.Equal(t, task.GetCreatedTime(), importTask.CreatedTime)
assert.Equal(t, task.GetCompleteTime(), importTask.CompleteTime)
}
func TestImportTask_MarshalJSON(t *testing.T) {
task := &importTask{
ImportTaskV2: &datapb.ImportTaskV2{
JobID: 1,
TaskID: 2,
CollectionID: 3,
NodeID: 4,
State: datapb.ImportTaskStateV2_Pending,
Reason: "test reason",
CreatedTime: time.Now().Format(time.RFC3339),
CompleteTime: time.Now().Add(time.Hour).Format(time.RFC3339),
},
tr: timerecord.NewTimeRecorder("test"),
}
jsonData, err := task.MarshalJSON()
assert.NoError(t, err)
var importTask metricsinfo.ImportTask
err = json.Unmarshal(jsonData, &importTask)
assert.NoError(t, err)
assert.Equal(t, task.GetJobID(), importTask.JobID)
assert.Equal(t, task.GetTaskID(), importTask.TaskID)
assert.Equal(t, task.GetCollectionID(), importTask.CollectionID)
assert.Equal(t, task.GetNodeID(), importTask.NodeID)
assert.Equal(t, task.GetState().String(), importTask.State)
assert.Equal(t, task.GetReason(), importTask.Reason)
assert.Equal(t, "ImportTask", importTask.TaskType)
assert.Equal(t, task.GetCreatedTime(), importTask.CreatedTime)
assert.Equal(t, task.GetCompleteTime(), importTask.CompleteTime)
}

View File

@ -19,11 +19,14 @@ package datacoord
import (
"context"
"encoding/json"
"fmt"
"strconv"
"sync"
"time"
"github.com/golang/protobuf/proto"
"github.com/hashicorp/golang-lru/v2/expirable"
"github.com/prometheus/client_golang/prometheus"
"github.com/samber/lo"
"go.uber.org/zap"
@ -52,22 +55,87 @@ type indexMeta struct {
// collectionIndexes records which indexes are on the collection
// collID -> indexID -> index
indexes map[UniqueID]map[UniqueID]*model.Index
// buildID2Meta records the meta information of the segment
// buildID -> segmentIndex
buildID2SegmentIndex map[UniqueID]*model.SegmentIndex
// buildID2Meta records building index meta information of the segment
segmentBuildInfo *segmentBuildInfo
// segmentID -> indexID -> segmentIndex
segmentIndexes map[UniqueID]map[UniqueID]*model.SegmentIndex
}
type indexTaskStats struct {
IndexID UniqueID `json:"index_id,omitempty"`
CollectionID UniqueID `json:"collection_id,omitempty"`
SegmentID UniqueID `json:"segment_id,omitempty"`
BuildID UniqueID `json:"build_id,omitempty"`
IndexState string `json:"index_state,omitempty"`
FailReason string `json:"fail_reason,omitempty"`
IndexSize uint64 `json:"index_size,omitempty"`
IndexVersion int64 `json:"index_version,omitempty"`
CreateTime uint64 `json:"create_time,omitempty"`
}
func newIndexTaskStats(s *model.SegmentIndex) *indexTaskStats {
return &indexTaskStats{
IndexID: s.IndexID,
CollectionID: s.CollectionID,
SegmentID: s.SegmentID,
BuildID: s.BuildID,
IndexState: s.IndexState.String(),
FailReason: s.FailReason,
IndexSize: s.IndexSize,
IndexVersion: s.IndexVersion,
CreateTime: s.CreateTime,
}
}
type segmentBuildInfo struct {
// buildID2Meta records the meta information of the segment
// buildID -> segmentIndex
buildID2SegmentIndex map[UniqueID]*model.SegmentIndex
// taskStats records the task stats of the segment
taskStats *expirable.LRU[UniqueID, *indexTaskStats]
}
func newSegmentIndexBuildInfo() *segmentBuildInfo {
return &segmentBuildInfo{
// build ID -> segment index
buildID2SegmentIndex: make(map[UniqueID]*model.SegmentIndex),
// build ID -> task stats
taskStats: expirable.NewLRU[UniqueID, *indexTaskStats](1024, nil, time.Minute*60),
}
}
func (m *segmentBuildInfo) Add(segIdx *model.SegmentIndex) {
m.buildID2SegmentIndex[segIdx.BuildID] = segIdx
m.taskStats.Add(segIdx.BuildID, newIndexTaskStats(segIdx))
}
func (m *segmentBuildInfo) Get(key UniqueID) (*model.SegmentIndex, bool) {
value, exists := m.buildID2SegmentIndex[key]
return value, exists
}
func (m *segmentBuildInfo) Remove(key UniqueID) {
delete(m.buildID2SegmentIndex, key)
}
func (m *segmentBuildInfo) List() map[UniqueID]*model.SegmentIndex {
return m.buildID2SegmentIndex
}
func (m *segmentBuildInfo) GetTaskStats() []*indexTaskStats {
return m.taskStats.Values()
}
// NewMeta creates meta from provided `kv.TxnKV`
func newIndexMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (*indexMeta, error) {
mt := &indexMeta{
ctx: ctx,
catalog: catalog,
indexes: make(map[UniqueID]map[UniqueID]*model.Index),
buildID2SegmentIndex: make(map[UniqueID]*model.SegmentIndex),
segmentIndexes: make(map[UniqueID]map[UniqueID]*model.SegmentIndex),
ctx: ctx,
catalog: catalog,
indexes: make(map[UniqueID]map[UniqueID]*model.Index),
segmentBuildInfo: newSegmentIndexBuildInfo(),
segmentIndexes: make(map[UniqueID]map[UniqueID]*model.SegmentIndex),
}
err := mt.reloadFromKV()
if err != nil {
@ -116,7 +184,7 @@ func (m *indexMeta) updateSegmentIndex(segIdx *model.SegmentIndex) {
m.segmentIndexes[segIdx.SegmentID] = make(map[UniqueID]*model.SegmentIndex)
m.segmentIndexes[segIdx.SegmentID][segIdx.IndexID] = segIdx
}
m.buildID2SegmentIndex[segIdx.BuildID] = segIdx
m.segmentBuildInfo.Add(segIdx)
}
func (m *indexMeta) alterSegmentIndexes(segIdxes []*model.SegmentIndex) error {
@ -142,7 +210,7 @@ func (m *indexMeta) updateSegIndexMeta(segIdx *model.SegmentIndex, updateFunc fu
func (m *indexMeta) updateIndexTasksMetrics() {
taskMetrics := make(map[UniqueID]map[commonpb.IndexState]int)
for _, segIdx := range m.buildID2SegmentIndex {
for _, segIdx := range m.segmentBuildInfo.List() {
if segIdx.IsDeleted {
continue
}
@ -674,7 +742,7 @@ func (m *indexMeta) GetIndexJob(buildID UniqueID) (*model.SegmentIndex, bool) {
m.RLock()
defer m.RUnlock()
segIdx, ok := m.buildID2SegmentIndex[buildID]
segIdx, ok := m.segmentBuildInfo.Get(buildID)
if ok {
return model.CloneSegmentIndex(segIdx), true
}
@ -703,7 +771,7 @@ func (m *indexMeta) UpdateVersion(buildID, nodeID UniqueID) error {
defer m.Unlock()
log.Info("IndexCoord metaTable UpdateVersion receive", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
segIdx, ok := m.buildID2SegmentIndex[buildID]
segIdx, ok := m.segmentBuildInfo.Get(buildID)
if !ok {
return fmt.Errorf("there is no index with buildID: %d", buildID)
}
@ -721,7 +789,7 @@ func (m *indexMeta) FinishTask(taskInfo *workerpb.IndexTaskInfo) error {
m.Lock()
defer m.Unlock()
segIdx, ok := m.buildID2SegmentIndex[taskInfo.GetBuildID()]
segIdx, ok := m.segmentBuildInfo.Get(taskInfo.GetBuildID())
if !ok {
log.Warn("there is no index with buildID", zap.Int64("buildID", taskInfo.GetBuildID()))
return nil
@ -752,7 +820,7 @@ func (m *indexMeta) DeleteTask(buildID int64) error {
m.Lock()
defer m.Unlock()
segIdx, ok := m.buildID2SegmentIndex[buildID]
segIdx, ok := m.segmentBuildInfo.Get(buildID)
if !ok {
log.Warn("there is no index with buildID", zap.Int64("buildID", buildID))
return nil
@ -777,7 +845,7 @@ func (m *indexMeta) BuildIndex(buildID UniqueID) error {
m.Lock()
defer m.Unlock()
segIdx, ok := m.buildID2SegmentIndex[buildID]
segIdx, ok := m.segmentBuildInfo.Get(buildID)
if !ok {
return fmt.Errorf("there is no index with buildID: %d", buildID)
}
@ -806,8 +874,9 @@ func (m *indexMeta) GetAllSegIndexes() map[int64]*model.SegmentIndex {
m.RLock()
defer m.RUnlock()
segIndexes := make(map[int64]*model.SegmentIndex, len(m.buildID2SegmentIndex))
for buildID, segIndex := range m.buildID2SegmentIndex {
tasks := m.segmentBuildInfo.List()
segIndexes := make(map[int64]*model.SegmentIndex, len(tasks))
for buildID, segIndex := range tasks {
segIndexes[buildID] = model.CloneSegmentIndex(segIndex)
}
return segIndexes
@ -821,7 +890,7 @@ func (m *indexMeta) SetStoredIndexFileSizeMetric(collections map[UniqueID]*colle
var total uint64
metrics.DataCoordStoredIndexFilesSize.Reset()
for _, segmentIdx := range m.buildID2SegmentIndex {
for _, segmentIdx := range m.segmentBuildInfo.List() {
coll, ok := collections[segmentIdx.CollectionID]
if ok {
metrics.DataCoordStoredIndexFilesSize.WithLabelValues(coll.DatabaseName, coll.Schema.GetName(),
@ -849,7 +918,7 @@ func (m *indexMeta) RemoveSegmentIndex(collID, partID, segID, indexID, buildID U
delete(m.segmentIndexes, segID)
}
delete(m.buildID2SegmentIndex, buildID)
m.segmentBuildInfo.Remove(buildID)
m.updateIndexTasksMetrics()
return nil
}
@ -896,7 +965,7 @@ func (m *indexMeta) CheckCleanSegmentIndex(buildID UniqueID) (bool, *model.Segme
m.RLock()
defer m.RUnlock()
if segIndex, ok := m.buildID2SegmentIndex[buildID]; ok {
if segIndex, ok := m.segmentBuildInfo.Get(buildID); ok {
if segIndex.IndexState == commonpb.IndexState_Finished {
return true, model.CloneSegmentIndex(segIndex)
}
@ -910,7 +979,7 @@ func (m *indexMeta) GetMetasByNodeID(nodeID UniqueID) []*model.SegmentIndex {
defer m.RUnlock()
metas := make([]*model.SegmentIndex, 0)
for _, segIndex := range m.buildID2SegmentIndex {
for _, segIndex := range m.segmentBuildInfo.List() {
if segIndex.IsDeleted {
continue
}
@ -1003,3 +1072,16 @@ func (m *indexMeta) HasIndex(collectionID int64) bool {
}
return false
}
func (m *indexMeta) TaskStatsJSON() string {
tasks := m.segmentBuildInfo.GetTaskStats()
if len(tasks) == 0 {
return ""
}
ret, err := json.Marshal(tasks)
if err != nil {
return ""
}
return string(ret)
}

View File

@ -19,8 +19,10 @@ package datacoord
import (
"context"
"encoding/json"
"sync"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
@ -435,12 +437,12 @@ func TestMeta_HasSameReq(t *testing.T) {
func newSegmentIndexMeta(catalog metastore.DataCoordCatalog) *indexMeta {
return &indexMeta{
RWMutex: sync.RWMutex{},
ctx: context.Background(),
catalog: catalog,
indexes: make(map[UniqueID]map[UniqueID]*model.Index),
buildID2SegmentIndex: make(map[UniqueID]*model.SegmentIndex),
segmentIndexes: make(map[UniqueID]map[UniqueID]*model.SegmentIndex),
RWMutex: sync.RWMutex{},
ctx: context.Background(),
catalog: catalog,
indexes: make(map[UniqueID]map[UniqueID]*model.Index),
segmentBuildInfo: newSegmentIndexBuildInfo(),
segmentIndexes: make(map[UniqueID]map[UniqueID]*model.SegmentIndex),
}
}
@ -765,24 +767,23 @@ func TestMeta_GetIndexedSegment(t *testing.T) {
},
},
}
m.buildID2SegmentIndex = map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: nil,
IndexSize: 0,
},
}
m.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: nil,
IndexSize: 0,
})
t.Run("success", func(t *testing.T) {
segments := m.GetIndexedSegments(collID, []int64{segID}, []int64{fieldID})
@ -1089,24 +1090,22 @@ func TestMeta_GetIndexParams(t *testing.T) {
func TestMeta_GetIndexJob(t *testing.T) {
m := newSegmentIndexMeta(nil)
m.buildID2SegmentIndex = map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
},
}
m.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
})
t.Run("exist", func(t *testing.T) {
segIndex, exist := m.GetIndexJob(buildID)
@ -1178,6 +1177,24 @@ func updateSegmentIndexMeta(t *testing.T) *indexMeta {
mock.Anything,
).Return(nil)
indexBuildInfo := newSegmentIndexBuildInfo()
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
})
return &indexMeta{
catalog: sc,
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
@ -1217,24 +1234,7 @@ func updateSegmentIndexMeta(t *testing.T) *indexMeta {
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
},
},
segmentBuildInfo: indexBuildInfo,
}
}
@ -1363,11 +1363,12 @@ func TestMeta_DeleteTask_Error(t *testing.T) {
).Return(errors.New("fail"))
m.catalog = ec
m.buildID2SegmentIndex[buildID] = &model.SegmentIndex{
m.segmentBuildInfo.Add(&model.SegmentIndex{
BuildID: buildID,
SegmentID: segID,
PartitionID: partID,
CollectionID: collID,
}
})
err := m.DeleteTask(buildID)
assert.Error(t, err)
@ -1488,16 +1489,14 @@ func TestRemoveSegmentIndex(t *testing.T) {
indexID: &model.SegmentIndex{},
},
},
buildID2SegmentIndex: map[int64]*model.SegmentIndex{
buildID: {},
},
segmentBuildInfo: newSegmentIndexBuildInfo(),
}
err := m.RemoveSegmentIndex(collID, partID, segID, indexID, buildID)
assert.NoError(t, err)
assert.Equal(t, len(m.segmentIndexes), 0)
assert.Equal(t, len(m.buildID2SegmentIndex), 0)
assert.Equal(t, len(m.segmentBuildInfo.List()), 0)
})
}
@ -1517,3 +1516,53 @@ func TestIndexMeta_GetUnindexedSegments(t *testing.T) {
unindexed = m.indexMeta.GetUnindexedSegments(collID+1, segmentIDs)
assert.Equal(t, 0, len(unindexed))
}
func TestBuildIndexTaskStatsJSON(t *testing.T) {
im := &indexMeta{segmentBuildInfo: newSegmentIndexBuildInfo()}
si1 := &model.SegmentIndex{
BuildID: 1,
CollectionID: 100,
SegmentID: 1000,
IndexID: 10,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IndexSize: 1024,
IndexVersion: 1,
CreateTime: uint64(time.Now().Unix()),
}
si2 := &model.SegmentIndex{
BuildID: 2,
CollectionID: 101,
SegmentID: 1001,
IndexID: 11,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IndexSize: 2048,
IndexVersion: 1,
CreateTime: uint64(time.Now().Unix()),
}
actualJSON := im.TaskStatsJSON()
assert.Equal(t, "", actualJSON)
im.segmentBuildInfo.Add(si1)
im.segmentBuildInfo.Add(si2)
assert.Equal(t, 2, len(im.segmentBuildInfo.List()))
ret1, ok := im.segmentBuildInfo.Get(si1.BuildID)
assert.True(t, ok)
assert.EqualValues(t, si1, ret1)
expectedTasks := []*indexTaskStats{
newIndexTaskStats(si1),
newIndexTaskStats(si2),
}
expectedJSON, err := json.Marshal(expectedTasks)
assert.NoError(t, err)
actualJSON = im.TaskStatsJSON()
assert.JSONEq(t, string(expectedJSON), actualJSON)
im.segmentBuildInfo.Remove(si1.BuildID)
assert.Equal(t, 1, len(im.segmentBuildInfo.List()))
}

View File

@ -2341,8 +2341,8 @@ func TestMeta_GetHasUnindexTaskSegments(t *testing.T) {
m := &meta{
segments: NewSegmentsInfo(),
indexMeta: &indexMeta{
buildID2SegmentIndex: make(map[UniqueID]*model.SegmentIndex),
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{},
segmentBuildInfo: newSegmentIndexBuildInfo(),
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{},
indexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {

View File

@ -6,6 +6,7 @@ import (
"testing"
"time"
"github.com/hashicorp/golang-lru/v2/expirable"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
@ -103,6 +104,7 @@ func (s *jobManagerSuite) TestJobManager_triggerStatsTaskLoop() {
allocator: alloc,
tasks: make(map[int64]Task),
meta: mt,
taskStats: expirable.NewLRU[UniqueID, Task](1024, nil, time.Minute*5),
},
allocator: alloc,
}

View File

@ -18,9 +18,11 @@ package datacoord
import (
"context"
"encoding/json"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -80,11 +82,81 @@ func (s *Server) getCollectionMetrics(ctx context.Context) *metricsinfo.DataCoor
return ret
}
// GetSyncTaskMetrics retrieves and aggregates the sync task metrics of the datanode.
func (s *Server) GetSyncTaskMetrics(
ctx context.Context,
req *milvuspb.GetMetricsRequest,
) (string, error) {
resp, err := s.requestDataNodeGetMetrics(ctx, req)
if err != nil {
return "", err
}
tasks := make(map[string][]*metricsinfo.SyncTask, resp.Len())
resp.Range(func(key string, value *milvuspb.GetMetricsResponse) bool {
if value.Response != "" {
var sts []*metricsinfo.SyncTask
if err1 := json.Unmarshal([]byte(value.Response), &sts); err1 != nil {
log.Warn("failed to unmarshal sync task metrics")
err = err1
return false
}
tasks[key] = sts
}
return true
})
if err != nil {
return "", err
}
if len(tasks) == 0 {
return "", nil
}
bs, err := json.Marshal(tasks)
if err != nil {
return "", err
}
return (string)(bs), nil
}
func (s *Server) requestDataNodeGetMetrics(
ctx context.Context,
req *milvuspb.GetMetricsRequest,
) (*typeutil.ConcurrentMap[string, *milvuspb.GetMetricsResponse], error) {
nodes := s.cluster.GetSessions()
rets := typeutil.NewConcurrentMap[string, *milvuspb.GetMetricsResponse]()
wg, ctx := errgroup.WithContext(ctx)
for _, node := range nodes {
wg.Go(func() error {
cli, err := node.GetOrCreateClient(ctx)
if err != nil {
return err
}
ret, err := cli.GetMetrics(ctx, req)
if err != nil {
return err
}
key := metricsinfo.ConstructComponentName(typeutil.DataNodeRole, node.NodeID())
rets.Insert(key, ret)
return nil
})
}
err := wg.Wait()
if err != nil {
return nil, err
}
return rets, nil
}
// getSystemInfoMetrics composes data cluster metrics
func (s *Server) getSystemInfoMetrics(
ctx context.Context,
req *milvuspb.GetMetricsRequest,
) (*milvuspb.GetMetricsResponse, error) {
) (string, error) {
// TODO(dragondriver): add more detail metrics
// get datacoord info
@ -125,18 +197,11 @@ func (s *Server) getSystemInfoMetrics(
},
}
resp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, paramtable.GetNodeID()),
}
var err error
resp.Response, err = metricsinfo.MarshalTopology(coordTopology)
ret, err := metricsinfo.MarshalTopology(coordTopology)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
return "", err
}
return resp, nil
return ret, nil
}
// getDataCoordMetrics composes datacoord infos

View File

@ -199,3 +199,116 @@ func TestGetIndexNodeMetrics(t *testing.T) {
assert.False(t, info.HasError)
assert.Equal(t, metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, 100), info.BaseComponentInfos.Name)
}
func TestGetSyncTaskMetrics(t *testing.T) {
svr := Server{}
t.Run("ReturnsCorrectJSON", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{}
ctx := context.Background()
task := `[{"segment_id": 1, "batch_rows": 100, "segment_level": "L0", "ts_from": 1000, "ts_to": 2000,"delta_row_count": 10, "flush_size": 1024, "running_time": 2000000000}]`
mockResp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
Response: task,
}
mockClient := &mockMetricDataNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
return mockResp, nil
},
}
dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return mockClient, nil
}
mockCluster := NewMockCluster(t)
mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)})
svr.cluster = mockCluster
actualJSON, err := svr.GetSyncTaskMetrics(ctx, req)
assert.NoError(t, err)
expectedJSON := `{"datanode1":[{"segment_id":1,"batch_rows":100,"segment_level":"L0","ts_from":1000,"ts_to":2000,"delta_row_count":10,"flush_size":1024,"running_time":2000000000}]}`
assert.Equal(t, expectedJSON, actualJSON)
})
t.Run("ReturnsErrorOnRequestFailure", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{}
ctx := context.Background()
mockClient := &mockMetricDataNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
return nil, errors.New("request failed")
},
}
dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return mockClient, nil
}
mockCluster := NewMockCluster(t)
mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)})
svr.cluster = mockCluster
actualJSON, err := svr.GetSyncTaskMetrics(ctx, req)
assert.Error(t, err)
assert.Equal(t, "", actualJSON)
})
t.Run("ReturnsErrorOnUnmarshalFailure", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{}
ctx := context.Background()
mockResp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
Response: `invalid json`,
}
mockClient := &mockMetricDataNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
return mockResp, nil
},
}
dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return mockClient, nil
}
mockCluster := NewMockCluster(t)
mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)})
svr.cluster = mockCluster
actualJSON, err := svr.GetSyncTaskMetrics(ctx, req)
assert.Error(t, err)
assert.Equal(t, "", actualJSON)
})
t.Run("ReturnsEmptyJSONWhenNoTasks", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{}
ctx := context.Background()
mockResp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
Response: "",
}
mockClient := &mockMetricDataNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
return mockResp, nil
},
}
dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return mockClient, nil
}
mockCluster := NewMockCluster(t)
mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)})
svr.cluster = mockCluster
expectedJSON := ""
actualJSON, err := svr.GetSyncTaskMetrics(ctx, req)
assert.NoError(t, err)
assert.Equal(t, expectedJSON, actualJSON)
})
}

View File

@ -29,12 +29,14 @@ import (
"github.com/blang/semver/v4"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/tidwall/gjson"
"github.com/tikv/client-go/v2/txnkv"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
globalIDAllocator "github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/broker"
@ -162,6 +164,8 @@ type Server struct {
// streamingcoord server is embedding in datacoord now.
streamingCoord *streamingcoord.Server
metricsRequest *metricsinfo.MetricsRequest
}
type CollectionNameInfo struct {
@ -214,6 +218,7 @@ func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Optio
rootCoordClientCreator: defaultRootCoordCreatorFunc,
metricsCacheManager: metricsinfo.NewMetricsCacheManager(),
enableActiveStandBy: Params.DataCoordCfg.EnableActiveStandby.GetAsBool(),
metricsRequest: metricsinfo.NewMetricsRequest(),
}
for _, opt := range opts {
@ -296,6 +301,7 @@ func (s *Server) initSession() error {
// Init change server state to Initializing
func (s *Server) Init() error {
var err error
s.registerMetricsRequest()
s.factory.Init(Params)
if err = s.initSession(); err != nil {
return err
@ -1125,20 +1131,33 @@ func (s *Server) stopServerLoop() {
s.serverLoopWg.Wait()
}
// func (s *Server) validateAllocRequest(collID UniqueID, partID UniqueID, channelName string) error {
// if !s.meta.HasCollection(collID) {
// return fmt.Errorf("can not find collection %d", collID)
// }
// if !s.meta.HasPartition(collID, partID) {
// return fmt.Errorf("can not find partition %d", partID)
// }
// for _, name := range s.insertChannels {
// if name == channelName {
// return nil
// }
// }
// return fmt.Errorf("can not find channel %s", channelName)
// }
func (s *Server) registerMetricsRequest() {
s.metricsRequest.RegisterMetricsRequest(metricsinfo.SystemInfoMetrics,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.getSystemInfoMetrics(ctx, req)
})
s.metricsRequest.RegisterMetricsRequest(metricsinfo.ImportTasks,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.importMeta.TaskStatsJSON(), nil
})
s.metricsRequest.RegisterMetricsRequest(metricsinfo.CompactionTasks,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.meta.compactionTaskMeta.TaskStatsJSON(), nil
})
s.metricsRequest.RegisterMetricsRequest(metricsinfo.BuildIndexTasks,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.meta.indexMeta.TaskStatsJSON(), nil
})
s.metricsRequest.RegisterMetricsRequest(metricsinfo.SyncTasks,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.GetSyncTaskMetrics(ctx, req)
})
log.Info("register metrics actions finished")
}
// loadCollectionFromRootCoord communicates with RootCoord and asks for collection information.
// collection information will be added to server meta info.

View File

@ -804,14 +804,11 @@ func TestServer_getSystemInfoMetrics(t *testing.T) {
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
assert.NoError(t, err)
resp, err := svr.getSystemInfoMetrics(svr.ctx, req)
ret, err := svr.getSystemInfoMetrics(svr.ctx, req)
assert.NoError(t, err)
log.Info("TestServer_getSystemInfoMetrics",
zap.String("name", resp.ComponentName),
zap.String("response", resp.Response))
var coordTopology metricsinfo.DataCoordTopology
err = metricsinfo.UnmarshalTopology(resp.Response, &coordTopology)
err = metricsinfo.UnmarshalTopology(ret, &coordTopology)
assert.NoError(t, err)
assert.Equal(t, len(svr.cluster.GetSessions()), len(coordTopology.Cluster.ConnectedDataNodes))
for _, nodeMetrics := range coordTopology.Cluster.ConnectedDataNodes {

View File

@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/tidwall/gjson"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
@ -1073,54 +1072,27 @@ func (s *Server) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
log := log.Ctx(ctx)
if err := merr.CheckHealthyStandby(s.GetStateCode()); err != nil {
msg := "failed to get metrics"
log.Warn(msg, zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
Status: merr.Status(errors.Wrap(err, msg)),
}, nil
}
ret := gjson.Parse(req.GetRequest())
metricType, err := metricsinfo.ParseMetricRequestType(ret)
resp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole,
paramtable.GetNodeID()),
}
ret, err := s.metricsRequest.ExecuteMetricsRequest(ctx, req)
if err != nil {
log.Warn("DataCoord.GetMetrics failed to parse metric type",
zap.Int64("nodeID", paramtable.GetNodeID()),
zap.String("req", req.Request),
zap.Error(err),
)
return &milvuspb.GetMetricsResponse{
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, paramtable.GetNodeID()),
Status: merr.Status(err),
}, nil
resp.Status = merr.Status(err)
return resp, nil
}
if metricType == metricsinfo.SystemInfoMetrics {
metrics, err := s.getSystemInfoMetrics(ctx, req)
if err != nil {
log.Warn("DataCoord GetMetrics failed", zap.Int64("nodeID", paramtable.GetNodeID()), zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
}, nil
}
log.RatedDebug(60, "DataCoord.GetMetrics",
zap.Int64("nodeID", paramtable.GetNodeID()),
zap.String("req", req.Request),
zap.String("metricType", metricType),
zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large
zap.Error(err))
return metrics, nil
}
log.RatedWarn(60.0, "DataCoord.GetMetrics failed, request metric type is not implemented yet",
zap.Int64("nodeID", paramtable.GetNodeID()),
zap.String("req", req.Request),
zap.String("metricType", metricType))
return &milvuspb.GetMetricsResponse{
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, paramtable.GetNodeID()),
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
}, nil
resp.Response = ret
return resp, nil
}
// ManualCompaction triggers a compaction for a collection

View File

@ -21,6 +21,7 @@ import (
"sync"
"time"
"github.com/hashicorp/golang-lru/v2/expirable"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
@ -61,6 +62,8 @@ type taskScheduler struct {
indexEngineVersionManager IndexEngineVersionManager
handler Handler
allocator allocator.Allocator
taskStats *expirable.LRU[UniqueID, Task]
}
func newTaskScheduler(
@ -88,6 +91,7 @@ func newTaskScheduler(
handler: handler,
indexEngineVersionManager: indexEngineVersionManager,
allocator: allocator,
taskStats: expirable.NewLRU[UniqueID, Task](1024, nil, time.Minute*5),
}
ts.reloadFromMeta()
return ts
@ -112,7 +116,7 @@ func (s *taskScheduler) reloadFromMeta() {
continue
}
if segIndex.IndexState != commonpb.IndexState_Finished && segIndex.IndexState != commonpb.IndexState_Failed {
s.tasks[segIndex.BuildID] = &indexBuildTask{
s.enqueue(&indexBuildTask{
taskID: segIndex.BuildID,
nodeID: segIndex.NodeID,
taskInfo: &workerpb.IndexTaskInfo{
@ -123,7 +127,7 @@ func (s *taskScheduler) reloadFromMeta() {
queueTime: time.Now(),
startTime: time.Now(),
endTime: time.Now(),
}
})
}
}
}
@ -131,7 +135,7 @@ func (s *taskScheduler) reloadFromMeta() {
allAnalyzeTasks := s.meta.analyzeMeta.GetAllTasks()
for taskID, t := range allAnalyzeTasks {
if t.State != indexpb.JobState_JobStateFinished && t.State != indexpb.JobState_JobStateFailed {
s.tasks[taskID] = &analyzeTask{
s.enqueue(&analyzeTask{
taskID: taskID,
nodeID: t.NodeID,
taskInfo: &workerpb.AnalyzeResult{
@ -142,14 +146,14 @@ func (s *taskScheduler) reloadFromMeta() {
queueTime: time.Now(),
startTime: time.Now(),
endTime: time.Now(),
}
})
}
}
allStatsTasks := s.meta.statsTaskMeta.GetAllTasks()
for taskID, t := range allStatsTasks {
if t.GetState() != indexpb.JobState_JobStateFinished && t.GetState() != indexpb.JobState_JobStateFailed {
s.tasks[taskID] = &statsTask{
s.enqueue(&statsTask{
taskID: taskID,
segmentID: t.GetSegmentID(),
targetSegmentID: t.GetTargetSegmentID(),
@ -163,7 +167,7 @@ func (s *taskScheduler) reloadFromMeta() {
startTime: time.Now(),
endTime: time.Now(),
subJobType: t.GetSubJobType(),
}
})
}
}
}
@ -184,6 +188,7 @@ func (s *taskScheduler) enqueue(task Task) {
taskID := task.GetTaskID()
if _, ok := s.tasks[taskID]; !ok {
s.tasks[taskID] = task
s.taskStats.Add(taskID, task)
task.SetQueueTime(time.Now())
log.Info("taskScheduler enqueue task", zap.Int64("taskID", taskID))
}

View File

@ -56,6 +56,194 @@ var (
)
func createIndexMeta(catalog metastore.DataCoordCatalog) *indexMeta {
indexBuildInfo := newSegmentIndexBuildInfo()
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 2,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 2,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: true,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 3,
CollectionID: collID,
PartitionID: partID,
NumRows: 500,
IndexID: indexID,
BuildID: buildID + 3,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 4,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 4,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 5,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 5,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 6,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 6,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 7,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 7,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Failed,
FailReason: "error",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 8,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 8,
NodeID: nodeID + 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 9,
CollectionID: collID,
PartitionID: partID,
NumRows: 500,
IndexID: indexID,
BuildID: buildID + 9,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
indexBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID + 10,
CollectionID: collID,
PartitionID: partID,
NumRows: 500,
IndexID: indexID,
BuildID: buildID + 10,
NodeID: nodeID,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
})
return &indexMeta{
catalog: catalog,
indexes: map[UniqueID]map[UniqueID]*model.Index{
@ -287,184 +475,7 @@ func createIndexMeta(catalog metastore.DataCoordCatalog) *indexMeta {
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 2: {
SegmentID: segID + 2,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 2,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: true,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 3: {
SegmentID: segID + 3,
CollectionID: collID,
PartitionID: partID,
NumRows: 500,
IndexID: indexID,
BuildID: buildID + 3,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 4: {
SegmentID: segID + 4,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 4,
NodeID: nodeID,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 5: {
SegmentID: segID + 5,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 5,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 6: {
SegmentID: segID + 6,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 6,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 7: {
SegmentID: segID + 7,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 7,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Failed,
FailReason: "error",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 8: {
SegmentID: segID + 8,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 8,
NodeID: nodeID + 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 9: {
SegmentID: segID + 9,
CollectionID: collID,
PartitionID: partID,
NumRows: 500,
IndexID: indexID,
BuildID: buildID + 9,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
buildID + 10: {
SegmentID: segID + 10,
CollectionID: collID,
PartitionID: partID,
NumRows: 500,
IndexID: indexID,
BuildID: buildID + 10,
NodeID: nodeID,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 1111,
IndexFileKeys: nil,
IndexSize: 1,
},
},
segmentBuildInfo: indexBuildInfo,
}
}
@ -1296,17 +1307,7 @@ func (s *taskSchedulerSuite) Test_indexTaskFailCase() {
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: s.collectionID,
PartitionID: s.partitionID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Unissued,
},
},
segmentBuildInfo: newSegmentIndexBuildInfo(),
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
buildID: {
@ -1326,6 +1327,15 @@ func (s *taskSchedulerSuite) Test_indexTaskFailCase() {
catalog: catalog,
}))
mt.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: s.collectionID,
PartitionID: s.partitionID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Unissued,
})
cm := mocks.NewChunkManager(s.T())
cm.EXPECT().RootPath().Return("ut-index")
@ -1548,24 +1558,7 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: minNumberOfRowsToBuild,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
},
},
segmentBuildInfo: newSegmentIndexBuildInfo(),
},
segments: &SegmentsInfo{
segments: map[UniqueID]*SegmentInfo{
@ -1589,6 +1582,22 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
},
}
mt.indexMeta.segmentBuildInfo.Add(&model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: minNumberOfRowsToBuild,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
})
cm := mocks.NewChunkManager(s.T())
cm.EXPECT().RootPath().Return("ut-index")
@ -1621,7 +1630,9 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
}
resetMetaFunc := func() {
mt.indexMeta.buildID2SegmentIndex[buildID].IndexState = commonpb.IndexState_Unissued
t, ok := mt.indexMeta.segmentBuildInfo.Get(buildID)
s.True(ok)
t.IndexState = commonpb.IndexState_Unissued
mt.indexMeta.segmentIndexes[segID][indexID].IndexState = commonpb.IndexState_Unissued
mt.indexMeta.indexes[collID][indexID].IndexParams[1].Value = "HNSW"
mt.collections[collID].Schema.Fields[0].DataType = schemapb.DataType_FloatVector

View File

@ -30,10 +30,12 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/tidwall/gjson"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/datanode/channel"
"github.com/milvus-io/milvus/internal/datanode/compaction"
@ -57,6 +59,7 @@ import (
"github.com/milvus-io/milvus/pkg/util/conc"
"github.com/milvus-io/milvus/pkg/util/expr"
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/typeutil"
@ -127,6 +130,8 @@ type DataNode struct {
reportImportRetryTimes uint // unitest set this value to 1 to save time, default is 10
pool *conc.Pool[any]
metricsRequest *metricsinfo.MetricsRequest
}
// NewDataNode will return a DataNode with abnormal state.
@ -144,6 +149,7 @@ func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode {
segmentCache: util.NewCache(),
compactionExecutor: compaction.NewExecutor(),
reportImportRetryTimes: 10,
metricsRequest: metricsinfo.NewMetricsRequest(),
}
node.UpdateStateCode(commonpb.StateCode_Abnormal)
expr.Register("datanode", node)
@ -221,6 +227,7 @@ func (node *DataNode) GetNodeID() int64 {
func (node *DataNode) Init() error {
var initError error
node.initOnce.Do(func() {
node.registerMetricsRequest()
logutil.Logger(node.ctx).Info("DataNode server initializing",
zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()),
)
@ -272,6 +279,18 @@ func (node *DataNode) Init() error {
return initError
}
func (node *DataNode) registerMetricsRequest() {
node.metricsRequest.RegisterMetricsRequest(metricsinfo.SystemInfoMetrics,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return node.getSystemInfoMetrics(ctx, req)
})
node.metricsRequest.RegisterMetricsRequest(metricsinfo.SyncTasks,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return node.syncMgr.TaskStatsJSON(), nil
})
log.Info("register metrics actions finished")
}
// tryToReleaseFlowgraph tries to release a flowgraph
func (node *DataNode) tryToReleaseFlowgraph(channel string) {
log.Info("try to release flowgraph", zap.String("channel", channel))

View File

@ -29,7 +29,6 @@ import (
"github.com/stretchr/testify/mock"
"go.uber.org/zap"
"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/internal/flushcommon/broker"
"github.com/milvus-io/milvus/internal/flushcommon/pipeline"
@ -187,10 +186,7 @@ func TestDataNode(t *testing.T) {
assert.NoError(t, err)
resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
log.Info("Test DataNode.getSystemInfoMetrics",
zap.String("name", resp.ComponentName),
zap.String("response", resp.Response))
assert.NotEmpty(t, resp)
})
t.Run("Test getSystemInfoMetrics with quotaMetric error", func(t *testing.T) {
@ -202,8 +198,8 @@ func TestDataNode(t *testing.T) {
assert.NoError(t, err)
util2.DeregisterRateCollector(metricsinfo.InsertConsumeThroughput)
resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req)
assert.NoError(t, err)
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Error(t, err)
assert.Empty(t, resp)
util2.RegisterRateCollector(metricsinfo.InsertConsumeThroughput)
})
}

View File

@ -95,7 +95,7 @@ func NewSyncTask(ctx context.Context,
WithTimeRange(ts, ts).
WithLevel(segmentLevel).
WithDataSource(metrics.BulkinsertDataSourceLabel).
WithBatchSize(int64(insertData.GetRowNum()))
WithBatchRows(int64(insertData.GetRowNum()))
if bm25Stats != nil {
syncPack.WithBM25Stats(bm25Stats)
}

View File

@ -22,7 +22,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/flushcommon/util"
"github.com/milvus-io/milvus/pkg/util/hardware"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/ratelimitutil"
@ -66,17 +65,14 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro
}, nil
}
func (node *DataNode) getSystemInfoMetrics(_ context.Context, _ *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (node *DataNode) getSystemInfoMetrics(_ context.Context, _ *milvuspb.GetMetricsRequest) (string, error) {
// TODO(dragondriver): add more metrics
usedMem := hardware.GetUsedMemoryCount()
totalMem := hardware.GetMemoryCount()
quotaMetrics, err := node.getQuotaMetrics()
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()),
}, nil
return "", err
}
hardwareMetrics := metricsinfo.HardwareMetrics{
IP: node.session.Address,
@ -106,19 +102,5 @@ func (node *DataNode) getSystemInfoMetrics(_ context.Context, _ *milvuspb.GetMet
}
metricsinfo.FillDeployMetricsWithEnv(&nodeInfos.SystemInfo)
resp, err := metricsinfo.MarshalComponentInfos(nodeInfos)
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
Response: "",
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()),
}, nil
}
return &milvuspb.GetMetricsResponse{
Status: merr.Success(),
Response: resp,
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()),
}, nil
return metricsinfo.MarshalComponentInfos(nodeInfos)
}

View File

@ -24,7 +24,6 @@ import (
"fmt"
"github.com/samber/lo"
"github.com/tidwall/gjson"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
@ -45,7 +44,9 @@ import (
"github.com/milvus-io/milvus/pkg/util/conc"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
// WatchDmChannels is not in use
@ -166,39 +167,20 @@ func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
}, nil
}
ret := gjson.Parse(req.GetRequest())
metricType, err := metricsinfo.ParseMetricRequestType(ret)
resp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole,
paramtable.GetNodeID()),
}
ret, err := node.metricsRequest.ExecuteMetricsRequest(ctx, req)
if err != nil {
log.Warn("DataNode.GetMetrics failed to parse metric type",
zap.Int64("nodeID", node.GetNodeID()),
zap.String("req", req.Request),
zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
}, nil
resp.Status = merr.Status(err)
return resp, nil
}
if metricType == metricsinfo.SystemInfoMetrics {
systemInfoMetrics, err := node.getSystemInfoMetrics(ctx, req)
if err != nil {
log.Warn("DataNode GetMetrics failed", zap.Int64("nodeID", node.GetNodeID()), zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
}, nil
}
return systemInfoMetrics, nil
}
log.RatedWarn(60, "DataNode.GetMetrics failed, request metric type is not implemented yet",
zap.Int64("nodeID", node.GetNodeID()),
zap.String("req", req.Request),
zap.String("metric_type", metricType))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
}, nil
resp.Response = ret
return resp, nil
}
// CompactionV2 handles compaction request from DataCoord

View File

@ -481,7 +481,8 @@ func (s *DataNodeServicesSuite) TestShowConfigurations() {
}
func (s *DataNodeServicesSuite) TestGetMetrics() {
node := &DataNode{}
node := NewDataNode(context.TODO(), nil)
node.registerMetricsRequest()
node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
node.flowgraphManager = pipeline.NewFlowgraphManager()
// server is closed

View File

@ -22,6 +22,7 @@ import (
"crypto/x509"
"fmt"
"io"
"net"
"net/http"
"os"
"strings"
@ -94,6 +95,9 @@ type Server struct {
ctx context.Context
wg sync.WaitGroup
proxy types.ProxyComponent
httpListener net.Listener
grpcListener net.Listener
tcpServer cmux.CMux
httpServer *http.Server
grpcInternalServer *grpc.Server
grpcExternalServer *grpc.Server

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.46.0. DO NOT EDIT.
package broker
@ -35,6 +35,10 @@ func (_m *MockBroker) AssignSegmentID(ctx context.Context, reqs ...*datapb.Segme
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for AssignSegmentID")
}
var r0 []int64
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, ...*datapb.SegmentIDRequest) ([]int64, error)); ok {
@ -97,6 +101,10 @@ func (_c *MockBroker_AssignSegmentID_Call) RunAndReturn(run func(context.Context
func (_m *MockBroker) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) {
ret := _m.Called(ctx, req)
if len(ret) == 0 {
panic("no return value specified for DropVirtualChannel")
}
var r0 *datapb.DropVirtualChannelResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error)); ok {
@ -152,6 +160,10 @@ func (_c *MockBroker_DropVirtualChannel_Call) RunAndReturn(run func(context.Cont
func (_m *MockBroker) GetSegmentInfo(ctx context.Context, segmentIDs []int64) ([]*datapb.SegmentInfo, error) {
ret := _m.Called(ctx, segmentIDs)
if len(ret) == 0 {
panic("no return value specified for GetSegmentInfo")
}
var r0 []*datapb.SegmentInfo
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, []int64) ([]*datapb.SegmentInfo, error)); ok {
@ -207,6 +219,10 @@ func (_c *MockBroker_GetSegmentInfo_Call) RunAndReturn(run func(context.Context,
func (_m *MockBroker) ReportTimeTick(ctx context.Context, msgs []*msgpb.DataNodeTtMsg) error {
ret := _m.Called(ctx, msgs)
if len(ret) == 0 {
panic("no return value specified for ReportTimeTick")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, []*msgpb.DataNodeTtMsg) error); ok {
r0 = rf(ctx, msgs)
@ -250,6 +266,10 @@ func (_c *MockBroker_ReportTimeTick_Call) RunAndReturn(run func(context.Context,
func (_m *MockBroker) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) error {
ret := _m.Called(ctx, req)
if len(ret) == 0 {
panic("no return value specified for SaveBinlogPaths")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SaveBinlogPathsRequest) error); ok {
r0 = rf(ctx, req)
@ -293,6 +313,10 @@ func (_c *MockBroker_SaveBinlogPaths_Call) RunAndReturn(run func(context.Context
func (_m *MockBroker) UpdateChannelCheckpoint(ctx context.Context, channelCPs []*msgpb.MsgPosition) error {
ret := _m.Called(ctx, channelCPs)
if len(ret) == 0 {
panic("no return value specified for UpdateChannelCheckpoint")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, []*msgpb.MsgPosition) error); ok {
r0 = rf(ctx, channelCPs)
@ -336,6 +360,10 @@ func (_c *MockBroker_UpdateChannelCheckpoint_Call) RunAndReturn(run func(context
func (_m *MockBroker) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) error {
ret := _m.Called(ctx, req)
if len(ret) == 0 {
panic("no return value specified for UpdateSegmentStatistics")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.UpdateSegmentStatisticsRequest) error); ok {
r0 = rf(ctx, req)

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.46.0. DO NOT EDIT.
package metacache
@ -80,6 +80,10 @@ func (_c *MockMetaCache_AddSegment_Call) RunAndReturn(run func(*datapb.SegmentIn
func (_m *MockMetaCache) Collection() int64 {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Collection")
}
var r0 int64
if rf, ok := ret.Get(0).(func() int64); ok {
r0 = rf()
@ -121,6 +125,10 @@ func (_c *MockMetaCache_Collection_Call) RunAndReturn(run func() int64) *MockMet
func (_m *MockMetaCache) DetectMissingSegments(segments map[int64]struct{}) []int64 {
ret := _m.Called(segments)
if len(ret) == 0 {
panic("no return value specified for DetectMissingSegments")
}
var r0 []int64
if rf, ok := ret.Get(0).(func(map[int64]struct{}) []int64); ok {
r0 = rf(segments)
@ -172,6 +180,10 @@ func (_m *MockMetaCache) GetSegmentByID(id int64, filters ...SegmentFilter) (*Se
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for GetSegmentByID")
}
var r0 *SegmentInfo
var r1 bool
if rf, ok := ret.Get(0).(func(int64, ...SegmentFilter) (*SegmentInfo, bool)); ok {
@ -240,6 +252,10 @@ func (_m *MockMetaCache) GetSegmentIDsBy(filters ...SegmentFilter) []int64 {
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for GetSegmentIDsBy")
}
var r0 []int64
if rf, ok := ret.Get(0).(func(...SegmentFilter) []int64); ok {
r0 = rf(filters...)
@ -297,6 +313,10 @@ func (_m *MockMetaCache) GetSegmentsBy(filters ...SegmentFilter) []*SegmentInfo
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for GetSegmentsBy")
}
var r0 []*SegmentInfo
if rf, ok := ret.Get(0).(func(...SegmentFilter) []*SegmentInfo); ok {
r0 = rf(filters...)
@ -355,6 +375,10 @@ func (_m *MockMetaCache) PredictSegments(pk storage.PrimaryKey, filters ...Segme
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for PredictSegments")
}
var r0 []int64
var r1 bool
if rf, ok := ret.Get(0).(func(storage.PrimaryKey, ...SegmentFilter) ([]int64, bool)); ok {
@ -423,6 +447,10 @@ func (_m *MockMetaCache) RemoveSegments(filters ...SegmentFilter) []int64 {
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for RemoveSegments")
}
var r0 []int64
if rf, ok := ret.Get(0).(func(...SegmentFilter) []int64); ok {
r0 = rf(filters...)
@ -474,6 +502,10 @@ func (_c *MockMetaCache_RemoveSegments_Call) RunAndReturn(run func(...SegmentFil
func (_m *MockMetaCache) Schema() *schemapb.CollectionSchema {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Schema")
}
var r0 *schemapb.CollectionSchema
if rf, ok := ret.Get(0).(func() *schemapb.CollectionSchema); ok {
r0 = rf()

View File

@ -60,7 +60,7 @@ func (b *brokerMetaWriter) UpdateSync(ctx context.Context, pack *SyncTask) error
}
checkPoints = append(checkPoints, &datapb.CheckPoint{
SegmentID: pack.segmentID,
NumOfRows: segment.FlushedRows() + pack.batchSize,
NumOfRows: segment.FlushedRows() + pack.batchRows,
Position: pack.checkpoint,
})

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.46.0. DO NOT EDIT.
package syncmgr
@ -34,6 +34,10 @@ func (_m *MockSyncManager) SyncData(ctx context.Context, task Task, callbacks ..
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for SyncData")
}
var r0 *conc.Future[struct{}]
if rf, ok := ret.Get(0).(func(context.Context, Task, ...func(error) error) *conc.Future[struct{}]); ok {
r0 = rf(ctx, task, callbacks...)
@ -83,6 +87,51 @@ func (_c *MockSyncManager_SyncData_Call) RunAndReturn(run func(context.Context,
return _c
}
// TaskStatsJSON provides a mock function with given fields:
func (_m *MockSyncManager) TaskStatsJSON() string {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for TaskStatsJSON")
}
var r0 string
if rf, ok := ret.Get(0).(func() string); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(string)
}
return r0
}
// MockSyncManager_TaskStatsJSON_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'TaskStatsJSON'
type MockSyncManager_TaskStatsJSON_Call struct {
*mock.Call
}
// TaskStatsJSON is a helper method to define mock.On call
func (_e *MockSyncManager_Expecter) TaskStatsJSON() *MockSyncManager_TaskStatsJSON_Call {
return &MockSyncManager_TaskStatsJSON_Call{Call: _e.mock.On("TaskStatsJSON")}
}
func (_c *MockSyncManager_TaskStatsJSON_Call) Run(run func()) *MockSyncManager_TaskStatsJSON_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockSyncManager_TaskStatsJSON_Call) Return(_a0 string) *MockSyncManager_TaskStatsJSON_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockSyncManager_TaskStatsJSON_Call) RunAndReturn(run func() string) *MockSyncManager_TaskStatsJSON_Call {
_c.Call.Return(run)
return _c
}
// NewMockSyncManager creates a new instance of MockSyncManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockSyncManager(t interface {

View File

@ -108,8 +108,8 @@ func (t *SyncTask) WithFailureCallback(callback func(error)) *SyncTask {
return t
}
func (t *SyncTask) WithBatchSize(batchSize int64) *SyncTask {
t.batchSize = batchSize
func (t *SyncTask) WithBatchRows(batchRows int64) *SyncTask {
t.batchRows = batchRows
return t
}

View File

@ -48,7 +48,7 @@ type SyncPack struct {
tsTo typeutil.Timestamp
startPosition *msgpb.MsgPosition
checkpoint *msgpb.MsgPosition
batchSize int64 // batchSize is the row number of this sync task,not the total num of rows of segemnt
batchRows int64 // batchRows is the row number of this sync task,not the total num of rows of segment
dataSource string
isFlush bool
isDrop bool
@ -124,8 +124,8 @@ func (p *SyncPack) WithDrop() *SyncPack {
return p
}
func (p *SyncPack) WithBatchSize(batchSize int64) *SyncPack {
p.batchSize = batchSize
func (p *SyncPack) WithBatchRows(batchRows int64) *SyncPack {
p.batchRows = batchRows
return p
}

View File

@ -169,7 +169,7 @@ func (s *storageV1Serializer) setTaskMeta(task *SyncTask, pack *SyncPack) {
WithPartitionID(pack.partitionID).
WithChannelName(pack.channelName).
WithSegmentID(pack.segmentID).
WithBatchSize(pack.batchSize).
WithBatchRows(pack.batchRows).
WithSchema(s.metacache.Schema()).
WithStartPosition(pack.startPosition).
WithCheckpoint(pack.checkpoint).
@ -235,7 +235,7 @@ func (s *storageV1Serializer) serializeStatslog(pack *SyncPack) (*storage.Primar
stats.UpdateByMsgs(chunkPkData)
}
blob, err := s.inCodec.SerializePkStats(stats, pack.batchSize)
blob, err := s.inCodec.SerializePkStats(stats, pack.batchRows)
if err != nil {
return nil, nil, err
}

View File

@ -198,7 +198,7 @@ func (s *StorageV1SerializerSuite) TestSerializeInsert() {
s.Run("with_empty_data", func() {
pack := s.getBasicPack()
pack.WithTimeRange(50, 100)
pack.WithInsertData([]*storage.InsertData{s.getEmptyInsertBuffer()}).WithBatchSize(0)
pack.WithInsertData([]*storage.InsertData{s.getEmptyInsertBuffer()}).WithBatchRows(0)
_, err := s.serializer.EncodeBuffer(ctx, pack)
s.Error(err)
@ -207,7 +207,7 @@ func (s *StorageV1SerializerSuite) TestSerializeInsert() {
s.Run("with_normal_data", func() {
pack := s.getBasicPack()
pack.WithTimeRange(50, 100)
pack.WithInsertData([]*storage.InsertData{s.getInsertBuffer()}).WithBatchSize(10)
pack.WithInsertData([]*storage.InsertData{s.getInsertBuffer()}).WithBatchRows(10)
s.mockCache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return().Once()
@ -241,7 +241,7 @@ func (s *StorageV1SerializerSuite) TestSerializeInsert() {
s.Run("with_flush", func() {
pack := s.getBasicPack()
pack.WithTimeRange(50, 100)
pack.WithInsertData([]*storage.InsertData{s.getInsertBuffer()}).WithBatchSize(10)
pack.WithInsertData([]*storage.InsertData{s.getInsertBuffer()}).WithBatchRows(10)
pack.WithFlush()
bfs := s.getBfs()

View File

@ -2,9 +2,12 @@ package syncmgr
import (
"context"
"encoding/json"
"fmt"
"strconv"
"time"
"github.com/hashicorp/golang-lru/v2/expirable"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
@ -45,13 +48,16 @@ type SyncMeta struct {
type SyncManager interface {
// SyncData is the method to submit sync task.
SyncData(ctx context.Context, task Task, callbacks ...func(error) error) *conc.Future[struct{}]
TaskStatsJSON() string
}
type syncManager struct {
*keyLockDispatcher[int64]
chunkManager storage.ChunkManager
tasks *typeutil.ConcurrentMap[string, Task]
tasks *typeutil.ConcurrentMap[string, Task]
taskStats *expirable.LRU[string, Task]
}
func NewSyncManager(chunkManager storage.ChunkManager) SyncManager {
@ -64,6 +70,7 @@ func NewSyncManager(chunkManager storage.ChunkManager) SyncManager {
keyLockDispatcher: dispatcher,
chunkManager: chunkManager,
tasks: typeutil.NewConcurrentMap[string, Task](),
taskStats: expirable.NewLRU[string, Task](512, nil, time.Minute*15),
}
// setup config update watcher
params.Watch(params.DataNodeCfg.MaxParallelSyncMgrTasks.Key, config.NewHandler("datanode.syncmgr.poolsize", syncMgr.resizeHandler))
@ -103,6 +110,7 @@ func (mgr *syncManager) SyncData(ctx context.Context, task Task, callbacks ...fu
func (mgr *syncManager) safeSubmitTask(ctx context.Context, task Task, callbacks ...func(error) error) *conc.Future[struct{}] {
taskKey := fmt.Sprintf("%d-%d", task.SegmentID(), task.Checkpoint().GetTimestamp())
mgr.tasks.Insert(taskKey, task)
mgr.taskStats.Add(taskKey, task)
key := task.SegmentID()
return mgr.submit(ctx, key, task, callbacks...)
@ -120,3 +128,17 @@ func (mgr *syncManager) submit(ctx context.Context, key int64, task Task, callba
log.Info("sync mgr sumbit task with key", zap.Int64("key", key))
return mgr.Submit(ctx, key, task, callbacks...)
}
func (mgr *syncManager) TaskStatsJSON() string {
tasks := mgr.taskStats.Values()
if len(tasks) == 0 {
return ""
}
ret, err := json.Marshal(tasks)
if err != nil {
log.Warn("failed to marshal sync task stats", zap.Error(err))
return ""
}
return string(ret)
}

View File

@ -2,6 +2,7 @@ package syncmgr
import (
"context"
"encoding/json"
"math/rand"
"strconv"
"testing"
@ -9,6 +10,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"go.uber.org/atomic"
@ -271,6 +273,44 @@ func (s *SyncManagerSuite) TestTargetUpdateSameID() {
s.Error(err)
}
func (s *SyncManagerSuite) TestSyncManager_TaskStatsJSON() {
manager := NewSyncManager(s.chunkManager)
syncMgr, ok := manager.(*syncManager)
assert.True(s.T(), ok)
task1 := &SyncTask{
segmentID: 12345,
collectionID: 1,
partitionID: 1,
channelName: "channel1",
schema: &schemapb.CollectionSchema{},
checkpoint: &msgpb.MsgPosition{},
tsFrom: 1000,
tsTo: 2000,
}
task2 := &SyncTask{
segmentID: 67890,
collectionID: 2,
partitionID: 2,
channelName: "channel2",
schema: &schemapb.CollectionSchema{},
checkpoint: &msgpb.MsgPosition{},
tsFrom: 3000,
tsTo: 4000,
}
syncMgr.taskStats.Add("12345-1000", task1)
syncMgr.taskStats.Add("67890-3000", task2)
expectedTasks := []SyncTask{*task1, *task2}
expectedJSON, err := json.Marshal(expectedTasks)
s.NoError(err)
actualJSON := syncMgr.TaskStatsJSON()
s.JSONEq(string(expectedJSON), actualJSON)
}
func TestSyncManager(t *testing.T) {
suite.Run(t, new(SyncManagerSuite))
}

View File

@ -18,8 +18,10 @@ package syncmgr
import (
"context"
"encoding/json"
"fmt"
"path"
"time"
"github.com/samber/lo"
"go.uber.org/zap"
@ -36,6 +38,7 @@ import (
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/timerecord"
@ -56,9 +59,9 @@ type SyncTask struct {
startPosition *msgpb.MsgPosition
checkpoint *msgpb.MsgPosition
dataSource string
// batchSize is the row number of this sync task,
// batchRows is the row number of this sync task,
// not the total num of rows of segemnt
batchSize int64
batchRows int64
level datapb.SegmentLevel
tsFrom typeutil.Timestamp
@ -97,6 +100,9 @@ type SyncTask struct {
failureCallback func(err error)
tr *timerecord.TimeRecorder
flushedSize int64
execTime time.Duration
}
func (t *SyncTask) getLogger() *log.MLogger {
@ -162,16 +168,17 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
return err
}
var totalSize float64
totalSize += lo.SumBy(lo.Values(t.binlogMemsize), func(fieldSize int64) float64 {
return float64(fieldSize)
})
if t.deltaBlob != nil {
totalSize += float64(len(t.deltaBlob.Value))
var totalSize int64
for _, size := range t.binlogMemsize {
totalSize += size
}
if t.deltaBlob != nil {
totalSize += int64(len(t.deltaBlob.Value))
}
t.flushedSize = totalSize
metrics.DataNodeFlushedSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.dataSource, t.level.String()).Add(totalSize)
metrics.DataNodeFlushedRows.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.dataSource).Add(float64(t.batchSize))
metrics.DataNodeFlushedSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.dataSource, t.level.String()).Add(float64(t.flushedSize))
metrics.DataNodeFlushedRows.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.dataSource).Add(float64(t.batchRows))
metrics.DataNodeSave2StorageLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.level.String()).Observe(float64(t.tr.RecordSpan().Milliseconds()))
@ -183,7 +190,7 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
}
}
actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchSize)}
actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchRows)}
if t.isFlush {
actions = append(actions, metacache.UpdateState(commonpb.SegmentState_Flushed))
}
@ -194,7 +201,8 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
log.Info("segment removed", zap.Int64("segmentID", t.segment.SegmentID()), zap.String("channel", t.channelName))
}
log.Info("task done", zap.Float64("flushedSize", totalSize), zap.Duration("interval", t.tr.RecordSpan()))
t.execTime = t.tr.RecordSpan()
log.Info("task done", zap.Int64("flushedSize", totalSize), zap.Duration("timeTaken", t.execTime))
if !t.isFlush {
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.SuccessLabel, t.level.String()).Inc()
@ -288,7 +296,7 @@ func (t *SyncTask) processBM25StastBlob() {
func (t *SyncTask) processStatsBlob() {
if t.batchStatsBlob != nil {
t.convertBlob2StatsBinlog(t.batchStatsBlob, t.pkField.GetFieldID(), t.nextID(), t.batchSize)
t.convertBlob2StatsBinlog(t.batchStatsBlob, t.pkField.GetFieldID(), t.nextID(), t.batchRows)
}
if t.mergedStatsBlob != nil {
totalRowNum := t.segment.NumOfRows()
@ -408,3 +416,16 @@ func (t *SyncTask) IsFlush() bool {
func (t *SyncTask) Binlogs() (map[int64]*datapb.FieldBinlog, map[int64]*datapb.FieldBinlog, *datapb.FieldBinlog) {
return t.insertBinlogs, t.statsBinlogs, t.deltaBinlog
}
func (t *SyncTask) MarshalJSON() ([]byte, error) {
return json.Marshal(&metricsinfo.SyncTask{
SegmentID: t.segmentID,
BatchRows: t.batchRows,
SegmentLevel: t.level.String(),
TsFrom: t.tsFrom,
TsTo: t.tsTo,
DeltaRowCount: t.deltaRowCount,
FlushSize: t.flushedSize,
RunningTime: t.execTime,
})
}

View File

@ -382,6 +382,34 @@ func (s *SyncTaskSuite) TestNextID() {
})
}
func (s *SyncTaskSuite) TestSyncTask_MarshalJSON() {
task := &SyncTask{
segmentID: 12345,
batchRows: 100,
level: datapb.SegmentLevel_L0,
tsFrom: 1000,
tsTo: 2000,
deltaRowCount: 10,
flushedSize: 1024,
execTime: 2 * time.Second,
}
expectedJSON := `{
"segment_id": 12345,
"batch_rows": 100,
"segment_level": "L0",
"ts_from": 1000,
"ts_to": 2000,
"delta_row_count": 10,
"flush_size": 1024,
"running_time": 2000000000
}`
data, err := task.MarshalJSON()
s.NoError(err)
s.JSONEq(expectedJSON, string(data))
}
func TestSyncTask(t *testing.T) {
suite.Run(t, new(SyncTaskSuite))
}

View File

@ -604,7 +604,7 @@ func (wb *writeBufferBase) getSyncTask(ctx context.Context, segmentID int64) (sy
WithLevel(segmentInfo.Level()).
WithDataSource(metrics.StreamingDataSourceLabel).
WithCheckpoint(wb.checkpoint).
WithBatchSize(batchSize).
WithBatchRows(batchSize).
WithErrorHandler(wb.errHandler)
if len(bm25) != 0 {

View File

@ -70,7 +70,14 @@ const (
ClusterClientsPath = "/_cluster/clients"
ClusterDependenciesPath = "/_cluster/dependencies"
HookConfigsPath = "/_hook/configs"
QcoordSegmentsPath = "/_qcoord/segments"
QcoordChannelsPath = "/_qcoord/channels"
QcoordTasksPath = "/_qcoord/tasks"
QCoordSegmentsPath = "/_qcoord/segments"
QCoordChannelsPath = "/_qcoord/channels"
QCoordAllTasksPath = "/_qcoord/tasks/all"
DCoordAllTasksPath = "/_dcoord/tasks/all"
DCoordImportTasksPath = "/_dcoord/tasks/import"
DCoordCompactionTasksPath = "/_dcoord/tasks/compaction"
DCoordBuildIndexTasksPath = "/_dcoord/tasks/build_index"
DNodeSyncTasksPath = "/_dnode/tasks/sync"
)

View File

@ -8,32 +8,21 @@
<meta name="description" content="Milvus Management WebUI">
<link href="./static/css/bootstrap.min.css" rel="stylesheet">
<link href="./static/css/style.css" rel="stylesheet">
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
<script>
function getQueryParams() {
const params = {};
const queryString = window.location.search.substring(1);
const queryArray = queryString.split('&');
queryArray.forEach(param => {
const [key, value] = param.split('=');
params[decodeURIComponent(key)] = decodeURIComponent(value || '');
});
return params;
}
document.addEventListener("DOMContentLoaded", function() {
const params = getQueryParams();
const errorMessage = params.error || 'No error message available.';
document.getElementById('error-message').innerText = errorMessage;
document.getElementById('error-message').innerText = "Reason: "+ errorMessage;
});
</script>
</head>
<body>
<h1>Service Unavailable</h1>
<p id="error-message"></p>
<p>Please try again later.</p>
<div class="alert alert-danger" role="alert">
<p>Service Unavailable, please try again later.</p>
<p id="error-message"></p>
</div>
</body>
</html>

View File

@ -11,28 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
// renderSysInfo(data)
// renderComponentInfo(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -154,5 +135,24 @@
</div>
<div id="footer"></div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/")
.then(text => {
//TODO add channel render
throw new Error("Unimplemented API")
})
.catch(error => {
handleError(error);
});
});
</script>
</body>
</html>

View File

@ -11,28 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
// renderSysInfo(data)
// renderComponentInfo(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -149,5 +130,20 @@
</div>
<div id="footer"></div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
fetchData(MILVUS_URI + "/", clientInfos)
.then(data => {
//TODO add collection render
})
.catch(error => {
handleError(new Error("Unimplemented API"));
});
</script>
</body>
</html>

View File

@ -11,26 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster configurations
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + '/_cluster/configs')
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
renderConfigs(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -51,6 +34,22 @@
<div class="col-md-2"></div>
</div>
<div id="footer"></div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster configurations
fetchData(MILVUS_URI + '/_cluster/configs', mconfigs)
.then(data => {
renderConfigs(data)
})
.catch(error => {
handleError(error);
});
</script>
</div>
</body>
</html>

View File

@ -1,4 +1 @@
<!--footer layout-->
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<!--footer layout-->

View File

@ -11,84 +11,74 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
const handleError = (error) => {
console.error('Error fetching data:', error);
const errorMessage = encodeURIComponent(error.message || 'Unknown error');
window.location.href = `5xx.html?error=${errorMessage}`;
};
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/_cluster/info")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
renderSysInfo(data)
renderComponentInfo(data)
})
.catch(error => {
handleError(error);
});
fetch(MILVUS_URI + "/_cluster/clients")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
renderClientsInfo(data)
})
.catch(error => {
handleError(error);
});
fetch(MILVUS_URI + "/_cluster/dependencies")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
renderDependencies(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
<div id="header"></div>
<div class="row">
<div class="col-md-2">
</div>
<div class="col-md-8">
<h2>
System Information
</h2>
<table id="sysInfo" class="table table-hover"></table>
<div class="container-fluid">
<div id="header"></div>
<div class="row">
<div class="col-md-2">
</div>
<div class="col-md-8">
<h2>
System Information
</h2>
<table id="sysInfo" class="table table-hover"></table>
<h2>
Component Information
</h2>
<table id="components" class="table table-hover"></table>
<h2>
Component Information
</h2>
<table id="components" class="table table-hover"></table>
<h2>
Connected Clients
</h2>
<table id="clients" class="table table-hover"></table>
<h2>
Connected Clients
</h2>
<table id="clients" class="table table-hover"></table>
<h2>
System Dependencies
</h2>
<table id="3rdDependency" class="table table-hover"></table>
</div>
<div class="col-md-2">
</div>
</div>
<div id="footer"></div>
<h2>
System Dependencies
</h2>
<table id="3rdDependency" class="table table-hover"></table>
</div>
<div class="col-md-2">
</div>
</div>
<div id="footer"></div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
document.addEventListener("DOMContentLoaded", function() {
fetchData(MILVUS_URI + "/_cluster/info", sysmetrics)
.then(data => {
renderSysInfo(data);
renderComponentInfo(data);
})
.catch(error => {
handleError(error);
});
fetchData(MILVUS_URI + "/_cluster/clients", clientInfos)
.then(data => {
renderClientsInfo(data);
})
.catch(error => {
handleError(error);
});
fetchData(MILVUS_URI + "/_cluster/dependencies", dependencies)
.then(data => {
renderDependencies(data);
})
.catch(error => {
handleError(error);
});
});
</script>
</body>
</html>

View File

@ -11,26 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/_cluster/info")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
renderNodesMetrics(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -57,5 +40,23 @@
<div id="footer"></div>
</div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load nodes information data
document.addEventListener("DOMContentLoaded", function() {
fetchData(MILVUS_URI + "/_cluster/info", sysmetrics)
.then(data => {
renderNodesMetrics(data)
})
.catch(error => {
handleError(error);
});
});
</script>
</body>
</html>

View File

@ -11,28 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/_cluster/info")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
// renderSysInfo(data)
// renderComponentInfo(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -193,5 +174,23 @@
</div>
<div id="footer"></div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
document.addEventListener("DOMContentLoaded", function() {
fetchData(MILVUS_URI + "/", sysmetrics)
.then(data => {
//TODO add segment render
})
.catch(error => {
handleError(new Error("Unimplemented API"));
});
});
</script>
</body>
</html>

View File

@ -0,0 +1,54 @@
var DEBUG_MODE = false; // Set this to false to disable debug mode
var MILVUS_URI = "http://127.0.0.1:9091/api/v1"
// Function to check URL for "debug" parameter and switch debug mode
function toggleDebugMode() {
// Get the current URL's query parameters
const urlParams = new URLSearchParams(window.location.search);
// Check if "debug" parameter is present and its value
if (urlParams.has('debug')) {
if (urlParams.get('debug') === 'true') {
console.log("Debug mode is ON");
// Enable debug mode: Add any additional debug functionality here
localStorage.setItem('debug', 'true');
} else {
console.log("Debug mode is OFF");
localStorage.setItem('debug', 'false');
}
}
// Check if debug mode is enabled
DEBUG_MODE = localStorage.getItem('debug') === 'true';
}
// Call the function to check the URL and apply debug mode
toggleDebugMode();
const handleError = (error) => {
console.error('Error fetching data:', error);
const errorMessage = encodeURIComponent(error.message || 'Unknown error');
window.location.href = `5xx.html?error=${errorMessage}`;
};
const fetchData = (url, localData) => {
if (DEBUG_MODE) {
return new Promise((resolve) => {
resolve(JSON.parse(localData));
});
} else {
return fetch(url)
.then(response => response.json())
}
};
function getQueryParams() {
const params = {};
const queryString = window.location.search.substring(1);
const queryArray = queryString.split('&');
queryArray.forEach(param => {
const [key, value] = param.split('=');
params[decodeURIComponent(key)] = decodeURIComponent(value || '');
});
return params;
}

View File

@ -0,0 +1,572 @@
var sysmetrics = `{
"nodes_info": [
{
"identifier": 1,
"connected": null,
"infos": {
"has_error": false,
"error_reason": "",
"name": "querynode1",
"hardware_infos": {
"ip": "172.18.20.7:21123",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "querynode",
"id": 1,
"system_configurations": {
"simd_type": "auto"
},
"quota_metrics": {
"Hms": {
"ip": "172.18.20.7:21123",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"Rms": [
{
"Label": "InsertConsumeThroughput",
"Rate": 0
},
{
"Label": "DeleteConsumeThroughput",
"Rate": 0
}
],
"Fgm": {
"MinFlowGraphChannel": "",
"MinFlowGraphTt": 18446744073709551615,
"NumFlowGraph": 0
},
"GrowingSegmentsSize": 0,
"Effect": {
"NodeID": 1,
"CollectionIDs": []
},
"DeleteBufferInfo": {
"CollectionDeleteBufferNum": {},
"CollectionDeleteBufferSize": {}
}
},
"collection_metrics": {
"CollectionRows": {}
}
}
},
{
"identifier": 1,
"connected": [
{
"connected_identifier": 1,
"type": "manage",
"target_type": "querynode"
}
],
"infos": {
"has_error": false,
"error_reason": "",
"name": "querycoord1",
"hardware_infos": {
"ip": "172.18.20.7:19531",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "querycoord",
"id": 1,
"system_configurations": {
"search_channel_prefix": "",
"search_result_channel_prefix": ""
}
}
},
{
"identifier": 1,
"connected": null,
"infos": {
"has_error": false,
"error_reason": "",
"name": "datanode1",
"hardware_infos": {
"ip": "172.18.20.7:21124",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "datanode",
"id": 1,
"system_configurations": {
"flush_insert_buffer_size": 16777216
},
"quota_metrics": {
"Hms": {
"ip": "172.18.20.7:21124",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"Rms": [
{
"Label": "InsertConsumeThroughput",
"Rate": 0
},
{
"Label": "DeleteConsumeThroughput",
"Rate": 0
}
],
"Fgm": {
"MinFlowGraphChannel": "",
"MinFlowGraphTt": 18446744073709551615,
"NumFlowGraph": 0
},
"Effect": {
"NodeID": 1,
"CollectionIDs": []
}
}
}
},
{
"identifier": 1,
"connected": null,
"infos": {
"has_error": false,
"error_reason": "",
"name": "indexnode1",
"hardware_infos": {
"ip": "172.18.20.7:21121",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "indexnode",
"id": 1,
"system_configurations": {
"minio_bucket_name": "a-bucket",
"simd_type": "auto"
}
}
},
{
"identifier": 1,
"connected": [
{
"connected_identifier": 1,
"type": "manage",
"target_type": "datanode"
},
{
"connected_identifier": 1,
"type": "manage",
"target_type": "indexnode"
}
],
"infos": {
"has_error": false,
"error_reason": "",
"name": "datacoord1",
"hardware_infos": {
"ip": "172.18.20.7:13333",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "datacoord",
"id": 1,
"system_configurations": {
"segment_max_size": 1024
},
"quota_metrics": {
"TotalBinlogSize": 0,
"CollectionBinlogSize": {},
"PartitionsBinlogSize": {},
"CollectionL0RowCount": {}
},
"collection_metrics": {
"Collections": {}
}
}
},
{
"identifier": 1,
"connected": [],
"infos": {
"has_error": false,
"error_reason": "",
"name": "rootcoord1",
"hardware_infos": {
"ip": "172.18.20.7:53100",
"cpu_core_count": 8,
"cpu_core_usage": 0,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "rootcoord",
"id": 1,
"system_configurations": {
"min_segment_size_to_enable_index": 1024
}
}
},
{
"identifier": 1,
"connected": [
{
"connected_identifier": 1,
"type": "forward",
"target_type": "querycoord"
},
{
"connected_identifier": 1,
"type": "forward",
"target_type": "datacoord"
},
{
"connected_identifier": 1,
"type": "forward",
"target_type": "rootcoord"
}
],
"infos": {
"has_error": false,
"error_reason": "",
"name": "proxy1",
"hardware_infos": {
"ip": "172.18.20.7:19529",
"cpu_core_count": 8,
"cpu_core_usage": 30.52004762940343,
"memory": 34359738368,
"memory_usage": 18362302464,
"disk": 104857600,
"disk_usage": 2097152
},
"system_info": {
"system_version": "34cf5352ec",
"deploy_mode": "STANDALONE[MOCK]",
"build_version": "v2.2-testing-20240702-804-g34cf5352ec-dev",
"build_time": "Wed, Oct 23, 2024 13:58:54 UTC",
"used_go_version": "go version go1.22.3 darwin/amd64"
},
"created_time": "2024-10-23 22:01:45.891414 +0800 CST m=+8.035797839",
"updated_time": "2024-10-23 22:01:45.891415 +0800 CST m=+8.035798239",
"type": "proxy",
"id": 1,
"system_configurations": {
"default_partition_name": "_default",
"default_index_name": "_default_idx"
},
"quota_metrics": null
}
}
]
}`
var clientInfos = `[
{
"sdk_type": "python",
"sdk_version": "1.0.0",
"local_time": "2023-10-01T12:00:00Z",
"user": "user1",
"host": "127.0.0.1",
"reserved": {
"last_active_time": ""
}
},
{
"sdk_type": "golang",
"sdk_version": "1.1.0",
"local_time": "2023-10-01T12:05:00Z",
"user": "user2",
"host": "127.0.0.2",
"reserved": {
"last_active_time": ""
}
}
]`
var dependencies = `
{
"metastore": {
"health_status": true,
"unhealthy_reason": "",
"members_health": [
{
"endpoint": "http://127.0.0.1:2379",
"health": true
}
],
"meta_type": "etcd"
},
"mq": {
"health_status": false,
"unhealthy_reason": "health check failed, err: Get \\"http://localhost:80/admin/v2/brokers/health\\": dial tcp [::1]:80: connect: connection refused",
"members_health": null,
"mq_type": "pulsar"
}
}
`
var mconfigs = `
{
"MILVUS_GIT_BUILD_TAGS": "v2.2-testing-20240702-811-g38211f2b81-dev",
"MILVUS_GIT_COMMIT": "38211f2b81",
"common.bloomfilterapplybatchsize": "1000",
"common.bloomfiltersize": "100000",
"common.bloomfiltertype": "BlockedBloomFilter",
"common.buildindexthreadpoolratio": "0.75",
"common.defaultindexname": "_default_idx",
"common.defaultpartitionname": "_default",
"common.diskindex.beamwidthratio": "4",
"common.diskindex.buildnumthreadsratio": "1",
"common.diskindex.loadnumthreadratio": "8",
"common.diskindex.maxdegree": "56",
"common.diskindex.pqcodebudgetgbratio": "0.125",
"common.diskindex.searchcachebudgetgbratio": "0.1",
"common.diskindex.searchlistsize": "100",
"common.enablevectorclusteringkey": "false",
"common.entityexpiration": "-1",
"common.gracefulstoptimeout": "1800",
"common.gracefultime": "5000",
"common.indexslicesize": "16",
"common.locks.metrics.enable": "false"
}
`;
var qcTasks = `
[
{
"task_name": "balance_checker-ChannelTask[1]-ch1",
"collection_id": 67890,
"replica_id": 11111,
"task_type": "Move",
"task_status": "started",
"priority": "Normal",
"actions": [
"type:Grow node id : 1 channel name:channel_1"
],
"step": 1,
"reason": "some reason"
},
{
"task_name": "index_checker-SegmentTask[2]-54321",
"collection_id": 12345,
"replica_id": 22222,
"task_type": "Grow",
"task_status": "completed",
"priority": "High",
"actions": [
"type:Grow node id: 2 segment id:123 scope:DataScope_Streaming"
],
"step": 2,
"reason": "another reason"
},
{
"task_name": "leader_checker-LeaderSegmentTask[3]-1",
"collection_id": 54321,
"replica_id": 33333,
"task_type": "Grow",
"task_status": "failed",
"priority": "Low",
"actions": [
"type:Grow node id: 3 leader id:456 segment id:789 version:1"
],
"step": 3,
"reason": "yet another reason"
}
]
`
var dc_build_index_task = `
[
{
"index_id": 1,
"collection_id": 1001,
"segment_id": 2001,
"build_id": 3001,
"index_state": "Finished",
"index_size": 1024,
"index_version": 1,
"create_time": 1633036800
},
{
"index_id": 2,
"collection_id": 1002,
"segment_id": 2002,
"build_id": 3002,
"index_state": "Failed",
"fail_reason": "Disk full",
"index_size": 2048,
"index_version": 2,
"create_time": 1633123200
}
]`
var dc_compaction_task = `
[
{
"plan_id": 1,
"collection_id": 1001,
"type": "Merge",
"state": "Completed",
"start_time": 1633036800,
"end_time": 1633040400,
"total_rows": 100000,
"input_segments": [1, 2, 3],
"result_segments": [4]
},
{
"plan_id": 2,
"collection_id": 1002,
"type": "Merge",
"state": "Failed",
"fail_reason": "Disk full",
"start_time": 1633123200,
"end_time": 1633126800,
"total_rows": 200000,
"input_segments": [5, 6, 7],
"result_segments": [8]
}
]`
var dc_sync_task = `
[
{
"segment_id": 1,
"batch_rows": 1000,
"segment_level": "L1",
"ts_from": 1633036800,
"ts_to": 1633040400,
"delta_row_count": 10,
"flush_size": 1024,
"running_time": 100000000
},
{
"segment_id": 2,
"batch_rows": 2000,
"segment_level": "L2",
"ts_from": 1633123200,
"ts_to": 1633126800,
"delta_row_count": 20,
"flush_size": 2048,
"running_time": 200000000
}
]
`
var dc_import_task = `
[
{
"job_id": 1,
"task_id": 2,
"collection_id": 3,
"node_id": 4,
"state": "Pending",
"reason": "",
"task_type": "PreImportTask",
"created_time": "2023-10-01T00:00:00Z",
"complete_time": "2023-10-01T01:00:00Z"
},
{
"job_id": 5,
"task_id": 6,
"collection_id": 7,
"node_id": 8,
"state": "ImportTaskStateCompleted",
"reason": "",
"task_type": "Completed",
"created_time": "2023-10-01T00:00:00Z",
"complete_time": "2023-10-01T01:00:00Z"
},
{
"job_id": 9,
"task_id": 10,
"collection_id": 11,
"node_id": 12,
"state": "Failed",
"reason": "some failure reason",
"task_type": "ImportTask",
"created_time": "2023-10-01T00:00:00Z",
"complete_time": "2023-10-01T01:00:00Z"
}
]
`

View File

@ -1,4 +1,3 @@
var MILVUS_URI = "http://127.0.0.1:9091/api/v1"
function renderNodesMetrics(data) {
let tableHTML = '<thead class="thead-light"><tr>' +
@ -104,7 +103,7 @@ function renderClientsInfo(data) {
<td>${client['sdk_type']}</td>
<td>${client['sdk_version']}</td>
<td>${client['local_time']}</td>
<td>${client['reserved']['last_active_time']}</td>
<td>${client['reserved'] ? client['reserved']['last_active_time']: ""}</td>
</tr>`;
tableHTML += tr
});

View File

@ -11,28 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
// renderSysInfo(data)
// renderComponentInfo(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -167,5 +148,22 @@
</div>
<div id="footer"></div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
document.addEventListener("DOMContentLoaded", function() {
fetchData(MILVUS_URI + "/", sysmetrics)
.then(data => {
//TODO add tasks render
})
.catch(error => {
handleError(new Error("Unimplemented API"));
});
});
</script>
</body>
</html>

View File

@ -11,28 +11,9 @@
<script src="./static/js/jquery.min.js"></script>
<script src="./static/js/bootstrap.min.js"></script>
<script src="./static/js/bootstrap.bundle.min.js"></script>
<script src="./static/js/scripts.js"></script>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
// load cluster information data
document.addEventListener("DOMContentLoaded", function() {
fetch(MILVUS_URI + "/")
.then(response => response.json())
.then(text => {
data = JSON.parse(text)
// renderSysInfo(data)
// renderComponentInfo(data)
})
.catch(error => {
handleError(error);
});
});
</script>
<script src="./static/js/render.js"></script>
<script src="./static/js/common.js"></script>
<script src="./static/js/mockdata.js"></script>
</head>
<body>
<div class="container-fluid">
@ -48,5 +29,21 @@
</div>
<div id="footer"></div>
</div>
<script>
$(document).ready(function(){
$('#header').load("header.html");
$('#footer').load("footer.html");
});
document.addEventListener("DOMContentLoaded", function() {
fetchData(MILVUS_URI + "/", null)
.then(data => {
})
.catch(error => {
handleError(new Error("Unimplemented API"));
});
});
</script>
</body>
</html>

View File

@ -900,6 +900,8 @@ message PreImportTask {
ImportTaskStateV2 state = 7;
string reason = 8;
repeated ImportFileStats file_stats = 10;
string created_time = 11;
string complete_time = 12;
}
message ImportTaskV2 {
@ -913,6 +915,7 @@ message ImportTaskV2 {
string complete_time = 8;
repeated ImportFileStats file_stats = 9;
repeated int64 stats_segmentIDs = 10;
string created_time = 11;
}
enum GcCommand {

View File

@ -32,16 +32,18 @@ import (
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
var contentType = "application/json"
func getConfigs(configs map[string]string) gin.HandlerFunc {
return func(c *gin.Context) {
bs, err := json.Marshal(configs)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
}
c.IndentedJSON(http.StatusOK, string(bs))
c.Data(http.StatusOK, contentType, bs)
}
}
@ -49,7 +51,7 @@ func getClusterInfo(node *Proxy) gin.HandlerFunc {
return func(c *gin.Context) {
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
@ -61,7 +63,7 @@ func getClusterInfo(node *Proxy) gin.HandlerFunc {
var err1 error
resp, err1 = getSystemInfoMetrics(c, req, node)
if err1 != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err1.Error(),
})
return
@ -70,13 +72,12 @@ func getClusterInfo(node *Proxy) gin.HandlerFunc {
}
if !merr.Ok(resp.GetStatus()) {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: resp.Status.Reason,
})
return
}
c.IndentedJSON(http.StatusOK, resp.GetResponse())
c.Data(http.StatusOK, contentType, []byte(resp.GetResponse()))
}
}
@ -84,12 +85,12 @@ func getConnectedClients(c *gin.Context) {
clients := connection.GetManager().List()
ret, err := json.Marshal(clients)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
}
c.IndentedJSON(http.StatusOK, string(ret))
c.Data(http.StatusOK, contentType, ret)
}
func getDependencies(c *gin.Context) {
@ -109,12 +110,12 @@ func getDependencies(c *gin.Context) {
etcdConfig.EtcdTLSMinVersion.GetValue())
ret, err := json.Marshal(dependencies)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
}
c.IndentedJSON(http.StatusOK, string(ret))
c.Data(http.StatusOK, contentType, ret)
}
// buildReqParams fetch all parameters from query parameter of URL, add them into a map data structure.
@ -139,7 +140,7 @@ func getQueryComponentMetrics(node *Proxy, metricsType string) gin.HandlerFunc {
params := buildReqParams(c, metricsType)
req, err := metricsinfo.ConstructGetMetricsRequest(params)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
@ -147,12 +148,33 @@ func getQueryComponentMetrics(node *Proxy, metricsType string) gin.HandlerFunc {
resp, err := node.queryCoord.GetMetrics(c, req)
if err := merr.CheckRPCCall(resp, err); err != nil {
c.JSON(http.StatusInternalServerError, gin.H{
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
}
c.Data(http.StatusOK, contentType, []byte(resp.GetResponse()))
}
}
func getDataComponentMetrics(node *Proxy, metricsType string) gin.HandlerFunc {
return func(c *gin.Context) {
params := buildReqParams(c, metricsType)
req, err := metricsinfo.ConstructGetMetricsRequest(params)
if err != nil {
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
}
c.IndentedJSON(http.StatusOK, resp.Response)
resp, err := node.dataCoord.GetMetrics(c, req)
if err := merr.CheckRPCCall(resp, err); err != nil {
c.AbortWithStatusJSON(http.StatusInternalServerError, gin.H{
mhttp.HTTPReturnMessage: err.Error(),
})
return
}
c.Data(http.StatusOK, contentType, []byte(resp.GetResponse()))
}
}

View File

@ -118,3 +118,36 @@ func TestGetQueryComponentMetrics(t *testing.T) {
assert.Contains(t, w.Body.String(), "test_response")
})
}
func TestGetDataComponentMetrics(t *testing.T) {
t.Run("get metrics failed", func(t *testing.T) {
w := httptest.NewRecorder()
c, _ := gin.CreateTestContext(w)
c.Request, _ = http.NewRequest("GET", "/?key=value", nil)
dc := mocks.NewMockDataCoordClient(t)
dc.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(nil, errors.New("error"))
proxy := &Proxy{dataCoord: dc}
handler := getDataComponentMetrics(proxy, "system_info")
handler(c)
assert.Equal(t, http.StatusInternalServerError, w.Code)
assert.Contains(t, w.Body.String(), "error")
})
t.Run("ok", func(t *testing.T) {
w := httptest.NewRecorder()
c, _ := gin.CreateTestContext(w)
c.Request, _ = http.NewRequest("GET", "/?key=value", nil)
dc := mocks.NewMockDataCoordClient(t)
dc.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Response: "test_response",
}, nil)
proxy := &Proxy{dataCoord: dc}
handler := getDataComponentMetrics(proxy, "test_metric")
handler(c)
assert.Equal(t, http.StatusOK, w.Code)
assert.Contains(t, w.Body.String(), "test_response")
})
}

View File

@ -6513,7 +6513,16 @@ func (node *Proxy) RegisterRestRouter(router gin.IRouter) {
router.GET(http.HookConfigsPath, getConfigs(paramtable.GetHookParams().GetAll()))
// QueryCoord request
router.GET(http.QcoordSegmentsPath, getQueryComponentMetrics(node, metricsinfo.QuerySegmentDist))
router.GET(http.QcoordChannelsPath, getQueryComponentMetrics(node, metricsinfo.QueryChannelDist))
router.GET(http.QcoordTasksPath, getQueryComponentMetrics(node, metricsinfo.QueryTasks))
router.GET(http.QCoordSegmentsPath, getQueryComponentMetrics(node, metricsinfo.QuerySegmentDist))
router.GET(http.QCoordChannelsPath, getQueryComponentMetrics(node, metricsinfo.QueryChannelDist))
router.GET(http.QCoordAllTasksPath, getQueryComponentMetrics(node, metricsinfo.QueryCoordAllTasks))
// DataCoord request
router.GET(http.DCoordAllTasksPath, getDataComponentMetrics(node, metricsinfo.DataCoordAllTasks))
router.GET(http.DCoordCompactionTasksPath, getDataComponentMetrics(node, metricsinfo.CompactionTasks))
router.GET(http.DCoordImportTasksPath, getDataComponentMetrics(node, metricsinfo.ImportTasks))
router.GET(http.DCoordBuildIndexTasksPath, getDataComponentMetrics(node, metricsinfo.BuildIndexTasks))
// Datanode request
router.GET(http.DNodeSyncTasksPath, getDataComponentMetrics(node, metricsinfo.SyncTasks))
}

View File

@ -20,10 +20,13 @@ import (
"context"
"encoding/base64"
"fmt"
"net/http"
"net/http/httptest"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/gin-gonic/gin"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"go.uber.org/zap"
@ -36,6 +39,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
mhttp "github.com/milvus-io/milvus/internal/http"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
@ -1856,3 +1860,42 @@ func TestProxy_InvalidateShardLeaderCache(t *testing.T) {
assert.True(t, merr.Ok(resp))
})
}
func TestRegisterRestRouter(t *testing.T) {
gin.SetMode(gin.TestMode)
router := gin.New()
dc := mocks.NewMockDataCoordClient(t)
dc.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(nil, errors.New("error"))
qc := mocks.NewMockQueryCoordClient(t)
qc.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(nil, errors.New("error"))
proxy := &Proxy{
dataCoord: dc,
queryCoord: qc,
}
proxy.RegisterRestRouter(router)
tests := []struct {
path string
statusCode int
}{
{path: mhttp.QCoordSegmentsPath, statusCode: http.StatusInternalServerError},
{path: mhttp.QCoordChannelsPath, statusCode: http.StatusInternalServerError},
{path: mhttp.QCoordAllTasksPath, statusCode: http.StatusInternalServerError},
{path: mhttp.DNodeSyncTasksPath, statusCode: http.StatusInternalServerError},
{path: mhttp.DCoordCompactionTasksPath, statusCode: http.StatusInternalServerError},
{path: mhttp.DCoordImportTasksPath, statusCode: http.StatusInternalServerError},
{path: mhttp.DCoordBuildIndexTasksPath, statusCode: http.StatusInternalServerError},
{path: mhttp.DNodeSyncTasksPath, statusCode: http.StatusInternalServerError},
}
for _, tt := range tests {
t.Run(tt.path, func(t *testing.T) {
req, _ := http.NewRequest("GET", tt.path, nil)
w := httptest.NewRecorder()
router.ServeHTTP(w, req)
assert.Equal(t, tt.statusCode, w.Code)
})
}
}

View File

@ -142,7 +142,7 @@ func (suite *IndexCheckerSuite) TestLoadIndex() {
suite.Require().True(ok)
suite.EqualValues(200, t.ReplicaID())
suite.Equal(task.ActionTypeUpdate, action.Type())
suite.EqualValues(2, action.SegmentID())
suite.EqualValues(2, action.GetSegmentID())
// test skip load index for read only node
suite.nodeMgr.Stopping(1)

View File

@ -154,7 +154,7 @@ func (suite *SegmentCheckerTestSuite) TestLoadSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeGrow, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
// test activation
@ -223,7 +223,7 @@ func (suite *SegmentCheckerTestSuite) TestLoadL0Segments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeGrow, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
@ -236,7 +236,7 @@ func (suite *SegmentCheckerTestSuite) TestLoadL0Segments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeGrow, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
@ -251,7 +251,7 @@ func (suite *SegmentCheckerTestSuite) TestLoadL0Segments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeGrow, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
@ -326,7 +326,7 @@ func (suite *SegmentCheckerTestSuite) TestReleaseL0Segments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
@ -405,7 +405,7 @@ func (suite *SegmentCheckerTestSuite) TestReleaseSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(2, action.SegmentID())
suite.EqualValues(2, action.GetSegmentID())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
@ -447,7 +447,7 @@ func (suite *SegmentCheckerTestSuite) TestReleaseRepeatedSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(1, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityLow)
@ -504,7 +504,7 @@ func (suite *SegmentCheckerTestSuite) TestReleaseDirtySegments() {
suite.True(ok)
suite.EqualValues(-1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
@ -558,7 +558,7 @@ func (suite *SegmentCheckerTestSuite) TestSkipReleaseSealedSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(segmentID, action.SegmentID())
suite.EqualValues(segmentID, action.GetSegmentID())
suite.EqualValues(nodeID, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
@ -573,7 +573,7 @@ func (suite *SegmentCheckerTestSuite) TestSkipReleaseSealedSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(segmentID, action.SegmentID())
suite.EqualValues(segmentID, action.GetSegmentID())
suite.EqualValues(nodeID, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
@ -625,14 +625,14 @@ func (suite *SegmentCheckerTestSuite) TestReleaseGrowingSegments() {
tasks := checker.Check(context.TODO())
suite.Len(tasks, 2)
sort.Slice(tasks, func(i, j int) bool {
return tasks[i].Actions()[0].(*task.SegmentAction).SegmentID() < tasks[j].Actions()[0].(*task.SegmentAction).SegmentID()
return tasks[i].Actions()[0].(*task.SegmentAction).GetSegmentID() < tasks[j].Actions()[0].(*task.SegmentAction).GetSegmentID()
})
suite.Len(tasks[0].Actions(), 1)
action, ok := tasks[0].Actions()[0].(*task.SegmentAction)
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(2, action.SegmentID())
suite.EqualValues(2, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
@ -641,7 +641,7 @@ func (suite *SegmentCheckerTestSuite) TestReleaseGrowingSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[1].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(3, action.SegmentID())
suite.EqualValues(3, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[1].Priority(), task.TaskPriorityNormal)
}
@ -689,7 +689,7 @@ func (suite *SegmentCheckerTestSuite) TestSkipReleaseGrowingSegments() {
suite.True(ok)
suite.EqualValues(1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(2, action.SegmentID())
suite.EqualValues(2, action.GetSegmentID())
suite.EqualValues(2, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}
@ -704,7 +704,7 @@ func (suite *SegmentCheckerTestSuite) TestReleaseDroppedSegments() {
suite.True(ok)
suite.EqualValues(-1, tasks[0].ReplicaID())
suite.Equal(task.ActionTypeReduce, action.Type())
suite.EqualValues(1, action.SegmentID())
suite.EqualValues(1, action.GetSegmentID())
suite.EqualValues(1, action.Node())
suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal)
}

View File

@ -135,6 +135,8 @@ type Server struct {
proxyCreator proxyutil.ProxyCreator
proxyWatcher proxyutil.ProxyWatcherInterface
proxyClientManager proxyutil.ProxyClientManagerInterface
metricsRequest *metricsinfo.MetricsRequest
}
func NewQueryCoord(ctx context.Context) (*Server, error) {
@ -145,6 +147,7 @@ func NewQueryCoord(ctx context.Context) (*Server, error) {
nodeUpEventChan: make(chan int64, 10240),
notifyNodeUp: make(chan struct{}),
balancerMap: make(map[string]balance.Balance),
metricsRequest: metricsinfo.NewMetricsRequest(),
}
server.UpdateStateCode(commonpb.StateCode_Abnormal)
server.queryNodeCreator = session.DefaultQueryNodeCreator
@ -191,7 +194,13 @@ func (s *Server) registerMetricsRequest() {
getSystemInfoAction := func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.getSystemInfoMetrics(ctx, req)
}
metricsinfo.RegisterMetricsRequest(metricsinfo.SystemInfoMetrics, getSystemInfoAction)
QueryTasksAction := func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return s.taskScheduler.GetTasksJSON(), nil
}
s.metricsRequest.RegisterMetricsRequest(metricsinfo.SystemInfoMetrics, getSystemInfoAction)
s.metricsRequest.RegisterMetricsRequest(metricsinfo.QueryCoordAllTasks, QueryTasksAction)
log.Info("register metrics actions finished")
}

View File

@ -847,7 +847,7 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
paramtable.GetNodeID()),
}
ret, err := metricsinfo.ExecuteMetricsRequest(ctx, req)
ret, err := s.metricsRequest.ExecuteMetricsRequest(ctx, req)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil

View File

@ -213,6 +213,7 @@ func (suite *ServiceSuite) SetupTest() {
getBalancerFunc: func() balance.Balance { return suite.balancer },
distController: suite.distController,
ctx: context.Background(),
metricsRequest: metricsinfo.NewMetricsRequest(),
}
suite.server.registerMetricsRequest()
@ -1357,8 +1358,8 @@ func (suite *ServiceSuite) TestLoadBalanceWithEmptySegmentList() {
suite.Len(actions, 2)
growAction := actions[0].(*task.SegmentAction)
reduceAction := actions[1].(*task.SegmentAction)
suite.True(lo.Contains(segmentOnCollection[collection], growAction.SegmentID()))
suite.True(lo.Contains(segmentOnCollection[collection], reduceAction.SegmentID()))
suite.True(lo.Contains(segmentOnCollection[collection], growAction.GetSegmentID()))
suite.True(lo.Contains(segmentOnCollection[collection], reduceAction.GetSegmentID()))
suite.Equal(dstNode, growAction.Node())
suite.Equal(srcNode, reduceAction.Node())
t.Cancel(nil)

View File

@ -51,44 +51,45 @@ type Action interface {
Node() int64
Type() ActionType
IsFinished(distMgr *meta.DistributionManager) bool
Desc() string
String() string
}
type BaseAction struct {
nodeID typeutil.UniqueID
typ ActionType
shard string
NodeID typeutil.UniqueID
Typ ActionType
Shard string
}
func NewBaseAction(nodeID typeutil.UniqueID, typ ActionType, shard string) *BaseAction {
return &BaseAction{
nodeID: nodeID,
typ: typ,
shard: shard,
NodeID: nodeID,
Typ: typ,
Shard: shard,
}
}
func (action *BaseAction) Node() int64 {
return action.nodeID
return action.NodeID
}
func (action *BaseAction) Type() ActionType {
return action.typ
return action.Typ
}
func (action *BaseAction) Shard() string {
return action.shard
func (action *BaseAction) GetShard() string {
return action.Shard
}
func (action *BaseAction) String() string {
return fmt.Sprintf(`{[type=%v][node=%d][shard=%v]}`, action.Type(), action.Node(), action.Shard())
return fmt.Sprintf(`{[type=%v][node=%d][shard=%v]}`, action.Type(), action.Node(), action.Shard)
}
type SegmentAction struct {
*BaseAction
segmentID typeutil.UniqueID
scope querypb.DataScope
SegmentID typeutil.UniqueID
Scope querypb.DataScope
rpcReturned atomic.Bool
}
@ -101,18 +102,18 @@ func NewSegmentActionWithScope(nodeID typeutil.UniqueID, typ ActionType, shard s
base := NewBaseAction(nodeID, typ, shard)
return &SegmentAction{
BaseAction: base,
segmentID: segmentID,
scope: scope,
SegmentID: segmentID,
Scope: scope,
rpcReturned: *atomic.NewBool(false),
}
}
func (action *SegmentAction) SegmentID() typeutil.UniqueID {
return action.segmentID
func (action *SegmentAction) GetSegmentID() typeutil.UniqueID {
return action.SegmentID
}
func (action *SegmentAction) Scope() querypb.DataScope {
return action.scope
func (action *SegmentAction) GetScope() querypb.DataScope {
return action.Scope
}
func (action *SegmentAction) IsFinished(distMgr *meta.DistributionManager) bool {
@ -123,13 +124,13 @@ func (action *SegmentAction) IsFinished(distMgr *meta.DistributionManager) bool
}
// segment found in leader view
views := distMgr.LeaderViewManager.GetByFilter(meta.WithSegment2LeaderView(action.segmentID, false))
views := distMgr.LeaderViewManager.GetByFilter(meta.WithSegment2LeaderView(action.SegmentID, false))
if len(views) == 0 {
return false
}
// segment found in dist
segmentInTargetNode := distMgr.SegmentDistManager.GetByFilter(meta.WithNodeID(action.Node()), meta.WithSegmentID(action.SegmentID()))
segmentInTargetNode := distMgr.SegmentDistManager.GetByFilter(meta.WithNodeID(action.Node()), meta.WithSegmentID(action.SegmentID))
return len(segmentInTargetNode) > 0
} else if action.Type() == ActionTypeReduce {
// FIXME: Now shard leader's segment view is a map of segment ID to node ID,
@ -148,7 +149,7 @@ func (action *SegmentAction) IsFinished(distMgr *meta.DistributionManager) bool
segments = append(segments, segment.GetID())
}
segments = append(segments, growing...)
if !funcutil.SliceContain(segments, action.SegmentID()) {
if !funcutil.SliceContain(segments, action.GetSegmentID()) {
return true
}
return action.rpcReturned.Load()
@ -159,8 +160,12 @@ func (action *SegmentAction) IsFinished(distMgr *meta.DistributionManager) bool
return true
}
func (action *SegmentAction) Desc() string {
return fmt.Sprintf("type:%s node id: %d, data scope:%s", action.Type().String(), action.Node(), action.Scope.String())
}
func (action *SegmentAction) String() string {
return action.BaseAction.String() + fmt.Sprintf(`{[segmentID=%d][scope=%d]}`, action.SegmentID(), action.Scope())
return action.BaseAction.String() + fmt.Sprintf(`{[segmentID=%d][scope=%d]}`, action.SegmentID, action.Scope)
}
type ChannelAction struct {
@ -174,7 +179,11 @@ func NewChannelAction(nodeID typeutil.UniqueID, typ ActionType, channelName stri
}
func (action *ChannelAction) ChannelName() string {
return action.shard
return action.Shard
}
func (action *ChannelAction) Desc() string {
return fmt.Sprintf("type:%s node id: %d", action.Type().String(), action.Node())
}
func (action *ChannelAction) IsFinished(distMgr *meta.DistributionManager) bool {
@ -232,6 +241,11 @@ func (action *LeaderAction) PartStats() map[int64]int64 {
return action.partStatsVersions
}
func (action *LeaderAction) Desc() string {
return fmt.Sprintf("type:%s, node id: %d, segment id:%d ,version:%d, leader id:%d",
action.Type().String(), action.Node(), action.SegmentID(), action.Version(), action.GetLeaderID())
}
func (action *LeaderAction) String() string {
partStatsStr := ""
if action.PartStats() != nil {
@ -246,7 +260,7 @@ func (action *LeaderAction) GetLeaderID() typeutil.UniqueID {
}
func (action *LeaderAction) IsFinished(distMgr *meta.DistributionManager) bool {
views := distMgr.LeaderViewManager.GetByFilter(meta.WithNodeID2LeaderView(action.leaderID), meta.WithChannelName2LeaderView(action.Shard()))
views := distMgr.LeaderViewManager.GetByFilter(meta.WithNodeID2LeaderView(action.leaderID), meta.WithChannelName2LeaderView(action.Shard))
if len(views) == 0 {
return false
}

View File

@ -192,7 +192,7 @@ func (ex *Executor) loadSegment(task *SegmentTask, step int) error {
return err
}
loadInfo, indexInfos, err := ex.getLoadInfo(ctx, task.CollectionID(), action.SegmentID(), channel)
loadInfo, indexInfos, err := ex.getLoadInfo(ctx, task.CollectionID(), action.SegmentID, channel)
if err != nil {
return err
}
@ -215,7 +215,7 @@ func (ex *Executor) loadSegment(task *SegmentTask, step int) error {
log.Warn(msg, zap.Error(err))
return err
}
view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard()))
view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard))
if view == nil {
msg := "no shard leader for the segment to execute loading"
err = merr.WrapErrChannelNotFound(task.Shard(), "shard delegator not found")
@ -265,7 +265,7 @@ func (ex *Executor) releaseSegment(task *SegmentTask, step int) {
req.Checkpoint = channel.GetSeekPosition()
}
if action.Scope() == querypb.DataScope_Streaming {
if action.Scope == querypb.DataScope_Streaming {
// Any modification to the segment distribution have to set NeedTransfer true,
// to protect the version, which serves search/query
req.NeedTransfer = true
@ -282,7 +282,7 @@ func (ex *Executor) releaseSegment(task *SegmentTask, step int) {
dstNode = action.Node()
req.NeedTransfer = false
} else {
view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard()))
view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard))
if view == nil {
msg := "no shard leader for the segment to execute releasing"
err := merr.WrapErrChannelNotFound(task.Shard(), "shard delegator not found")

View File

@ -361,6 +361,47 @@ func (_c *MockScheduler_GetSegmentTaskNum_Call) RunAndReturn(run func() int) *Mo
return _c
}
// GetTasksJSON provides a mock function with given fields:
func (_m *MockScheduler) GetTasksJSON() string {
ret := _m.Called()
var r0 string
if rf, ok := ret.Get(0).(func() string); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(string)
}
return r0
}
// MockScheduler_GetTasksJSON_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetTasksJSON'
type MockScheduler_GetTasksJSON_Call struct {
*mock.Call
}
// GetTasksJSON is a helper method to define mock.On call
func (_e *MockScheduler_Expecter) GetTasksJSON() *MockScheduler_GetTasksJSON_Call {
return &MockScheduler_GetTasksJSON_Call{Call: _e.mock.On("GetTasksJSON")}
}
func (_c *MockScheduler_GetTasksJSON_Call) Run(run func()) *MockScheduler_GetTasksJSON_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockScheduler_GetTasksJSON_Call) Return(_a0 string) *MockScheduler_GetTasksJSON_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockScheduler_GetTasksJSON_Call) RunAndReturn(run func() string) *MockScheduler_GetTasksJSON_Call {
_c.Call.Return(run)
return _c
}
// RemoveByNode provides a mock function with given fields: node
func (_m *MockScheduler) RemoveByNode(node int64) {
_m.Called(node)

View File

@ -23,6 +23,8 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/goccy/go-json"
"github.com/hashicorp/golang-lru/v2/expirable"
"github.com/samber/lo"
"go.uber.org/atomic"
"go.uber.org/zap"
@ -67,7 +69,7 @@ type replicaSegmentIndex struct {
}
func NewReplicaSegmentIndex(task *SegmentTask) replicaSegmentIndex {
isGrowing := task.Actions()[0].(*SegmentAction).Scope() == querypb.DataScope_Streaming
isGrowing := task.Actions()[0].(*SegmentAction).GetScope() == querypb.DataScope_Streaming
return replicaSegmentIndex{
ReplicaID: task.ReplicaID(),
SegmentID: task.SegmentID(),
@ -144,6 +146,7 @@ type Scheduler interface {
GetExecutedFlag(nodeID int64) <-chan struct{}
GetChannelTaskNum() int
GetSegmentTaskNum() int
GetTasksJSON() string
GetSegmentTaskDelta(nodeID int64, collectionID int64) int
GetChannelTaskDelta(nodeID int64, collectionID int64) int
@ -167,6 +170,7 @@ type taskScheduler struct {
channelTasks map[replicaChannelIndex]Task
processQueue *taskQueue
waitQueue *taskQueue
taskStats *expirable.LRU[UniqueID, Task]
}
func NewScheduler(ctx context.Context,
@ -198,6 +202,7 @@ func NewScheduler(ctx context.Context,
channelTasks: make(map[replicaChannelIndex]Task),
processQueue: newTaskQueue(),
waitQueue: newTaskQueue(),
taskStats: expirable.NewLRU[UniqueID, Task](512, nil, time.Minute*30),
}
}
@ -279,6 +284,7 @@ func (scheduler *taskScheduler) Add(task Task) error {
scheduler.segmentTasks[index] = task
}
scheduler.taskStats.Add(task.ID(), task)
scheduler.updateTaskMetrics()
log.Ctx(task.Context()).Info("task added", zap.String("task", task.String()))
task.RecordStartTs()
@ -526,8 +532,8 @@ func (scheduler *taskScheduler) calculateTaskDelta(collectionID int64, targetAct
switch action := action.(type) {
case *SegmentAction:
// skip growing segment's count, cause doesn't know realtime row number of growing segment
if action.Scope() == querypb.DataScope_Historical {
segment := scheduler.targetMgr.GetSealedSegment(collectionID, action.segmentID, meta.NextTargetFirst)
if action.Scope == querypb.DataScope_Historical {
segment := scheduler.targetMgr.GetSealedSegment(collectionID, action.SegmentID, meta.NextTargetFirst)
if segment != nil {
sum += int(segment.GetNumOfRows()) * delta
}
@ -565,6 +571,22 @@ func (scheduler *taskScheduler) GetSegmentTaskNum() int {
return len(scheduler.segmentTasks)
}
// GetTasksJSON returns the JSON string of all tasks.
// the task stats object is thread safe and can be accessed without lock
func (scheduler *taskScheduler) GetTasksJSON() string {
if scheduler.taskStats.Len() == 0 {
return ""
}
tasks := scheduler.taskStats.Values()
ret, err := json.Marshal(tasks)
if err != nil {
log.Warn("marshal tasks fail", zap.Error(err))
return ""
}
return string(ret)
}
// schedule selects some tasks to execute, follow these steps for each started selected tasks:
// 1. check whether this task is stale, set status to canceled if stale
// 2. step up the task's actions, set status to succeeded if all actions finished
@ -685,7 +707,7 @@ func (scheduler *taskScheduler) preProcess(task Task) bool {
// causes a few time to load delta log, if reduce the old delegator in advance,
// new delegator can't service search and query, will got no available channel error
channelAction := actions[step].(*ChannelAction)
leader := scheduler.distMgr.LeaderViewManager.GetLeaderShardView(channelAction.Node(), channelAction.Shard())
leader := scheduler.distMgr.LeaderViewManager.GetLeaderShardView(channelAction.Node(), channelAction.Shard)
ready = leader.UnServiceableError == nil
default:
ready = true

View File

@ -18,10 +18,12 @@ package task
import (
"context"
"encoding/json"
"fmt"
"time"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/trace"
"go.uber.org/atomic"
@ -97,6 +99,11 @@ type Task interface {
SetReason(reason string)
String() string
// MarshalJSON marshal task info to json
MarshalJSON() ([]byte, error)
Name() string
GetReason() string
RecordStartTs()
GetTaskLatency() int64
}
@ -123,6 +130,7 @@ type baseTask struct {
// span for tracing
span trace.Span
name string
// startTs
startTs time.Time
@ -278,6 +286,14 @@ func (task *baseTask) SetReason(reason string) {
task.reason = reason
}
func (task *baseTask) GetReason() string {
return task.reason
}
func (task *baseTask) MarshalJSON() ([]byte, error) {
return marshalJSON(task)
}
func (task *baseTask) String() string {
var actionsStr string
for _, action := range task.actions {
@ -298,6 +314,10 @@ func (task *baseTask) String() string {
)
}
func (task *baseTask) Name() string {
return fmt.Sprintf("%s-%s-%d", task.source.String(), GetTaskType(task).String(), task.id)
}
type SegmentTask struct {
*baseTask
@ -326,10 +346,10 @@ func NewSegmentTask(ctx context.Context,
return nil, errors.WithStack(merr.WrapErrParameterInvalid("SegmentAction", "other action", "all actions must be with the same type"))
}
if segmentID == -1 {
segmentID = action.SegmentID()
shard = action.Shard()
} else if segmentID != action.SegmentID() {
return nil, errors.WithStack(merr.WrapErrParameterInvalid(segmentID, action.SegmentID(), "all actions must operate the same segment"))
segmentID = action.GetSegmentID()
shard = action.GetShard()
} else if segmentID != action.GetSegmentID() {
return nil, errors.WithStack(merr.WrapErrParameterInvalid(segmentID, action.GetSegmentID(), "all actions must operate the same segment"))
}
}
@ -346,13 +366,21 @@ func (task *SegmentTask) SegmentID() typeutil.UniqueID {
}
func (task *SegmentTask) Index() string {
return fmt.Sprintf("%s[segment=%d][growing=%t]", task.baseTask.Index(), task.segmentID, task.Actions()[0].(*SegmentAction).Scope() == querypb.DataScope_Streaming)
return fmt.Sprintf("%s[segment=%d][growing=%t]", task.baseTask.Index(), task.segmentID, task.Actions()[0].(*SegmentAction).GetScope() == querypb.DataScope_Streaming)
}
func (task *SegmentTask) Name() string {
return fmt.Sprintf("%s-SegmentTask[%d]-%d", task.source.String(), task.ID(), task.segmentID)
}
func (task *SegmentTask) String() string {
return fmt.Sprintf("%s [segmentID=%d]", task.baseTask.String(), task.segmentID)
}
func (task *SegmentTask) MarshalJSON() ([]byte, error) {
return marshalJSON(task)
}
type ChannelTask struct {
*baseTask
}
@ -399,15 +427,24 @@ func (task *ChannelTask) Index() string {
return fmt.Sprintf("%s[channel=%s]", task.baseTask.Index(), task.shard)
}
func (task *ChannelTask) Name() string {
return fmt.Sprintf("%s-ChannelTask[%d]-%s", task.source.String(), task.ID(), task.shard)
}
func (task *ChannelTask) String() string {
return fmt.Sprintf("%s [channel=%s]", task.baseTask.String(), task.Channel())
}
func (task *ChannelTask) MarshalJSON() ([]byte, error) {
return marshalJSON(task)
}
type LeaderTask struct {
*baseTask
segmentID typeutil.UniqueID
leaderID int64
innerName string
}
func NewLeaderSegmentTask(ctx context.Context,
@ -418,12 +455,13 @@ func NewLeaderSegmentTask(ctx context.Context,
action *LeaderAction,
) *LeaderTask {
segmentID := action.SegmentID()
base := newBaseTask(ctx, source, collectionID, replica, action.Shard(), fmt.Sprintf("LeaderSegmentTask-%s-%d", action.Type().String(), segmentID))
base := newBaseTask(ctx, source, collectionID, replica, action.Shard, fmt.Sprintf("LeaderSegmentTask-%s-%d", action.Type().String(), segmentID))
base.actions = []Action{action}
return &LeaderTask{
baseTask: base,
segmentID: segmentID,
leaderID: leaderID,
innerName: fmt.Sprintf("%s-LeaderSegmentTask", source.String()),
}
}
@ -434,11 +472,12 @@ func NewLeaderPartStatsTask(ctx context.Context,
leaderID int64,
action *LeaderAction,
) *LeaderTask {
base := newBaseTask(ctx, source, collectionID, replica, action.Shard(), fmt.Sprintf("LeaderPartitionStatsTask-%s", action.Type().String()))
base := newBaseTask(ctx, source, collectionID, replica, action.Shard, fmt.Sprintf("LeaderPartitionStatsTask-%s", action.Type().String()))
base.actions = []Action{action}
return &LeaderTask{
baseTask: base,
leaderID: leaderID,
baseTask: base,
leaderID: leaderID,
innerName: fmt.Sprintf("%s-LeaderPartitionStatsTask", source.String()),
}
}
@ -453,3 +492,37 @@ func (task *LeaderTask) Index() string {
func (task *LeaderTask) String() string {
return fmt.Sprintf("%s [segmentID=%d][leader=%d]", task.baseTask.String(), task.segmentID, task.leaderID)
}
func (task *LeaderTask) Name() string {
return fmt.Sprintf("%s[%d]-%d", task.innerName, task.ID(), task.leaderID)
}
func (task *LeaderTask) MarshalJSON() ([]byte, error) {
return marshalJSON(task)
}
func marshalJSON(task Task) ([]byte, error) {
return json.Marshal(&struct {
TaskName string `json:"task_name,omitempty"`
CollectionID int64 `json:"collection_id,omitempty"`
Replica int64 `json:"replica_id,omitempty"`
TaskType string `json:"task_type,omitempty"`
TaskStatus string `json:"task_status,omitempty"`
Priority string `json:"priority,omitempty"`
Actions []string `json:"actions,omitempty"`
Step int `json:"step,omitempty"`
Reason string `json:"reason,omitempty"`
}{
TaskName: task.Name(),
CollectionID: task.CollectionID(),
Replica: task.ReplicaID(),
TaskType: GetTaskType(task).String(),
TaskStatus: task.Status(),
Priority: task.Priority().String(),
Actions: lo.Map(task.Actions(), func(t Action, i int) string {
return t.Desc()
}),
Step: task.Step(),
Reason: task.GetReason(),
})
}

View File

@ -1807,6 +1807,40 @@ func (suite *TaskSuite) TestBalanceChannelWithL0SegmentTask() {
suite.Equal(2, task.step)
}
func (suite *TaskSuite) TestGetTasksJSON() {
ctx := context.Background()
scheduler := suite.newScheduler()
// Add some tasks to the scheduler
task1, err := NewSegmentTask(
ctx,
10*time.Second,
WrapIDSource(0),
suite.collection,
suite.replica,
NewSegmentAction(1, ActionTypeGrow, "", 1),
)
suite.NoError(err)
err = scheduler.Add(task1)
suite.NoError(err)
task2, err := NewChannelTask(
ctx,
10*time.Second,
WrapIDSource(0),
suite.collection,
suite.replica,
NewChannelAction(1, ActionTypeGrow, "channel-1"),
)
suite.NoError(err)
err = scheduler.Add(task2)
suite.NoError(err)
actualJSON := scheduler.GetTasksJSON()
suite.Contains(actualJSON, "SegmentTask")
suite.Contains(actualJSON, "ChannelTask")
}
func TestTask(t *testing.T) {
suite.Run(t, new(TaskSuite))
}

View File

@ -177,8 +177,8 @@ func packReleaseSegmentRequest(task *SegmentTask, action *SegmentAction) *queryp
NodeID: action.Node(),
CollectionID: task.CollectionID(),
SegmentIDs: []int64{task.SegmentID()},
Scope: action.Scope(),
Shard: action.Shard(),
Scope: action.GetScope(),
Shard: action.GetShard(),
NeedTransfer: false,
}
}

View File

@ -28,7 +28,6 @@ import (
"github.com/milvus-io/milvus/internal/querynodev2/segments"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/hardware"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/ratelimitutil"
@ -172,16 +171,13 @@ func getCollectionMetrics(node *QueryNode) (*metricsinfo.QueryNodeCollectionMetr
}
// getSystemInfoMetrics returns metrics info of QueryNode
func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, node *QueryNode) (*milvuspb.GetMetricsResponse, error) {
func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, node *QueryNode) (string, error) {
usedMem := hardware.GetUsedMemoryCount()
totalMem := hardware.GetMemoryCount()
quotaMetrics, err := getQuotaMetrics(node)
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, node.GetNodeID()),
}, nil
return "", err
}
hardwareInfos := metricsinfo.HardwareMetrics{
IP: node.session.Address,
@ -196,10 +192,7 @@ func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest,
collectionMetrics, err := getCollectionMetrics(node)
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, node.GetNodeID()),
}, nil
return "", err
}
nodeInfos := metricsinfo.QueryNodeInfos{
@ -220,18 +213,5 @@ func getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest,
}
metricsinfo.FillDeployMetricsWithEnv(&nodeInfos.SystemInfo)
resp, err := metricsinfo.MarshalComponentInfos(nodeInfos)
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
Response: "",
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, node.GetNodeID()),
}, nil
}
return &milvuspb.GetMetricsResponse{
Status: merr.Success(),
Response: resp,
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, node.GetNodeID()),
}, nil
return metricsinfo.MarshalComponentInfos(nodeInfos)
}

View File

@ -41,10 +41,12 @@ import (
"unsafe"
"github.com/samber/lo"
"github.com/tidwall/gjson"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
grpcquerynodeclient "github.com/milvus-io/milvus/internal/distributed/querynode/client"
"github.com/milvus-io/milvus/internal/querynodev2/cluster"
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
@ -67,6 +69,7 @@ import (
"github.com/milvus-io/milvus/pkg/util/hardware"
"github.com/milvus-io/milvus/pkg/util/lifetime"
"github.com/milvus-io/milvus/pkg/util/lock"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
@ -134,16 +137,19 @@ type QueryNode struct {
// record the last modify ts of segment/channel distribution
lastModifyLock lock.RWMutex
lastModifyTs int64
metricsRequest *metricsinfo.MetricsRequest
}
// NewQueryNode will return a QueryNode with abnormal state.
func NewQueryNode(ctx context.Context, factory dependency.Factory) *QueryNode {
ctx, cancel := context.WithCancel(ctx)
node := &QueryNode{
ctx: ctx,
cancel: cancel,
factory: factory,
lifetime: lifetime.NewLifetime(commonpb.StateCode_Abnormal),
ctx: ctx,
cancel: cancel,
factory: factory,
lifetime: lifetime.NewLifetime(commonpb.StateCode_Abnormal),
metricsRequest: metricsinfo.NewMetricsRequest(),
}
node.tSafeManager = tsafe.NewTSafeReplica()
@ -272,10 +278,19 @@ func (node *QueryNode) CloseSegcore() {
initcore.CleanGlogManager()
}
func (node *QueryNode) registerMetricsRequest() {
node.metricsRequest.RegisterMetricsRequest(metricsinfo.SystemInfoMetrics,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return getSystemInfoMetrics(ctx, req, node)
})
log.Info("register metrics actions finished")
}
// Init function init historical and streaming module to manage segments
func (node *QueryNode) Init() error {
var initError error
node.initOnce.Do(func() {
node.registerMetricsRequest()
// ctx := context.Background()
log.Info("QueryNode session info", zap.String("metaPath", paramtable.Get().EtcdCfg.MetaRootPath.GetValue()))
err := node.initSession()

View File

@ -25,7 +25,6 @@ import (
"time"
"github.com/samber/lo"
"github.com/tidwall/gjson"
"go.opentelemetry.io/otel/trace"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
@ -1103,48 +1102,19 @@ func (node *QueryNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsR
}
defer node.lifetime.Done()
ret := gjson.Parse(req.GetRequest())
metricType, err := metricsinfo.ParseMetricRequestType(ret)
resp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryNodeRole, paramtable.GetNodeID()),
}
ret, err := node.metricsRequest.ExecuteMetricsRequest(ctx, req)
if err != nil {
log.Warn("QueryNode.GetMetrics failed to parse metric type",
zap.Int64("nodeId", node.GetNodeID()),
zap.String("req", req.Request),
zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
}, nil
resp.Status = merr.Status(err)
return resp, nil
}
if metricType == metricsinfo.SystemInfoMetrics {
queryNodeMetrics, err := getSystemInfoMetrics(ctx, req, node)
if err != nil {
log.Warn("QueryNode.GetMetrics failed",
zap.Int64("nodeId", node.GetNodeID()),
zap.String("req", req.Request),
zap.String("metricType", metricType),
zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
}, nil
}
log.RatedDebug(50, "QueryNode.GetMetrics",
zap.Int64("nodeID", node.GetNodeID()),
zap.String("req", req.Request),
zap.String("metricType", metricType),
zap.Any("queryNodeMetrics", queryNodeMetrics))
return queryNodeMetrics, nil
}
log.Debug("QueryNode.GetMetrics failed, request metric type is not implemented yet",
zap.Int64("nodeID", node.GetNodeID()),
zap.String("req", req.Request),
zap.String("metricType", metricType))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
}, nil
resp.Response = ret
return resp, nil
}
func (node *QueryNode) GetDataDistribution(ctx context.Context, req *querypb.GetDataDistributionRequest) (*querypb.GetDataDistributionResponse, error) {

View File

@ -124,6 +124,7 @@ func (suite *ServiceSuite) SetupTest() {
suite.factory = dependency.NewMockFactory(suite.T())
suite.msgStream = msgstream.NewMockMsgStream(suite.T())
// TODO:: cpp chunk manager not support local chunk manager
paramtable.Get().Save(paramtable.Get().LocalStorageCfg.Path.Key, suite.T().TempDir())
// suite.chunkManagerFactory = storage.NewChunkManagerFactory("local", storage.RootPath("/tmp/milvus-test"))
suite.chunkManagerFactory = storage.NewTestChunkManagerFactory(paramtable.Get(), suite.rootPath)
suite.factory.EXPECT().Init(mock.Anything).Return()
@ -168,9 +169,10 @@ func (suite *ServiceSuite) TearDownTest() {
})
suite.NoError(err)
suite.Equal(commonpb.ErrorCode_Success, resp.ErrorCode)
suite.node.chunkManager.RemoveWithPrefix(ctx, suite.rootPath)
suite.node.chunkManager.RemoveWithPrefix(ctx, paramtable.Get().LocalStorageCfg.Path.GetValue())
suite.node.Stop()
suite.etcdClient.Close()
paramtable.Get().Reset(paramtable.Get().LocalStorageCfg.Path.Key)
}
func (suite *ServiceSuite) TestGetComponentStatesNormal() {
@ -1748,30 +1750,22 @@ func (suite *ServiceSuite) TestShowConfigurations_Failed() {
func (suite *ServiceSuite) TestGetMetric_Normal() {
ctx := context.Background()
metricReq := make(map[string]string)
metricReq[metricsinfo.MetricTypeKey] = metricsinfo.SystemInfoMetrics
mReq, err := json.Marshal(metricReq)
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
suite.NoError(err)
req := &milvuspb.GetMetricsRequest{
Base: &commonpb.MsgBase{
MsgID: rand.Int63(),
TargetID: suite.node.session.ServerID,
},
Request: string(mReq),
}
sd1 := delegator.NewMockShardDelegator(suite.T())
sd1.EXPECT().Collection().Return(100)
sd1.EXPECT().GetDeleteBufferSize().Return(10, 1000)
sd1.EXPECT().Close().Maybe()
suite.node.delegators.Insert("qn_unitest_dml_0_100v0", sd1)
defer suite.node.delegators.GetAndRemove("qn_unitest_dml_0_100v0")
sd2 := delegator.NewMockShardDelegator(suite.T())
sd2.EXPECT().Collection().Return(100)
sd2.EXPECT().GetDeleteBufferSize().Return(10, 1000)
sd2.EXPECT().Close().Maybe()
suite.node.delegators.Insert("qn_unitest_dml_1_100v1", sd2)
defer suite.node.delegators.GetAndRemove("qn_unitest_dml_1_100v1")
resp, err := suite.node.GetMetrics(ctx, req)
err = merr.CheckRPCCall(resp, err)
@ -1808,7 +1802,7 @@ func (suite *ServiceSuite) TestGetMetric_Failed() {
resp, err := suite.node.GetMetrics(ctx, req)
suite.NoError(err)
err = merr.Error(resp.GetStatus())
suite.ErrorIs(err, merr.ErrMetricNotFound)
suite.Contains(err.Error(), metricsinfo.MsgUnimplementedMetric)
// metric parse failed
req.Request = "---"
@ -1929,6 +1923,7 @@ func (suite *ServiceSuite) TestSyncDistribution_Normal() {
return nil
})
suite.node.delegators.Insert(testChannel, mockDelegator)
defer suite.node.delegators.GetAndRemove(testChannel)
status, err = suite.node.SyncDistribution(ctx, req)
suite.NoError(err)

View File

@ -19,18 +19,14 @@ package rootcoord
import (
"context"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/hardware"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
func (c *Core) getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Core) getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (string, error) {
rootCoordTopology := metricsinfo.RootCoordTopology{
Self: metricsinfo.RootCoordInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
@ -61,22 +57,5 @@ func (c *Core) getSystemInfoMetrics(ctx context.Context, req *milvuspb.GetMetric
},
}
metricsinfo.FillDeployMetricsWithEnv(&rootCoordTopology.Self.SystemInfo)
resp, err := metricsinfo.MarshalTopology(rootCoordTopology)
if err != nil {
log.Warn("Failed to marshal system info metrics of root coordinator",
zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
Response: "",
ComponentName: metricsinfo.ConstructComponentName(typeutil.RootCoordRole, c.session.ServerID),
}, nil
}
return &milvuspb.GetMetricsResponse{
Status: merr.Success(),
Response: resp,
ComponentName: metricsinfo.ConstructComponentName(typeutil.RootCoordRole, c.session.ServerID),
}, nil
return metricsinfo.MarshalTopology(rootCoordTopology)
}

View File

@ -375,7 +375,8 @@ func newMockProxy() *mockProxy {
func newTestCore(opts ...Opt) *Core {
c := &Core{
session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: TestRootCoordID}},
metricsRequest: metricsinfo.NewMetricsRequest(),
session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: TestRootCoordID}},
}
executor := newMockStepExecutor()
executor.AddStepsFunc = func(s *stepStack) {
@ -387,6 +388,7 @@ func newTestCore(opts ...Opt) *Core {
for _, opt := range opts {
opt(c)
}
c.registerMetricsRequest()
return c
}

View File

@ -128,6 +128,8 @@ type Core struct {
enableActiveStandBy bool
activateFunc func() error
metricsRequest *metricsinfo.MetricsRequest
}
// --------------------- function --------------------------
@ -141,6 +143,7 @@ func NewCore(c context.Context, factory dependency.Factory) (*Core, error) {
cancel: cancel,
factory: factory,
enableActiveStandBy: Params.RootCoordCfg.EnableActiveStandby.GetAsBool(),
metricsRequest: metricsinfo.NewMetricsRequest(),
}
core.UpdateStateCode(commonpb.StateCode_Abnormal)
@ -486,9 +489,18 @@ func (c *Core) initInternal() error {
return nil
}
func (c *Core) registerMetricsRequest() {
c.metricsRequest.RegisterMetricsRequest(metricsinfo.SystemInfoMetrics,
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
return c.getSystemInfoMetrics(ctx, req)
})
log.Info("register metrics actions finished")
}
// Init initialize routine
func (c *Core) Init() error {
var initError error
c.registerMetricsRequest()
c.factory.Init(Params)
if err := c.initSession(); err != nil {
return err
@ -1705,45 +1717,19 @@ func (c *Core) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest) (
}, nil
}
ret := gjson.Parse(in.GetRequest())
metricType, err := metricsinfo.ParseMetricRequestType(ret)
resp := &milvuspb.GetMetricsResponse{
Status: merr.Success(),
ComponentName: metricsinfo.ConstructComponentName(typeutil.RootCoordRole, paramtable.GetNodeID()),
}
ret, err := c.metricsRequest.ExecuteMetricsRequest(ctx, in)
if err != nil {
log.Warn("ParseMetricType failed", zap.String("role", typeutil.RootCoordRole),
zap.Int64("nodeID", c.session.ServerID), zap.String("req", in.Request), zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
Response: "",
}, nil
resp.Status = merr.Status(err)
return resp, nil
}
if metricType == metricsinfo.SystemInfoMetrics {
metrics, err := c.metricsCacheManager.GetSystemInfoMetrics()
if err != nil {
metrics, err = c.getSystemInfoMetrics(ctx, in)
}
if err != nil {
log.Warn("GetSystemInfoMetrics failed",
zap.String("role", typeutil.RootCoordRole),
zap.String("metricType", metricType),
zap.Error(err))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(err),
Response: "",
}, nil
}
c.metricsCacheManager.UpdateSystemInfoMetrics(metrics)
return metrics, err
}
log.RatedWarn(60, "GetMetrics failed, metric type not implemented", zap.String("role", typeutil.RootCoordRole),
zap.String("metricType", metricType))
return &milvuspb.GetMetricsResponse{
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
Response: "",
}, nil
resp.Response = ret
return resp, nil
}
// CreateAlias create collection alias

View File

@ -1139,9 +1139,9 @@ func TestRootCoord_GetMetrics(t *testing.T) {
ctx := context.Background()
c := newTestCore(withHealthyCode(),
withMetricsCacheManager())
resp, err := c.getSystemInfoMetrics(ctx, req)
ret, err := c.getSystemInfoMetrics(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.NotEmpty(t, ret)
})
}
@ -1992,7 +1992,7 @@ func TestCore_RestoreRBAC(t *testing.T) {
meta := mockrootcoord.NewIMetaTable(t)
c := newTestCore(withHealthyCode(), withMeta(meta))
mockProxyClientManager := proxyutil.NewMockProxyClientManager(t)
mockProxyClientManager.EXPECT().RefreshPolicyInfoCache(mock.Anything, mock.Anything).Return(nil)
mockProxyClientManager.EXPECT().RefreshPolicyInfoCache(mock.Anything, mock.Anything).Return(nil).Maybe()
c.proxyClientManager = mockProxyClientManager
meta.EXPECT().RestoreRBAC(mock.Anything, mock.Anything, mock.Anything).Return(nil)

View File

@ -15,7 +15,7 @@ import "github.com/cockroachdb/errors"
const (
// MsgUnimplementedMetric represents that user requests an unimplemented metric type
MsgUnimplementedMetric = "sorry, but this metric request type is not implemented"
MsgUnimplementedMetric = "metric request type is not implemented"
msgInvalidSystemInfosMetricCache = "system infos metric is invalid"
)

View File

@ -15,6 +15,7 @@ import (
"context"
"encoding/json"
"fmt"
"sync"
"github.com/cockroachdb/errors"
"github.com/tidwall/gjson"
@ -44,19 +45,34 @@ const (
MetricRequestParamsSeparator = ","
// QuerySegmentDist request for segment distribution on the query node
QuerySegmentDist = "query_segment_dist"
QuerySegmentDist = "qc_segment_dist"
// QueryChannelDist request for channel distribution on the query node
QueryChannelDist = "query_channel_dist"
QueryChannelDist = "qc_channel_dist"
// QueryTasks request for get tasks on the querycoord
QueryTasks = "query_tasks"
// QueryCoordAllTasks request for get tasks on the querycoord
QueryCoordAllTasks = "qc_tasks_all"
// QueryReplicas request for get replica on the querycoord
QueryReplicas = "query_replica"
QueryReplicas = "qc_replica"
// QueryResourceGroups request for get resource groups on the querycoord
QueryResourceGroups = "query_resource_group"
QueryResourceGroups = "qc_resource_group"
// DataCoordAllTasks request for get tasks on the datacoord
DataCoordAllTasks = "dc_tasks_all"
// ImportTasks request for get import tasks from the datacoord
ImportTasks = "dc_import_tasks"
// CompactionTasks request for get compaction tasks from the datacoord
CompactionTasks = "dc_compaction_tasks"
// BuildIndexTasks request for get building index tasks from the datacoord
BuildIndexTasks = "dc_build_index_tasks"
// SyncTasks request for get sync tasks from the datanode
SyncTasks = "dn_sync_tasks"
// MetricRequestParamVerboseKey as a request parameter decide to whether return verbose value
MetricRequestParamVerboseKey = "verbose"
@ -64,19 +80,30 @@ const (
type MetricsRequestAction func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error)
var metricsReqType2Action = make(map[string]MetricsRequestAction)
type MetricsRequest struct {
metricsReqType2Action map[string]MetricsRequestAction
lock sync.Mutex
}
func RegisterMetricsRequest(reqType string, action MetricsRequestAction) {
_, ok := metricsReqType2Action[reqType]
func NewMetricsRequest() *MetricsRequest {
return &MetricsRequest{
metricsReqType2Action: make(map[string]MetricsRequestAction),
}
}
func (mr *MetricsRequest) RegisterMetricsRequest(reqType string, action MetricsRequestAction) {
mr.lock.Lock()
defer mr.lock.Unlock()
_, ok := mr.metricsReqType2Action[reqType]
if ok {
log.Info("metrics request type already exists", zap.String("reqType", reqType))
return
}
metricsReqType2Action[reqType] = action
mr.metricsReqType2Action[reqType] = action
}
func ExecuteMetricsRequest(ctx context.Context, req *milvuspb.GetMetricsRequest) (string, error) {
func (mr *MetricsRequest) ExecuteMetricsRequest(ctx context.Context, req *milvuspb.GetMetricsRequest) (string, error) {
jsonReq := gjson.Parse(req.Request)
reqType, err := ParseMetricRequestType(jsonReq)
if err != nil {
@ -84,11 +111,14 @@ func ExecuteMetricsRequest(ctx context.Context, req *milvuspb.GetMetricsRequest)
return "", err
}
action, ok := metricsReqType2Action[reqType]
mr.lock.Lock()
action, ok := mr.metricsReqType2Action[reqType]
if !ok {
mr.lock.Unlock()
log.Warn("unimplemented metric request type", zap.String("req_type", reqType))
return "", errors.New(MsgUnimplementedMetric)
}
mr.lock.Unlock()
actionRet, err := action(ctx, req, jsonReq)
if err != nil {

View File

@ -13,6 +13,9 @@ package metricsinfo
import (
"encoding/json"
"time"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
// ComponentInfos defines the interface of all component infos
@ -160,6 +163,17 @@ type DataNodeConfiguration struct {
FlushInsertBufferSize int64 `json:"flush_insert_buffer_size"`
}
type SyncTask struct {
SegmentID int64 `json:"segment_id,omitempty"`
BatchRows int64 `json:"batch_rows,omitempty"`
SegmentLevel string `json:"segment_level,omitempty"`
TsFrom typeutil.Timestamp `json:"ts_from,omitempty"`
TsTo typeutil.Timestamp `json:"ts_to,omitempty"`
DeltaRowCount int64 `json:"delta_row_count,omitempty"`
FlushSize int64 `json:"flush_size,omitempty"`
RunningTime time.Duration `json:"running_time,omitempty"`
}
// DataNodeInfos implements ComponentInfos
type DataNodeInfos struct {
BaseComponentInfos
@ -195,6 +209,31 @@ type DataCoordInfos struct {
CollectionMetrics *DataCoordCollectionMetrics `json:"collection_metrics"`
}
type ImportTask struct {
JobID int64 `json:"job_id,omitempty"`
TaskID int64 `json:"task_id,omitempty"`
CollectionID int64 `json:"collection_id,omitempty"`
NodeID int64 `json:"node_id,omitempty"`
State string `json:"state,omitempty"`
Reason string `json:"reason,omitempty"`
TaskType string `json:"task_type,omitempty"`
CreatedTime string `json:"created_time,omitempty"`
CompleteTime string `json:"complete_time,omitempty"`
}
type CompactionTask struct {
PlanID int64 `json:"plan_id,omitempty"`
CollectionID int64 `json:"collection_id,omitempty"`
Type string `json:"type,omitempty"`
State string `json:"state,omitempty"`
FailReason string `json:"fail_reason,omitempty"`
StartTime int64 `json:"start_time,omitempty"`
EndTime int64 `json:"end_time,omitempty"`
TotalRows int64 `json:"total_rows,omitempty"`
InputSegments []int64 `json:"input_segments,omitempty"`
ResultSegments []int64 `json:"result_segments,omitempty"`
}
// RootCoordConfiguration records the configuration of RootCoord.
type RootCoordConfiguration struct {
MinSegmentSizeToEnableIndex int64 `json:"min_segment_size_to_enable_index"`