mirror of https://github.com/milvus-io/milvus.git
enhance: Update stats task to optional (#35947)
issue: #33744 --------- Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>pull/36149/head
parent
e8840a1b41
commit
8395c8a8db
|
@ -317,8 +317,7 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
|
|||
!segment.isCompacting && // not compacting now
|
||||
!segment.GetIsImporting() && // not importing now
|
||||
segment.GetLevel() != datapb.SegmentLevel_L0 && // ignore level zero segments
|
||||
segment.GetLevel() != datapb.SegmentLevel_L2 && // ignore l2 segment
|
||||
segment.GetIsSorted() // segment is sorted
|
||||
segment.GetLevel() != datapb.SegmentLevel_L2 // ignore l2 segment
|
||||
}) // partSegments is list of chanPartSegments, which is channel-partition organized segments
|
||||
|
||||
if len(partSegments) == 0 {
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datacoord
|
||||
|
||||
import "sync"
|
||||
|
||||
var buildIndexCh chan UniqueID
|
||||
var statsTaskCh chan UniqueID
|
||||
var buildIndexChOnce sync.Once
|
||||
var statsTaskChOnce sync.Once
|
||||
|
||||
func getBuildIndexChSingleton() chan UniqueID {
|
||||
buildIndexChOnce.Do(func() {
|
||||
buildIndexCh = make(chan UniqueID, 1024)
|
||||
})
|
||||
|
||||
return buildIndexCh
|
||||
}
|
||||
|
||||
func getStatsTaskChSingleton() chan UniqueID {
|
||||
statsTaskChOnce.Do(func() {
|
||||
statsTaskCh = make(chan UniqueID, 1024)
|
||||
})
|
||||
return statsTaskCh
|
||||
}
|
|
@ -51,8 +51,6 @@ type importScheduler struct {
|
|||
alloc allocator.Allocator
|
||||
imeta ImportMeta
|
||||
|
||||
buildIndexCh chan UniqueID
|
||||
|
||||
closeOnce sync.Once
|
||||
closeChan chan struct{}
|
||||
}
|
||||
|
@ -61,15 +59,13 @@ func NewImportScheduler(meta *meta,
|
|||
cluster Cluster,
|
||||
alloc allocator.Allocator,
|
||||
imeta ImportMeta,
|
||||
buildIndexCh chan UniqueID,
|
||||
) ImportScheduler {
|
||||
return &importScheduler{
|
||||
meta: meta,
|
||||
cluster: cluster,
|
||||
alloc: alloc,
|
||||
imeta: imeta,
|
||||
buildIndexCh: buildIndexCh,
|
||||
closeChan: make(chan struct{}),
|
||||
meta: meta,
|
||||
cluster: cluster,
|
||||
alloc: alloc,
|
||||
imeta: imeta,
|
||||
closeChan: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -320,7 +316,7 @@ func (s *importScheduler) processInProgressImport(task ImportTask) {
|
|||
return
|
||||
}
|
||||
select {
|
||||
case s.buildIndexCh <- info.GetSegmentID(): // accelerate index building:
|
||||
case getBuildIndexChSingleton() <- info.GetSegmentID(): // accelerate index building:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,8 +74,8 @@ func (s *ImportSchedulerSuite) SetupTest() {
|
|||
})
|
||||
s.imeta, err = NewImportMeta(s.catalog)
|
||||
s.NoError(err)
|
||||
buildIndexCh := make(chan UniqueID, 1024)
|
||||
s.scheduler = NewImportScheduler(s.meta, s.cluster, s.alloc, s.imeta, buildIndexCh).(*importScheduler)
|
||||
|
||||
s.scheduler = NewImportScheduler(s.meta, s.cluster, s.alloc, s.imeta).(*importScheduler)
|
||||
}
|
||||
|
||||
func (s *ImportSchedulerSuite) TestProcessPreImport() {
|
||||
|
|
|
@ -53,7 +53,7 @@ func (s *Server) startIndexService(ctx context.Context) {
|
|||
}
|
||||
|
||||
func (s *Server) createIndexForSegment(segment *SegmentInfo, indexID UniqueID) error {
|
||||
if !segment.GetIsSorted() && !segment.GetIsImporting() && segment.Level != datapb.SegmentLevel_L0 {
|
||||
if !segment.GetIsSorted() && Params.DataCoordCfg.EnableStatsTask.GetAsBool() && !segment.GetIsImporting() && segment.Level != datapb.SegmentLevel_L0 {
|
||||
log.Info("segment not sorted, skip create index", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
|
@ -80,10 +80,15 @@ func (s *Server) createIndexForSegment(segment *SegmentInfo, indexID UniqueID) e
|
|||
}
|
||||
|
||||
func (s *Server) createIndexesForSegment(segment *SegmentInfo) error {
|
||||
if !segment.GetIsSorted() && !segment.GetIsImporting() && segment.GetLevel() != datapb.SegmentLevel_L0 {
|
||||
log.Debug("segment is not sorted by pk, skip create index", zap.Int64("segmentID", segment.ID))
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() && !segment.GetIsSorted() && !segment.GetIsImporting() {
|
||||
log.Debug("segment is not sorted by pk, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
if segment.GetLevel() == datapb.SegmentLevel_L0 {
|
||||
log.Debug("segment is level zero, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
|
||||
indexes := s.meta.indexMeta.GetIndexesForCollection(segment.CollectionID, "")
|
||||
indexIDToSegIndexes := s.meta.indexMeta.GetSegmentIndexes(segment.CollectionID, segment.ID)
|
||||
for _, index := range indexes {
|
||||
|
@ -134,7 +139,7 @@ func (s *Server) createIndexForSegmentLoop(ctx context.Context) {
|
|||
case collectionID := <-s.notifyIndexChan:
|
||||
log.Info("receive create index notify", zap.Int64("collectionID", collectionID))
|
||||
segments := s.meta.SelectSegments(WithCollection(collectionID), SegmentFilterFunc(func(info *SegmentInfo) bool {
|
||||
return isFlush(info) && info.GetIsSorted()
|
||||
return isFlush(info) && (!Params.DataCoordCfg.EnableStatsTask.GetAsBool() || info.GetIsSorted())
|
||||
}))
|
||||
for _, segment := range segments {
|
||||
if err := s.createIndexesForSegment(segment); err != nil {
|
||||
|
@ -142,7 +147,7 @@ func (s *Server) createIndexForSegmentLoop(ctx context.Context) {
|
|||
continue
|
||||
}
|
||||
}
|
||||
case segID := <-s.buildIndexCh:
|
||||
case segID := <-getBuildIndexChSingleton():
|
||||
log.Info("receive new flushed segment", zap.Int64("segmentID", segID))
|
||||
segment := s.meta.GetSegment(segID)
|
||||
if segment == nil {
|
||||
|
|
|
@ -0,0 +1,302 @@
|
|||
package datacoord
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datacoord/allocator"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
type StatsTaskManager interface {
|
||||
Start()
|
||||
Stop()
|
||||
SubmitStatsTask(originSegmentID, targetSegmentID int64, subJobType indexpb.StatsSubJob, canRecycle bool) error
|
||||
GetStatsTaskState(originSegmentID int64, subJobType indexpb.StatsSubJob) indexpb.JobState
|
||||
DropStatsTask(originSegmentID int64, subJobType indexpb.StatsSubJob) error
|
||||
}
|
||||
|
||||
var _ StatsTaskManager = (*statsJobManager)(nil)
|
||||
|
||||
type statsJobManager struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
loopWg sync.WaitGroup
|
||||
|
||||
mt *meta
|
||||
|
||||
scheduler *taskScheduler
|
||||
allocator allocator.Allocator
|
||||
}
|
||||
|
||||
func newJobManager(ctx context.Context,
|
||||
mt *meta,
|
||||
scheduler *taskScheduler,
|
||||
allocator allocator.Allocator) *statsJobManager {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
return &statsJobManager{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
loopWg: sync.WaitGroup{},
|
||||
mt: mt,
|
||||
scheduler: scheduler,
|
||||
allocator: allocator,
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) Start() {
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
jm.loopWg.Add(2)
|
||||
go jm.triggerStatsTaskLoop()
|
||||
go jm.cleanupStatsTasksLoop()
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) Stop() {
|
||||
jm.cancel()
|
||||
jm.loopWg.Wait()
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) triggerStatsTaskLoop() {
|
||||
log.Info("start checkStatsTaskLoop...")
|
||||
defer jm.loopWg.Done()
|
||||
|
||||
ticker := time.NewTicker(Params.DataCoordCfg.TaskCheckInterval.GetAsDuration(time.Second))
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-jm.ctx.Done():
|
||||
log.Warn("DataCoord context done, exit checkStatsTaskLoop...")
|
||||
return
|
||||
case <-ticker.C:
|
||||
jm.triggerSortStatsTask()
|
||||
jm.triggerTextStatsTask()
|
||||
jm.triggerBM25StatsTask()
|
||||
|
||||
case segID := <-getStatsTaskChSingleton():
|
||||
log.Info("receive new segment to trigger stats task", zap.Int64("segmentID", segID))
|
||||
segment := jm.mt.GetSegment(segID)
|
||||
if segment == nil {
|
||||
log.Warn("segment is not exist, no need to do stats task", zap.Int64("segmentID", segID))
|
||||
continue
|
||||
}
|
||||
// TODO @xiaocai2333 @bigsheeper: remove code after allow create stats task for importing segment
|
||||
if segment.GetIsImporting() {
|
||||
log.Info("segment is importing, skip stats task", zap.Int64("segmentID", segID))
|
||||
select {
|
||||
case getBuildIndexChSingleton() <- segID:
|
||||
default:
|
||||
}
|
||||
continue
|
||||
}
|
||||
jm.createSortStatsTaskForSegment(segment)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) triggerSortStatsTask() {
|
||||
segments := jm.mt.SelectSegments(SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return isFlush(seg) && seg.GetLevel() != datapb.SegmentLevel_L0 && !seg.GetIsSorted()
|
||||
}))
|
||||
for _, segment := range segments {
|
||||
if !segment.GetIsSorted() {
|
||||
// TODO @xiaocai2333, @bigsheeper:
|
||||
if segment.GetIsImporting() {
|
||||
log.Warn("segment is importing, skip stats task, wait @bigsheeper support it")
|
||||
continue
|
||||
}
|
||||
jm.createSortStatsTaskForSegment(segment)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) createSortStatsTaskForSegment(segment *SegmentInfo) {
|
||||
targetSegmentID, err := jm.allocator.AllocID(jm.ctx)
|
||||
if err != nil {
|
||||
log.Warn("allocID for segment stats task failed",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
return
|
||||
}
|
||||
if err := jm.SubmitStatsTask(segment.GetID(), targetSegmentID, indexpb.StatsSubJob_Sort, true); err != nil {
|
||||
log.Warn("create stats task with sort for segment failed, wait for retry",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) enableBM25() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func needDoTextIndex(segment *SegmentInfo, fieldIDs []UniqueID) bool {
|
||||
if !(isFlush(segment) && segment.GetLevel() != datapb.SegmentLevel_L0 &&
|
||||
segment.GetIsSorted()) {
|
||||
return false
|
||||
}
|
||||
|
||||
for _, fieldID := range fieldIDs {
|
||||
if segment.GetTextStatsLogs() == nil {
|
||||
return true
|
||||
}
|
||||
if segment.GetTextStatsLogs()[fieldID] == nil {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func needDoBM25(segment *SegmentInfo, fieldIDs []UniqueID) bool {
|
||||
// TODO: docking bm25 stats task
|
||||
return false
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) triggerTextStatsTask() {
|
||||
collections := jm.mt.GetCollections()
|
||||
for _, collection := range collections {
|
||||
needTriggerFieldIDs := make([]UniqueID, 0)
|
||||
for _, field := range collection.Schema.GetFields() {
|
||||
// TODO @longjiquan: please replace it to fieldSchemaHelper.EnableMath
|
||||
h := typeutil.CreateFieldSchemaHelper(field)
|
||||
if !h.EnableMatch() {
|
||||
continue
|
||||
}
|
||||
needTriggerFieldIDs = append(needTriggerFieldIDs, field.GetFieldID())
|
||||
}
|
||||
segments := jm.mt.SelectSegments(WithCollection(collection.ID), SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return needDoTextIndex(seg, needTriggerFieldIDs)
|
||||
}))
|
||||
|
||||
for _, segment := range segments {
|
||||
if err := jm.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_TextIndexJob, true); err != nil {
|
||||
log.Warn("create stats task with text index for segment failed, wait for retry",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) triggerBM25StatsTask() {
|
||||
collections := jm.mt.GetCollections()
|
||||
for _, collection := range collections {
|
||||
needTriggerFieldIDs := make([]UniqueID, 0)
|
||||
for _, field := range collection.Schema.GetFields() {
|
||||
// TODO: docking bm25 stats task
|
||||
if jm.enableBM25() {
|
||||
needTriggerFieldIDs = append(needTriggerFieldIDs, field.GetFieldID())
|
||||
}
|
||||
}
|
||||
segments := jm.mt.SelectSegments(WithCollection(collection.ID), SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return needDoBM25(seg, needTriggerFieldIDs)
|
||||
}))
|
||||
|
||||
for _, segment := range segments {
|
||||
if err := jm.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_BM25Job, true); err != nil {
|
||||
log.Warn("create stats task with bm25 for segment failed, wait for retry",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// cleanupStatsTasks clean up the finished/failed stats tasks
|
||||
func (jm *statsJobManager) cleanupStatsTasksLoop() {
|
||||
log.Info("start cleanupStatsTasksLoop...")
|
||||
defer jm.loopWg.Done()
|
||||
|
||||
ticker := time.NewTicker(Params.DataCoordCfg.GCInterval.GetAsDuration(time.Second))
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-jm.ctx.Done():
|
||||
log.Warn("DataCoord context done, exit cleanupStatsTasksLoop...")
|
||||
return
|
||||
case <-ticker.C:
|
||||
start := time.Now()
|
||||
log.Info("start cleanupUnusedStatsTasks...", zap.Time("startAt", start))
|
||||
|
||||
taskIDs := jm.mt.statsTaskMeta.CanCleanedTasks()
|
||||
for _, taskID := range taskIDs {
|
||||
// waiting for queue processing tasks to complete
|
||||
if jm.scheduler.getTask(taskID) == nil {
|
||||
if err := jm.mt.statsTaskMeta.DropStatsTask(taskID); err != nil {
|
||||
// ignore err, if remove failed, wait next GC
|
||||
log.Warn("clean up stats task failed", zap.Int64("taskID", taskID), zap.Error(err))
|
||||
}
|
||||
}
|
||||
}
|
||||
log.Info("cleanupUnusedStatsTasks done", zap.Duration("timeCost", time.Since(start)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) SubmitStatsTask(originSegmentID, targetSegmentID int64,
|
||||
subJobType indexpb.StatsSubJob, canRecycle bool) error {
|
||||
originSegment := jm.mt.GetHealthySegment(originSegmentID)
|
||||
if originSegment == nil {
|
||||
return merr.WrapErrSegmentNotFound(originSegmentID)
|
||||
}
|
||||
taskID, err := jm.allocator.AllocID(context.Background())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t := &indexpb.StatsTask{
|
||||
CollectionID: originSegment.GetCollectionID(),
|
||||
PartitionID: originSegment.GetPartitionID(),
|
||||
SegmentID: originSegmentID,
|
||||
InsertChannel: originSegment.GetInsertChannel(),
|
||||
TaskID: taskID,
|
||||
Version: 0,
|
||||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
FailReason: "",
|
||||
TargetSegmentID: targetSegmentID,
|
||||
SubJobType: subJobType,
|
||||
CanRecycle: canRecycle,
|
||||
}
|
||||
if err = jm.mt.statsTaskMeta.AddStatsTask(t); err != nil {
|
||||
if errors.Is(err, merr.ErrTaskDuplicate) {
|
||||
log.Info("stats task already exists", zap.Int64("taskID", taskID),
|
||||
zap.Int64("collectionID", originSegment.GetCollectionID()),
|
||||
zap.Int64("segmentID", originSegment.GetID()))
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
jm.scheduler.enqueue(newStatsTask(t.GetTaskID(), t.GetSegmentID(), t.GetTargetSegmentID(), subJobType))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) GetStatsTaskState(originSegmentID int64, subJobType indexpb.StatsSubJob) indexpb.JobState {
|
||||
state := jm.mt.statsTaskMeta.GetStatsTaskStateBySegmentID(originSegmentID, subJobType)
|
||||
log.Info("statsJobManager get stats task state", zap.Int64("segmentID", originSegmentID),
|
||||
zap.String("subJobType", subJobType.String()), zap.String("state", state.String()))
|
||||
return state
|
||||
}
|
||||
|
||||
func (jm *statsJobManager) DropStatsTask(originSegmentID int64, subJobType indexpb.StatsSubJob) error {
|
||||
task := jm.mt.statsTaskMeta.GetStatsTaskBySegmentID(originSegmentID, subJobType)
|
||||
if task == nil {
|
||||
return nil
|
||||
}
|
||||
jm.scheduler.AbortTask(task.GetTaskID())
|
||||
if err := jm.mt.statsTaskMeta.MarkTaskCanRecycle(task.GetTaskID()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
log.Info("statsJobManager drop stats task success", zap.Int64("segmentID", originSegmentID),
|
||||
zap.Int64("taskID", task.GetTaskID()), zap.String("subJobType", subJobType.String()))
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,116 @@
|
|||
package datacoord
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"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/datacoord/allocator"
|
||||
"github.com/milvus-io/milvus/internal/metastore/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
)
|
||||
|
||||
type jobManagerSuite struct {
|
||||
suite.Suite
|
||||
}
|
||||
|
||||
func Test_jobManagerSuite(t *testing.T) {
|
||||
suite.Run(t, new(jobManagerSuite))
|
||||
}
|
||||
|
||||
func (s *jobManagerSuite) TestJobManager_triggerStatsTaskLoop() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
var start int64
|
||||
alloc := allocator.NewMockAllocator(s.T())
|
||||
alloc.EXPECT().AllocID(mock.Anything).RunAndReturn(func(ctx context.Context) (int64, error) {
|
||||
start++
|
||||
return start, nil
|
||||
})
|
||||
Params.Save(Params.DataCoordCfg.TaskCheckInterval.Key, "1")
|
||||
|
||||
catalog := mocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
mt := &meta{
|
||||
collections: map[UniqueID]*collectionInfo{
|
||||
1: {
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 100,
|
||||
Name: "pk",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "var",
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "enable_match", Value: "true",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
segments: &SegmentsInfo{
|
||||
segments: map[UniqueID]*SegmentInfo{
|
||||
10: {
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 10,
|
||||
CollectionID: 1,
|
||||
PartitionID: 2,
|
||||
IsSorted: false,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
},
|
||||
},
|
||||
20: {
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 20,
|
||||
CollectionID: 1,
|
||||
PartitionID: 2,
|
||||
IsSorted: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
statsTaskMeta: &statsTaskMeta{
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
tasks: make(map[int64]*indexpb.StatsTask),
|
||||
},
|
||||
}
|
||||
|
||||
jm := &statsJobManager{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
loopWg: sync.WaitGroup{},
|
||||
mt: mt,
|
||||
scheduler: &taskScheduler{
|
||||
allocator: alloc,
|
||||
tasks: make(map[int64]Task),
|
||||
meta: mt,
|
||||
},
|
||||
allocator: alloc,
|
||||
}
|
||||
|
||||
jm.loopWg.Add(1)
|
||||
go jm.triggerStatsTaskLoop()
|
||||
|
||||
time.Sleep(2 * time.Second)
|
||||
cancel()
|
||||
|
||||
jm.loopWg.Wait()
|
||||
|
||||
s.Equal(2, len(jm.scheduler.tasks))
|
||||
}
|
|
@ -72,7 +72,6 @@ type CompactionMeta interface {
|
|||
GetAnalyzeMeta() *analyzeMeta
|
||||
GetPartitionStatsMeta() *partitionStatsMeta
|
||||
GetCompactionTaskMeta() *compactionTaskMeta
|
||||
GetStatsTaskMeta() *statsTaskMeta
|
||||
}
|
||||
|
||||
var _ CompactionMeta = (*meta)(nil)
|
||||
|
@ -109,10 +108,6 @@ func (m *meta) GetCompactionTaskMeta() *compactionTaskMeta {
|
|||
return m.compactionTaskMeta
|
||||
}
|
||||
|
||||
func (m *meta) GetStatsTaskMeta() *statsTaskMeta {
|
||||
return m.statsTaskMeta
|
||||
}
|
||||
|
||||
type channelCPs struct {
|
||||
lock.RWMutex
|
||||
checkpoints map[string]*msgpb.MsgPosition
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
|
||||
package datacoord
|
||||
|
||||
import "github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
|
||||
// SegmentOperator is function type to update segment info.
|
||||
type SegmentOperator func(segment *SegmentInfo) bool
|
||||
|
||||
|
@ -29,6 +31,18 @@ func SetMaxRowCount(maxRow int64) SegmentOperator {
|
|||
}
|
||||
}
|
||||
|
||||
func SetTextIndexLogs(textIndexLogs map[int64]*datapb.TextIndexStats) SegmentOperator {
|
||||
return func(segment *SegmentInfo) bool {
|
||||
if segment.TextStatsLogs == nil {
|
||||
segment.TextStatsLogs = make(map[int64]*datapb.TextIndexStats)
|
||||
}
|
||||
for field, logs := range textIndexLogs {
|
||||
segment.TextStatsLogs[field] = logs
|
||||
}
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
type segmentCriterion struct {
|
||||
collectionID int64
|
||||
channel string
|
||||
|
|
|
@ -129,8 +129,6 @@ type Server struct {
|
|||
metricsCacheManager *metricsinfo.MetricsCacheManager
|
||||
|
||||
flushCh chan UniqueID
|
||||
statsCh chan UniqueID
|
||||
buildIndexCh chan UniqueID
|
||||
notifyIndexChan chan UniqueID
|
||||
factory dependency.Factory
|
||||
|
||||
|
@ -154,6 +152,7 @@ type Server struct {
|
|||
indexEngineVersionManager IndexEngineVersionManager
|
||||
|
||||
taskScheduler *taskScheduler
|
||||
jobManager *statsJobManager
|
||||
|
||||
// manage ways that data coord access other coord
|
||||
broker broker.Broker
|
||||
|
@ -206,8 +205,6 @@ func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Optio
|
|||
quitCh: make(chan struct{}),
|
||||
factory: factory,
|
||||
flushCh: make(chan UniqueID, 1024),
|
||||
statsCh: make(chan UniqueID, 1024),
|
||||
buildIndexCh: make(chan UniqueID, 1024),
|
||||
notifyIndexChan: make(chan UniqueID),
|
||||
dataNodeCreator: defaultDataNodeCreatorFunc,
|
||||
indexNodeCreator: defaultIndexNodeCreatorFunc,
|
||||
|
@ -381,6 +378,9 @@ func (s *Server) initDataCoord() error {
|
|||
s.initTaskScheduler(storageCli)
|
||||
log.Info("init task scheduler done")
|
||||
|
||||
s.initJobManager()
|
||||
log.Info("init statsJobManager done")
|
||||
|
||||
s.initCompaction()
|
||||
log.Info("init compaction done")
|
||||
|
||||
|
@ -395,7 +395,7 @@ func (s *Server) initDataCoord() error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
s.importScheduler = NewImportScheduler(s.meta, s.cluster, s.allocator, s.importMeta, s.statsCh)
|
||||
s.importScheduler = NewImportScheduler(s.meta, s.cluster, s.allocator, s.importMeta)
|
||||
s.importChecker = NewImportChecker(s.meta, s.broker, s.cluster, s.allocator, s.segmentManager, s.importMeta)
|
||||
|
||||
s.syncSegmentsScheduler = newSyncSegmentsScheduler(s.meta, s.channelManager, s.sessionManager)
|
||||
|
@ -675,6 +675,12 @@ func (s *Server) initTaskScheduler(manager storage.ChunkManager) {
|
|||
}
|
||||
}
|
||||
|
||||
func (s *Server) initJobManager() {
|
||||
if s.jobManager == nil {
|
||||
s.jobManager = newJobManager(s.ctx, s.meta, s.taskScheduler, s.allocator)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Server) initIndexNodeManager() {
|
||||
if s.indexNodeManager == nil {
|
||||
s.indexNodeManager = session.NewNodeManager(s.ctx, s.indexNodeCreator)
|
||||
|
@ -733,9 +739,9 @@ func (s *Server) startServerLoop() {
|
|||
|
||||
func (s *Server) startTaskScheduler() {
|
||||
s.taskScheduler.Start()
|
||||
s.jobManager.Start()
|
||||
|
||||
s.startIndexService(s.serverLoopCtx)
|
||||
s.startStatsTasksCheckLoop(s.serverLoopCtx)
|
||||
}
|
||||
|
||||
func (s *Server) updateSegmentStatistics(stats []*commonpb.SegmentStats) {
|
||||
|
@ -989,7 +995,7 @@ func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
|
|||
return err
|
||||
}
|
||||
select {
|
||||
case s.statsCh <- segmentID:
|
||||
case getStatsTaskChSingleton() <- segmentID:
|
||||
default:
|
||||
}
|
||||
|
||||
|
@ -1065,6 +1071,9 @@ func (s *Server) Stop() error {
|
|||
s.stopCompaction()
|
||||
logutil.Logger(s.ctx).Info("datacoord compaction stopped")
|
||||
|
||||
s.jobManager.Stop()
|
||||
logutil.Logger(s.ctx).Info("datacoord statsJobManager stopped")
|
||||
|
||||
s.taskScheduler.Stop()
|
||||
logutil.Logger(s.ctx).Info("datacoord index builder stopped")
|
||||
|
||||
|
|
|
@ -41,17 +41,14 @@ type statsTaskMeta struct {
|
|||
catalog metastore.DataCoordCatalog
|
||||
|
||||
// taskID -> analyzeStats
|
||||
// TODO: when to mark as dropped?
|
||||
tasks map[int64]*indexpb.StatsTask
|
||||
segmentStatsTaskIndex map[int64]*indexpb.StatsTask
|
||||
tasks map[int64]*indexpb.StatsTask
|
||||
}
|
||||
|
||||
func newStatsTaskMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (*statsTaskMeta, error) {
|
||||
stm := &statsTaskMeta{
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
tasks: make(map[int64]*indexpb.StatsTask),
|
||||
segmentStatsTaskIndex: make(map[int64]*indexpb.StatsTask),
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
tasks: make(map[int64]*indexpb.StatsTask),
|
||||
}
|
||||
if err := stm.reloadFromKV(); err != nil {
|
||||
return nil, err
|
||||
|
@ -69,7 +66,6 @@ func (stm *statsTaskMeta) reloadFromKV() error {
|
|||
}
|
||||
for _, t := range statsTasks {
|
||||
stm.tasks[t.GetTaskID()] = t
|
||||
stm.tasks[t.GetSegmentID()] = t
|
||||
}
|
||||
|
||||
log.Info("statsTaskMeta reloadFromKV done", zap.Duration("duration", record.ElapseSpan()))
|
||||
|
@ -103,36 +99,41 @@ func (stm *statsTaskMeta) AddStatsTask(t *indexpb.StatsTask) error {
|
|||
stm.Lock()
|
||||
defer stm.Unlock()
|
||||
|
||||
if _, ok := stm.segmentStatsTaskIndex[t.GetSegmentID()]; ok {
|
||||
msg := fmt.Sprintf("stats task already exist in meta of segment %d", t.GetSegmentID())
|
||||
log.Warn(msg)
|
||||
return merr.WrapErrTaskDuplicate(indexpb.JobType_JobTypeStatsJob.String(), msg)
|
||||
for _, st := range stm.tasks {
|
||||
if st.GetSegmentID() == t.GetSegmentID() && st.GetSubJobType() == t.GetSubJobType() {
|
||||
msg := fmt.Sprintf("stats task already exist in meta of segment %d with subJobType: %s",
|
||||
t.GetSegmentID(), t.GetSubJobType().String())
|
||||
log.Warn(msg)
|
||||
return merr.WrapErrTaskDuplicate(indexpb.JobType_JobTypeStatsJob.String(), msg)
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("add stats task", zap.Int64("taskID", t.GetTaskID()), zap.Int64("segmentID", t.GetSegmentID()))
|
||||
log.Info("add stats task", zap.Int64("taskID", t.GetTaskID()),
|
||||
zap.Int64("segmentID", t.GetSegmentID()), zap.String("subJobType", t.GetSubJobType().String()))
|
||||
t.State = indexpb.JobState_JobStateInit
|
||||
|
||||
if err := stm.catalog.SaveStatsTask(stm.ctx, t); err != nil {
|
||||
log.Warn("adding stats task failed",
|
||||
zap.Int64("taskID", t.GetTaskID()),
|
||||
zap.Int64("segmentID", t.GetSegmentID()),
|
||||
zap.String("subJobType", t.GetSubJobType().String()),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
stm.tasks[t.GetTaskID()] = t
|
||||
stm.segmentStatsTaskIndex[t.GetSegmentID()] = t
|
||||
stm.updateMetrics()
|
||||
|
||||
log.Info("add stats task success", zap.Int64("taskID", t.GetTaskID()), zap.Int64("segmentID", t.GetSegmentID()))
|
||||
log.Info("add stats task success", zap.Int64("taskID", t.GetTaskID()),
|
||||
zap.Int64("segmentID", t.GetSegmentID()), zap.String("subJobType", t.GetSubJobType().String()))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) RemoveStatsTaskByTaskID(taskID int64) error {
|
||||
func (stm *statsTaskMeta) DropStatsTask(taskID int64) error {
|
||||
stm.Lock()
|
||||
defer stm.Unlock()
|
||||
|
||||
log.Info("remove stats task by taskID", zap.Int64("taskID", taskID))
|
||||
log.Info("drop stats task by taskID", zap.Int64("taskID", taskID))
|
||||
|
||||
t, ok := stm.tasks[taskID]
|
||||
if !ok {
|
||||
|
@ -140,7 +141,7 @@ func (stm *statsTaskMeta) RemoveStatsTaskByTaskID(taskID int64) error {
|
|||
return nil
|
||||
}
|
||||
if err := stm.catalog.DropStatsTask(stm.ctx, taskID); err != nil {
|
||||
log.Warn("meta update: removing stats task failed",
|
||||
log.Warn("drop stats task failed",
|
||||
zap.Int64("taskID", taskID),
|
||||
zap.Int64("segmentID", taskID),
|
||||
zap.Error(err))
|
||||
|
@ -148,39 +149,12 @@ func (stm *statsTaskMeta) RemoveStatsTaskByTaskID(taskID int64) error {
|
|||
}
|
||||
|
||||
delete(stm.tasks, taskID)
|
||||
delete(stm.segmentStatsTaskIndex, t.SegmentID)
|
||||
stm.updateMetrics()
|
||||
|
||||
log.Info("remove stats task success", zap.Int64("taskID", taskID), zap.Int64("segmentID", t.SegmentID))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) RemoveStatsTaskBySegmentID(segmentID int64) error {
|
||||
stm.Lock()
|
||||
defer stm.Unlock()
|
||||
|
||||
log.Info("remove stats task by segmentID", zap.Int64("segmentID", segmentID))
|
||||
t, ok := stm.segmentStatsTaskIndex[segmentID]
|
||||
if !ok {
|
||||
log.Info("remove stats task success, task already not exist", zap.Int64("segmentID", segmentID))
|
||||
return nil
|
||||
}
|
||||
if err := stm.catalog.DropStatsTask(stm.ctx, t.TaskID); err != nil {
|
||||
log.Warn("meta update: removing stats task failed",
|
||||
zap.Int64("taskID", t.TaskID),
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
delete(stm.tasks, t.TaskID)
|
||||
delete(stm.segmentStatsTaskIndex, segmentID)
|
||||
stm.updateMetrics()
|
||||
|
||||
log.Info("remove stats task success", zap.Int64("taskID", t.TaskID), zap.Int64("segmentID", segmentID))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) UpdateVersion(taskID int64) error {
|
||||
stm.Lock()
|
||||
defer stm.Unlock()
|
||||
|
@ -202,9 +176,9 @@ func (stm *statsTaskMeta) UpdateVersion(taskID int64) error {
|
|||
}
|
||||
|
||||
stm.tasks[t.TaskID] = cloneT
|
||||
stm.segmentStatsTaskIndex[t.SegmentID] = cloneT
|
||||
stm.updateMetrics()
|
||||
log.Info("update stats task version success", zap.Int64("taskID", taskID), zap.Int64("newVersion", cloneT.GetVersion()))
|
||||
log.Info("update stats task version success", zap.Int64("taskID", taskID),
|
||||
zap.Int64("newVersion", cloneT.GetVersion()))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -230,7 +204,6 @@ func (stm *statsTaskMeta) UpdateBuildingTask(taskID, nodeID int64) error {
|
|||
}
|
||||
|
||||
stm.tasks[t.TaskID] = cloneT
|
||||
stm.segmentStatsTaskIndex[t.SegmentID] = cloneT
|
||||
stm.updateMetrics()
|
||||
|
||||
log.Info("update building stats task success", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
|
||||
|
@ -259,7 +232,6 @@ func (stm *statsTaskMeta) FinishTask(taskID int64, result *workerpb.StatsResult)
|
|||
}
|
||||
|
||||
stm.tasks[t.TaskID] = cloneT
|
||||
stm.segmentStatsTaskIndex[t.SegmentID] = cloneT
|
||||
stm.updateMetrics()
|
||||
|
||||
log.Info("finish stats task meta success", zap.Int64("taskID", taskID), zap.Int64("segmentID", t.SegmentID),
|
||||
|
@ -278,15 +250,17 @@ func (stm *statsTaskMeta) GetStatsTaskState(taskID int64) indexpb.JobState {
|
|||
return t.GetState()
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) GetStatsTaskStateBySegmentID(segmentID int64) indexpb.JobState {
|
||||
func (stm *statsTaskMeta) GetStatsTaskStateBySegmentID(segmentID int64, jobType indexpb.StatsSubJob) indexpb.JobState {
|
||||
stm.RLock()
|
||||
defer stm.RUnlock()
|
||||
|
||||
t, ok := stm.segmentStatsTaskIndex[segmentID]
|
||||
if !ok {
|
||||
return indexpb.JobState_JobStateNone
|
||||
for _, t := range stm.tasks {
|
||||
if segmentID == t.GetSegmentID() && jobType == t.GetSubJobType() {
|
||||
return t.GetState()
|
||||
}
|
||||
}
|
||||
return t.GetState()
|
||||
|
||||
return indexpb.JobState_JobStateNone
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) CanCleanedTasks() []int64 {
|
||||
|
@ -295,10 +269,74 @@ func (stm *statsTaskMeta) CanCleanedTasks() []int64 {
|
|||
|
||||
needCleanedTaskIDs := make([]int64, 0)
|
||||
for taskID, t := range stm.tasks {
|
||||
if t.GetState() == indexpb.JobState_JobStateFinished ||
|
||||
t.GetState() == indexpb.JobState_JobStateFailed {
|
||||
if t.GetCanRecycle() && (t.GetState() == indexpb.JobState_JobStateFinished ||
|
||||
t.GetState() == indexpb.JobState_JobStateFailed) {
|
||||
needCleanedTaskIDs = append(needCleanedTaskIDs, taskID)
|
||||
}
|
||||
}
|
||||
return needCleanedTaskIDs
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) GetAllTasks() map[int64]*indexpb.StatsTask {
|
||||
tasks := make(map[int64]*indexpb.StatsTask)
|
||||
|
||||
stm.RLock()
|
||||
defer stm.RUnlock()
|
||||
for k, v := range stm.tasks {
|
||||
tasks[k] = proto.Clone(v).(*indexpb.StatsTask)
|
||||
}
|
||||
return tasks
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) GetStatsTaskBySegmentID(segmentID int64, subJobType indexpb.StatsSubJob) *indexpb.StatsTask {
|
||||
stm.RLock()
|
||||
defer stm.RUnlock()
|
||||
|
||||
log.Info("get stats task by segmentID", zap.Int64("segmentID", segmentID),
|
||||
zap.String("subJobType", subJobType.String()))
|
||||
|
||||
for taskID, t := range stm.tasks {
|
||||
if t.GetSegmentID() == segmentID && t.GetSubJobType() == subJobType {
|
||||
log.Info("get stats task by segmentID success",
|
||||
zap.Int64("taskID", taskID),
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.String("subJobType", subJobType.String()))
|
||||
return t
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("get stats task by segmentID failed, task not exist", zap.Int64("segmentID", segmentID),
|
||||
zap.String("subJobType", subJobType.String()))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (stm *statsTaskMeta) MarkTaskCanRecycle(taskID int64) error {
|
||||
stm.Lock()
|
||||
defer stm.Unlock()
|
||||
|
||||
log.Info("mark stats task can recycle", zap.Int64("taskID", taskID))
|
||||
|
||||
t, ok := stm.tasks[taskID]
|
||||
if !ok {
|
||||
return fmt.Errorf("task %d not found", taskID)
|
||||
}
|
||||
|
||||
cloneT := proto.Clone(t).(*indexpb.StatsTask)
|
||||
cloneT.CanRecycle = true
|
||||
|
||||
if err := stm.catalog.SaveStatsTask(stm.ctx, cloneT); err != nil {
|
||||
log.Warn("mark stats task can recycle failed",
|
||||
zap.Int64("taskID", taskID),
|
||||
zap.Int64("segmentID", t.GetSegmentID()),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
stm.tasks[t.TaskID] = cloneT
|
||||
stm.updateMetrics()
|
||||
|
||||
log.Info("mark stats task can recycle success", zap.Int64("taskID", taskID),
|
||||
zap.Int64("segmentID", t.SegmentID),
|
||||
zap.String("subJobType", t.GetSubJobType().String()))
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -54,15 +54,18 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
catalog := mocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().ListStatsTasks(mock.Anything).Return([]*indexpb.StatsTask{
|
||||
{
|
||||
CollectionID: s.collectionID,
|
||||
PartitionID: s.partitionID,
|
||||
SegmentID: 10000,
|
||||
InsertChannel: "ch1",
|
||||
TaskID: 10001,
|
||||
Version: 1,
|
||||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateFinished,
|
||||
FailReason: "",
|
||||
CollectionID: s.collectionID,
|
||||
PartitionID: s.partitionID,
|
||||
SegmentID: 10000,
|
||||
InsertChannel: "ch1",
|
||||
TaskID: 10001,
|
||||
Version: 1,
|
||||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateFinished,
|
||||
FailReason: "",
|
||||
TargetSegmentID: 10002,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
CanRecycle: true,
|
||||
},
|
||||
}, nil)
|
||||
|
||||
|
@ -97,6 +100,7 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
FailReason: "",
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
}
|
||||
|
||||
s.Run("AddStatsTask", func() {
|
||||
|
@ -106,9 +110,6 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
s.Error(m.AddStatsTask(t))
|
||||
_, ok := m.tasks[1]
|
||||
s.False(ok)
|
||||
|
||||
_, ok = m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.False(ok)
|
||||
})
|
||||
|
||||
s.Run("normal case", func() {
|
||||
|
@ -117,18 +118,12 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
s.NoError(m.AddStatsTask(t))
|
||||
_, ok := m.tasks[1]
|
||||
s.True(ok)
|
||||
|
||||
_, ok = m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.True(ok)
|
||||
})
|
||||
|
||||
s.Run("already exist", func() {
|
||||
s.Error(m.AddStatsTask(t))
|
||||
_, ok := m.tasks[1]
|
||||
s.True(ok)
|
||||
|
||||
_, ok = m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.True(ok)
|
||||
})
|
||||
})
|
||||
|
||||
|
@ -140,10 +135,6 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
task, ok := m.tasks[1]
|
||||
s.True(ok)
|
||||
s.Equal(int64(1), task.GetVersion())
|
||||
|
||||
sTask, ok := m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.True(ok)
|
||||
s.Equal(int64(1), sTask.GetVersion())
|
||||
})
|
||||
|
||||
s.Run("task not exist", func() {
|
||||
|
@ -161,10 +152,6 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
s.True(ok)
|
||||
// still 1
|
||||
s.Equal(int64(1), task.GetVersion())
|
||||
|
||||
sTask, ok := m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.True(ok)
|
||||
s.Equal(int64(1), sTask.GetVersion())
|
||||
})
|
||||
})
|
||||
|
||||
|
@ -215,7 +202,6 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
StatsLogs: []*datapb.FieldBinlog{
|
||||
{FieldID: 100, Binlogs: []*datapb.Binlog{{LogID: 9}}},
|
||||
},
|
||||
DeltaLogs: nil,
|
||||
TextStatsLogs: map[int64]*datapb.TextIndexStats{
|
||||
100: {
|
||||
FieldID: 100,
|
||||
|
@ -264,42 +250,36 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
|||
|
||||
s.Run("GetStatsTaskStateBySegmentID", func() {
|
||||
s.Run("task not exist", func() {
|
||||
state := m.GetStatsTaskStateBySegmentID(100)
|
||||
state := m.GetStatsTaskStateBySegmentID(100, indexpb.StatsSubJob_Sort)
|
||||
s.Equal(indexpb.JobState_JobStateNone, state)
|
||||
|
||||
state = m.GetStatsTaskStateBySegmentID(s.segmentID, indexpb.StatsSubJob_BM25Job)
|
||||
s.Equal(indexpb.JobState_JobStateNone, state)
|
||||
})
|
||||
|
||||
s.Run("normal case", func() {
|
||||
state := m.GetStatsTaskStateBySegmentID(s.segmentID)
|
||||
state := m.GetStatsTaskStateBySegmentID(s.segmentID, indexpb.StatsSubJob_Sort)
|
||||
s.Equal(indexpb.JobState_JobStateFinished, state)
|
||||
})
|
||||
})
|
||||
|
||||
s.Run("RemoveStatsTask", func() {
|
||||
s.Run("DropStatsTask", func() {
|
||||
s.Run("failed case", func() {
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(fmt.Errorf("mock error")).Twice()
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(fmt.Errorf("mock error")).Once()
|
||||
|
||||
s.Error(m.RemoveStatsTaskByTaskID(1))
|
||||
s.Error(m.DropStatsTask(1))
|
||||
_, ok := m.tasks[1]
|
||||
s.True(ok)
|
||||
|
||||
s.Error(m.RemoveStatsTaskBySegmentID(s.segmentID))
|
||||
_, ok = m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.True(ok)
|
||||
})
|
||||
|
||||
s.Run("normal case", func() {
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(nil).Twice()
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
|
||||
s.NoError(m.RemoveStatsTaskByTaskID(1))
|
||||
s.NoError(m.DropStatsTask(1))
|
||||
_, ok := m.tasks[1]
|
||||
s.False(ok)
|
||||
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
s.NoError(m.AddStatsTask(t))
|
||||
|
||||
s.NoError(m.RemoveStatsTaskBySegmentID(s.segmentID))
|
||||
_, ok = m.segmentStatsTaskIndex[s.segmentID]
|
||||
s.False(ok)
|
||||
s.NoError(m.DropStatsTask(1000))
|
||||
})
|
||||
})
|
||||
}
|
||||
|
|
|
@ -254,6 +254,8 @@ func (at *analyzeTask) setResult(result *workerpb.AnalyzeResult) {
|
|||
}
|
||||
|
||||
func (at *analyzeTask) QueryResult(ctx context.Context, client types.IndexNodeClient) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
|
||||
defer cancel()
|
||||
resp, err := client.QueryJobsV2(ctx, &workerpb.QueryJobsV2Request{
|
||||
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
|
||||
TaskIDs: []int64{at.GetTaskID()},
|
||||
|
@ -292,6 +294,8 @@ func (at *analyzeTask) QueryResult(ctx context.Context, client types.IndexNodeCl
|
|||
}
|
||||
|
||||
func (at *analyzeTask) DropTaskOnWorker(ctx context.Context, client types.IndexNodeClient) bool {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
|
||||
defer cancel()
|
||||
resp, err := client.DropJobsV2(ctx, &workerpb.DropJobsV2Request{
|
||||
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
|
||||
TaskIDs: []UniqueID{at.GetTaskID()},
|
||||
|
|
|
@ -244,7 +244,8 @@ func (it *indexBuildTask) PreCheck(ctx context.Context, dependency *taskSchedule
|
|||
PartitionKeyIsolation: partitionKeyIsolation,
|
||||
}
|
||||
|
||||
log.Ctx(ctx).Info("index task pre check successfully", zap.Int64("taskID", it.GetTaskID()))
|
||||
log.Ctx(ctx).Info("index task pre check successfully", zap.Int64("taskID", it.GetTaskID()),
|
||||
zap.Int64("segID", segment.GetID()))
|
||||
return true
|
||||
}
|
||||
|
||||
|
@ -278,6 +279,8 @@ func (it *indexBuildTask) setResult(info *workerpb.IndexTaskInfo) {
|
|||
}
|
||||
|
||||
func (it *indexBuildTask) QueryResult(ctx context.Context, node types.IndexNodeClient) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
|
||||
defer cancel()
|
||||
resp, err := node.QueryJobsV2(ctx, &workerpb.QueryJobsV2Request{
|
||||
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
|
||||
TaskIDs: []UniqueID{it.GetTaskID()},
|
||||
|
@ -314,6 +317,8 @@ func (it *indexBuildTask) QueryResult(ctx context.Context, node types.IndexNodeC
|
|||
}
|
||||
|
||||
func (it *indexBuildTask) DropTaskOnWorker(ctx context.Context, client types.IndexNodeClient) bool {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
|
||||
defer cancel()
|
||||
resp, err := client.DropJobsV2(ctx, &workerpb.DropJobsV2Request{
|
||||
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
|
||||
TaskIDs: []UniqueID{it.GetTaskID()},
|
||||
|
|
|
@ -89,7 +89,7 @@ func newTaskScheduler(
|
|||
indexEngineVersionManager: indexEngineVersionManager,
|
||||
allocator: allocator,
|
||||
}
|
||||
ts.reloadFromKV()
|
||||
ts.reloadFromMeta()
|
||||
return ts
|
||||
}
|
||||
|
||||
|
@ -104,7 +104,7 @@ func (s *taskScheduler) Stop() {
|
|||
s.wg.Wait()
|
||||
}
|
||||
|
||||
func (s *taskScheduler) reloadFromKV() {
|
||||
func (s *taskScheduler) reloadFromMeta() {
|
||||
segments := s.meta.GetAllSegmentsUnsafe()
|
||||
for _, segment := range segments {
|
||||
for _, segIndex := range s.meta.indexMeta.getSegmentIndexes(segment.ID) {
|
||||
|
@ -145,6 +145,27 @@ func (s *taskScheduler) reloadFromKV() {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
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{
|
||||
taskID: taskID,
|
||||
segmentID: t.GetSegmentID(),
|
||||
targetSegmentID: t.GetTargetSegmentID(),
|
||||
nodeID: t.NodeID,
|
||||
taskInfo: &workerpb.StatsResult{
|
||||
TaskID: taskID,
|
||||
State: t.GetState(),
|
||||
FailReason: t.GetFailReason(),
|
||||
},
|
||||
queueTime: time.Now(),
|
||||
startTime: time.Now(),
|
||||
endTime: time.Now(),
|
||||
subJobType: t.GetSubJobType(),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// notify is an unblocked notify function
|
||||
|
@ -169,6 +190,7 @@ func (s *taskScheduler) enqueue(task Task) {
|
|||
}
|
||||
|
||||
func (s *taskScheduler) AbortTask(taskID int64) {
|
||||
log.Info("task scheduler receive abort task request", zap.Int64("taskID", taskID))
|
||||
s.RLock()
|
||||
task, ok := s.tasks[taskID]
|
||||
s.RUnlock()
|
||||
|
@ -291,13 +313,12 @@ func (s *taskScheduler) collectTaskMetrics() {
|
|||
maxTaskRunningTime := make(map[string]int64)
|
||||
|
||||
collectMetricsFunc := func(taskID int64) {
|
||||
s.taskLock.Lock(taskID)
|
||||
defer s.taskLock.Unlock(taskID)
|
||||
|
||||
task := s.getTask(taskID)
|
||||
if task == nil {
|
||||
return
|
||||
}
|
||||
s.taskLock.Lock(taskID)
|
||||
defer s.taskLock.Unlock(taskID)
|
||||
|
||||
state := task.GetState()
|
||||
switch state {
|
||||
|
|
|
@ -833,7 +833,11 @@ func (s *taskSchedulerSuite) scheduler(handler Handler) {
|
|||
workerManager.EXPECT().PickClient().Return(s.nodeID, in)
|
||||
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true)
|
||||
|
||||
mt := createMeta(catalog, withAnalyzeMeta(s.createAnalyzeMeta(catalog)), withIndexMeta(createIndexMeta(catalog)))
|
||||
mt := createMeta(catalog, withAnalyzeMeta(s.createAnalyzeMeta(catalog)), withIndexMeta(createIndexMeta(catalog)),
|
||||
withStatsTaskMeta(&statsTaskMeta{
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
}))
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
cm.EXPECT().RootPath().Return("root")
|
||||
|
@ -975,9 +979,13 @@ func (s *taskSchedulerSuite) Test_analyzeTaskFailCase() {
|
|||
},
|
||||
}),
|
||||
withIndexMeta(&indexMeta{
|
||||
RWMutex: sync.RWMutex{},
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
}),
|
||||
withStatsTaskMeta(&statsTaskMeta{
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
tasks: nil,
|
||||
}))
|
||||
|
||||
handler := NewNMockHandler(s.T())
|
||||
|
@ -1016,11 +1024,14 @@ func (s *taskSchedulerSuite) Test_analyzeTaskFailCase() {
|
|||
|
||||
workerManager := session.NewMockWorkerManager(s.T())
|
||||
|
||||
mt := createMeta(catalog, withAnalyzeMeta(s.createAnalyzeMeta(catalog)), withIndexMeta(&indexMeta{
|
||||
RWMutex: sync.RWMutex{},
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
}))
|
||||
mt := createMeta(catalog, withAnalyzeMeta(s.createAnalyzeMeta(catalog)),
|
||||
withIndexMeta(&indexMeta{
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
}), withStatsTaskMeta(&statsTaskMeta{
|
||||
ctx: ctx,
|
||||
catalog: catalog,
|
||||
}))
|
||||
|
||||
handler := NewNMockHandler(s.T())
|
||||
handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{
|
||||
|
@ -1309,6 +1320,10 @@ func (s *taskSchedulerSuite) Test_indexTaskFailCase() {
|
|||
},
|
||||
},
|
||||
},
|
||||
}),
|
||||
withStatsTaskMeta(&statsTaskMeta{
|
||||
ctx: context.Background(),
|
||||
catalog: catalog,
|
||||
}))
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
|
@ -1546,6 +1561,10 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
|
|||
},
|
||||
},
|
||||
},
|
||||
statsTaskMeta: &statsTaskMeta{
|
||||
ctx: context.Background(),
|
||||
catalog: catalog,
|
||||
},
|
||||
}
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
|
|
|
@ -21,10 +21,8 @@ import (
|
|||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/workerpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
|
@ -33,122 +31,6 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
)
|
||||
|
||||
func (s *Server) startStatsTasksCheckLoop(ctx context.Context) {
|
||||
s.serverLoopWg.Add(2)
|
||||
go s.checkStatsTaskLoop(ctx)
|
||||
go s.cleanupStatsTasksLoop(ctx)
|
||||
}
|
||||
|
||||
func (s *Server) checkStatsTaskLoop(ctx context.Context) {
|
||||
log.Info("start checkStatsTaskLoop...")
|
||||
defer s.serverLoopWg.Done()
|
||||
|
||||
ticker := time.NewTicker(Params.DataCoordCfg.TaskCheckInterval.GetAsDuration(time.Second))
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Warn("DataCoord context done, exit checkStatsTaskLoop...")
|
||||
return
|
||||
case <-ticker.C:
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
segments := s.meta.SelectSegments(SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return isFlush(seg) && seg.GetLevel() != datapb.SegmentLevel_L0 && !seg.GetIsSorted() && !seg.isCompacting
|
||||
}))
|
||||
for _, segment := range segments {
|
||||
if err := s.createStatsSegmentTask(segment); err != nil {
|
||||
log.Warn("create stats task for segment failed, wait for retry",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
case segID := <-s.statsCh:
|
||||
log.Info("receive new flushed segment", zap.Int64("segmentID", segID))
|
||||
segment := s.meta.GetSegment(segID)
|
||||
if segment == nil {
|
||||
log.Warn("segment is not exist, no need to do stats task", zap.Int64("segmentID", segID))
|
||||
continue
|
||||
}
|
||||
// TODO @xiaocai2333: remove code after allow create stats task for importing segment
|
||||
if segment.GetIsImporting() {
|
||||
log.Info("segment is importing, skip stats task", zap.Int64("segmentID", segID))
|
||||
select {
|
||||
case s.buildIndexCh <- segID:
|
||||
default:
|
||||
}
|
||||
continue
|
||||
}
|
||||
if err := s.createStatsSegmentTask(segment); err != nil {
|
||||
log.Warn("create stats task for segment failed, wait for retry",
|
||||
zap.Int64("segmentID", segment.ID), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// cleanupStatsTasks clean up the finished/failed stats tasks
|
||||
func (s *Server) cleanupStatsTasksLoop(ctx context.Context) {
|
||||
log.Info("start cleanupStatsTasksLoop...")
|
||||
defer s.serverLoopWg.Done()
|
||||
|
||||
ticker := time.NewTicker(Params.DataCoordCfg.GCInterval.GetAsDuration(time.Second))
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Warn("DataCoord context done, exit cleanupStatsTasksLoop...")
|
||||
return
|
||||
case <-ticker.C:
|
||||
start := time.Now()
|
||||
log.Info("start cleanupUnusedStatsTasks...", zap.Time("startAt", start))
|
||||
|
||||
taskIDs := s.meta.statsTaskMeta.CanCleanedTasks()
|
||||
for _, taskID := range taskIDs {
|
||||
if err := s.meta.statsTaskMeta.RemoveStatsTaskByTaskID(taskID); err != nil {
|
||||
// ignore err, if remove failed, wait next GC
|
||||
log.Warn("clean up stats task failed", zap.Int64("taskID", taskID), zap.Error(err))
|
||||
}
|
||||
}
|
||||
log.Info("recycleUnusedStatsTasks done", zap.Duration("timeCost", time.Since(start)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Server) createStatsSegmentTask(segment *SegmentInfo) error {
|
||||
if segment.GetIsSorted() || segment.GetIsImporting() {
|
||||
// TODO @xiaocai2333: allow importing segment stats
|
||||
log.Info("segment is sorted by segmentID", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
start, _, err := s.allocator.AllocN(2)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t := &indexpb.StatsTask{
|
||||
CollectionID: segment.GetCollectionID(),
|
||||
PartitionID: segment.GetPartitionID(),
|
||||
SegmentID: segment.GetID(),
|
||||
InsertChannel: segment.GetInsertChannel(),
|
||||
TaskID: start,
|
||||
Version: 0,
|
||||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
FailReason: "",
|
||||
TargetSegmentID: start + 1,
|
||||
}
|
||||
if err = s.meta.statsTaskMeta.AddStatsTask(t); err != nil {
|
||||
if errors.Is(err, merr.ErrTaskDuplicate) {
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
s.taskScheduler.enqueue(newStatsTask(t.GetTaskID(), t.GetSegmentID(), t.GetTargetSegmentID(), s.buildIndexCh))
|
||||
return nil
|
||||
}
|
||||
|
||||
type statsTask struct {
|
||||
taskID int64
|
||||
segmentID int64
|
||||
|
@ -162,12 +44,12 @@ type statsTask struct {
|
|||
|
||||
req *workerpb.CreateStatsRequest
|
||||
|
||||
buildIndexCh chan UniqueID
|
||||
subJobType indexpb.StatsSubJob
|
||||
}
|
||||
|
||||
var _ Task = (*statsTask)(nil)
|
||||
|
||||
func newStatsTask(taskID int64, segmentID, targetSegmentID int64, buildIndexCh chan UniqueID) *statsTask {
|
||||
func newStatsTask(taskID int64, segmentID, targetSegmentID int64, subJobType indexpb.StatsSubJob) *statsTask {
|
||||
return &statsTask{
|
||||
taskID: taskID,
|
||||
segmentID: segmentID,
|
||||
|
@ -176,7 +58,7 @@ func newStatsTask(taskID int64, segmentID, targetSegmentID int64, buildIndexCh c
|
|||
TaskID: taskID,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
},
|
||||
buildIndexCh: buildIndexCh,
|
||||
subJobType: subJobType,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -272,7 +154,7 @@ func (st *statsTask) PreCheck(ctx context.Context, dependency *taskScheduler) bo
|
|||
return false
|
||||
}
|
||||
|
||||
if segment.GetIsSorted() {
|
||||
if segment.GetIsSorted() && st.subJobType == indexpb.StatsSubJob_Sort {
|
||||
log.Info("stats task is marked as sorted, skip stats")
|
||||
st.SetState(indexpb.JobState_JobStateNone, "segment is marked as sorted")
|
||||
return false
|
||||
|
@ -311,6 +193,7 @@ func (st *statsTask) PreCheck(ctx context.Context, dependency *taskScheduler) bo
|
|||
DeltaLogs: segment.GetDeltalogs(),
|
||||
StorageConfig: createStorageConfig(),
|
||||
Schema: collInfo.Schema,
|
||||
SubJobType: st.subJobType,
|
||||
TargetSegmentID: st.targetSegmentID,
|
||||
StartLogID: start,
|
||||
EndLogID: end,
|
||||
|
@ -347,6 +230,8 @@ func (st *statsTask) AssignTask(ctx context.Context, client types.IndexNodeClien
|
|||
}
|
||||
|
||||
func (st *statsTask) QueryResult(ctx context.Context, client types.IndexNodeClient) {
|
||||
ctx, cancel := context.WithTimeout(ctx, reqTimeoutInterval)
|
||||
defer cancel()
|
||||
resp, err := client.QueryJobsV2(ctx, &workerpb.QueryJobsV2Request{
|
||||
ClusterID: st.req.GetClusterID(),
|
||||
TaskIDs: []int64{st.GetTaskID()},
|
||||
|
@ -381,6 +266,8 @@ func (st *statsTask) QueryResult(ctx context.Context, client types.IndexNodeClie
|
|||
}
|
||||
|
||||
func (st *statsTask) DropTaskOnWorker(ctx context.Context, client types.IndexNodeClient) bool {
|
||||
ctx, cancel := context.WithTimeout(ctx, reqTimeoutInterval)
|
||||
defer cancel()
|
||||
resp, err := client.DropJobsV2(ctx, &workerpb.DropJobsV2Request{
|
||||
ClusterID: st.req.GetClusterID(),
|
||||
TaskIDs: []int64{st.GetTaskID()},
|
||||
|
@ -398,29 +285,42 @@ func (st *statsTask) DropTaskOnWorker(ctx context.Context, client types.IndexNod
|
|||
}
|
||||
|
||||
func (st *statsTask) SetJobInfo(meta *meta) error {
|
||||
// first update segment
|
||||
metricMutation, err := meta.SaveStatsResultSegment(st.segmentID, st.taskInfo)
|
||||
if err != nil {
|
||||
log.Warn("save stats result failed", zap.Int64("taskID", st.taskID),
|
||||
zap.Int64("segmentID", st.segmentID), zap.Error(err))
|
||||
return err
|
||||
if st.GetState() == indexpb.JobState_JobStateFinished {
|
||||
switch st.subJobType {
|
||||
case indexpb.StatsSubJob_Sort:
|
||||
// first update segment, failed state cannot generate new segment
|
||||
metricMutation, err := meta.SaveStatsResultSegment(st.segmentID, st.taskInfo)
|
||||
if err != nil {
|
||||
log.Warn("save sort stats result failed", zap.Int64("taskID", st.taskID),
|
||||
zap.Int64("segmentID", st.segmentID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
metricMutation.commit()
|
||||
|
||||
select {
|
||||
case getBuildIndexChSingleton() <- st.taskInfo.GetSegmentID():
|
||||
default:
|
||||
}
|
||||
case indexpb.StatsSubJob_TextIndexJob:
|
||||
err := meta.UpdateSegment(st.taskInfo.GetSegmentID(), SetTextIndexLogs(st.taskInfo.GetTextStatsLogs()))
|
||||
if err != nil {
|
||||
log.Warn("save text index stats result failed", zap.Int64("taskID", st.taskID),
|
||||
zap.Int64("segmentID", st.segmentID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
case indexpb.StatsSubJob_BM25Job:
|
||||
// TODO: support bm25 job
|
||||
}
|
||||
}
|
||||
|
||||
// second update the task meta
|
||||
if err = meta.statsTaskMeta.FinishTask(st.taskID, st.taskInfo); err != nil {
|
||||
if err := meta.statsTaskMeta.FinishTask(st.taskID, st.taskInfo); err != nil {
|
||||
log.Warn("save stats result failed", zap.Int64("taskID", st.taskID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
metricMutation.commit()
|
||||
log.Info("SetJobInfo for stats task success", zap.Int64("taskID", st.taskID),
|
||||
zap.Int64("oldSegmentID", st.segmentID), zap.Int64("targetSegmentID", st.taskInfo.GetSegmentID()))
|
||||
|
||||
if st.buildIndexCh != nil {
|
||||
select {
|
||||
case st.buildIndexCh <- st.taskInfo.GetSegmentID():
|
||||
default:
|
||||
}
|
||||
}
|
||||
zap.Int64("oldSegmentID", st.segmentID), zap.Int64("targetSegmentID", st.taskInfo.GetSegmentID()),
|
||||
zap.String("subJobType", st.subJobType.String()), zap.String("state", st.taskInfo.GetState().String()))
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -122,25 +122,12 @@ func (s *statsTaskSuite) SetupSuite() {
|
|||
FailReason: "",
|
||||
},
|
||||
},
|
||||
segmentStatsTaskIndex: map[int64]*indexpb.StatsTask{
|
||||
s.segID: {
|
||||
CollectionID: 1,
|
||||
PartitionID: 2,
|
||||
SegmentID: s.segID,
|
||||
InsertChannel: "ch1",
|
||||
TaskID: s.taskID,
|
||||
Version: 0,
|
||||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
FailReason: "",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *statsTaskSuite) TestTaskStats_PreCheck() {
|
||||
st := newStatsTask(s.taskID, s.segID, s.targetID, nil)
|
||||
st := newStatsTask(s.taskID, s.segID, s.targetID, indexpb.StatsSubJob_Sort)
|
||||
|
||||
s.Equal(s.taskID, st.GetTaskID())
|
||||
|
||||
|
@ -421,7 +408,6 @@ func (s *statsTaskSuite) TestTaskStats_PreCheck() {
|
|||
Channel: "ch1",
|
||||
InsertLogs: nil,
|
||||
StatsLogs: nil,
|
||||
DeltaLogs: nil,
|
||||
TextStatsLogs: nil,
|
||||
NumRows: 65535,
|
||||
},
|
||||
|
|
|
@ -123,13 +123,13 @@ func mergeSortMultipleSegments(ctx context.Context,
|
|||
return nil, err
|
||||
}
|
||||
|
||||
v, err = segmentReaders[smallest.Index].Next()
|
||||
iv, err := segmentReaders[smallest.Index].Next()
|
||||
if err != nil && err != sio.EOF {
|
||||
return nil, err
|
||||
}
|
||||
if err == nil {
|
||||
next := &PQItem{
|
||||
Value: v,
|
||||
Value: iv,
|
||||
Index: smallest.Index,
|
||||
}
|
||||
heap.Push(&pq, next)
|
||||
|
|
|
@ -243,6 +243,7 @@ func (s *MixCompactionTaskSuite) TestCompactSortedSegment() {
|
|||
s.EqualValues(19531, segment.GetSegmentID())
|
||||
s.EqualValues(300, segment.GetNumOfRows())
|
||||
s.NotEmpty(segment.InsertLogs)
|
||||
|
||||
s.NotEmpty(segment.Field2StatslogPaths)
|
||||
s.Empty(segment.Deltalogs)
|
||||
}
|
||||
|
|
|
@ -379,6 +379,7 @@ func (i *IndexNode) CreateJobV2(ctx context.Context, req *workerpb.CreateJobV2Re
|
|||
zap.Int64("partitionID", statsRequest.GetPartitionID()),
|
||||
zap.Int64("segmentID", statsRequest.GetSegmentID()),
|
||||
zap.Int64("targetSegmentID", statsRequest.GetTargetSegmentID()),
|
||||
zap.String("subJobType", statsRequest.GetSubJobType().String()),
|
||||
zap.Int64("startLogID", statsRequest.GetStartLogID()),
|
||||
zap.Int64("endLogID", statsRequest.GetEndLogID()),
|
||||
)
|
||||
|
@ -512,7 +513,6 @@ func (i *IndexNode) QueryJobsV2(ctx context.Context, req *workerpb.QueryJobsV2Re
|
|||
Channel: info.insertChannel,
|
||||
InsertLogs: info.insertLogs,
|
||||
StatsLogs: info.statsLogs,
|
||||
DeltaLogs: nil,
|
||||
TextStatsLogs: info.textStatsLogs,
|
||||
NumRows: info.numRows,
|
||||
})
|
||||
|
|
|
@ -634,6 +634,7 @@ func (s *IndexNodeSuite) Test_CreateStatsTask() {
|
|||
EndLogID: s.logID + 200,
|
||||
NumRows: s.numRows,
|
||||
BinlogMaxSize: 131000,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
}
|
||||
|
||||
status, err := s.in.CreateJobV2(ctx, &workerpb.CreateJobV2Request{
|
||||
|
@ -661,7 +662,6 @@ func (s *IndexNodeSuite) Test_CreateStatsTask() {
|
|||
if resp.GetStatsJobResults().GetResults()[0].GetState() == indexpb.JobState_JobStateFinished {
|
||||
s.NotZero(len(resp.GetStatsJobResults().GetResults()[0].GetInsertLogs()))
|
||||
s.NotZero(len(resp.GetStatsJobResults().GetResults()[0].GetStatsLogs()))
|
||||
s.Zero(len(resp.GetStatsJobResults().GetResults()[0].GetDeltaLogs()))
|
||||
s.Equal(s.numRows, resp.GetStatsJobResults().GetResults()[0].GetNumRows())
|
||||
break
|
||||
}
|
||||
|
|
|
@ -153,23 +153,13 @@ func (st *statsTask) PreExecute(ctx context.Context) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (st *statsTask) Execute(ctx context.Context) error {
|
||||
// sort segment and check need to do text index.
|
||||
ctx, span := otel.Tracer(typeutil.IndexNodeRole).Start(ctx, fmt.Sprintf("Stats-Execute-%s-%d", st.req.GetClusterID(), st.req.GetTaskID()))
|
||||
defer span.End()
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("clusterID", st.req.GetClusterID()),
|
||||
zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("collectionID", st.req.GetCollectionID()),
|
||||
zap.Int64("partitionID", st.req.GetPartitionID()),
|
||||
zap.Int64("segmentID", st.req.GetSegmentID()),
|
||||
)
|
||||
|
||||
func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, error) {
|
||||
numRows := st.req.GetNumRows()
|
||||
writer, err := compaction.NewSegmentWriter(st.req.GetSchema(), numRows, st.req.GetTargetSegmentID(), st.req.GetPartitionID(), st.req.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("sort segment wrong, unable to init segment writer", zap.Error(err))
|
||||
return err
|
||||
log.Warn("sort segment wrong, unable to init segment writer",
|
||||
zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var (
|
||||
|
@ -186,8 +176,8 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
|
||||
values, err := st.downloadData(ctx, numRows, writer.GetPkID())
|
||||
if err != nil {
|
||||
log.Warn("download data failed", zap.Error(err))
|
||||
return err
|
||||
log.Warn("download data failed", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
sortStart := time.Now()
|
||||
|
@ -199,8 +189,8 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
for _, v := range values {
|
||||
err := writer.Write(v)
|
||||
if err != nil {
|
||||
log.Warn("write value wrong, failed to writer row", zap.Error(err))
|
||||
return err
|
||||
log.Warn("write value wrong, failed to writer row", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
unFlushedRowCount++
|
||||
|
||||
|
@ -208,15 +198,15 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
serWriteStart := time.Now()
|
||||
binlogNum, kvs, partialBinlogs, err := serializeWrite(ctx, st.req.GetStartLogID()+st.logIDOffset, writer)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to serialize writer", zap.Error(err))
|
||||
return err
|
||||
log.Warn("stats wrong, failed to serialize writer", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
serWriteTimeCost += time.Since(serWriteStart)
|
||||
|
||||
uploadStart := time.Now()
|
||||
if err := st.binlogIO.Upload(ctx, kvs); err != nil {
|
||||
log.Warn("stats wrong, failed to upload kvs", zap.Error(err))
|
||||
return err
|
||||
log.Warn("stats wrong, failed to upload kvs", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
uploadTimeCost += time.Since(uploadStart)
|
||||
|
||||
|
@ -226,10 +216,10 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
unFlushedRowCount = 0
|
||||
st.logIDOffset += binlogNum
|
||||
if st.req.GetStartLogID()+st.logIDOffset >= st.req.GetEndLogID() {
|
||||
log.Warn("binlog files too much, log is not enough",
|
||||
log.Warn("binlog files too much, log is not enough", zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("binlog num", binlogNum), zap.Int64("startLogID", st.req.GetStartLogID()),
|
||||
zap.Int64("endLogID", st.req.GetEndLogID()), zap.Int64("logIDOffset", st.logIDOffset))
|
||||
return fmt.Errorf("binlog files too much, log is not enough")
|
||||
return nil, fmt.Errorf("binlog files too much, log is not enough")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -238,15 +228,15 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
serWriteStart := time.Now()
|
||||
binlogNum, kvs, partialBinlogs, err := serializeWrite(ctx, st.req.GetStartLogID()+st.logIDOffset, writer)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to serialize writer", zap.Error(err))
|
||||
return err
|
||||
log.Warn("stats wrong, failed to serialize writer", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
serWriteTimeCost += time.Since(serWriteStart)
|
||||
st.logIDOffset += binlogNum
|
||||
|
||||
uploadStart := time.Now()
|
||||
if err := st.binlogIO.Upload(ctx, kvs); err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
uploadTimeCost += time.Since(uploadStart)
|
||||
|
||||
|
@ -257,9 +247,9 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
serWriteStart := time.Now()
|
||||
binlogNums, sPath, err := statSerializeWrite(ctx, st.binlogIO, st.req.GetStartLogID()+st.logIDOffset, writer, numRows)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to serialize write segment stats",
|
||||
log.Warn("stats wrong, failed to serialize write segment stats", zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("remaining row count", numRows), zap.Error(err))
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
serWriteTimeCost += time.Since(serWriteStart)
|
||||
|
||||
|
@ -269,15 +259,29 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
|
||||
insertLogs := lo.Values(allBinlogs)
|
||||
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
statsLogs := []*datapb.FieldBinlog{sPath}
|
||||
if err := binlog.CompressFieldBinlogs(statsLogs); err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
st.node.storePKSortStatsResult(st.req.GetClusterID(),
|
||||
st.req.GetTaskID(),
|
||||
st.req.GetCollectionID(),
|
||||
st.req.GetPartitionID(),
|
||||
st.req.GetTargetSegmentID(),
|
||||
st.req.GetInsertChannel(),
|
||||
int64(len(values)), insertLogs, statsLogs)
|
||||
|
||||
log.Info("sort segment end",
|
||||
zap.String("clusterID", st.req.GetClusterID()),
|
||||
zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("collectionID", st.req.GetCollectionID()),
|
||||
zap.Int64("partitionID", st.req.GetPartitionID()),
|
||||
zap.Int64("segmentID", st.req.GetSegmentID()),
|
||||
zap.String("subTaskType", st.req.GetSubJobType().String()),
|
||||
zap.Int64("target segmentID", st.req.GetTargetSegmentID()),
|
||||
zap.Int64("old rows", numRows),
|
||||
zap.Int("valid rows", len(values)),
|
||||
|
@ -286,27 +290,39 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
zap.Duration("sort elapse", sortTimeCost),
|
||||
zap.Duration("serWrite elapse", serWriteTimeCost),
|
||||
zap.Duration("total elapse", totalElapse))
|
||||
return insertLogs, nil
|
||||
}
|
||||
|
||||
textIndexStatsLogs, err := st.createTextIndex(ctx,
|
||||
st.req.GetStorageConfig(),
|
||||
st.req.GetCollectionID(),
|
||||
st.req.GetPartitionID(),
|
||||
st.req.GetTargetSegmentID(),
|
||||
st.req.GetTaskVersion(),
|
||||
st.req.GetTaskID(),
|
||||
lo.Values(allBinlogs))
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to create text index", zap.Error(err))
|
||||
return err
|
||||
func (st *statsTask) Execute(ctx context.Context) error {
|
||||
// sort segment and check need to do text index.
|
||||
ctx, span := otel.Tracer(typeutil.IndexNodeRole).Start(ctx, fmt.Sprintf("Stats-Execute-%s-%d", st.req.GetClusterID(), st.req.GetTaskID()))
|
||||
defer span.End()
|
||||
|
||||
insertLogs := st.req.GetInsertLogs()
|
||||
var err error
|
||||
if st.req.GetSubJobType() == indexpb.StatsSubJob_Sort {
|
||||
insertLogs, err = st.sortSegment(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
st.node.storeStatsResult(st.req.GetClusterID(),
|
||||
st.req.GetTaskID(),
|
||||
st.req.GetCollectionID(),
|
||||
st.req.GetPartitionID(),
|
||||
st.req.GetTargetSegmentID(),
|
||||
st.req.GetInsertChannel(),
|
||||
int64(len(values)), insertLogs, statsLogs, textIndexStatsLogs)
|
||||
if st.req.GetSubJobType() == indexpb.StatsSubJob_Sort || st.req.GetSubJobType() == indexpb.StatsSubJob_TextIndexJob {
|
||||
err = st.createTextIndex(ctx,
|
||||
st.req.GetStorageConfig(),
|
||||
st.req.GetCollectionID(),
|
||||
st.req.GetPartitionID(),
|
||||
st.req.GetTargetSegmentID(),
|
||||
st.req.GetTaskVersion(),
|
||||
st.req.GetTaskID(),
|
||||
insertLogs)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to create text index", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: support bm25
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -568,9 +584,9 @@ func (st *statsTask) createTextIndex(ctx context.Context,
|
|||
partitionID int64,
|
||||
segmentID int64,
|
||||
version int64,
|
||||
buildID int64,
|
||||
taskID int64,
|
||||
insertBinlogs []*datapb.FieldBinlog,
|
||||
) (map[int64]*datapb.TextIndexStats, error) {
|
||||
) error {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("clusterID", st.req.GetClusterID()),
|
||||
zap.Int64("taskID", st.req.GetTaskID()),
|
||||
|
@ -599,10 +615,10 @@ func (st *statsTask) createTextIndex(ctx context.Context,
|
|||
|
||||
newStorageConfig, err := ParseStorageConfig(storageConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
fieldStatsLogs := make(map[int64]*datapb.TextIndexStats)
|
||||
textIndexLogs := make(map[int64]*datapb.TextIndexStats)
|
||||
for _, field := range st.req.GetSchema().GetFields() {
|
||||
h := typeutil.CreateFieldSchemaHelper(field)
|
||||
if !h.EnableMatch() {
|
||||
|
@ -612,11 +628,11 @@ func (st *statsTask) createTextIndex(ctx context.Context,
|
|||
// create text index and upload the text index files.
|
||||
files, err := getInsertFiles(field.GetFieldID())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
|
||||
buildIndexParams := &indexcgopb.BuildIndexInfo{
|
||||
BuildID: buildID,
|
||||
BuildID: taskID,
|
||||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
SegmentID: segmentID,
|
||||
|
@ -628,12 +644,12 @@ func (st *statsTask) createTextIndex(ctx context.Context,
|
|||
|
||||
uploaded, err := indexcgowrapper.CreateTextIndex(ctx, buildIndexParams)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
fieldStatsLogs[field.GetFieldID()] = &datapb.TextIndexStats{
|
||||
textIndexLogs[field.GetFieldID()] = &datapb.TextIndexStats{
|
||||
FieldID: field.GetFieldID(),
|
||||
Version: version,
|
||||
BuildID: buildID,
|
||||
BuildID: taskID,
|
||||
Files: lo.Keys(uploaded),
|
||||
}
|
||||
log.Info("field enable match, create text index done",
|
||||
|
@ -644,8 +660,16 @@ func (st *statsTask) createTextIndex(ctx context.Context,
|
|||
|
||||
totalElapse := st.tr.RecordSpan()
|
||||
|
||||
st.node.storeStatsTextIndexResult(st.req.GetClusterID(),
|
||||
st.req.GetTaskID(),
|
||||
st.req.GetCollectionID(),
|
||||
st.req.GetPartitionID(),
|
||||
st.req.GetTargetSegmentID(),
|
||||
st.req.GetInsertChannel(),
|
||||
textIndexLogs)
|
||||
|
||||
log.Info("create text index done",
|
||||
zap.Int64("target segmentID", st.req.GetTargetSegmentID()),
|
||||
zap.Duration("total elapse", totalElapse))
|
||||
return fieldStatsLogs, nil
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -360,7 +360,7 @@ func (i *IndexNode) storeStatsTaskState(clusterID string, taskID UniqueID, state
|
|||
}
|
||||
}
|
||||
|
||||
func (i *IndexNode) storeStatsResult(
|
||||
func (i *IndexNode) storePKSortStatsResult(
|
||||
ClusterID string,
|
||||
taskID UniqueID,
|
||||
collID UniqueID,
|
||||
|
@ -370,7 +370,6 @@ func (i *IndexNode) storeStatsResult(
|
|||
numRows int64,
|
||||
insertLogs []*datapb.FieldBinlog,
|
||||
statsLogs []*datapb.FieldBinlog,
|
||||
fieldStatsLogs map[int64]*datapb.TextIndexStats,
|
||||
) {
|
||||
key := taskKey{ClusterID: ClusterID, TaskID: taskID}
|
||||
i.stateLock.Lock()
|
||||
|
@ -383,11 +382,31 @@ func (i *IndexNode) storeStatsResult(
|
|||
info.numRows = numRows
|
||||
info.insertLogs = insertLogs
|
||||
info.statsLogs = statsLogs
|
||||
info.textStatsLogs = fieldStatsLogs
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func (i *IndexNode) storeStatsTextIndexResult(
|
||||
ClusterID string,
|
||||
taskID UniqueID,
|
||||
collID UniqueID,
|
||||
partID UniqueID,
|
||||
segID UniqueID,
|
||||
channel string,
|
||||
texIndexLogs map[int64]*datapb.TextIndexStats,
|
||||
) {
|
||||
key := taskKey{ClusterID: ClusterID, TaskID: taskID}
|
||||
i.stateLock.Lock()
|
||||
defer i.stateLock.Unlock()
|
||||
if info, ok := i.statsTasks[key]; ok {
|
||||
info.textStatsLogs = texIndexLogs
|
||||
info.segID = segID
|
||||
info.collID = collID
|
||||
info.partID = partID
|
||||
info.insertChannel = channel
|
||||
}
|
||||
}
|
||||
|
||||
func (i *IndexNode) getStatsTaskInfo(clusterID string, taskID UniqueID) *statsTaskInfo {
|
||||
i.stateLock.Lock()
|
||||
defer i.stateLock.Unlock()
|
||||
|
|
|
@ -76,9 +76,14 @@ func (s *statsTaskInfoSuite) Test_Methods() {
|
|||
})
|
||||
|
||||
s.Run("storeStatsResult", func() {
|
||||
s.node.storeStatsResult(s.cluster, s.taskID, 1, 2, 3, "ch1", 65535,
|
||||
s.node.storePKSortStatsResult(s.cluster, s.taskID, 1, 2, 3, "ch1", 65535,
|
||||
[]*datapb.FieldBinlog{{FieldID: 100, Binlogs: []*datapb.Binlog{{LogID: 1}}}},
|
||||
[]*datapb.FieldBinlog{{FieldID: 100, Binlogs: []*datapb.Binlog{{LogID: 2}}}},
|
||||
)
|
||||
})
|
||||
|
||||
s.Run("storeStatsTextIndexResult", func() {
|
||||
s.node.storeStatsTextIndexResult(s.cluster, s.taskID, 1, 2, 3, "ch1",
|
||||
map[int64]*datapb.TextIndexStats{
|
||||
100: {
|
||||
FieldID: 100,
|
||||
|
@ -87,8 +92,7 @@ func (s *statsTaskInfoSuite) Test_Methods() {
|
|||
LogSize: 1024,
|
||||
MemorySize: 1024,
|
||||
},
|
||||
},
|
||||
)
|
||||
})
|
||||
})
|
||||
|
||||
s.Run("getStatsTaskInfo", func() {
|
||||
|
|
|
@ -111,6 +111,12 @@ func (kc *Catalog) listSegments() ([]*datapb.SegmentInfo, error) {
|
|||
return nil
|
||||
}
|
||||
|
||||
// due to StatsTaskPrefix has the same prefix with SegmentPrefix, so skip it.
|
||||
// when the WalkWithPrefix is refactored, this patch can be removed.
|
||||
if strings.Contains(string(value), StatsTaskPrefix) {
|
||||
return nil
|
||||
}
|
||||
|
||||
segmentInfo := &datapb.SegmentInfo{}
|
||||
err := proto.Unmarshal(value, segmentInfo)
|
||||
if err != nil {
|
||||
|
|
|
@ -311,4 +311,13 @@ message StatsTask {
|
|||
JobState state = 8;
|
||||
string fail_reason = 9;
|
||||
int64 target_segmentID = 10;
|
||||
index.StatsSubJob subJobType = 11;
|
||||
bool canRecycle = 12;
|
||||
}
|
||||
|
||||
enum StatsSubJob {
|
||||
None = 0;
|
||||
Sort = 1;
|
||||
TextIndexJob = 2;
|
||||
BM25Job=3;
|
||||
}
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
syntax = "proto3";
|
||||
|
||||
package milvus.proto.worker;
|
||||
package milvus.proto.index;
|
||||
|
||||
option go_package = "github.com/milvus-io/milvus/internal/proto/workerpb";
|
||||
|
||||
|
@ -133,14 +133,15 @@ message CreateStatsRequest {
|
|||
repeated data.FieldBinlog delta_logs = 8;
|
||||
index.StorageConfig storage_config = 9;
|
||||
schema.CollectionSchema schema = 10;
|
||||
int64 targetSegmentID = 11;
|
||||
int64 startLogID = 12;
|
||||
int64 endLogID = 13;
|
||||
int64 num_rows = 14;
|
||||
int64 collection_ttl = 15;
|
||||
uint64 current_ts = 16;
|
||||
int64 task_version = 17;
|
||||
uint64 binlogMaxSize = 18;
|
||||
index.StatsSubJob subJobType = 11;
|
||||
int64 targetSegmentID = 12;
|
||||
int64 startLogID = 13;
|
||||
int64 endLogID = 14;
|
||||
int64 num_rows = 15;
|
||||
int64 collection_ttl = 16;
|
||||
uint64 current_ts = 17;
|
||||
int64 task_version = 18;
|
||||
uint64 binlogMaxSize = 19;
|
||||
}
|
||||
|
||||
message CreateJobV2Request {
|
||||
|
@ -195,9 +196,8 @@ message StatsResult {
|
|||
string channel = 7;
|
||||
repeated data.FieldBinlog insert_logs = 8;
|
||||
repeated data.FieldBinlog stats_logs = 9;
|
||||
repeated data.FieldBinlog delta_logs = 10;
|
||||
map<int64, data.TextIndexStats> text_stats_logs = 11;
|
||||
int64 num_rows = 12;
|
||||
map<int64, data.TextIndexStats> text_stats_logs = 10;
|
||||
int64 num_rows = 11;
|
||||
}
|
||||
|
||||
message StatsResults {
|
||||
|
|
Loading…
Reference in New Issue