mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
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:
parent
d7b2ffe5aa
commit
9d16b972ea
2
go.mod
2
go.mod
@ -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
|
||||
|
||||
@ -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() {
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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{}},
|
||||
}
|
||||
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()))
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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"),
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()))
|
||||
}
|
||||
|
||||
@ -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: {
|
||||
|
||||
@ -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,
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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,
|
||||
})
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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,
|
||||
})
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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"
|
||||
)
|
||||
|
||||
@ -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>
|
||||
@ -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>
|
||||
@ -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>
|
||||
@ -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>
|
||||
@ -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-->
|
||||
@ -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>
|
||||
@ -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>
|
||||
@ -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>
|
||||
54
internal/http/webui/static/js/common.js
Normal file
54
internal/http/webui/static/js/common.js
Normal 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;
|
||||
}
|
||||
572
internal/http/webui/static/js/mockdata.js
Normal file
572
internal/http/webui/static/js/mockdata.js
Normal 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"
|
||||
}
|
||||
]
|
||||
`
|
||||
@ -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
|
||||
});
|
||||
@ -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>
|
||||
@ -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>
|
||||
@ -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 {
|
||||
|
||||
@ -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()))
|
||||
}
|
||||
}
|
||||
|
||||
@ -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")
|
||||
})
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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")
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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(),
|
||||
})
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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"
|
||||
)
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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"`
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user