mirror of https://github.com/milvus-io/milvus.git
parent
b7644c0c03
commit
c26cd79532
configs/advanced
internal
datanode
distributed
datacoord
datanode
proxy
proto
types
|
@ -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
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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",
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue