Add compaction ()

issue: 
Signed-off-by: sunby <bingyi.sun@zilliz.com>
pull/11354/head
sunby 2021-11-05 22:25:00 +08:00 committed by GitHub
parent b7644c0c03
commit c26cd79532
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 3908 additions and 259 deletions

View File

@ -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

View File

@ -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
}

View File

@ -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
}
}

View File

@ -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
}

View File

@ -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)
})
}
}

View File

@ -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)
})
}
}

View File

@ -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)
}

View File

@ -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)
})
}
}

View File

@ -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 {

View File

@ -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)
})
}
}

View File

@ -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")
}

View File

@ -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)
}

View File

@ -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 {

View File

@ -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
}

View File

@ -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)

View File

@ -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
}

View File

@ -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()

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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) {

View File

@ -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)
}

View File

@ -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()

View File

@ -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
}

View File

@ -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) {

View File

@ -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)
}

View File

@ -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

View File

@ -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

View File

@ -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;
}
}

View File

@ -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",

View File

@ -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()),

View File

@ -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