diff --git a/configs/advanced/data_coord.yaml b/configs/advanced/data_coord.yaml index 36cbb5d102..9d7dcd3347 100644 --- a/configs/advanced/data_coord.yaml +++ b/configs/advanced/data_coord.yaml @@ -19,3 +19,5 @@ dataCoord: maxSize: 512 # Maximum size of a segment in MB sealProportion: 0.75 # It's the minimum proportion for a segment which can be sealed assignmentExpiration: 2000 # ms + + enableCompaction: false \ No newline at end of file diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index e00b66b085..6116579d9c 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -293,3 +293,27 @@ func (c *ChannelManager) Match(nodeID int64, channel string) bool { } return false } + +// FindWatcher finds the datanode watching the provided channel +func (c *ChannelManager) FindWatcher(channel string) (int64, error) { + c.mu.RLock() + defer c.mu.RUnlock() + + infos := c.store.GetNodesChannels() + for _, info := range infos { + for _, channelInfo := range info.Channels { + if channelInfo.Name == channel { + return info.NodeID, nil + } + } + } + + // channel in buffer + bufferInfo := c.store.GetBufferChannelInfo() + for _, channelInfo := range bufferInfo.Channels { + if channelInfo.Name == channel { + return bufferID, errChannelInBuffer + } + } + return 0, errChannelNotWatched +} diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go new file mode 100644 index 0000000000..f2e9c68643 --- /dev/null +++ b/internal/datacoord/compaction.go @@ -0,0 +1,288 @@ +package datacoord + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/util/tsoutil" + "go.uber.org/zap" +) + +// TODO this num should be determined by resources of datanode, for now, we set to a fixed value for simple +const ( + maxParallelCompactionTaskNum = 100 + compactionTimeout = 10 * time.Second + compactionExpirationCheckInterval = 60 * time.Second +) + +type compactionPlanContext interface { + start() + stop() + // execCompactionPlan start to execute plan and return immediately + execCompactionPlan(plan *datapb.CompactionPlan) error + // completeCompaction record the result of a compaction + completeCompaction(result *datapb.CompactionResult) error + // getCompaction return compaction task. If planId does not exist, return nil. + getCompaction(planID int64) *compactionTask + // expireCompaction set the compaction state to expired + expireCompaction(ts Timestamp) error + // isFull return true if the task pool is full + isFull() bool + // get compaction by signal id and return the number of executing/completed/timeout plans + getCompactionBySignalID(signalID int64) (executing, completed, timeout int) +} + +type compactionTaskState int8 + +const ( + executing compactionTaskState = iota + 1 + completed + timeout +) + +var ( + errChannelNotWatched = errors.New("channel is not watched") + errChannelInBuffer = errors.New("channel is in buffer") +) + +type compactionTask struct { + triggerInfo *compactionSignal + plan *datapb.CompactionPlan + state compactionTaskState + dataNodeID int64 +} + +func (t *compactionTask) shadowClone(opts ...compactionTaskOpt) *compactionTask { + task := &compactionTask{ + plan: t.plan, + state: t.state, + dataNodeID: t.dataNodeID, + } + for _, opt := range opts { + opt(task) + } + return task +} + +var _ compactionPlanContext = (*compactionPlanHandler)(nil) + +type compactionPlanHandler struct { + plans map[int64]*compactionTask // planid -> task + sessions *SessionManager + meta *meta + chManager *ChannelManager + mu sync.RWMutex + executingTaskNum int + allocator allocator + quit chan struct{} + wg sync.WaitGroup + flushCh chan UniqueID +} + +func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta *meta, + allocator allocator, flush chan UniqueID) *compactionPlanHandler { + return &compactionPlanHandler{ + plans: make(map[int64]*compactionTask), + chManager: cm, + meta: meta, + sessions: sessions, + allocator: allocator, + flushCh: flush, + } +} + +func (c *compactionPlanHandler) start() { + ticker := time.NewTicker(compactionExpirationCheckInterval) + c.quit = make(chan struct{}) + c.wg.Add(1) + + go func() { + defer c.wg.Done() + for { + select { + case <-c.quit: + ticker.Stop() + log.Info("compaction handler quit") + return + case <-ticker.C: + cctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + ts, err := c.allocator.allocTimestamp(cctx) + if err != nil { + log.Warn("unable to alloc timestamp", zap.Error(err)) + cancel() + continue + } + cancel() + _ = c.expireCompaction(ts) + } + } + }() +} + +func (c *compactionPlanHandler) stop() { + close(c.quit) + c.wg.Wait() +} + +// execCompactionPlan start to execute plan and return immediately +func (c *compactionPlanHandler) execCompactionPlan(plan *datapb.CompactionPlan) error { + c.mu.Lock() + defer c.mu.Unlock() + + nodeID, err := c.chManager.FindWatcher(plan.GetChannel()) + if err != nil { + return err + } + + c.setSegmentsCompacting(plan, true) + + // FIXME: check response of compaction call and restore segment state if failed + c.sessions.Compaction(nodeID, plan) + + task := &compactionTask{ + plan: plan, + state: executing, + dataNodeID: nodeID, + } + c.plans[plan.PlanID] = task + c.executingTaskNum++ + return nil +} + +func (c *compactionPlanHandler) setSegmentsCompacting(plan *datapb.CompactionPlan, compacting bool) { + for _, segmentBinlogs := range plan.GetSegmentBinlogs() { + c.meta.SetSegmentCompacting(segmentBinlogs.GetSegmentID(), compacting) + } +} + +// completeCompaction record the result of a compaction +func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResult) error { + c.mu.Lock() + defer c.mu.Unlock() + + planID := result.PlanID + if _, ok := c.plans[planID]; !ok { + return fmt.Errorf("plan %d is not found", planID) + } + + if c.plans[planID].state != executing { + return fmt.Errorf("plan %d's state is %v", planID, c.plans[planID].state) + } + + plan := c.plans[planID].plan + switch plan.GetType() { + case datapb.CompactionType_InnerCompaction: + if err := c.handleInnerCompactionResult(plan, result); err != nil { + return err + } + case datapb.CompactionType_MergeCompaction: + if err := c.handleMergeCompactionResult(plan, result); err != nil { + return err + } + default: + return errors.New("unknown compaction type") + } + c.plans[planID] = c.plans[planID].shadowClone(setState(completed)) + c.executingTaskNum-- + if c.plans[planID].plan.GetType() == datapb.CompactionType_MergeCompaction { + c.flushCh <- result.GetSegmentID() + } + // TODO: when to clean task list + + return nil +} + +func (c *compactionPlanHandler) handleInnerCompactionResult(plan *datapb.CompactionPlan, result *datapb.CompactionResult) error { + return c.meta.CompleteInnerCompaction(plan.GetSegmentBinlogs()[0], result) +} + +func (c *compactionPlanHandler) handleMergeCompactionResult(plan *datapb.CompactionPlan, result *datapb.CompactionResult) error { + return c.meta.CompleteMergeCompaction(plan.GetSegmentBinlogs(), result) +} + +// getCompaction return compaction task. If planId does not exist, return nil. +func (c *compactionPlanHandler) getCompaction(planID int64) *compactionTask { + c.mu.RLock() + defer c.mu.RUnlock() + + return c.plans[planID] +} + +// expireCompaction set the compaction state to expired +func (c *compactionPlanHandler) expireCompaction(ts Timestamp) error { + c.mu.Lock() + defer c.mu.Unlock() + + tasks := c.getExecutingCompactions() + for _, task := range tasks { + if !c.isTimeout(ts, task.plan.GetStartTime(), task.plan.GetTimeoutInSeconds()) { + continue + } + + c.setSegmentsCompacting(task.plan, false) + + planID := task.plan.PlanID + c.plans[planID] = c.plans[planID].shadowClone(setState(timeout)) + c.executingTaskNum-- + } + + return nil +} + +func (c *compactionPlanHandler) isTimeout(now Timestamp, start Timestamp, timeout int32) bool { + starttime, _ := tsoutil.ParseTS(start) + ts, _ := tsoutil.ParseTS(now) + return int32(ts.Sub(starttime).Seconds()) >= timeout +} + +// isFull return true if the task pool is full +func (c *compactionPlanHandler) isFull() bool { + c.mu.RLock() + defer c.mu.RUnlock() + + return c.executingTaskNum >= maxParallelCompactionTaskNum +} + +func (c *compactionPlanHandler) getExecutingCompactions() []*compactionTask { + tasks := make([]*compactionTask, 0, len(c.plans)) + for _, plan := range c.plans { + if plan.state == executing { + tasks = append(tasks, plan) + } + } + return tasks +} + +// get compaction by signal id and return the number of executing/completed/timeout plans +func (c *compactionPlanHandler) getCompactionBySignalID(signalID int64) (executingPlans int, completedPlans int, timeoutPlans int) { + c.mu.RLock() + defer c.mu.RUnlock() + + for _, t := range c.plans { + if t.triggerInfo.id != signalID { + continue + } + switch t.state { + case executing: + executingPlans++ + case completed: + completedPlans++ + case timeout: + timeoutPlans++ + } + } + return +} + +type compactionTaskOpt func(task *compactionTask) + +func setState(state compactionTaskState) compactionTaskOpt { + return func(task *compactionTask) { + task.state = state + } +} diff --git a/internal/datacoord/compaction_policy.go b/internal/datacoord/compaction_policy.go new file mode 100644 index 0000000000..52349025c4 --- /dev/null +++ b/internal/datacoord/compaction_policy.go @@ -0,0 +1,109 @@ +package datacoord + +import ( + "sort" + + "github.com/milvus-io/milvus/internal/proto/datapb" +) + +type singleCompactionPolicy interface { + // shouldSingleCompaction generates a compaction plan for single comapction, return nil if no plan can be generated. + generatePlan(segment *SegmentInfo, timetravel *timetravel) *datapb.CompactionPlan +} + +type mergeCompactionPolicy interface { + // shouldMergeCompaction generates a compaction plan for merge compaction, return nil if no plan can be generated. + generatePlan(segments []*SegmentInfo, timetravel *timetravel) []*datapb.CompactionPlan +} + +type singleCompactionFunc func(segment *SegmentInfo, timetravel *timetravel) *datapb.CompactionPlan + +func (f singleCompactionFunc) generatePlan(segment *SegmentInfo, timetravel *timetravel) *datapb.CompactionPlan { + return f(segment, timetravel) +} + +func chooseAllBinlogs(segment *SegmentInfo, timetravel *timetravel) *datapb.CompactionPlan { + deltaLogs := make([]*datapb.DeltaLogInfo, 0) + for _, l := range segment.GetDeltalogs() { + if l.TimestampTo < timetravel.time { + deltaLogs = append(deltaLogs, l) + } + } + + return &datapb.CompactionPlan{ + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: segment.GetID(), + FieldBinlogs: segment.GetBinlogs(), + Field2StatslogPaths: segment.GetStatslogs(), + Deltalogs: deltaLogs, + }, + }, + Type: datapb.CompactionType_InnerCompaction, + Timetravel: timetravel.time, + Channel: segment.GetInsertChannel(), + } +} + +type mergeCompactionFunc func(segments []*SegmentInfo, timetravel *timetravel) []*datapb.CompactionPlan + +func (f mergeCompactionFunc) generatePlan(segments []*SegmentInfo, timetravel *timetravel) []*datapb.CompactionPlan { + return f(segments, timetravel) +} + +func greedyMergeCompaction(segments []*SegmentInfo, timetravel *timetravel) []*datapb.CompactionPlan { + if len(segments) == 0 { + return nil + } + + sort.Slice(segments, func(i, j int) bool { + return segments[i].NumOfRows < segments[j].NumOfRows + }) + + return greedyGeneratePlans(segments, timetravel) +} + +func greedyGeneratePlans(sortedSegments []*SegmentInfo, timetravel *timetravel) []*datapb.CompactionPlan { + maxRowNumPerSegment := sortedSegments[0].MaxRowNum + + plans := make([]*datapb.CompactionPlan, 0) + free := maxRowNumPerSegment + plan := &datapb.CompactionPlan{ + Timetravel: timetravel.time, + Type: datapb.CompactionType_MergeCompaction, + Channel: sortedSegments[0].GetInsertChannel(), + } + + for _, s := range sortedSegments { + segmentBinlogs := &datapb.CompactionSegmentBinlogs{ + SegmentID: s.GetID(), + FieldBinlogs: s.GetBinlogs(), + Field2StatslogPaths: s.GetStatslogs(), + Deltalogs: s.GetDeltalogs(), + } + + if s.NumOfRows > free { + // if the plan size is less than or equal to 1, it means that every unchecked segment is larger than half of max segment size + // so there's no need to merge them + if len(plan.SegmentBinlogs) <= 1 { + break + } + plans = append(plans, plan) + plan = &datapb.CompactionPlan{ + Timetravel: timetravel.time, + Type: datapb.CompactionType_MergeCompaction, + Channel: sortedSegments[0].GetInsertChannel(), + } + free = maxRowNumPerSegment + } + plan.SegmentBinlogs = append(plan.SegmentBinlogs, segmentBinlogs) + free -= s.GetNumOfRows() + } + + // if plan contains zero or one segment, dont need to merge + if len(plan.SegmentBinlogs) > 1 { + plans = append(plans, plan) + } + + return plans +} diff --git a/internal/datacoord/compaction_policy_test.go b/internal/datacoord/compaction_policy_test.go new file mode 100644 index 0000000000..7a0e6fd626 --- /dev/null +++ b/internal/datacoord/compaction_policy_test.go @@ -0,0 +1,110 @@ +package datacoord + +import ( + "testing" + + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/stretchr/testify/assert" +) + +func Test_greedyMergeCompaction(t *testing.T) { + type args struct { + segments []*SegmentInfo + timetravel *timetravel + } + tests := []struct { + name string + args args + want []*datapb.CompactionPlan + }{ + { + "test normal merge", + args{ + []*SegmentInfo{ + {SegmentInfo: &datapb.SegmentInfo{ID: 1, NumOfRows: 1, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 2, NumOfRows: 1, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}}}, + }, + &timetravel{1000}, + }, + []*datapb.CompactionPlan{ + { + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 1, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}, + {SegmentID: 2, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}}, + }, + + Type: datapb.CompactionType_MergeCompaction, + Timetravel: 1000, + }, + }, + }, + { + "test unmergable segments", + args{ + []*SegmentInfo{ + {SegmentInfo: &datapb.SegmentInfo{ID: 1, NumOfRows: 1, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 2, NumOfRows: 99, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 3, NumOfRows: 99, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}}}, + }, + &timetravel{1000}, + }, + []*datapb.CompactionPlan{ + { + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 1, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}, + {SegmentID: 2, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}}, + }, + Type: datapb.CompactionType_MergeCompaction, + Timetravel: 1000, + }, + }, + }, + { + "test multi plans", + args{ + []*SegmentInfo{ + {SegmentInfo: &datapb.SegmentInfo{ID: 1, NumOfRows: 50, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 2, NumOfRows: 50, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 3, NumOfRows: 50, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 4, NumOfRows: 50, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log4"}}}}}, + }, + &timetravel{1000}, + }, + []*datapb.CompactionPlan{ + { + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 1, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}, + {SegmentID: 2, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}}, + }, + Type: datapb.CompactionType_MergeCompaction, + Timetravel: 1000, + }, + { + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 3, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}}, + {SegmentID: 4, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log4"}}}}, + }, + Type: datapb.CompactionType_MergeCompaction, + Timetravel: 1000, + }, + }, + }, + { + "test empty plan", + args{ + []*SegmentInfo{ + {SegmentInfo: &datapb.SegmentInfo{ID: 1, NumOfRows: 50, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}}, + {SegmentInfo: &datapb.SegmentInfo{ID: 2, NumOfRows: 51, MaxRowNum: 100, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}}}, + }, + &timetravel{1000}, + }, + []*datapb.CompactionPlan{}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := greedyMergeCompaction(tt.args.segments, tt.args.timetravel) + assert.EqualValues(t, tt.want, got) + }) + } +} diff --git a/internal/datacoord/compaction_test.go b/internal/datacoord/compaction_test.go new file mode 100644 index 0000000000..17a43f8f25 --- /dev/null +++ b/internal/datacoord/compaction_test.go @@ -0,0 +1,367 @@ +package datacoord + +import ( + "sync" + "testing" + "time" + + memkv "github.com/milvus-io/milvus/internal/kv/mem" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/util/tsoutil" + "github.com/stretchr/testify/assert" +) + +// TODO not completed + +func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) { + ch := make(chan interface{}, 1) + type fields struct { + plans map[int64]*compactionTask + sessions *SessionManager + chManager *ChannelManager + } + type args struct { + plan *datapb.CompactionPlan + } + tests := []struct { + name string + fields fields + args args + wantErr bool + err error + }{ + { + "test exec compaction", + fields{ + plans: map[int64]*compactionTask{}, + sessions: &SessionManager{ + sessions: struct { + sync.RWMutex + data map[int64]*Session + }{ + data: map[int64]*Session{ + 1: {client: &mockDataNodeClient{ch: ch}}, + }, + }, + }, + chManager: &ChannelManager{ + store: &ChannelStore{ + channelsInfo: map[int64]*NodeChannelInfo{ + 1: {NodeID: 1, Channels: []*channel{{Name: "ch1"}}}, + }, + }, + }, + }, + args{ + plan: &datapb.CompactionPlan{PlanID: 1, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}, + }, + false, + nil, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := &compactionPlanHandler{ + plans: tt.fields.plans, + sessions: tt.fields.sessions, + chManager: tt.fields.chManager, + } + err := c.execCompactionPlan(tt.args.plan) + assert.Equal(t, tt.err, err) + if err == nil { + <-ch + task := c.getCompaction(tt.args.plan.PlanID) + assert.Equal(t, tt.args.plan, task.plan) + } + }) + } +} + +func Test_compactionPlanHandler_completeCompaction(t *testing.T) { + type fields struct { + plans map[int64]*compactionTask + sessions *SessionManager + meta *meta + flushCh chan UniqueID + } + type args struct { + result *datapb.CompactionResult + } + tests := []struct { + name string + fields fields + args args + wantErr bool + }{ + { + "test complete non existed compaction task", + fields{ + plans: map[int64]*compactionTask{1: {}}, + }, + args{ + result: &datapb.CompactionResult{PlanID: 2}, + }, + true, + }, + { + "test complete completed task", + fields{ + plans: map[int64]*compactionTask{1: {state: completed}}, + }, + args{ + result: &datapb.CompactionResult{PlanID: 1}, + }, + true, + }, + { + "test complete inner compaction", + fields{ + map[int64]*compactionTask{ + 1: { + state: executing, + plan: &datapb.CompactionPlan{ + PlanID: 1, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 1, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}, + }, + Type: datapb.CompactionType_InnerCompaction, + }, + }, + }, + nil, + &meta{ + client: memkv.NewMemoryKV(), + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: {SegmentInfo: &datapb.SegmentInfo{ID: 1, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}}, + }, + }, + }, + make(chan UniqueID, 1), + }, + args{ + result: &datapb.CompactionResult{ + PlanID: 1, + SegmentID: 1, + InsertLogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}, + }, + }, + false, + }, + { + "test complete merge compaction", + fields{ + map[int64]*compactionTask{ + 1: { + state: executing, + plan: &datapb.CompactionPlan{ + PlanID: 1, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 1, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}, + {SegmentID: 2, FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}}, + }, + Type: datapb.CompactionType_MergeCompaction, + }, + }, + }, + nil, + &meta{ + client: memkv.NewMemoryKV(), + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: {SegmentInfo: &datapb.SegmentInfo{ID: 1, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}}}, + 2: {SegmentInfo: &datapb.SegmentInfo{ID: 2, Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2"}}}}}, + }, + }, + }, + make(chan UniqueID, 1), + }, + args{ + result: &datapb.CompactionResult{ + PlanID: 1, + SegmentID: 3, + InsertLogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}, + }, + }, + false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := &compactionPlanHandler{ + plans: tt.fields.plans, + sessions: tt.fields.sessions, + meta: tt.fields.meta, + flushCh: tt.fields.flushCh, + } + err := c.completeCompaction(tt.args.result) + assert.Equal(t, tt.wantErr, err != nil) + }) + } +} + +func Test_compactionPlanHandler_getCompaction(t *testing.T) { + type fields struct { + plans map[int64]*compactionTask + sessions *SessionManager + } + type args struct { + planID int64 + } + tests := []struct { + name string + fields fields + args args + want *compactionTask + }{ + { + "test get non existed task", + fields{plans: map[int64]*compactionTask{}}, + args{planID: 1}, + nil, + }, + { + "test get existed task", + fields{ + plans: map[int64]*compactionTask{1: { + state: executing, + }}, + }, + args{planID: 1}, + &compactionTask{ + state: executing, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := &compactionPlanHandler{ + plans: tt.fields.plans, + sessions: tt.fields.sessions, + } + got := c.getCompaction(tt.args.planID) + assert.EqualValues(t, tt.want, got) + }) + } +} + +func Test_compactionPlanHandler_expireCompaction(t *testing.T) { + type fields struct { + plans map[int64]*compactionTask + sessions *SessionManager + meta *meta + } + type args struct { + ts Timestamp + } + + ts := time.Now() + tests := []struct { + name string + fields fields + args args + wantErr bool + expired []int64 + unexpired []int64 + }{ + { + "test expire compaction task", + fields{ + plans: map[int64]*compactionTask{ + 1: { + state: executing, + plan: &datapb.CompactionPlan{ + PlanID: 1, + StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0), + TimeoutInSeconds: 10, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + {SegmentID: 1}, + }, + }, + }, + 2: { + state: executing, + plan: &datapb.CompactionPlan{ + PlanID: 2, + StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0), + TimeoutInSeconds: 1, + }, + }, + }, + meta: &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: {SegmentInfo: &datapb.SegmentInfo{ID: 1}}, + }, + }, + }, + }, + args{ts: tsoutil.ComposeTS(ts.Add(5*time.Second).UnixNano()/int64(time.Millisecond), 0)}, + false, + []int64{2}, + []int64{1}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := &compactionPlanHandler{ + plans: tt.fields.plans, + sessions: tt.fields.sessions, + meta: tt.fields.meta, + } + + err := c.expireCompaction(tt.args.ts) + assert.Equal(t, tt.wantErr, err != nil) + + for _, id := range tt.expired { + task := c.getCompaction(id) + assert.Equal(t, timeout, task.state) + } + + for _, id := range tt.unexpired { + task := c.getCompaction(id) + assert.NotEqual(t, timeout, task.state) + } + }) + } +} + +func Test_newCompactionPlanHandler(t *testing.T) { + type args struct { + sessions *SessionManager + cm *ChannelManager + meta *meta + allocator allocator + flush chan UniqueID + } + tests := []struct { + name string + args args + want *compactionPlanHandler + }{ + { + "test new handler", + args{ + &SessionManager{}, + &ChannelManager{}, + &meta{}, + newMockAllocator(), + nil, + }, + &compactionPlanHandler{ + plans: map[int64]*compactionTask{}, + sessions: &SessionManager{}, + chManager: &ChannelManager{}, + meta: &meta{}, + allocator: newMockAllocator(), + flushCh: nil, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := newCompactionPlanHandler(tt.args.sessions, tt.args.cm, tt.args.meta, tt.args.allocator, tt.args.flush) + assert.EqualValues(t, tt.want, got) + }) + } +} diff --git a/internal/datacoord/compaction_trigger.go b/internal/datacoord/compaction_trigger.go new file mode 100644 index 0000000000..f9d0849746 --- /dev/null +++ b/internal/datacoord/compaction_trigger.go @@ -0,0 +1,436 @@ +package datacoord + +import ( + "context" + "sync" + "time" + + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/logutil" + "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/proto/datapb" + "go.uber.org/zap" +) + +const ( + signalBufferSize = 100 + maxLittleSegmentNum = 10 + maxCompactionTimeoutInSeconds = 60 + singleCompactionRatioThreshold = 0.2 + singleCompactionDeltaLogMaxSize = 10 * 1024 * 1024 //10MiB + globalCompactionInterval = 60 * time.Second + timetravelRange = 5 * 24 * time.Hour +) + +type timetravel struct { + time Timestamp +} + +type trigger interface { + start() + stop() + // triggerCompaction trigger a compaction if any compaction condition satisfy. + triggerCompaction(timetravel *timetravel) error + // triggerSingleCompaction trigerr a compaction bundled with collection-partiiton-channel-segment + triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string, timetravel *timetravel) error + // forceTriggerCompaction force to start a compaction + forceTriggerCompaction(collectionID int64, timetravel *timetravel) (UniqueID, error) +} + +type compactionSignal struct { + id UniqueID + isForce bool + isGlobal bool + collectionID UniqueID + partitionID UniqueID + segmentID UniqueID + channel string + timetravel *timetravel +} + +var _ trigger = (*compactionTrigger)(nil) + +type compactionTrigger struct { + meta *meta + allocator allocator + signals chan *compactionSignal + singleCompactionPolicy singleCompactionPolicy + mergeCompactionPolicy mergeCompactionPolicy + compactionHandler compactionPlanContext + globalTrigger *time.Ticker + forceMu sync.Mutex + mergeCompactionSegmentThreshold int + quit chan struct{} + wg sync.WaitGroup +} + +func newCompactionTrigger(meta *meta, compactionHandler compactionPlanContext, allocator allocator) *compactionTrigger { + return &compactionTrigger{ + meta: meta, + allocator: allocator, + signals: make(chan *compactionSignal, signalBufferSize), + singleCompactionPolicy: (singleCompactionFunc)(chooseAllBinlogs), + mergeCompactionPolicy: (mergeCompactionFunc)(greedyMergeCompaction), + compactionHandler: compactionHandler, + mergeCompactionSegmentThreshold: maxLittleSegmentNum, + } +} + +func (t *compactionTrigger) start() { + t.quit = make(chan struct{}) + t.globalTrigger = time.NewTicker(globalCompactionInterval) + t.wg.Add(2) + go func() { + defer logutil.LogPanic() + defer t.wg.Done() + + for { + select { + case <-t.quit: + log.Debug("compaction trigger quit") + return + case signal := <-t.signals: + switch { + case signal.isGlobal: + t.handleGlobalSignal(signal) + default: + t.handleSignal(signal) + t.globalTrigger.Reset(globalCompactionInterval) + } + } + } + }() + + go t.startGlobalCompactionLoop() +} + +func (t *compactionTrigger) startGlobalCompactionLoop() { + defer logutil.LogPanic() + defer t.wg.Done() + + for { + select { + case <-t.quit: + t.globalTrigger.Stop() + log.Info("global compaction loop exit") + return + case <-t.globalTrigger.C: + cctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + tt, err := getTimetravel(cctx, t.allocator) + if err != nil { + log.Warn("unbale to get compaction timetravel", zap.Error(err)) + cancel() + continue + } + cancel() + t.triggerCompaction(tt) + } + } +} + +func (t *compactionTrigger) stop() { + close(t.quit) + t.wg.Wait() +} + +// triggerCompaction trigger a compaction if any compaction condition satisfy. +func (t *compactionTrigger) triggerCompaction(timetravel *timetravel) error { + id, err := t.allocSignalID() + if err != nil { + return err + } + signal := &compactionSignal{ + id: id, + isForce: false, + isGlobal: true, + timetravel: timetravel, + } + t.signals <- signal + return nil +} + +// triggerSingleCompaction triger a compaction bundled with collection-partiiton-channel-segment +func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string, timetravel *timetravel) error { + id, err := t.allocSignalID() + if err != nil { + return err + } + signal := &compactionSignal{ + id: id, + isForce: false, + isGlobal: false, + collectionID: collectionID, + partitionID: partitionID, + segmentID: segmentID, + channel: channel, + timetravel: timetravel, + } + t.signals <- signal + return nil +} + +// forceTriggerCompaction force to start a compaction +func (t *compactionTrigger) forceTriggerCompaction(collectionID int64, timetravel *timetravel) (UniqueID, error) { + id, err := t.allocSignalID() + if err != nil { + return -1, err + } + signal := &compactionSignal{ + id: id, + isForce: true, + isGlobal: false, + collectionID: collectionID, + timetravel: timetravel, + } + t.handleForceSignal(signal) + return id, nil +} + +func (t *compactionTrigger) allocSignalID() (UniqueID, error) { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + return t.allocator.allocID(ctx) +} + +func (t *compactionTrigger) handleForceSignal(signal *compactionSignal) { + t.forceMu.Lock() + defer t.forceMu.Unlock() + + t1 := time.Now() + + segments := t.meta.GetSegmentsOfCollection(signal.collectionID) + singleCompactionPlans := t.globalSingleCompaction(segments, true, signal.timetravel) + if len(singleCompactionPlans) != 0 { + log.Debug("force single compaction plans", zap.Int64("signalID", signal.id), zap.Int64s("planIDs", getPlanIDs(singleCompactionPlans))) + } + + mergeCompactionPlans := t.globalMergeCompaction(signal.timetravel, true, signal.collectionID) + if len(mergeCompactionPlans) != 0 { + log.Debug("force merge compaction plans", zap.Int64("signalID", signal.id), zap.Int64s("planIDs", getPlanIDs(mergeCompactionPlans))) + } + log.Info("handle force signal cost", zap.Int64("milliseconds", time.Since(t1).Milliseconds()), + zap.Int64("collectionID", signal.collectionID), zap.Int64("signalID", signal.id)) +} + +func getPlanIDs(plans []*datapb.CompactionPlan) []int64 { + ids := make([]int64, 0, len(plans)) + for _, p := range plans { + ids = append(ids, p.GetPlanID()) + } + return ids +} + +func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) { + t.forceMu.Lock() + defer t.forceMu.Unlock() + + // 1. try global single compaction + t1 := time.Now() + if t.compactionHandler.isFull() { + return + } + segments := t.meta.segments.GetSegments() + singleCompactionPlans := t.globalSingleCompaction(segments, false, signal.timetravel) + if len(singleCompactionPlans) != 0 { + log.Debug("global single compaction plans", zap.Int64("signalID", signal.id), zap.Int64s("plans", getPlanIDs(singleCompactionPlans))) + } + + // 2. try global merge compaction + if t.compactionHandler.isFull() { + return + } + + mergeCompactionPlans := t.globalMergeCompaction(signal.timetravel, false) + if len(mergeCompactionPlans) != 0 { + log.Debug("global merge compaction plans", zap.Int64("signalID", signal.id), zap.Int64s("plans", getPlanIDs(mergeCompactionPlans))) + } + + log.Info("handle glonbal compaction cost", zap.Int64("millliseconds", time.Since(t1).Milliseconds())) +} + +func (t *compactionTrigger) handleSignal(signal *compactionSignal) { + t.forceMu.Lock() + defer t.forceMu.Unlock() + + t1 := time.Now() + // 1. check whether segment's binlogs should be compacted or not + if t.compactionHandler.isFull() { + return + } + + segment := t.meta.GetSegment(signal.segmentID) + singleCompactionPlan, err := t.singleCompaction(segment, signal.isForce, signal.timetravel) + if err != nil { + log.Warn("failed to do single compaction", zap.Int64("segmentID", segment.ID), zap.Error(err)) + } else { + log.Info("time cost of generating single compaction plan", zap.Int64("milllis", time.Since(t1).Milliseconds()), + zap.Int64("planID", singleCompactionPlan.GetPlanID()), zap.Int64("signalID", signal.id)) + } + + // 2. check whether segments of partition&channel level should be compacted or not + if t.compactionHandler.isFull() { + return + } + + channel := segment.GetInsertChannel() + partitionID := segment.GetPartitionID() + + segments := t.getCandidateSegments(channel, partitionID) + + plans := t.mergeCompaction(segments, signal.timetravel, false) + if len(plans) != 0 { + log.Debug("merge compaction plans", zap.Int64("signalID", signal.id), zap.Int64s("plans", getPlanIDs(plans))) + } + + // log.Info("time cost of generating merge compaction", zap.Int64("planID", plan.PlanID), zap.Any("time cost", time.Since(t1).Milliseconds()), + // zap.String("channel", channel), zap.Int64("partitionID", partitionID)) +} + +func (t *compactionTrigger) globalMergeCompaction(timetravel *timetravel, isForce bool, collections ...UniqueID) []*datapb.CompactionPlan { + colls := make(map[int64]struct{}) + for _, collID := range collections { + colls[collID] = struct{}{} + } + m := t.meta.GetSegmentsChanPart(func(segment *SegmentInfo) bool { + _, has := colls[segment.GetCollectionID()] + return (has || len(collections) == 0) && // if filters collection + segment.State == commonpb.SegmentState_Flushed && // flushed only + !segment.isCompacting // not compacting now + }) // m is list of chanPartSegments, which is channel-partition organized segments + plans := make([]*datapb.CompactionPlan, 0) + for _, segments := range m { + if !isForce && t.compactionHandler.isFull() { + return plans + } + mplans := t.mergeCompaction(segments.segments, timetravel, isForce) + plans = append(plans, mplans...) + } + + return plans +} + +func (t *compactionTrigger) mergeCompaction(segments []*SegmentInfo, timetravel *timetravel, isForce bool) []*datapb.CompactionPlan { + if !isForce && !t.shouldDoMergeCompaction(segments) { + return nil + } + + plans := t.mergeCompactionPolicy.generatePlan(segments, timetravel) + if len(plans) == 0 { + return nil + } + + res := make([]*datapb.CompactionPlan, 0, len(plans)) + for _, plan := range plans { + if !isForce && t.compactionHandler.isFull() { + return nil + } + + if err := t.fillOriginPlan(plan); err != nil { + log.Warn("failed to fill plan", zap.Error(err)) + continue + } + + log.Debug("exec merge compaction plan", zap.Any("plan", plan)) + if err := t.compactionHandler.execCompactionPlan(plan); err != nil { + log.Warn("failed to execute compaction plan", zap.Error(err)) + continue + } + res = append(res, plan) + } + return res +} + +func (t *compactionTrigger) getCandidateSegments(channel string, partitionID UniqueID) []*SegmentInfo { + segments := t.meta.GetSegmentsByChannel(channel) + res := make([]*SegmentInfo, 0) + for _, s := range segments { + if s.GetState() != commonpb.SegmentState_Flushed || s.GetInsertChannel() != channel || + s.GetPartitionID() != partitionID || s.isCompacting { + continue + } + res = append(res, s) + } + return res +} + +func (t *compactionTrigger) shouldDoMergeCompaction(segments []*SegmentInfo) bool { + littleSegmentNum := 0 + for _, s := range segments { + if s.GetNumOfRows() < s.GetMaxRowNum()/2 { + littleSegmentNum++ + } + } + return littleSegmentNum >= t.mergeCompactionSegmentThreshold +} + +func (t *compactionTrigger) fillOriginPlan(plan *datapb.CompactionPlan) error { + // TODO context + id, err := t.allocator.allocID(context.Background()) + if err != nil { + return err + } + plan.PlanID = id + plan.TimeoutInSeconds = maxCompactionTimeoutInSeconds + return nil +} + +func (t *compactionTrigger) shouldDoSingleCompaction(segment *SegmentInfo, timetravel *timetravel) bool { + // single compaction only merge insert and delta log beyond the timetravel + // segment's insert binlogs dont have time range info, so we wait until the segment's last expire time is less than timetravel + // to ensure that all insert logs is beyond the timetravel. + // TODO: add meta in insert binlog + if segment.LastExpireTime >= timetravel.time { + return false + } + + totalDeletedRows := 0 + totalDeleteLogSize := int64(0) + for _, l := range segment.GetDeltalogs() { + if l.TimestampTo < timetravel.time { + totalDeletedRows += int(l.GetRecordEntries()) + totalDeleteLogSize += l.GetDeltaLogSize() + } + } + + // currently delta log size and delete ratio policy is applied + return float32(totalDeletedRows)/float32(segment.NumOfRows) >= singleCompactionRatioThreshold || totalDeleteLogSize > singleCompactionDeltaLogMaxSize +} + +func (t *compactionTrigger) globalSingleCompaction(segments []*SegmentInfo, isForce bool, timetravel *timetravel) []*datapb.CompactionPlan { + plans := make([]*datapb.CompactionPlan, 0) + for _, segment := range segments { + if !isForce && t.compactionHandler.isFull() { + return plans + } + plan, err := t.singleCompaction(segment, isForce, timetravel) + if err != nil { + log.Warn("failed to exec single compaction", zap.Error(err)) + continue + } + if plan != nil { + plans = append(plans, plan) + log.Debug("exec single compaction plan", zap.Any("plan", plan)) + } + } + return plans +} + +func (t *compactionTrigger) singleCompaction(segment *SegmentInfo, isForce bool, timetravel *timetravel) (*datapb.CompactionPlan, error) { + if segment == nil { + return nil, nil + } + + if !t.shouldDoSingleCompaction(segment, timetravel) { + return nil, nil + } + + plan := t.singleCompactionPolicy.generatePlan(segment, timetravel) + if plan == nil { + return nil, nil + } + + if err := t.fillOriginPlan(plan); err != nil { + return nil, err + } + return plan, t.compactionHandler.execCompactionPlan(plan) +} diff --git a/internal/datacoord/compaction_trigger_test.go b/internal/datacoord/compaction_trigger_test.go new file mode 100644 index 0000000000..5bc684dfac --- /dev/null +++ b/internal/datacoord/compaction_trigger_test.go @@ -0,0 +1,757 @@ +package datacoord + +import ( + "context" + "sort" + "testing" + "time" + + "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/stretchr/testify/assert" +) + +type spyCompactionHandler struct { + spyChan chan *datapb.CompactionPlan +} + +// execCompactionPlan start to execute plan and return immediately +func (h *spyCompactionHandler) execCompactionPlan(plan *datapb.CompactionPlan) error { + h.spyChan <- plan + return nil +} + +// completeCompaction record the result of a compaction +func (h *spyCompactionHandler) completeCompaction(result *datapb.CompactionResult) error { + return nil +} + +// getCompaction return compaction task. If planId does not exist, return nil. +func (h *spyCompactionHandler) getCompaction(planID int64) *compactionTask { + panic("not implemented") // TODO: Implement +} + +// expireCompaction set the compaction state to expired +func (h *spyCompactionHandler) expireCompaction(ts Timestamp) error { + panic("not implemented") // TODO: Implement +} + +// isFull return true if the task pool is full +func (h *spyCompactionHandler) isFull() bool { + return false +} + +// get compaction by signal id and return the number of executing/completed/timeout plans +func (h *spyCompactionHandler) getCompactionBySignalID(signalID int64) (executing int, completed int, timeout int) { + panic("not implemented") // TODO: Implement +} + +func (h *spyCompactionHandler) start() {} + +func (h *spyCompactionHandler) stop() {} + +var _ compactionPlanContext = (*spyCompactionHandler)(nil) + +func Test_compactionTrigger_forceTriggerCompaction(t *testing.T) { + type fields struct { + meta *meta + allocator allocator + signals chan *compactionSignal + singleCompactionPolicy singleCompactionPolicy + mergeCompactionPolicy mergeCompactionPolicy + compactionHandler compactionPlanContext + globalTrigger *time.Ticker + } + type args struct { + collectionID int64 + timetravel *timetravel + } + tests := []struct { + name string + fields fields + args args + wantErr bool + wantPlans []*datapb.CompactionPlan + }{ + { + "test only merge compaction", + fields{ + &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: 2, + PartitionID: 1, + LastExpireTime: 100, + NumOfRows: 100, + MaxRowNum: 300, + InsertChannel: "ch1", + State: commonpb.SegmentState_Flushed, + Binlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"log1"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog1"}, + }, + }, + }, + 2: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + CollectionID: 2, + PartitionID: 1, + LastExpireTime: 100, + NumOfRows: 100, + MaxRowNum: 300, + InsertChannel: "ch1", + State: commonpb.SegmentState_Flushed, + Binlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"log2"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog2"}, + }, + }, + }, + }, + }, + }, + newMockAllocator(), + nil, + (singleCompactionFunc)(chooseAllBinlogs), + (mergeCompactionFunc)(greedyGeneratePlans), + &spyCompactionHandler{spyChan: make(chan *datapb.CompactionPlan, 1)}, + nil, + }, + args{ + 2, + &timetravel{time: 200}, + }, + false, + []*datapb.CompactionPlan{ + { + PlanID: 2, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 1, + FieldBinlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"log1"}}, + }, + Field2StatslogPaths: nil, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog1"}, + }, + }, + { + SegmentID: 2, + FieldBinlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"log2"}}, + }, + Field2StatslogPaths: nil, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog2"}, + }, + }, + }, + TimeoutInSeconds: maxCompactionTimeoutInSeconds, + Type: datapb.CompactionType_MergeCompaction, + Timetravel: 200, + Channel: "ch1", + }, + }, + }, + { + "test only single compaction", + fields{ + &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: 2, + LastExpireTime: 100, + NumOfRows: 10, + InsertChannel: "ch1", + State: commonpb.SegmentState_Flushed, + Binlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"log1"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5}, + }, + }, + }, + }, + }, + }, + newMockAllocator(), + nil, + (singleCompactionFunc)(chooseAllBinlogs), + (mergeCompactionFunc)(greedyGeneratePlans), + &spyCompactionHandler{spyChan: make(chan *datapb.CompactionPlan, 1)}, + nil, + }, + args{ + 2, + &timetravel{time: 200}, + }, + false, + []*datapb.CompactionPlan{ + { + PlanID: 2, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 1, + FieldBinlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"log1"}}, + }, + Field2StatslogPaths: nil, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5}, + }, + }, + }, + TimeoutInSeconds: maxCompactionTimeoutInSeconds, + Type: datapb.CompactionType_InnerCompaction, + Timetravel: 200, + Channel: "ch1", + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tr := &compactionTrigger{ + meta: tt.fields.meta, + allocator: tt.fields.allocator, + signals: tt.fields.signals, + singleCompactionPolicy: tt.fields.singleCompactionPolicy, + mergeCompactionPolicy: tt.fields.mergeCompactionPolicy, + compactionHandler: tt.fields.compactionHandler, + globalTrigger: tt.fields.globalTrigger, + } + _, err := tr.forceTriggerCompaction(tt.args.collectionID, tt.args.timetravel) + assert.Equal(t, tt.wantErr, err != nil) + spy := (tt.fields.compactionHandler).(*spyCompactionHandler) + plan := <-spy.spyChan + sortPlanCompactionBinlogs(plan) + assert.EqualValues(t, tt.wantPlans[0], plan) + }) + } +} + +func sortPlanCompactionBinlogs(plan *datapb.CompactionPlan) { + sort.Slice(plan.SegmentBinlogs, func(i, j int) bool { + return plan.SegmentBinlogs[i].SegmentID < plan.SegmentBinlogs[j].SegmentID + }) +} + +func Test_compactionTrigger_triggerCompaction(t *testing.T) { + type fields struct { + meta *meta + allocator allocator + signals chan *compactionSignal + singleCompactionPolicy singleCompactionPolicy + mergeCompactionPolicy mergeCompactionPolicy + compactionHandler compactionPlanContext + mergeCompactionSegmentThreshold int + } + type args struct { + timetravel *timetravel + } + tests := []struct { + name string + fields fields + args args + wantErr bool + wantPlans []*datapb.CompactionPlan + }{ + { + "test normal case", + fields{ + &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: 1, + PartitionID: 1, + LastExpireTime: 100, + NumOfRows: 10, + MaxRowNum: 100, + InsertChannel: "ch1", + State: commonpb.SegmentState_Flushed, + Binlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"binlog1"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog1"}, + }, + }, + }, + 2: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + CollectionID: 2, + PartitionID: 1, + LastExpireTime: 100, + NumOfRows: 300, + MaxRowNum: 1000, + InsertChannel: "ch2", + State: commonpb.SegmentState_Flushed, + Binlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"binlog2"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog2"}, + }, + }, + }, + 3: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 3, + CollectionID: 2, + PartitionID: 1, + LastExpireTime: 100, + NumOfRows: 300, + MaxRowNum: 1000, + InsertChannel: "ch2", + State: commonpb.SegmentState_Flushed, + Binlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"binlog3"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog3"}, + }, + }, + }, + }, + }, + }, + newMockAllocator(), + make(chan *compactionSignal, 1), + (singleCompactionFunc)(chooseAllBinlogs), + (mergeCompactionFunc)(greedyMergeCompaction), + &spyCompactionHandler{spyChan: make(chan *datapb.CompactionPlan, 2)}, + 2, + }, + args{ + &timetravel{200}, + }, + false, + []*datapb.CompactionPlan{ + { + PlanID: 2, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 1, + FieldBinlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"binlog1"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog1"}, + }, + }, + }, + TimeoutInSeconds: maxCompactionTimeoutInSeconds, + Type: datapb.CompactionType_InnerCompaction, + Timetravel: 200, + Channel: "ch1", + }, + { + PlanID: 3, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 2, + FieldBinlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"binlog2"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog2"}, + }, + }, + { + SegmentID: 3, + FieldBinlogs: []*datapb.FieldBinlog{ + {FieldID: 1, Binlogs: []string{"binlog3"}}, + }, + Deltalogs: []*datapb.DeltaLogInfo{ + {RecordEntries: 5, DeltaLogPath: "deltalog3"}, + }, + }, + }, + TimeoutInSeconds: maxCompactionTimeoutInSeconds, + Type: datapb.CompactionType_MergeCompaction, + Timetravel: 200, + Channel: "ch2", + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tr := &compactionTrigger{ + meta: tt.fields.meta, + allocator: tt.fields.allocator, + signals: tt.fields.signals, + singleCompactionPolicy: tt.fields.singleCompactionPolicy, + mergeCompactionPolicy: tt.fields.mergeCompactionPolicy, + compactionHandler: tt.fields.compactionHandler, + mergeCompactionSegmentThreshold: tt.fields.mergeCompactionSegmentThreshold, + } + tr.start() + defer tr.stop() + err := tr.triggerCompaction(tt.args.timetravel) + assert.Equal(t, tt.wantErr, err != nil) + spy := tt.fields.compactionHandler.(*spyCompactionHandler) + gotPlans := make([]*datapb.CompactionPlan, 0, len(tt.wantPlans)) + for i := 0; i < len(tt.wantPlans); i++ { + plan := <-spy.spyChan + gotPlans = append(gotPlans, plan) + } + for _, plan := range gotPlans { + sortPlanCompactionBinlogs(plan) + } + assert.EqualValues(t, tt.wantPlans, gotPlans) + }) + } +} + +func Test_compactionTrigger_singleTriggerCompaction(t *testing.T) { + type fields struct { + meta *meta + allocator allocator + signals chan *compactionSignal + singleCompactionPolicy singleCompactionPolicy + mergeCompactionPolicy mergeCompactionPolicy + compactionHandler compactionPlanContext + globalTrigger *time.Ticker + } + type args struct { + collectionID int64 + partitionID int64 + segmentID int64 + channelName string + timetravel *timetravel + } + tests := []struct { + name string + fields fields + args args + wantErr bool + wantPlan bool + wantBinlogs []*datapb.CompactionSegmentBinlogs + }{ + { + name: "normal single flush", + fields: fields{ + meta: &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 101: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 101, + CollectionID: 1, + PartitionID: 10, + InsertChannel: "test_chan_01", + NumOfRows: 10000, + State: commonpb.SegmentState_Flushed, + MaxRowNum: 12000, + LastExpireTime: 50, + StartPosition: &internalpb.MsgPosition{ + ChannelName: "", + MsgID: []byte{}, + MsgGroup: "", + Timestamp: 10, + }, + DmlPosition: &internalpb.MsgPosition{ + ChannelName: "", + MsgID: []byte{}, + MsgGroup: "", + Timestamp: 45, + }, + Binlogs: []*datapb.FieldBinlog{}, + Statslogs: []*datapb.FieldBinlog{}, + Deltalogs: []*datapb.DeltaLogInfo{ + { + RecordEntries: 2001, + }, + }, + CreatedByCompaction: false, + CompactionFrom: []int64{}, + }, + isCompacting: false, + }, + }, + }, + }, + allocator: newMockAllocator(), + signals: make(chan *compactionSignal, 1), + singleCompactionPolicy: (singleCompactionFunc)(chooseAllBinlogs), + mergeCompactionPolicy: (mergeCompactionFunc)(greedyGeneratePlans), + compactionHandler: &spyCompactionHandler{ + spyChan: make(chan *datapb.CompactionPlan, 1), + }, + globalTrigger: time.NewTicker(time.Hour), + }, + args: args{ + collectionID: 1, + partitionID: 10, + segmentID: 101, + channelName: "test_ch_01", + timetravel: &timetravel{ + time: 100, + }, + }, + wantErr: false, + wantPlan: true, + wantBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 101, + FieldBinlogs: []*datapb.FieldBinlog{}, + Field2StatslogPaths: []*datapb.FieldBinlog{}, + Deltalogs: []*datapb.DeltaLogInfo{ + { + RecordEntries: 2001, + }, + }, + }, + }, + }, + { + name: "part delta out of range", + fields: fields{ + meta: &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 101: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 101, + CollectionID: 1, + PartitionID: 10, + InsertChannel: "test_chan_01", + NumOfRows: 10000, + State: commonpb.SegmentState_Flushed, + MaxRowNum: 12000, + LastExpireTime: 100, + StartPosition: &internalpb.MsgPosition{ + ChannelName: "", + MsgID: []byte{}, + MsgGroup: "", + Timestamp: 10, + }, + DmlPosition: &internalpb.MsgPosition{ + ChannelName: "", + MsgID: []byte{}, + MsgGroup: "", + Timestamp: 45, + }, + Binlogs: []*datapb.FieldBinlog{}, + Statslogs: []*datapb.FieldBinlog{}, + Deltalogs: []*datapb.DeltaLogInfo{ + { + RecordEntries: 1000, + TimestampFrom: 10, + TimestampTo: 20, + }, + { + RecordEntries: 1001, + TimestampFrom: 30, + TimestampTo: 45, + }, + }, + CreatedByCompaction: false, + CompactionFrom: []int64{}, + }, + isCompacting: false, + }, + }, + }, + }, + allocator: newMockAllocator(), + signals: make(chan *compactionSignal, 1), + singleCompactionPolicy: (singleCompactionFunc)(chooseAllBinlogs), + mergeCompactionPolicy: (mergeCompactionFunc)(greedyGeneratePlans), + compactionHandler: &spyCompactionHandler{ + spyChan: make(chan *datapb.CompactionPlan, 1), + }, + globalTrigger: time.NewTicker(time.Hour), + }, + args: args{ + collectionID: 1, + partitionID: 10, + segmentID: 101, + channelName: "test_ch_01", + timetravel: &timetravel{ + time: 30, + }, + }, + wantErr: false, + wantPlan: false, + wantBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 101, + FieldBinlogs: []*datapb.FieldBinlog{}, + Field2StatslogPaths: []*datapb.FieldBinlog{}, + Deltalogs: []*datapb.DeltaLogInfo{ + { + RecordEntries: 2001, + }, + }, + }, + }, + }, + { + name: "delte log size", + fields: fields{ + meta: &meta{ + segments: &SegmentsInfo{ + map[int64]*SegmentInfo{ + 101: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 101, + CollectionID: 1, + PartitionID: 10, + InsertChannel: "test_chan_01", + NumOfRows: 10000, + State: commonpb.SegmentState_Flushed, + MaxRowNum: 12000, + LastExpireTime: 100, + StartPosition: &internalpb.MsgPosition{ + ChannelName: "", + MsgID: []byte{}, + MsgGroup: "", + Timestamp: 10, + }, + DmlPosition: &internalpb.MsgPosition{ + ChannelName: "", + MsgID: []byte{}, + MsgGroup: "", + Timestamp: 45, + }, + Binlogs: []*datapb.FieldBinlog{}, + Statslogs: []*datapb.FieldBinlog{}, + Deltalogs: []*datapb.DeltaLogInfo{ + { + RecordEntries: 1000, + TimestampFrom: 10, + TimestampTo: 20, + DeltaLogSize: 10*1024*1024 + 1, + }, + }, + CreatedByCompaction: false, + CompactionFrom: []int64{}, + }, + isCompacting: false, + }, + }, + }, + }, + allocator: newMockAllocator(), + signals: make(chan *compactionSignal, 1), + singleCompactionPolicy: (singleCompactionFunc)(chooseAllBinlogs), + mergeCompactionPolicy: (mergeCompactionFunc)(greedyGeneratePlans), + compactionHandler: &spyCompactionHandler{ + spyChan: make(chan *datapb.CompactionPlan, 1), + }, + globalTrigger: time.NewTicker(time.Hour), + }, + args: args{ + collectionID: 1, + partitionID: 10, + segmentID: 101, + channelName: "test_ch_01", + timetravel: &timetravel{ + time: 120, + }, + }, + wantErr: false, + wantPlan: true, + wantBinlogs: []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 101, + FieldBinlogs: []*datapb.FieldBinlog{}, + Field2StatslogPaths: []*datapb.FieldBinlog{}, + Deltalogs: []*datapb.DeltaLogInfo{ + { + RecordEntries: 1000, + TimestampFrom: 10, + TimestampTo: 20, + DeltaLogSize: 10*1024*1024 + 1, + }, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tr := &compactionTrigger{ + meta: tt.fields.meta, + allocator: tt.fields.allocator, + signals: tt.fields.signals, + singleCompactionPolicy: tt.fields.singleCompactionPolicy, + mergeCompactionPolicy: tt.fields.mergeCompactionPolicy, + compactionHandler: tt.fields.compactionHandler, + globalTrigger: tt.fields.globalTrigger, + } + tr.start() + defer tr.stop() + + err := tr.triggerSingleCompaction(tt.args.collectionID, tt.args.partitionID, + tt.args.segmentID, tt.args.channelName, tt.args.timetravel) + assert.Equal(t, tt.wantErr, err != nil) + spy := (tt.fields.compactionHandler).(*spyCompactionHandler) + ctx, cancel := context.WithTimeout(context.TODO(), time.Millisecond*50) + select { + case plan := <-spy.spyChan: + if tt.wantPlan { + assert.EqualValues(t, tt.wantBinlogs, plan.GetSegmentBinlogs()) + } else { + t.Fail() + } + case <-ctx.Done(): + if tt.wantPlan { + t.Fail() + } + } + cancel() + }) + } +} + +func Test_newCompactionTrigger(t *testing.T) { + type args struct { + meta *meta + compactionHandler compactionPlanContext + allocator allocator + } + tests := []struct { + name string + args args + }{ + { + "test new trigger", + args{ + &meta{}, + &compactionPlanHandler{}, + newMockAllocator(), + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := newCompactionTrigger(tt.args.meta, tt.args.compactionHandler, tt.args.allocator) + assert.Equal(t, tt.args.meta, got.meta) + assert.Equal(t, tt.args.compactionHandler, got.compactionHandler) + assert.Equal(t, tt.args.allocator, got.allocator) + }) + } +} diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 8ee718d5f5..5262076b6d 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/querypb" ) @@ -72,6 +73,9 @@ func (m *meta) reloadFromKV() error { if err != nil { return fmt.Errorf("DataCoord reloadFromKV UnMarshal datapb.SegmentInfo err:%w", err) } + if segmentInfo.State == commonpb.SegmentState_NotExist { + continue + } m.segments.SetSegment(segmentInfo.GetID(), NewSegmentInfo(segmentInfo)) } @@ -97,6 +101,55 @@ func (m *meta) GetCollection(collectionID UniqueID) *datapb.CollectionInfo { return collection } +// GetCollections get all collections id from local cache +func (m *meta) GetCollectionsID() []UniqueID { + m.RLock() + defer m.RUnlock() + + res := make([]UniqueID, 0, len(m.collections)) + for _, c := range m.collections { + res = append(res, c.GetID()) + } + return res +} + +type chanPartSegments struct { + collecionID UniqueID + partitionID UniqueID + channelName string + segments []*SegmentInfo +} + +// GetSegmentsChanPart get segments organized in Channel-Parition dimension with selector applied +func (m *meta) GetSegmentsChanPart(selector SegmentInfoSelector) []*chanPartSegments { + m.RLock() + defer m.RUnlock() + mDimEntry := make(map[string]*chanPartSegments) + + for _, segmentInfo := range m.segments.segments { + if !selector(segmentInfo) { + continue + } + dim := fmt.Sprintf("%d-%s", segmentInfo.PartitionID, segmentInfo.InsertChannel) + entry, ok := mDimEntry[dim] + if !ok { + entry = &chanPartSegments{ + collecionID: segmentInfo.CollectionID, + partitionID: segmentInfo.PartitionID, + channelName: segmentInfo.InsertChannel, + } + mDimEntry[dim] = entry + } + entry.segments = append(entry.segments, segmentInfo) + } + + result := make([]*chanPartSegments, 0, len(mDimEntry)) + for _, entry := range mDimEntry { + result = append(result, entry) + } + return result +} + // GetNumRowsOfCollection returns total rows count of segments belongs to provided collection func (m *meta) GetNumRowsOfCollection(collectionID UniqueID) int64 { m.RLock() @@ -310,8 +363,19 @@ func (m *meta) GetSegmentsByChannel(dmlCh string) []*SegmentInfo { return infos } -// GetSegmentsOfCollection returns all segment ids which collection equals to provided `collectionID` -func (m *meta) GetSegmentsOfCollection(collectionID UniqueID) []UniqueID { +// GetSegmentsOfCollection get all segments of collection +func (m *meta) GetSegmentsOfCollection(collectionID UniqueID) []*SegmentInfo { + m.RLock() + defer m.RUnlock() + + ret := make([]*SegmentInfo, 0) + segments := m.segments.GetSegments() + ret = append(ret, segments...) + return ret +} + +// GetSegmentsIDOfCollection returns all segment ids which collection equals to provided `collectionID` +func (m *meta) GetSegmentsIDOfCollection(collectionID UniqueID) []UniqueID { m.RLock() defer m.RUnlock() ret := make([]UniqueID, 0) @@ -324,8 +388,8 @@ func (m *meta) GetSegmentsOfCollection(collectionID UniqueID) []UniqueID { return ret } -// GetSegmentsOfPartition returns all segments ids which collection & partition equals to provided `collectionID`, `partitionID` -func (m *meta) GetSegmentsOfPartition(collectionID, partitionID UniqueID) []UniqueID { +// GetSegmentsIDOfPartition returns all segments ids which collection & partition equals to provided `collectionID`, `partitionID` +func (m *meta) GetSegmentsIDOfPartition(collectionID, partitionID UniqueID) []UniqueID { m.RLock() defer m.RUnlock() ret := make([]UniqueID, 0) @@ -429,6 +493,14 @@ func (m *meta) SetLastFlushTime(segmentID UniqueID, t time.Time) { m.segments.SetFlushTime(segmentID, t) } +// SetSegmentCompacting sets compaction state for segment +func (m *meta) SetSegmentCompacting(segmentID UniqueID, compacting bool) { + m.Lock() + defer m.Unlock() + + m.segments.SetIsCompacting(segmentID, compacting) +} + // MoveSegmentBinlogs migration logic, moving segment binlong information for legacy keys func (m *meta) MoveSegmentBinlogs(segmentID UniqueID, oldPathPrefix string, field2Binlogs map[UniqueID][]string) error { m.Lock() @@ -451,6 +523,187 @@ func (m *meta) MoveSegmentBinlogs(segmentID UniqueID, oldPathPrefix string, fiel return m.client.MultiSaveAndRemoveWithPrefix(kv, removals) } +func (m *meta) CompleteMergeCompaction(compactionLogs []*datapb.CompactionSegmentBinlogs, result *datapb.CompactionResult) error { + m.Lock() + defer m.Unlock() + + segments := make([]*SegmentInfo, 0, len(compactionLogs)) + for _, cl := range compactionLogs { + if segment := m.segments.GetSegment(cl.GetSegmentID()); segment != nil { + cloned := segment.Clone() + cloned.State = commonpb.SegmentState_NotExist + segments = append(segments, cloned) + } + } + + var dmlPosition *internalpb.MsgPosition + for _, s := range segments { + if dmlPosition == nil || s.GetDmlPosition().Timestamp > dmlPosition.Timestamp { + dmlPosition = s.GetDmlPosition() + } + } + + // find new added delta logs when executing compaction + originDeltalogs := make([]*datapb.DeltaLogInfo, 0) + for _, s := range segments { + originDeltalogs = append(originDeltalogs, s.GetDeltalogs()...) + } + + deletedDeltalogs := make([]*datapb.DeltaLogInfo, 0) + for _, l := range compactionLogs { + deletedDeltalogs = append(deletedDeltalogs, l.GetDeltalogs()...) + } + + newAddedDeltalogs := m.updateDeltalogs(originDeltalogs, deletedDeltalogs, nil) + deltalogs := append(result.GetDeltalogs(), newAddedDeltalogs...) + + compactionFrom := make([]UniqueID, 0, len(segments)) + for _, s := range segments { + compactionFrom = append(compactionFrom, s.GetID()) + } + + segment := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: result.GetSegmentID(), + CollectionID: segments[0].CollectionID, + PartitionID: segments[0].PartitionID, + InsertChannel: segments[0].InsertChannel, + NumOfRows: result.NumOfRows, + State: commonpb.SegmentState_Flushing, + MaxRowNum: segments[0].MaxRowNum, + Binlogs: result.GetInsertLogs(), + Statslogs: result.GetField2StatslogPaths(), + Deltalogs: deltalogs, + DmlPosition: dmlPosition, + CreatedByCompaction: true, + CompactionFrom: compactionFrom, + }, + isCompacting: false, + } + + data := make(map[string]string) + + for _, s := range segments { + k, v, err := m.marshal(s) + if err != nil { + return err + } + data[k] = v + } + k, v, err := m.marshal(segment) + if err != nil { + return err + } + data[k] = v + + if err := m.saveKvTxn(data); err != nil { + return err + } + + for _, s := range segments { + m.segments.DropSegment(s.GetID()) + } + + m.segments.SetSegment(segment.GetID(), segment) + return nil +} + +func (m *meta) CompleteInnerCompaction(segmentBinlogs *datapb.CompactionSegmentBinlogs, result *datapb.CompactionResult) error { + m.Lock() + defer m.Unlock() + + if segment := m.segments.GetSegment(segmentBinlogs.SegmentID); segment != nil { + cloned := segment.Clone() + cloned.Binlogs = m.updateBinlogs(cloned.GetBinlogs(), segmentBinlogs.GetFieldBinlogs(), result.GetInsertLogs()) + cloned.Statslogs = m.updateBinlogs(cloned.GetStatslogs(), segmentBinlogs.GetField2StatslogPaths(), result.GetField2StatslogPaths()) + cloned.Deltalogs = m.updateDeltalogs(cloned.GetDeltalogs(), segmentBinlogs.GetDeltalogs(), result.GetDeltalogs()) + if err := m.saveSegmentInfo(cloned); err != nil { + return err + } + + cloned.isCompacting = false + + m.segments.SetSegment(cloned.GetID(), cloned) + } + return nil +} + +func (m *meta) updateBinlogs(origin []*datapb.FieldBinlog, removes []*datapb.FieldBinlog, adds []*datapb.FieldBinlog) []*datapb.FieldBinlog { + fieldBinlogs := make(map[int64]map[string]struct{}) + for _, f := range origin { + fid := f.GetFieldID() + if _, ok := fieldBinlogs[fid]; !ok { + fieldBinlogs[fid] = make(map[string]struct{}) + } + for _, p := range f.GetBinlogs() { + fieldBinlogs[fid][p] = struct{}{} + } + } + + for _, f := range removes { + fid := f.GetFieldID() + if _, ok := fieldBinlogs[fid]; !ok { + continue + } + for _, p := range f.GetBinlogs() { + delete(fieldBinlogs[fid], p) + } + } + + for _, f := range adds { + fid := f.GetFieldID() + if _, ok := fieldBinlogs[fid]; !ok { + fieldBinlogs[fid] = make(map[string]struct{}) + } + for _, p := range f.GetBinlogs() { + fieldBinlogs[fid][p] = struct{}{} + } + } + + res := make([]*datapb.FieldBinlog, 0, len(fieldBinlogs)) + for fid, logs := range fieldBinlogs { + if len(logs) == 0 { + continue + } + + binlogs := make([]string, 0, len(logs)) + for path := range logs { + binlogs = append(binlogs, path) + } + + field := &datapb.FieldBinlog{FieldID: fid, Binlogs: binlogs} + res = append(res, field) + } + return res +} + +func (m *meta) updateDeltalogs(origin []*datapb.DeltaLogInfo, removes []*datapb.DeltaLogInfo, adds []*datapb.DeltaLogInfo) []*datapb.DeltaLogInfo { + deltalogs := make(map[string]*datapb.DeltaLogInfo) + for _, d := range origin { + deltalogs[d.GetDeltaLogPath()] = d + } + + for _, r := range removes { + delete(deltalogs, r.GetDeltaLogPath()) + } + + res := make([]*datapb.DeltaLogInfo, 0, len(deltalogs)) + for _, log := range deltalogs { + res = append(res, log) + } + res = append(res, adds...) + return res +} + +func (m *meta) marshal(segment *SegmentInfo) (string, string, error) { + segBytes, err := proto.Marshal(segment.SegmentInfo) + if err != nil { + return "", "", fmt.Errorf("failed to marshal segment info, %v", err) + } + key := buildSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID()) + return key, string(segBytes), nil +} + // saveSegmentInfo utility function saving segment info into kv store func (m *meta) saveSegmentInfo(segment *SegmentInfo) error { segBytes, err := proto.Marshal(segment.SegmentInfo) @@ -463,11 +716,13 @@ func (m *meta) saveSegmentInfo(segment *SegmentInfo) error { kvs[dataKey] = string(segBytes) if segment.State == commonpb.SegmentState_Flushed { handoffSegmentInfo := &querypb.SegmentInfo{ - SegmentID: segment.ID, - CollectionID: segment.CollectionID, - PartitionID: segment.PartitionID, - ChannelID: segment.InsertChannel, - SegmentState: querypb.SegmentState_sealed, + SegmentID: segment.ID, + CollectionID: segment.CollectionID, + PartitionID: segment.PartitionID, + ChannelID: segment.InsertChannel, + SegmentState: querypb.SegmentState_sealed, + CreatedByCompaction: segment.GetCreatedByCompaction(), + CompactionFrom: segment.GetCompactionFrom(), } handoffSegBytes, err := proto.Marshal(handoffSegmentInfo) if err != nil { diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index ca95832775..73f3409796 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -23,6 +23,7 @@ import ( "testing" "github.com/golang/protobuf/proto" + "github.com/milvus-io/milvus/internal/kv" memkv "github.com/milvus-io/milvus/internal/kv/mem" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -97,17 +98,17 @@ func TestMeta_Basic(t *testing.T) { assert.True(t, proto.Equal(info1_0, segInfo1_0)) // check GetSegmentsOfCollection - segIDs := meta.GetSegmentsOfCollection(collID) + segIDs := meta.GetSegmentsIDOfCollection(collID) assert.EqualValues(t, 3, len(segIDs)) assert.Contains(t, segIDs, segID0_0) assert.Contains(t, segIDs, segID1_0) assert.Contains(t, segIDs, segID1_1) // check GetSegmentsOfPartition - segIDs = meta.GetSegmentsOfPartition(collID, partID0) + segIDs = meta.GetSegmentsIDOfPartition(collID, partID0) assert.EqualValues(t, 1, len(segIDs)) assert.Contains(t, segIDs, segID0_0) - segIDs = meta.GetSegmentsOfPartition(collID, partID1) + segIDs = meta.GetSegmentsIDOfPartition(collID, partID1) assert.EqualValues(t, 2, len(segIDs)) assert.Contains(t, segIDs, segID1_0) assert.Contains(t, segIDs, segID1_1) @@ -115,7 +116,7 @@ func TestMeta_Basic(t *testing.T) { // check DropSegment err = meta.DropSegment(segID1_0) assert.Nil(t, err) - segIDs = meta.GetSegmentsOfPartition(collID, partID1) + segIDs = meta.GetSegmentsIDOfPartition(collID, partID1) assert.EqualValues(t, 1, len(segIDs)) assert.Contains(t, segIDs, segID1_1) @@ -183,6 +184,22 @@ func TestMeta_Basic(t *testing.T) { nums = meta.GetNumRowsOfCollection(collID) assert.EqualValues(t, (rowCount0 + rowCount1), nums) }) + + t.Run("Test GetSegmentsChanPart", func(t *testing.T) { + result := meta.GetSegmentsChanPart(func(*SegmentInfo) bool { return true }) + assert.Equal(t, 2, len(result)) + for _, entry := range result { + assert.Equal(t, "c1", entry.channelName) + if entry.partitionID == UniqueID(100) { + assert.Equal(t, 3, len(entry.segments)) + } + if entry.partitionID == UniqueID(101) { + assert.Equal(t, 1, len(entry.segments)) + } + } + result = meta.GetSegmentsChanPart(func(seg *SegmentInfo) bool { return seg.GetCollectionID() == 10 }) + assert.Equal(t, 0, len(result)) + }) } func TestGetUnFlushedSegments(t *testing.T) { @@ -315,3 +332,211 @@ func TestSaveHandoffMeta(t *testing.T) { assert.Nil(t, err) assert.Equal(t, 100, int(segmentID)) } + +func Test_meta_CompleteMergeCompaction(t *testing.T) { + type fields struct { + client kv.TxnKV + collections map[UniqueID]*datapb.CollectionInfo + segments *SegmentsInfo + } + type args struct { + compactionLogs []*datapb.CompactionSegmentBinlogs + result *datapb.CompactionResult + } + tests := []struct { + name string + fields fields + args args + wantErr bool + }{ + { + "test normal merge", + fields{ + memkv.NewMemoryKV(), + nil, + &SegmentsInfo{map[int64]*SegmentInfo{ + 1: {SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1", "log2"}}}, + Statslogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog1", "statlog2"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog1"}, {DeltaLogPath: "deltalog2"}}, + }}, + 2: {SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3", "log4"}}}, + Statslogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog3", "statlog4"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog3"}, {DeltaLogPath: "deltalog4"}}, + }}, + }}, + }, + args{ + []*datapb.CompactionSegmentBinlogs{ + { + SegmentID: 1, + FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1", "log2"}}}, + Field2StatslogPaths: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog1", "statlog2"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog1"}, {DeltaLogPath: "deltalog2"}}, + }, + { + SegmentID: 2, + FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3", "log4"}}}, + Field2StatslogPaths: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog3", "statlog4"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog3"}, {DeltaLogPath: "deltalog4"}}, + }, + }, + &datapb.CompactionResult{ + SegmentID: 3, + InsertLogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log5"}}}, + Field2StatslogPaths: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statslog5"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog5"}}, + }, + }, + false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + m := &meta{ + client: tt.fields.client, + collections: tt.fields.collections, + segments: tt.fields.segments, + } + err := m.CompleteMergeCompaction(tt.args.compactionLogs, tt.args.result) + assert.Equal(t, tt.wantErr, err != nil) + if err == nil { + for _, l := range tt.args.compactionLogs { + assert.Nil(t, m.GetSegment(l.GetSegmentID())) + } + segment := m.GetSegment(tt.args.result.SegmentID) + assert.NotNil(t, segment) + assert.EqualValues(t, tt.args.result.GetInsertLogs(), segment.GetBinlogs()) + assert.EqualValues(t, tt.args.result.GetField2StatslogPaths(), segment.GetStatslogs()) + assert.EqualValues(t, tt.args.result.GetDeltalogs(), segment.GetDeltalogs()) + } + }) + } +} + +func Test_meta_CompleteInnerCompaction(t *testing.T) { + type fields struct { + client kv.TxnKV + collections map[UniqueID]*datapb.CollectionInfo + segments *SegmentsInfo + } + type args struct { + segmentBinlogs *datapb.CompactionSegmentBinlogs + result *datapb.CompactionResult + } + tests := []struct { + name string + fields fields + args args + wantErr bool + want *SegmentInfo + }{ + { + "test normal merge", + fields{ + memkv.NewMemoryKV(), + nil, + &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: {SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1", "log2"}}}, + Statslogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog1", "statlog2"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog1"}, {DeltaLogPath: "deltalog2"}}, + }}, + }, + }, + }, + args{ + &datapb.CompactionSegmentBinlogs{ + SegmentID: 1, + FieldBinlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log1"}}}, + Field2StatslogPaths: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog1"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog1"}}, + }, + &datapb.CompactionResult{ + SegmentID: 1, + InsertLogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log3"}}}, + Field2StatslogPaths: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog3"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog3"}}, + }, + }, + false, + &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + Binlogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"log2", "log3"}}}, + Statslogs: []*datapb.FieldBinlog{{FieldID: 1, Binlogs: []string{"statlog2", "statlog3"}}}, + Deltalogs: []*datapb.DeltaLogInfo{{DeltaLogPath: "deltalog2"}, {DeltaLogPath: "deltalog3"}}, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + m := &meta{ + client: tt.fields.client, + collections: tt.fields.collections, + segments: tt.fields.segments, + } + err := m.CompleteInnerCompaction(tt.args.segmentBinlogs, tt.args.result) + assert.Equal(t, tt.wantErr, err != nil) + + if err != nil { + segment := m.GetSegment(tt.args.result.SegmentID) + assert.EqualValues(t, tt.want, segment) + } + }) + } +} + +func Test_meta_SetSegmentCompacting(t *testing.T) { + type fields struct { + client kv.TxnKV + segments *SegmentsInfo + } + type args struct { + segmentID UniqueID + compacting bool + } + tests := []struct { + name string + fields fields + args args + }{ + { + "test set segment compacting", + fields{ + memkv.NewMemoryKV(), + &SegmentsInfo{ + map[int64]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + }, + isCompacting: false, + }, + }, + }, + }, + args{ + segmentID: 1, + compacting: true, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + m := &meta{ + client: tt.fields.client, + segments: tt.fields.segments, + } + m.SetSegmentCompacting(tt.args.segmentID, tt.args.compacting) + segment := m.GetSegment(tt.args.segmentID) + assert.Equal(t, tt.args.compacting, segment.isCompacting) + }) + } +} diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index d91c679564..5cc141ea4a 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -193,6 +193,13 @@ func (c *mockDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMe }, nil } +func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) { + if c.ch != nil { + c.ch <- struct{}{} + } + return &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: "not implemented"}, nil +} + func (c *mockDataNodeClient) Stop() error { c.state = internalpb.StateCode_Abnormal return nil @@ -439,3 +446,141 @@ func (m *mockRootCoordService) GetMetrics(ctx context.Context, req *milvuspb.Get ComponentName: metricsinfo.ConstructComponentName(typeutil.RootCoordRole, nodeID), }, nil } + +type mockCompactionHandler struct { + methods map[string]interface{} +} + +func (h *mockCompactionHandler) start() { + if f, ok := h.methods["start"]; ok { + if ff, ok := f.(func()); ok { + ff() + return + } + } + panic("not implemented") +} + +func (h *mockCompactionHandler) stop() { + if f, ok := h.methods["stop"]; ok { + if ff, ok := f.(func()); ok { + ff() + return + } + } + panic("not implemented") +} + +// execCompactionPlan start to execute plan and return immediately +func (h *mockCompactionHandler) execCompactionPlan(plan *datapb.CompactionPlan) error { + if f, ok := h.methods["execCompactionPlan"]; ok { + if ff, ok := f.(func(plan *datapb.CompactionPlan) error); ok { + return ff(plan) + } + } + panic("not implemented") +} + +// completeCompaction record the result of a compaction +func (h *mockCompactionHandler) completeCompaction(result *datapb.CompactionResult) error { + if f, ok := h.methods["completeCompaction"]; ok { + if ff, ok := f.(func(result *datapb.CompactionResult) error); ok { + return ff(result) + } + } + panic("not implemented") +} + +// getCompaction return compaction task. If planId does not exist, return nil. +func (h *mockCompactionHandler) getCompaction(planID int64) *compactionTask { + if f, ok := h.methods["getCompaction"]; ok { + if ff, ok := f.(func(planID int64) *compactionTask); ok { + return ff(planID) + } + } + panic("not implemented") +} + +// expireCompaction set the compaction state to expired +func (h *mockCompactionHandler) expireCompaction(ts Timestamp) error { + if f, ok := h.methods["expireCompaction"]; ok { + if ff, ok := f.(func(ts Timestamp) error); ok { + return ff(ts) + } + } + panic("not implemented") +} + +// isFull return true if the task pool is full +func (h *mockCompactionHandler) isFull() bool { + if f, ok := h.methods["isFull"]; ok { + if ff, ok := f.(func() bool); ok { + return ff() + } + } + panic("not implemented") +} + +// get compaction by signal id and return the number of executing/completed/timeout plans +func (h *mockCompactionHandler) getCompactionBySignalID(signalID int64) (executing int, completed int, timeout int) { + if f, ok := h.methods["getCompactionBySignalID"]; ok { + if ff, ok := f.(func(signalID int64) (executing int, completed int, timeout int)); ok { + return ff(signalID) + } + } + panic("not implemented") +} + +type mockCompactionTrigger struct { + methods map[string]interface{} +} + +// triggerCompaction trigger a compaction if any compaction condition satisfy. +func (t *mockCompactionTrigger) triggerCompaction(tt *timetravel) error { + if f, ok := t.methods["triggerCompaction"]; ok { + if ff, ok := f.(func(tt *timetravel) error); ok { + return ff(tt) + } + } + panic("not implemented") +} + +// triggerSingleCompaction trigerr a compaction bundled with collection-partiiton-channel-segment +func (t *mockCompactionTrigger) triggerSingleCompaction(collectionID int64, partitionID int64, segmentID int64, channel string, tt *timetravel) error { + if f, ok := t.methods["triggerSingleCompaction"]; ok { + if ff, ok := f.(func(collectionID int64, partitionID int64, segmentID int64, channel string, tt *timetravel) error); ok { + return ff(collectionID, partitionID, segmentID, channel, tt) + } + } + panic("not implemented") +} + +// forceTriggerCompaction force to start a compaction +func (t *mockCompactionTrigger) forceTriggerCompaction(collectionID int64, tt *timetravel) (UniqueID, error) { + if f, ok := t.methods["forceTriggerCompaction"]; ok { + if ff, ok := f.(func(collectionID int64, tt *timetravel) (UniqueID, error)); ok { + return ff(collectionID, tt) + } + } + panic("not implemented") +} + +func (t *mockCompactionTrigger) start() { + if f, ok := t.methods["start"]; ok { + if ff, ok := f.(func()); ok { + ff() + return + } + } + panic("not implemented") +} + +func (t *mockCompactionTrigger) stop() { + if f, ok := t.methods["stop"]; ok { + if ff, ok := f.(func()); ok { + ff() + return + } + } + panic("not implemented") +} diff --git a/internal/datacoord/param_table.go b/internal/datacoord/param_table.go index 40dd4d06fd..7c1875dbab 100644 --- a/internal/datacoord/param_table.go +++ b/internal/datacoord/param_table.go @@ -66,6 +66,8 @@ type ParamTable struct { CreatedTime time.Time UpdatedTime time.Time + + EnableCompaction bool } // Params is a package scoped variable of type ParamTable. @@ -108,6 +110,8 @@ func (p *ParamTable) Init() { p.initFlushStreamPosSubPath() p.initStatsStreamPosSubPath() + + p.initEnableCompaction() } // InitOnce ensures param table is a singleton @@ -271,3 +275,7 @@ func (p *ParamTable) initChannelWatchPrefix() { // This will be removed after we reconstruct our config module. p.ChannelWatchSubPath = "channelwatch" } + +func (p *ParamTable) initEnableCompaction() { + p.EnableCompaction = p.ParseBool("datacoord.enableCompaction", false) +} diff --git a/internal/datacoord/segment_info.go b/internal/datacoord/segment_info.go index 0b326ab994..55609f5520 100644 --- a/internal/datacoord/segment_info.go +++ b/internal/datacoord/segment_info.go @@ -36,6 +36,7 @@ type SegmentInfo struct { currRows int64 allocations []*Allocation lastFlushTime time.Time + isCompacting bool } // NewSegmentInfo create `SegmentInfo` wrapper from `datapb.SegmentInfo` @@ -173,6 +174,13 @@ func (s *SegmentsInfo) AddSegmentBinlogs(segmentID UniqueID, field2Binlogs map[U } } +// SetIsCompacting sets compactino status for segment +func (s *SegmentsInfo) SetIsCompacting(segmentID UniqueID, isCompacting bool) { + if segment, ok := s.segments[segmentID]; ok { + s.segments[segmentID] = segment.ShadowClone(SetIsCompacting(isCompacting)) + } +} + // Clone deep clone the segment info and return a new instance func (s *SegmentInfo) Clone(opts ...SegmentInfoOption) *SegmentInfo { info := proto.Clone(s.SegmentInfo).(*datapb.SegmentInfo) @@ -277,6 +285,13 @@ func SetFlushTime(t time.Time) SegmentInfoOption { } } +// SetIsCompacting is the option to set compacton state for segment info +func SetIsCompacting(isCompacting bool) SegmentInfoOption { + return func(segment *SegmentInfo) { + segment.isCompacting = isCompacting + } +} + func addSegmentBinlogs(field2Binlogs map[UniqueID][]string) SegmentInfoOption { return func(segment *SegmentInfo) { for fieldID, binlogPaths := range field2Binlogs { diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index e1c6fad719..45358f2244 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -102,9 +102,13 @@ type Server struct { segmentManager Manager allocator allocator cluster *Cluster + sessionManager *SessionManager channelManager *ChannelManager rootCoordClient types.RootCoord + compactionTrigger trigger + compactionHandler compactionPlanContext + metricsCacheManager *metricsinfo.MetricsCacheManager flushCh chan UniqueID @@ -235,6 +239,10 @@ func (s *Server) Start() error { } s.allocator = newRootCoordAllocator(s.rootCoordClient) + if Params.EnableCompaction { + s.createCompactionHandler() + s.createCompactionTrigger() + } s.startSegmentManager() if err = s.initServiceDiscovery(); err != nil { @@ -260,11 +268,29 @@ func (s *Server) initCluster() error { if err != nil { return err } - sessionManager := NewSessionManager(withSessionCreator(s.dataNodeCreator)) - s.cluster = NewCluster(sessionManager, s.channelManager) + s.sessionManager = NewSessionManager(withSessionCreator(s.dataNodeCreator)) + s.cluster = NewCluster(s.sessionManager, s.channelManager) return nil } +func (s *Server) createCompactionHandler() { + s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh) + s.compactionHandler.start() +} + +func (s *Server) stopCompactionHandler() { + s.compactionHandler.stop() +} + +func (s *Server) createCompactionTrigger() { + s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator) + s.compactionTrigger.start() +} + +func (s *Server) stopCompactionTrigger() { + s.compactionTrigger.stop() +} + func (s *Server) initServiceDiscovery() error { sessions, rev, err := s.session.GetSessions(typeutil.DataNodeRole) if err != nil { @@ -624,6 +650,11 @@ func (s *Server) Stop() error { log.Debug("dataCoord server shutdown") s.cluster.Close() s.stopServerLoop() + + if Params.EnableCompaction { + s.stopCompactionTrigger() + s.stopCompactionHandler() + } return nil } diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 8f29c5f305..7099aff0da 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -1304,6 +1304,162 @@ func TestGetRecoveryInfo(t *testing.T) { }) } +func TestGetCompactionState(t *testing.T) { + Params.EnableCompaction = true + t.Run("test get compaction state with new compactionhandler", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateHealthy + + svr.compactionHandler = &mockCompactionHandler{ + methods: map[string]interface{}{ + "getCompactionBySignalID": func(signalID int64) (executing, completed, timeout int) { + return 0, 1, 0 + }, + }, + } + + resp, err := svr.GetCompactionState(context.Background(), &datapb.GetCompactionStateRequest{}) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + assert.Equal(t, datapb.CompactionState_Completed, resp.GetState()) + }) + t.Run("test get compaction state in running", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateHealthy + + svr.compactionHandler = &mockCompactionHandler{ + methods: map[string]interface{}{ + "getCompactionBySignalID": func(signalID int64) (executing, completed, timeout int) { + return 3, 2, 1 + }, + }, + } + + resp, err := svr.GetCompactionState(context.Background(), &datapb.GetCompactionStateRequest{CompactionID: 1}) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + assert.Equal(t, datapb.CompactionState_Executing, resp.GetState()) + assert.EqualValues(t, 3, resp.GetExecutingPlanNo()) + assert.EqualValues(t, 2, resp.GetCompletedPlanNo()) + assert.EqualValues(t, 1, resp.GetTimeoutPlanNo()) + }) + + t.Run("with closed server", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateStopped + + resp, err := svr.GetCompactionState(context.Background(), &datapb.GetCompactionStateRequest{}) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetStatus().GetErrorCode()) + assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.GetStatus().GetReason()) + }) +} + +func TestCompleteCompaction(t *testing.T) { + Params.EnableCompaction = true + t.Run("test complete compaction successfully", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateHealthy + + svr.compactionHandler = &mockCompactionHandler{ + methods: map[string]interface{}{ + "completeCompaction": func(result *datapb.CompactionResult) error { + return nil + }, + }, + } + status, err := svr.CompleteCompaction(context.TODO(), &datapb.CompactionResult{}) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode) + }) + + t.Run("test complete compaction failure", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateHealthy + svr.compactionHandler = &mockCompactionHandler{ + methods: map[string]interface{}{ + "completeCompaction": func(result *datapb.CompactionResult) error { + return errors.New("mock error") + }, + }, + } + status, err := svr.CompleteCompaction(context.TODO(), &datapb.CompactionResult{}) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.ErrorCode) + }) + + t.Run("with closed server", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateStopped + + resp, err := svr.CompleteCompaction(context.Background(), &datapb.CompactionResult{}) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode()) + assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.GetReason()) + }) +} + +func TestManualCompaction(t *testing.T) { + Params.EnableCompaction = true + t.Run("test manual compaction successfully", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateHealthy + svr.compactionTrigger = &mockCompactionTrigger{ + methods: map[string]interface{}{ + "forceTriggerCompaction": func(collectionID int64, tt *timetravel) (UniqueID, error) { + return 1, nil + }, + }, + } + + resp, err := svr.ManualCompaction(context.TODO(), &datapb.ManualCompactionRequest{ + CollectionID: 1, + Timetravel: 1, + }) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) + }) + + t.Run("test manual compaction failure", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateHealthy + svr.compactionTrigger = &mockCompactionTrigger{ + methods: map[string]interface{}{ + "forceTriggerCompaction": func(collectionID int64, tt *timetravel) (UniqueID, error) { + return 0, errors.New("mock error") + }, + }, + } + + resp, err := svr.ManualCompaction(context.TODO(), &datapb.ManualCompactionRequest{ + CollectionID: 1, + Timetravel: 1, + }) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode) + }) + + t.Run("test manual compaction with closed server", func(t *testing.T) { + svr := &Server{} + svr.isServing = ServerStateStopped + svr.compactionTrigger = &mockCompactionTrigger{ + methods: map[string]interface{}{ + "forceTriggerCompaction": func(collectionID int64, tt *timetravel) (UniqueID, error) { + return 1, nil + }, + }, + } + + resp, err := svr.ManualCompaction(context.TODO(), &datapb.ManualCompactionRequest{ + CollectionID: 1, + Timetravel: 1, + }) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode) + assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.Status.Reason) + }) +} + func TestOptions(t *testing.T) { t.Run("SetRootCoordCreator", func(t *testing.T) { svr := newTestServer(t, nil) diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index fb70322fa8..4e3f272f93 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -21,6 +21,7 @@ import ( "fmt" "strconv" "sync/atomic" + "time" "github.com/milvus-io/milvus/internal/util/trace" @@ -343,6 +344,19 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath if req.Flushed { s.segmentManager.DropSegment(ctx, req.SegmentID) s.flushCh <- req.SegmentID + + if Params.EnableCompaction { + cctx, cancel := context.WithTimeout(s.ctx, 5*time.Second) + defer cancel() + + tt, err := getTimetravel(cctx, s.allocator) + if err == nil { + if err = s.compactionTrigger.triggerSingleCompaction(segment.GetCollectionID(), segment.GetPartitionID(), + segmentID, segment.GetInsertChannel(), tt); err != nil { + log.Warn("failed to trigger single compaction", zap.Int64("segmentID", segmentID)) + } + } + } } resp.ErrorCode = commonpb.ErrorCode_Success return resp, nil @@ -389,7 +403,7 @@ func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf resp.Status.Reason = serverNotServingErrMsg return resp, nil } - segmentIDs := s.meta.GetSegmentsOfPartition(collectionID, partitionID) + segmentIDs := s.meta.GetSegmentsIDOfPartition(collectionID, partitionID) segment2Binlogs := make(map[UniqueID][]*datapb.FieldBinlog) segment2StatsBinlogs := make(map[UniqueID][]*datapb.FieldBinlog) segment2DeltaBinlogs := make(map[UniqueID][]*datapb.DeltaLogInfo) @@ -505,16 +519,17 @@ func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS } var segmentIDs []UniqueID if partitionID < 0 { - segmentIDs = s.meta.GetSegmentsOfCollection(collectionID) + segmentIDs = s.meta.GetSegmentsIDOfCollection(collectionID) } else { - segmentIDs = s.meta.GetSegmentsOfPartition(collectionID, partitionID) + segmentIDs = s.meta.GetSegmentsIDOfPartition(collectionID, partitionID) } ret := make([]UniqueID, 0, len(segmentIDs)) for _, id := range segmentIDs { s := s.meta.GetSegment(id) - if s == nil || s.GetState() != commonpb.SegmentState_Flushed { + if s != nil && s.GetState() != commonpb.SegmentState_Flushed { continue } + // if this segment == nil, we assume this segment has been compacted and flushed ret = append(ret, id) } resp.Segments = ret @@ -598,3 +613,102 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest Response: "", }, nil } + +func (s *Server) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { + log.Debug("receive complete compaction request", zap.Int64("planID", req.PlanID), zap.Int64("segmentID", req.GetSegmentID())) + + resp := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + } + + if s.isClosed() { + log.Warn("failed to complete compaction", zap.Int64("planID", req.PlanID), + zap.Error(errDataCoordIsUnhealthy(Params.NodeID))) + + resp.Reason = msgDataCoordIsUnhealthy(Params.NodeID) + return resp, nil + } + + if !Params.EnableCompaction { + resp.Reason = "compaction disabled" + return resp, nil + } + + if err := s.compactionHandler.completeCompaction(req); err != nil { + log.Error("failed to complete compaction", zap.Int64("planID", req.PlanID), zap.Error(err)) + resp.Reason = err.Error() + return resp, nil + } + + log.Debug("success to complete compaction", zap.Int64("planID", req.PlanID)) + resp.ErrorCode = commonpb.ErrorCode_Success + return resp, nil +} + +func (s *Server) ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) { + log.Debug("receive manual compaction", zap.Int64("collectionID", req.GetCollectionID())) + + resp := &datapb.ManualCompactionResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + + if s.isClosed() { + log.Warn("failed to execute manual compaction", zap.Int64("collectionID", req.GetCollectionID()), + zap.Error(errDataCoordIsUnhealthy(Params.NodeID))) + resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID) + return resp, nil + } + + if !Params.EnableCompaction { + resp.Status.Reason = "compaction disabled" + return resp, nil + } + + id, err := s.compactionTrigger.forceTriggerCompaction(req.CollectionID, &timetravel{req.Timetravel}) + if err != nil { + log.Error("failed to trigger manual compaction", zap.Int64("collectionID", req.GetCollectionID()), zap.Error(err)) + resp.Status.Reason = err.Error() + return resp, nil + } + + log.Debug("success to trigger manual compaction", zap.Int64("collectionID", req.GetCollectionID()), zap.Int64("compactionID", id)) + resp.Status.ErrorCode = commonpb.ErrorCode_Success + resp.CompactionID = id + return resp, nil +} + +func (s *Server) GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) { + log.Debug("receive get compaction state request", zap.Int64("compactionID", req.GetCompactionID())) + resp := &datapb.GetCompactionStateResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + } + + if s.isClosed() { + log.Warn("failed to get compaction state", zap.Int64("compactionID", req.GetCompactionID()), + zap.Error(errDataCoordIsUnhealthy(Params.NodeID))) + resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID) + return resp, nil + } + + if !Params.EnableCompaction { + resp.Status.Reason = "compaction disabled" + return resp, nil + } + + executing, completed, timeout := s.compactionHandler.getCompactionBySignalID(req.GetCompactionID()) + if executing != 0 { + resp.State = datapb.CompactionState_Executing + } else { + resp.State = datapb.CompactionState_Completed + } + + resp.ExecutingPlanNo = int64(executing) + resp.CompletedPlanNo = int64(completed) + resp.TimeoutPlanNo = int64(timeout) + resp.Status.ErrorCode = commonpb.ErrorCode_Success + return resp, nil +} diff --git a/internal/datacoord/session_manager.go b/internal/datacoord/session_manager.go index fb760dc60f..ed4a0256e7 100644 --- a/internal/datacoord/session_manager.go +++ b/internal/datacoord/session_manager.go @@ -18,6 +18,7 @@ package datacoord import ( "context" + "fmt" "sync" "time" @@ -28,7 +29,9 @@ import ( "go.uber.org/zap" ) -const flushTimeout = 5 * time.Second +const ( + flushTimeout = 5 * time.Second +) // SessionManager provides the grpc interfaces of cluster type SessionManager struct { @@ -105,20 +108,11 @@ func (c *SessionManager) Flush(ctx context.Context, nodeID int64, req *datapb.Fl } func (c *SessionManager) execFlush(ctx context.Context, nodeID int64, req *datapb.FlushSegmentsRequest) { - c.sessions.RLock() - session, ok := c.sessions.data[nodeID] - c.sessions.RUnlock() - - if !ok { - return - } - - cli, err := session.GetOrCreateClient(ctx) + cli, err := c.getClient(ctx, nodeID) if err != nil { - log.Warn("unable to connect to node", zap.Int64("node", nodeID), zap.Error(err)) + log.Warn("failed to get client", zap.Int64("nodeID", nodeID), zap.Error(err)) return } - ctx, cancel := context.WithTimeout(ctx, flushTimeout) defer cancel() @@ -131,6 +125,40 @@ func (c *SessionManager) execFlush(ctx context.Context, nodeID int64, req *datap log.Debug("success to flush", zap.Int64("node", nodeID), zap.Any("segments", req)) } +func (c *SessionManager) Compaction(nodeID int64, plan *datapb.CompactionPlan) { + go c.execCompaction(nodeID, plan) +} + +func (c *SessionManager) execCompaction(nodeID int64, plan *datapb.CompactionPlan) { + ctx, cancel := context.WithTimeout(context.Background(), compactionTimeout) + defer cancel() + cli, err := c.getClient(ctx, nodeID) + if err != nil { + log.Warn("failed to get client", zap.Int64("nodeID", nodeID), zap.Error(err)) + return + } + + resp, err := cli.Compaction(ctx, plan) + if err := VerifyResponse(resp, err); err != nil { + log.Warn("failed to execute compaction", zap.Int64("node", nodeID), zap.Error(err), zap.Int64("planID", plan.GetPlanID())) + return + } + + log.Debug("success to execute compaction", zap.Int64("node", nodeID), zap.Any("planID", plan.GetPlanID())) +} + +func (c *SessionManager) getClient(ctx context.Context, nodeID int64) (types.DataNode, error) { + c.sessions.RLock() + session, ok := c.sessions.data[nodeID] + c.sessions.RUnlock() + + if !ok { + return nil, fmt.Errorf("can not find session of node %d", nodeID) + } + + return session.GetOrCreateClient(ctx) +} + // Close release sessions func (c *SessionManager) Close() { c.sessions.Lock() diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index 042821ff84..68c0e6818c 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -24,6 +24,7 @@ import ( "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/util/tsoutil" ) // Response response interface for verification @@ -114,3 +115,15 @@ func (c *LongTermChecker) Stop() { c.t.Stop() close(c.ch) } + +func getTimetravel(ctx context.Context, allocator allocator) (*timetravel, error) { + ts, err := allocator.allocTimestamp(ctx) + if err != nil { + return nil, err + } + + pts, _ := tsoutil.ParseTS(ts) + ttpts := pts.Add(-timetravelRange) + tt := tsoutil.ComposeTS(ttpts.UnixNano()/int64(time.Millisecond), 0) + return &timetravel{tt}, nil +} diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 6195595eae..eb24c70749 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -702,3 +702,7 @@ func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe Response: "", }, nil } + +func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) { + panic("not implemented") // TODO: Implement +} diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index f227c792cb..0f492da4ae 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -528,3 +528,48 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest } return ret.(*milvuspb.GetMetricsResponse), err } + +func (c *Client) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { + ret, err := c.recall(func() (interface{}, error) { + client, err := c.getGrpcClient() + if err != nil { + return nil, err + } + + return client.CompleteCompaction(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} + +func (c *Client) ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) { + ret, err := c.recall(func() (interface{}, error) { + client, err := c.getGrpcClient() + if err != nil { + return nil, err + } + + return client.ManualCompaction(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*datapb.ManualCompactionResponse), err +} + +func (c *Client) GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) { + ret, err := c.recall(func() (interface{}, error) { + client, err := c.getGrpcClient() + if err != nil { + return nil, err + } + + return client.GetCompactionState(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*datapb.GetCompactionStateResponse), err +} diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index 5b1ef01c97..f41283ef73 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -94,6 +94,18 @@ func (m *MockDataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMe return &milvuspb.GetMetricsResponse{}, m.err } +func (m *MockDataCoordClient) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.err +} + +func (m *MockDataCoordClient) ManualCompaction(ctx context.Context, in *datapb.ManualCompactionRequest, opts ...grpc.CallOption) (*datapb.ManualCompactionResponse, error) { + return &datapb.ManualCompactionResponse{}, m.err +} + +func (m *MockDataCoordClient) GetCompactionState(ctx context.Context, in *datapb.GetCompactionStateRequest, opts ...grpc.CallOption) (*datapb.GetCompactionStateResponse, error) { + return &datapb.GetCompactionStateResponse{}, m.err +} + func Test_NewClient(t *testing.T) { proxy.Params.InitOnce() @@ -166,6 +178,15 @@ func Test_NewClient(t *testing.T) { r15, err := client.GetMetrics(ctx, nil) retCheck(retNotNil, r15, err) + + r16, err := client.CompleteCompaction(ctx, nil) + retCheck(retNotNil, r16, err) + + r17, err := client.GetCompactionState(ctx, nil) + retCheck(retNotNil, r17, err) + + r18, err := client.ManualCompaction(ctx, nil) + retCheck(retNotNil, r18, err) } client.getGrpcClient = func() (datapb.DataCoordClient, error) { diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index d20c36c687..7645cb04fe 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -262,3 +262,15 @@ func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { return s.dataCoord.GetMetrics(ctx, req) } + +func (s *Server) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { + return s.dataCoord.CompleteCompaction(ctx, req) +} + +func (s *Server) ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) { + return s.dataCoord.ManualCompaction(ctx, req) +} + +func (s *Server) GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) { + return s.dataCoord.GetCompactionState(ctx, req) +} diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index 63881eaea6..9d7e89b37e 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -30,24 +30,26 @@ import ( /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// type MockDataCoord struct { - states *internalpb.ComponentStates - status *commonpb.Status - err error - initErr error - startErr error - stopErr error - regErr error - strResp *milvuspb.StringResponse - infoResp *datapb.GetSegmentInfoResponse - flushResp *datapb.FlushResponse - assignResp *datapb.AssignSegmentIDResponse - segStateResp *datapb.GetSegmentStatesResponse - binResp *datapb.GetInsertBinlogPathsResponse - colStatResp *datapb.GetCollectionStatisticsResponse - partStatResp *datapb.GetPartitionStatisticsResponse - recoverResp *datapb.GetRecoveryInfoResponse - flushSegResp *datapb.GetFlushedSegmentsResponse - metricResp *milvuspb.GetMetricsResponse + states *internalpb.ComponentStates + status *commonpb.Status + err error + initErr error + startErr error + stopErr error + regErr error + strResp *milvuspb.StringResponse + infoResp *datapb.GetSegmentInfoResponse + flushResp *datapb.FlushResponse + assignResp *datapb.AssignSegmentIDResponse + segStateResp *datapb.GetSegmentStatesResponse + binResp *datapb.GetInsertBinlogPathsResponse + colStatResp *datapb.GetCollectionStatisticsResponse + partStatResp *datapb.GetPartitionStatisticsResponse + recoverResp *datapb.GetRecoveryInfoResponse + flushSegResp *datapb.GetFlushedSegmentsResponse + metricResp *milvuspb.GetMetricsResponse + compactionStateResp *datapb.GetCompactionStateResponse + manualCompactionResp *datapb.ManualCompactionResponse } func (m *MockDataCoord) Init() error { @@ -126,6 +128,18 @@ func (m *MockDataCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetrics return m.metricResp, m.err } +func (m *MockDataCoord) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { + return m.status, m.err +} + +func (m *MockDataCoord) ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) { + return m.manualCompactionResp, m.err +} + +func (m *MockDataCoord) GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) { + return m.compactionStateResp, m.err +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// func Test_NewServer(t *testing.T) { ctx := context.Background() diff --git a/internal/distributed/datanode/client/client.go b/internal/distributed/datanode/client/client.go index 04f5d2c498..b2ee005b4d 100644 --- a/internal/distributed/datanode/client/client.go +++ b/internal/distributed/datanode/client/client.go @@ -289,3 +289,18 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest } return ret.(*milvuspb.GetMetricsResponse), err } + +func (c *Client) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) { + ret, err := c.recall(func() (interface{}, error) { + client, err := c.getGrpcClient() + if err != nil { + return nil, err + } + + return client.Compaction(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} diff --git a/internal/distributed/datanode/client/client_test.go b/internal/distributed/datanode/client/client_test.go index d030aa74e2..6b64d62530 100644 --- a/internal/distributed/datanode/client/client_test.go +++ b/internal/distributed/datanode/client/client_test.go @@ -54,6 +54,10 @@ func (m *MockDataNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMet return &milvuspb.GetMetricsResponse{}, m.err } +func (m *MockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.err +} + func Test_NewClient(t *testing.T) { proxy.Params.InitOnce() @@ -100,6 +104,9 @@ func Test_NewClient(t *testing.T) { r5, err := client.GetMetrics(ctx, nil) retCheck(retNotNil, r5, err) + + r6, err := client.Compaction(ctx, nil) + retCheck(retNotNil, r6, err) } client.getGrpcClient = func() (datapb.DataNodeClient, error) { diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index 39ee142f31..11ed44deaa 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -293,3 +293,7 @@ func (s *Server) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsReq func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { return s.datanode.GetMetrics(ctx, request) } + +func (s *Server) Compaction(ctx context.Context, request *datapb.CompactionPlan) (*commonpb.Status, error) { + return s.datanode.Compaction(ctx, request) +} diff --git a/internal/distributed/datanode/service_test.go b/internal/distributed/datanode/service_test.go index 500931da3e..7afb25b254 100644 --- a/internal/distributed/datanode/service_test.go +++ b/internal/distributed/datanode/service_test.go @@ -103,6 +103,10 @@ func (m *MockDataNode) GetMetrics(ctx context.Context, request *milvuspb.GetMetr return m.metricResp, m.err } +func (m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) { + return m.status, m.err +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// type mockDataCoord struct { types.DataCoord diff --git a/internal/distributed/proxy/service_test.go b/internal/distributed/proxy/service_test.go index b3b741edd4..60cbba137d 100644 --- a/internal/distributed/proxy/service_test.go +++ b/internal/distributed/proxy/service_test.go @@ -375,6 +375,18 @@ func (m *MockDataCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetrics return nil, nil } +func (m *MockDataCoord) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { + return nil, nil +} + +func (m *MockDataCoord) ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) { + return nil, nil +} + +func (m *MockDataCoord) GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) { + return nil, nil +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// type MockProxy struct { MockBase diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 569fb8019f..d8c1b49210 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -33,6 +33,9 @@ service DataCoord { // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {} + rpc CompleteCompaction(CompactionResult) returns (common.Status) {} + rpc ManualCompaction(ManualCompactionRequest) returns (ManualCompactionResponse) {} + rpc GetCompactionState(GetCompactionStateRequest) returns (GetCompactionStateResponse) {} } service DataNode { @@ -44,6 +47,7 @@ service DataNode { // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {} + rpc Compaction(CompactionPlan) returns (common.Status) {} } message FlushRequest { @@ -201,6 +205,9 @@ message SegmentInfo { repeated FieldBinlog binlogs = 11; repeated FieldBinlog statslogs = 12; repeated DeltaLogInfo deltalogs = 13; + + bool createdByCompaction = 14; + repeated int64 compactionFrom = 15; } message SegmentStartPosition { @@ -317,17 +324,14 @@ message CompactionSegmentBinlogs { repeated DeltaLogInfo deltalogs = 4; } -message CompactionMergeGroup { - repeated CompactionSegmentBinlogs segmentBinlogs = 1; -} - message CompactionPlan { int64 planID = 1; - repeated CompactionMergeGroup mergeGroup = 2; + repeated CompactionSegmentBinlogs segmentBinlogs = 2; uint64 start_time = 3; int32 timeout_in_seconds = 4; CompactionType type = 5; uint64 timetravel = 6; + string channel = 7; } message CompactionResult { @@ -339,8 +343,36 @@ message CompactionResult { repeated DeltaLogInfo deltalogs = 6; } +message ManualCompactionRequest { + int64 collectionID = 1; + uint64 timetravel = 2; +} + +message ManualCompactionResponse { + common.Status status = 1; + int64 compactionID = 2; +} + +message GetCompactionStateRequest { + int64 compactionID = 1; +} + +enum CompactionState { + UndefiedState = 0; + Executing = 1; + Completed = 2; +} + +message GetCompactionStateResponse { + common.Status status = 1; + CompactionState state = 2; + int64 executingPlanNo = 3; + int64 timeoutPlanNo = 4; + int64 completedPlanNo = 5; +} + // Deprecated message SegmentFieldBinlogMeta { int64 fieldID = 1; string binlog_path = 2; -} +} \ No newline at end of file diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index 50adaafe5d..91d5c35705 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -81,6 +81,34 @@ func (CompactionType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_82cd95f524594f49, []int{1} } +type CompactionState int32 + +const ( + CompactionState_UndefiedState CompactionState = 0 + CompactionState_Executing CompactionState = 1 + CompactionState_Completed CompactionState = 2 +) + +var CompactionState_name = map[int32]string{ + 0: "UndefiedState", + 1: "Executing", + 2: "Completed", +} + +var CompactionState_value = map[string]int32{ + "UndefiedState": 0, + "Executing": 1, + "Completed": 2, +} + +func (x CompactionState) String() string { + return proto.EnumName(CompactionState_name, int32(x)) +} + +func (CompactionState) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{2} +} + type FlushRequest struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` @@ -1369,6 +1397,8 @@ type SegmentInfo struct { Binlogs []*FieldBinlog `protobuf:"bytes,11,rep,name=binlogs,proto3" json:"binlogs,omitempty"` Statslogs []*FieldBinlog `protobuf:"bytes,12,rep,name=statslogs,proto3" json:"statslogs,omitempty"` Deltalogs []*DeltaLogInfo `protobuf:"bytes,13,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + CreatedByCompaction bool `protobuf:"varint,14,opt,name=createdByCompaction,proto3" json:"createdByCompaction,omitempty"` + CompactionFrom []int64 `protobuf:"varint,15,rep,packed,name=compactionFrom,proto3" json:"compactionFrom,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1490,6 +1520,20 @@ func (m *SegmentInfo) GetDeltalogs() []*DeltaLogInfo { return nil } +func (m *SegmentInfo) GetCreatedByCompaction() bool { + if m != nil { + return m.CreatedByCompaction + } + return false +} + +func (m *SegmentInfo) GetCompactionFrom() []int64 { + if m != nil { + return m.CompactionFrom + } + return nil +} + type SegmentStartPosition struct { StartPosition *internalpb.MsgPosition `protobuf:"bytes,1,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` @@ -2426,62 +2470,24 @@ func (m *CompactionSegmentBinlogs) GetDeltalogs() []*DeltaLogInfo { return nil } -type CompactionMergeGroup struct { - SegmentBinlogs []*CompactionSegmentBinlogs `protobuf:"bytes,1,rep,name=segmentBinlogs,proto3" json:"segmentBinlogs,omitempty"` +type CompactionPlan struct { + PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` + SegmentBinlogs []*CompactionSegmentBinlogs `protobuf:"bytes,2,rep,name=segmentBinlogs,proto3" json:"segmentBinlogs,omitempty"` + StartTime uint64 `protobuf:"varint,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + TimeoutInSeconds int32 `protobuf:"varint,4,opt,name=timeout_in_seconds,json=timeoutInSeconds,proto3" json:"timeout_in_seconds,omitempty"` + Type CompactionType `protobuf:"varint,5,opt,name=type,proto3,enum=milvus.proto.data.CompactionType" json:"type,omitempty"` + Timetravel uint64 `protobuf:"varint,6,opt,name=timetravel,proto3" json:"timetravel,omitempty"` + Channel string `protobuf:"bytes,7,opt,name=channel,proto3" json:"channel,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` } -func (m *CompactionMergeGroup) Reset() { *m = CompactionMergeGroup{} } -func (m *CompactionMergeGroup) String() string { return proto.CompactTextString(m) } -func (*CompactionMergeGroup) ProtoMessage() {} -func (*CompactionMergeGroup) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{40} -} - -func (m *CompactionMergeGroup) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CompactionMergeGroup.Unmarshal(m, b) -} -func (m *CompactionMergeGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CompactionMergeGroup.Marshal(b, m, deterministic) -} -func (m *CompactionMergeGroup) XXX_Merge(src proto.Message) { - xxx_messageInfo_CompactionMergeGroup.Merge(m, src) -} -func (m *CompactionMergeGroup) XXX_Size() int { - return xxx_messageInfo_CompactionMergeGroup.Size(m) -} -func (m *CompactionMergeGroup) XXX_DiscardUnknown() { - xxx_messageInfo_CompactionMergeGroup.DiscardUnknown(m) -} - -var xxx_messageInfo_CompactionMergeGroup proto.InternalMessageInfo - -func (m *CompactionMergeGroup) GetSegmentBinlogs() []*CompactionSegmentBinlogs { - if m != nil { - return m.SegmentBinlogs - } - return nil -} - -type CompactionPlan struct { - PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` - MergeGroup []*CompactionMergeGroup `protobuf:"bytes,2,rep,name=mergeGroup,proto3" json:"mergeGroup,omitempty"` - StartTime uint64 `protobuf:"varint,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` - TimeoutInSeconds int32 `protobuf:"varint,4,opt,name=timeout_in_seconds,json=timeoutInSeconds,proto3" json:"timeout_in_seconds,omitempty"` - Type CompactionType `protobuf:"varint,5,opt,name=type,proto3,enum=milvus.proto.data.CompactionType" json:"type,omitempty"` - Timetravel uint64 `protobuf:"varint,6,opt,name=timetravel,proto3" json:"timetravel,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - func (m *CompactionPlan) Reset() { *m = CompactionPlan{} } func (m *CompactionPlan) String() string { return proto.CompactTextString(m) } func (*CompactionPlan) ProtoMessage() {} func (*CompactionPlan) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{41} + return fileDescriptor_82cd95f524594f49, []int{40} } func (m *CompactionPlan) XXX_Unmarshal(b []byte) error { @@ -2509,9 +2515,9 @@ func (m *CompactionPlan) GetPlanID() int64 { return 0 } -func (m *CompactionPlan) GetMergeGroup() []*CompactionMergeGroup { +func (m *CompactionPlan) GetSegmentBinlogs() []*CompactionSegmentBinlogs { if m != nil { - return m.MergeGroup + return m.SegmentBinlogs } return nil } @@ -2544,6 +2550,13 @@ func (m *CompactionPlan) GetTimetravel() uint64 { return 0 } +func (m *CompactionPlan) GetChannel() string { + if m != nil { + return m.Channel + } + return "" +} + type CompactionResult struct { PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` @@ -2560,7 +2573,7 @@ func (m *CompactionResult) Reset() { *m = CompactionResult{} } func (m *CompactionResult) String() string { return proto.CompactTextString(m) } func (*CompactionResult) ProtoMessage() {} func (*CompactionResult) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{42} + return fileDescriptor_82cd95f524594f49, []int{41} } func (m *CompactionResult) XXX_Unmarshal(b []byte) error { @@ -2623,6 +2636,210 @@ func (m *CompactionResult) GetDeltalogs() []*DeltaLogInfo { return nil } +type ManualCompactionRequest struct { + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Timetravel uint64 `protobuf:"varint,2,opt,name=timetravel,proto3" json:"timetravel,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ManualCompactionRequest) Reset() { *m = ManualCompactionRequest{} } +func (m *ManualCompactionRequest) String() string { return proto.CompactTextString(m) } +func (*ManualCompactionRequest) ProtoMessage() {} +func (*ManualCompactionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{42} +} + +func (m *ManualCompactionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ManualCompactionRequest.Unmarshal(m, b) +} +func (m *ManualCompactionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ManualCompactionRequest.Marshal(b, m, deterministic) +} +func (m *ManualCompactionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ManualCompactionRequest.Merge(m, src) +} +func (m *ManualCompactionRequest) XXX_Size() int { + return xxx_messageInfo_ManualCompactionRequest.Size(m) +} +func (m *ManualCompactionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ManualCompactionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ManualCompactionRequest proto.InternalMessageInfo + +func (m *ManualCompactionRequest) GetCollectionID() int64 { + if m != nil { + return m.CollectionID + } + return 0 +} + +func (m *ManualCompactionRequest) GetTimetravel() uint64 { + if m != nil { + return m.Timetravel + } + return 0 +} + +type ManualCompactionResponse struct { + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + CompactionID int64 `protobuf:"varint,2,opt,name=compactionID,proto3" json:"compactionID,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ManualCompactionResponse) Reset() { *m = ManualCompactionResponse{} } +func (m *ManualCompactionResponse) String() string { return proto.CompactTextString(m) } +func (*ManualCompactionResponse) ProtoMessage() {} +func (*ManualCompactionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{43} +} + +func (m *ManualCompactionResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ManualCompactionResponse.Unmarshal(m, b) +} +func (m *ManualCompactionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ManualCompactionResponse.Marshal(b, m, deterministic) +} +func (m *ManualCompactionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ManualCompactionResponse.Merge(m, src) +} +func (m *ManualCompactionResponse) XXX_Size() int { + return xxx_messageInfo_ManualCompactionResponse.Size(m) +} +func (m *ManualCompactionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ManualCompactionResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ManualCompactionResponse proto.InternalMessageInfo + +func (m *ManualCompactionResponse) GetStatus() *commonpb.Status { + if m != nil { + return m.Status + } + return nil +} + +func (m *ManualCompactionResponse) GetCompactionID() int64 { + if m != nil { + return m.CompactionID + } + return 0 +} + +type GetCompactionStateRequest struct { + CompactionID int64 `protobuf:"varint,1,opt,name=compactionID,proto3" json:"compactionID,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetCompactionStateRequest) Reset() { *m = GetCompactionStateRequest{} } +func (m *GetCompactionStateRequest) String() string { return proto.CompactTextString(m) } +func (*GetCompactionStateRequest) ProtoMessage() {} +func (*GetCompactionStateRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{44} +} + +func (m *GetCompactionStateRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetCompactionStateRequest.Unmarshal(m, b) +} +func (m *GetCompactionStateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetCompactionStateRequest.Marshal(b, m, deterministic) +} +func (m *GetCompactionStateRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetCompactionStateRequest.Merge(m, src) +} +func (m *GetCompactionStateRequest) XXX_Size() int { + return xxx_messageInfo_GetCompactionStateRequest.Size(m) +} +func (m *GetCompactionStateRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetCompactionStateRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetCompactionStateRequest proto.InternalMessageInfo + +func (m *GetCompactionStateRequest) GetCompactionID() int64 { + if m != nil { + return m.CompactionID + } + return 0 +} + +type GetCompactionStateResponse struct { + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + State CompactionState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.data.CompactionState" json:"state,omitempty"` + ExecutingPlanNo int64 `protobuf:"varint,3,opt,name=executingPlanNo,proto3" json:"executingPlanNo,omitempty"` + TimeoutPlanNo int64 `protobuf:"varint,4,opt,name=timeoutPlanNo,proto3" json:"timeoutPlanNo,omitempty"` + CompletedPlanNo int64 `protobuf:"varint,5,opt,name=completedPlanNo,proto3" json:"completedPlanNo,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetCompactionStateResponse) Reset() { *m = GetCompactionStateResponse{} } +func (m *GetCompactionStateResponse) String() string { return proto.CompactTextString(m) } +func (*GetCompactionStateResponse) ProtoMessage() {} +func (*GetCompactionStateResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{45} +} + +func (m *GetCompactionStateResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_GetCompactionStateResponse.Unmarshal(m, b) +} +func (m *GetCompactionStateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_GetCompactionStateResponse.Marshal(b, m, deterministic) +} +func (m *GetCompactionStateResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetCompactionStateResponse.Merge(m, src) +} +func (m *GetCompactionStateResponse) XXX_Size() int { + return xxx_messageInfo_GetCompactionStateResponse.Size(m) +} +func (m *GetCompactionStateResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetCompactionStateResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GetCompactionStateResponse proto.InternalMessageInfo + +func (m *GetCompactionStateResponse) GetStatus() *commonpb.Status { + if m != nil { + return m.Status + } + return nil +} + +func (m *GetCompactionStateResponse) GetState() CompactionState { + if m != nil { + return m.State + } + return CompactionState_UndefiedState +} + +func (m *GetCompactionStateResponse) GetExecutingPlanNo() int64 { + if m != nil { + return m.ExecutingPlanNo + } + return 0 +} + +func (m *GetCompactionStateResponse) GetTimeoutPlanNo() int64 { + if m != nil { + return m.TimeoutPlanNo + } + return 0 +} + +func (m *GetCompactionStateResponse) GetCompletedPlanNo() int64 { + if m != nil { + return m.CompletedPlanNo + } + return 0 +} + // Deprecated type SegmentFieldBinlogMeta struct { FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` @@ -2636,7 +2853,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{} func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{43} + return fileDescriptor_82cd95f524594f49, []int{46} } func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { @@ -2674,6 +2891,7 @@ func (m *SegmentFieldBinlogMeta) GetBinlogPath() string { func init() { proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value) proto.RegisterEnum("milvus.proto.data.CompactionType", CompactionType_name, CompactionType_value) + proto.RegisterEnum("milvus.proto.data.CompactionState", CompactionState_name, CompactionState_value) proto.RegisterType((*FlushRequest)(nil), "milvus.proto.data.FlushRequest") proto.RegisterType((*FlushResponse)(nil), "milvus.proto.data.FlushResponse") proto.RegisterType((*SegmentIDRequest)(nil), "milvus.proto.data.SegmentIDRequest") @@ -2714,167 +2932,184 @@ func init() { proto.RegisterType((*SegmentFlushCompletedMsg)(nil), "milvus.proto.data.SegmentFlushCompletedMsg") proto.RegisterType((*ChannelWatchInfo)(nil), "milvus.proto.data.ChannelWatchInfo") proto.RegisterType((*CompactionSegmentBinlogs)(nil), "milvus.proto.data.CompactionSegmentBinlogs") - proto.RegisterType((*CompactionMergeGroup)(nil), "milvus.proto.data.CompactionMergeGroup") proto.RegisterType((*CompactionPlan)(nil), "milvus.proto.data.CompactionPlan") proto.RegisterType((*CompactionResult)(nil), "milvus.proto.data.CompactionResult") + proto.RegisterType((*ManualCompactionRequest)(nil), "milvus.proto.data.ManualCompactionRequest") + proto.RegisterType((*ManualCompactionResponse)(nil), "milvus.proto.data.ManualCompactionResponse") + proto.RegisterType((*GetCompactionStateRequest)(nil), "milvus.proto.data.GetCompactionStateRequest") + proto.RegisterType((*GetCompactionStateResponse)(nil), "milvus.proto.data.GetCompactionStateResponse") proto.RegisterType((*SegmentFieldBinlogMeta)(nil), "milvus.proto.data.SegmentFieldBinlogMeta") } func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 2412 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3a, 0xcb, 0x6f, 0x1b, 0xc7, - 0xf9, 0x5e, 0x3e, 0x64, 0xf1, 0xe3, 0x43, 0xf4, 0xd8, 0x3f, 0x99, 0x3f, 0xda, 0x96, 0xe5, 0x6d, - 0x62, 0x2b, 0x4e, 0x22, 0xc5, 0x72, 0x83, 0x06, 0x75, 0xd2, 0x20, 0xb6, 0x6c, 0x96, 0xa8, 0xe4, - 0xaa, 0x4b, 0x25, 0x01, 0x9a, 0x03, 0xb1, 0xe2, 0x8e, 0xa8, 0xad, 0xf6, 0xc1, 0xec, 0x0c, 0x65, - 0x3b, 0x97, 0x04, 0x2e, 0x10, 0xa0, 0x41, 0xdb, 0xb4, 0x28, 0x0a, 0xf4, 0x50, 0xa0, 0x45, 0x4f, - 0x05, 0x7a, 0xe9, 0xa5, 0x97, 0xfe, 0x05, 0x45, 0xfb, 0xff, 0xf4, 0x5c, 0xcc, 0x63, 0x67, 0x97, - 0xcb, 0x25, 0xb9, 0x94, 0x6c, 0xeb, 0xc6, 0xf9, 0xf6, 0x7b, 0xcd, 0x37, 0xdf, 0x73, 0x86, 0x50, - 0xb7, 0x4c, 0x6a, 0x76, 0x7b, 0xbe, 0x1f, 0x58, 0xeb, 0x83, 0xc0, 0xa7, 0x3e, 0xba, 0xe0, 0xda, - 0xce, 0xf1, 0x90, 0x88, 0xd5, 0x3a, 0xfb, 0xdc, 0xac, 0xf4, 0x7c, 0xd7, 0xf5, 0x3d, 0x01, 0x6a, - 0xd6, 0x6c, 0x8f, 0xe2, 0xc0, 0x33, 0x1d, 0xb9, 0xae, 0xc4, 0x09, 0x9a, 0x15, 0xd2, 0x3b, 0xc4, - 0xae, 0x29, 0x56, 0xfa, 0x53, 0xa8, 0x3c, 0x72, 0x86, 0xe4, 0xd0, 0xc0, 0x9f, 0x0f, 0x31, 0xa1, - 0xe8, 0x1d, 0x28, 0xec, 0x9b, 0x04, 0x37, 0xb4, 0x55, 0x6d, 0xad, 0xbc, 0x79, 0x75, 0x7d, 0x44, - 0x96, 0x94, 0xb2, 0x43, 0xfa, 0xf7, 0x4d, 0x82, 0x0d, 0x8e, 0x89, 0x10, 0x14, 0xac, 0xfd, 0xf6, - 0x56, 0x23, 0xb7, 0xaa, 0xad, 0xe5, 0x0d, 0xfe, 0x1b, 0xe9, 0x50, 0xe9, 0xf9, 0x8e, 0x83, 0x7b, - 0xd4, 0xf6, 0xbd, 0xf6, 0x56, 0xa3, 0xc0, 0xbf, 0x8d, 0xc0, 0xf4, 0x3f, 0x6a, 0x50, 0x95, 0xa2, - 0xc9, 0xc0, 0xf7, 0x08, 0x46, 0x77, 0x61, 0x81, 0x50, 0x93, 0x0e, 0x89, 0x94, 0x7e, 0x25, 0x55, - 0x7a, 0x87, 0xa3, 0x18, 0x12, 0x35, 0x93, 0xf8, 0xfc, 0xb8, 0x78, 0xb4, 0x02, 0x40, 0x70, 0xdf, - 0xc5, 0x1e, 0x6d, 0x6f, 0x91, 0x46, 0x61, 0x35, 0xbf, 0x96, 0x37, 0x62, 0x10, 0xfd, 0xb7, 0x1a, - 0xd4, 0x3b, 0xe1, 0x32, 0xb4, 0xce, 0x25, 0x28, 0xf6, 0xfc, 0xa1, 0x47, 0xb9, 0x82, 0x55, 0x43, - 0x2c, 0xd0, 0x0d, 0xa8, 0xf4, 0x0e, 0x4d, 0xcf, 0xc3, 0x4e, 0xd7, 0x33, 0x5d, 0xcc, 0x55, 0x29, - 0x19, 0x65, 0x09, 0x7b, 0x6c, 0xba, 0x38, 0x93, 0x46, 0xab, 0x50, 0x1e, 0x98, 0x01, 0xb5, 0x47, - 0x6c, 0x16, 0x07, 0xe9, 0x7f, 0xd6, 0x60, 0xf9, 0x23, 0x42, 0xec, 0xbe, 0x37, 0xa6, 0xd9, 0x32, - 0x2c, 0x78, 0xbe, 0x85, 0xdb, 0x5b, 0x5c, 0xb5, 0xbc, 0x21, 0x57, 0xe8, 0x0a, 0x94, 0x06, 0x18, - 0x07, 0xdd, 0xc0, 0x77, 0x42, 0xc5, 0x16, 0x19, 0xc0, 0xf0, 0x1d, 0x8c, 0x7e, 0x02, 0x17, 0x48, - 0x82, 0x11, 0x69, 0xe4, 0x57, 0xf3, 0x6b, 0xe5, 0xcd, 0xef, 0xac, 0x8f, 0x79, 0xd9, 0x7a, 0x52, - 0xa8, 0x31, 0x4e, 0xad, 0x7f, 0x95, 0x83, 0x8b, 0x0a, 0x4f, 0xe8, 0xca, 0x7e, 0x33, 0xcb, 0x11, - 0xdc, 0x57, 0xea, 0x89, 0x45, 0x16, 0xcb, 0x29, 0x93, 0xe7, 0xe3, 0x26, 0xcf, 0xe0, 0x60, 0x49, - 0x7b, 0x16, 0xc7, 0xec, 0x89, 0xae, 0x43, 0x19, 0x3f, 0x1d, 0xd8, 0x01, 0xee, 0x52, 0xdb, 0xc5, - 0x8d, 0x85, 0x55, 0x6d, 0xad, 0x60, 0x80, 0x00, 0xed, 0xd9, 0x6e, 0xdc, 0x23, 0xcf, 0x67, 0xf6, - 0x48, 0xfd, 0x2f, 0x1a, 0x5c, 0x1e, 0x3b, 0x25, 0xe9, 0xe2, 0x06, 0xd4, 0xf9, 0xce, 0x23, 0xcb, - 0x30, 0x67, 0x67, 0x06, 0xbf, 0x39, 0xcd, 0xe0, 0x11, 0xba, 0x31, 0x46, 0x1f, 0x53, 0x32, 0x97, - 0x5d, 0xc9, 0x23, 0xb8, 0xdc, 0xc2, 0x54, 0x0a, 0x60, 0xdf, 0x30, 0x39, 0x79, 0x0a, 0x18, 0x8d, - 0xa5, 0xdc, 0x58, 0x2c, 0xfd, 0x3d, 0xa7, 0x62, 0x89, 0x8b, 0x6a, 0x7b, 0x07, 0x3e, 0xba, 0x0a, - 0x25, 0x85, 0x22, 0xbd, 0x22, 0x02, 0xa0, 0xef, 0x41, 0x91, 0x69, 0x2a, 0x5c, 0xa2, 0xb6, 0x79, - 0x23, 0x7d, 0x4f, 0x31, 0x9e, 0x86, 0xc0, 0x47, 0x6d, 0xa8, 0x11, 0x6a, 0x06, 0xb4, 0x3b, 0xf0, - 0x09, 0x3f, 0x67, 0xee, 0x38, 0xe5, 0x4d, 0x7d, 0x94, 0x83, 0x4a, 0x91, 0x3b, 0xa4, 0xbf, 0x2b, - 0x31, 0x8d, 0x2a, 0xa7, 0x0c, 0x97, 0xe8, 0x21, 0x54, 0xb0, 0x67, 0x45, 0x8c, 0x0a, 0x99, 0x19, - 0x95, 0xb1, 0x67, 0x29, 0x36, 0xd1, 0xf9, 0x14, 0xb3, 0x9f, 0xcf, 0x2f, 0x35, 0x68, 0x8c, 0x1f, - 0xd0, 0x69, 0x12, 0xe5, 0x3d, 0x41, 0x84, 0xc5, 0x01, 0x4d, 0x8d, 0x70, 0x75, 0x48, 0x86, 0x24, - 0xd1, 0x6d, 0xf8, 0xbf, 0x48, 0x1b, 0xfe, 0xe5, 0xa5, 0x39, 0xcb, 0xcf, 0x35, 0x58, 0x4e, 0xca, - 0x3a, 0xcd, 0xbe, 0xbf, 0x0b, 0x45, 0xdb, 0x3b, 0xf0, 0xc3, 0x6d, 0xaf, 0x4c, 0x89, 0x33, 0x26, - 0x4b, 0x20, 0xeb, 0x2e, 0x5c, 0x69, 0x61, 0xda, 0xf6, 0x08, 0x0e, 0xe8, 0x7d, 0xdb, 0x73, 0xfc, - 0xfe, 0xae, 0x49, 0x0f, 0x4f, 0x11, 0x23, 0x23, 0xee, 0x9e, 0x4b, 0xb8, 0xbb, 0xfe, 0x57, 0x0d, - 0xae, 0xa6, 0xcb, 0x93, 0x5b, 0x6f, 0xc2, 0xe2, 0x81, 0x8d, 0x1d, 0x8b, 0xd9, 0x4c, 0xe3, 0x36, - 0x53, 0x6b, 0x16, 0x2b, 0x03, 0x86, 0x2c, 0x77, 0x78, 0x63, 0x82, 0x83, 0x76, 0x68, 0x60, 0x7b, - 0xfd, 0x6d, 0x9b, 0x50, 0x43, 0xe0, 0xc7, 0xec, 0x99, 0xcf, 0xee, 0x99, 0xdf, 0x68, 0xb0, 0xd2, - 0xc2, 0xf4, 0x81, 0x4a, 0xb5, 0xec, 0xbb, 0x4d, 0xa8, 0xdd, 0x23, 0x2f, 0xb7, 0x89, 0x48, 0xa9, - 0x99, 0xfa, 0xb7, 0x1a, 0x5c, 0x9f, 0xa8, 0x8c, 0x34, 0x9d, 0x4c, 0x25, 0x61, 0xa2, 0x4d, 0x4f, - 0x25, 0x3f, 0xc2, 0xcf, 0x3e, 0x31, 0x9d, 0x21, 0xde, 0x35, 0xed, 0x40, 0xa4, 0x92, 0x13, 0x26, - 0xd6, 0xbf, 0x69, 0x70, 0xad, 0x85, 0xe9, 0x6e, 0x58, 0x66, 0xce, 0xd0, 0x3a, 0x19, 0x3a, 0x8a, - 0x5f, 0x8b, 0xc3, 0x4c, 0xd5, 0xf6, 0x4c, 0xcc, 0xb7, 0xc2, 0xe3, 0x20, 0x16, 0x90, 0x0f, 0x44, - 0x2f, 0x20, 0x8d, 0xa7, 0xff, 0x23, 0x07, 0x95, 0x4f, 0x64, 0x7f, 0xc0, 0xcb, 0x48, 0xd2, 0x0e, - 0x5a, 0xba, 0x1d, 0x62, 0x2d, 0x45, 0x5a, 0x97, 0xd1, 0x82, 0x2a, 0xc1, 0xf8, 0xe8, 0x24, 0x45, - 0xa3, 0xc2, 0x08, 0x55, 0xb2, 0xdf, 0x86, 0x0b, 0x43, 0xef, 0x80, 0xb5, 0xb5, 0xd8, 0x92, 0xbb, - 0x10, 0xdd, 0xe5, 0xec, 0xcc, 0x33, 0x4e, 0x88, 0x7e, 0x08, 0x4b, 0x49, 0x5e, 0xc5, 0x4c, 0xbc, - 0x92, 0x64, 0xfa, 0x2f, 0x34, 0x58, 0xfe, 0xd4, 0xa4, 0xbd, 0xc3, 0x2d, 0x57, 0x5a, 0xf4, 0x14, - 0xfe, 0xf8, 0x01, 0x94, 0x8e, 0xa5, 0xf5, 0xc2, 0xa4, 0x73, 0x3d, 0x45, 0xa1, 0xf8, 0x39, 0x19, - 0x11, 0x85, 0xfe, 0x2f, 0x0d, 0x2e, 0xf1, 0xce, 0x3f, 0xd4, 0xee, 0xd5, 0x47, 0xc6, 0x8c, 0xee, - 0x1f, 0xdd, 0x84, 0x9a, 0x6b, 0x06, 0x47, 0x9d, 0x08, 0xa7, 0xc8, 0x71, 0x12, 0x50, 0xfd, 0x29, - 0x80, 0x5c, 0xed, 0x90, 0xfe, 0x09, 0xf4, 0x7f, 0x0f, 0xce, 0x4b, 0xa9, 0x32, 0x48, 0x66, 0x1d, - 0x6c, 0x88, 0xae, 0xff, 0x5b, 0x83, 0x5a, 0x94, 0xf6, 0x78, 0x28, 0xd4, 0x20, 0xa7, 0x02, 0x20, - 0xd7, 0xde, 0x42, 0x1f, 0xc0, 0x82, 0x98, 0xf5, 0x24, 0xef, 0xd7, 0x47, 0x79, 0xcb, 0x39, 0x30, - 0x96, 0x3b, 0x39, 0xc0, 0x90, 0x44, 0xcc, 0x46, 0x2a, 0x55, 0x88, 0xb1, 0x20, 0x6f, 0xc4, 0x20, - 0xa8, 0x0d, 0x4b, 0xa3, 0x9d, 0x56, 0xe8, 0xe8, 0xab, 0x93, 0x52, 0xc4, 0x96, 0x49, 0x4d, 0x9e, - 0x21, 0x6a, 0x23, 0x8d, 0x16, 0xd1, 0xff, 0x5b, 0x80, 0x72, 0x6c, 0x97, 0x63, 0x3b, 0x49, 0x1e, - 0x69, 0x6e, 0x76, 0xb2, 0xcb, 0x8f, 0xb7, 0xfb, 0xaf, 0x43, 0xcd, 0xe6, 0x05, 0xb6, 0x2b, 0x5d, - 0x91, 0x67, 0xc4, 0x92, 0x51, 0x15, 0x50, 0x19, 0x17, 0x68, 0x05, 0xca, 0xde, 0xd0, 0xed, 0xfa, - 0x07, 0xdd, 0xc0, 0x7f, 0x42, 0xe4, 0xdc, 0x50, 0xf2, 0x86, 0xee, 0x8f, 0x0f, 0x0c, 0xff, 0x09, - 0x89, 0x5a, 0xd3, 0x85, 0x39, 0x5b, 0xd3, 0x15, 0x28, 0xbb, 0xe6, 0x53, 0xc6, 0xb5, 0xeb, 0x0d, - 0x5d, 0x3e, 0x52, 0xe4, 0x8d, 0x92, 0x6b, 0x3e, 0x35, 0xfc, 0x27, 0x8f, 0x87, 0x2e, 0x5a, 0x83, - 0xba, 0x63, 0x12, 0xda, 0x8d, 0xcf, 0x24, 0x8b, 0x7c, 0x26, 0xa9, 0x31, 0xf8, 0xc3, 0x68, 0x2e, - 0x19, 0x6f, 0x72, 0x4b, 0xa7, 0x68, 0x72, 0x2d, 0xd7, 0x89, 0x18, 0x41, 0xf6, 0x26, 0xd7, 0x72, - 0x1d, 0xc5, 0xe6, 0x3d, 0x38, 0xbf, 0xcf, 0xdb, 0x16, 0xd2, 0x28, 0x4f, 0xcc, 0x50, 0x8f, 0x58, - 0xc7, 0x22, 0xba, 0x1b, 0x23, 0x44, 0x47, 0xef, 0x43, 0x89, 0xd7, 0x0b, 0x4e, 0x5b, 0xc9, 0x44, - 0x1b, 0x11, 0xb0, 0x54, 0x64, 0x61, 0x87, 0x9a, 0x9c, 0xba, 0x3a, 0x31, 0x15, 0x6d, 0x31, 0x9c, - 0x6d, 0xbf, 0x2f, 0x52, 0x91, 0xa2, 0xd0, 0xbf, 0x84, 0x4b, 0xd1, 0x49, 0xc5, 0xac, 0x32, 0x6e, - 0x60, 0xed, 0xa4, 0x06, 0x9e, 0xde, 0xf8, 0xfd, 0xa1, 0x00, 0xcb, 0x1d, 0xf3, 0x18, 0xbf, 0xfc, - 0x1e, 0x33, 0x53, 0x5e, 0xdc, 0x86, 0x0b, 0xbc, 0xad, 0xdc, 0x8c, 0xe9, 0x33, 0xa5, 0x7c, 0xc5, - 0x0f, 0x65, 0x9c, 0x10, 0x7d, 0xc8, 0xea, 0x2e, 0xee, 0x1d, 0xed, 0xfa, 0x76, 0x54, 0xba, 0xae, - 0xa5, 0xf0, 0x79, 0xa0, 0xb0, 0x8c, 0x38, 0x05, 0xda, 0x1d, 0x4f, 0x31, 0x0b, 0x9c, 0xc9, 0xad, - 0xa9, 0xc3, 0x4b, 0x64, 0xfd, 0x64, 0xa6, 0x41, 0x0d, 0x38, 0x2f, 0x4b, 0x23, 0x8f, 0xbf, 0x45, - 0x23, 0x5c, 0xa2, 0x5d, 0xb8, 0x28, 0x76, 0xd0, 0x91, 0xce, 0x25, 0x36, 0xbf, 0x98, 0x69, 0xf3, - 0x69, 0xa4, 0xa3, 0xbe, 0x59, 0x9a, 0xdb, 0x37, 0xbf, 0xd1, 0x00, 0x22, 0xc3, 0xcc, 0x98, 0x97, - 0x7f, 0x00, 0x8b, 0xca, 0x55, 0x73, 0x99, 0x5d, 0x55, 0xd1, 0x24, 0x93, 0x5e, 0x3e, 0x91, 0xf4, - 0xf4, 0xff, 0x68, 0x50, 0x89, 0x2b, 0xca, 0x92, 0x69, 0x80, 0x7b, 0x7e, 0x60, 0x75, 0xb1, 0x47, - 0x03, 0x1b, 0x8b, 0x99, 0xac, 0x60, 0x54, 0x05, 0xf4, 0xa1, 0x00, 0x32, 0x34, 0x96, 0xc7, 0x08, - 0x35, 0xdd, 0x41, 0xf7, 0x20, 0xf0, 0x5d, 0xae, 0x5d, 0xc1, 0xa8, 0x2a, 0xe8, 0xa3, 0xc0, 0x77, - 0xd1, 0x0d, 0xa8, 0x44, 0x68, 0xd4, 0xe7, 0xf2, 0x0b, 0x46, 0x59, 0xc1, 0xf6, 0x7c, 0xf4, 0x1a, - 0xd4, 0xb8, 0x6d, 0xba, 0x8e, 0xdf, 0xef, 0xb2, 0xf9, 0x45, 0x66, 0xef, 0x8a, 0x25, 0xd5, 0x62, - 0x46, 0x1f, 0xc5, 0x22, 0xf6, 0x17, 0x58, 0xe6, 0x6f, 0x85, 0xd5, 0xb1, 0xbf, 0xc0, 0xfa, 0x73, - 0x0d, 0xaa, 0xac, 0x18, 0x3d, 0xf6, 0x2d, 0xbc, 0x77, 0xc2, 0xd2, 0x9d, 0xe1, 0xee, 0xea, 0x2a, - 0x94, 0xd4, 0x0e, 0xe4, 0x96, 0x22, 0x00, 0x1b, 0x74, 0xab, 0xb2, 0xe6, 0x74, 0xd4, 0x5d, 0x26, - 0x67, 0xa5, 0x71, 0x56, 0xfc, 0x37, 0xfa, 0xfe, 0xe8, 0x45, 0xc8, 0x6b, 0xa9, 0xd1, 0xc3, 0x99, - 0xf0, 0xf6, 0x6e, 0xa4, 0xe0, 0x64, 0x99, 0xa0, 0xbe, 0x62, 0x07, 0x2b, 0x4d, 0xc1, 0x0f, 0xb6, - 0x01, 0xe7, 0x4d, 0xcb, 0x0a, 0x30, 0x21, 0x52, 0x8f, 0x70, 0xc9, 0xbe, 0x1c, 0xe3, 0x80, 0x84, - 0x2e, 0x96, 0x37, 0xc2, 0x25, 0x7a, 0x1f, 0x16, 0x55, 0x3f, 0x98, 0x4f, 0xeb, 0x01, 0xe2, 0x7a, - 0xca, 0x8e, 0x5f, 0x51, 0xe8, 0xdf, 0xe6, 0xa0, 0x26, 0x83, 0xf7, 0xbe, 0x2c, 0x0a, 0xd3, 0x9d, - 0xfd, 0x3e, 0x54, 0x0e, 0xa2, 0xe0, 0x9b, 0x36, 0xd9, 0xc7, 0x63, 0x74, 0x84, 0x66, 0x96, 0xc3, - 0x8f, 0x96, 0xa5, 0xc2, 0xa9, 0xca, 0x52, 0x71, 0xee, 0xd0, 0xff, 0x08, 0xca, 0x31, 0xc6, 0x3c, - 0x69, 0x89, 0x61, 0x5f, 0xda, 0x22, 0x5c, 0xb2, 0x2f, 0xfb, 0x31, 0x23, 0x94, 0x54, 0x59, 0x65, - 0x4d, 0xf6, 0xe5, 0x16, 0xa6, 0x06, 0xee, 0xf9, 0xc7, 0x38, 0x78, 0x76, 0xfa, 0x7b, 0x94, 0x7b, - 0xb1, 0x33, 0xce, 0xd8, 0xf3, 0x2b, 0x02, 0x74, 0x2f, 0xd2, 0x33, 0x9f, 0x36, 0x46, 0xc6, 0x13, - 0xb8, 0x3c, 0xa1, 0x68, 0x2b, 0xbf, 0x11, 0x37, 0x42, 0xa3, 0x5b, 0x39, 0x69, 0x8d, 0x7c, 0x21, - 0xad, 0xa4, 0xfe, 0x3b, 0x0d, 0xfe, 0xbf, 0x85, 0xe9, 0xa3, 0xd1, 0x29, 0xeb, 0xac, 0xb5, 0x72, - 0xa1, 0x99, 0xa6, 0xd4, 0x69, 0x4e, 0xbd, 0x09, 0x8b, 0x24, 0x1c, 0x3d, 0xc5, 0x5d, 0x9d, 0x5a, - 0xeb, 0x5f, 0x6b, 0xd0, 0x90, 0x52, 0xb8, 0xcc, 0x07, 0xbe, 0x3b, 0x70, 0x30, 0xc5, 0xd6, 0xab, - 0x9e, 0x85, 0xfe, 0xa4, 0x41, 0x3d, 0x9e, 0x04, 0x79, 0x1e, 0x7b, 0x17, 0x8a, 0x7c, 0xe4, 0x94, - 0x1a, 0xcc, 0x74, 0x56, 0x81, 0xcd, 0x22, 0x8a, 0xb7, 0x0c, 0x7b, 0x24, 0x4c, 0x72, 0x72, 0x19, - 0x65, 0xe2, 0xfc, 0xdc, 0x99, 0x58, 0xff, 0x55, 0x0e, 0x1a, 0xcc, 0x3c, 0xa6, 0x18, 0xb4, 0x5e, - 0x75, 0xb2, 0x9b, 0xd0, 0xdb, 0xe4, 0x5f, 0x50, 0x6f, 0x53, 0x98, 0x3b, 0xc1, 0x1d, 0xc1, 0xa5, - 0xc8, 0x1c, 0x3b, 0x38, 0xe8, 0xe3, 0x56, 0xe0, 0x0f, 0x07, 0xa8, 0x03, 0x35, 0x32, 0x62, 0x1c, - 0x79, 0xeb, 0xf4, 0x66, 0x9a, 0xb1, 0x27, 0xd8, 0xd3, 0x48, 0xb0, 0xd0, 0x7f, 0x9f, 0x63, 0xa3, - 0x72, 0x88, 0xbc, 0xeb, 0x98, 0x1e, 0x5a, 0x86, 0x85, 0x81, 0x63, 0x46, 0xf7, 0x45, 0x72, 0x85, - 0x5a, 0x00, 0xae, 0xd2, 0x46, 0x9a, 0xfa, 0xd6, 0x54, 0xd9, 0x91, 0xf2, 0x46, 0x8c, 0x14, 0x5d, - 0x03, 0x10, 0x9d, 0x2b, 0x9f, 0xe2, 0x64, 0xed, 0x17, 0x9e, 0xc4, 0x06, 0xb8, 0xb7, 0x00, 0xb1, - 0x0f, 0xfe, 0x90, 0x76, 0x6d, 0xaf, 0x4b, 0x70, 0xcf, 0xf7, 0x2c, 0xc2, 0x1b, 0x9a, 0xa2, 0x51, - 0x97, 0x5f, 0xda, 0x5e, 0x47, 0xc0, 0xd1, 0xbb, 0x50, 0xa0, 0xcf, 0x06, 0xa2, 0x95, 0xa9, 0xa5, - 0xa6, 0xce, 0x48, 0x9f, 0xbd, 0x67, 0x03, 0x6c, 0x70, 0x74, 0x36, 0xc0, 0x33, 0x56, 0x34, 0x30, - 0x8f, 0xb1, 0x13, 0xbe, 0x6e, 0x45, 0x10, 0xfd, 0x9f, 0x39, 0xa8, 0x47, 0x84, 0x06, 0x26, 0x43, - 0x87, 0x4e, 0xb4, 0xcc, 0xf4, 0xd9, 0x62, 0x56, 0x35, 0xfd, 0x10, 0xca, 0x72, 0xf4, 0x9e, 0xa3, - 0x9e, 0x82, 0x20, 0xd9, 0x9e, 0xe2, 0xc1, 0xc5, 0x17, 0xe4, 0xc1, 0x0b, 0x73, 0x7b, 0x70, 0x07, - 0x96, 0xc3, 0xdc, 0x17, 0x49, 0xda, 0xc1, 0xd4, 0x9c, 0x52, 0xad, 0xaf, 0x43, 0x59, 0xd4, 0x34, - 0xd1, 0xbf, 0x8a, 0x8e, 0x11, 0xf6, 0xd5, 0xc4, 0x74, 0xfb, 0x0e, 0x5c, 0x18, 0x4b, 0x21, 0xa8, - 0x06, 0xf0, 0xb1, 0xd7, 0x93, 0xb9, 0xb5, 0x7e, 0x0e, 0x55, 0x60, 0x31, 0xcc, 0xb4, 0x75, 0xed, - 0x76, 0x27, 0xee, 0xdb, 0xec, 0xf0, 0xd1, 0x65, 0xb8, 0xf8, 0xb1, 0x67, 0xe1, 0x03, 0xdb, 0xc3, - 0x56, 0xf4, 0xa9, 0x7e, 0x0e, 0x5d, 0x84, 0xa5, 0xb6, 0xe7, 0xe1, 0x20, 0x06, 0xd4, 0x18, 0x90, - 0xbb, 0x70, 0x0c, 0x98, 0xdb, 0x7c, 0x5e, 0x85, 0x12, 0x6b, 0x0a, 0x1f, 0xf8, 0x7e, 0x60, 0xa1, - 0x01, 0x20, 0x7e, 0xc7, 0xee, 0x0e, 0x7c, 0x4f, 0x3d, 0x46, 0xa1, 0x77, 0x26, 0xcc, 0x17, 0xe3, - 0xa8, 0xb2, 0x2c, 0x36, 0x6f, 0x4e, 0xa0, 0x48, 0xa0, 0xeb, 0xe7, 0x90, 0xcb, 0x25, 0xb2, 0x48, - 0xd9, 0xb3, 0x7b, 0x47, 0xe1, 0xc5, 0xcc, 0x14, 0x89, 0x09, 0xd4, 0x50, 0x62, 0xe2, 0x8d, 0x4b, - 0x2e, 0xc4, 0x43, 0x48, 0x58, 0x17, 0xf5, 0x73, 0xe8, 0x73, 0xb8, 0xd4, 0xc2, 0x34, 0xba, 0xfb, - 0x0e, 0x05, 0x6e, 0x4e, 0x16, 0x38, 0x86, 0x3c, 0xa7, 0xc8, 0x6d, 0x28, 0xf2, 0x9a, 0x89, 0xd2, - 0x7c, 0x2e, 0xfe, 0x8f, 0x8c, 0xe6, 0xea, 0x64, 0x04, 0xc5, 0xed, 0x67, 0xb0, 0x94, 0x78, 0x71, - 0x46, 0x6f, 0xa4, 0x90, 0xa5, 0xff, 0x77, 0xa0, 0x79, 0x3b, 0x0b, 0xaa, 0x92, 0xd5, 0x87, 0xda, - 0xe8, 0x0d, 0x3d, 0x5a, 0x4b, 0xa1, 0x4f, 0x7d, 0x2d, 0x6c, 0xbe, 0x91, 0x01, 0x53, 0x09, 0x72, - 0xa1, 0x9e, 0x7c, 0x01, 0x45, 0xb7, 0xa7, 0x32, 0x18, 0x75, 0xb7, 0x37, 0x33, 0xe1, 0x2a, 0x71, - 0xcf, 0xb8, 0x13, 0x8c, 0xbd, 0xc0, 0xa1, 0xf5, 0x74, 0x36, 0x93, 0x9e, 0x06, 0x9b, 0x1b, 0x99, - 0xf1, 0x95, 0xe8, 0xe7, 0xa2, 0x57, 0x4f, 0x7b, 0xc5, 0x42, 0x77, 0xd2, 0xd9, 0x4d, 0x79, 0x7e, - 0x6b, 0x6e, 0xce, 0x43, 0xa2, 0x94, 0xf8, 0x92, 0x37, 0xd9, 0x29, 0x2f, 0x41, 0xc9, 0xb8, 0x0b, - 0xf9, 0x4d, 0x7e, 0xe2, 0x6a, 0xde, 0x99, 0x83, 0x42, 0x29, 0xe0, 0x27, 0xdf, 0x98, 0xc3, 0x30, - 0xdc, 0x98, 0xe9, 0x35, 0x27, 0x8b, 0xc1, 0xcf, 0x60, 0x29, 0x71, 0xf5, 0x96, 0x1a, 0x35, 0xe9, - 0xd7, 0x73, 0xcd, 0x69, 0xed, 0xb3, 0x08, 0xc9, 0xc4, 0xcc, 0x82, 0x26, 0x78, 0x7f, 0xca, 0x5c, - 0xd3, 0xbc, 0x9d, 0x05, 0x55, 0x6d, 0x84, 0xf0, 0x74, 0x99, 0xe8, 0xfb, 0xd1, 0x5b, 0xe9, 0x3c, - 0xd2, 0x67, 0x96, 0xe6, 0xdb, 0x19, 0xb1, 0x95, 0xd0, 0x2e, 0x40, 0x0b, 0xd3, 0x1d, 0x4c, 0x03, - 0xe6, 0x23, 0x37, 0x53, 0x4d, 0x1e, 0x21, 0x84, 0x62, 0x6e, 0xcd, 0xc4, 0x0b, 0x05, 0x6c, 0x7e, - 0x5d, 0x80, 0xc5, 0xf0, 0x66, 0xe2, 0x0c, 0x6a, 0xd0, 0x19, 0x14, 0x85, 0xcf, 0x60, 0x29, 0xf1, - 0x46, 0x97, 0xea, 0x33, 0xe9, 0xef, 0x78, 0xb3, 0x1c, 0xf2, 0x53, 0xf9, 0x77, 0x3b, 0xe5, 0x1f, - 0xb7, 0x26, 0x15, 0x96, 0xa4, 0x6b, 0xcc, 0x60, 0xfc, 0xb2, 0x1d, 0xe1, 0xfe, 0xdd, 0x9f, 0xde, - 0xe9, 0xdb, 0xf4, 0x70, 0xb8, 0xcf, 0x44, 0x6f, 0x08, 0xcc, 0xb7, 0x6d, 0x5f, 0xfe, 0xda, 0x08, - 0x4f, 0x60, 0x83, 0x73, 0xda, 0x60, 0xfb, 0x18, 0xec, 0xef, 0x2f, 0xf0, 0xd5, 0xdd, 0xff, 0x05, - 0x00, 0x00, 0xff, 0xff, 0x21, 0x93, 0x69, 0xf9, 0x40, 0x29, 0x00, 0x00, + // 2631 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3a, 0xcb, 0x6f, 0x1c, 0x49, + 0xf9, 0xee, 0x79, 0xd8, 0x9e, 0x6f, 0x1e, 0x1e, 0x57, 0xf2, 0x73, 0xe6, 0x37, 0x49, 0x1c, 0xa7, + 0x37, 0x9b, 0x78, 0xbd, 0x59, 0x3b, 0x71, 0x58, 0x11, 0x91, 0x5d, 0xa2, 0x75, 0x9c, 0x98, 0x11, + 0x76, 0x30, 0x3d, 0xde, 0x5d, 0xc4, 0x0a, 0x8d, 0xda, 0xd3, 0xe5, 0x71, 0x93, 0xe9, 0xee, 0xd9, + 0xae, 0x1e, 0xc7, 0xde, 0xcb, 0xae, 0x16, 0x81, 0x04, 0x02, 0x16, 0xc4, 0x85, 0x03, 0x12, 0x88, + 0x13, 0x12, 0x17, 0x2e, 0x5c, 0xe0, 0xc2, 0x11, 0xc1, 0x85, 0x13, 0x7f, 0x0f, 0xaa, 0x47, 0x57, + 0x3f, 0x67, 0xa6, 0x6d, 0xe7, 0x71, 0x9b, 0xfa, 0xfa, 0x7b, 0xd5, 0x57, 0xdf, 0xb3, 0x6a, 0xa0, + 0x6e, 0xe8, 0x9e, 0xde, 0xe9, 0x3a, 0x8e, 0x6b, 0xac, 0x0e, 0x5c, 0xc7, 0x73, 0xd0, 0xbc, 0x65, + 0xf6, 0x8f, 0x86, 0x84, 0xaf, 0x56, 0xe9, 0xe7, 0x66, 0xa5, 0xeb, 0x58, 0x96, 0x63, 0x73, 0x50, + 0xb3, 0x66, 0xda, 0x1e, 0x76, 0x6d, 0xbd, 0x2f, 0xd6, 0x95, 0x30, 0x41, 0xb3, 0x42, 0xba, 0x87, + 0xd8, 0xd2, 0xf9, 0x4a, 0x3d, 0x86, 0xca, 0x93, 0xfe, 0x90, 0x1c, 0x6a, 0xf8, 0xd3, 0x21, 0x26, + 0x1e, 0xba, 0x03, 0x85, 0x7d, 0x9d, 0xe0, 0x86, 0xb2, 0xa4, 0x2c, 0x97, 0xd7, 0xaf, 0xac, 0x46, + 0x64, 0x09, 0x29, 0x3b, 0xa4, 0xb7, 0xa1, 0x13, 0xac, 0x31, 0x4c, 0x84, 0xa0, 0x60, 0xec, 0xb7, + 0x36, 0x1b, 0xb9, 0x25, 0x65, 0x39, 0xaf, 0xb1, 0xdf, 0x48, 0x85, 0x4a, 0xd7, 0xe9, 0xf7, 0x71, + 0xd7, 0x33, 0x1d, 0xbb, 0xb5, 0xd9, 0x28, 0xb0, 0x6f, 0x11, 0x98, 0xfa, 0x3b, 0x05, 0xaa, 0x42, + 0x34, 0x19, 0x38, 0x36, 0xc1, 0xe8, 0x1e, 0x4c, 0x13, 0x4f, 0xf7, 0x86, 0x44, 0x48, 0xbf, 0x9c, + 0x2a, 0xbd, 0xcd, 0x50, 0x34, 0x81, 0x9a, 0x49, 0x7c, 0x3e, 0x29, 0x1e, 0x2d, 0x02, 0x10, 0xdc, + 0xb3, 0xb0, 0xed, 0xb5, 0x36, 0x49, 0xa3, 0xb0, 0x94, 0x5f, 0xce, 0x6b, 0x21, 0x88, 0xfa, 0x6b, + 0x05, 0xea, 0x6d, 0x7f, 0xe9, 0x5b, 0xe7, 0x22, 0x14, 0xbb, 0xce, 0xd0, 0xf6, 0x98, 0x82, 0x55, + 0x8d, 0x2f, 0xd0, 0x75, 0xa8, 0x74, 0x0f, 0x75, 0xdb, 0xc6, 0xfd, 0x8e, 0xad, 0x5b, 0x98, 0xa9, + 0x52, 0xd2, 0xca, 0x02, 0xf6, 0x54, 0xb7, 0x70, 0x26, 0x8d, 0x96, 0xa0, 0x3c, 0xd0, 0x5d, 0xcf, + 0x8c, 0xd8, 0x2c, 0x0c, 0x52, 0xff, 0xa0, 0xc0, 0xc2, 0x07, 0x84, 0x98, 0x3d, 0x3b, 0xa1, 0xd9, + 0x02, 0x4c, 0xdb, 0x8e, 0x81, 0x5b, 0x9b, 0x4c, 0xb5, 0xbc, 0x26, 0x56, 0xe8, 0x32, 0x94, 0x06, + 0x18, 0xbb, 0x1d, 0xd7, 0xe9, 0xfb, 0x8a, 0xcd, 0x52, 0x80, 0xe6, 0xf4, 0x31, 0xfa, 0x2e, 0xcc, + 0x93, 0x18, 0x23, 0xd2, 0xc8, 0x2f, 0xe5, 0x97, 0xcb, 0xeb, 0x6f, 0xac, 0x26, 0xbc, 0x6c, 0x35, + 0x2e, 0x54, 0x4b, 0x52, 0xab, 0x5f, 0xe4, 0xe0, 0x82, 0xc4, 0xe3, 0xba, 0xd2, 0xdf, 0xd4, 0x72, + 0x04, 0xf7, 0xa4, 0x7a, 0x7c, 0x91, 0xc5, 0x72, 0xd2, 0xe4, 0xf9, 0xb0, 0xc9, 0x33, 0x38, 0x58, + 0xdc, 0x9e, 0xc5, 0x84, 0x3d, 0xd1, 0x35, 0x28, 0xe3, 0xe3, 0x81, 0xe9, 0xe2, 0x8e, 0x67, 0x5a, + 0xb8, 0x31, 0xbd, 0xa4, 0x2c, 0x17, 0x34, 0xe0, 0xa0, 0x3d, 0xd3, 0x0a, 0x7b, 0xe4, 0x4c, 0x66, + 0x8f, 0x54, 0xff, 0xa8, 0xc0, 0xa5, 0xc4, 0x29, 0x09, 0x17, 0xd7, 0xa0, 0xce, 0x76, 0x1e, 0x58, + 0x86, 0x3a, 0x3b, 0x35, 0xf8, 0xcd, 0x71, 0x06, 0x0f, 0xd0, 0xb5, 0x04, 0x7d, 0x48, 0xc9, 0x5c, + 0x76, 0x25, 0x9f, 0xc1, 0xa5, 0x2d, 0xec, 0x09, 0x01, 0xf4, 0x1b, 0x26, 0x67, 0x4f, 0x01, 0xd1, + 0x58, 0xca, 0x25, 0x62, 0xe9, 0x2f, 0x39, 0x19, 0x4b, 0x4c, 0x54, 0xcb, 0x3e, 0x70, 0xd0, 0x15, + 0x28, 0x49, 0x14, 0xe1, 0x15, 0x01, 0x00, 0x7d, 0x1d, 0x8a, 0x54, 0x53, 0xee, 0x12, 0xb5, 0xf5, + 0xeb, 0xe9, 0x7b, 0x0a, 0xf1, 0xd4, 0x38, 0x3e, 0x6a, 0x41, 0x8d, 0x78, 0xba, 0xeb, 0x75, 0x06, + 0x0e, 0x61, 0xe7, 0xcc, 0x1c, 0xa7, 0xbc, 0xae, 0x46, 0x39, 0xc8, 0x14, 0xb9, 0x43, 0x7a, 0xbb, + 0x02, 0x53, 0xab, 0x32, 0x4a, 0x7f, 0x89, 0x1e, 0x43, 0x05, 0xdb, 0x46, 0xc0, 0xa8, 0x90, 0x99, + 0x51, 0x19, 0xdb, 0x86, 0x64, 0x13, 0x9c, 0x4f, 0x31, 0xfb, 0xf9, 0xfc, 0x5c, 0x81, 0x46, 0xf2, + 0x80, 0xce, 0x93, 0x28, 0x1f, 0x70, 0x22, 0xcc, 0x0f, 0x68, 0x6c, 0x84, 0xcb, 0x43, 0xd2, 0x04, + 0x89, 0x6a, 0xc2, 0xff, 0x05, 0xda, 0xb0, 0x2f, 0x2f, 0xcd, 0x59, 0x7e, 0xa4, 0xc0, 0x42, 0x5c, + 0xd6, 0x79, 0xf6, 0xfd, 0x35, 0x28, 0x9a, 0xf6, 0x81, 0xe3, 0x6f, 0x7b, 0x71, 0x4c, 0x9c, 0x51, + 0x59, 0x1c, 0x59, 0xb5, 0xe0, 0xf2, 0x16, 0xf6, 0x5a, 0x36, 0xc1, 0xae, 0xb7, 0x61, 0xda, 0x7d, + 0xa7, 0xb7, 0xab, 0x7b, 0x87, 0xe7, 0x88, 0x91, 0x88, 0xbb, 0xe7, 0x62, 0xee, 0xae, 0xfe, 0x49, + 0x81, 0x2b, 0xe9, 0xf2, 0xc4, 0xd6, 0x9b, 0x30, 0x7b, 0x60, 0xe2, 0xbe, 0x41, 0x6d, 0xa6, 0x30, + 0x9b, 0xc9, 0x35, 0x8d, 0x95, 0x01, 0x45, 0x16, 0x3b, 0xbc, 0x3e, 0xc2, 0x41, 0xdb, 0x9e, 0x6b, + 0xda, 0xbd, 0x6d, 0x93, 0x78, 0x1a, 0xc7, 0x0f, 0xd9, 0x33, 0x9f, 0xdd, 0x33, 0x7f, 0xa6, 0xc0, + 0xe2, 0x16, 0xf6, 0x1e, 0xc9, 0x54, 0x4b, 0xbf, 0x9b, 0xc4, 0x33, 0xbb, 0xe4, 0xe5, 0x36, 0x11, + 0x29, 0x35, 0x53, 0xfd, 0x4a, 0x81, 0x6b, 0x23, 0x95, 0x11, 0xa6, 0x13, 0xa9, 0xc4, 0x4f, 0xb4, + 0xe9, 0xa9, 0xe4, 0xdb, 0xf8, 0xe4, 0x23, 0xbd, 0x3f, 0xc4, 0xbb, 0xba, 0xe9, 0xf2, 0x54, 0x72, + 0xc6, 0xc4, 0xfa, 0x67, 0x05, 0xae, 0x6e, 0x61, 0x6f, 0xd7, 0x2f, 0x33, 0xaf, 0xd1, 0x3a, 0x19, + 0x3a, 0x8a, 0x5f, 0xf2, 0xc3, 0x4c, 0xd5, 0xf6, 0xb5, 0x98, 0x6f, 0x91, 0xc5, 0x41, 0x28, 0x20, + 0x1f, 0xf1, 0x5e, 0x40, 0x18, 0x4f, 0xfd, 0x6b, 0x0e, 0x2a, 0x1f, 0x89, 0xfe, 0x80, 0x95, 0x91, + 0xb8, 0x1d, 0x94, 0x74, 0x3b, 0x84, 0x5a, 0x8a, 0xb4, 0x2e, 0x63, 0x0b, 0xaa, 0x04, 0xe3, 0x67, + 0x67, 0x29, 0x1a, 0x15, 0x4a, 0x28, 0x93, 0xfd, 0x36, 0xcc, 0x0f, 0xed, 0x03, 0xda, 0xd6, 0x62, + 0x43, 0xec, 0x82, 0x77, 0x97, 0x93, 0x33, 0x4f, 0x92, 0x10, 0x7d, 0x0b, 0xe6, 0xe2, 0xbc, 0x8a, + 0x99, 0x78, 0xc5, 0xc9, 0xd4, 0x9f, 0x2a, 0xb0, 0xf0, 0xb1, 0xee, 0x75, 0x0f, 0x37, 0x2d, 0x61, + 0xd1, 0x73, 0xf8, 0xe3, 0xfb, 0x50, 0x3a, 0x12, 0xd6, 0xf3, 0x93, 0xce, 0xb5, 0x14, 0x85, 0xc2, + 0xe7, 0xa4, 0x05, 0x14, 0xea, 0x3f, 0x15, 0xb8, 0xc8, 0x3a, 0x7f, 0x5f, 0xbb, 0x57, 0x1f, 0x19, + 0x13, 0xba, 0x7f, 0x74, 0x13, 0x6a, 0x96, 0xee, 0x3e, 0x6b, 0x07, 0x38, 0x45, 0x86, 0x13, 0x83, + 0xaa, 0xc7, 0x00, 0x62, 0xb5, 0x43, 0x7a, 0x67, 0xd0, 0xff, 0x3e, 0xcc, 0x08, 0xa9, 0x22, 0x48, + 0x26, 0x1d, 0xac, 0x8f, 0xae, 0xfe, 0x4b, 0x81, 0x5a, 0x90, 0xf6, 0x58, 0x28, 0xd4, 0x20, 0x27, + 0x03, 0x20, 0xd7, 0xda, 0x44, 0xef, 0xc3, 0x34, 0x9f, 0xf5, 0x04, 0xef, 0x37, 0xa3, 0xbc, 0xc5, + 0x1c, 0x18, 0xca, 0x9d, 0x0c, 0xa0, 0x09, 0x22, 0x6a, 0x23, 0x99, 0x2a, 0xf8, 0x58, 0x90, 0xd7, + 0x42, 0x10, 0xd4, 0x82, 0xb9, 0x68, 0xa7, 0xe5, 0x3b, 0xfa, 0xd2, 0xa8, 0x14, 0xb1, 0xa9, 0x7b, + 0x3a, 0xcb, 0x10, 0xb5, 0x48, 0xa3, 0x45, 0xd4, 0xff, 0x14, 0xa1, 0x1c, 0xda, 0x65, 0x62, 0x27, + 0xf1, 0x23, 0xcd, 0x4d, 0x4e, 0x76, 0xf9, 0x64, 0xbb, 0xff, 0x26, 0xd4, 0x4c, 0x56, 0x60, 0x3b, + 0xc2, 0x15, 0x59, 0x46, 0x2c, 0x69, 0x55, 0x0e, 0x15, 0x71, 0x81, 0x16, 0xa1, 0x6c, 0x0f, 0xad, + 0x8e, 0x73, 0xd0, 0x71, 0x9d, 0xe7, 0x44, 0xcc, 0x0d, 0x25, 0x7b, 0x68, 0x7d, 0xe7, 0x40, 0x73, + 0x9e, 0x93, 0xa0, 0x35, 0x9d, 0x3e, 0x65, 0x6b, 0xba, 0x08, 0x65, 0x4b, 0x3f, 0xa6, 0x5c, 0x3b, + 0xf6, 0xd0, 0x62, 0x23, 0x45, 0x5e, 0x2b, 0x59, 0xfa, 0xb1, 0xe6, 0x3c, 0x7f, 0x3a, 0xb4, 0xd0, + 0x32, 0xd4, 0xfb, 0x3a, 0xf1, 0x3a, 0xe1, 0x99, 0x64, 0x96, 0xcd, 0x24, 0x35, 0x0a, 0x7f, 0x1c, + 0xcc, 0x25, 0xc9, 0x26, 0xb7, 0x74, 0x8e, 0x26, 0xd7, 0xb0, 0xfa, 0x01, 0x23, 0xc8, 0xde, 0xe4, + 0x1a, 0x56, 0x5f, 0xb2, 0xb9, 0x0f, 0x33, 0xfb, 0xac, 0x6d, 0x21, 0x8d, 0xf2, 0xc8, 0x0c, 0xf5, + 0x84, 0x76, 0x2c, 0xbc, 0xbb, 0xd1, 0x7c, 0x74, 0xf4, 0x1e, 0x94, 0x58, 0xbd, 0x60, 0xb4, 0x95, + 0x4c, 0xb4, 0x01, 0x01, 0x4d, 0x45, 0x06, 0xee, 0x7b, 0x3a, 0xa3, 0xae, 0x8e, 0x4c, 0x45, 0x9b, + 0x14, 0x67, 0xdb, 0xe9, 0xf1, 0x54, 0x24, 0x29, 0xd0, 0x1d, 0xb8, 0xd0, 0x75, 0xb1, 0xee, 0x61, + 0x63, 0xe3, 0xe4, 0x91, 0x63, 0x0d, 0x74, 0xe6, 0x4d, 0x8d, 0xda, 0x92, 0xb2, 0x3c, 0xab, 0xa5, + 0x7d, 0xa2, 0x99, 0xa1, 0x2b, 0x57, 0x4f, 0x5c, 0xc7, 0x6a, 0xcc, 0xf1, 0xcc, 0x10, 0x85, 0xaa, + 0x9f, 0xc3, 0xc5, 0xc0, 0x07, 0x42, 0xf6, 0x4e, 0x1e, 0x9d, 0x72, 0xd6, 0xa3, 0x1b, 0xdf, 0x52, + 0xfe, 0xb6, 0x00, 0x0b, 0x6d, 0xfd, 0x08, 0xbf, 0xfc, 0xee, 0x35, 0x53, 0xc6, 0xdd, 0x86, 0x79, + 0xd6, 0xb0, 0xae, 0x87, 0xf4, 0x19, 0x53, 0x18, 0xc3, 0xc7, 0x9d, 0x24, 0x44, 0x0f, 0x69, 0x45, + 0xc7, 0xdd, 0x67, 0xbb, 0x8e, 0x19, 0x14, 0xc5, 0xab, 0x29, 0x7c, 0x1e, 0x49, 0x2c, 0x2d, 0x4c, + 0x81, 0x76, 0x93, 0xc9, 0x6b, 0x9a, 0x31, 0xb9, 0x35, 0x76, 0x2c, 0x0a, 0xac, 0x1f, 0xcf, 0x61, + 0xa8, 0x01, 0x33, 0xa2, 0xe8, 0xb2, 0xc8, 0x9e, 0xd5, 0xfc, 0x25, 0xda, 0x85, 0x0b, 0x7c, 0x07, + 0x6d, 0xe1, 0xb6, 0x7c, 0xf3, 0xb3, 0x99, 0x36, 0x9f, 0x46, 0x1a, 0xf5, 0xfa, 0xd2, 0x69, 0xbd, + 0x9e, 0xb6, 0xf0, 0x10, 0x18, 0x66, 0xc2, 0x24, 0xfe, 0x4d, 0x98, 0x95, 0xae, 0x9a, 0xcb, 0xec, + 0xaa, 0x92, 0x26, 0x9e, 0x4e, 0xf3, 0xb1, 0x74, 0xaa, 0xfe, 0x5b, 0x81, 0x4a, 0x58, 0x51, 0x9a, + 0xa6, 0x5d, 0xdc, 0x75, 0x5c, 0xa3, 0x83, 0x6d, 0xcf, 0x35, 0x31, 0x9f, 0xf6, 0x0a, 0x5a, 0x95, + 0x43, 0x1f, 0x73, 0x20, 0x45, 0xa3, 0x19, 0x92, 0x78, 0xba, 0x35, 0xe8, 0x1c, 0xd0, 0x40, 0xcc, + 0x71, 0x34, 0x09, 0xa5, 0x71, 0x88, 0xae, 0x43, 0x25, 0x40, 0xf3, 0x1c, 0x26, 0xbf, 0xa0, 0x95, + 0x25, 0x6c, 0xcf, 0x41, 0x37, 0xa0, 0xc6, 0x6c, 0xd3, 0xe9, 0x3b, 0xbd, 0x0e, 0x9d, 0x8c, 0x44, + 0x5d, 0xa8, 0x18, 0x42, 0x2d, 0x6a, 0xf4, 0x28, 0x16, 0x31, 0x3f, 0xc3, 0xa2, 0x32, 0x48, 0xac, + 0xb6, 0xf9, 0x19, 0x56, 0xbf, 0x54, 0xa0, 0x4a, 0xcb, 0xdc, 0x53, 0xc7, 0xc0, 0x7b, 0x67, 0x6c, + 0x0a, 0x32, 0xdc, 0x8a, 0x5d, 0x81, 0x92, 0xdc, 0x81, 0xd8, 0x52, 0x00, 0xa0, 0x23, 0x74, 0x55, + 0x54, 0xb3, 0xb6, 0xbc, 0x25, 0x65, 0xac, 0x14, 0xc6, 0x8a, 0xfd, 0x46, 0xdf, 0x88, 0x5e, 0xb1, + 0xdc, 0x48, 0x8d, 0x1e, 0xc6, 0x84, 0x35, 0x8e, 0x91, 0x52, 0x96, 0x65, 0x36, 0xfb, 0x82, 0x1e, + 0xac, 0x30, 0x05, 0x3b, 0xd8, 0x06, 0xcc, 0xe8, 0x86, 0xe1, 0x62, 0x42, 0x84, 0x1e, 0xfe, 0x92, + 0x7e, 0x39, 0xc2, 0x2e, 0xf1, 0x5d, 0x2c, 0xaf, 0xf9, 0x4b, 0xf4, 0x1e, 0xcc, 0xca, 0x4e, 0x33, + 0x9f, 0xd6, 0x5d, 0x84, 0xf5, 0x14, 0xb3, 0x84, 0xa4, 0x50, 0xbf, 0xca, 0x41, 0x4d, 0x04, 0xef, + 0x86, 0x28, 0x37, 0xe3, 0x9d, 0x7d, 0x03, 0x2a, 0x07, 0x41, 0xf0, 0x8d, 0xbb, 0x33, 0x08, 0xc7, + 0x68, 0x84, 0x66, 0x92, 0xc3, 0x47, 0x0b, 0x5e, 0xe1, 0x5c, 0x05, 0xaf, 0x78, 0xea, 0xd0, 0xff, + 0x00, 0xca, 0x21, 0xc6, 0x2c, 0x69, 0xf1, 0x6b, 0x04, 0x61, 0x0b, 0x7f, 0x49, 0xbf, 0xec, 0x87, + 0x8c, 0x50, 0x92, 0x05, 0x9b, 0xb6, 0xef, 0x97, 0xb6, 0xb0, 0xa7, 0xe1, 0xae, 0x73, 0x84, 0xdd, + 0x93, 0xf3, 0xdf, 0xd0, 0x3c, 0x08, 0x9d, 0x71, 0xc6, 0x69, 0x42, 0x12, 0xa0, 0x07, 0x81, 0x9e, + 0xf9, 0xb4, 0x01, 0x35, 0x9c, 0xc0, 0xc5, 0x09, 0x05, 0x5b, 0xf9, 0x15, 0xbf, 0x6b, 0x8a, 0x6e, + 0xe5, 0xac, 0x35, 0xf2, 0x85, 0x34, 0xa9, 0xea, 0x6f, 0x14, 0xf8, 0xff, 0x2d, 0xec, 0x3d, 0x89, + 0xce, 0x6f, 0xaf, 0x5b, 0x2b, 0x0b, 0x9a, 0x69, 0x4a, 0x9d, 0xe7, 0xd4, 0x9b, 0x30, 0x4b, 0xfc, + 0xa1, 0x96, 0xdf, 0x02, 0xca, 0xb5, 0xfa, 0x13, 0x05, 0x1a, 0x42, 0x0a, 0x93, 0x49, 0xfb, 0xaf, + 0x3e, 0xf6, 0xb0, 0xf1, 0xaa, 0xa7, 0xac, 0xdf, 0x2b, 0x50, 0x0f, 0x27, 0x41, 0x96, 0xc7, 0xde, + 0x85, 0x22, 0x1b, 0x66, 0x85, 0x06, 0x13, 0x9d, 0x95, 0x63, 0xd3, 0x88, 0x62, 0x2d, 0xc3, 0x1e, + 0xf1, 0x93, 0x9c, 0x58, 0x06, 0x99, 0x38, 0x7f, 0xea, 0x4c, 0xac, 0xfe, 0x22, 0x07, 0x8d, 0xa0, + 0x3d, 0x7d, 0xe5, 0xc9, 0x6e, 0x44, 0x6f, 0x93, 0x7f, 0x41, 0xbd, 0x4d, 0xe1, 0xd4, 0x09, 0xee, + 0xef, 0x39, 0x3a, 0x17, 0xfb, 0xf6, 0xd8, 0xed, 0xeb, 0x36, 0x5a, 0x80, 0xe9, 0x41, 0x5f, 0x0f, + 0x2e, 0x87, 0xc4, 0x0a, 0xb5, 0xa1, 0x46, 0x22, 0xf6, 0x12, 0x16, 0x78, 0x3b, 0xcd, 0xfe, 0x23, + 0x4c, 0xac, 0xc5, 0x58, 0xa0, 0xab, 0x00, 0xbc, 0xb1, 0x64, 0xe3, 0x9b, 0x28, 0xcd, 0xfc, 0xa0, + 0xe9, 0xe4, 0x76, 0x1b, 0x10, 0xfd, 0xe0, 0x0c, 0xbd, 0x8e, 0x69, 0x77, 0x08, 0xee, 0x3a, 0xb6, + 0x41, 0x58, 0xbf, 0x51, 0xd4, 0xea, 0xe2, 0x4b, 0xcb, 0x6e, 0x73, 0x38, 0x7a, 0x17, 0x0a, 0xde, + 0xc9, 0x80, 0x77, 0x1a, 0xb5, 0xd4, 0xcc, 0x16, 0xe8, 0xb5, 0x77, 0x32, 0xc0, 0x1a, 0x43, 0xa7, + 0x93, 0x3b, 0x65, 0xe5, 0xb9, 0xfa, 0x11, 0xee, 0xfb, 0xcf, 0x5a, 0x01, 0x84, 0x7a, 0xa2, 0x3f, + 0x01, 0xcf, 0xf0, 0x42, 0x2c, 0x96, 0xea, 0xdf, 0x72, 0x50, 0x0f, 0x58, 0x6a, 0x98, 0x0c, 0xfb, + 0xde, 0x48, 0xfb, 0x8d, 0x1f, 0x0a, 0x26, 0x95, 0xc1, 0x87, 0x50, 0x16, 0xd3, 0xf8, 0x29, 0x0a, + 0x21, 0x70, 0x92, 0xed, 0x31, 0xae, 0x57, 0x7c, 0x41, 0xae, 0x37, 0x7d, 0x6a, 0xd7, 0xfb, 0x01, + 0x5c, 0xda, 0xd1, 0xed, 0xa1, 0xde, 0x0f, 0x5b, 0x90, 0xe7, 0xed, 0x2c, 0xb7, 0x94, 0xd1, 0x53, + 0xcb, 0xc5, 0x4f, 0x4d, 0x25, 0xd0, 0x48, 0xb2, 0x3f, 0x4f, 0x06, 0x66, 0x4a, 0xf9, 0xac, 0xc2, + 0xa5, 0x21, 0x80, 0xa9, 0x0f, 0x59, 0x35, 0x0a, 0x79, 0x3f, 0xcb, 0x3d, 0xe1, 0x5d, 0x85, 0x18, + 0x28, 0x29, 0x0c, 0x7e, 0x9c, 0x63, 0xa5, 0x23, 0xc1, 0xe1, 0x3c, 0x8a, 0xdf, 0x8f, 0x76, 0xae, + 0xea, 0xf8, 0x78, 0x0d, 0xf7, 0xad, 0xcb, 0x30, 0x87, 0x8f, 0x71, 0x77, 0xe8, 0x99, 0x76, 0x8f, + 0xe6, 0x86, 0xa7, 0x8e, 0x70, 0xcc, 0x38, 0x18, 0xdd, 0x80, 0xaa, 0x08, 0x47, 0x81, 0xc7, 0x6f, + 0xcf, 0xa3, 0x40, 0xca, 0xaf, 0xeb, 0xd7, 0x26, 0x81, 0xc7, 0xa7, 0x82, 0x38, 0x58, 0x6d, 0xc3, + 0x82, 0x5f, 0xd1, 0x02, 0x37, 0xdc, 0xc1, 0x9e, 0x3e, 0xa6, 0x07, 0xbb, 0x06, 0x65, 0xde, 0xa9, + 0xf0, 0xa9, 0x84, 0xcf, 0x01, 0xb0, 0x2f, 0xe7, 0xe0, 0x95, 0xbb, 0x30, 0x9f, 0x28, 0x0c, 0xa8, + 0x06, 0xf0, 0xa1, 0xed, 0x8b, 0xaf, 0x4f, 0xa1, 0x0a, 0xcc, 0xfa, 0xf5, 0xb3, 0xae, 0xac, 0xb4, + 0xc3, 0xe9, 0x91, 0xe6, 0x0c, 0x74, 0x09, 0x2e, 0x7c, 0x68, 0x1b, 0xf8, 0xc0, 0xb4, 0xb1, 0x11, + 0x7c, 0xaa, 0x4f, 0xa1, 0x0b, 0x30, 0xd7, 0xb2, 0x6d, 0xec, 0x86, 0x80, 0x0a, 0x05, 0xee, 0x60, + 0xb7, 0x87, 0x43, 0xc0, 0xdc, 0xca, 0x06, 0xcc, 0xc5, 0x0c, 0x8e, 0xe6, 0xa1, 0xca, 0xb9, 0x62, + 0x83, 0x01, 0xea, 0x53, 0xa8, 0x0a, 0xa5, 0xc7, 0xbe, 0x95, 0xeb, 0x0a, 0x5d, 0xca, 0xba, 0x5e, + 0xcf, 0xad, 0xff, 0x63, 0x0e, 0x4a, 0x74, 0x5c, 0x78, 0xe4, 0x38, 0xae, 0x81, 0x06, 0x80, 0x84, + 0xd7, 0x38, 0xb6, 0x7c, 0x00, 0x45, 0x77, 0x46, 0x4c, 0x9e, 0x49, 0x54, 0xe1, 0xa2, 0xcd, 0x9b, + 0x23, 0x28, 0x62, 0xe8, 0xea, 0x14, 0xb2, 0x98, 0x44, 0x9a, 0xa4, 0xf7, 0xcc, 0xee, 0x33, 0xff, + 0x32, 0x70, 0x8c, 0xc4, 0x18, 0xaa, 0x2f, 0x31, 0xf6, 0xae, 0x2a, 0x16, 0xfc, 0xf1, 0xcd, 0x77, + 0x7b, 0x75, 0x0a, 0x7d, 0x0a, 0x17, 0xb7, 0xb0, 0x17, 0xbc, 0xb7, 0xf8, 0x02, 0xd7, 0x47, 0x0b, + 0x4c, 0x20, 0x9f, 0x52, 0xe4, 0x36, 0x14, 0x59, 0x37, 0x85, 0xd2, 0x92, 0x5a, 0xf8, 0x5f, 0x40, + 0xcd, 0xa5, 0xd1, 0x08, 0x92, 0xdb, 0x0f, 0x61, 0x2e, 0xf6, 0x2f, 0x07, 0xf4, 0x56, 0x0a, 0x59, + 0xfa, 0xff, 0x55, 0x9a, 0x2b, 0x59, 0x50, 0xa5, 0xac, 0x1e, 0xd4, 0xa2, 0xaf, 0x42, 0x68, 0x39, + 0x85, 0x3e, 0xf5, 0x85, 0xba, 0xf9, 0x56, 0x06, 0x4c, 0x29, 0xc8, 0x82, 0x7a, 0xfc, 0xd5, 0x1d, + 0xad, 0x8c, 0x65, 0x10, 0x75, 0xb7, 0xb7, 0x33, 0xe1, 0x4a, 0x71, 0x27, 0xcc, 0x09, 0x12, 0xaf, + 0xbe, 0x68, 0x35, 0x9d, 0xcd, 0xa8, 0xe7, 0xe8, 0xe6, 0x5a, 0x66, 0x7c, 0x29, 0xfa, 0x4b, 0x3e, + 0xc5, 0xa5, 0xbd, 0x9c, 0xa2, 0xbb, 0xe9, 0xec, 0xc6, 0x3c, 0xf9, 0x36, 0xd7, 0x4f, 0x43, 0x22, + 0x95, 0xf8, 0x9c, 0x8d, 0x5f, 0x29, 0xaf, 0x8f, 0xf1, 0xb8, 0xf3, 0xf9, 0x8d, 0x7e, 0x56, 0x6d, + 0xde, 0x3d, 0x05, 0x85, 0x54, 0xc0, 0x89, 0xff, 0xaf, 0xc1, 0x0f, 0xc3, 0xb5, 0x89, 0x5e, 0x73, + 0xb6, 0x18, 0xfc, 0x04, 0xe6, 0x62, 0x97, 0xb2, 0xa9, 0x51, 0x93, 0x7e, 0x71, 0xdb, 0x1c, 0x57, + 0x1d, 0x79, 0x48, 0xc6, 0xa6, 0x59, 0x34, 0xc2, 0xfb, 0x53, 0x26, 0xde, 0xe6, 0x4a, 0x16, 0x54, + 0xb9, 0x11, 0xc2, 0xd2, 0x65, 0x6c, 0x22, 0x44, 0xb7, 0xd3, 0x79, 0xa4, 0x4f, 0xb3, 0xcd, 0x77, + 0x32, 0x62, 0x4b, 0xa1, 0x1d, 0x80, 0x2d, 0xec, 0xed, 0x60, 0xcf, 0xa5, 0x3e, 0x72, 0x33, 0xd5, + 0xe4, 0x01, 0x82, 0x2f, 0xe6, 0xd6, 0x44, 0x3c, 0x29, 0xe0, 0x7b, 0x80, 0xfc, 0x9a, 0x14, 0xba, + 0xf3, 0x7f, 0x63, 0x6c, 0x83, 0xc1, 0xbb, 0xe4, 0x49, 0x67, 0x63, 0x41, 0x3d, 0xde, 0xbd, 0xa5, + 0x66, 0x96, 0x11, 0x1d, 0x64, 0x6a, 0x66, 0x19, 0xd5, 0x0e, 0xca, 0xe3, 0x89, 0x17, 0xe5, 0xdb, + 0xa3, 0xa2, 0x34, 0xad, 0xbd, 0x1b, 0x75, 0x3c, 0x23, 0x5a, 0x39, 0x75, 0x6a, 0xfd, 0xbf, 0x05, + 0x98, 0xf5, 0x6f, 0xfc, 0x5e, 0x43, 0x05, 0x7f, 0x0d, 0x25, 0xf5, 0x13, 0x98, 0x8b, 0xbd, 0xaa, + 0xa7, 0x46, 0x5c, 0xfa, 0xcb, 0xfb, 0x24, 0x97, 0xf9, 0x58, 0xfc, 0x41, 0x56, 0x46, 0xd7, 0xad, + 0x51, 0x65, 0x39, 0x1e, 0x58, 0x13, 0x18, 0xbf, 0xf4, 0x30, 0x7a, 0x0a, 0x10, 0x72, 0xf3, 0xf1, + 0x73, 0x2b, 0xed, 0x8f, 0x27, 0x28, 0xbc, 0x71, 0xef, 0xfb, 0x77, 0x7b, 0xa6, 0x77, 0x38, 0xdc, + 0xa7, 0x5f, 0xd6, 0x38, 0xea, 0x3b, 0xa6, 0x23, 0x7e, 0xad, 0xf9, 0x27, 0xba, 0xc6, 0xa8, 0xd7, + 0xa8, 0x80, 0xc1, 0xfe, 0xfe, 0x34, 0x5b, 0xdd, 0xfb, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x6f, + 0x95, 0xc4, 0x82, 0x42, 0x2d, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -2905,6 +3140,9 @@ type DataCoordClient interface { GetFlushedSegments(ctx context.Context, in *GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*GetFlushedSegmentsResponse, error) // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + CompleteCompaction(ctx context.Context, in *CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) + ManualCompaction(ctx context.Context, in *ManualCompactionRequest, opts ...grpc.CallOption) (*ManualCompactionResponse, error) + GetCompactionState(ctx context.Context, in *GetCompactionStateRequest, opts ...grpc.CallOption) (*GetCompactionStateResponse, error) } type dataCoordClient struct { @@ -3050,6 +3288,33 @@ func (c *dataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetric return out, nil } +func (c *dataCoordClient) CompleteCompaction(ctx context.Context, in *CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/CompleteCompaction", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ManualCompaction(ctx context.Context, in *ManualCompactionRequest, opts ...grpc.CallOption) (*ManualCompactionResponse, error) { + out := new(ManualCompactionResponse) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/ManualCompaction", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) GetCompactionState(ctx context.Context, in *GetCompactionStateRequest, opts ...grpc.CallOption) (*GetCompactionStateResponse, error) { + out := new(GetCompactionStateResponse) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/GetCompactionState", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DataCoordServer is the server API for DataCoord service. type DataCoordServer interface { GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) @@ -3068,6 +3333,9 @@ type DataCoordServer interface { GetFlushedSegments(context.Context, *GetFlushedSegmentsRequest) (*GetFlushedSegmentsResponse, error) // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + CompleteCompaction(context.Context, *CompactionResult) (*commonpb.Status, error) + ManualCompaction(context.Context, *ManualCompactionRequest) (*ManualCompactionResponse, error) + GetCompactionState(context.Context, *GetCompactionStateRequest) (*GetCompactionStateResponse, error) } // UnimplementedDataCoordServer can be embedded to have forward compatible implementations. @@ -3119,6 +3387,15 @@ func (*UnimplementedDataCoordServer) GetFlushedSegments(ctx context.Context, req func (*UnimplementedDataCoordServer) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") } +func (*UnimplementedDataCoordServer) CompleteCompaction(ctx context.Context, req *CompactionResult) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method CompleteCompaction not implemented") +} +func (*UnimplementedDataCoordServer) ManualCompaction(ctx context.Context, req *ManualCompactionRequest) (*ManualCompactionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ManualCompaction not implemented") +} +func (*UnimplementedDataCoordServer) GetCompactionState(ctx context.Context, req *GetCompactionStateRequest) (*GetCompactionStateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetCompactionState not implemented") +} func RegisterDataCoordServer(s *grpc.Server, srv DataCoordServer) { s.RegisterService(&_DataCoord_serviceDesc, srv) @@ -3394,6 +3671,60 @@ func _DataCoord_GetMetrics_Handler(srv interface{}, ctx context.Context, dec fun return interceptor(ctx, in, info, handler) } +func _DataCoord_CompleteCompaction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CompactionResult) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).CompleteCompaction(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/CompleteCompaction", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).CompleteCompaction(ctx, req.(*CompactionResult)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ManualCompaction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ManualCompactionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ManualCompaction(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/ManualCompaction", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ManualCompaction(ctx, req.(*ManualCompactionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_GetCompactionState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetCompactionStateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).GetCompactionState(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/GetCompactionState", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).GetCompactionState(ctx, req.(*GetCompactionStateRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _DataCoord_serviceDesc = grpc.ServiceDesc{ ServiceName: "milvus.proto.data.DataCoord", HandlerType: (*DataCoordServer)(nil), @@ -3458,6 +3789,18 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{ MethodName: "GetMetrics", Handler: _DataCoord_GetMetrics_Handler, }, + { + MethodName: "CompleteCompaction", + Handler: _DataCoord_CompleteCompaction_Handler, + }, + { + MethodName: "ManualCompaction", + Handler: _DataCoord_ManualCompaction_Handler, + }, + { + MethodName: "GetCompactionState", + Handler: _DataCoord_GetCompactionState_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "data_coord.proto", @@ -3473,6 +3816,7 @@ type DataNodeClient interface { FlushSegments(ctx context.Context, in *FlushSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) + Compaction(ctx context.Context, in *CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) } type dataNodeClient struct { @@ -3528,6 +3872,15 @@ func (c *dataNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetrics return out, nil } +func (c *dataNodeClient) Compaction(ctx context.Context, in *CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/Compaction", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DataNodeServer is the server API for DataNode service. type DataNodeServer interface { GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) @@ -3536,6 +3889,7 @@ type DataNodeServer interface { FlushSegments(context.Context, *FlushSegmentsRequest) (*commonpb.Status, error) // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + Compaction(context.Context, *CompactionPlan) (*commonpb.Status, error) } // UnimplementedDataNodeServer can be embedded to have forward compatible implementations. @@ -3557,6 +3911,9 @@ func (*UnimplementedDataNodeServer) FlushSegments(ctx context.Context, req *Flus func (*UnimplementedDataNodeServer) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") } +func (*UnimplementedDataNodeServer) Compaction(ctx context.Context, req *CompactionPlan) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method Compaction not implemented") +} func RegisterDataNodeServer(s *grpc.Server, srv DataNodeServer) { s.RegisterService(&_DataNode_serviceDesc, srv) @@ -3652,6 +4009,24 @@ func _DataNode_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func return interceptor(ctx, in, info, handler) } +func _DataNode_Compaction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CompactionPlan) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).Compaction(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataNode/Compaction", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).Compaction(ctx, req.(*CompactionPlan)) + } + return interceptor(ctx, in, info, handler) +} + var _DataNode_serviceDesc = grpc.ServiceDesc{ ServiceName: "milvus.proto.data.DataNode", HandlerType: (*DataNodeServer)(nil), @@ -3676,6 +4051,10 @@ var _DataNode_serviceDesc = grpc.ServiceDesc{ MethodName: "GetMetrics", Handler: _DataNode_GetMetrics_Handler, }, + { + MethodName: "Compaction", + Handler: _DataNode_Compaction_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "data_coord.proto", diff --git a/internal/proxy/data_coord_mock_test.go b/internal/proxy/data_coord_mock_test.go index f094cd567c..ae5fbd86bd 100644 --- a/internal/proxy/data_coord_mock_test.go +++ b/internal/proxy/data_coord_mock_test.go @@ -164,6 +164,18 @@ func (coord *DataCoordMock) GetMetrics(ctx context.Context, req *milvuspb.GetMet }, nil } +func (coord *DataCoordMock) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { + return &commonpb.Status{}, nil +} + +func (coord *DataCoordMock) ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) { + return &datapb.ManualCompactionResponse{}, nil +} + +func (coord *DataCoordMock) GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) { + return &datapb.GetCompactionStateResponse{}, nil +} + func NewDataCoordMock() *DataCoordMock { return &DataCoordMock{ nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()), diff --git a/internal/types/types.go b/internal/types/types.go index 578f1b854b..9fc85360be 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -59,6 +59,8 @@ type DataNode interface { // GetMetrics gets the metrics about DataNode. GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + + Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) } // DataNodeComponent is used by grpc server of DataNode @@ -220,6 +222,9 @@ type DataCoord interface { // GetMetrics gets the metrics about DataCoord. GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) + ManualCompaction(ctx context.Context, req *datapb.ManualCompactionRequest) (*datapb.ManualCompactionResponse, error) + GetCompactionState(ctx context.Context, req *datapb.GetCompactionStateRequest) (*datapb.GetCompactionStateResponse, error) } // IndexNode is the interface `indexnode` package implements