refactor compaction concurency logic (#18660)

Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com>

Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com>
This commit is contained in:
Enwei Jiao 2022-08-23 15:50:52 +08:00 committed by GitHub
parent dad060af2e
commit d9c7519107
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 1175 additions and 749 deletions

3
go.mod
View File

@ -49,7 +49,7 @@ require (
go.uber.org/automaxprocs v1.4.0 go.uber.org/automaxprocs v1.4.0
go.uber.org/zap v1.17.0 go.uber.org/zap v1.17.0
golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 golang.org/x/crypto v0.0.0-20210921155107-089bfa567519
golang.org/x/exp v0.0.0-20211216164055-b2b84827b756 golang.org/x/exp v0.0.0-20220303212507-bbda1eaf7a17
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sync v0.0.0-20210220032951-036812b2e83c
google.golang.org/grpc v1.46.0 google.golang.org/grpc v1.46.0
google.golang.org/grpc/examples v0.0.0-20220617181431-3e7b97febc7f google.golang.org/grpc/examples v0.0.0-20220617181431-3e7b97febc7f
@ -139,6 +139,7 @@ require (
github.com/prometheus/procfs v0.6.0 // indirect github.com/prometheus/procfs v0.6.0 // indirect
github.com/quasilyte/go-ruleguard/dsl v0.3.21 // indirect github.com/quasilyte/go-ruleguard/dsl v0.3.21 // indirect
github.com/rs/xid v1.2.1 // indirect github.com/rs/xid v1.2.1 // indirect
github.com/samber/lo v1.27.0 // indirect
github.com/sirupsen/logrus v1.8.1 // indirect github.com/sirupsen/logrus v1.8.1 // indirect
github.com/soheilhy/cmux v0.1.5 // indirect github.com/soheilhy/cmux v0.1.5 // indirect
github.com/spf13/afero v1.6.0 // indirect github.com/spf13/afero v1.6.0 // indirect

4
go.sum
View File

@ -605,6 +605,8 @@ github.com/rs/zerolog v1.21.0/go.mod h1:ZPhntP/xmq1nnND05hhpAh2QMhSsA4UN3MGZ6O2J
github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w=
github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
github.com/samber/lo v1.27.0 h1:GOyDWxsblvqYobqsmUuMddPa2/mMzkKyojlXol4+LaQ=
github.com/samber/lo v1.27.0/go.mod h1:it33p9UtPMS7z72fP4gw/EIfQB2eI8ke7GR2wc6+Rhg=
github.com/sanity-io/litter v1.2.0/go.mod h1:JF6pZUFgu2Q0sBZ+HSV35P8TVPI1TTzEwyu9FXAw2W4= github.com/sanity-io/litter v1.2.0/go.mod h1:JF6pZUFgu2Q0sBZ+HSV35P8TVPI1TTzEwyu9FXAw2W4=
github.com/santhosh-tekuri/jsonschema/v5 v5.0.0/go.mod h1:FKdcjfQW6rpZSnxxUvEA5H/cDPdvJ/SZJQLWWXWGrZ0= github.com/santhosh-tekuri/jsonschema/v5 v5.0.0/go.mod h1:FKdcjfQW6rpZSnxxUvEA5H/cDPdvJ/SZJQLWWXWGrZ0=
github.com/sbinet/npyio v0.6.0 h1:IyqqQIzRjDym9xnIXsToCKei/qCzxDP+Y74KoMlMgXo= github.com/sbinet/npyio v0.6.0 h1:IyqqQIzRjDym9xnIXsToCKei/qCzxDP+Y74KoMlMgXo=
@ -785,6 +787,8 @@ golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EH
golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU=
golang.org/x/exp v0.0.0-20211216164055-b2b84827b756 h1:/5Bs7sWi0i3rOVO5KnM55OwugpsD4bRW1zywKoZjbkI= golang.org/x/exp v0.0.0-20211216164055-b2b84827b756 h1:/5Bs7sWi0i3rOVO5KnM55OwugpsD4bRW1zywKoZjbkI=
golang.org/x/exp v0.0.0-20211216164055-b2b84827b756/go.mod h1:b9TAUYHmRtqA6klRHApnXMnj+OyLce4yF5cZCUbk2ps= golang.org/x/exp v0.0.0-20211216164055-b2b84827b756/go.mod h1:b9TAUYHmRtqA6klRHApnXMnj+OyLce4yF5cZCUbk2ps=
golang.org/x/exp v0.0.0-20220303212507-bbda1eaf7a17 h1:3MTrJm4PyNL9NBqvYDSj3DHl46qQakyfqfWo4jgfaEM=
golang.org/x/exp v0.0.0-20220303212507-bbda1eaf7a17/go.mod h1:lgLbSvA5ygNOMpwM/9anMpWVlVJ7Z+cHWq/eFuinpGE=
golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs=
golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=

View File

@ -24,6 +24,7 @@ import (
"time" "time"
"github.com/milvus-io/milvus/internal/log" "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/datapb"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"go.uber.org/zap" "go.uber.org/zap"
@ -32,9 +33,8 @@ import (
// TODO this num should be determined by resources of datanode, for now, we set to a fixed value for simple // TODO this num should be determined by resources of datanode, for now, we set to a fixed value for simple
// TODO we should split compaction into different priorities, small compaction helps to merge segment, large compaction helps to handle delta and expiration of large segments // TODO we should split compaction into different priorities, small compaction helps to merge segment, large compaction helps to handle delta and expiration of large segments
const ( const (
maxParallelCompactionTaskNum = 100 maxParallelCompactionTaskNum = 100
compactionTimeout = 10 * time.Second rpcCompactionTimeout = 10 * time.Second
compactionExpirationCheckInterval = 60 * time.Second
) )
type compactionPlanContext interface { type compactionPlanContext interface {
@ -42,12 +42,10 @@ type compactionPlanContext interface {
stop() stop()
// execCompactionPlan start to execute plan and return immediately // execCompactionPlan start to execute plan and return immediately
execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) error execCompactionPlan(signal *compactionSignal, 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 return compaction task. If planId does not exist, return nil.
getCompaction(planID int64) *compactionTask getCompaction(planID int64) *compactionTask
// expireCompaction set the compaction state to expired // updateCompaction set the compaction state to timeout or completed
expireCompaction(ts Timestamp) error updateCompaction(ts Timestamp) error
// isFull return true if the task pool is full // isFull return true if the task pool is full
isFull() bool isFull() bool
// get compaction tasks by signal id // get compaction tasks by signal id
@ -59,6 +57,7 @@ type compactionTaskState int8
const ( const (
executing compactionTaskState = iota + 1 executing compactionTaskState = iota + 1
completed completed
failed
timeout timeout
) )
@ -102,23 +101,26 @@ type compactionPlanHandler struct {
wg sync.WaitGroup wg sync.WaitGroup
flushCh chan UniqueID flushCh chan UniqueID
segRefer *SegmentReferenceManager segRefer *SegmentReferenceManager
parallelCh map[int64]chan struct{}
} }
func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta *meta, func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta *meta,
allocator allocator, flush chan UniqueID, segRefer *SegmentReferenceManager) *compactionPlanHandler { allocator allocator, flush chan UniqueID, segRefer *SegmentReferenceManager) *compactionPlanHandler {
return &compactionPlanHandler{ return &compactionPlanHandler{
plans: make(map[int64]*compactionTask), plans: make(map[int64]*compactionTask),
chManager: cm, chManager: cm,
meta: meta, meta: meta,
sessions: sessions, sessions: sessions,
allocator: allocator, allocator: allocator,
flushCh: flush, flushCh: flush,
segRefer: segRefer, segRefer: segRefer,
parallelCh: make(map[int64]chan struct{}),
} }
} }
func (c *compactionPlanHandler) start() { func (c *compactionPlanHandler) start() {
ticker := time.NewTicker(compactionExpirationCheckInterval) interval := time.Duration(Params.DataCoordCfg.CompactionCheckIntervalInSeconds) * time.Second
ticker := time.NewTicker(interval)
c.quit = make(chan struct{}) c.quit = make(chan struct{})
c.wg.Add(1) c.wg.Add(1)
@ -139,7 +141,7 @@ func (c *compactionPlanHandler) start() {
continue continue
} }
cancel() cancel()
_ = c.expireCompaction(ts) _ = c.updateCompaction(ts)
} }
} }
}() }()
@ -162,17 +164,40 @@ func (c *compactionPlanHandler) execCompactionPlan(signal *compactionSignal, pla
c.setSegmentsCompacting(plan, true) c.setSegmentsCompacting(plan, true)
// FIXME: check response of compaction call and restore segment state if failed go func() {
c.sessions.Compaction(nodeID, plan) log.Debug("acquire queue", zap.Int64("nodeID", nodeID), zap.Int64("planID", plan.GetPlanID()))
c.acquireQueue(nodeID)
task := &compactionTask{ ts, err := c.allocator.allocTimestamp(context.TODO())
triggerInfo: signal, if err != nil {
plan: plan, log.Warn("Alloc start time for CompactionPlan failed", zap.Int64("planID", plan.GetPlanID()))
state: executing, return
dataNodeID: nodeID, }
} plan.StartTime = ts
c.plans[plan.PlanID] = task
c.executingTaskNum++ c.mu.Lock()
task := &compactionTask{
triggerInfo: signal,
plan: plan,
state: executing,
dataNodeID: nodeID,
}
c.plans[plan.PlanID] = task
c.executingTaskNum++
c.mu.Unlock()
err = c.sessions.Compaction(nodeID, plan)
if err != nil {
log.Warn("Try to Compaction but DataNode rejected", zap.Any("TargetNodeId", nodeID), zap.Any("planId", plan.GetPlanID()))
c.mu.Lock()
delete(c.plans, plan.PlanID)
c.executingTaskNum--
c.mu.Unlock()
return
}
log.Debug("start compaction", zap.Int64("nodeID", nodeID), zap.Int64("planID", plan.GetPlanID()))
}()
return nil return nil
} }
@ -182,11 +207,9 @@ func (c *compactionPlanHandler) setSegmentsCompacting(plan *datapb.CompactionPla
} }
} }
// completeCompaction record the result of a compaction // complete a compaction task
// not threadsafe, only can be used internally
func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResult) error { func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResult) error {
c.mu.Lock()
defer c.mu.Unlock()
planID := result.PlanID planID := result.PlanID
if _, ok := c.plans[planID]; !ok { if _, ok := c.plans[planID]; !ok {
return fmt.Errorf("plan %d is not found", planID) return fmt.Errorf("plan %d is not found", planID)
@ -219,6 +242,8 @@ func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResu
} }
// TODO: when to clean task list // TODO: when to clean task list
nodeID := c.plans[planID].dataNodeID
c.releaseQueue(nodeID)
return nil return nil
} }
@ -241,21 +266,35 @@ func (c *compactionPlanHandler) getCompaction(planID int64) *compactionTask {
} }
// expireCompaction set the compaction state to expired // expireCompaction set the compaction state to expired
func (c *compactionPlanHandler) expireCompaction(ts Timestamp) error { func (c *compactionPlanHandler) updateCompaction(ts Timestamp) error {
planStates := c.sessions.GetCompactionState()
c.mu.Lock() c.mu.Lock()
defer c.mu.Unlock() defer c.mu.Unlock()
tasks := c.getExecutingCompactions() tasks := c.getExecutingCompactions()
for _, task := range tasks { for _, task := range tasks {
if !c.isTimeout(ts, task.plan.GetStartTime(), task.plan.GetTimeoutInSeconds()) { stateResult, ok := planStates[task.plan.PlanID]
continue state := stateResult.GetState()
planID := task.plan.PlanID
// check wether the state of CompactionPlan is working
if ok {
// check wether the CompactionPlan is timeout
if state == commonpb.CompactionState_Executing && !c.isTimeout(ts, task.plan.GetStartTime(), task.plan.GetTimeoutInSeconds()) {
continue
}
if state == commonpb.CompactionState_Completed {
c.completeCompaction(stateResult.GetResult())
continue
}
c.plans[planID] = c.plans[planID].shadowClone(setState(timeout))
} }
c.plans[planID] = c.plans[planID].shadowClone(setState(failed))
c.setSegmentsCompacting(task.plan, false) c.setSegmentsCompacting(task.plan, false)
planID := task.plan.PlanID
c.plans[planID] = c.plans[planID].shadowClone(setState(timeout))
c.executingTaskNum-- c.executingTaskNum--
c.releaseQueue(task.dataNodeID)
} }
return nil return nil
@ -267,6 +306,29 @@ func (c *compactionPlanHandler) isTimeout(now Timestamp, start Timestamp, timeou
return int32(ts.Sub(startTime).Seconds()) >= timeout return int32(ts.Sub(startTime).Seconds()) >= timeout
} }
func (c *compactionPlanHandler) acquireQueue(nodeID int64) {
c.mu.Lock()
_, ok := c.parallelCh[nodeID]
if !ok {
c.parallelCh[nodeID] = make(chan struct{}, calculateParallel())
}
c.mu.Unlock()
c.mu.RLock()
ch := c.parallelCh[nodeID]
c.mu.RUnlock()
ch <- struct{}{}
}
func (c *compactionPlanHandler) releaseQueue(nodeID int64) {
log.Debug("try to release queue", zap.Int64("nodeID", nodeID))
ch, ok := c.parallelCh[nodeID]
if !ok {
return
}
<-ch
}
// isFull return true if the task pool is full // isFull return true if the task pool is full
func (c *compactionPlanHandler) isFull() bool { func (c *compactionPlanHandler) isFull() bool {
c.mu.RLock() c.mu.RLock()
@ -285,13 +347,17 @@ func (c *compactionPlanHandler) getExecutingCompactions() []*compactionTask {
return tasks return tasks
} }
// get compaction tasks by signal id // get compaction tasks by signal id; if signalID == 0 return all tasks
func (c *compactionPlanHandler) getCompactionTasksBySignalID(signalID int64) []*compactionTask { func (c *compactionPlanHandler) getCompactionTasksBySignalID(signalID int64) []*compactionTask {
c.mu.RLock() c.mu.RLock()
defer c.mu.RUnlock() defer c.mu.RUnlock()
var tasks []*compactionTask var tasks []*compactionTask
for _, t := range c.plans { for _, t := range c.plans {
if signalID == 0 {
tasks = append(tasks, t)
continue
}
if t.triggerInfo.id != signalID { if t.triggerInfo.id != signalID {
continue continue
} }
@ -313,3 +379,13 @@ func setResult(result *datapb.CompactionResult) compactionTaskOpt {
task.result = result task.result = result
} }
} }
// 0.5*min(8, NumCPU/2)
func calculateParallel() int {
return 2
//cores := runtime.NumCPU()
//if cores < 16 {
//return 4
//}
//return cores / 2
}

View File

@ -22,14 +22,15 @@ import (
"time" "time"
memkv "github.com/milvus-io/milvus/internal/kv/mem" memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"go.uber.org/zap"
) )
// TODO not completed
func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) { func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
ch := make(chan interface{}, 1) ch := make(chan interface{}, 1)
type fields struct { type fields struct {
@ -81,9 +82,11 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
c := &compactionPlanHandler{ c := &compactionPlanHandler{
plans: tt.fields.plans, plans: tt.fields.plans,
sessions: tt.fields.sessions, sessions: tt.fields.sessions,
chManager: tt.fields.chManager, chManager: tt.fields.chManager,
parallelCh: make(map[int64]chan struct{}),
allocator: newMockAllocator(),
} }
err := c.execCompactionPlan(tt.args.signal, tt.args.plan) err := c.execCompactionPlan(tt.args.signal, tt.args.plan)
assert.Equal(t, tt.err, err) assert.Equal(t, tt.err, err)
@ -92,11 +95,69 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
task := c.getCompaction(tt.args.plan.PlanID) task := c.getCompaction(tt.args.plan.PlanID)
assert.Equal(t, tt.args.plan, task.plan) assert.Equal(t, tt.args.plan, task.plan)
assert.Equal(t, tt.args.signal, task.triggerInfo) assert.Equal(t, tt.args.signal, task.triggerInfo)
assert.Equal(t, 1, c.executingTaskNum)
} }
}) })
} }
} }
func Test_compactionPlanHandler_execWithParallels(t *testing.T) {
Params.DataCoordCfg.CompactionCheckIntervalInSeconds = 1
c := &compactionPlanHandler{
plans: map[int64]*compactionTask{},
sessions: &SessionManager{
sessions: struct {
sync.RWMutex
data map[int64]*Session
}{
data: map[int64]*Session{
1: {client: &mockDataNodeClient{ch: make(chan interface{}, 1)}},
},
},
},
chManager: &ChannelManager{
store: &ChannelStore{
channelsInfo: map[int64]*NodeChannelInfo{
1: {NodeID: 1, Channels: []*channel{{Name: "ch1"}}},
},
},
},
parallelCh: make(map[int64]chan struct{}),
allocator: newMockAllocator(),
}
signal := &compactionSignal{id: 100}
plan1 := &datapb.CompactionPlan{PlanID: 1, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}
plan2 := &datapb.CompactionPlan{PlanID: 2, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}
plan3 := &datapb.CompactionPlan{PlanID: 3, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}
c.parallelCh[1] = make(chan struct{}, 2)
go func() {
c.execCompactionPlan(signal, plan1)
c.execCompactionPlan(signal, plan2)
c.execCompactionPlan(signal, plan3)
}()
<-c.parallelCh[1]
<-c.parallelCh[1]
<-c.parallelCh[1]
tasks := c.getCompactionTasksBySignalID(0)
max, min := uint64(0), uint64(0)
for _, v := range tasks {
if max < v.plan.GetStartTime() {
max = v.plan.GetStartTime()
}
if min > v.plan.GetStartTime() {
min = v.plan.GetStartTime()
}
}
log.Debug("start time", zap.Uint64("min", min), zap.Uint64("max", max))
assert.Less(t, uint64(2), max-min)
}
func Test_compactionPlanHandler_completeCompaction(t *testing.T) { func Test_compactionPlanHandler_completeCompaction(t *testing.T) {
type fields struct { type fields struct {
plans map[int64]*compactionTask plans map[int64]*compactionTask
@ -378,7 +439,7 @@ func Test_compactionPlanHandler_getCompaction(t *testing.T) {
} }
} }
func Test_compactionPlanHandler_expireCompaction(t *testing.T) { func Test_compactionPlanHandler_updateCompaction(t *testing.T) {
type fields struct { type fields struct {
plans map[int64]*compactionTask plans map[int64]*compactionTask
sessions *SessionManager sessions *SessionManager
@ -398,11 +459,12 @@ func Test_compactionPlanHandler_expireCompaction(t *testing.T) {
unexpired []int64 unexpired []int64
}{ }{
{ {
"test expire compaction task", "test update compaction task",
fields{ fields{
plans: map[int64]*compactionTask{ plans: map[int64]*compactionTask{
1: { 1: {
state: executing, state: executing,
dataNodeID: 1,
plan: &datapb.CompactionPlan{ plan: &datapb.CompactionPlan{
PlanID: 1, PlanID: 1,
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0), StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0),
@ -413,13 +475,32 @@ func Test_compactionPlanHandler_expireCompaction(t *testing.T) {
}, },
}, },
2: { 2: {
state: executing, state: executing,
dataNodeID: 2,
plan: &datapb.CompactionPlan{ plan: &datapb.CompactionPlan{
PlanID: 2, PlanID: 2,
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0), StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0),
TimeoutInSeconds: 1, TimeoutInSeconds: 1,
}, },
}, },
3: {
state: completed,
dataNodeID: 2,
plan: &datapb.CompactionPlan{
PlanID: 3,
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0),
TimeoutInSeconds: 1,
},
},
4: {
state: executing,
dataNodeID: 2,
plan: &datapb.CompactionPlan{
PlanID: 4,
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0) - 200*1000,
TimeoutInSeconds: 1,
},
},
}, },
meta: &meta{ meta: &meta{
segments: &SegmentsInfo{ segments: &SegmentsInfo{
@ -428,11 +509,27 @@ func Test_compactionPlanHandler_expireCompaction(t *testing.T) {
}, },
}, },
}, },
sessions: &SessionManager{
sessions: struct {
sync.RWMutex
data map[int64]*Session
}{
data: map[int64]*Session{
1: {client: &mockDataNodeClient{
compactionStateResp: &datapb.CompactionStateResponse{
Results: []*datapb.CompactionStateResult{
{PlanID: 1, State: commonpb.CompactionState_Executing},
},
},
}},
},
},
},
}, },
args{ts: tsoutil.ComposeTS(ts.Add(5*time.Second).UnixNano()/int64(time.Millisecond), 0)}, args{ts: tsoutil.ComposeTS(ts.Add(5*time.Second).UnixNano()/int64(time.Millisecond), 0)},
false, false,
[]int64{2}, []int64{2, 4},
[]int64{1}, []int64{1, 3},
}, },
} }
for _, tt := range tests { for _, tt := range tests {
@ -443,17 +540,17 @@ func Test_compactionPlanHandler_expireCompaction(t *testing.T) {
meta: tt.fields.meta, meta: tt.fields.meta,
} }
err := c.expireCompaction(tt.args.ts) err := c.updateCompaction(tt.args.ts)
assert.Equal(t, tt.wantErr, err != nil) assert.Equal(t, tt.wantErr, err != nil)
for _, id := range tt.expired { for _, id := range tt.expired {
task := c.getCompaction(id) task := c.getCompaction(id)
assert.Equal(t, timeout, task.state) assert.Equal(t, failed, task.state)
} }
for _, id := range tt.unexpired { for _, id := range tt.unexpired {
task := c.getCompaction(id) task := c.getCompaction(id)
assert.NotEqual(t, timeout, task.state) assert.NotEqual(t, failed, task.state)
} }
}) })
} }
@ -484,13 +581,14 @@ func Test_newCompactionPlanHandler(t *testing.T) {
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
}, },
&compactionPlanHandler{ &compactionPlanHandler{
plans: map[int64]*compactionTask{}, plans: map[int64]*compactionTask{},
sessions: &SessionManager{}, sessions: &SessionManager{},
chManager: &ChannelManager{}, chManager: &ChannelManager{},
meta: &meta{}, meta: &meta{},
allocator: newMockAllocator(), allocator: newMockAllocator(),
flushCh: nil, flushCh: nil,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
parallelCh: make(map[int64]chan struct{}),
}, },
}, },
} }
@ -529,7 +627,7 @@ func Test_getCompactionTasksBySignalID(t *testing.T) {
}, },
3: { 3: {
triggerInfo: &compactionSignal{id: 1}, triggerInfo: &compactionSignal{id: 1},
state: timeout, state: failed,
}, },
}, },
}, },
@ -545,7 +643,7 @@ func Test_getCompactionTasksBySignalID(t *testing.T) {
}, },
{ {
triggerInfo: &compactionSignal{id: 1}, triggerInfo: &compactionSignal{id: 1},
state: timeout, state: failed,
}, },
}, },
}, },

View File

@ -49,7 +49,7 @@ func (h *spyCompactionHandler) getCompaction(planID int64) *compactionTask {
} }
// expireCompaction set the compaction state to expired // expireCompaction set the compaction state to expired
func (h *spyCompactionHandler) expireCompaction(ts Timestamp) error { func (h *spyCompactionHandler) updateCompaction(ts Timestamp) error {
panic("not implemented") // TODO: Implement panic("not implemented") // TODO: Implement
} }

View File

@ -118,9 +118,11 @@ func newTestSchema() *schemapb.CollectionSchema {
} }
type mockDataNodeClient struct { type mockDataNodeClient struct {
id int64 id int64
state internalpb.StateCode state internalpb.StateCode
ch chan interface{} ch chan interface{}
compactionStateResp *datapb.CompactionStateResponse
compactionResp *commonpb.Status
} }
func newMockDataNodeClient(id int64, ch chan interface{}) (*mockDataNodeClient, error) { func newMockDataNodeClient(id int64, ch chan interface{}) (*mockDataNodeClient, error) {
@ -221,10 +223,21 @@ func (c *mockDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMe
func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) { func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
if c.ch != nil { if c.ch != nil {
c.ch <- struct{}{} c.ch <- struct{}{}
if c.compactionResp != nil {
return c.compactionResp, nil
}
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
if c.compactionResp != nil {
return c.compactionResp, nil
} }
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: "not implemented"}, nil return &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: "not implemented"}, nil
} }
func (c *mockDataNodeClient) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
return c.compactionStateResp, nil
}
func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRequest) (*commonpb.Status, error) { func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRequest) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
} }
@ -556,15 +569,15 @@ func (h *mockCompactionHandler) execCompactionPlan(signal *compactionSignal, pla
panic("not implemented") panic("not implemented")
} }
// completeCompaction record the result of a compaction // // completeCompaction record the result of a compaction
func (h *mockCompactionHandler) completeCompaction(result *datapb.CompactionResult) error { // func (h *mockCompactionHandler) completeCompaction(result *datapb.CompactionResult) error {
if f, ok := h.methods["completeCompaction"]; ok { // if f, ok := h.methods["completeCompaction"]; ok {
if ff, ok := f.(func(result *datapb.CompactionResult) error); ok { // if ff, ok := f.(func(result *datapb.CompactionResult) error); ok {
return ff(result) // return ff(result)
} // }
} // }
panic("not implemented") // panic("not implemented")
} // }
// getCompaction return compaction task. If planId does not exist, return nil. // getCompaction return compaction task. If planId does not exist, return nil.
func (h *mockCompactionHandler) getCompaction(planID int64) *compactionTask { func (h *mockCompactionHandler) getCompaction(planID int64) *compactionTask {
@ -577,7 +590,7 @@ func (h *mockCompactionHandler) getCompaction(planID int64) *compactionTask {
} }
// expireCompaction set the compaction state to expired // expireCompaction set the compaction state to expired
func (h *mockCompactionHandler) expireCompaction(ts Timestamp) error { func (h *mockCompactionHandler) updateCompaction(ts Timestamp) error {
if f, ok := h.methods["expireCompaction"]; ok { if f, ok := h.methods["expireCompaction"]; ok {
if ff, ok := f.(func(ts Timestamp) error); ok { if ff, ok := f.(func(ts Timestamp) error); ok {
return ff(ts) return ff(ts)

View File

@ -2105,6 +2105,10 @@ func TestGetCompactionState(t *testing.T) {
{state: executing}, {state: executing},
{state: completed}, {state: completed},
{state: completed}, {state: completed},
{state: failed, plan: &datapb.CompactionPlan{PlanID: 1}},
{state: timeout, plan: &datapb.CompactionPlan{PlanID: 2}},
{state: timeout},
{state: timeout},
{state: timeout}, {state: timeout},
} }
}, },
@ -2117,7 +2121,8 @@ func TestGetCompactionState(t *testing.T) {
assert.Equal(t, commonpb.CompactionState_Executing, resp.GetState()) assert.Equal(t, commonpb.CompactionState_Executing, resp.GetState())
assert.EqualValues(t, 3, resp.GetExecutingPlanNo()) assert.EqualValues(t, 3, resp.GetExecutingPlanNo())
assert.EqualValues(t, 2, resp.GetCompletedPlanNo()) assert.EqualValues(t, 2, resp.GetCompletedPlanNo())
assert.EqualValues(t, 1, resp.GetTimeoutPlanNo()) assert.EqualValues(t, 1, resp.GetFailedPlanNo())
assert.EqualValues(t, 4, resp.GetTimeoutPlanNo())
}) })
t.Run("with closed server", func(t *testing.T) { t.Run("with closed server", func(t *testing.T) {
@ -2131,50 +2136,6 @@ func TestGetCompactionState(t *testing.T) {
}) })
} }
func TestCompleteCompaction(t *testing.T) {
Params.DataCoordCfg.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.DataCoordCfg.GetNodeID()), resp.GetReason())
})
}
func TestManualCompaction(t *testing.T) { func TestManualCompaction(t *testing.T) {
Params.DataCoordCfg.EnableCompaction = true Params.DataCoordCfg.EnableCompaction = true
t.Run("test manual compaction successfully", func(t *testing.T) { t.Run("test manual compaction successfully", func(t *testing.T) {

View File

@ -35,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/trace" "github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/samber/lo"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -816,38 +817,6 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
}, nil }, nil
} }
// CompleteCompaction completes a compaction with the result
func (s *Server) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) {
log.Info("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.DataCoordCfg.GetNodeID())))
resp.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID())
return resp, nil
}
if !Params.DataCoordCfg.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.Info("success to complete compaction", zap.Int64("planID", req.PlanID))
resp.ErrorCode = commonpb.ErrorCode_Success
return resp, nil
}
// ManualCompaction triggers a compaction for a collection // ManualCompaction triggers a compaction for a collection
func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) { func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
log.Info("received manual compaction", zap.Int64("collectionID", req.GetCollectionID())) log.Info("received manual compaction", zap.Int64("collectionID", req.GetCollectionID()))
@ -912,15 +881,22 @@ func (s *Server) GetCompactionState(ctx context.Context, req *milvuspb.GetCompac
} }
tasks := s.compactionHandler.getCompactionTasksBySignalID(req.GetCompactionID()) tasks := s.compactionHandler.getCompactionTasksBySignalID(req.GetCompactionID())
state, executingCnt, completedCnt, timeoutCnt := getCompactionState(tasks) state, executingCnt, completedCnt, failedCnt, timeoutCnt := getCompactionState(tasks)
resp.State = state resp.State = state
resp.ExecutingPlanNo = int64(executingCnt) resp.ExecutingPlanNo = int64(executingCnt)
resp.CompletedPlanNo = int64(completedCnt) resp.CompletedPlanNo = int64(completedCnt)
resp.TimeoutPlanNo = int64(timeoutCnt) resp.TimeoutPlanNo = int64(timeoutCnt)
resp.FailedPlanNo = int64(failedCnt)
resp.Status.ErrorCode = commonpb.ErrorCode_Success resp.Status.ErrorCode = commonpb.ErrorCode_Success
log.Info("success to get compaction state", zap.Any("state", state), zap.Int("executing", executingCnt), log.Info("success to get compaction state", zap.Any("state", state), zap.Int("executing", executingCnt),
zap.Int("completed", completedCnt), zap.Int("timeout", timeoutCnt)) zap.Int("completed", completedCnt), zap.Int("failed", failedCnt), zap.Int("timeout", timeoutCnt),
zap.Int64s("plans", lo.Map(tasks, func(t *compactionTask, _ int) int64 {
if t.plan == nil {
return -1
}
return t.plan.PlanID
})))
return resp, nil return resp, nil
} }
@ -948,11 +924,17 @@ func (s *Server) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.
resp.MergeInfos = append(resp.MergeInfos, getCompactionMergeInfo(task)) resp.MergeInfos = append(resp.MergeInfos, getCompactionMergeInfo(task))
} }
state, _, _, _ := getCompactionState(tasks) state, _, _, _, _ := getCompactionState(tasks)
resp.Status.ErrorCode = commonpb.ErrorCode_Success resp.Status.ErrorCode = commonpb.ErrorCode_Success
resp.State = state resp.State = state
log.Info("success to get state with plans", zap.Any("state", state), zap.Any("merge infos", resp.MergeInfos)) log.Info("success to get state with plans", zap.Any("state", state), zap.Any("merge infos", resp.MergeInfos),
zap.Int64s("plans", lo.Map(tasks, func(t *compactionTask, _ int) int64 {
if t.plan == nil {
return -1
}
return t.plan.PlanID
})))
return resp, nil return resp, nil
} }
@ -974,13 +956,19 @@ func getCompactionMergeInfo(task *compactionTask) *milvuspb.CompactionMergeInfo
} }
} }
func getCompactionState(tasks []*compactionTask) (state commonpb.CompactionState, executingCnt, completedCnt, timeoutCnt int) { func getCompactionState(tasks []*compactionTask) (state commonpb.CompactionState, executingCnt, completedCnt, failedCnt, timeoutCnt int) {
if len(tasks) == 0 {
state = commonpb.CompactionState_Executing
return
}
for _, t := range tasks { for _, t := range tasks {
switch t.state { switch t.state {
case executing: case executing:
executingCnt++ executingCnt++
case completed: case completed:
completedCnt++ completedCnt++
case failed:
failedCnt++
case timeout: case timeout:
timeoutCnt++ timeoutCnt++
} }

View File

@ -141,27 +141,24 @@ func (c *SessionManager) execFlush(ctx context.Context, nodeID int64, req *datap
} }
} }
// Compaction is a grpc interface. It will send request to DataNode with provided `nodeID` asynchronously. // Compaction is a grpc interface. It will send request to DataNode with provided `nodeID` synchronously.
func (c *SessionManager) Compaction(nodeID int64, plan *datapb.CompactionPlan) { func (c *SessionManager) Compaction(nodeID int64, plan *datapb.CompactionPlan) error {
go c.execCompaction(nodeID, plan) ctx, cancel := context.WithTimeout(context.Background(), rpcCompactionTimeout)
}
func (c *SessionManager) execCompaction(nodeID int64, plan *datapb.CompactionPlan) {
ctx, cancel := context.WithTimeout(context.Background(), compactionTimeout)
defer cancel() defer cancel()
cli, err := c.getClient(ctx, nodeID) cli, err := c.getClient(ctx, nodeID)
if err != nil { if err != nil {
log.Warn("failed to get client", zap.Int64("nodeID", nodeID), zap.Error(err)) log.Warn("failed to get client", zap.Int64("nodeID", nodeID), zap.Error(err))
return return err
} }
resp, err := cli.Compaction(ctx, plan) resp, err := cli.Compaction(ctx, plan)
if err := VerifyResponse(resp, err); err != nil { 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())) log.Warn("failed to execute compaction", zap.Int64("node", nodeID), zap.Error(err), zap.Int64("planID", plan.GetPlanID()))
return return err
} }
log.Info("success to execute compaction", zap.Int64("node", nodeID), zap.Any("planID", plan.GetPlanID())) log.Info("success to execute compaction", zap.Int64("node", nodeID), zap.Any("planID", plan.GetPlanID()))
return nil
} }
// Import is a grpc interface. It will send request to DataNode with provided `nodeID` asynchronously. // Import is a grpc interface. It will send request to DataNode with provided `nodeID` asynchronously.
@ -216,6 +213,57 @@ func (c *SessionManager) execReCollectSegmentStats(ctx context.Context, nodeID i
} }
} }
func (c *SessionManager) GetCompactionState() map[int64]*datapb.CompactionStateResult {
wg := sync.WaitGroup{}
ctx := context.Background()
c.sessions.RLock()
wg.Add(len(c.sessions.data))
c.sessions.RUnlock()
plans := sync.Map{}
c.sessions.RLock()
for nodeID, s := range c.sessions.data {
go func(nodeID int64, s *Session) {
defer wg.Done()
cli, err := s.GetOrCreateClient(ctx)
if err != nil {
log.Info("Cannot Create Client", zap.Int64("NodeID", nodeID))
return
}
ctx, cancel := context.WithTimeout(ctx, rpcCompactionTimeout)
defer cancel()
resp, err := cli.GetCompactionState(ctx, &datapb.CompactionStateRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_GetSystemConfigs,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
})
if err != nil {
log.Info("Get State failed", zap.Error(err))
return
}
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Info("Get State failed", zap.String("Reason", resp.GetStatus().GetReason()))
return
}
for _, rst := range resp.GetResults() {
plans.Store(rst.PlanID, rst)
}
}(nodeID, s)
}
c.sessions.RUnlock()
wg.Wait()
rst := make(map[int64]*datapb.CompactionStateResult)
plans.Range(func(key, value any) bool {
rst[key.(int64)] = value.(*datapb.CompactionStateResult)
return true
})
return rst
}
// AddSegment calls DataNode with ID == `nodeID` to put the segment into this node. // AddSegment calls DataNode with ID == `nodeID` to put the segment into this node.
func (c *SessionManager) AddSegment(ctx context.Context, nodeID int64, req *datapb.AddSegmentRequest) { func (c *SessionManager) AddSegment(ctx context.Context, nodeID int64, req *datapb.AddSegmentRequest) {
go c.execAddSegment(ctx, nodeID, req) go c.execAddSegment(ctx, nodeID, req)

View File

@ -29,30 +29,17 @@ const (
maxTaskNum = 1024 maxTaskNum = 1024
) )
var maxParallelCompactionNum = calculateParallel()
type compactionExecutor struct { type compactionExecutor struct {
parallelCh chan struct{} executing sync.Map // planID to compactor
executing sync.Map // planID to compactor completed sync.Map // planID to CompactionResult
taskCh chan compactor taskCh chan compactor
dropped sync.Map // vchannel dropped dropped sync.Map // vchannel dropped
}
// 0.5*min(8, NumCPU/2)
func calculateParallel() int {
return 2
//cores := runtime.NumCPU()
//if cores < 16 {
//return 4
//}
//return cores / 2
} }
func newCompactionExecutor() *compactionExecutor { func newCompactionExecutor() *compactionExecutor {
return &compactionExecutor{ return &compactionExecutor{
parallelCh: make(chan struct{}, maxParallelCompactionNum), executing: sync.Map{},
executing: sync.Map{}, taskCh: make(chan compactor, maxTaskNum),
taskCh: make(chan compactor, maxTaskNum),
} }
} }
@ -88,20 +75,20 @@ func (c *compactionExecutor) start(ctx context.Context) {
} }
func (c *compactionExecutor) executeTask(task compactor) { func (c *compactionExecutor) executeTask(task compactor) {
c.parallelCh <- struct{}{}
defer func() { defer func() {
c.toCompleteState(task) c.toCompleteState(task)
<-c.parallelCh
}() }()
log.Info("start to execute compaction", zap.Int64("planID", task.getPlanID())) log.Info("start to execute compaction", zap.Int64("planID", task.getPlanID()))
err := task.compact() result, err := task.compact()
if err != nil { if err != nil {
log.Warn("compaction task failed", log.Warn("compaction task failed",
zap.Int64("planID", task.getPlanID()), zap.Int64("planID", task.getPlanID()),
zap.Error(err), zap.Error(err),
) )
} else {
c.completed.Store(task.getPlanID(), result)
} }
log.Info("end to execute compaction", zap.Int64("planID", task.getPlanID())) log.Info("end to execute compaction", zap.Int64("planID", task.getPlanID()))

View File

@ -21,6 +21,7 @@ import (
"sync" "sync"
"testing" "testing"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -142,15 +143,15 @@ func (mc *mockCompactor) complete() {
mc.wg.Done() mc.wg.Done()
} }
func (mc *mockCompactor) compact() error { func (mc *mockCompactor) compact() (*datapb.CompactionResult, error) {
if !mc.isvalid { if !mc.isvalid {
return errStart return nil, errStart
} }
if mc.alwaysWorking { if mc.alwaysWorking {
<-mc.ctx.Done() <-mc.ctx.Done()
return mc.ctx.Err() return nil, mc.ctx.Err()
} }
return nil return nil, nil
} }
func (mc *mockCompactor) getPlanID() UniqueID { func (mc *mockCompactor) getPlanID() UniqueID {

View File

@ -27,12 +27,10 @@ import (
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/timerecord" "github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
@ -55,7 +53,7 @@ type iterator = storage.Iterator
type compactor interface { type compactor interface {
start() start()
complete() complete()
compact() error compact() (*datapb.CompactionResult, error)
stop() stop()
getPlanID() UniqueID getPlanID() UniqueID
getCollection() UniqueID getCollection() UniqueID
@ -73,7 +71,6 @@ type compactionTask struct {
flushManager flushManager
allocatorInterface allocatorInterface
dc types.DataCoord
plan *datapb.CompactionPlan plan *datapb.CompactionPlan
ctx context.Context ctx context.Context
@ -93,7 +90,6 @@ func newCompactionTask(
replica Replica, replica Replica,
fm flushManager, fm flushManager,
alloc allocatorInterface, alloc allocatorInterface,
dc types.DataCoord,
plan *datapb.CompactionPlan) *compactionTask { plan *datapb.CompactionPlan) *compactionTask {
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
@ -106,7 +102,6 @@ func newCompactionTask(
Replica: replica, Replica: replica,
flushManager: fm, flushManager: fm,
allocatorInterface: alloc, allocatorInterface: alloc,
dc: dc,
plan: plan, plan: plan,
tr: timerecord.NewTimeRecorder("compactionTask"), tr: timerecord.NewTimeRecorder("compactionTask"),
} }
@ -312,11 +307,11 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
return iDatas, numRows, nil return iDatas, numRows, nil
} }
func (t *compactionTask) compact() error { func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
compactStart := time.Now() compactStart := time.Now()
if ok := funcutil.CheckCtxValid(t.ctx); !ok { if ok := funcutil.CheckCtxValid(t.ctx); !ok {
log.Error("compact wrong, task context done or timeout") log.Error("compact wrong, task context done or timeout")
return errContext return nil, errContext
} }
ctxTimeout, cancelAll := context.WithTimeout(t.ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second) ctxTimeout, cancelAll := context.WithTimeout(t.ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
@ -328,17 +323,17 @@ func (t *compactionTask) compact() error {
case t.plan.GetType() == datapb.CompactionType_UndefinedCompaction: case t.plan.GetType() == datapb.CompactionType_UndefinedCompaction:
log.Error("compact wrong, compaction type undefined") log.Error("compact wrong, compaction type undefined")
return errCompactionTypeUndifined return nil, errCompactionTypeUndifined
case len(t.plan.GetSegmentBinlogs()) < 1: case len(t.plan.GetSegmentBinlogs()) < 1:
log.Error("compact wrong, there's no segments in segment binlogs") log.Error("compact wrong, there's no segments in segment binlogs")
return errIllegalCompactionPlan return nil, errIllegalCompactionPlan
case t.plan.GetType() == datapb.CompactionType_MergeCompaction || t.plan.GetType() == datapb.CompactionType_MixCompaction: case t.plan.GetType() == datapb.CompactionType_MergeCompaction || t.plan.GetType() == datapb.CompactionType_MixCompaction:
targetSegID, err = t.allocID() targetSegID, err = t.allocID()
if err != nil { if err != nil {
log.Error("compact wrong", zap.Error(err)) log.Error("compact wrong", zap.Error(err))
return err return nil, err
} }
case t.plan.GetType() == datapb.CompactionType_InnerCompaction: case t.plan.GetType() == datapb.CompactionType_InnerCompaction:
@ -354,7 +349,7 @@ func (t *compactionTask) compact() error {
collID, partID, meta, err := t.getSegmentMeta(segIDs[0]) collID, partID, meta, err := t.getSegmentMeta(segIDs[0])
if err != nil { if err != nil {
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err)) log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
return err return nil, err
} }
// Inject to stop flush // Inject to stop flush
@ -407,7 +402,7 @@ func (t *compactionTask) compact() error {
// Unable to deal with all empty segments cases, so return error // Unable to deal with all empty segments cases, so return error
if binlogNum == 0 { if binlogNum == 0 {
log.Error("compact wrong, all segments' binlogs are empty", zap.Int64("planID", t.plan.GetPlanID())) log.Error("compact wrong, all segments' binlogs are empty", zap.Int64("planID", t.plan.GetPlanID()))
return errIllegalCompactionPlan return nil, errIllegalCompactionPlan
} }
for idx := 0; idx < binlogNum; idx++ { for idx := 0; idx < binlogNum; idx++ {
@ -466,27 +461,27 @@ func (t *compactionTask) compact() error {
if err != nil { if err != nil {
log.Error("compaction IO wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err)) log.Error("compaction IO wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
return err return nil, err
} }
mergeItr := storage.NewMergeIterator(iItr) mergeItr := storage.NewMergeIterator(iItr)
deltaPk2Ts, deltaBuf, err := t.mergeDeltalogs(dblobs, t.plan.GetTimetravel()) deltaPk2Ts, deltaBuf, err := t.mergeDeltalogs(dblobs, t.plan.GetTimetravel())
if err != nil { if err != nil {
return err return nil, err
} }
iDatas, numRows, err := t.merge(mergeItr, deltaPk2Ts, meta.GetSchema(), t.GetCurrentTime()) iDatas, numRows, err := t.merge(mergeItr, deltaPk2Ts, meta.GetSchema(), t.GetCurrentTime())
if err != nil { if err != nil {
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err)) log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
return err return nil, err
} }
uploadStart := time.Now() uploadStart := time.Now()
segPaths, err := t.upload(ctxTimeout, targetSegID, partID, iDatas, deltaBuf.delData, meta) segPaths, err := t.upload(ctxTimeout, targetSegID, partID, iDatas, deltaBuf.delData, meta)
if err != nil { if err != nil {
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err)) log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
return err return nil, err
} }
uploadEnd := time.Now() uploadEnd := time.Now()
@ -512,21 +507,21 @@ func (t *compactionTask) compact() error {
NumOfRows: numRows, NumOfRows: numRows,
} }
rpcStart := time.Now() // rpcStart := time.Now()
status, err := t.dc.CompleteCompaction(ctxTimeout, pack) // status, err := t.dc.CompleteCompaction(ctxTimeout, pack)
if err != nil { // if err != nil {
log.Error("complete compaction rpc wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err)) // log.Error("complete compaction rpc wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
return err // return err
} // }
if status.ErrorCode != commonpb.ErrorCode_Success { // if status.ErrorCode != commonpb.ErrorCode_Success {
log.Error("complete compaction wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.String("reason", status.GetReason())) // log.Error("complete compaction wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.String("reason", status.GetReason()))
return fmt.Errorf("complete comapction wrong: %s", status.GetReason()) // return fmt.Errorf("complete comapction wrong: %s", status.GetReason())
} // }
rpcEnd := time.Now() // rpcEnd := time.Now()
defer func() { // defer func() {
log.Debug("rpc elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(rpcEnd.Sub(rpcStart)))) // log.Debug("rpc elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(rpcEnd.Sub(rpcStart))))
}() // }()
//
// Compaction I: update pk range. // Compaction I: update pk range.
// Compaction II: remove the segments and add a new flushed segment with pk range. // Compaction II: remove the segments and add a new flushed segment with pk range.
if t.hasSegment(targetSegID, true) { if t.hasSegment(targetSegID, true) {
@ -540,7 +535,7 @@ func (t *compactionTask) compact() error {
err = t.mergeFlushedSegments(targetSegID, collID, partID, t.plan.GetPlanID(), segIDs, t.plan.GetChannel(), numRows) err = t.mergeFlushedSegments(targetSegID, collID, partID, t.plan.GetPlanID(), segIDs, t.plan.GetChannel(), numRows)
if err != nil { if err != nil {
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err)) log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
return err return nil, err
} }
} }
@ -562,7 +557,7 @@ func (t *compactionTask) compact() error {
log.Info("overall elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(time.Since(compactStart)))) log.Info("overall elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(time.Since(compactStart))))
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Observe(float64(t.tr.ElapseSpan().Milliseconds())) metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
return nil return pack, nil
} }
// TODO copy maybe expensive, but this seems to be the only convinent way. // TODO copy maybe expensive, but this seems to be the only convinent way.

View File

@ -436,18 +436,18 @@ func TestCompactorInterfaceMethods(t *testing.T) {
} }
emptyTask.plan = plan emptyTask.plan = plan
err := emptyTask.compact() _, err := emptyTask.compact()
assert.Error(t, err) assert.Error(t, err)
plan.Type = datapb.CompactionType_InnerCompaction plan.Type = datapb.CompactionType_InnerCompaction
plan.SegmentBinlogs = emptySegmentBinlogs plan.SegmentBinlogs = emptySegmentBinlogs
err = emptyTask.compact() _, err = emptyTask.compact()
assert.Error(t, err) assert.Error(t, err)
plan.Type = datapb.CompactionType_MergeCompaction plan.Type = datapb.CompactionType_MergeCompaction
emptyTask.allocatorInterface = invalidAlloc emptyTask.allocatorInterface = invalidAlloc
plan.SegmentBinlogs = notEmptySegmentBinlogs plan.SegmentBinlogs = notEmptySegmentBinlogs
err = emptyTask.compact() _, err = emptyTask.compact()
assert.Error(t, err) assert.Error(t, err)
emptyTask.stop() emptyTask.stop()
@ -490,7 +490,6 @@ func TestCompactorInterfaceMethods(t *testing.T) {
rc := &RootCoordFactory{ rc := &RootCoordFactory{
pkType: c.pkType, pkType: c.pkType,
} }
dc := &DataCoordFactory{}
mockfm := &mockFlushManager{} mockfm := &mockFlushManager{}
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
replica, err := newReplica(context.TODO(), rc, cm, c.colID) replica, err := newReplica(context.TODO(), rc, cm, c.colID)
@ -527,12 +526,12 @@ func TestCompactorInterfaceMethods(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO()) ctx, cancel := context.WithCancel(context.TODO())
cancel() cancel()
canceledTask := newCompactionTask(ctx, mockbIO, mockbIO, replica, mockfm, alloc, dc, plan) canceledTask := newCompactionTask(ctx, mockbIO, mockbIO, replica, mockfm, alloc, plan)
err = canceledTask.compact() _, err = canceledTask.compact()
assert.Error(t, err) assert.Error(t, err)
task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, dc, plan) task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, plan)
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
updates, err := replica.getSegmentStatisticsUpdates(c.segID) updates, err := replica.getSegmentStatisticsUpdates(c.segID)
@ -558,7 +557,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
plan.PlanID++ plan.PlanID++
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
// The segment should be removed // The segment should be removed
assert.False(t, replica.hasSegment(c.segID, true)) assert.False(t, replica.hasSegment(c.segID, true))
@ -578,7 +577,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
}, },
} }
plan.SegmentBinlogs = segmentBinlogsWithEmptySegment plan.SegmentBinlogs = segmentBinlogsWithEmptySegment
err = task.compact() _, err = task.compact()
assert.Error(t, err) assert.Error(t, err)
plan.SegmentBinlogs = segBinlogs plan.SegmentBinlogs = segBinlogs
@ -591,7 +590,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
plan.PlanID++ plan.PlanID++
plan.Timetravel = Timestamp(10000) plan.Timetravel = Timestamp(10000)
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
updates, err = replica.getSegmentStatisticsUpdates(c.segID) updates, err = replica.getSegmentStatisticsUpdates(c.segID)
@ -607,7 +606,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
plan.PlanID++ plan.PlanID++
mockfm.sleepSeconds = plan.TimeoutInSeconds + int32(1) mockfm.sleepSeconds = plan.TimeoutInSeconds + int32(1)
err = task.compact() _, err = task.compact()
assert.Error(t, err) assert.Error(t, err)
} }
}) })
@ -654,7 +653,6 @@ func TestCompactorInterfaceMethods(t *testing.T) {
rc := &RootCoordFactory{ rc := &RootCoordFactory{
pkType: c.pkType, pkType: c.pkType,
} }
dc := &DataCoordFactory{}
mockfm := &mockFlushManager{} mockfm := &mockFlushManager{}
mockKv := memkv.NewMemoryKV() mockKv := memkv.NewMemoryKV()
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
@ -712,8 +710,8 @@ func TestCompactorInterfaceMethods(t *testing.T) {
} }
alloc.random = false // generated ID = 19530 alloc.random = false // generated ID = 19530
task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, dc, plan) task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, plan)
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
assert.False(t, replica.hasSegment(c.segID1, true)) assert.False(t, replica.hasSegment(c.segID1, true))
@ -737,7 +735,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
require.True(t, replica.hasSegment(c.segID2, true)) require.True(t, replica.hasSegment(c.segID2, true))
require.False(t, replica.hasSegment(19530, true)) require.False(t, replica.hasSegment(19530, true))
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
assert.False(t, replica.hasSegment(c.segID1, true)) assert.False(t, replica.hasSegment(c.segID1, true))
@ -761,7 +759,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
require.True(t, replica.hasSegment(c.segID2, true)) require.True(t, replica.hasSegment(c.segID2, true))
require.False(t, replica.hasSegment(19530, true)) require.False(t, replica.hasSegment(19530, true))
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
assert.False(t, replica.hasSegment(c.segID1, true)) assert.False(t, replica.hasSegment(c.segID1, true))
@ -783,7 +781,6 @@ func TestCompactorInterfaceMethods(t *testing.T) {
rc := &RootCoordFactory{ rc := &RootCoordFactory{
pkType: schemapb.DataType_Int64, pkType: schemapb.DataType_Int64,
} }
dc := &DataCoordFactory{}
mockfm := &mockFlushManager{} mockfm := &mockFlushManager{}
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
replica, err := newReplica(context.TODO(), rc, cm, collID) replica, err := newReplica(context.TODO(), rc, cm, collID)
@ -845,8 +842,8 @@ func TestCompactorInterfaceMethods(t *testing.T) {
} }
alloc.random = false // generated ID = 19530 alloc.random = false // generated ID = 19530
task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, dc, plan) task := newCompactionTask(context.TODO(), mockbIO, mockbIO, replica, mockfm, alloc, plan)
err = task.compact() _, err = task.compact()
assert.NoError(t, err) assert.NoError(t, err)
assert.False(t, replica.hasSegment(segID1, true)) assert.False(t, replica.hasSegment(segID1, true))

View File

@ -809,7 +809,6 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan
ds.replica, ds.replica,
ds.flushManager, ds.flushManager,
ds.idAllocator, ds.idAllocator,
node.dataCoord,
req, req,
) )
@ -820,6 +819,42 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan
}, nil }, nil
} }
// GetCompactionState called by DataCoord
// return status of all compaction plans
func (node *DataNode) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
log.Info("DataNode.GetCompactionState")
if !node.isHealthy() {
return &datapb.CompactionStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "DataNode is unhealthy",
},
}, nil
}
results := make([]*datapb.CompactionStateResult, 0)
node.compactionExecutor.executing.Range(func(k, v any) bool {
results = append(results, &datapb.CompactionStateResult{
State: commonpb.CompactionState_Executing,
PlanID: k.(UniqueID),
})
return true
})
node.compactionExecutor.completed.Range(func(k, v any) bool {
results = append(results, &datapb.CompactionStateResult{
State: commonpb.CompactionState_Completed,
PlanID: k.(UniqueID),
Result: v.(*datapb.CompactionResult),
})
node.compactionExecutor.completed.Delete(k)
return true
})
log.Debug("Compaction results", zap.Any("results", results))
return &datapb.CompactionStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Results: results,
}, nil
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments // Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) { func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
log.Info("DataNode receive import request", log.Info("DataNode receive import request",

View File

@ -149,6 +149,41 @@ func TestDataNode(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, stat.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, stat.Status.ErrorCode)
}) })
t.Run("Test GetCompactionState", func(t *testing.T) {
node.compactionExecutor.executing.Store(int64(3), 0)
node.compactionExecutor.executing.Store(int64(2), 0)
node.compactionExecutor.completed.Store(int64(1), &datapb.CompactionResult{
PlanID: 1,
SegmentID: 10,
})
stat, err := node.GetCompactionState(node.ctx, nil)
assert.NoError(t, err)
assert.Equal(t, 3, len(stat.GetResults()))
cnt := 0
for _, v := range stat.GetResults() {
if v.GetState() == commonpb.CompactionState_Completed {
cnt++
}
}
assert.Equal(t, 1, cnt)
cnt = 0
node.compactionExecutor.completed.Range(func(k, v any) bool {
cnt++
return true
})
assert.Equal(t, 0, cnt)
})
t.Run("Test GetCompactionState unhealthy", func(t *testing.T) {
node.UpdateStateCode(internalpb.StateCode_Abnormal)
resp, _ := node.GetCompactionState(ctx, nil)
assert.Equal(t, "DataNode is unhealthy", resp.GetStatus().GetReason())
node.UpdateStateCode(internalpb.StateCode_Healthy)
})
t.Run("Test FlushSegments", func(t *testing.T) { t.Run("Test FlushSegments", func(t *testing.T) {
dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments" dmChannelName := "fake-by-dev-rootcoord-dml-channel-test-FlushSegments"

View File

@ -25,6 +25,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/grpcclient" "github.com/milvus-io/milvus/internal/util/grpcclient"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
@ -38,6 +39,8 @@ import (
// ClientParams is the parameters of client singleton // ClientParams is the parameters of client singleton
var ClientParams paramtable.GrpcClientConfig var ClientParams paramtable.GrpcClientConfig
var _ types.DataCoord = (*Client)(nil)
// Client is the datacoord grpc client // Client is the datacoord grpc client
type Client struct { type Client struct {
grpcClient grpcclient.GrpcClient grpcClient grpcclient.GrpcClient
@ -412,20 +415,6 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
return ret.(*milvuspb.GetMetricsResponse), err return ret.(*milvuspb.GetMetricsResponse), err
} }
// CompleteCompaction completes a compaction with the result
func (c *Client) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.(datapb.DataCoordClient).CompleteCompaction(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}
// ManualCompaction triggers a compaction for a collection // ManualCompaction triggers a compaction for a collection
func (c *Client) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) { func (c *Client) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) { ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {

View File

@ -103,9 +103,6 @@ func Test_NewClient(t *testing.T) {
r15, err := client.GetMetrics(ctx, nil) r15, err := client.GetMetrics(ctx, nil)
retCheck(retNotNil, r15, err) retCheck(retNotNil, r15, err)
r16, err := client.CompleteCompaction(ctx, nil)
retCheck(retNotNil, r16, err)
r17, err := client.GetCompactionState(ctx, nil) r17, err := client.GetCompactionState(ctx, nil)
retCheck(retNotNil, r17, err) retCheck(retNotNil, r17, err)

View File

@ -301,11 +301,6 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
return s.dataCoord.GetMetrics(ctx, req) return s.dataCoord.GetMetrics(ctx, req)
} }
// CompleteCompaction completes a compaction with the result
func (s *Server) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) {
return s.dataCoord.CompleteCompaction(ctx, req)
}
// ManualCompaction triggers a compaction for a collection // ManualCompaction triggers a compaction for a collection
func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) { func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
return s.dataCoord.ManualCompaction(ctx, req) return s.dataCoord.ManualCompaction(ctx, req)

View File

@ -382,15 +382,6 @@ func Test_NewServer(t *testing.T) {
assert.NotNil(t, resp) assert.NotNil(t, resp)
}) })
t.Run("CompleteCompaction", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
status: &commonpb.Status{},
}
resp, err := server.CompleteCompaction(ctx, nil)
assert.Nil(t, err)
assert.NotNil(t, resp)
})
t.Run("ManualCompaction", func(t *testing.T) { t.Run("ManualCompaction", func(t *testing.T) {
server.dataCoord = &MockDataCoord{ server.dataCoord = &MockDataCoord{
manualCompactionResp: &milvuspb.ManualCompactionResponse{}, manualCompactionResp: &milvuspb.ManualCompactionResponse{},

View File

@ -205,6 +205,19 @@ func (c *Client) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*c
return ret.(*commonpb.Status), err return ret.(*commonpb.Status), err
} }
func (c *Client) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.(datapb.DataNodeClient).GetCompactionState(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*datapb.CompactionStateResponse), err
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments // Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) { func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) { ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {

View File

@ -88,6 +88,9 @@ func Test_NewClient(t *testing.T) {
r10, err := client.ShowConfigurations(ctx, nil) r10, err := client.ShowConfigurations(ctx, nil)
retCheck(retNotNil, r10, err) retCheck(retNotNil, r10, err)
r11, err := client.GetCompactionState(ctx, nil)
retCheck(retNotNil, r11, err)
} }
client.grpcClient = &mock.GRPCClientBase{ client.grpcClient = &mock.GRPCClientBase{

View File

@ -365,6 +365,11 @@ func (s *Server) Compaction(ctx context.Context, request *datapb.CompactionPlan)
return s.datanode.Compaction(ctx, request) return s.datanode.Compaction(ctx, request)
} }
// GetCompactionState gets the Compaction tasks state of DataNode
func (s *Server) GetCompactionState(ctx context.Context, request *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
return s.datanode.GetCompactionState(ctx, request)
}
func (s *Server) Import(ctx context.Context, request *datapb.ImportTaskRequest) (*commonpb.Status, error) { func (s *Server) Import(ctx context.Context, request *datapb.ImportTaskRequest) (*commonpb.Status, error) {
return s.datanode.Import(ctx, request) return s.datanode.Import(ctx, request)
} }

View File

@ -36,18 +36,19 @@ import (
type MockDataNode struct { type MockDataNode struct {
nodeID typeutil.UniqueID nodeID typeutil.UniqueID
stateCode internalpb.StateCode stateCode internalpb.StateCode
states *internalpb.ComponentStates states *internalpb.ComponentStates
status *commonpb.Status status *commonpb.Status
err error err error
initErr error initErr error
startErr error startErr error
stopErr error stopErr error
regErr error regErr error
strResp *milvuspb.StringResponse strResp *milvuspb.StringResponse
configResp *internalpb.ShowConfigurationsResponse configResp *internalpb.ShowConfigurationsResponse
metricResp *milvuspb.GetMetricsResponse metricResp *milvuspb.GetMetricsResponse
resendResp *datapb.ResendSegmentStatsResponse resendResp *datapb.ResendSegmentStatsResponse
compactionResp *datapb.CompactionStateResponse
} }
func (m *MockDataNode) Init() error { func (m *MockDataNode) Init() error {
@ -114,6 +115,10 @@ func (m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPla
return m.status, m.err return m.status, m.err
} }
func (m *MockDataNode) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
return m.compactionResp, m.err
}
func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) { func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) {
} }

View File

@ -37,7 +37,6 @@ service DataCoord {
rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){} rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){}
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {} rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
rpc CompleteCompaction(CompactionResult) returns (common.Status) {}
rpc ManualCompaction(milvus.ManualCompactionRequest) returns (milvus.ManualCompactionResponse) {} rpc ManualCompaction(milvus.ManualCompactionRequest) returns (milvus.ManualCompactionResponse) {}
rpc GetCompactionState(milvus.GetCompactionStateRequest) returns (milvus.GetCompactionStateResponse) {} rpc GetCompactionState(milvus.GetCompactionStateRequest) returns (milvus.GetCompactionStateResponse) {}
rpc GetCompactionStateWithPlans(milvus.GetCompactionPlansRequest) returns (milvus.GetCompactionPlansResponse) {} rpc GetCompactionStateWithPlans(milvus.GetCompactionPlansRequest) returns (milvus.GetCompactionPlansResponse) {}
@ -59,7 +58,7 @@ service DataCoord {
service DataNode { service DataNode {
rpc GetComponentStates(internal.GetComponentStatesRequest) returns (internal.ComponentStates) {} rpc GetComponentStates(internal.GetComponentStatesRequest) returns (internal.ComponentStates) {}
rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){} rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns (milvus.StringResponse) {}
rpc WatchDmChannels(WatchDmChannelsRequest) returns (common.Status) {} rpc WatchDmChannels(WatchDmChannelsRequest) returns (common.Status) {}
rpc FlushSegments(FlushSegmentsRequest) returns(common.Status) {} rpc FlushSegments(FlushSegmentsRequest) returns(common.Status) {}
@ -67,7 +66,9 @@ service DataNode {
rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){} rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){}
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {} rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
rpc Compaction(CompactionPlan) returns (common.Status) {} rpc Compaction(CompactionPlan) returns (common.Status) {}
rpc GetCompactionState(CompactionStateRequest) returns (CompactionStateResponse) {}
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
rpc Import(ImportTaskRequest) returns(common.Status) {} rpc Import(ImportTaskRequest) returns(common.Status) {}
@ -393,6 +394,10 @@ enum CompactionType {
MixCompaction = 3; MixCompaction = 3;
} }
message CompactionStateRequest {
common.MsgBase base = 1;
}
message CompactionSegmentBinlogs { message CompactionSegmentBinlogs {
int64 segmentID = 1; int64 segmentID = 1;
repeated FieldBinlog fieldBinlogs = 2; repeated FieldBinlog fieldBinlogs = 2;
@ -420,6 +425,17 @@ message CompactionResult {
repeated FieldBinlog deltalogs = 6; repeated FieldBinlog deltalogs = 6;
} }
message CompactionStateResult {
int64 planID = 1;
common.CompactionState state = 2;
CompactionResult result = 3;
}
message CompactionStateResponse {
common.Status status = 1;
repeated CompactionStateResult results = 2;
}
// Deprecated // Deprecated
message SegmentFieldBinlogMeta { message SegmentFieldBinlogMeta {
int64 fieldID = 1; int64 fieldID = 1;

View File

@ -2791,6 +2791,45 @@ func (m *ChannelWatchInfo) GetTimeoutTs() int64 {
return 0 return 0
} }
type CompactionStateRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *CompactionStateRequest) Reset() { *m = CompactionStateRequest{} }
func (m *CompactionStateRequest) String() string { return proto.CompactTextString(m) }
func (*CompactionStateRequest) ProtoMessage() {}
func (*CompactionStateRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{44}
}
func (m *CompactionStateRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CompactionStateRequest.Unmarshal(m, b)
}
func (m *CompactionStateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_CompactionStateRequest.Marshal(b, m, deterministic)
}
func (m *CompactionStateRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_CompactionStateRequest.Merge(m, src)
}
func (m *CompactionStateRequest) XXX_Size() int {
return xxx_messageInfo_CompactionStateRequest.Size(m)
}
func (m *CompactionStateRequest) XXX_DiscardUnknown() {
xxx_messageInfo_CompactionStateRequest.DiscardUnknown(m)
}
var xxx_messageInfo_CompactionStateRequest proto.InternalMessageInfo
func (m *CompactionStateRequest) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
type CompactionSegmentBinlogs struct { type CompactionSegmentBinlogs struct {
SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"` FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"`
@ -2806,7 +2845,7 @@ func (m *CompactionSegmentBinlogs) Reset() { *m = CompactionSegmentBinlo
func (m *CompactionSegmentBinlogs) String() string { return proto.CompactTextString(m) } func (m *CompactionSegmentBinlogs) String() string { return proto.CompactTextString(m) }
func (*CompactionSegmentBinlogs) ProtoMessage() {} func (*CompactionSegmentBinlogs) ProtoMessage() {}
func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) { func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{44} return fileDescriptor_82cd95f524594f49, []int{45}
} }
func (m *CompactionSegmentBinlogs) XXX_Unmarshal(b []byte) error { func (m *CompactionSegmentBinlogs) XXX_Unmarshal(b []byte) error {
@ -2879,7 +2918,7 @@ func (m *CompactionPlan) Reset() { *m = CompactionPlan{} }
func (m *CompactionPlan) String() string { return proto.CompactTextString(m) } func (m *CompactionPlan) String() string { return proto.CompactTextString(m) }
func (*CompactionPlan) ProtoMessage() {} func (*CompactionPlan) ProtoMessage() {}
func (*CompactionPlan) Descriptor() ([]byte, []int) { func (*CompactionPlan) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{45} return fileDescriptor_82cd95f524594f49, []int{46}
} }
func (m *CompactionPlan) XXX_Unmarshal(b []byte) error { func (m *CompactionPlan) XXX_Unmarshal(b []byte) error {
@ -2965,7 +3004,7 @@ func (m *CompactionResult) Reset() { *m = CompactionResult{} }
func (m *CompactionResult) String() string { return proto.CompactTextString(m) } func (m *CompactionResult) String() string { return proto.CompactTextString(m) }
func (*CompactionResult) ProtoMessage() {} func (*CompactionResult) ProtoMessage() {}
func (*CompactionResult) Descriptor() ([]byte, []int) { func (*CompactionResult) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{46} return fileDescriptor_82cd95f524594f49, []int{47}
} }
func (m *CompactionResult) XXX_Unmarshal(b []byte) error { func (m *CompactionResult) XXX_Unmarshal(b []byte) error {
@ -3028,6 +3067,108 @@ func (m *CompactionResult) GetDeltalogs() []*FieldBinlog {
return nil return nil
} }
type CompactionStateResult struct {
PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"`
State commonpb.CompactionState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.common.CompactionState" json:"state,omitempty"`
Result *CompactionResult `protobuf:"bytes,3,opt,name=result,proto3" json:"result,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *CompactionStateResult) Reset() { *m = CompactionStateResult{} }
func (m *CompactionStateResult) String() string { return proto.CompactTextString(m) }
func (*CompactionStateResult) ProtoMessage() {}
func (*CompactionStateResult) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{48}
}
func (m *CompactionStateResult) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CompactionStateResult.Unmarshal(m, b)
}
func (m *CompactionStateResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_CompactionStateResult.Marshal(b, m, deterministic)
}
func (m *CompactionStateResult) XXX_Merge(src proto.Message) {
xxx_messageInfo_CompactionStateResult.Merge(m, src)
}
func (m *CompactionStateResult) XXX_Size() int {
return xxx_messageInfo_CompactionStateResult.Size(m)
}
func (m *CompactionStateResult) XXX_DiscardUnknown() {
xxx_messageInfo_CompactionStateResult.DiscardUnknown(m)
}
var xxx_messageInfo_CompactionStateResult proto.InternalMessageInfo
func (m *CompactionStateResult) GetPlanID() int64 {
if m != nil {
return m.PlanID
}
return 0
}
func (m *CompactionStateResult) GetState() commonpb.CompactionState {
if m != nil {
return m.State
}
return commonpb.CompactionState_UndefiedState
}
func (m *CompactionStateResult) GetResult() *CompactionResult {
if m != nil {
return m.Result
}
return nil
}
type CompactionStateResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
Results []*CompactionStateResult `protobuf:"bytes,2,rep,name=results,proto3" json:"results,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *CompactionStateResponse) Reset() { *m = CompactionStateResponse{} }
func (m *CompactionStateResponse) String() string { return proto.CompactTextString(m) }
func (*CompactionStateResponse) ProtoMessage() {}
func (*CompactionStateResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{49}
}
func (m *CompactionStateResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CompactionStateResponse.Unmarshal(m, b)
}
func (m *CompactionStateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_CompactionStateResponse.Marshal(b, m, deterministic)
}
func (m *CompactionStateResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_CompactionStateResponse.Merge(m, src)
}
func (m *CompactionStateResponse) XXX_Size() int {
return xxx_messageInfo_CompactionStateResponse.Size(m)
}
func (m *CompactionStateResponse) XXX_DiscardUnknown() {
xxx_messageInfo_CompactionStateResponse.DiscardUnknown(m)
}
var xxx_messageInfo_CompactionStateResponse proto.InternalMessageInfo
func (m *CompactionStateResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
func (m *CompactionStateResponse) GetResults() []*CompactionStateResult {
if m != nil {
return m.Results
}
return nil
}
// Deprecated // Deprecated
type SegmentFieldBinlogMeta struct { type SegmentFieldBinlogMeta struct {
FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"` FieldID int64 `protobuf:"varint,1,opt,name=fieldID,proto3" json:"fieldID,omitempty"`
@ -3041,7 +3182,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{}
func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) ProtoMessage() {}
func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{47} return fileDescriptor_82cd95f524594f49, []int{50}
} }
func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error {
@ -3089,7 +3230,7 @@ func (m *WatchChannelsRequest) Reset() { *m = WatchChannelsRequest{} }
func (m *WatchChannelsRequest) String() string { return proto.CompactTextString(m) } func (m *WatchChannelsRequest) String() string { return proto.CompactTextString(m) }
func (*WatchChannelsRequest) ProtoMessage() {} func (*WatchChannelsRequest) ProtoMessage() {}
func (*WatchChannelsRequest) Descriptor() ([]byte, []int) { func (*WatchChannelsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{48} return fileDescriptor_82cd95f524594f49, []int{51}
} }
func (m *WatchChannelsRequest) XXX_Unmarshal(b []byte) error { func (m *WatchChannelsRequest) XXX_Unmarshal(b []byte) error {
@ -3142,7 +3283,7 @@ func (m *WatchChannelsResponse) Reset() { *m = WatchChannelsResponse{} }
func (m *WatchChannelsResponse) String() string { return proto.CompactTextString(m) } func (m *WatchChannelsResponse) String() string { return proto.CompactTextString(m) }
func (*WatchChannelsResponse) ProtoMessage() {} func (*WatchChannelsResponse) ProtoMessage() {}
func (*WatchChannelsResponse) Descriptor() ([]byte, []int) { func (*WatchChannelsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{49} return fileDescriptor_82cd95f524594f49, []int{52}
} }
func (m *WatchChannelsResponse) XXX_Unmarshal(b []byte) error { func (m *WatchChannelsResponse) XXX_Unmarshal(b []byte) error {
@ -3183,7 +3324,7 @@ func (m *SetSegmentStateRequest) Reset() { *m = SetSegmentStateRequest{}
func (m *SetSegmentStateRequest) String() string { return proto.CompactTextString(m) } func (m *SetSegmentStateRequest) String() string { return proto.CompactTextString(m) }
func (*SetSegmentStateRequest) ProtoMessage() {} func (*SetSegmentStateRequest) ProtoMessage() {}
func (*SetSegmentStateRequest) Descriptor() ([]byte, []int) { func (*SetSegmentStateRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{50} return fileDescriptor_82cd95f524594f49, []int{53}
} }
func (m *SetSegmentStateRequest) XXX_Unmarshal(b []byte) error { func (m *SetSegmentStateRequest) XXX_Unmarshal(b []byte) error {
@ -3236,7 +3377,7 @@ func (m *SetSegmentStateResponse) Reset() { *m = SetSegmentStateResponse
func (m *SetSegmentStateResponse) String() string { return proto.CompactTextString(m) } func (m *SetSegmentStateResponse) String() string { return proto.CompactTextString(m) }
func (*SetSegmentStateResponse) ProtoMessage() {} func (*SetSegmentStateResponse) ProtoMessage() {}
func (*SetSegmentStateResponse) Descriptor() ([]byte, []int) { func (*SetSegmentStateResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{51} return fileDescriptor_82cd95f524594f49, []int{54}
} }
func (m *SetSegmentStateResponse) XXX_Unmarshal(b []byte) error { func (m *SetSegmentStateResponse) XXX_Unmarshal(b []byte) error {
@ -3277,7 +3418,7 @@ func (m *DropVirtualChannelRequest) Reset() { *m = DropVirtualChannelReq
func (m *DropVirtualChannelRequest) String() string { return proto.CompactTextString(m) } func (m *DropVirtualChannelRequest) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelRequest) ProtoMessage() {} func (*DropVirtualChannelRequest) ProtoMessage() {}
func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) { func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{52} return fileDescriptor_82cd95f524594f49, []int{55}
} }
func (m *DropVirtualChannelRequest) XXX_Unmarshal(b []byte) error { func (m *DropVirtualChannelRequest) XXX_Unmarshal(b []byte) error {
@ -3337,7 +3478,7 @@ func (m *DropVirtualChannelSegment) Reset() { *m = DropVirtualChannelSeg
func (m *DropVirtualChannelSegment) String() string { return proto.CompactTextString(m) } func (m *DropVirtualChannelSegment) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelSegment) ProtoMessage() {} func (*DropVirtualChannelSegment) ProtoMessage() {}
func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) { func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{53} return fileDescriptor_82cd95f524594f49, []int{56}
} }
func (m *DropVirtualChannelSegment) XXX_Unmarshal(b []byte) error { func (m *DropVirtualChannelSegment) XXX_Unmarshal(b []byte) error {
@ -3425,7 +3566,7 @@ func (m *DropVirtualChannelResponse) Reset() { *m = DropVirtualChannelRe
func (m *DropVirtualChannelResponse) String() string { return proto.CompactTextString(m) } func (m *DropVirtualChannelResponse) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelResponse) ProtoMessage() {} func (*DropVirtualChannelResponse) ProtoMessage() {}
func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) { func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{54} return fileDescriptor_82cd95f524594f49, []int{57}
} }
func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error { func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error {
@ -3471,7 +3612,7 @@ func (m *ImportTask) Reset() { *m = ImportTask{} }
func (m *ImportTask) String() string { return proto.CompactTextString(m) } func (m *ImportTask) String() string { return proto.CompactTextString(m) }
func (*ImportTask) ProtoMessage() {} func (*ImportTask) ProtoMessage() {}
func (*ImportTask) Descriptor() ([]byte, []int) { func (*ImportTask) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{55} return fileDescriptor_82cd95f524594f49, []int{58}
} }
func (m *ImportTask) XXX_Unmarshal(b []byte) error { func (m *ImportTask) XXX_Unmarshal(b []byte) error {
@ -3563,7 +3704,7 @@ func (m *ImportTaskState) Reset() { *m = ImportTaskState{} }
func (m *ImportTaskState) String() string { return proto.CompactTextString(m) } func (m *ImportTaskState) String() string { return proto.CompactTextString(m) }
func (*ImportTaskState) ProtoMessage() {} func (*ImportTaskState) ProtoMessage() {}
func (*ImportTaskState) Descriptor() ([]byte, []int) { func (*ImportTaskState) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{56} return fileDescriptor_82cd95f524594f49, []int{59}
} }
func (m *ImportTaskState) XXX_Unmarshal(b []byte) error { func (m *ImportTaskState) XXX_Unmarshal(b []byte) error {
@ -3642,7 +3783,7 @@ func (m *ImportTaskInfo) Reset() { *m = ImportTaskInfo{} }
func (m *ImportTaskInfo) String() string { return proto.CompactTextString(m) } func (m *ImportTaskInfo) String() string { return proto.CompactTextString(m) }
func (*ImportTaskInfo) ProtoMessage() {} func (*ImportTaskInfo) ProtoMessage() {}
func (*ImportTaskInfo) Descriptor() ([]byte, []int) { func (*ImportTaskInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{57} return fileDescriptor_82cd95f524594f49, []int{60}
} }
func (m *ImportTaskInfo) XXX_Unmarshal(b []byte) error { func (m *ImportTaskInfo) XXX_Unmarshal(b []byte) error {
@ -3767,7 +3908,7 @@ func (m *ImportTaskResponse) Reset() { *m = ImportTaskResponse{} }
func (m *ImportTaskResponse) String() string { return proto.CompactTextString(m) } func (m *ImportTaskResponse) String() string { return proto.CompactTextString(m) }
func (*ImportTaskResponse) ProtoMessage() {} func (*ImportTaskResponse) ProtoMessage() {}
func (*ImportTaskResponse) Descriptor() ([]byte, []int) { func (*ImportTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{58} return fileDescriptor_82cd95f524594f49, []int{61}
} }
func (m *ImportTaskResponse) XXX_Unmarshal(b []byte) error { func (m *ImportTaskResponse) XXX_Unmarshal(b []byte) error {
@ -3815,7 +3956,7 @@ func (m *ImportTaskRequest) Reset() { *m = ImportTaskRequest{} }
func (m *ImportTaskRequest) String() string { return proto.CompactTextString(m) } func (m *ImportTaskRequest) String() string { return proto.CompactTextString(m) }
func (*ImportTaskRequest) ProtoMessage() {} func (*ImportTaskRequest) ProtoMessage() {}
func (*ImportTaskRequest) Descriptor() ([]byte, []int) { func (*ImportTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{59} return fileDescriptor_82cd95f524594f49, []int{62}
} }
func (m *ImportTaskRequest) XXX_Unmarshal(b []byte) error { func (m *ImportTaskRequest) XXX_Unmarshal(b []byte) error {
@ -3869,7 +4010,7 @@ func (m *UpdateSegmentStatisticsRequest) Reset() { *m = UpdateSegmentSta
func (m *UpdateSegmentStatisticsRequest) String() string { return proto.CompactTextString(m) } func (m *UpdateSegmentStatisticsRequest) String() string { return proto.CompactTextString(m) }
func (*UpdateSegmentStatisticsRequest) ProtoMessage() {} func (*UpdateSegmentStatisticsRequest) ProtoMessage() {}
func (*UpdateSegmentStatisticsRequest) Descriptor() ([]byte, []int) { func (*UpdateSegmentStatisticsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{60} return fileDescriptor_82cd95f524594f49, []int{63}
} }
func (m *UpdateSegmentStatisticsRequest) XXX_Unmarshal(b []byte) error { func (m *UpdateSegmentStatisticsRequest) XXX_Unmarshal(b []byte) error {
@ -3915,7 +4056,7 @@ func (m *ResendSegmentStatsRequest) Reset() { *m = ResendSegmentStatsReq
func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) } func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) }
func (*ResendSegmentStatsRequest) ProtoMessage() {} func (*ResendSegmentStatsRequest) ProtoMessage() {}
func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) { func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{61} return fileDescriptor_82cd95f524594f49, []int{64}
} }
func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error { func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error {
@ -3955,7 +4096,7 @@ func (m *ResendSegmentStatsResponse) Reset() { *m = ResendSegmentStatsRe
func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) } func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) }
func (*ResendSegmentStatsResponse) ProtoMessage() {} func (*ResendSegmentStatsResponse) ProtoMessage() {}
func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) { func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{62} return fileDescriptor_82cd95f524594f49, []int{65}
} }
func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error { func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error {
@ -4006,7 +4147,7 @@ func (m *AddSegmentRequest) Reset() { *m = AddSegmentRequest{} }
func (m *AddSegmentRequest) String() string { return proto.CompactTextString(m) } func (m *AddSegmentRequest) String() string { return proto.CompactTextString(m) }
func (*AddSegmentRequest) ProtoMessage() {} func (*AddSegmentRequest) ProtoMessage() {}
func (*AddSegmentRequest) Descriptor() ([]byte, []int) { func (*AddSegmentRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{63} return fileDescriptor_82cd95f524594f49, []int{66}
} }
func (m *AddSegmentRequest) XXX_Unmarshal(b []byte) error { func (m *AddSegmentRequest) XXX_Unmarshal(b []byte) error {
@ -4082,7 +4223,7 @@ func (m *SegmentReferenceLock) Reset() { *m = SegmentReferenceLock{} }
func (m *SegmentReferenceLock) String() string { return proto.CompactTextString(m) } func (m *SegmentReferenceLock) String() string { return proto.CompactTextString(m) }
func (*SegmentReferenceLock) ProtoMessage() {} func (*SegmentReferenceLock) ProtoMessage() {}
func (*SegmentReferenceLock) Descriptor() ([]byte, []int) { func (*SegmentReferenceLock) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{64} return fileDescriptor_82cd95f524594f49, []int{67}
} }
func (m *SegmentReferenceLock) XXX_Unmarshal(b []byte) error { func (m *SegmentReferenceLock) XXX_Unmarshal(b []byte) error {
@ -4171,9 +4312,12 @@ func init() {
proto.RegisterType((*GetFlushedSegmentsResponse)(nil), "milvus.proto.data.GetFlushedSegmentsResponse") proto.RegisterType((*GetFlushedSegmentsResponse)(nil), "milvus.proto.data.GetFlushedSegmentsResponse")
proto.RegisterType((*SegmentFlushCompletedMsg)(nil), "milvus.proto.data.SegmentFlushCompletedMsg") proto.RegisterType((*SegmentFlushCompletedMsg)(nil), "milvus.proto.data.SegmentFlushCompletedMsg")
proto.RegisterType((*ChannelWatchInfo)(nil), "milvus.proto.data.ChannelWatchInfo") proto.RegisterType((*ChannelWatchInfo)(nil), "milvus.proto.data.ChannelWatchInfo")
proto.RegisterType((*CompactionStateRequest)(nil), "milvus.proto.data.CompactionStateRequest")
proto.RegisterType((*CompactionSegmentBinlogs)(nil), "milvus.proto.data.CompactionSegmentBinlogs") proto.RegisterType((*CompactionSegmentBinlogs)(nil), "milvus.proto.data.CompactionSegmentBinlogs")
proto.RegisterType((*CompactionPlan)(nil), "milvus.proto.data.CompactionPlan") proto.RegisterType((*CompactionPlan)(nil), "milvus.proto.data.CompactionPlan")
proto.RegisterType((*CompactionResult)(nil), "milvus.proto.data.CompactionResult") proto.RegisterType((*CompactionResult)(nil), "milvus.proto.data.CompactionResult")
proto.RegisterType((*CompactionStateResult)(nil), "milvus.proto.data.CompactionStateResult")
proto.RegisterType((*CompactionStateResponse)(nil), "milvus.proto.data.CompactionStateResponse")
proto.RegisterType((*SegmentFieldBinlogMeta)(nil), "milvus.proto.data.SegmentFieldBinlogMeta") proto.RegisterType((*SegmentFieldBinlogMeta)(nil), "milvus.proto.data.SegmentFieldBinlogMeta")
proto.RegisterType((*WatchChannelsRequest)(nil), "milvus.proto.data.WatchChannelsRequest") proto.RegisterType((*WatchChannelsRequest)(nil), "milvus.proto.data.WatchChannelsRequest")
proto.RegisterType((*WatchChannelsResponse)(nil), "milvus.proto.data.WatchChannelsResponse") proto.RegisterType((*WatchChannelsResponse)(nil), "milvus.proto.data.WatchChannelsResponse")
@ -4197,243 +4341,248 @@ func init() {
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{ var fileDescriptor_82cd95f524594f49 = []byte{
// 3776 bytes of a gzipped FileDescriptorProto // 3847 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3b, 0xdd, 0x6f, 0x1c, 0x57, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0x5b, 0x6f, 0x1b, 0x47,
0xf5, 0x99, 0xfd, 0xf2, 0xee, 0xd9, 0x0f, 0xaf, 0x6f, 0x52, 0x67, 0xb3, 0x49, 0x9c, 0x64, 0xd2, 0x77, 0x5e, 0xde, 0x44, 0x1e, 0x5e, 0x44, 0x8d, 0x1d, 0x89, 0xa6, 0x6d, 0xd9, 0x5e, 0xc7, 0x8e,
0xa4, 0x69, 0x9a, 0x26, 0xad, 0xf3, 0xab, 0x5a, 0xfd, 0xfa, 0xa5, 0x38, 0x4e, 0x9c, 0x15, 0x76, 0xed, 0x38, 0x76, 0x22, 0x37, 0x48, 0x50, 0xe7, 0x02, 0xcb, 0xb2, 0x65, 0xb6, 0x92, 0x2b, 0xaf,
0x70, 0xc6, 0x4e, 0x83, 0x28, 0xd2, 0x6a, 0xbc, 0x73, 0xbd, 0x9e, 0x7a, 0x67, 0x66, 0x33, 0x33, 0xe4, 0x18, 0x68, 0x0a, 0x10, 0x2b, 0xee, 0x88, 0xda, 0x88, 0xbb, 0x4b, 0xef, 0x2e, 0x2d, 0x2b,
0x1b, 0xc7, 0xe5, 0xa1, 0x15, 0x95, 0x90, 0x8a, 0x10, 0x45, 0x42, 0x48, 0x20, 0x81, 0x84, 0x78, 0x7d, 0x48, 0xd0, 0x00, 0x05, 0x52, 0x14, 0x4d, 0xd1, 0xa2, 0x40, 0x0b, 0xb4, 0x40, 0xd1, 0xa7,
0x02, 0x24, 0x24, 0xa4, 0x8a, 0x07, 0x40, 0x7d, 0xaf, 0xe0, 0x01, 0xf1, 0xc6, 0x7f, 0x00, 0xe2, 0xb4, 0x40, 0x81, 0x02, 0x41, 0x1f, 0xda, 0x22, 0xef, 0x41, 0xfb, 0x50, 0xf4, 0xb1, 0xbf, 0xa0,
0x81, 0xbf, 0x01, 0xdd, 0x8f, 0xb9, 0xf3, 0xbd, 0x3b, 0x5e, 0x27, 0x0d, 0xe2, 0x6d, 0xef, 0x99, 0x7d, 0xfa, 0x7e, 0xc2, 0x87, 0x0f, 0x73, 0xd9, 0xd9, 0x3b, 0xb9, 0x22, 0xed, 0xf8, 0xc3, 0xf7,
0x73, 0xef, 0x3d, 0xf7, 0x7c, 0x9f, 0x73, 0xef, 0x42, 0x53, 0x53, 0x5d, 0xb5, 0xdb, 0xb3, 0x2c, 0xa6, 0x39, 0x7b, 0xce, 0xcc, 0x99, 0x33, 0xe7, 0x3e, 0x43, 0x41, 0x53, 0x53, 0x5d, 0xb5, 0xdb,
0x5b, 0xbb, 0x3a, 0xb4, 0x2d, 0xd7, 0x42, 0x73, 0x86, 0x3e, 0x78, 0x34, 0x72, 0xd8, 0xe8, 0x2a, 0xb3, 0x2c, 0x5b, 0xbb, 0x39, 0xb4, 0x2d, 0xd7, 0x42, 0x0b, 0x86, 0x3e, 0x78, 0x3e, 0x72, 0xd8,
0xf9, 0xdc, 0xae, 0xf5, 0x2c, 0xc3, 0xb0, 0x4c, 0x06, 0x6a, 0x37, 0x74, 0xd3, 0xc5, 0xb6, 0xa9, 0xe8, 0x26, 0xf9, 0xdc, 0xae, 0xf5, 0x2c, 0xc3, 0xb0, 0x4c, 0x06, 0x6a, 0x37, 0x74, 0xd3, 0xc5,
0x0e, 0xf8, 0xb8, 0x16, 0x9c, 0xd0, 0xae, 0x39, 0xbd, 0x1d, 0x6c, 0xa8, 0x6c, 0x24, 0xcf, 0x40, 0xb6, 0xa9, 0x0e, 0xf8, 0xb8, 0x16, 0x24, 0x68, 0xd7, 0x9c, 0xde, 0x3e, 0x36, 0x54, 0x36, 0x92,
0xf1, 0x96, 0x31, 0x74, 0xf7, 0xe5, 0x9f, 0x48, 0x50, 0xbb, 0x3d, 0x18, 0x39, 0x3b, 0x0a, 0x7e, 0xe7, 0xa0, 0x78, 0xdf, 0x18, 0xba, 0x47, 0xf2, 0x5f, 0x4b, 0x50, 0x7b, 0x30, 0x18, 0x39, 0xfb,
0x38, 0xc2, 0x8e, 0x8b, 0x5e, 0x81, 0xc2, 0x96, 0xea, 0xe0, 0x96, 0x74, 0x56, 0xba, 0x54, 0x5d, 0x0a, 0x7e, 0x36, 0xc2, 0x8e, 0x8b, 0xde, 0x85, 0xc2, 0xae, 0xea, 0xe0, 0x96, 0x74, 0x41, 0xba,
0x3c, 0x75, 0x35, 0xb4, 0x2b, 0xdf, 0x6f, 0xcd, 0xe9, 0x2f, 0xa9, 0x0e, 0x56, 0x28, 0x26, 0x42, 0x5a, 0x5d, 0x39, 0x7b, 0x33, 0xb4, 0x2a, 0x5f, 0x6f, 0xd3, 0xe9, 0xaf, 0xaa, 0x0e, 0x56, 0x28,
0x50, 0xd0, 0xb6, 0x3a, 0xcb, 0xad, 0xdc, 0x59, 0xe9, 0x52, 0x5e, 0xa1, 0xbf, 0xd1, 0x02, 0x80, 0x26, 0x42, 0x50, 0xd0, 0x76, 0x3b, 0x6b, 0xad, 0xdc, 0x05, 0xe9, 0x6a, 0x5e, 0xa1, 0x7f, 0xa3,
0x83, 0xfb, 0x06, 0x36, 0xdd, 0xce, 0xb2, 0xd3, 0xca, 0x9f, 0xcd, 0x5f, 0xca, 0x2b, 0x01, 0x08, 0x65, 0x00, 0x07, 0xf7, 0x0d, 0x6c, 0xba, 0x9d, 0x35, 0xa7, 0x95, 0xbf, 0x90, 0xbf, 0x9a, 0x57,
0x92, 0xa1, 0xd6, 0xb3, 0x06, 0x03, 0xdc, 0x73, 0x75, 0xcb, 0xec, 0x2c, 0xb7, 0x0a, 0x74, 0x6e, 0x02, 0x10, 0x24, 0x43, 0xad, 0x67, 0x0d, 0x06, 0xb8, 0xe7, 0xea, 0x96, 0xd9, 0x59, 0x6b, 0x15,
0x08, 0x26, 0xff, 0x4c, 0x82, 0x3a, 0x27, 0xcd, 0x19, 0x5a, 0xa6, 0x83, 0xd1, 0x75, 0x28, 0x39, 0x28, 0x6d, 0x08, 0x26, 0xff, 0xad, 0x04, 0x75, 0xce, 0x9a, 0x33, 0xb4, 0x4c, 0x07, 0xa3, 0xdb,
0xae, 0xea, 0x8e, 0x1c, 0x4e, 0xdd, 0xc9, 0x44, 0xea, 0x36, 0x28, 0x8a, 0xc2, 0x51, 0x13, 0xc9, 0x50, 0x72, 0x5c, 0xd5, 0x1d, 0x39, 0x9c, 0xbb, 0x33, 0x89, 0xdc, 0x6d, 0x53, 0x14, 0x85, 0xa3,
0x8b, 0x6e, 0x9f, 0x8f, 0x6f, 0x1f, 0x39, 0x42, 0x21, 0x7a, 0x04, 0xf9, 0x6f, 0x12, 0x34, 0x37, 0x26, 0xb2, 0x17, 0x5d, 0x3e, 0x1f, 0x5f, 0x3e, 0xb2, 0x85, 0x42, 0x74, 0x0b, 0xf2, 0xff, 0x48,
0xbc, 0xa1, 0xc7, 0xbd, 0x63, 0x50, 0xec, 0x59, 0x23, 0xd3, 0xa5, 0x04, 0xd6, 0x15, 0x36, 0x40, 0xd0, 0xdc, 0xf6, 0x86, 0x9e, 0xf4, 0x4e, 0x41, 0xb1, 0x67, 0x8d, 0x4c, 0x97, 0x32, 0x58, 0x57,
0xe7, 0xa0, 0xd6, 0xdb, 0x51, 0x4d, 0x13, 0x0f, 0xba, 0xa6, 0x6a, 0x60, 0x4a, 0x4a, 0x45, 0xa9, 0xd8, 0x00, 0x5d, 0x84, 0x5a, 0x6f, 0x5f, 0x35, 0x4d, 0x3c, 0xe8, 0x9a, 0xaa, 0x81, 0x29, 0x2b,
0x72, 0xd8, 0x5d, 0xd5, 0xc0, 0x99, 0x28, 0x3a, 0x0b, 0xd5, 0xa1, 0x6a, 0xbb, 0x7a, 0x88, 0x67, 0x15, 0xa5, 0xca, 0x61, 0x8f, 0x54, 0x03, 0x67, 0xe2, 0xe8, 0x02, 0x54, 0x87, 0xaa, 0xed, 0xea,
0x41, 0x10, 0x6a, 0x43, 0x59, 0x77, 0x3a, 0xc6, 0xd0, 0xb2, 0xdd, 0x56, 0xf1, 0xac, 0x74, 0xa9, 0x21, 0x99, 0x05, 0x41, 0xa8, 0x0d, 0x65, 0xdd, 0xe9, 0x18, 0x43, 0xcb, 0x76, 0x5b, 0xc5, 0x0b,
0xac, 0x88, 0x31, 0xd9, 0x41, 0xa7, 0xbf, 0x36, 0x55, 0x67, 0xb7, 0xb3, 0xdc, 0x2a, 0xb1, 0x1d, 0xd2, 0xd5, 0xb2, 0x22, 0xc6, 0x64, 0x05, 0x9d, 0xfe, 0xb5, 0xa3, 0x3a, 0x07, 0x9d, 0xb5, 0x56,
0x82, 0x30, 0xf9, 0x17, 0x12, 0xcc, 0xdf, 0x70, 0x1c, 0xbd, 0x6f, 0xc6, 0x4e, 0x36, 0x0f, 0x25, 0x89, 0xad, 0x10, 0x84, 0xc9, 0x7f, 0x2f, 0xc1, 0xe2, 0x5d, 0xc7, 0xd1, 0xfb, 0x66, 0x6c, 0x67,
0xd3, 0xd2, 0x70, 0x67, 0x99, 0x1e, 0x2d, 0xaf, 0xf0, 0x11, 0x3a, 0x09, 0x95, 0x21, 0xc6, 0x76, 0x8b, 0x50, 0x32, 0x2d, 0x0d, 0x77, 0xd6, 0xe8, 0xd6, 0xf2, 0x0a, 0x1f, 0xa1, 0x33, 0x50, 0x19,
0xd7, 0xb6, 0x06, 0xde, 0xc1, 0xca, 0x04, 0xa0, 0x58, 0x03, 0x8c, 0xee, 0xc1, 0x9c, 0x13, 0x59, 0x62, 0x6c, 0x77, 0x6d, 0x6b, 0xe0, 0x6d, 0xac, 0x4c, 0x00, 0x8a, 0x35, 0xc0, 0xe8, 0x31, 0x2c,
0x88, 0x69, 0x43, 0x75, 0xf1, 0xfc, 0xd5, 0x98, 0x3e, 0x5f, 0x8d, 0x6e, 0xaa, 0xc4, 0x67, 0xcb, 0x38, 0x91, 0x89, 0x98, 0x36, 0x54, 0x57, 0x2e, 0xdd, 0x8c, 0xe9, 0xf3, 0xcd, 0xe8, 0xa2, 0x4a,
0x1f, 0xe7, 0xe0, 0xa8, 0xc0, 0x63, 0xb4, 0x92, 0xdf, 0x84, 0xf3, 0x0e, 0xee, 0x0b, 0xf2, 0xd8, 0x9c, 0x5a, 0xfe, 0x3a, 0x07, 0x27, 0x05, 0x1e, 0xe3, 0x95, 0xfc, 0x4d, 0x24, 0xef, 0xe0, 0xbe,
0x20, 0x0b, 0xe7, 0x85, 0xc8, 0xf2, 0x41, 0x91, 0x65, 0x50, 0xd0, 0xa8, 0x3c, 0x8a, 0x71, 0x79, 0x60, 0x8f, 0x0d, 0xb2, 0x48, 0x5e, 0x1c, 0x59, 0x3e, 0x78, 0x64, 0x19, 0x14, 0x34, 0x7a, 0x1e,
0x9c, 0x81, 0x2a, 0x7e, 0x3c, 0xd4, 0x6d, 0xdc, 0x75, 0x75, 0x03, 0x53, 0x96, 0x17, 0x14, 0x60, 0xc5, 0xf8, 0x79, 0x9c, 0x87, 0x2a, 0x7e, 0x31, 0xd4, 0x6d, 0xdc, 0x75, 0x75, 0x03, 0x53, 0x91,
0xa0, 0x4d, 0xdd, 0x08, 0x6a, 0xf4, 0x4c, 0x66, 0x8d, 0x96, 0x7f, 0x29, 0xc1, 0xf1, 0x98, 0x94, 0x17, 0x14, 0x60, 0xa0, 0x1d, 0xdd, 0x08, 0x6a, 0xf4, 0x5c, 0x66, 0x8d, 0x96, 0xff, 0x41, 0x82,
0xb8, 0x89, 0x28, 0xd0, 0xa4, 0x27, 0xf7, 0x39, 0x43, 0x8c, 0x85, 0x30, 0xfc, 0xe2, 0x38, 0x86, 0xa5, 0xd8, 0x29, 0x71, 0x13, 0x51, 0xa0, 0x49, 0x77, 0xee, 0x4b, 0x86, 0x18, 0x0b, 0x11, 0xf8,
0xfb, 0xe8, 0x4a, 0x6c, 0x7e, 0x80, 0xc8, 0x5c, 0x76, 0x22, 0x77, 0xe1, 0xf8, 0x0a, 0x76, 0xf9, 0x95, 0x71, 0x02, 0xf7, 0xd1, 0x95, 0x18, 0x7d, 0x80, 0xc9, 0x5c, 0x76, 0x26, 0x0f, 0x60, 0x69,
0x06, 0xe4, 0x1b, 0x76, 0xa6, 0x77, 0x31, 0x61, 0x5b, 0xcc, 0xc5, 0x6c, 0xf1, 0x77, 0x39, 0x61, 0x1d, 0xbb, 0x7c, 0x01, 0xf2, 0x0d, 0x3b, 0xd3, 0xbb, 0x98, 0xb0, 0x2d, 0xe6, 0x62, 0xb6, 0xf8,
0x8b, 0x74, 0xab, 0x8e, 0xb9, 0x6d, 0xa1, 0x53, 0x50, 0x11, 0x28, 0x5c, 0x2b, 0x7c, 0x00, 0x7a, 0x2f, 0x39, 0x61, 0x8b, 0x74, 0xa9, 0x8e, 0xb9, 0x67, 0xa1, 0xb3, 0x50, 0x11, 0x28, 0x5c, 0x2b,
0x1d, 0x8a, 0x84, 0x52, 0xa6, 0x12, 0x8d, 0xc5, 0x73, 0xc9, 0x67, 0x0a, 0xac, 0xa9, 0x30, 0x7c, 0x7c, 0x00, 0xfa, 0x00, 0x8a, 0x84, 0x53, 0xa6, 0x12, 0x8d, 0x95, 0x8b, 0xc9, 0x7b, 0x0a, 0xcc,
0xd4, 0x81, 0x86, 0xe3, 0xaa, 0xb6, 0xdb, 0x1d, 0x5a, 0x0e, 0x95, 0x33, 0x55, 0x9c, 0xea, 0xa2, 0xa9, 0x30, 0x7c, 0xd4, 0x81, 0x86, 0xe3, 0xaa, 0xb6, 0xdb, 0x1d, 0x5a, 0x0e, 0x3d, 0x67, 0xaa,
0x1c, 0x5e, 0x41, 0x38, 0xe3, 0x35, 0xa7, 0xbf, 0xce, 0x31, 0x95, 0x3a, 0x9d, 0xe9, 0x0d, 0xd1, 0x38, 0xd5, 0x15, 0x39, 0x3c, 0x83, 0x70, 0xc6, 0x9b, 0x4e, 0x7f, 0x8b, 0x63, 0x2a, 0x75, 0x4a,
0x2d, 0xa8, 0x61, 0x53, 0xf3, 0x17, 0x2a, 0x64, 0x5e, 0xa8, 0x8a, 0x4d, 0x4d, 0x2c, 0xe3, 0xcb, 0xe9, 0x0d, 0xd1, 0x7d, 0xa8, 0x61, 0x53, 0xf3, 0x27, 0x2a, 0x64, 0x9e, 0xa8, 0x8a, 0x4d, 0x4d,
0xa7, 0x98, 0x5d, 0x3e, 0xdf, 0x97, 0xa0, 0x15, 0x17, 0xd0, 0x61, 0x1c, 0xed, 0x9b, 0x6c, 0x12, 0x4c, 0xe3, 0x9f, 0x4f, 0x31, 0xfb, 0xf9, 0xfc, 0xa9, 0x04, 0xad, 0xf8, 0x01, 0xcd, 0xe2, 0x68,
0x66, 0x02, 0x1a, 0x6b, 0xe1, 0x42, 0x48, 0x0a, 0x9f, 0x22, 0xff, 0x58, 0x82, 0xe7, 0x7c, 0x72, 0xef, 0x30, 0x22, 0xcc, 0x0e, 0x68, 0xac, 0x85, 0x8b, 0x43, 0x52, 0x38, 0x89, 0xfc, 0x57, 0x12,
0xe8, 0xa7, 0xa7, 0xa5, 0x2d, 0xe8, 0x32, 0x34, 0x75, 0xb3, 0x37, 0x18, 0x69, 0xf8, 0xbe, 0x79, 0xbc, 0xe1, 0xb3, 0x43, 0x3f, 0xbd, 0x2a, 0x6d, 0x41, 0xd7, 0xa1, 0xa9, 0x9b, 0xbd, 0xc1, 0x48,
0x07, 0xab, 0x03, 0x77, 0x67, 0x9f, 0xca, 0xb0, 0xac, 0xc4, 0xe0, 0xf2, 0x27, 0x12, 0xcc, 0x47, 0xc3, 0x4f, 0xcc, 0x87, 0x58, 0x1d, 0xb8, 0xfb, 0x47, 0xf4, 0x0c, 0xcb, 0x4a, 0x0c, 0x2e, 0x7f,
0xe9, 0x3a, 0x0c, 0x93, 0xfe, 0x0f, 0x8a, 0xba, 0xb9, 0x6d, 0x79, 0x3c, 0x5a, 0x18, 0x63, 0x94, 0x23, 0xc1, 0x62, 0x94, 0xaf, 0x59, 0x84, 0xf4, 0x5b, 0x50, 0xd4, 0xcd, 0x3d, 0xcb, 0x93, 0xd1,
0x64, 0x2f, 0x86, 0x2c, 0x1b, 0x70, 0x72, 0x05, 0xbb, 0x1d, 0xd3, 0xc1, 0xb6, 0xbb, 0xa4, 0x9b, 0xf2, 0x18, 0xa3, 0x24, 0x6b, 0x31, 0x64, 0xd9, 0x80, 0x33, 0xeb, 0xd8, 0xed, 0x98, 0x0e, 0xb6,
0x03, 0xab, 0xbf, 0xae, 0xba, 0x3b, 0x87, 0x30, 0xa8, 0x90, 0x6d, 0xe4, 0x22, 0xb6, 0x21, 0xff, 0xdd, 0x55, 0xdd, 0x1c, 0x58, 0xfd, 0x2d, 0xd5, 0xdd, 0x9f, 0xc1, 0xa0, 0x42, 0xb6, 0x91, 0x8b,
0x4a, 0x82, 0x53, 0xc9, 0xfb, 0xf1, 0xa3, 0xb7, 0xa1, 0xbc, 0xad, 0xe3, 0x81, 0x46, 0xf8, 0x2b, 0xd8, 0x86, 0xfc, 0xbd, 0x04, 0x67, 0x93, 0xd7, 0xe3, 0x5b, 0x6f, 0x43, 0x79, 0x4f, 0xc7, 0x03,
0x51, 0xfe, 0x8a, 0x31, 0x31, 0xac, 0x21, 0x41, 0xe6, 0x27, 0x3c, 0x97, 0xa2, 0xcd, 0x1b, 0xae, 0x8d, 0xc8, 0x57, 0xa2, 0xf2, 0x15, 0x63, 0x62, 0x58, 0x43, 0x82, 0xcc, 0x77, 0x78, 0x31, 0x45,
0xad, 0x9b, 0xfd, 0x55, 0xdd, 0x71, 0x15, 0x86, 0x1f, 0xe0, 0x67, 0x3e, 0xbb, 0x1a, 0x7f, 0x4f, 0x9b, 0xb7, 0x5d, 0x5b, 0x37, 0xfb, 0x1b, 0xba, 0xe3, 0x2a, 0x0c, 0x3f, 0x20, 0xcf, 0x7c, 0x76,
0x82, 0x85, 0x15, 0xec, 0xde, 0x14, 0x7e, 0x99, 0x7c, 0xd7, 0x1d, 0x57, 0xef, 0x39, 0x4f, 0x36, 0x35, 0xfe, 0x13, 0x09, 0x96, 0xd7, 0xb1, 0x7b, 0x4f, 0xf8, 0x65, 0xf2, 0x5d, 0x77, 0x5c, 0xbd,
0xa3, 0xc9, 0x10, 0xa0, 0xe5, 0xcf, 0x24, 0x38, 0x93, 0x4a, 0x0c, 0x67, 0x1d, 0xf7, 0x3b, 0x9e, 0xe7, 0xbc, 0xdc, 0x8c, 0x26, 0x43, 0x80, 0x96, 0xbf, 0x93, 0xe0, 0x7c, 0x2a, 0x33, 0x5c, 0x74,
0x57, 0x4e, 0xf6, 0x3b, 0x5f, 0xc3, 0xfb, 0xef, 0xa9, 0x83, 0x11, 0x5e, 0x57, 0x75, 0x9b, 0xf9, 0xdc, 0xef, 0x78, 0x5e, 0x39, 0xd9, 0xef, 0xfc, 0x2e, 0x3e, 0xfa, 0x4c, 0x1d, 0x8c, 0xf0, 0x96,
0x9d, 0x29, 0xbd, 0xf0, 0x6f, 0x25, 0x38, 0xbd, 0x82, 0xdd, 0x75, 0x2f, 0x26, 0x3d, 0x43, 0xee, 0xaa, 0xdb, 0xcc, 0xef, 0x4c, 0xe9, 0x85, 0xff, 0x59, 0x82, 0x73, 0xeb, 0xd8, 0xdd, 0xf2, 0x62,
0x10, 0x9c, 0x40, 0x6c, 0xf4, 0x52, 0xaa, 0x10, 0x4c, 0xfe, 0x01, 0x13, 0x67, 0x22, 0xbd, 0xcf, 0xd2, 0x6b, 0x94, 0x0e, 0xc1, 0x09, 0xc4, 0x46, 0x2f, 0xa5, 0x0a, 0xc1, 0xe4, 0x3f, 0x63, 0xc7,
0x84, 0x81, 0x0b, 0xd4, 0x12, 0x02, 0x26, 0x79, 0x93, 0xa5, 0x0e, 0x9c, 0x7d, 0xf2, 0xcf, 0x25, 0x99, 0xc8, 0xef, 0x6b, 0x11, 0xe0, 0x32, 0xb5, 0x84, 0x80, 0x49, 0xde, 0x63, 0xa9, 0x03, 0x17,
0x38, 0x71, 0xa3, 0xf7, 0x70, 0xa4, 0xdb, 0x98, 0x23, 0xad, 0x5a, 0xbd, 0xdd, 0xe9, 0x99, 0xeb, 0x9f, 0xfc, 0x77, 0x12, 0x9c, 0xbe, 0xdb, 0x7b, 0x36, 0xd2, 0x6d, 0xcc, 0x91, 0x36, 0xac, 0xde,
0xa7, 0x59, 0xb9, 0x50, 0x9a, 0x35, 0x29, 0xa1, 0x9e, 0x87, 0x92, 0xcb, 0xf2, 0x3a, 0x96, 0xa9, 0xc1, 0xf4, 0xc2, 0xf5, 0xd3, 0xac, 0x5c, 0x28, 0xcd, 0x9a, 0x94, 0x50, 0x2f, 0x42, 0xc9, 0x65,
0xf0, 0x11, 0xa5, 0x4f, 0xc1, 0x03, 0xac, 0x3a, 0xff, 0x9d, 0xf4, 0x7d, 0x56, 0x80, 0xda, 0x7b, 0x79, 0x1d, 0xcb, 0x54, 0xf8, 0x88, 0xf2, 0xa7, 0xe0, 0x01, 0x56, 0x9d, 0x5f, 0x4f, 0xfe, 0xbe,
0x3c, 0x1d, 0xa3, 0x51, 0x3b, 0xaa, 0x49, 0x52, 0x72, 0xe2, 0x15, 0xc8, 0xe0, 0x92, 0x92, 0xba, 0x2b, 0x40, 0xed, 0x33, 0x9e, 0x8e, 0xd1, 0xa8, 0x1d, 0xd5, 0x24, 0x29, 0x39, 0xf1, 0x0a, 0x64,
0x15, 0xa8, 0x3b, 0x18, 0xef, 0x4e, 0x13, 0xa3, 0x6b, 0x64, 0xa2, 0x88, 0xad, 0xab, 0x30, 0x37, 0x70, 0x49, 0x49, 0xdd, 0x3a, 0xd4, 0x1d, 0x8c, 0x0f, 0xa6, 0x89, 0xd1, 0x35, 0x42, 0x28, 0x62,
0x32, 0xb7, 0x49, 0x15, 0x82, 0x35, 0xce, 0x40, 0xa6, 0xb9, 0x93, 0x7d, 0x77, 0x7c, 0x22, 0xba, 0xeb, 0x06, 0x2c, 0x8c, 0xcc, 0x3d, 0x52, 0x85, 0x60, 0x8d, 0x0b, 0x90, 0x69, 0xee, 0x64, 0xdf,
0x03, 0xb3, 0xd1, 0xb5, 0x8a, 0x99, 0xd6, 0x8a, 0x4e, 0x43, 0x1d, 0x68, 0x6a, 0xb6, 0x35, 0x1c, 0x1d, 0x27, 0x44, 0x0f, 0x61, 0x3e, 0x3a, 0x57, 0x31, 0xd3, 0x5c, 0x51, 0x32, 0xd4, 0x81, 0xa6,
0x62, 0xad, 0xeb, 0x78, 0x4b, 0x95, 0xb2, 0x2d, 0xc5, 0xe7, 0x89, 0xa5, 0x5e, 0x81, 0xa3, 0x51, 0x66, 0x5b, 0xc3, 0x21, 0xd6, 0xba, 0x8e, 0x37, 0x55, 0x29, 0xdb, 0x54, 0x9c, 0x4e, 0x4c, 0xf5,
0x4a, 0x3b, 0x1a, 0x49, 0x48, 0x89, 0x0c, 0x93, 0x3e, 0xa1, 0x2b, 0x30, 0x17, 0xc7, 0x2f, 0x53, 0x2e, 0x9c, 0x8c, 0x72, 0xda, 0xd1, 0x48, 0x42, 0x4a, 0xce, 0x30, 0xe9, 0x13, 0xba, 0x01, 0x0b,
0xfc, 0xf8, 0x07, 0xf4, 0x32, 0xa0, 0x08, 0xa9, 0x04, 0xbd, 0xc2, 0xd0, 0xc3, 0xc4, 0x74, 0x34, 0x71, 0xfc, 0x32, 0xc5, 0x8f, 0x7f, 0x40, 0xef, 0x00, 0x8a, 0xb0, 0x4a, 0xd0, 0x2b, 0x0c, 0x3d,
0x47, 0xfe, 0x54, 0x82, 0xf9, 0x07, 0xaa, 0xdb, 0xdb, 0x59, 0x36, 0xb8, 0xad, 0x1d, 0xc2, 0x57, 0xcc, 0x4c, 0x47, 0x73, 0xe4, 0x6f, 0x25, 0x58, 0x7c, 0xaa, 0xba, 0xbd, 0xfd, 0x35, 0x83, 0xdb,
0xbd, 0x0d, 0x95, 0x47, 0x5c, 0x2f, 0xbc, 0x80, 0x74, 0x26, 0x81, 0x3f, 0x41, 0x0d, 0x54, 0xfc, 0xda, 0x0c, 0xbe, 0xea, 0x63, 0xa8, 0x3c, 0xe7, 0x7a, 0xe1, 0x05, 0xa4, 0xf3, 0x09, 0xf2, 0x09,
0x19, 0xf2, 0x97, 0x12, 0x1c, 0xa3, 0x25, 0xa8, 0xc7, 0xac, 0xaf, 0xde, 0x6b, 0x4e, 0x28, 0x43, 0x6a, 0xa0, 0xe2, 0x53, 0xc8, 0x3f, 0x49, 0x70, 0x8a, 0x96, 0xa0, 0x9e, 0xb0, 0x7e, 0x7e, 0xaf,
0xd1, 0x45, 0x68, 0x18, 0xaa, 0xbd, 0xbb, 0xe1, 0xe3, 0x14, 0x29, 0x4e, 0x04, 0x2a, 0x3f, 0x06, 0x39, 0xa1, 0x0c, 0x45, 0x57, 0xa0, 0x61, 0xa8, 0xf6, 0xc1, 0xb6, 0x8f, 0x53, 0xa4, 0x38, 0x11,
0xe0, 0xa3, 0x35, 0xa7, 0x3f, 0x05, 0xfd, 0x6f, 0xc0, 0x0c, 0xdf, 0x95, 0xbb, 0xcf, 0x49, 0x7a, 0xa8, 0xfc, 0x02, 0x80, 0x8f, 0x36, 0x9d, 0xfe, 0x14, 0xfc, 0x7f, 0x08, 0x73, 0x7c, 0x55, 0xee,
0xe6, 0xa1, 0xcb, 0x7f, 0x96, 0xa0, 0xe1, 0x87, 0x44, 0x6a, 0xe4, 0x0d, 0xc8, 0x09, 0xd3, 0xce, 0x3e, 0x27, 0xe9, 0x99, 0x87, 0x2e, 0xff, 0xa7, 0x04, 0x0d, 0x3f, 0x24, 0x52, 0x23, 0x6f, 0x40,
0x75, 0x96, 0xd1, 0xdb, 0x50, 0x62, 0xed, 0x09, 0xbe, 0xf6, 0x85, 0xf0, 0xda, 0xbc, 0x75, 0x11, 0x4e, 0x98, 0x76, 0xae, 0xb3, 0x86, 0x3e, 0x86, 0x12, 0x6b, 0x4f, 0xf0, 0xb9, 0x2f, 0x87, 0xe7,
0x88, 0xab, 0x14, 0xa0, 0xf0, 0x49, 0x84, 0x47, 0x22, 0x8a, 0x08, 0xe7, 0xe3, 0x43, 0x50, 0x07, 0xe6, 0xad, 0x8b, 0x40, 0x5c, 0xa5, 0x00, 0x85, 0x13, 0x11, 0x19, 0x89, 0x28, 0x22, 0x9c, 0x8f,
0x66, 0xc3, 0x29, 0xbb, 0x67, 0xc2, 0x67, 0xd3, 0x82, 0xc7, 0xb2, 0xea, 0xaa, 0x34, 0x76, 0x34, 0x0f, 0x41, 0x1d, 0x98, 0x0f, 0xa7, 0xec, 0x9e, 0x09, 0x5f, 0x48, 0x0b, 0x1e, 0x6b, 0xaa, 0xab,
0x42, 0x19, 0xbb, 0x23, 0xff, 0xbb, 0x08, 0xd5, 0xc0, 0x29, 0x63, 0x27, 0x89, 0x8a, 0x34, 0x37, 0xd2, 0xd8, 0xd1, 0x08, 0x65, 0xec, 0x8e, 0xfc, 0x8b, 0x22, 0x54, 0x03, 0xbb, 0x8c, 0xed, 0x24,
0xb9, 0x6e, 0xcc, 0xc7, 0xeb, 0xc6, 0x0b, 0xd0, 0xd0, 0x69, 0xf2, 0xd5, 0xe5, 0xaa, 0x48, 0xbd, 0x7a, 0xa4, 0xb9, 0xc9, 0x75, 0x63, 0x3e, 0x5e, 0x37, 0x5e, 0x86, 0x86, 0x4e, 0x93, 0xaf, 0x2e,
0x66, 0x45, 0xa9, 0x33, 0x28, 0xb7, 0x0b, 0xb4, 0x00, 0x55, 0x73, 0x64, 0x74, 0xad, 0xed, 0xae, 0x57, 0x45, 0xea, 0x35, 0x2b, 0x4a, 0x9d, 0x41, 0xb9, 0x5d, 0xa0, 0x65, 0xa8, 0x9a, 0x23, 0xa3,
0x6d, 0xed, 0x39, 0xbc, 0x00, 0xad, 0x98, 0x23, 0xe3, 0xeb, 0xdb, 0x8a, 0xb5, 0xe7, 0xf8, 0x35, 0x6b, 0xed, 0x75, 0x6d, 0xeb, 0xd0, 0xe1, 0x05, 0x68, 0xc5, 0x1c, 0x19, 0xbf, 0xb7, 0xa7, 0x58,
0x4e, 0xe9, 0x80, 0x35, 0xce, 0x02, 0x54, 0x0d, 0xf5, 0x31, 0x59, 0xb5, 0x6b, 0x8e, 0x0c, 0x5a, 0x87, 0x8e, 0x5f, 0xe3, 0x94, 0x8e, 0x59, 0xe3, 0x2c, 0x43, 0xd5, 0x50, 0x5f, 0x90, 0x59, 0xbb,
0x9b, 0xe6, 0x95, 0x8a, 0xa1, 0x3e, 0x56, 0xac, 0xbd, 0xbb, 0x23, 0x03, 0x5d, 0x82, 0xe6, 0x40, 0xe6, 0xc8, 0xa0, 0xb5, 0x69, 0x5e, 0xa9, 0x18, 0xea, 0x0b, 0xc5, 0x3a, 0x7c, 0x34, 0x32, 0xd0,
0x75, 0xdc, 0x6e, 0xb0, 0xb8, 0x2d, 0xd3, 0xe2, 0xb6, 0x41, 0xe0, 0xb7, 0xfc, 0x02, 0x37, 0x5e, 0x55, 0x68, 0x0e, 0x54, 0xc7, 0xed, 0x06, 0x8b, 0xdb, 0x32, 0x2d, 0x6e, 0x1b, 0x04, 0x7e, 0xdf,
0x2d, 0x55, 0x0e, 0x51, 0x2d, 0x69, 0xc6, 0xc0, 0x5f, 0x08, 0xb2, 0x57, 0x4b, 0x9a, 0x31, 0x10, 0x2f, 0x70, 0xe3, 0xd5, 0x52, 0x65, 0x86, 0x6a, 0x49, 0x33, 0x06, 0xfe, 0x44, 0x90, 0xbd, 0x5a,
0xcb, 0xbc, 0x01, 0x33, 0x5b, 0x34, 0xa5, 0x75, 0x5a, 0xd5, 0x54, 0x87, 0x79, 0x9b, 0x64, 0xb3, 0xd2, 0x8c, 0x81, 0x98, 0xe6, 0x43, 0x98, 0xdb, 0xa5, 0x29, 0xad, 0xd3, 0xaa, 0xa6, 0x3a, 0xcc,
0x2c, 0xf3, 0x55, 0x3c, 0x74, 0xf4, 0x16, 0x54, 0x68, 0x26, 0x41, 0xe7, 0xd6, 0x32, 0xcd, 0xf5, 0x07, 0x24, 0x9b, 0x65, 0x99, 0xaf, 0xe2, 0xa1, 0xa3, 0x8f, 0xa0, 0x42, 0x33, 0x09, 0x4a, 0x5b,
0x27, 0x90, 0xd9, 0x1a, 0x1e, 0xb8, 0x2a, 0x9d, 0x5d, 0xcf, 0x36, 0x5b, 0x4c, 0x20, 0x4e, 0xba, 0xcb, 0x44, 0xeb, 0x13, 0x10, 0x6a, 0x0d, 0x0f, 0x5c, 0x95, 0x52, 0xd7, 0xb3, 0x51, 0x0b, 0x02,
0x67, 0x63, 0xd5, 0xc5, 0xda, 0xd2, 0xfe, 0x4d, 0xcb, 0x18, 0xaa, 0x54, 0x99, 0x5a, 0x0d, 0x5a, 0xe2, 0xa4, 0x7b, 0x36, 0x56, 0x5d, 0xac, 0xad, 0x1e, 0xdd, 0xb3, 0x8c, 0xa1, 0x4a, 0x95, 0xa9,
0xb6, 0x24, 0x7d, 0x22, 0x8e, 0xa1, 0x27, 0x46, 0xb7, 0x6d, 0xcb, 0x68, 0xcd, 0x32, 0xc7, 0x10, 0xd5, 0xa0, 0x65, 0x4b, 0xd2, 0x27, 0xe2, 0x18, 0x7a, 0x62, 0xf4, 0xc0, 0xb6, 0x8c, 0xd6, 0x3c,
0x86, 0xa2, 0xd3, 0x00, 0x9e, 0x7b, 0x56, 0xdd, 0x56, 0x93, 0x4a, 0xb1, 0xc2, 0x21, 0x37, 0x5c, 0x73, 0x0c, 0x61, 0x28, 0x3a, 0x07, 0xe0, 0xb9, 0x67, 0xd5, 0x6d, 0x35, 0xe9, 0x29, 0x56, 0x38,
0xf9, 0x23, 0x38, 0xe6, 0x6b, 0x48, 0x40, 0x1a, 0x71, 0xc1, 0x4a, 0xd3, 0x0a, 0x76, 0x7c, 0x31, 0xe4, 0xae, 0x2b, 0x7f, 0x05, 0xa7, 0x7c, 0x0d, 0x09, 0x9c, 0x46, 0xfc, 0x60, 0xa5, 0x69, 0x0f,
0xf2, 0xd7, 0x02, 0xcc, 0x6f, 0xa8, 0x8f, 0xf0, 0xd3, 0xaf, 0x7b, 0x32, 0xf9, 0xe3, 0x55, 0x98, 0x76, 0x7c, 0x31, 0xf2, 0xdf, 0x05, 0x58, 0xdc, 0x56, 0x9f, 0xe3, 0x57, 0x5f, 0xf7, 0x64, 0xf2,
0xa3, 0xa5, 0xce, 0x62, 0x80, 0x9e, 0x31, 0x09, 0x41, 0x50, 0x9c, 0xf1, 0x89, 0xe8, 0x5d, 0x92, 0xc7, 0x1b, 0xb0, 0x40, 0x4b, 0x9d, 0x95, 0x00, 0x3f, 0x63, 0x12, 0x82, 0xe0, 0x71, 0xc6, 0x09,
0xc9, 0xe0, 0xde, 0xee, 0xba, 0xa5, 0xfb, 0xc9, 0xc0, 0xe9, 0x84, 0x75, 0x6e, 0x0a, 0x2c, 0x25, 0xd1, 0xa7, 0x24, 0x93, 0xc1, 0xbd, 0x83, 0x2d, 0x4b, 0xf7, 0x93, 0x81, 0x73, 0x09, 0xf3, 0xdc,
0x38, 0x03, 0xad, 0xc7, 0x5d, 0x1b, 0x4b, 0x03, 0x5e, 0x18, 0x5b, 0x7d, 0xfb, 0xdc, 0x8f, 0x7a, 0x13, 0x58, 0x4a, 0x90, 0x02, 0x6d, 0xc5, 0x5d, 0x1b, 0x4b, 0x03, 0xde, 0x1a, 0x5b, 0x7d, 0xfb,
0x38, 0xd4, 0x82, 0x19, 0x1e, 0xc3, 0xa9, 0xdd, 0x97, 0x15, 0x6f, 0x88, 0xd6, 0xe1, 0x28, 0x3b, 0xd2, 0x8f, 0x7a, 0x38, 0xd4, 0x82, 0x39, 0x1e, 0xc3, 0xa9, 0xdd, 0x97, 0x15, 0x6f, 0x88, 0xb6,
0xc1, 0x06, 0x57, 0x6a, 0x76, 0xf8, 0x72, 0xa6, 0xc3, 0x27, 0x4d, 0x0d, 0xdb, 0x44, 0xe5, 0xa0, 0xe0, 0x24, 0xdb, 0xc1, 0x36, 0x57, 0x6a, 0xb6, 0xf9, 0x72, 0xa6, 0xcd, 0x27, 0x91, 0x86, 0x6d,
0x36, 0xd1, 0x82, 0x19, 0xae, 0xa7, 0xd4, 0x17, 0x94, 0x15, 0x6f, 0x48, 0xc4, 0xcc, 0xfa, 0x9a, 0xa2, 0x72, 0x5c, 0x9b, 0x68, 0xc1, 0x1c, 0xd7, 0x53, 0xea, 0x0b, 0xca, 0x8a, 0x37, 0x24, 0xc7,
0xba, 0xd9, 0x6f, 0x55, 0xe9, 0x37, 0x1f, 0x40, 0x6a, 0x46, 0xf0, 0xf9, 0x39, 0xa1, 0x4f, 0xf4, 0xcc, 0xfa, 0x9a, 0xba, 0xd9, 0x6f, 0x55, 0xe9, 0x37, 0x1f, 0x40, 0x6a, 0x46, 0xf0, 0xe5, 0x39,
0x0e, 0x94, 0x85, 0x86, 0xe7, 0x32, 0x6b, 0xb8, 0x98, 0x13, 0xf5, 0xd1, 0xf9, 0x88, 0x8f, 0x96, 0xa1, 0x4f, 0xf4, 0x09, 0x94, 0x85, 0x86, 0xe7, 0x32, 0x6b, 0xb8, 0xa0, 0x89, 0xfa, 0xe8, 0x7c,
0xff, 0x22, 0x41, 0x6d, 0x99, 0x1c, 0x69, 0xd5, 0xea, 0xd3, 0x88, 0x72, 0x01, 0x1a, 0x36, 0xee, 0xc4, 0x47, 0xcb, 0xff, 0x25, 0x41, 0x6d, 0x8d, 0x6c, 0x69, 0xc3, 0xea, 0xd3, 0x88, 0x72, 0x19,
0x59, 0xb6, 0xd6, 0xc5, 0xa6, 0x6b, 0xeb, 0x98, 0xb5, 0x17, 0x0a, 0x4a, 0x9d, 0x41, 0x6f, 0x31, 0x1a, 0x36, 0xee, 0x59, 0xb6, 0xd6, 0xc5, 0xa6, 0x6b, 0xeb, 0x98, 0xb5, 0x17, 0x0a, 0x4a, 0x9d,
0x20, 0x41, 0x23, 0x6e, 0xd7, 0x71, 0x55, 0x63, 0xd8, 0xdd, 0x26, 0xe6, 0x9d, 0x63, 0x68, 0x02, 0x41, 0xef, 0x33, 0x20, 0x41, 0x23, 0x6e, 0xd7, 0x71, 0x55, 0x63, 0xd8, 0xdd, 0x23, 0xe6, 0x9d,
0x4a, 0xad, 0xfb, 0x1c, 0xd4, 0x7c, 0x34, 0xd7, 0xa2, 0xfb, 0x17, 0x94, 0xaa, 0x80, 0x6d, 0x5a, 0x63, 0x68, 0x02, 0x4a, 0xad, 0xfb, 0x22, 0xd4, 0x7c, 0x34, 0xd7, 0xa2, 0xeb, 0x17, 0x94, 0xaa,
0xe8, 0x79, 0x68, 0x50, 0x9e, 0x76, 0x07, 0x56, 0xbf, 0x4b, 0x4a, 0x71, 0x1e, 0x6c, 0x6a, 0x1a, 0x80, 0xed, 0x58, 0xe8, 0x4d, 0x68, 0x50, 0x99, 0x76, 0x07, 0x56, 0xbf, 0x4b, 0x4a, 0x71, 0x1e,
0x27, 0x8b, 0xc8, 0x2a, 0x8c, 0xe5, 0xe8, 0x1f, 0x62, 0x1e, 0x6e, 0x04, 0xd6, 0x86, 0xfe, 0x21, 0x6c, 0x6a, 0x1a, 0x67, 0x8b, 0x9c, 0x55, 0x18, 0xcb, 0xd1, 0xbf, 0xc4, 0x3c, 0xdc, 0x08, 0xac,
0x26, 0xb1, 0xbe, 0x4e, 0x62, 0xe7, 0x5d, 0x4b, 0xc3, 0x9b, 0x53, 0x66, 0x1a, 0x19, 0x7a, 0xb6, 0x6d, 0xfd, 0x4b, 0x4c, 0x62, 0x7d, 0x9d, 0xc4, 0xce, 0x47, 0x96, 0x86, 0x77, 0xa6, 0xcc, 0x34,
0xa7, 0xa0, 0x22, 0x4e, 0xc0, 0x8f, 0xe4, 0x03, 0xd0, 0x6d, 0x68, 0x78, 0x39, 0x71, 0x97, 0x95, 0x32, 0xf4, 0x6c, 0xcf, 0x42, 0x45, 0xec, 0x80, 0x6f, 0xc9, 0x07, 0xa0, 0x07, 0xd0, 0xf0, 0x72,
0x8a, 0x85, 0xd4, 0xcc, 0x2f, 0x10, 0xfd, 0x1c, 0xa5, 0xee, 0x4d, 0xa3, 0x43, 0xf9, 0x36, 0xd4, 0xe2, 0x2e, 0x2b, 0x15, 0x0b, 0xa9, 0x99, 0x5f, 0x20, 0xfa, 0x39, 0x4a, 0xdd, 0x23, 0xa3, 0x43,
0x82, 0x9f, 0xc9, 0xae, 0x1b, 0x51, 0x45, 0x11, 0x00, 0xa2, 0x8d, 0x77, 0x47, 0x06, 0x91, 0x29, 0xf9, 0x01, 0xd4, 0x82, 0x9f, 0xc9, 0xaa, 0xdb, 0x51, 0x45, 0x11, 0x00, 0xa2, 0x8d, 0x8f, 0x46,
0x77, 0x2c, 0xde, 0x50, 0xfe, 0x44, 0x82, 0x3a, 0x0f, 0xd9, 0x1b, 0xe2, 0x4e, 0x82, 0x1e, 0x4d, 0x06, 0x39, 0x53, 0xee, 0x58, 0xbc, 0xa1, 0xfc, 0x8d, 0x04, 0x75, 0x1e, 0xb2, 0xb7, 0xc5, 0x9d,
0xa2, 0x47, 0xa3, 0xbf, 0xd1, 0xff, 0x87, 0x1b, 0x92, 0xcf, 0x27, 0x3a, 0x01, 0xba, 0x08, 0xcd, 0x04, 0xdd, 0x9a, 0x44, 0xb7, 0x46, 0xff, 0x46, 0xbf, 0x1d, 0x6e, 0x48, 0xbe, 0x99, 0xe8, 0x04,
0x8e, 0x43, 0xf1, 0x3a, 0x4b, 0x73, 0xe2, 0x63, 0xa2, 0x68, 0x5c, 0x34, 0x54, 0xd1, 0x5a, 0x30, 0xe8, 0x24, 0x34, 0x3b, 0x0e, 0xc5, 0xeb, 0x2c, 0xcd, 0x89, 0xaf, 0x89, 0xa2, 0xf1, 0xa3, 0xa1,
0xa3, 0x6a, 0x9a, 0x8d, 0x1d, 0x87, 0xd3, 0xe1, 0x0d, 0xc9, 0x97, 0x47, 0xd8, 0x76, 0x3c, 0x95, 0x8a, 0xd6, 0x82, 0x39, 0x55, 0xd3, 0x6c, 0xec, 0x38, 0x9c, 0x0f, 0x6f, 0x48, 0xbe, 0x3c, 0xc7,
0xcf, 0x2b, 0xde, 0x10, 0xbd, 0x05, 0x65, 0x91, 0x4e, 0xe7, 0x93, 0x52, 0xa8, 0x20, 0x9d, 0xbc, 0xb6, 0xe3, 0xa9, 0x7c, 0x5e, 0xf1, 0x86, 0xe8, 0x23, 0x28, 0x8b, 0x74, 0x3a, 0x9f, 0x94, 0x42,
0x94, 0x16, 0x33, 0xe4, 0xdf, 0xe7, 0xa0, 0xc1, 0x19, 0xb6, 0xc4, 0x63, 0xea, 0x78, 0xe3, 0x5b, 0x05, 0xf9, 0xe4, 0xa5, 0xb4, 0xa0, 0x90, 0xff, 0x35, 0x07, 0x0d, 0x2e, 0xb0, 0x55, 0x1e, 0x53,
0x82, 0xda, 0xb6, 0x6f, 0xfb, 0xe3, 0x9a, 0x66, 0x41, 0x17, 0x11, 0x9a, 0x33, 0xc9, 0x00, 0xc3, 0xc7, 0x1b, 0xdf, 0x2a, 0xd4, 0xf6, 0x7c, 0xdb, 0x1f, 0xd7, 0x34, 0x0b, 0xba, 0x88, 0x10, 0xcd,
0x51, 0xbd, 0x70, 0xa8, 0xa8, 0x5e, 0x3c, 0xa8, 0x07, 0x8b, 0xe7, 0x79, 0xa5, 0x84, 0x3c, 0x4f, 0x24, 0x03, 0x0c, 0x47, 0xf5, 0xc2, 0x4c, 0x51, 0xbd, 0x78, 0x5c, 0x0f, 0x16, 0xcf, 0xf3, 0x4a,
0xfe, 0x16, 0x54, 0x03, 0x0b, 0x50, 0x0f, 0xcd, 0xba, 0x6d, 0x9c, 0x63, 0xde, 0x10, 0x5d, 0xf7, 0x09, 0x79, 0x9e, 0xfc, 0x07, 0x50, 0x0d, 0x4c, 0x40, 0x3d, 0x34, 0xeb, 0xb6, 0x71, 0x89, 0x79,
0x73, 0x1b, 0xc6, 0xaa, 0x13, 0x09, 0xb4, 0x44, 0xd2, 0x1a, 0xf9, 0xd7, 0x12, 0x94, 0xf8, 0xca, 0x43, 0x74, 0xdb, 0xcf, 0x6d, 0x98, 0xa8, 0x4e, 0x27, 0xf0, 0x12, 0x49, 0x6b, 0xe4, 0x7f, 0x94,
0x67, 0xa0, 0xca, 0x9d, 0x0e, 0xcd, 0xfb, 0xd8, 0xea, 0xc0, 0x41, 0x24, 0xf1, 0x7b, 0x72, 0x5e, 0xa0, 0xc4, 0x67, 0x3e, 0x0f, 0x55, 0xee, 0x74, 0x68, 0xde, 0xc7, 0x66, 0x07, 0x0e, 0x22, 0x89,
0xe7, 0x04, 0x94, 0x23, 0xfe, 0x66, 0x86, 0x87, 0x05, 0xef, 0x53, 0xc0, 0xc9, 0x90, 0x4f, 0xd4, 0xdf, 0xcb, 0xf3, 0x3a, 0xa7, 0xa1, 0x1c, 0xf1, 0x37, 0x73, 0x3c, 0x2c, 0x78, 0x9f, 0x02, 0x4e,
0xbf, 0x7c, 0x29, 0xd1, 0x6b, 0x05, 0x05, 0xf7, 0xac, 0x47, 0xd8, 0xde, 0x3f, 0x7c, 0x3f, 0xf6, 0x86, 0x7c, 0xa2, 0xfe, 0xe5, 0x27, 0x89, 0x5e, 0x2b, 0x28, 0xb8, 0x67, 0x3d, 0xc7, 0xf6, 0xd1,
0xcd, 0x80, 0x42, 0x67, 0xac, 0x0f, 0xc5, 0x04, 0xf4, 0xa6, 0xcf, 0xee, 0x7c, 0x52, 0x33, 0x2a, 0xec, 0xfd, 0xd8, 0x3b, 0x01, 0x85, 0xce, 0x58, 0x1f, 0x0a, 0x02, 0x74, 0xc7, 0x17, 0x77, 0x3e,
0xe8, 0x61, 0xb8, 0x3a, 0xfa, 0x6c, 0xff, 0x21, 0xeb, 0x2c, 0x87, 0x8f, 0x32, 0x6d, 0x5e, 0xf3, 0xa9, 0x19, 0x15, 0xf4, 0x30, 0x5c, 0x1d, 0x7d, 0xb1, 0xff, 0x39, 0xeb, 0x2c, 0x87, 0xb7, 0x32,
0x44, 0xca, 0x0e, 0xf9, 0x47, 0x12, 0x9c, 0x58, 0xc1, 0xee, 0xed, 0x70, 0xaf, 0xe1, 0x59, 0x53, 0x6d, 0x5e, 0xf3, 0x52, 0xca, 0x0e, 0xf9, 0x2f, 0x25, 0x38, 0xbd, 0x8e, 0xdd, 0x07, 0xe1, 0x5e,
0x65, 0x40, 0x3b, 0x89, 0xa8, 0xc3, 0x48, 0xbd, 0x0d, 0x65, 0xd1, 0x35, 0x61, 0xf7, 0x03, 0x62, 0xc3, 0xeb, 0xe6, 0xca, 0x80, 0x76, 0x12, 0x53, 0xb3, 0x9c, 0x7a, 0x1b, 0xca, 0xa2, 0x6b, 0xc2,
0x2c, 0x7f, 0x57, 0x82, 0x16, 0xdf, 0x85, 0xee, 0x49, 0x52, 0xea, 0x01, 0x76, 0xb1, 0xf6, 0x55, 0xee, 0x07, 0xc4, 0x58, 0xfe, 0x63, 0x09, 0x5a, 0x7c, 0x15, 0xba, 0x26, 0x49, 0xa9, 0x07, 0xd8,
0xd7, 0xcd, 0x5f, 0x48, 0xd0, 0x0c, 0x7a, 0x7c, 0xea, 0xb4, 0x5f, 0x83, 0x22, 0x6d, 0x4f, 0x70, 0xc5, 0xda, 0xcf, 0x5d, 0x37, 0xff, 0x28, 0x41, 0x33, 0xe8, 0xf1, 0xa9, 0xd3, 0x7e, 0x1f, 0x8a,
0x0a, 0x26, 0x2a, 0x2b, 0xc3, 0x26, 0x2e, 0x83, 0xa6, 0x79, 0x9b, 0x22, 0x38, 0xf1, 0xa1, 0x1f, 0xb4, 0x3d, 0xc1, 0x39, 0x98, 0xa8, 0xac, 0x0c, 0x9b, 0xb8, 0x0c, 0x9a, 0xe6, 0xed, 0x88, 0xe0,
0x76, 0xf2, 0x07, 0x0f, 0x3b, 0x3c, 0x0c, 0x5b, 0x23, 0xb2, 0x2e, 0xeb, 0xeb, 0xf9, 0x00, 0xf9, 0xc4, 0x87, 0x7e, 0xd8, 0xc9, 0x1f, 0x3f, 0xec, 0xf0, 0x30, 0x6c, 0x8d, 0xc8, 0xbc, 0xac, 0xaf,
0xf3, 0x1c, 0xb4, 0xfc, 0x7a, 0xe4, 0x2b, 0xf7, 0xfb, 0x29, 0xd9, 0x6a, 0xfe, 0x09, 0x65, 0xab, 0xe7, 0x03, 0xe4, 0xdf, 0x81, 0x45, 0xbf, 0x1c, 0x61, 0x74, 0xd3, 0x6a, 0x92, 0xfc, 0x43, 0x0e,
0x85, 0xc3, 0xfb, 0xfa, 0x62, 0x92, 0xaf, 0xff, 0x53, 0x0e, 0x1a, 0x3e, 0xd7, 0xd6, 0x07, 0xaa, 0x5a, 0x81, 0xc9, 0x7e, 0xee, 0x18, 0x92, 0x92, 0xf9, 0xe6, 0x5f, 0x52, 0xe6, 0x5b, 0x98, 0x3d,
0x89, 0xe6, 0xa1, 0x34, 0x1c, 0xa8, 0x7e, 0x33, 0x94, 0x8f, 0xd0, 0x86, 0xc8, 0x73, 0xc2, 0x7c, 0x6e, 0x14, 0x93, 0xe2, 0xc6, 0x7f, 0xe4, 0xa0, 0xe1, 0x4b, 0x6d, 0x6b, 0xa0, 0x9a, 0x68, 0x11,
0x7a, 0x29, 0x49, 0x86, 0x29, 0x82, 0x50, 0x22, 0x4b, 0x90, 0x72, 0x90, 0x15, 0x14, 0xb4, 0xa8, 0x4a, 0xc3, 0x81, 0xea, 0x37, 0x56, 0xf9, 0x08, 0x6d, 0x8b, 0x9c, 0x29, 0x2c, 0xa7, 0xb7, 0x93,
0xe7, 0xb9, 0x15, 0x53, 0x16, 0x52, 0xcf, 0x5f, 0x01, 0xc4, 0x25, 0xdc, 0xd5, 0xcd, 0xae, 0x83, 0xf4, 0x21, 0xe5, 0x20, 0x94, 0xc8, 0x14, 0xa4, 0xb4, 0x64, 0xc5, 0x09, 0x6d, 0x10, 0xf0, 0x3c,
0x7b, 0x96, 0xa9, 0x31, 0xd9, 0x17, 0x95, 0x26, 0xff, 0xd2, 0x31, 0x37, 0x18, 0x1c, 0xbd, 0x06, 0x8d, 0x29, 0x9e, 0x6e, 0x60, 0x74, 0x03, 0x10, 0xd7, 0x96, 0xae, 0x6e, 0x76, 0x1d, 0xdc, 0xb3,
0x05, 0x77, 0x7f, 0xc8, 0xbc, 0x78, 0x23, 0xd1, 0x3b, 0xfa, 0x74, 0x6d, 0xee, 0x0f, 0xb1, 0x42, 0x4c, 0x8d, 0xe9, 0x51, 0x51, 0x69, 0xf2, 0x2f, 0x1d, 0x73, 0x9b, 0xc1, 0xd1, 0xfb, 0x50, 0x70,
0xd1, 0xd1, 0x02, 0x00, 0x59, 0xca, 0xb5, 0xd5, 0x47, 0x3c, 0x24, 0x16, 0x94, 0x00, 0x84, 0x68, 0x8f, 0x86, 0x2c, 0x22, 0x34, 0x12, 0x3d, 0xad, 0xcf, 0xd7, 0xce, 0xd1, 0x10, 0x2b, 0x14, 0x1d,
0xb3, 0xc7, 0xc3, 0x19, 0x16, 0x3a, 0xf8, 0x50, 0xfe, 0x43, 0x0e, 0x9a, 0xfe, 0x92, 0x0a, 0x76, 0x2d, 0x03, 0x90, 0xa9, 0x5c, 0x5b, 0x7d, 0xce, 0xc3, 0x6b, 0x41, 0x09, 0x40, 0x88, 0x65, 0x78,
0x46, 0x03, 0x37, 0x95, 0x7f, 0xe3, 0x8b, 0xc1, 0x49, 0x79, 0xc3, 0xbb, 0x50, 0xe5, 0xf2, 0x3c, 0x32, 0x9c, 0x63, 0x61, 0x88, 0x0f, 0xe5, 0x7f, 0xcb, 0x41, 0xd3, 0x9f, 0x52, 0xc1, 0xce, 0x68,
0x80, 0x3e, 0x00, 0x9b, 0xb2, 0x3a, 0x46, 0x41, 0x8b, 0x4f, 0x48, 0x41, 0x4b, 0x07, 0x54, 0x50, 0xe0, 0xa6, 0xca, 0x6f, 0x7c, 0x61, 0x39, 0x29, 0x07, 0xf9, 0x14, 0xaa, 0xfc, 0x3c, 0x8f, 0xa1,
0x79, 0x03, 0xe6, 0x3d, 0xbf, 0xe7, 0x23, 0xac, 0x61, 0x57, 0x1d, 0x93, 0x70, 0x9c, 0x81, 0x2a, 0x0f, 0xc0, 0x48, 0x36, 0xc6, 0x28, 0x68, 0xf1, 0x25, 0x29, 0x68, 0xe9, 0x98, 0x0a, 0x2a, 0x7f,
0x8b, 0x67, 0x2c, 0x90, 0xb3, 0x54, 0x1d, 0xb6, 0x44, 0x85, 0x4b, 0x92, 0x8b, 0x63, 0xd4, 0x71, 0x2f, 0xc1, 0x1b, 0x31, 0xe3, 0x1f, 0x2b, 0xc0, 0xf1, 0xe9, 0x2f, 0x77, 0x0a, 0xd1, 0x29, 0xb9,
0x44, 0x7b, 0xb9, 0x59, 0xfa, 0xfc, 0xb2, 0xa8, 0x04, 0x48, 0xd6, 0xcf, 0xd4, 0xbb, 0xa2, 0x84, 0x1f, 0xba, 0x03, 0x25, 0x9b, 0xce, 0xce, 0xdb, 0xfc, 0x97, 0xc6, 0x2a, 0x07, 0x63, 0x44, 0xe1,
0x60, 0x49, 0xbd, 0xbd, 0xfc, 0x94, 0xbd, 0xbd, 0x55, 0x78, 0x2e, 0x42, 0xea, 0x21, 0x62, 0x0c, 0x24, 0xf2, 0x5f, 0x48, 0xb0, 0x14, 0x67, 0x75, 0x86, 0xe0, 0xb2, 0x0a, 0x73, 0x6c, 0x6a, 0xcf,
0x39, 0xf9, 0xfc, 0x46, 0xf8, 0x82, 0x7d, 0xfa, 0x48, 0x7a, 0x5a, 0x74, 0x81, 0xbb, 0xba, 0x16, 0x86, 0xae, 0x8e, 0xb7, 0x21, 0x5f, 0x38, 0x8a, 0x47, 0x28, 0x6f, 0xc3, 0xa2, 0x17, 0x83, 0x7c,
0xd5, 0x55, 0x0d, 0xbd, 0x03, 0x15, 0x13, 0xef, 0x75, 0x83, 0x8e, 0x3c, 0x43, 0xb3, 0xaf, 0x6c, 0x01, 0x6f, 0x62, 0x57, 0x1d, 0x93, 0xfc, 0x9d, 0x87, 0x2a, 0xcb, 0x2d, 0x58, 0x52, 0xc5, 0xca,
0xe2, 0x3d, 0xfa, 0x4b, 0xbe, 0x0b, 0xc7, 0x63, 0xa4, 0x1e, 0xe6, 0xec, 0x7f, 0x94, 0xe0, 0xc4, 0x26, 0xd8, 0x15, 0xdd, 0x06, 0x92, 0xe8, 0x9d, 0xa2, 0x4e, 0x3c, 0xda, 0x57, 0xcf, 0x72, 0xe7,
0xb2, 0x6d, 0x0d, 0xdf, 0xd3, 0x6d, 0x77, 0xa4, 0x0e, 0xc2, 0x77, 0x66, 0x4f, 0xa7, 0x24, 0xbc, 0x22, 0x8b, 0xaa, 0x8c, 0x54, 0x60, 0x6c, 0x6b, 0x15, 0x25, 0x04, 0x4b, 0xea, 0xb3, 0xe6, 0xa7,
0x13, 0x08, 0xe9, 0x4c, 0x01, 0xae, 0x24, 0xa8, 0x7e, 0x9c, 0x28, 0x7e, 0xe8, 0x40, 0x02, 0xf0, 0xec, 0xb3, 0x6e, 0xc0, 0x1b, 0x11, 0x56, 0x67, 0x38, 0x12, 0xb2, 0xf3, 0xc5, 0xed, 0xf0, 0x63,
0x8f, 0x7c, 0x12, 0xf1, 0x1c, 0x6f, 0x42, 0xe0, 0xca, 0x92, 0xf1, 0x24, 0x76, 0x90, 0xf2, 0xd3, 0x87, 0xe9, 0xb3, 0x9a, 0x73, 0xa2, 0x23, 0xdf, 0xd5, 0xb5, 0xa8, 0xad, 0x6b, 0xe8, 0x13, 0xa8,
0x76, 0x90, 0x52, 0xbc, 0x48, 0xe1, 0x09, 0x79, 0x91, 0x03, 0x97, 0x34, 0x77, 0x20, 0xdc, 0xdd, 0x98, 0xf8, 0xb0, 0x1b, 0x0c, 0xaa, 0x19, 0x1a, 0xaf, 0x65, 0x13, 0x1f, 0xd2, 0xbf, 0xe4, 0x47,
0xa3, 0xee, 0x7b, 0xaa, 0xb6, 0xe0, 0x12, 0x80, 0xdf, 0xe9, 0xe2, 0xef, 0xa3, 0xb2, 0x2c, 0x13, 0xb0, 0x14, 0x63, 0x75, 0x96, 0xbd, 0xff, 0xbb, 0x04, 0xa7, 0xd7, 0x6c, 0x6b, 0xf8, 0x99, 0x6e,
0x98, 0x45, 0xa4, 0x25, 0x3c, 0x36, 0xed, 0x50, 0x87, 0x7a, 0x2f, 0xf7, 0xa0, 0x9d, 0xa4, 0xa5, 0xbb, 0x23, 0x75, 0x10, 0xbe, 0xbf, 0x7c, 0x35, 0xe5, 0xf9, 0xc3, 0x40, 0x7a, 0xc5, 0x14, 0xe0,
0x87, 0xd1, 0xfc, 0xcf, 0x73, 0x00, 0x1d, 0xf1, 0xa4, 0x6e, 0xba, 0xec, 0xf4, 0x3c, 0xd4, 0x7d, 0x46, 0x82, 0x09, 0xc4, 0x99, 0xe2, 0x9b, 0x0e, 0x24, 0x63, 0xff, 0x97, 0x4f, 0x62, 0x9e, 0xe3,
0x85, 0xf1, 0xed, 0x3d, 0xa8, 0x45, 0x1a, 0x31, 0x09, 0x91, 0x24, 0x13, 0x9c, 0x58, 0xe2, 0xac, 0x4d, 0x08, 0xfc, 0x59, 0xb2, 0xcf, 0xc4, 0x6e, 0x5e, 0x7e, 0xda, 0x6e, 0x5e, 0x8a, 0x17, 0x2e,
0xd1, 0x75, 0x02, 0x56, 0xc3, 0x94, 0x22, 0xea, 0x3f, 0x4f, 0x42, 0xc5, 0xb6, 0xf6, 0xba, 0xc4, 0xbc, 0x24, 0x2f, 0x7c, 0xec, 0xf2, 0xf2, 0x21, 0x84, 0x3b, 0xad, 0x34, 0xfc, 0x4d, 0xd5, 0xa2,
0xcc, 0x34, 0xef, 0xcd, 0xa0, 0x6d, 0xed, 0x11, 0xe3, 0xd3, 0xd0, 0x71, 0x98, 0x71, 0x55, 0x67, 0x5d, 0x05, 0xf0, 0xbb, 0x8e, 0xfc, 0xad, 0x5a, 0x96, 0x69, 0x02, 0x54, 0xe4, 0xb4, 0x44, 0xc4,
0x97, 0xac, 0x5f, 0x0a, 0x5c, 0xdb, 0x6a, 0xe8, 0x18, 0x14, 0xb7, 0xf5, 0x01, 0x66, 0xb7, 0x84, 0xa3, 0xb7, 0x05, 0xa1, 0x3e, 0xd8, 0x63, 0x68, 0x27, 0x69, 0xe9, 0x2c, 0x9a, 0xff, 0x43, 0x0e,
0x15, 0x85, 0x0d, 0xd0, 0xeb, 0xde, 0xe3, 0x96, 0x72, 0xe6, 0xab, 0x79, 0xf6, 0xbe, 0xe5, 0x4b, 0xa0, 0x23, 0x9e, 0x37, 0x4e, 0xe7, 0xcc, 0x2f, 0x41, 0xdd, 0x57, 0x18, 0xdf, 0xde, 0x83, 0x5a,
0x09, 0x66, 0x7d, 0xae, 0x51, 0x07, 0x44, 0x7c, 0x1a, 0xf5, 0x67, 0x37, 0x2d, 0x8d, 0xb9, 0x8a, 0xa4, 0x11, 0x93, 0x10, 0x05, 0x0b, 0xc1, 0x89, 0x15, 0x31, 0x1a, 0x9d, 0x27, 0x60, 0x35, 0x4c,
0x46, 0x8a, 0x4b, 0x67, 0x13, 0x99, 0xd7, 0xf2, 0xa7, 0x8c, 0xcb, 0xf1, 0xc9, 0xb9, 0xc8, 0xa1, 0x29, 0xa2, 0xfe, 0xf3, 0x0c, 0x54, 0x6c, 0xeb, 0xb0, 0x4b, 0xcc, 0x4c, 0xf3, 0xde, 0x6f, 0xda,
0x75, 0xcd, 0xbb, 0x2d, 0x2a, 0xd9, 0xd6, 0x5e, 0x47, 0x13, 0xdc, 0x60, 0x0f, 0x02, 0x59, 0x46, 0xd6, 0x21, 0x31, 0x3e, 0x0d, 0x2d, 0xc1, 0x9c, 0xab, 0x3a, 0x07, 0x64, 0xfe, 0x52, 0xe0, 0x0a,
0x4b, 0xb8, 0x71, 0x93, 0xbe, 0x09, 0x3c, 0x0f, 0x75, 0x6c, 0xdb, 0x96, 0xdd, 0x35, 0xb0, 0xe3, 0x5d, 0x43, 0xa7, 0xa0, 0xb8, 0xa7, 0x0f, 0x30, 0xbb, 0xb1, 0xad, 0x28, 0x6c, 0x80, 0x3e, 0xf0,
0xa8, 0x7d, 0xcc, 0x13, 0xb8, 0x1a, 0x05, 0xae, 0x31, 0x98, 0xfc, 0x45, 0x1e, 0x1a, 0xfe, 0x51, 0x1e, 0x1a, 0x95, 0x33, 0x3f, 0x93, 0x60, 0x6f, 0x8d, 0x7e, 0x92, 0x60, 0xde, 0x97, 0x1a, 0x75,
0xbc, 0x3b, 0x22, 0x5d, 0xf3, 0xee, 0x88, 0x74, 0x22, 0x3a, 0xb0, 0x99, 0x2b, 0x14, 0xc2, 0x5d, 0x40, 0xc4, 0xa7, 0x51, 0x7f, 0x76, 0xcf, 0xd2, 0x98, 0xab, 0x68, 0xa4, 0xb8, 0x74, 0x46, 0xc8,
0xca, 0xb5, 0x24, 0xa5, 0xc2, 0xa1, 0x1d, 0x8d, 0xc4, 0x55, 0x62, 0x64, 0xa6, 0xa5, 0x61, 0x5f, 0xbc, 0x96, 0x4f, 0x32, 0xae, 0xde, 0x22, 0xfb, 0x22, 0x9b, 0xd6, 0x35, 0xef, 0xe6, 0xae, 0x64,
0xb8, 0xe0, 0x81, 0xb8, 0x6c, 0x43, 0x3a, 0x52, 0xc8, 0xa0, 0x23, 0xc5, 0x0c, 0x3a, 0x52, 0x4a, 0x5b, 0x87, 0x1d, 0x4d, 0x48, 0x83, 0x3d, 0xce, 0x64, 0xd5, 0x05, 0x91, 0xc6, 0x3d, 0xfa, 0x3e,
0xd0, 0x91, 0x79, 0x28, 0x6d, 0x8d, 0x7a, 0xbb, 0xd8, 0xe5, 0xe9, 0x16, 0x1f, 0x85, 0x75, 0xa7, 0xf3, 0x12, 0xd4, 0xb1, 0x6d, 0x5b, 0x76, 0xd7, 0xc0, 0x8e, 0xa3, 0xf6, 0x31, 0x4f, 0x80, 0x6b,
0x1c, 0xd1, 0x1d, 0xa1, 0x22, 0x95, 0xa0, 0x8a, 0x9c, 0x84, 0x0a, 0xbb, 0xac, 0xe8, 0xba, 0x0e, 0x14, 0xb8, 0xc9, 0x60, 0xf2, 0x8f, 0x79, 0x68, 0xf8, 0x5b, 0xf1, 0xee, 0xeb, 0x74, 0xcd, 0xbb,
0xed, 0xda, 0xe6, 0x95, 0x32, 0x03, 0x6c, 0x3a, 0xe8, 0x0d, 0xaf, 0x16, 0xa9, 0x26, 0x19, 0x3b, 0xaf, 0xd3, 0xc9, 0xd1, 0x81, 0xcd, 0x5c, 0xa1, 0x38, 0xdc, 0xd5, 0x5c, 0x4b, 0x52, 0x2a, 0x1c,
0xf5, 0x3a, 0x11, 0x2d, 0xf1, 0x2a, 0x91, 0x0b, 0xd0, 0xa0, 0x0f, 0xa6, 0x1f, 0x8e, 0xb0, 0xbd, 0xda, 0xd1, 0x48, 0x5c, 0x25, 0x46, 0x66, 0x5a, 0x1a, 0xf6, 0x0f, 0x17, 0x3c, 0x10, 0x3f, 0xdb,
0xaf, 0x6e, 0x0d, 0x70, 0xab, 0x46, 0xc9, 0xa9, 0x13, 0xe8, 0x3d, 0x0f, 0x48, 0x18, 0x42, 0xd1, 0x90, 0x8e, 0x14, 0x32, 0xe8, 0x48, 0x31, 0x83, 0x8e, 0x94, 0x12, 0x74, 0x64, 0x11, 0x4a, 0xbb,
0x74, 0x53, 0xc3, 0x8f, 0xb1, 0xd6, 0xaa, 0x53, 0x24, 0xca, 0xea, 0x0e, 0x03, 0xc9, 0x1f, 0x00, 0xa3, 0xde, 0x01, 0x76, 0x79, 0xba, 0xca, 0x47, 0x61, 0xdd, 0x29, 0x47, 0x74, 0x47, 0xa8, 0x48,
0xf2, 0xf7, 0x38, 0x5c, 0x95, 0x19, 0x11, 0x62, 0x2e, 0x2a, 0x44, 0xf9, 0x37, 0x12, 0xcc, 0x05, 0x25, 0xa8, 0x22, 0x67, 0xa0, 0xc2, 0x2e, 0x8e, 0xba, 0xae, 0x43, 0x3b, 0xe8, 0x79, 0xa5, 0xcc,
0x37, 0x9b, 0x36, 0x3c, 0xbe, 0x03, 0x55, 0xd6, 0xdd, 0xee, 0x12, 0xf3, 0xe4, 0x75, 0xe6, 0xe9, 0x00, 0x3b, 0x0e, 0xfa, 0xd0, 0xcb, 0xc7, 0xaa, 0x49, 0xc6, 0x4e, 0xbd, 0x4e, 0x44, 0x4b, 0xbc,
0xb1, 0xdc, 0x53, 0xc0, 0x7f, 0xf8, 0x4b, 0x94, 0x60, 0xcf, 0xb2, 0x77, 0x75, 0xb3, 0xdf, 0x25, 0x6c, 0xec, 0x32, 0x34, 0xe8, 0xe3, 0xf5, 0x67, 0x23, 0x6c, 0x1f, 0xa9, 0xbb, 0x03, 0xdc, 0xaa,
0x94, 0x79, 0x46, 0x51, 0xe3, 0xc0, 0xbb, 0x04, 0x26, 0x7f, 0x2a, 0xc1, 0xc2, 0xfd, 0xa1, 0xa6, 0x51, 0x76, 0xea, 0x04, 0xfa, 0xd8, 0x03, 0x12, 0x81, 0x50, 0x34, 0xdd, 0xd4, 0xf0, 0x0b, 0xac,
0xba, 0x38, 0x90, 0x27, 0x1c, 0xf6, 0x2d, 0xd1, 0x6b, 0xde, 0x63, 0x9e, 0x5c, 0xb6, 0x0e, 0x2d, 0xb5, 0xea, 0x14, 0x89, 0x8a, 0xba, 0xc3, 0x40, 0xf2, 0x17, 0x80, 0xfc, 0x35, 0x66, 0x4b, 0xca,
0xc3, 0x96, 0xd7, 0xe0, 0x84, 0x82, 0x1d, 0x6c, 0x6a, 0xa1, 0x8f, 0xd3, 0x52, 0x21, 0x0f, 0xa1, 0x22, 0x87, 0x98, 0x8b, 0x1e, 0xa2, 0xfc, 0x4f, 0x12, 0x2c, 0x04, 0x17, 0x9b, 0x36, 0x3c, 0x7e,
0x9d, 0xb4, 0xdc, 0x61, 0x64, 0xcf, 0x12, 0xb6, 0xae, 0x4d, 0x96, 0x75, 0xb9, 0xff, 0x21, 0x79, 0x02, 0x55, 0x76, 0xd3, 0xd0, 0x25, 0xe6, 0xc9, 0x6b, 0xfe, 0x73, 0x63, 0xa5, 0xa7, 0x80, 0xff,
0x02, 0xdd, 0xc7, 0x95, 0xff, 0x29, 0xc1, 0xdc, 0x0d, 0xcd, 0xdb, 0xef, 0xa9, 0xe5, 0x85, 0xd1, 0x08, 0x9b, 0x28, 0xc1, 0xa1, 0x65, 0x1f, 0xe8, 0x66, 0xbf, 0x4b, 0x38, 0xf3, 0x8c, 0xa2, 0xc6,
0xbc, 0x29, 0x1f, 0xcf, 0x9b, 0x9e, 0x94, 0x23, 0xe1, 0x2e, 0xd5, 0x1c, 0x19, 0x5e, 0xa8, 0xb0, 0x81, 0x8f, 0x08, 0x4c, 0xfe, 0x56, 0x82, 0xe5, 0x27, 0x43, 0x4d, 0x75, 0x71, 0x20, 0x4f, 0x98,
0xe9, 0x5d, 0xb1, 0xbc, 0x2d, 0x2e, 0x10, 0x15, 0xbc, 0x8d, 0x6d, 0x6c, 0xf6, 0xf0, 0xaa, 0xd5, 0xf5, 0x5d, 0xd7, 0xfb, 0xde, 0xc3, 0xaa, 0x5c, 0xb6, 0x6e, 0x39, 0xc3, 0x96, 0x37, 0xe1, 0xb4,
0xdb, 0x0d, 0xbc, 0x08, 0x92, 0x82, 0x2f, 0x82, 0xa6, 0x7d, 0x61, 0x74, 0xf9, 0xa7, 0x12, 0xcc, 0x82, 0x1d, 0x6c, 0x6a, 0xa1, 0x8f, 0x53, 0x57, 0xfa, 0x43, 0x68, 0x27, 0x4d, 0x37, 0xcb, 0xd9,
0xc5, 0x3a, 0x15, 0xa8, 0x01, 0x70, 0xdf, 0xec, 0xf1, 0x16, 0x4e, 0xf3, 0x08, 0xaa, 0x41, 0xd9, 0xb3, 0x84, 0xad, 0x6b, 0x93, 0x69, 0x5d, 0xee, 0x7f, 0x48, 0x9e, 0x40, 0xd7, 0x71, 0xe5, 0xff,
0x6b, 0xe8, 0x34, 0x25, 0x54, 0x85, 0x99, 0x4d, 0x8b, 0x62, 0x37, 0x73, 0xa8, 0x09, 0x35, 0x36, 0x97, 0x60, 0xe1, 0xae, 0xe6, 0xad, 0xf7, 0xca, 0xf2, 0xc2, 0x68, 0xde, 0x94, 0x8f, 0xe7, 0x4d,
0x71, 0xd4, 0xeb, 0x61, 0xc7, 0x69, 0xe6, 0x05, 0xe4, 0xb6, 0xaa, 0x0f, 0x46, 0x36, 0x6e, 0x16, 0x2f, 0xcb, 0x91, 0x70, 0x97, 0x6a, 0x8e, 0x0c, 0x2f, 0x54, 0xd8, 0xf4, 0xde, 0x5e, 0xde, 0x13,
0x50, 0x1d, 0x2a, 0x9b, 0x16, 0x7f, 0x4f, 0xd5, 0x2c, 0x22, 0x04, 0x0d, 0xef, 0x71, 0x15, 0x9f, 0x97, 0xb9, 0x0a, 0xde, 0xc3, 0x36, 0x36, 0x7b, 0x78, 0xc3, 0xea, 0x1d, 0x04, 0x5e, 0x67, 0x49,
0x54, 0x0a, 0xc0, 0xbc, 0x69, 0x33, 0x97, 0xb7, 0x83, 0x35, 0x3d, 0x29, 0x74, 0xd1, 0x71, 0x38, 0xc1, 0xd7, 0x59, 0xd3, 0xbe, 0xf6, 0xba, 0xfe, 0x37, 0x12, 0x2c, 0xc4, 0xba, 0x46, 0xa8, 0x01,
0x7a, 0xdf, 0xd4, 0xf0, 0xb6, 0x6e, 0x62, 0xcd, 0xff, 0xd4, 0x3c, 0x82, 0x8e, 0xc2, 0x6c, 0xc7, 0xf0, 0xc4, 0xec, 0xf1, 0x76, 0x5a, 0xf3, 0x04, 0xaa, 0x41, 0xd9, 0x6b, 0xae, 0x35, 0x25, 0x54,
0x34, 0xb1, 0x1d, 0x00, 0x4a, 0x04, 0xb8, 0x86, 0xed, 0x3e, 0x0e, 0x00, 0x73, 0x68, 0x0e, 0xea, 0x85, 0xb9, 0x1d, 0x8b, 0x62, 0x37, 0x73, 0xa8, 0x09, 0x35, 0x46, 0x38, 0xea, 0xf5, 0xb0, 0xe3,
0x6b, 0xfa, 0xe3, 0x00, 0x28, 0xbf, 0xf8, 0xf7, 0x16, 0x54, 0x48, 0x79, 0x73, 0xd3, 0xb2, 0x6c, 0x34, 0xf3, 0x02, 0xf2, 0x40, 0xd5, 0x07, 0x23, 0x1b, 0x37, 0x0b, 0xa8, 0x0e, 0x95, 0x1d, 0x8b,
0x0d, 0x0d, 0x01, 0xd1, 0xd7, 0x88, 0xc6, 0xd0, 0x32, 0xc5, 0x1b, 0x5f, 0xf4, 0x4a, 0x4a, 0x0a, 0xbf, 0x6d, 0x6b, 0x16, 0x11, 0x82, 0x86, 0xf7, 0xd0, 0x8d, 0x13, 0x95, 0x02, 0x30, 0x8f, 0x6c,
0x15, 0x47, 0xe5, 0x6a, 0xd9, 0xbe, 0x98, 0x32, 0x23, 0x82, 0x2e, 0x1f, 0x41, 0x06, 0xdd, 0x71, 0xee, 0xfa, 0x5e, 0xb0, 0x27, 0xb2, 0x73, 0x34, 0xc4, 0x68, 0x09, 0x4e, 0x3e, 0x31, 0x35, 0xbc,
0x53, 0x37, 0xf0, 0xa6, 0xde, 0xdb, 0xf5, 0x9e, 0x29, 0x8c, 0xd9, 0x31, 0x82, 0xea, 0xed, 0x18, 0xa7, 0x9b, 0x58, 0xf3, 0x3f, 0x35, 0x4f, 0xa0, 0x93, 0x30, 0xdf, 0x31, 0x4d, 0x6c, 0x07, 0x80,
0x79, 0x3a, 0xcc, 0x07, 0xec, 0xc9, 0xa8, 0x67, 0x97, 0xf2, 0x11, 0xf4, 0x10, 0x8e, 0xad, 0xe0, 0x12, 0x01, 0x6e, 0x62, 0xbb, 0x8f, 0x03, 0xc0, 0x1c, 0x5a, 0x80, 0xfa, 0xa6, 0xfe, 0x22, 0x00,
0x80, 0x1f, 0xf2, 0x36, 0x5c, 0x4c, 0xdf, 0x30, 0x86, 0x7c, 0xc0, 0x2d, 0x57, 0xa1, 0x48, 0xbb, 0xca, 0xaf, 0xfc, 0x72, 0x09, 0x2a, 0xa4, 0xbc, 0xb9, 0x67, 0x59, 0xb6, 0x86, 0x86, 0x80, 0xe8,
0x82, 0x28, 0xc9, 0x55, 0x05, 0xff, 0x48, 0xd3, 0x3e, 0x9b, 0x8e, 0x20, 0x56, 0xfb, 0x00, 0x66, 0xcb, 0x50, 0x63, 0x68, 0x99, 0xe2, 0xbd, 0x35, 0x7a, 0x37, 0x25, 0x85, 0x8a, 0xa3, 0x72, 0xb5,
0x23, 0x0f, 0xf9, 0xd1, 0x8b, 0x09, 0xd3, 0x92, 0xff, 0x92, 0xd1, 0xbe, 0x9c, 0x05, 0x55, 0xec, 0x6c, 0x5f, 0x49, 0xa1, 0x88, 0xa0, 0xcb, 0x27, 0x90, 0x41, 0x57, 0xdc, 0xd1, 0x0d, 0xbc, 0xa3,
0xd5, 0x87, 0x46, 0xf8, 0x25, 0x23, 0xba, 0x94, 0x30, 0x3f, 0xf1, 0x0d, 0x76, 0xfb, 0xc5, 0x0c, 0xf7, 0x0e, 0xbc, 0x27, 0x23, 0x63, 0x56, 0x8c, 0xa0, 0x7a, 0x2b, 0x46, 0x8a, 0x69, 0x3e, 0x60,
0x98, 0x62, 0x23, 0x03, 0x9a, 0xd1, 0x87, 0xe5, 0xe8, 0xf2, 0xd8, 0x05, 0xc2, 0xea, 0xf6, 0x52, 0xcf, 0x77, 0x3d, 0xbb, 0x94, 0x4f, 0xa0, 0x67, 0x70, 0x6a, 0x1d, 0x07, 0xfc, 0x90, 0xb7, 0xe0,
0x26, 0x5c, 0xb1, 0xdd, 0x3e, 0x55, 0x82, 0xd8, 0x5b, 0x65, 0x74, 0x35, 0x79, 0x99, 0xb4, 0x47, 0x4a, 0xfa, 0x82, 0x31, 0xe4, 0x63, 0x2e, 0xb9, 0x01, 0x45, 0xda, 0xa1, 0x45, 0x49, 0xae, 0x2a,
0xd4, 0xed, 0x6b, 0x99, 0xf1, 0xc5, 0xd6, 0xdf, 0x61, 0xb7, 0x11, 0x49, 0xef, 0x7d, 0xd1, 0xab, 0xf8, 0xa3, 0xa6, 0xf6, 0x85, 0x74, 0x04, 0x31, 0xdb, 0x17, 0x30, 0x1f, 0xf9, 0x51, 0x05, 0xba,
0xc9, 0xcb, 0x8d, 0x79, 0xa8, 0xdc, 0x5e, 0x3c, 0xc8, 0x14, 0x41, 0xc4, 0x47, 0xf4, 0x1a, 0x21, 0x96, 0x40, 0x96, 0xfc, 0xf3, 0x98, 0xf6, 0xf5, 0x2c, 0xa8, 0x62, 0xad, 0x3e, 0x34, 0xc2, 0xaf,
0xe1, 0xc5, 0x6c, 0xd4, 0xee, 0xbc, 0xf5, 0xd2, 0x1f, 0x03, 0xb7, 0x5f, 0x3d, 0xc0, 0x0c, 0x41, 0x4a, 0x51, 0x52, 0x8f, 0x20, 0xf1, 0x3d, 0x7c, 0xfb, 0x5a, 0x06, 0x4c, 0xb1, 0x90, 0x01, 0xcd,
0x80, 0x15, 0x7d, 0xb9, 0xef, 0x99, 0xe1, 0xb5, 0x89, 0x5a, 0x33, 0x9d, 0x0d, 0xbe, 0x0f, 0xb3, 0xe8, 0x23, 0x7f, 0x74, 0x7d, 0xec, 0x04, 0x61, 0x75, 0x7b, 0x3b, 0x13, 0xae, 0x58, 0xee, 0x88,
0x91, 0x07, 0x21, 0x89, 0x56, 0x93, 0xfc, 0x68, 0xa4, 0x3d, 0x2e, 0x7c, 0x33, 0x93, 0x8c, 0xdc, 0x2a, 0x41, 0xec, 0xdd, 0x38, 0xba, 0x99, 0x3c, 0x4d, 0xda, 0x83, 0xf6, 0xf6, 0xad, 0xcc, 0xf8,
0xca, 0xa0, 0x14, 0xed, 0x4f, 0xb8, 0xb9, 0x69, 0x5f, 0xce, 0x82, 0x2a, 0x0e, 0xe2, 0x50, 0x77, 0x62, 0xe9, 0x3f, 0x62, 0x37, 0x43, 0x49, 0x6f, 0xaf, 0xd1, 0x7b, 0xc9, 0xd3, 0x8d, 0x79, 0x34,
0x19, 0xb9, 0xd9, 0x40, 0x57, 0x92, 0xd7, 0x48, 0xbe, 0x95, 0x69, 0xbf, 0x9c, 0x11, 0x5b, 0x6c, 0xde, 0x5e, 0x39, 0x0e, 0x89, 0x60, 0xe2, 0x2b, 0x7a, 0xa5, 0x93, 0xf0, 0x7a, 0x39, 0x6a, 0x77,
0xfa, 0x6d, 0x40, 0x1b, 0x3b, 0xa4, 0xa4, 0x31, 0xb7, 0xf5, 0xfe, 0xc8, 0x56, 0xd9, 0xab, 0x8f, 0xde, 0x7c, 0xe9, 0x0f, 0xb3, 0xdb, 0xef, 0x1d, 0x83, 0x42, 0x30, 0x60, 0x45, 0x7f, 0x45, 0xe1,
0x34, 0x1f, 0x1d, 0x47, 0x4d, 0xd1, 0x95, 0xb1, 0x33, 0xc4, 0xe6, 0x5d, 0x80, 0x15, 0xec, 0xae, 0x99, 0xe1, 0xad, 0x89, 0x5a, 0x33, 0x9d, 0x0d, 0x7e, 0x0e, 0xf3, 0x91, 0xc7, 0x39, 0x89, 0x56,
0x61, 0xd7, 0x26, 0x0a, 0x7a, 0x31, 0x51, 0xde, 0x3e, 0x82, 0xb7, 0xd5, 0x0b, 0x13, 0xf1, 0xc4, 0x93, 0xfc, 0x80, 0xa7, 0x3d, 0x2e, 0x7c, 0x33, 0x93, 0x8c, 0xdc, 0x90, 0xa1, 0x14, 0xed, 0x4f,
0x06, 0xdf, 0x00, 0xe4, 0xc5, 0xf7, 0xc0, 0x5b, 0xa8, 0xf3, 0x63, 0x3b, 0xcf, 0xac, 0x4d, 0x3c, 0xb8, 0x45, 0x6b, 0x5f, 0xcf, 0x82, 0x2a, 0x36, 0xe2, 0x50, 0x77, 0x19, 0xb9, 0x65, 0x42, 0x37,
0x49, 0x31, 0x1e, 0x42, 0x73, 0x4d, 0x35, 0x47, 0xea, 0x20, 0xb0, 0xee, 0x95, 0x44, 0xc2, 0xa2, 0x92, 0xe7, 0x48, 0xbe, 0x21, 0x6b, 0xbf, 0x93, 0x11, 0x5b, 0x2c, 0xfa, 0x87, 0x80, 0xb6, 0xf7,
0x68, 0x29, 0xa2, 0x4a, 0xc5, 0x16, 0x87, 0xd9, 0x13, 0x01, 0x5c, 0x15, 0xf6, 0x8f, 0xa3, 0x8e, 0x49, 0x49, 0x63, 0xee, 0xe9, 0xfd, 0x91, 0xad, 0xb2, 0x17, 0x38, 0x69, 0x3e, 0x3a, 0x8e, 0x9a,
0xcd, 0xe7, 0x46, 0x04, 0x31, 0xc5, 0xb1, 0x8d, 0xc1, 0x17, 0x1b, 0x7f, 0x2c, 0xd1, 0x3f, 0x9c, 0xa2, 0x2b, 0x63, 0x29, 0xc4, 0xe2, 0x5d, 0x80, 0x75, 0xec, 0x6e, 0x62, 0xd7, 0x26, 0x0a, 0x7a,
0x44, 0x10, 0x1e, 0xe8, 0xee, 0xce, 0xfa, 0x40, 0x35, 0x9d, 0x2c, 0x24, 0x50, 0xc4, 0x03, 0x90, 0x25, 0xf1, 0xbc, 0x7d, 0x04, 0x6f, 0xa9, 0xb7, 0x26, 0xe2, 0x05, 0x42, 0x42, 0x73, 0x53, 0x35,
0xc0, 0xf1, 0x05, 0x09, 0x1a, 0xd4, 0x43, 0xcd, 0x58, 0x94, 0xf4, 0xa0, 0x29, 0xa9, 0xb3, 0xdc, 0x49, 0x35, 0xef, 0xbf, 0x4a, 0xbb, 0x91, 0x48, 0x1e, 0x45, 0x4b, 0x11, 0x68, 0x2a, 0xb6, 0x58,
0xbe, 0x34, 0x19, 0x51, 0xec, 0xb2, 0x03, 0x75, 0xcf, 0x58, 0x18, 0x73, 0x5f, 0x4c, 0xa3, 0xd4, 0xf2, 0x50, 0x84, 0xd9, 0x40, 0x73, 0x35, 0xea, 0x7e, 0x7c, 0x9e, 0x93, 0xef, 0xa7, 0xa2, 0xee,
0xc7, 0x49, 0xb1, 0xf5, 0x64, 0xd4, 0xa0, 0xad, 0xc7, 0x7b, 0x4d, 0x28, 0x5b, 0x8f, 0x72, 0x9c, 0x67, 0x0c, 0xbe, 0x58, 0xf8, 0x6b, 0x89, 0xfe, 0x44, 0x27, 0x82, 0xf0, 0x54, 0x77, 0xf7, 0xb7,
0xad, 0xa7, 0x37, 0xb0, 0x98, 0x33, 0x8b, 0xf4, 0x75, 0x93, 0x3d, 0x65, 0x62, 0x9b, 0x3a, 0xd1, 0x06, 0xaa, 0xe9, 0x64, 0x61, 0x81, 0x22, 0x1e, 0x83, 0x05, 0x8e, 0x2f, 0x58, 0xd0, 0xa0, 0x1e,
0x99, 0xa5, 0xb4, 0x89, 0xe5, 0x23, 0xe8, 0x01, 0x94, 0xf8, 0x3f, 0x4d, 0x9f, 0x1f, 0x5f, 0x79, 0x6a, 0x99, 0xa2, 0xa4, 0x27, 0x60, 0x49, 0xfd, 0xdf, 0xf6, 0xd5, 0xc9, 0x88, 0x62, 0x95, 0x7d,
0xf2, 0xd5, 0x2f, 0x4c, 0xc0, 0x12, 0x0b, 0xef, 0xc2, 0xf1, 0x94, 0xba, 0x33, 0x31, 0xc8, 0x8e, 0xa8, 0x7b, 0x2a, 0xcd, 0x84, 0x7b, 0x2d, 0x8d, 0x53, 0x1f, 0x27, 0xc5, 0x22, 0x93, 0x51, 0x83,
0xaf, 0x51, 0x27, 0x59, 0xb9, 0x0a, 0x28, 0xfe, 0x77, 0x8e, 0x44, 0x31, 0xa5, 0xfe, 0xeb, 0x23, 0x16, 0x19, 0xef, 0x08, 0xa1, 0x6c, 0x9d, 0xc4, 0x71, 0x16, 0x99, 0xde, 0x66, 0x62, 0x2e, 0x27,
0xc3, 0x16, 0xf1, 0x7f, 0x64, 0x24, 0x6e, 0x91, 0xfa, 0xc7, 0x8d, 0x49, 0x5b, 0xdc, 0x03, 0xf0, 0xd2, 0x7d, 0x4d, 0xf6, 0x67, 0x89, 0xcd, 0xe4, 0x44, 0x97, 0x93, 0xd2, 0xcc, 0x95, 0x4f, 0xa0,
0xab, 0xcb, 0x44, 0x79, 0xc4, 0x8a, 0xcf, 0x09, 0x4b, 0x2e, 0xfe, 0x6b, 0x06, 0xca, 0xde, 0xf3, 0xa7, 0x50, 0xe2, 0xbf, 0xcd, 0x7d, 0x73, 0x7c, 0x7d, 0xc8, 0x67, 0xbf, 0x3c, 0x01, 0x4b, 0x4c,
0xa1, 0x67, 0x50, 0x59, 0x3c, 0x83, 0x54, 0xff, 0x7d, 0x98, 0x8d, 0xfc, 0x0f, 0x21, 0xd1, 0x78, 0x7c, 0x00, 0x4b, 0x29, 0xd5, 0x61, 0x62, 0x28, 0x1c, 0x5f, 0x49, 0x4e, 0x72, 0xd2, 0x2a, 0xa0,
0x92, 0xff, 0xab, 0x30, 0x49, 0x42, 0x0f, 0xf8, 0x7f, 0xdb, 0x45, 0xd4, 0x7f, 0x21, 0xad, 0x5c, 0xf8, 0x0f, 0x60, 0x12, 0x8f, 0x29, 0xf5, 0x77, 0x32, 0x19, 0x96, 0x88, 0xff, 0x86, 0x25, 0x71,
0x88, 0x06, 0xfc, 0x09, 0x0b, 0xff, 0x6f, 0x87, 0xf7, 0xbb, 0x00, 0x81, 0xf0, 0x3b, 0xfe, 0x42, 0x89, 0xd4, 0x9f, 0xba, 0x4c, 0x5a, 0xe2, 0x31, 0x80, 0x5f, 0x03, 0x26, 0x9e, 0x47, 0xac, 0x44,
0x99, 0x44, 0x94, 0x49, 0xdc, 0x5a, 0x3b, 0xa0, 0xd3, 0x9a, 0xb0, 0x9c, 0x43, 0x4c, 0x3b, 0xda, 0x9c, 0x30, 0xe5, 0xca, 0xff, 0x96, 0xa1, 0xec, 0x3d, 0xb8, 0x7a, 0x0d, 0xf9, 0xff, 0x6b, 0x48,
0x48, 0x4a, 0x31, 0xed, 0x94, 0xf6, 0x55, 0xa2, 0x93, 0x4f, 0xef, 0x4e, 0x3d, 0x15, 0x63, 0x5f, 0xc8, 0x3f, 0x87, 0xf9, 0xc8, 0x2f, 0x37, 0x12, 0x8d, 0x27, 0xf9, 0xd7, 0x1d, 0x93, 0x4e, 0xe8,
0xba, 0xfe, 0xcd, 0x57, 0xfb, 0xba, 0xbb, 0x33, 0xda, 0x22, 0x5f, 0xae, 0x31, 0xd4, 0x97, 0x75, 0x29, 0xff, 0x6f, 0x00, 0x22, 0x36, 0xbf, 0x95, 0x96, 0xd4, 0x47, 0xc3, 0xf2, 0x84, 0x89, 0x7f,
0x8b, 0xff, 0xba, 0xe6, 0x29, 0xc3, 0x35, 0x3a, 0xfb, 0x1a, 0xd9, 0x63, 0xb8, 0xb5, 0x55, 0xa2, 0xb3, 0x83, 0xf0, 0x23, 0x80, 0x40, 0xf8, 0x1d, 0x7f, 0x6d, 0x4e, 0x22, 0xca, 0x24, 0x69, 0x19,
0xa3, 0xeb, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0xf9, 0x60, 0xe6, 0x31, 0xbb, 0x42, 0x00, 0x00, 0x89, 0x11, 0xf6, 0x5a, 0x96, 0x2b, 0xce, 0x74, 0x1f, 0x99, 0x1e, 0x57, 0x37, 0x8f, 0xe9, 0x23,
0x27, 0x70, 0xef, 0x10, 0x4f, 0x12, 0xed, 0x2e, 0xa5, 0x78, 0x92, 0x94, 0x9e, 0x56, 0x62, 0x4c,
0x49, 0x6f, 0x59, 0xbd, 0x12, 0xdf, 0xb2, 0x7a, 0xfb, 0xf7, 0xdf, 0xeb, 0xeb, 0xee, 0xfe, 0x68,
0x97, 0x7c, 0xb9, 0xc5, 0x50, 0xdf, 0xd1, 0x2d, 0xfe, 0xd7, 0x2d, 0x4f, 0xf7, 0x6e, 0x51, 0xea,
0x5b, 0x64, 0x8d, 0xe1, 0xee, 0x6e, 0x89, 0x8e, 0x6e, 0xff, 0x2a, 0x00, 0x00, 0xff, 0xff, 0xed,
0xbc, 0xc3, 0xee, 0x5c, 0x44, 0x00, 0x00,
} }
// Reference imports to suppress errors if they are not otherwise used. // Reference imports to suppress errors if they are not otherwise used.
@ -4465,7 +4614,6 @@ type DataCoordClient interface {
ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // 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) 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 *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error)
GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error)
GetCompactionStateWithPlans(ctx context.Context, in *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) GetCompactionStateWithPlans(ctx context.Context, in *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error)
@ -4633,15 +4781,6 @@ func (c *dataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetric
return out, nil 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 *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) { func (c *dataCoordClient) ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) {
out := new(milvuspb.ManualCompactionResponse) out := new(milvuspb.ManualCompactionResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/ManualCompaction", in, out, opts...) err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/ManualCompaction", in, out, opts...)
@ -4769,7 +4908,6 @@ type DataCoordServer interface {
ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) ShowConfigurations(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
CompleteCompaction(context.Context, *CompactionResult) (*commonpb.Status, error)
ManualCompaction(context.Context, *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) ManualCompaction(context.Context, *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error)
GetCompactionState(context.Context, *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) GetCompactionState(context.Context, *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error)
GetCompactionStateWithPlans(context.Context, *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) GetCompactionStateWithPlans(context.Context, *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error)
@ -4837,9 +4975,6 @@ func (*UnimplementedDataCoordServer) ShowConfigurations(ctx context.Context, req
func (*UnimplementedDataCoordServer) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { func (*UnimplementedDataCoordServer) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") 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 *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) { func (*UnimplementedDataCoordServer) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ManualCompaction not implemented") return nil, status.Errorf(codes.Unimplemented, "method ManualCompaction not implemented")
} }
@ -5169,24 +5304,6 @@ func _DataCoord_GetMetrics_Handler(srv interface{}, ctx context.Context, dec fun
return interceptor(ctx, in, info, handler) 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) { func _DataCoord_ManualCompaction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.ManualCompactionRequest) in := new(milvuspb.ManualCompactionRequest)
if err := dec(in); err != nil { if err := dec(in); err != nil {
@ -5471,10 +5588,6 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{
MethodName: "GetMetrics", MethodName: "GetMetrics",
Handler: _DataCoord_GetMetrics_Handler, Handler: _DataCoord_GetMetrics_Handler,
}, },
{
MethodName: "CompleteCompaction",
Handler: _DataCoord_CompleteCompaction_Handler,
},
{ {
MethodName: "ManualCompaction", MethodName: "ManualCompaction",
Handler: _DataCoord_ManualCompaction_Handler, Handler: _DataCoord_ManualCompaction_Handler,
@ -5540,6 +5653,7 @@ type DataNodeClient interface {
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // 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) 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) Compaction(ctx context.Context, in *CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error)
GetCompactionState(ctx context.Context, in *CompactionStateRequest, opts ...grpc.CallOption) (*CompactionStateResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
ResendSegmentStats(ctx context.Context, in *ResendSegmentStatsRequest, opts ...grpc.CallOption) (*ResendSegmentStatsResponse, error) ResendSegmentStats(ctx context.Context, in *ResendSegmentStatsRequest, opts ...grpc.CallOption) (*ResendSegmentStatsResponse, error)
@ -5617,6 +5731,15 @@ func (c *dataNodeClient) Compaction(ctx context.Context, in *CompactionPlan, opt
return out, nil return out, nil
} }
func (c *dataNodeClient) GetCompactionState(ctx context.Context, in *CompactionStateRequest, opts ...grpc.CallOption) (*CompactionStateResponse, error) {
out := new(CompactionStateResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/GetCompactionState", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataNodeClient) Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { func (c *dataNodeClient) Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
out := new(commonpb.Status) out := new(commonpb.Status)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/Import", in, out, opts...) err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/Import", in, out, opts...)
@ -5654,6 +5777,7 @@ type DataNodeServer interface {
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
Compaction(context.Context, *CompactionPlan) (*commonpb.Status, error) Compaction(context.Context, *CompactionPlan) (*commonpb.Status, error)
GetCompactionState(context.Context, *CompactionStateRequest) (*CompactionStateResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(context.Context, *ImportTaskRequest) (*commonpb.Status, error) Import(context.Context, *ImportTaskRequest) (*commonpb.Status, error)
ResendSegmentStats(context.Context, *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error) ResendSegmentStats(context.Context, *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error)
@ -5685,6 +5809,9 @@ func (*UnimplementedDataNodeServer) GetMetrics(ctx context.Context, req *milvusp
func (*UnimplementedDataNodeServer) Compaction(ctx context.Context, req *CompactionPlan) (*commonpb.Status, error) { func (*UnimplementedDataNodeServer) Compaction(ctx context.Context, req *CompactionPlan) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method Compaction not implemented") return nil, status.Errorf(codes.Unimplemented, "method Compaction not implemented")
} }
func (*UnimplementedDataNodeServer) GetCompactionState(ctx context.Context, req *CompactionStateRequest) (*CompactionStateResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetCompactionState not implemented")
}
func (*UnimplementedDataNodeServer) Import(ctx context.Context, req *ImportTaskRequest) (*commonpb.Status, error) { func (*UnimplementedDataNodeServer) Import(ctx context.Context, req *ImportTaskRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method Import not implemented") return nil, status.Errorf(codes.Unimplemented, "method Import not implemented")
} }
@ -5825,6 +5952,24 @@ func _DataNode_Compaction_Handler(srv interface{}, ctx context.Context, dec func
return interceptor(ctx, in, info, handler) return interceptor(ctx, in, info, handler)
} }
func _DataNode_GetCompactionState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(CompactionStateRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataNodeServer).GetCompactionState(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataNode/GetCompactionState",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataNodeServer).GetCompactionState(ctx, req.(*CompactionStateRequest))
}
return interceptor(ctx, in, info, handler)
}
func _DataNode_Import_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { func _DataNode_Import_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ImportTaskRequest) in := new(ImportTaskRequest)
if err := dec(in); err != nil { if err := dec(in); err != nil {
@ -5911,6 +6056,10 @@ var _DataNode_serviceDesc = grpc.ServiceDesc{
MethodName: "Compaction", MethodName: "Compaction",
Handler: _DataNode_Compaction_Handler, Handler: _DataNode_Compaction_Handler,
}, },
{
MethodName: "GetCompactionState",
Handler: _DataNode_GetCompactionState_Handler,
},
{ {
MethodName: "Import", MethodName: "Import",
Handler: _DataNode_Import_Handler, Handler: _DataNode_Import_Handler,

View File

@ -908,6 +908,7 @@ message GetCompactionStateResponse {
int64 executingPlanNo = 3; int64 executingPlanNo = 3;
int64 timeoutPlanNo = 4; int64 timeoutPlanNo = 4;
int64 completedPlanNo = 5; int64 completedPlanNo = 5;
int64 failedPlanNo = 6;
} }
message GetCompactionPlansRequest { message GetCompactionPlansRequest {

View File

@ -4773,6 +4773,7 @@ type GetCompactionStateResponse struct {
ExecutingPlanNo int64 `protobuf:"varint,3,opt,name=executingPlanNo,proto3" json:"executingPlanNo,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"` TimeoutPlanNo int64 `protobuf:"varint,4,opt,name=timeoutPlanNo,proto3" json:"timeoutPlanNo,omitempty"`
CompletedPlanNo int64 `protobuf:"varint,5,opt,name=completedPlanNo,proto3" json:"completedPlanNo,omitempty"` CompletedPlanNo int64 `protobuf:"varint,5,opt,name=completedPlanNo,proto3" json:"completedPlanNo,omitempty"`
FailedPlanNo int64 `protobuf:"varint,6,opt,name=failedPlanNo,proto3" json:"failedPlanNo,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -4838,6 +4839,13 @@ func (m *GetCompactionStateResponse) GetCompletedPlanNo() int64 {
return 0 return 0
} }
func (m *GetCompactionStateResponse) GetFailedPlanNo() int64 {
if m != nil {
return m.FailedPlanNo
}
return 0
}
type GetCompactionPlansRequest struct { type GetCompactionPlansRequest struct {
CompactionID int64 `protobuf:"varint,1,opt,name=compactionID,proto3" json:"compactionID,omitempty"` CompactionID int64 `protobuf:"varint,1,opt,name=compactionID,proto3" json:"compactionID,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -7049,34 +7057,34 @@ func init() {
func init() { proto.RegisterFile("milvus.proto", fileDescriptor_02345ba45cc0e303) } func init() { proto.RegisterFile("milvus.proto", fileDescriptor_02345ba45cc0e303) }
var fileDescriptor_02345ba45cc0e303 = []byte{ var fileDescriptor_02345ba45cc0e303 = []byte{
// 5188 bytes of a gzipped FileDescriptorProto // 5200 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3d, 0x4b, 0x6c, 0x1c, 0x47, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3d, 0x4b, 0x6c, 0x1c, 0x47,
0x76, 0xec, 0x19, 0xce, 0xef, 0xcd, 0x87, 0xc3, 0xe2, 0x6f, 0x3c, 0x92, 0x2c, 0xaa, 0x6d, 0xd9, 0x76, 0xec, 0x19, 0xce, 0xef, 0xcd, 0x87, 0xc3, 0xe2, 0x6f, 0x3c, 0x92, 0x2c, 0xaa, 0x6d, 0xd9,
0xb4, 0xb4, 0xa6, 0x6c, 0xca, 0x96, 0xd7, 0xb2, 0x57, 0xb6, 0x24, 0x5a, 0x12, 0x61, 0x7d, 0xe8, 0xb4, 0xb4, 0xa6, 0x6c, 0xca, 0x96, 0xd7, 0xb2, 0x57, 0xb6, 0x24, 0x5a, 0x12, 0x61, 0x7d, 0xe8,
0xa6, 0xec, 0x60, 0xe3, 0x18, 0x8d, 0xe6, 0x74, 0x71, 0xd8, 0x56, 0x4f, 0xf7, 0xa8, 0xbb, 0x87, 0xa6, 0xec, 0x60, 0xe3, 0x18, 0x8d, 0xe6, 0x74, 0x71, 0xd8, 0x56, 0x4f, 0xf7, 0xa8, 0xbb, 0x87,
0x14, 0x9d, 0xcb, 0x02, 0x9b, 0xdd, 0x6c, 0x90, 0xcd, 0x2e, 0x36, 0xbf, 0x45, 0x0e, 0xf9, 0x20, 0x14, 0x9d, 0xcb, 0x02, 0x9b, 0x5d, 0x6c, 0x90, 0xcd, 0x2e, 0x36, 0xbf, 0x45, 0x0e, 0xf9, 0x20,
0xd8, 0x1c, 0x02, 0x6c, 0x82, 0x38, 0x09, 0x10, 0x60, 0x73, 0xc8, 0x39, 0x46, 0xbe, 0x87, 0x20, 0xd8, 0x1c, 0x82, 0x6c, 0x82, 0x38, 0x09, 0x10, 0x60, 0x73, 0xc8, 0x39, 0x46, 0xbe, 0x87, 0x20,
0x59, 0x24, 0xc7, 0x45, 0x80, 0x1c, 0x02, 0xe4, 0x90, 0x63, 0x82, 0x04, 0xf5, 0xe9, 0x9e, 0xea, 0x59, 0x24, 0xc7, 0x45, 0x80, 0x1c, 0x02, 0xe4, 0x90, 0x63, 0x82, 0x04, 0xf5, 0xe9, 0x9e, 0xea,
0x9e, 0xea, 0xe1, 0x90, 0xb3, 0x32, 0x29, 0xf3, 0x34, 0xf5, 0xea, 0x55, 0xd5, 0xab, 0x57, 0xef, 0x9e, 0xea, 0xe1, 0x90, 0xb3, 0x32, 0x29, 0xf3, 0x34, 0xf5, 0xea, 0x55, 0xd5, 0xab, 0x57, 0xef,
0xbd, 0x7a, 0x55, 0xef, 0x55, 0x13, 0x2a, 0x1d, 0xcb, 0xde, 0xe9, 0xf9, 0xcb, 0x5d, 0xcf, 0x0d, 0xbd, 0x7a, 0x55, 0xef, 0x55, 0x13, 0x2a, 0x1d, 0xcb, 0xde, 0xe9, 0xf9, 0xcb, 0x5d, 0xcf, 0x0d,
0x5c, 0x34, 0x23, 0x96, 0x96, 0x59, 0xa1, 0x59, 0x69, 0xb9, 0x9d, 0x8e, 0xeb, 0x30, 0x60, 0xb3, 0x5c, 0x34, 0x23, 0x96, 0x96, 0x59, 0xa1, 0x59, 0x69, 0xb9, 0x9d, 0x8e, 0xeb, 0x30, 0x60, 0xb3,
0xe2, 0xb7, 0xb6, 0x71, 0xc7, 0xe0, 0xa5, 0xc5, 0xb6, 0xeb, 0xb6, 0x6d, 0x7c, 0x81, 0x96, 0x36, 0xe2, 0xb7, 0xb6, 0x71, 0xc7, 0xe0, 0xa5, 0xc5, 0xb6, 0xeb, 0xb6, 0x6d, 0x7c, 0x81, 0x96, 0x36,
0x7b, 0x5b, 0x17, 0x4c, 0xec, 0xb7, 0x3c, 0xab, 0x1b, 0xb8, 0x1e, 0xc3, 0x50, 0x7f, 0x47, 0x01, 0x7b, 0x5b, 0x17, 0x4c, 0xec, 0xb7, 0x3c, 0xab, 0x1b, 0xb8, 0x1e, 0xc3, 0x50, 0x7f, 0x5b, 0x01,
0x74, 0xdd, 0xc3, 0x46, 0x80, 0xaf, 0xda, 0x96, 0xe1, 0x6b, 0xf8, 0x61, 0x0f, 0xfb, 0x01, 0x7a, 0x74, 0xdd, 0xc3, 0x46, 0x80, 0xaf, 0xda, 0x96, 0xe1, 0x6b, 0xf8, 0x61, 0x0f, 0xfb, 0x01, 0x7a,
0x09, 0x26, 0x37, 0x0d, 0x1f, 0x37, 0x94, 0x45, 0x65, 0xa9, 0xbc, 0x72, 0x72, 0x39, 0x36, 0x30, 0x09, 0x26, 0x37, 0x0d, 0x1f, 0x37, 0x94, 0x45, 0x65, 0xa9, 0xbc, 0x72, 0x72, 0x39, 0x36, 0x30,
0x1f, 0xf0, 0x8e, 0xdf, 0xbe, 0x66, 0xf8, 0x58, 0xa3, 0x98, 0x68, 0x01, 0x0a, 0xe6, 0xa6, 0xee, 0x1f, 0xf0, 0x8e, 0xdf, 0xbe, 0x66, 0xf8, 0x58, 0xa3, 0x98, 0x68, 0x01, 0x0a, 0xe6, 0xa6, 0xee,
0x18, 0x1d, 0xdc, 0xc8, 0x2c, 0x2a, 0x4b, 0x25, 0x2d, 0x6f, 0x6e, 0xde, 0x35, 0x3a, 0x18, 0x3d, 0x18, 0x1d, 0xdc, 0xc8, 0x2c, 0x2a, 0x4b, 0x25, 0x2d, 0x6f, 0x6e, 0xde, 0x35, 0x3a, 0x18, 0x3d,
0x0f, 0x53, 0x2d, 0xd7, 0xb6, 0x71, 0x2b, 0xb0, 0x5c, 0x87, 0x21, 0x64, 0x29, 0x42, 0xad, 0x0f, 0x0f, 0x53, 0x2d, 0xd7, 0xb6, 0x71, 0x2b, 0xb0, 0x5c, 0x87, 0x21, 0x64, 0x29, 0x42, 0xad, 0x0f,
0xa6, 0x88, 0xb3, 0x90, 0x33, 0x08, 0x0d, 0x8d, 0x49, 0x5a, 0xcd, 0x0a, 0xaa, 0x0f, 0xf5, 0x55, 0xa6, 0x88, 0xb3, 0x90, 0x33, 0x08, 0x0d, 0x8d, 0x49, 0x5a, 0xcd, 0x0a, 0xaa, 0x0f, 0xf5, 0x55,
0xcf, 0xed, 0x3e, 0x2e, 0xea, 0xa2, 0x41, 0xb3, 0xe2, 0xa0, 0xbf, 0xad, 0xc0, 0xf4, 0x55, 0x3b, 0xcf, 0xed, 0x3e, 0x2e, 0xea, 0xa2, 0x41, 0xb3, 0xe2, 0xa0, 0xbf, 0xa5, 0xc0, 0xf4, 0x55, 0x3b,
0xc0, 0xde, 0x31, 0x65, 0xca, 0x0f, 0x33, 0xb0, 0xc0, 0x56, 0xed, 0x7a, 0x84, 0x7e, 0x94, 0x54, 0xc0, 0xde, 0x31, 0x65, 0xca, 0x0f, 0x33, 0xb0, 0xc0, 0x56, 0xed, 0x7a, 0x84, 0x7e, 0x94, 0x54,
0xce, 0x43, 0x9e, 0xc9, 0x1d, 0x25, 0xb3, 0xa2, 0xf1, 0x12, 0x3a, 0x05, 0xe0, 0x6f, 0x1b, 0x9e, 0xce, 0x43, 0x9e, 0xc9, 0x1d, 0x25, 0xb3, 0xa2, 0xf1, 0x12, 0x3a, 0x05, 0xe0, 0x6f, 0x1b, 0x9e,
0xe9, 0xeb, 0x4e, 0xaf, 0xd3, 0xc8, 0x2d, 0x2a, 0x4b, 0x39, 0xad, 0xc4, 0x20, 0x77, 0x7b, 0x1d, 0xe9, 0xeb, 0x4e, 0xaf, 0xd3, 0xc8, 0x2d, 0x2a, 0x4b, 0x39, 0xad, 0xc4, 0x20, 0x77, 0x7b, 0x1d,
0xa4, 0xc1, 0x74, 0xcb, 0x75, 0x7c, 0xcb, 0x0f, 0xb0, 0xd3, 0xda, 0xd3, 0x6d, 0xbc, 0x83, 0xed, 0xa4, 0xc1, 0x74, 0xcb, 0x75, 0x7c, 0xcb, 0x0f, 0xb0, 0xd3, 0xda, 0xd3, 0x6d, 0xbc, 0x83, 0xed,
0x46, 0x7e, 0x51, 0x59, 0xaa, 0xad, 0x9c, 0x95, 0xd2, 0x7d, 0xbd, 0x8f, 0x7d, 0x9b, 0x20, 0x6b, 0x46, 0x7e, 0x51, 0x59, 0xaa, 0xad, 0x9c, 0x95, 0xd2, 0x7d, 0xbd, 0x8f, 0x7d, 0x9b, 0x20, 0x6b,
0xf5, 0x56, 0x02, 0x72, 0x19, 0x7d, 0x76, 0x65, 0xaa, 0xa8, 0xd4, 0x95, 0xc6, 0xff, 0x85, 0x7f, 0xf5, 0x56, 0x02, 0x72, 0x19, 0x7d, 0x76, 0x65, 0xaa, 0xa8, 0xd4, 0x95, 0xc6, 0xff, 0x85, 0x7f,
0x8a, 0xfa, 0xbb, 0x0a, 0xcc, 0x11, 0x21, 0x3a, 0x16, 0xcc, 0x0a, 0x29, 0xcc, 0x88, 0x14, 0xfe, 0x8a, 0xfa, 0x3b, 0x0a, 0xcc, 0x11, 0x21, 0x3a, 0x16, 0xcc, 0x0a, 0x29, 0xcc, 0x88, 0x14, 0xfe,
0xa1, 0x02, 0xb3, 0xb7, 0x0c, 0xff, 0x78, 0xac, 0xe6, 0x29, 0x80, 0xc0, 0xea, 0x60, 0xdd, 0x0f, 0x81, 0x02, 0xb3, 0xb7, 0x0c, 0xff, 0x78, 0xac, 0xe6, 0x29, 0x80, 0xc0, 0xea, 0x60, 0xdd, 0x0f,
0x8c, 0x4e, 0x97, 0xae, 0xe8, 0xa4, 0x56, 0x22, 0x90, 0x0d, 0x02, 0x50, 0xbf, 0x0a, 0x95, 0x6b, 0x8c, 0x4e, 0x97, 0xae, 0xe8, 0xa4, 0x56, 0x22, 0x90, 0x0d, 0x02, 0x50, 0xbf, 0x0a, 0x95, 0x6b,
0xae, 0x6b, 0x6b, 0xd8, 0xef, 0xba, 0x8e, 0x8f, 0xd1, 0x45, 0xc8, 0xfb, 0x81, 0x11, 0xf4, 0x7c, 0xae, 0x6b, 0x6b, 0xd8, 0xef, 0xba, 0x8e, 0x8f, 0xd1, 0x45, 0xc8, 0xfb, 0x81, 0x11, 0xf4, 0x7c,
0x4e, 0xe4, 0x09, 0x29, 0x91, 0x1b, 0x14, 0x45, 0xe3, 0xa8, 0x44, 0xae, 0x77, 0x0c, 0xbb, 0xc7, 0x4e, 0xe4, 0x09, 0x29, 0x91, 0x1b, 0x14, 0x45, 0xe3, 0xa8, 0x44, 0xae, 0x77, 0x0c, 0xbb, 0xc7,
@ -7122,7 +7130,7 @@ var fileDescriptor_02345ba45cc0e303 = []byte{
0xf2, 0xbf, 0x72, 0x47, 0x24, 0xa2, 0xf9, 0xc9, 0xd8, 0x31, 0x07, 0x3d, 0x96, 0x9c, 0xc4, 0x63, 0xf2, 0xbf, 0x72, 0x47, 0x24, 0xa2, 0xf9, 0xc9, 0xd8, 0x31, 0x07, 0x3d, 0x96, 0x9c, 0xc4, 0x63,
0x51, 0xff, 0xa2, 0xef, 0xa8, 0x3c, 0x59, 0x13, 0x54, 0x7f, 0xa4, 0xc0, 0xa9, 0x9b, 0x38, 0x88, 0x51, 0xff, 0xa2, 0xef, 0xa8, 0x3c, 0x59, 0x13, 0x54, 0x7f, 0xa4, 0xc0, 0xa9, 0x9b, 0x38, 0x88,
0xa8, 0x3e, 0x1e, 0x1e, 0xcd, 0x88, 0x42, 0xf5, 0x1d, 0xe6, 0x0d, 0x48, 0x89, 0x3f, 0x92, 0xcd, 0xa8, 0x3e, 0x1e, 0x1e, 0xcd, 0x88, 0x42, 0xf5, 0x1d, 0xe6, 0x0d, 0x48, 0x89, 0x3f, 0x92, 0xcd,
0xf6, 0x97, 0x33, 0x30, 0x47, 0x76, 0x9d, 0xe3, 0x21, 0x04, 0xa3, 0x1c, 0x6b, 0x25, 0x82, 0x92, 0xf6, 0x97, 0x32, 0x30, 0x47, 0x76, 0x9d, 0xe3, 0x21, 0x04, 0xa3, 0x1c, 0x6b, 0x25, 0x82, 0x92,
0x93, 0x6a, 0x42, 0xb8, 0x85, 0xe7, 0x47, 0xde, 0xc2, 0xd5, 0x3f, 0xcd, 0x30, 0xd7, 0x43, 0xe4, 0x93, 0x6a, 0x42, 0xb8, 0x85, 0xe7, 0x47, 0xde, 0xc2, 0xd5, 0x3f, 0xcd, 0x30, 0xd7, 0x43, 0xe4,
0xc6, 0x38, 0xcb, 0x22, 0xa1, 0x35, 0x23, 0xa5, 0x55, 0x85, 0x4a, 0x04, 0x59, 0x5b, 0x0d, 0xb7, 0xc6, 0x38, 0xcb, 0x22, 0xa1, 0x35, 0x23, 0xa5, 0x55, 0x85, 0x4a, 0x04, 0x59, 0x5b, 0x0d, 0xb7,
0xdf, 0x18, 0xec, 0xb8, 0xee, 0xbe, 0xea, 0xb7, 0x15, 0x98, 0x0f, 0x2f, 0x0d, 0x36, 0x70, 0xbb, 0xdf, 0x18, 0xec, 0xb8, 0xee, 0xbe, 0xea, 0xb7, 0x15, 0x98, 0x0f, 0x2f, 0x0d, 0x36, 0x70, 0xbb,
@ -7130,20 +7138,20 @@ var fileDescriptor_02345ba45cc0e303 = []byte{
0x44, 0xf7, 0x01, 0x7d, 0x80, 0xfa, 0x57, 0x0a, 0x2c, 0x0c, 0x90, 0x33, 0xce, 0x22, 0x36, 0xa0, 0x44, 0xf7, 0x01, 0x7d, 0x80, 0xfa, 0x57, 0x0a, 0x2c, 0x0c, 0x90, 0x33, 0xce, 0x22, 0x36, 0xa0,
0x60, 0x39, 0x26, 0x7e, 0x14, 0x51, 0x13, 0x16, 0x49, 0xcd, 0x66, 0xcf, 0xb2, 0xcd, 0x88, 0x8c, 0x60, 0x39, 0x26, 0x7e, 0x14, 0x51, 0x13, 0x16, 0x49, 0xcd, 0x66, 0xcf, 0xb2, 0xcd, 0x88, 0x8c,
0xb0, 0x88, 0xce, 0x40, 0x05, 0x3b, 0xc4, 0xc7, 0xd0, 0x29, 0x2e, 0x15, 0xe4, 0xa2, 0x56, 0x66, 0xb0, 0x88, 0xce, 0x40, 0x05, 0x3b, 0xc4, 0xc7, 0xd0, 0x29, 0x2e, 0x15, 0xe4, 0xa2, 0x56, 0x66,
0xb0, 0x35, 0x02, 0x22, 0x8d, 0xb7, 0x2c, 0x4c, 0x1b, 0xe7, 0x58, 0x63, 0x5e, 0x54, 0x7f, 0x45, 0xb0, 0x35, 0x02, 0x22, 0x8d, 0xb7, 0x2c, 0x4c, 0x1b, 0xe7, 0x58, 0x63, 0x5e, 0x54, 0x7f, 0x59,
0x81, 0x19, 0x22, 0x85, 0x9c, 0x7a, 0xff, 0xf1, 0x72, 0x73, 0x11, 0xca, 0x82, 0x98, 0xf1, 0x89, 0x81, 0x19, 0x22, 0x85, 0x9c, 0x7a, 0xff, 0xf1, 0x72, 0x73, 0x11, 0xca, 0x82, 0x98, 0xf1, 0x89,
0x88, 0x20, 0xf5, 0x01, 0xcc, 0xc6, 0xc9, 0x19, 0x87, 0x9b, 0x4f, 0x03, 0x44, 0x6b, 0xc5, 0xb4, 0x88, 0x20, 0xf5, 0x01, 0xcc, 0xc6, 0xc9, 0x19, 0x87, 0x9b, 0x4f, 0x03, 0x44, 0x6b, 0xc5, 0xb4,
0x21, 0xab, 0x09, 0x10, 0xf5, 0x37, 0x32, 0x61, 0x58, 0x81, 0xb2, 0xe9, 0x88, 0x6f, 0x33, 0xe9, 0x21, 0xab, 0x09, 0x10, 0xf5, 0xd7, 0x33, 0x61, 0x58, 0x81, 0xb2, 0xe9, 0x88, 0x6f, 0x33, 0xe9,
0x92, 0x88, 0xf6, 0xbc, 0x44, 0x21, 0xb4, 0x7a, 0x15, 0x2a, 0xf8, 0x51, 0xe0, 0x19, 0x7a, 0xd7, 0x92, 0x88, 0xf6, 0xbc, 0x44, 0x21, 0xb4, 0x7a, 0x15, 0x2a, 0xf8, 0x51, 0xe0, 0x19, 0x7a, 0xd7,
0xf0, 0x8c, 0x0e, 0x53, 0xab, 0x91, 0x4c, 0x6f, 0x99, 0x36, 0x5b, 0xa7, 0xad, 0xc8, 0x20, 0x54, 0xf0, 0x8c, 0x0e, 0x53, 0xab, 0x91, 0x4c, 0x6f, 0x99, 0x36, 0x5b, 0xa7, 0xad, 0xc8, 0x20, 0x54,
0x44, 0xd8, 0x20, 0x79, 0x36, 0x08, 0x85, 0xf4, 0xcf, 0x69, 0xe5, 0x46, 0x56, 0xfd, 0x47, 0xe2, 0x44, 0xd8, 0x20, 0x79, 0x36, 0x08, 0x85, 0xf4, 0xcf, 0x69, 0xe5, 0x46, 0x56, 0xfd, 0x47, 0xe2,
0xf5, 0x71, 0xb1, 0x3e, 0xee, 0x9c, 0x89, 0xcf, 0x29, 0x27, 0x9d, 0x53, 0xa5, 0x91, 0x55, 0xff, 0xf5, 0x71, 0xb1, 0x3e, 0xee, 0x9c, 0x89, 0xcf, 0x29, 0x27, 0x9d, 0x53, 0xa5, 0x91, 0x55, 0x7f,
0x40, 0x81, 0x3a, 0x9d, 0xcb, 0x2a, 0x0f, 0x2e, 0x59, 0xae, 0x93, 0x68, 0xac, 0x24, 0x1a, 0x0f, 0x5f, 0x81, 0x3a, 0x9d, 0xcb, 0x2a, 0x0f, 0x2e, 0x59, 0xae, 0x93, 0x68, 0xac, 0x24, 0x1a, 0x0f,
0xd1, 0xc6, 0xd7, 0x21, 0xcf, 0x57, 0x22, 0x3b, 0xea, 0x4a, 0xf0, 0x06, 0xfb, 0xcc, 0x47, 0xfd, 0xd1, 0xc6, 0xd7, 0x21, 0xcf, 0x57, 0x22, 0x3b, 0xea, 0x4a, 0xf0, 0x06, 0xfb, 0xcc, 0x47, 0xfd,
0x7d, 0x05, 0xe6, 0x12, 0xbc, 0x1f, 0x47, 0x05, 0xee, 0x03, 0x62, 0x33, 0x34, 0xfb, 0xd3, 0x0e, 0x3d, 0x05, 0xe6, 0x12, 0xbc, 0x1f, 0x47, 0x05, 0xee, 0x03, 0x62, 0x33, 0x34, 0xfb, 0xd3, 0x0e,
0x77, 0xee, 0xb3, 0xd2, 0x6d, 0x2a, 0xc9, 0x24, 0x6d, 0xda, 0x4a, 0x40, 0x7c, 0xf5, 0xc7, 0x0a, 0x77, 0xee, 0xb3, 0xd2, 0x6d, 0x2a, 0xc9, 0x24, 0x6d, 0xda, 0x4a, 0x40, 0x7c, 0xf5, 0xc7, 0x0a,
0x9c, 0xbc, 0x89, 0x03, 0x8a, 0x7a, 0x8d, 0x98, 0xa1, 0x75, 0xcf, 0x6d, 0x7b, 0xd8, 0xf7, 0xbf, 0x9c, 0xbc, 0x89, 0x03, 0x8a, 0x7a, 0x8d, 0x98, 0xa1, 0x75, 0xcf, 0x6d, 0x7b, 0xd8, 0xf7, 0xbf,
0x00, 0x82, 0xf2, 0x9b, 0xcc, 0xe7, 0x93, 0xcd, 0x6d, 0x9c, 0x85, 0x38, 0x03, 0x15, 0x3a, 0x18, 0x00, 0x82, 0xf2, 0x1b, 0xcc, 0xe7, 0x93, 0xcd, 0x6d, 0x9c, 0x85, 0x38, 0x03, 0x15, 0x3a, 0x18,
0x36, 0x75, 0xcf, 0xdd, 0xf5, 0xb9, 0x40, 0x95, 0x39, 0x4c, 0x73, 0x77, 0xa9, 0x64, 0x04, 0x6e, 0x36, 0x75, 0xcf, 0xdd, 0xf5, 0xb9, 0x40, 0x95, 0x39, 0x4c, 0x73, 0x77, 0xa9, 0x64, 0x04, 0x6e,
0x60, 0xd8, 0x0c, 0x81, 0x6f, 0x36, 0x14, 0x42, 0xaa, 0xa9, 0x56, 0x86, 0x84, 0x91, 0xce, 0xf1, 0x60, 0xd8, 0x0c, 0x81, 0x6f, 0x36, 0x14, 0x42, 0xaa, 0xa9, 0x56, 0x86, 0x84, 0x91, 0xce, 0xf1,
0x17, 0x80, 0xd9, 0x3f, 0x60, 0x37, 0x67, 0xe2, 0x9c, 0xc6, 0x61, 0xf2, 0xab, 0xcc, 0x35, 0x65, 0x17, 0x80, 0xd9, 0x3f, 0x60, 0x37, 0x67, 0xe2, 0x9c, 0xc6, 0x61, 0xf2, 0xab, 0xcc, 0x35, 0x65,
@ -7204,8 +7212,8 @@ var fileDescriptor_02345ba45cc0e303 = []byte{
0x5e, 0x81, 0xb9, 0x75, 0xec, 0xb1, 0x1c, 0xa3, 0x80, 0xdf, 0x4c, 0xae, 0x39, 0x5b, 0x6e, 0xfc, 0x5e, 0x81, 0xb9, 0x75, 0xec, 0xb1, 0x1c, 0xa3, 0x80, 0xdf, 0x4c, 0xae, 0x39, 0x5b, 0x6e, 0xfc,
0x72, 0x58, 0x49, 0x5c, 0x0e, 0xff, 0x74, 0x2e, 0x44, 0x63, 0xe7, 0x13, 0x16, 0xa2, 0x08, 0xcf, 0x72, 0x58, 0x49, 0x5c, 0x0e, 0xff, 0x74, 0x2e, 0x44, 0x63, 0xe7, 0x13, 0x16, 0xa2, 0x08, 0xcf,
0x27, 0x61, 0x20, 0x86, 0x1d, 0xf4, 0x6a, 0x29, 0xcb, 0xc4, 0xe9, 0x15, 0xcf, 0xbb, 0xea, 0xaf, 0x27, 0x61, 0x20, 0x86, 0x1d, 0xf4, 0x6a, 0x29, 0xcb, 0xc4, 0xe9, 0x15, 0xcf, 0xbb, 0xea, 0xaf,
0xb1, 0x3c, 0x0c, 0xe9, 0xa4, 0x0e, 0x2f, 0xb0, 0xf3, 0xc0, 0x4d, 0x7e, 0x62, 0x03, 0x78, 0x0e, 0xb2, 0x3c, 0x0c, 0xe9, 0xa4, 0x0e, 0x2f, 0xb0, 0xf3, 0xc0, 0x4d, 0x7e, 0x62, 0x03, 0x78, 0x0e,
0x12, 0xb6, 0x23, 0xc5, 0x10, 0xfd, 0x96, 0x02, 0x8b, 0xe9, 0x54, 0x8d, 0xb3, 0x57, 0xbf, 0x0d, 0x12, 0xb6, 0x23, 0xc5, 0x10, 0xfd, 0xa6, 0x02, 0x8b, 0xe9, 0x54, 0x8d, 0xb3, 0x57, 0xbf, 0x0d,
0x39, 0xcb, 0xd9, 0x72, 0xc3, 0x7b, 0xaf, 0x73, 0x52, 0x5d, 0x90, 0x8f, 0xcb, 0x1a, 0xaa, 0xff, 0x39, 0xcb, 0xd9, 0x72, 0xc3, 0x7b, 0xaf, 0x73, 0x52, 0x5d, 0x90, 0x8f, 0xcb, 0x1a, 0xaa, 0xff,
0x90, 0x81, 0xfa, 0x7b, 0x2c, 0xae, 0xff, 0xb9, 0x2f, 0x7f, 0x07, 0x77, 0x74, 0xdf, 0xfa, 0x04, 0x90, 0x81, 0xfa, 0x7b, 0x2c, 0xae, 0xff, 0xb9, 0x2f, 0x7f, 0x07, 0x77, 0x74, 0xdf, 0xfa, 0x04,
0x87, 0xcb, 0xdf, 0xc1, 0x9d, 0x0d, 0xeb, 0x13, 0x1c, 0x93, 0x8c, 0x5c, 0x5c, 0x32, 0x86, 0x5f, 0x87, 0xcb, 0xdf, 0xc1, 0x9d, 0x0d, 0xeb, 0x13, 0x1c, 0x93, 0x8c, 0x5c, 0x5c, 0x32, 0x86, 0x5f,
@ -7230,151 +7238,151 @@ var fileDescriptor_02345ba45cc0e303 = []byte{
0xd3, 0x2c, 0x95, 0x8e, 0xb9, 0xeb, 0x74, 0x72, 0x93, 0x9a, 0x00, 0x61, 0xe3, 0x14, 0x1a, 0x8a, 0xd3, 0x2c, 0x95, 0x8e, 0xb9, 0xeb, 0x74, 0x72, 0x93, 0x9a, 0x00, 0x61, 0xe3, 0x14, 0x1a, 0x8a,
0xea, 0x43, 0x63, 0x70, 0x9c, 0x71, 0x96, 0x98, 0x52, 0x17, 0x76, 0x25, 0xda, 0xf3, 0x3e, 0x4c, 0xea, 0x43, 0x63, 0x70, 0x9c, 0x71, 0x96, 0x98, 0x52, 0x17, 0x76, 0x25, 0xda, 0xf3, 0x3e, 0x4c,
0x7d, 0x0b, 0x9e, 0xa2, 0xf9, 0x8d, 0x21, 0x28, 0x76, 0x4d, 0x9f, 0xec, 0x40, 0x91, 0x74, 0xf0, 0x7d, 0x0b, 0x9e, 0xa2, 0xf9, 0x8d, 0x21, 0x28, 0x76, 0x4d, 0x9f, 0xec, 0x40, 0x91, 0x74, 0xf0,
0x8b, 0x19, 0x6a, 0x14, 0x07, 0x7a, 0x18, 0x87, 0xf0, 0xcb, 0xf1, 0x4b, 0xf1, 0x67, 0x53, 0xd2, 0x87, 0x19, 0x6a, 0x14, 0x07, 0x7a, 0x18, 0x87, 0xf0, 0xcb, 0xf1, 0x4b, 0xf1, 0x67, 0x53, 0xd2,
0xaa, 0xe3, 0x23, 0x72, 0xf3, 0xbd, 0x04, 0x53, 0xf8, 0x11, 0x6e, 0xf5, 0x02, 0xcb, 0x69, 0xaf, 0xaa, 0xe3, 0x23, 0x72, 0xf3, 0xbd, 0x04, 0x53, 0xf8, 0x11, 0x6e, 0xf5, 0x02, 0xcb, 0x69, 0xaf,
0xdb, 0x86, 0x73, 0xd7, 0xe5, 0x9b, 0x54, 0x12, 0x8c, 0x9e, 0x85, 0x2a, 0x59, 0x06, 0xb7, 0x17, 0xdb, 0x86, 0x73, 0xd7, 0xe5, 0x9b, 0x54, 0x12, 0x8c, 0x9e, 0x85, 0x2a, 0x59, 0x06, 0xb7, 0x17,
0x70, 0x3c, 0xb6, 0x5b, 0xc5, 0x81, 0xa4, 0x3f, 0x32, 0x5f, 0x1b, 0x07, 0xd8, 0xe4, 0x78, 0x6c, 0x70, 0x3c, 0xb6, 0x5b, 0xc5, 0x81, 0xa4, 0x3f, 0x32, 0x5f, 0x1b, 0x07, 0xd8, 0xe4, 0x78, 0x6c,
0xeb, 0x4a, 0x82, 0x07, 0x58, 0x49, 0xc0, 0xfe, 0x41, 0x58, 0xf9, 0xcf, 0x4a, 0x82, 0x95, 0xbc, 0xeb, 0x4a, 0x82, 0x09, 0xb7, 0xb6, 0x0c, 0xcb, 0x8e, 0xd0, 0xd8, 0x0d, 0x65, 0x0c, 0x36, 0xc0,
0x87, 0xa3, 0x62, 0xe5, 0x2d, 0x80, 0x0e, 0xf6, 0xda, 0x78, 0x8d, 0x6e, 0x07, 0xec, 0x5a, 0x60, 0x6e, 0x02, 0xf6, 0x0f, 0xc2, 0xee, 0x7f, 0x56, 0x12, 0xec, 0xe6, 0x3d, 0x1c, 0x15, 0xbb, 0x6f,
0x49, 0xba, 0x1d, 0xf4, 0x3b, 0xb8, 0x13, 0x36, 0xd0, 0x84, 0xb6, 0xea, 0x4d, 0x98, 0x91, 0xa0, 0x01, 0x74, 0xb0, 0xd7, 0xc6, 0x6b, 0x74, 0xcb, 0x60, 0x57, 0x07, 0x4b, 0xd2, 0x2d, 0xa3, 0xdf,
0x10, 0x4b, 0xe7, 0xbb, 0x3d, 0xaf, 0x85, 0xc3, 0x2b, 0xa6, 0xb0, 0x48, 0x76, 0xc6, 0xc0, 0xf0, 0xc1, 0x9d, 0xb0, 0x81, 0x26, 0xb4, 0x55, 0x6f, 0xc2, 0x8c, 0x04, 0x85, 0x58, 0x43, 0xdf, 0xed,
0xda, 0x38, 0xe0, 0x42, 0xcb, 0x4b, 0xea, 0x25, 0x1a, 0x50, 0xa2, 0xb7, 0x10, 0x31, 0x49, 0x8d, 0x79, 0x2d, 0x1c, 0x5e, 0x43, 0x85, 0x45, 0xb2, 0x7b, 0x06, 0x86, 0xd7, 0xc6, 0x01, 0x17, 0x6c,
0xc7, 0xcd, 0x95, 0x81, 0xb8, 0xf9, 0x16, 0x0d, 0xda, 0x88, 0xed, 0xc6, 0xcc, 0x79, 0xd8, 0x22, 0x5e, 0x52, 0x2f, 0xd1, 0xa0, 0x13, 0xbd, 0xa9, 0x88, 0x49, 0x73, 0x3c, 0xb6, 0xae, 0x0c, 0xc4,
0x5d, 0x61, 0x93, 0x3f, 0xb3, 0x09, 0x8b, 0xea, 0xff, 0x2a, 0x50, 0x5d, 0xeb, 0x74, 0xdd, 0x7e, 0xd6, 0xb7, 0x68, 0x60, 0x47, 0x6c, 0x37, 0x66, 0x5e, 0xc4, 0x16, 0xe9, 0x0a, 0x9b, 0xfc, 0x29,
0x98, 0x62, 0xe4, 0xe3, 0xe9, 0xe0, 0xed, 0x6e, 0x46, 0x76, 0xbb, 0xfb, 0x0c, 0x54, 0xe3, 0x0f, 0x4e, 0x58, 0x54, 0xff, 0x57, 0x81, 0xea, 0x5a, 0xa7, 0xeb, 0xf6, 0x43, 0x19, 0x23, 0x1f, 0x61,
0x32, 0xd8, 0xed, 0x51, 0xa5, 0x25, 0x3e, 0xc4, 0x38, 0x01, 0x25, 0xcf, 0xdd, 0xd5, 0x89, 0x71, 0x07, 0x6f, 0x80, 0x33, 0xb2, 0x1b, 0xe0, 0x67, 0xa0, 0x1a, 0x7f, 0xb4, 0xc1, 0x6e, 0x98, 0x2a,
0x35, 0x79, 0x76, 0x45, 0xd1, 0x73, 0x77, 0x89, 0xc9, 0x35, 0xd1, 0x2c, 0xe4, 0xb6, 0x2c, 0x3b, 0x2d, 0xf1, 0xb1, 0xc6, 0x09, 0x28, 0x79, 0xee, 0xae, 0x4e, 0x0c, 0xb0, 0xc9, 0x33, 0x30, 0x8a,
0x4a, 0x0c, 0x62, 0x05, 0xf4, 0x06, 0x39, 0xbc, 0xb1, 0x58, 0x6b, 0x7e, 0xd4, 0x33, 0x54, 0xd8, 0x9e, 0xbb, 0x4b, 0xcc, 0xb2, 0x89, 0x66, 0x21, 0xb7, 0x65, 0xd9, 0x51, 0xf2, 0x10, 0x2b, 0xa0,
0x82, 0xd9, 0x30, 0xd4, 0x50, 0xd4, 0x0f, 0xa1, 0x16, 0x4e, 0x7f, 0xcc, 0x87, 0x46, 0x81, 0xe1, 0x37, 0xc8, 0x01, 0x8f, 0xc5, 0x63, 0xf3, 0xa3, 0x9e, 0xb3, 0xc2, 0x16, 0xcc, 0xce, 0xa1, 0x86,
0x3f, 0x08, 0x33, 0x20, 0x58, 0x41, 0x3d, 0xcf, 0x22, 0x6f, 0xb4, 0xff, 0xd8, 0xea, 0x23, 0x98, 0xa2, 0x7e, 0x08, 0xb5, 0x70, 0xfa, 0x63, 0x3e, 0x46, 0x0a, 0x0c, 0xff, 0x41, 0x98, 0x25, 0xc1,
0x24, 0x18, 0x5c, 0xa9, 0xe8, 0x6f, 0xf5, 0xef, 0x32, 0x30, 0x9f, 0xc4, 0x1e, 0x87, 0xa4, 0x4b, 0x0a, 0xea, 0x79, 0x16, 0x9d, 0xa3, 0xfd, 0xc7, 0x56, 0x1f, 0xc1, 0x24, 0xc1, 0xe0, 0x4a, 0x45,
0x71, 0x45, 0x92, 0xbf, 0x1b, 0x11, 0x47, 0xe3, 0x4a, 0xc4, 0x97, 0xa2, 0xe5, 0xf6, 0x9c, 0x80, 0x7f, 0xab, 0x7f, 0x97, 0x81, 0xf9, 0x24, 0xf6, 0x38, 0x24, 0x5d, 0x8a, 0x2b, 0x92, 0xfc, 0x6d,
0x5b, 0x22, 0xb2, 0x14, 0xd7, 0x49, 0x19, 0x2d, 0x40, 0xc1, 0x32, 0x75, 0x9b, 0x1c, 0xf8, 0xd8, 0x89, 0x38, 0x1a, 0x57, 0x22, 0xbe, 0x14, 0x2d, 0xb7, 0xe7, 0x04, 0xdc, 0x5a, 0x91, 0xa5, 0xb8,
0x76, 0x95, 0xb7, 0xcc, 0xdb, 0xe4, 0x30, 0xf8, 0x5a, 0xe8, 0x84, 0x8d, 0x9c, 0x36, 0xc1, 0xf0, 0x4e, 0xca, 0x68, 0x01, 0x0a, 0x96, 0xa9, 0xdb, 0xe4, 0x50, 0xc8, 0xb6, 0xb4, 0xbc, 0x65, 0xde,
0x51, 0x0d, 0x32, 0x96, 0xc9, 0x83, 0x23, 0x19, 0xcb, 0x24, 0x52, 0x45, 0x6f, 0x0a, 0x68, 0x8a, 0x26, 0x07, 0xc6, 0xd7, 0x42, 0x47, 0x6d, 0xe4, 0xd4, 0x0a, 0x86, 0x8f, 0x6a, 0x90, 0xb1, 0x4c,
0x2f, 0xcf, 0xf9, 0x25, 0xe2, 0x50, 0x25, 0xd0, 0xf7, 0x42, 0x20, 0xf1, 0xd3, 0x28, 0x1a, 0x0f, 0x6e, 0x9e, 0x32, 0x96, 0x49, 0xa4, 0x8a, 0xde, 0x26, 0xd0, 0x34, 0x60, 0x9e, 0x17, 0x4c, 0xc4,
0xee, 0x52, 0x5f, 0xba, 0xa8, 0x95, 0x09, 0x6c, 0x8d, 0x81, 0xd4, 0x06, 0xcc, 0x13, 0xd2, 0xd8, 0xa1, 0x4a, 0xa0, 0xef, 0x85, 0x40, 0xe2, 0xcb, 0x51, 0x34, 0x1e, 0x00, 0xa6, 0xfe, 0x76, 0x51,
0x14, 0xef, 0x93, 0x05, 0x09, 0xbd, 0xaf, 0xef, 0x29, 0xb0, 0x30, 0x50, 0x35, 0x0e, 0xaf, 0xaf, 0x2b, 0x13, 0xd8, 0x1a, 0x03, 0xa9, 0x0d, 0x98, 0x27, 0xa4, 0xb1, 0x29, 0xde, 0x27, 0x0b, 0x12,
0x8a, 0xcb, 0x5f, 0x5e, 0x39, 0x2f, 0xb5, 0x39, 0xf2, 0xc5, 0x0d, 0x65, 0xe5, 0xd7, 0x99, 0xab, 0x7a, 0x68, 0xdf, 0x53, 0x60, 0x61, 0xa0, 0x6a, 0x1c, 0x5e, 0x5f, 0x15, 0x97, 0xbf, 0xbc, 0x72,
0xa4, 0xb1, 0xb4, 0xce, 0xc7, 0x9c, 0x24, 0xb4, 0x04, 0xf5, 0x5d, 0x2b, 0xd8, 0xd6, 0xe9, 0x4b, 0x5e, 0x6a, 0x73, 0xe4, 0x8b, 0x1b, 0xca, 0xca, 0xaf, 0x31, 0x77, 0x4a, 0x63, 0xa9, 0x9f, 0x8f,
0x24, 0xea, 0xa7, 0xb0, 0x60, 0x78, 0x51, 0xab, 0x11, 0xf8, 0x06, 0x01, 0x13, 0x5f, 0xc5, 0x57, 0x39, 0x91, 0x68, 0x09, 0xea, 0xbb, 0x56, 0xb0, 0xad, 0xd3, 0xd7, 0x4a, 0xd4, 0x97, 0x61, 0x01,
0xbf, 0xa5, 0xc0, 0x4c, 0x8c, 0xac, 0x71, 0xd8, 0xf4, 0x26, 0x71, 0xe1, 0x58, 0x47, 0x9c, 0x53, 0xf3, 0xa2, 0x56, 0x23, 0xf0, 0x0d, 0x02, 0x26, 0xfe, 0x8c, 0xaf, 0x7e, 0x4b, 0x81, 0x99, 0x18,
0x8b, 0x52, 0x4e, 0xf1, 0xd1, 0xa8, 0x55, 0x8e, 0x5a, 0xa8, 0x3f, 0x56, 0xa0, 0x2c, 0xd4, 0x90, 0x59, 0xe3, 0xb0, 0xe9, 0x4d, 0xe2, 0xe6, 0xb1, 0x8e, 0x38, 0xa7, 0x16, 0xa5, 0x9c, 0xe2, 0xa3,
0xb3, 0x21, 0xaf, 0xeb, 0x9f, 0x0d, 0x23, 0xc0, 0x48, 0x6c, 0x78, 0x06, 0xfa, 0xb6, 0x4a, 0x48, 0x51, 0xab, 0x1c, 0xb5, 0x50, 0x7f, 0xac, 0x40, 0x59, 0xa8, 0x21, 0xe7, 0x47, 0x5e, 0xd7, 0x3f,
0x93, 0x17, 0xf2, 0xf4, 0x4c, 0x1f, 0xdd, 0x82, 0x1a, 0x63, 0x53, 0x44, 0xba, 0xf4, 0xca, 0x26, 0x3f, 0x46, 0x80, 0x91, 0xd8, 0xf0, 0x0c, 0xf4, 0x6d, 0x95, 0x90, 0x4a, 0x2f, 0xe4, 0xf2, 0x99,
0xca, 0x40, 0x34, 0x3c, 0x93, 0x53, 0xa9, 0x55, 0x7d, 0xa1, 0xc4, 0xe2, 0xa0, 0xae, 0x89, 0xe9, 0x3e, 0xba, 0x05, 0x35, 0xc6, 0xa6, 0x88, 0x74, 0xe9, 0xb5, 0x4e, 0x94, 0xa5, 0x68, 0x78, 0x26,
0x48, 0xb9, 0x81, 0x93, 0x5a, 0x45, 0x6c, 0x4a, 0xbc, 0x5d, 0x1b, 0x1b, 0x26, 0xf6, 0xa2, 0xb9, 0xa7, 0x52, 0xab, 0xfa, 0x42, 0x89, 0xc5, 0x4a, 0x5d, 0x13, 0xd3, 0x91, 0x72, 0x03, 0xa7, 0xb9,
0x45, 0x65, 0xe2, 0x5e, 0xb2, 0xdf, 0x3a, 0xf1, 0xfe, 0xb9, 0xd5, 0x05, 0x06, 0x22, 0x07, 0x03, 0x8a, 0xd8, 0x94, 0x78, 0xc4, 0x36, 0x36, 0x4c, 0xec, 0x45, 0x73, 0x8b, 0xca, 0xc4, 0x05, 0x65,
0xf4, 0x1c, 0x4c, 0x99, 0x9d, 0xd8, 0x33, 0xb8, 0xd0, 0x1f, 0x36, 0x3b, 0xc2, 0xfb, 0xb7, 0x18, 0xbf, 0x75, 0x72, 0x42, 0xe0, 0x56, 0x17, 0x18, 0x88, 0x1c, 0x1e, 0xd0, 0x73, 0x30, 0x65, 0x76,
0x41, 0x93, 0x71, 0x82, 0xbe, 0xd1, 0x7f, 0x58, 0xec, 0x61, 0x13, 0x3b, 0x81, 0x65, 0xd8, 0x87, 0x62, 0x4f, 0xe5, 0x42, 0x9f, 0xd9, 0xec, 0x08, 0x6f, 0xe4, 0x62, 0x04, 0x4d, 0xc6, 0x09, 0xfa,
0x97, 0xc9, 0x26, 0x14, 0x7b, 0x3e, 0xf6, 0x84, 0x4d, 0x22, 0x2a, 0x93, 0xba, 0xae, 0xe1, 0xfb, 0x46, 0xff, 0xf1, 0xb1, 0x87, 0x4d, 0xec, 0x04, 0x96, 0x61, 0x1f, 0x5e, 0x26, 0x9b, 0x50, 0xec,
0xbb, 0xae, 0x67, 0x72, 0x2a, 0xa3, 0xf2, 0x90, 0xa4, 0x47, 0xf6, 0x18, 0x55, 0x9e, 0xf4, 0x78, 0xf9, 0xd8, 0x13, 0x36, 0x89, 0xa8, 0x4c, 0xea, 0xba, 0x86, 0xef, 0xef, 0xba, 0x9e, 0xc9, 0xa9,
0x09, 0x16, 0x3a, 0xae, 0x69, 0x6d, 0x59, 0xb2, 0x5c, 0x49, 0xd2, 0x6c, 0x2e, 0xac, 0x8e, 0xb5, 0x8c, 0xca, 0x43, 0x12, 0x23, 0xd9, 0x83, 0x55, 0x79, 0x62, 0xe4, 0x25, 0x58, 0xe8, 0xb8, 0xa6,
0x0b, 0x9f, 0x71, 0xcc, 0x88, 0xcf, 0x38, 0x7e, 0x90, 0x81, 0x85, 0xf7, 0xbb, 0xe6, 0xe7, 0xc0, 0xb5, 0x65, 0xc9, 0xf2, 0x29, 0x49, 0xb3, 0xb9, 0xb0, 0x3a, 0xd6, 0x2e, 0x7c, 0xea, 0x31, 0x23,
0x87, 0x45, 0x28, 0xbb, 0xb6, 0xb9, 0x1e, 0x67, 0x85, 0x08, 0x22, 0x18, 0x0e, 0xde, 0x8d, 0x30, 0x3e, 0xf5, 0xf8, 0x41, 0x06, 0x16, 0xde, 0xef, 0x9a, 0x9f, 0x03, 0x1f, 0x16, 0xa1, 0xec, 0xda,
0xd8, 0x25, 0xbe, 0x08, 0x1a, 0x9a, 0x24, 0x7a, 0x28, 0x7e, 0xe5, 0x87, 0xf1, 0xab, 0xf4, 0xd9, 0xe6, 0x7a, 0x9c, 0x15, 0x22, 0x88, 0x60, 0x38, 0x78, 0x37, 0xc2, 0x60, 0x17, 0xfd, 0x22, 0x68,
0x95, 0x7c, 0x31, 0x53, 0x9f, 0x6d, 0x64, 0xd4, 0x9f, 0x87, 0x05, 0x16, 0x6e, 0x7e, 0xcc, 0x5c, 0x68, 0x22, 0xe9, 0xa1, 0xf8, 0x95, 0x1f, 0xc6, 0xaf, 0xd2, 0x67, 0x57, 0xf2, 0xc5, 0x4c, 0x7d,
0x0a, 0xd7, 0x68, 0x4e, 0x5c, 0xa3, 0x8f, 0x61, 0x8e, 0x58, 0x73, 0x32, 0xf4, 0xfb, 0x3e, 0xf6, 0xb6, 0x91, 0x51, 0x7f, 0x1e, 0x16, 0x58, 0x48, 0xfa, 0x31, 0x73, 0x29, 0x5c, 0xa3, 0x39, 0x71,
0xc6, 0x34, 0x52, 0x27, 0xa1, 0x14, 0x8e, 0x16, 0xa6, 0xf7, 0xf6, 0x01, 0xea, 0xcf, 0xc1, 0x6c, 0x8d, 0x3e, 0x86, 0x39, 0x62, 0xcd, 0xc9, 0xd0, 0xef, 0xfb, 0xd8, 0x1b, 0xd3, 0x48, 0x9d, 0x84,
0x62, 0xac, 0x43, 0xce, 0x32, 0x9c, 0xc9, 0xbc, 0x38, 0x93, 0x45, 0x00, 0xcd, 0xb5, 0xf1, 0x3b, 0x52, 0x38, 0x5a, 0x98, 0x02, 0xdc, 0x07, 0xa8, 0x3f, 0x07, 0xb3, 0x89, 0xb1, 0x0e, 0x39, 0xcb,
0x4e, 0x60, 0x05, 0x7b, 0xc4, 0x4b, 0x10, 0xdc, 0x2f, 0xfa, 0x9b, 0x60, 0x90, 0x71, 0x87, 0x60, 0x70, 0x26, 0xf3, 0xe2, 0x4c, 0x16, 0x01, 0x34, 0xd7, 0xc6, 0xef, 0x38, 0x81, 0x15, 0xec, 0x11,
0xfc, 0xaa, 0x02, 0xd3, 0x4c, 0x73, 0x49, 0x57, 0x87, 0x5f, 0x85, 0xd7, 0x20, 0x8f, 0xe9, 0x28, 0x2f, 0x41, 0x70, 0xbf, 0xe8, 0x6f, 0x82, 0x41, 0xc6, 0x1d, 0x82, 0xf1, 0x2b, 0x0a, 0x4c, 0x33,
0xfc, 0x46, 0xe1, 0xb4, 0xdc, 0x54, 0x47, 0xe4, 0x6a, 0x1c, 0x5d, 0xaa, 0x46, 0x01, 0x4c, 0xad, 0xcd, 0x25, 0x5d, 0x1d, 0x7e, 0x15, 0x5e, 0x83, 0x3c, 0xa6, 0xa3, 0xf0, 0x5b, 0x87, 0xd3, 0x72,
0x7a, 0x6e, 0x77, 0x3c, 0x8a, 0xa8, 0x67, 0x62, 0x63, 0xd1, 0xd7, 0x2c, 0x12, 0xc0, 0xdd, 0x34, 0x53, 0x1d, 0x91, 0xab, 0x71, 0x74, 0xa9, 0x1a, 0x05, 0x30, 0xb5, 0xea, 0xb9, 0xdd, 0xf1, 0x28,
0xc1, 0xf8, 0x27, 0x05, 0xe6, 0xef, 0x75, 0xb1, 0x67, 0x04, 0x98, 0x30, 0x6d, 0xbc, 0xd1, 0x87, 0xa2, 0x9e, 0x89, 0x8d, 0x45, 0x5f, 0xb3, 0x48, 0x00, 0x77, 0xd3, 0x04, 0xe3, 0x9f, 0x14, 0x98,
0xe9, 0x6e, 0x8c, 0xb2, 0x6c, 0x9c, 0x32, 0xf4, 0x66, 0xec, 0xed, 0x99, 0xfc, 0x3c, 0x92, 0xa0, 0xbf, 0xd7, 0xc5, 0x9e, 0x11, 0x60, 0xc2, 0xb4, 0xf1, 0x46, 0x1f, 0xa6, 0xbb, 0x31, 0xca, 0xb2,
0xb2, 0x9f, 0xc3, 0x1e, 0xce, 0x6b, 0x41, 0x9c, 0xd7, 0x8f, 0x14, 0x98, 0xde, 0xc0, 0x64, 0x1f, 0x71, 0xca, 0xd0, 0x9b, 0xb1, 0xf7, 0x69, 0xf2, 0xf3, 0x48, 0x82, 0xca, 0x7e, 0x9e, 0x7b, 0x38,
0x1b, 0x6f, 0x4a, 0x17, 0x61, 0x92, 0x50, 0x39, 0xea, 0x02, 0x53, 0x64, 0x74, 0x0e, 0xa6, 0x2d, 0xaf, 0x05, 0x71, 0x5e, 0x3f, 0x52, 0x60, 0x7a, 0x03, 0x93, 0x7d, 0x6c, 0xbc, 0x29, 0x5d, 0x84,
0xa7, 0x65, 0xf7, 0x4c, 0xac, 0x93, 0xf9, 0xeb, 0xc4, 0x8d, 0xe3, 0xce, 0xc3, 0x14, 0xaf, 0x20, 0x49, 0x42, 0xe5, 0xa8, 0x0b, 0x4c, 0x91, 0xd1, 0x39, 0x98, 0xb6, 0x9c, 0x96, 0xdd, 0x33, 0xb1,
0xd3, 0x20, 0x5b, 0xb4, 0x54, 0xc6, 0x1f, 0x31, 0x19, 0x8f, 0x92, 0x7c, 0x18, 0x09, 0xca, 0x41, 0x4e, 0xe6, 0xaf, 0x13, 0x37, 0x8e, 0x3b, 0x0f, 0x53, 0xbc, 0x82, 0x4c, 0x83, 0x6c, 0xd1, 0x52,
0x48, 0x78, 0x15, 0x72, 0x64, 0xe8, 0xd0, 0x89, 0x90, 0xb7, 0xea, 0xab, 0x89, 0xc6, 0xb0, 0xd5, 0x19, 0x7f, 0xc4, 0x64, 0x3c, 0x4a, 0x04, 0x62, 0x24, 0x28, 0x07, 0x21, 0xe1, 0x55, 0xc8, 0x91,
0x5f, 0x50, 0x00, 0x89, 0x6c, 0x1b, 0xc7, 0x4a, 0xbc, 0x2e, 0x06, 0xf7, 0xb3, 0x43, 0x49, 0x67, 0xa1, 0x43, 0x27, 0x42, 0xde, 0xaa, 0xaf, 0x26, 0x1a, 0xc3, 0x56, 0x7f, 0x41, 0x01, 0x24, 0xb2,
0x33, 0x8d, 0xc2, 0xfa, 0xea, 0xa7, 0xd1, 0xea, 0xd1, 0xe5, 0x1e, 0x67, 0xf5, 0xc8, 0xbc, 0x86, 0x6d, 0x1c, 0x2b, 0xf1, 0xba, 0x98, 0x00, 0x90, 0x1d, 0x4a, 0x3a, 0x9b, 0x69, 0x14, 0xfa, 0x57,
0xae, 0x9e, 0xc0, 0x04, 0x8a, 0x2c, 0xae, 0x1e, 0x95, 0x58, 0xc9, 0xea, 0x11, 0x9a, 0xe9, 0xea, 0x3f, 0x8d, 0x56, 0x8f, 0x2e, 0xf7, 0x38, 0xab, 0x47, 0xe6, 0x35, 0x74, 0xf5, 0x04, 0x26, 0x50,
0x71, 0xfb, 0xde, 0x68, 0x64, 0xc8, 0xa2, 0x31, 0x62, 0xc3, 0x45, 0xa3, 0x23, 0x2b, 0x07, 0x19, 0x64, 0x71, 0xf5, 0xa8, 0xc4, 0x4a, 0x56, 0x8f, 0xd0, 0x4c, 0x57, 0x8f, 0xdb, 0xf7, 0x46, 0x23,
0xf9, 0x55, 0xc8, 0x91, 0x11, 0xf7, 0xe7, 0x57, 0xb8, 0x68, 0x14, 0x5b, 0x58, 0x34, 0x4e, 0xc0, 0x43, 0x16, 0x8d, 0x11, 0x1b, 0x2e, 0x1a, 0x1d, 0x59, 0x39, 0xc8, 0xc8, 0xaf, 0x42, 0x8e, 0x8c,
0xe3, 0x5f, 0xb4, 0xfe, 0x4c, 0xfb, 0x8b, 0xa6, 0x42, 0xe5, 0xde, 0xe6, 0xc7, 0xb8, 0x15, 0x0c, 0xb8, 0x3f, 0xbf, 0xc2, 0x45, 0xa3, 0xd8, 0xc2, 0xa2, 0x71, 0x02, 0x1e, 0xff, 0xa2, 0xf5, 0x67,
0xb1, 0xbc, 0x67, 0x61, 0x6a, 0xdd, 0xb3, 0x76, 0x2c, 0x1b, 0xb7, 0x87, 0x99, 0xf0, 0x6f, 0x29, 0xda, 0x5f, 0x34, 0x15, 0x2a, 0xf7, 0x36, 0x3f, 0xc6, 0xad, 0x60, 0x88, 0xe5, 0x3d, 0x0b, 0x53,
0x50, 0xbd, 0xe9, 0x19, 0x4e, 0xe0, 0x86, 0x66, 0xfc, 0x50, 0xfc, 0xbc, 0x06, 0xa5, 0x6e, 0x38, 0xeb, 0x9e, 0xb5, 0x63, 0xd9, 0xb8, 0x3d, 0xcc, 0x84, 0x7f, 0x4b, 0x81, 0xea, 0x4d, 0xcf, 0x70,
0x1a, 0x97, 0x81, 0x67, 0xe5, 0x51, 0x97, 0x38, 0x4d, 0x5a, 0xbf, 0x99, 0xfa, 0x01, 0xcc, 0x52, 0x02, 0x37, 0x34, 0xe3, 0x87, 0xe2, 0xe7, 0x35, 0x28, 0x75, 0xc3, 0xd1, 0xb8, 0x0c, 0x3c, 0x2b,
0x4a, 0x92, 0x64, 0x5f, 0x81, 0x22, 0x35, 0xe6, 0x16, 0xbf, 0xe8, 0x28, 0xaf, 0xa8, 0xf2, 0x23, 0x8f, 0xcc, 0xc4, 0x69, 0xd2, 0xfa, 0xcd, 0xd4, 0x0f, 0x60, 0x96, 0x52, 0x92, 0x24, 0xfb, 0x0a,
0x8d, 0x38, 0x0d, 0x2d, 0x6a, 0xa3, 0xfe, 0x9b, 0x02, 0x65, 0x5a, 0xd7, 0x9f, 0xe0, 0xc1, 0xb5, 0x14, 0xa9, 0x31, 0xb7, 0xf8, 0x45, 0x47, 0x79, 0x45, 0x95, 0x1f, 0x69, 0xc4, 0x69, 0x68, 0x51,
0xfc, 0x75, 0xc8, 0xbb, 0x94, 0xe5, 0x43, 0x83, 0xb3, 0xe2, 0xaa, 0x68, 0xbc, 0x01, 0xf1, 0x90, 0x1b, 0xf5, 0xdf, 0x14, 0x28, 0xd3, 0xba, 0xfe, 0x04, 0x0f, 0xae, 0xe5, 0xaf, 0x43, 0xde, 0xa5,
0xd9, 0x2f, 0xd1, 0x22, 0x03, 0x03, 0x71, 0x9b, 0x5c, 0x68, 0x33, 0xda, 0xa9, 0x59, 0x1e, 0x6d, 0x2c, 0x1f, 0x1a, 0xc0, 0x15, 0x57, 0x45, 0xe3, 0x0d, 0x88, 0x87, 0xcc, 0x7e, 0x89, 0x16, 0x19,
0x7e, 0x61, 0x13, 0x7a, 0x56, 0x63, 0x32, 0x49, 0x11, 0x0e, 0xaf, 0xc2, 0x5f, 0x4e, 0xec, 0xb1, 0x18, 0x88, 0xdb, 0xe4, 0x42, 0x9b, 0xd1, 0x4e, 0xcd, 0xf2, 0x68, 0xf3, 0x0b, 0x9b, 0xd0, 0xb3,
0x8b, 0xe9, 0x54, 0xc8, 0x37, 0xd9, 0x98, 0x65, 0x25, 0x67, 0xb5, 0x18, 0x59, 0x63, 0x9e, 0xd5, 0x1a, 0x93, 0x49, 0x8a, 0x70, 0x78, 0x15, 0xfe, 0x72, 0x62, 0x8f, 0x5d, 0x4c, 0xa7, 0x42, 0xbe,
0x22, 0x11, 0x18, 0x76, 0x56, 0x13, 0x89, 0xeb, 0x0b, 0xc0, 0xbf, 0x28, 0xb0, 0xc0, 0xf7, 0xb4, 0xc9, 0xc6, 0x2c, 0x2b, 0x39, 0xab, 0xc5, 0xc8, 0x1a, 0xf3, 0xac, 0x16, 0x89, 0xc0, 0xb0, 0xb3,
0x48, 0xb6, 0x8e, 0x80, 0x4d, 0xe8, 0x2b, 0x7c, 0xef, 0xcd, 0xd2, 0xbd, 0xf7, 0x85, 0x61, 0x7b, 0x9a, 0x48, 0x5c, 0x5f, 0x00, 0xfe, 0x45, 0x81, 0x05, 0xbe, 0xa7, 0x45, 0xb2, 0x75, 0x04, 0x6c,
0x6f, 0x44, 0xe7, 0x3e, 0x9b, 0xef, 0x59, 0x28, 0xdd, 0xa1, 0x0d, 0xdf, 0x79, 0x14, 0xa0, 0x06, 0x42, 0x5f, 0xe1, 0x7b, 0x6f, 0x96, 0xee, 0xbd, 0x2f, 0x0c, 0xdb, 0x7b, 0x23, 0x3a, 0xf7, 0xd9,
0x14, 0x76, 0xb0, 0xe7, 0x5b, 0xae, 0xc3, 0x55, 0x3c, 0x2c, 0x9e, 0x3b, 0x03, 0xc5, 0xf0, 0x35, 0x7c, 0xcf, 0x42, 0xe9, 0x0e, 0x6d, 0xf8, 0xce, 0xa3, 0x00, 0x35, 0xa0, 0xb0, 0x83, 0x3d, 0xdf,
0x1a, 0x2a, 0x40, 0xf6, 0xaa, 0x6d, 0xd7, 0x27, 0x50, 0x05, 0x8a, 0x6b, 0xfc, 0xc9, 0x55, 0x5d, 0x72, 0x1d, 0xae, 0xe2, 0x61, 0xf1, 0xdc, 0x19, 0x28, 0x86, 0x2f, 0xd6, 0x50, 0x01, 0xb2, 0x57,
0x39, 0xf7, 0x36, 0xcc, 0x48, 0xf6, 0x7d, 0x34, 0x0d, 0xd5, 0xab, 0x26, 0xf5, 0x2e, 0xef, 0xbb, 0x6d, 0xbb, 0x3e, 0x81, 0x2a, 0x50, 0x5c, 0xe3, 0xcf, 0xb2, 0xea, 0xca, 0xb9, 0xb7, 0x61, 0x46,
0x04, 0x58, 0x9f, 0x40, 0xf3, 0x80, 0x34, 0xdc, 0x71, 0x77, 0x28, 0xe2, 0x0d, 0xcf, 0xed, 0x50, 0xb2, 0xef, 0xa3, 0x69, 0xa8, 0x5e, 0x35, 0xa9, 0x77, 0x79, 0xdf, 0x25, 0xc0, 0xfa, 0x04, 0x9a,
0xb8, 0x72, 0xee, 0x45, 0x98, 0x95, 0x51, 0x8f, 0x4a, 0x90, 0xa3, 0xdc, 0xa8, 0x4f, 0x20, 0x80, 0x07, 0xa4, 0xe1, 0x8e, 0xbb, 0x43, 0x11, 0x6f, 0x78, 0x6e, 0x87, 0xc2, 0x95, 0x73, 0x2f, 0xc2,
0xbc, 0x86, 0x77, 0xdc, 0x07, 0xb8, 0xae, 0xac, 0xfc, 0xf9, 0x39, 0xa8, 0x32, 0xda, 0xf9, 0xdb, 0xac, 0x8c, 0x7a, 0x54, 0x82, 0x1c, 0xe5, 0x46, 0x7d, 0x02, 0x01, 0xe4, 0x35, 0xbc, 0xe3, 0x3e,
0x69, 0xa4, 0x43, 0x3d, 0xf9, 0xf9, 0x28, 0xf4, 0x25, 0xf9, 0x8d, 0xa9, 0xfc, 0x2b, 0x53, 0xcd, 0xc0, 0x75, 0x65, 0xe5, 0xcf, 0xcf, 0x41, 0x95, 0xd1, 0xce, 0xdf, 0x57, 0x23, 0x1d, 0xea, 0xc9,
0x61, 0xc2, 0xa4, 0x4e, 0xa0, 0x0f, 0xa1, 0x16, 0xff, 0xe0, 0x12, 0x92, 0x87, 0x86, 0xa5, 0x5f, 0x4f, 0x4c, 0xa1, 0x2f, 0xc9, 0x6f, 0x4c, 0xe5, 0x5f, 0xa2, 0x6a, 0x0e, 0x13, 0x26, 0x75, 0x02,
0x65, 0xda, 0xaf, 0x73, 0x1d, 0xaa, 0xb1, 0x6f, 0x25, 0x21, 0xf9, 0x02, 0xcb, 0xbe, 0xa7, 0xd4, 0x7d, 0x08, 0xb5, 0xf8, 0x47, 0x99, 0x90, 0x3c, 0x7c, 0x2c, 0xfd, 0x72, 0xd3, 0x7e, 0x9d, 0xeb,
0x94, 0x5b, 0x13, 0xf1, 0x7b, 0x46, 0x8c, 0xfa, 0xf8, 0xc7, 0x4b, 0x52, 0xa8, 0x97, 0x7e, 0xe1, 0x50, 0x8d, 0x7d, 0x4f, 0x09, 0xc9, 0x17, 0x58, 0xf6, 0xcd, 0xa5, 0xa6, 0xdc, 0x9a, 0x88, 0xdf,
0x64, 0x3f, 0xea, 0x0d, 0x98, 0x1e, 0xf8, 0xb6, 0x08, 0x7a, 0x31, 0xe5, 0x42, 0x44, 0xfe, 0x0d, 0x3c, 0x62, 0xd4, 0xc7, 0x3f, 0x70, 0x92, 0x42, 0xbd, 0xf4, 0x2b, 0x28, 0xfb, 0x51, 0x6f, 0xc0,
0x92, 0xfd, 0x86, 0xd8, 0x05, 0x34, 0xf8, 0xfd, 0x1f, 0xb4, 0x2c, 0x5f, 0x81, 0xb4, 0x2f, 0x22, 0xf4, 0xc0, 0xf7, 0x47, 0xd0, 0x8b, 0x29, 0x17, 0x22, 0xf2, 0xef, 0x94, 0xec, 0x37, 0xc4, 0x2e,
0x35, 0x2f, 0x8c, 0x8c, 0x1f, 0x31, 0xee, 0x9b, 0x0a, 0x2c, 0xa4, 0x7c, 0x86, 0x02, 0x5d, 0x4c, 0xa0, 0xc1, 0x6f, 0x04, 0xa1, 0x65, 0xf9, 0x0a, 0xa4, 0x7d, 0x35, 0xa9, 0x79, 0x61, 0x64, 0xfc,
0xbb, 0x1d, 0x1b, 0xf2, 0x51, 0x8d, 0xe6, 0x2b, 0x07, 0x6b, 0x14, 0x11, 0xe2, 0xc0, 0x54, 0xe2, 0x88, 0x71, 0xdf, 0x54, 0x60, 0x21, 0xe5, 0x53, 0x15, 0xe8, 0x62, 0xda, 0xed, 0xd8, 0x90, 0x0f,
0x2b, 0x0c, 0xe8, 0x7c, 0xea, 0xd3, 0xd1, 0xc1, 0x4f, 0x54, 0x34, 0xbf, 0x34, 0x1a, 0x72, 0x34, 0x6f, 0x34, 0x5f, 0x39, 0x58, 0xa3, 0x88, 0x10, 0x07, 0xa6, 0x12, 0x5f, 0x6a, 0x40, 0xe7, 0x53,
0xde, 0x47, 0x30, 0x95, 0xf8, 0x04, 0x41, 0xca, 0x78, 0xf2, 0x0f, 0x15, 0xec, 0xb7, 0xa0, 0x5f, 0x9f, 0x97, 0x0e, 0x7e, 0xc6, 0xa2, 0xf9, 0xa5, 0xd1, 0x90, 0xa3, 0xf1, 0x3e, 0x82, 0xa9, 0xc4,
0x85, 0x6a, 0xec, 0x5b, 0x01, 0x29, 0x12, 0x2f, 0xfb, 0x9e, 0xc0, 0x7e, 0x5d, 0x7f, 0x04, 0x15, 0x67, 0x0a, 0x52, 0xc6, 0x93, 0x7f, 0xcc, 0x60, 0xbf, 0x05, 0xfd, 0x2a, 0x54, 0x63, 0xdf, 0x13,
0xf1, 0x49, 0x3f, 0x5a, 0x4a, 0xd3, 0xa5, 0x81, 0x8e, 0x0f, 0xa2, 0x4a, 0xfd, 0xa7, 0xb8, 0x43, 0x48, 0x91, 0x78, 0xd9, 0x37, 0x07, 0xf6, 0xeb, 0xfa, 0x23, 0xa8, 0x88, 0xcf, 0xfe, 0xd1, 0x52,
0x54, 0x69, 0xe0, 0xf5, 0xf2, 0xe8, 0xaa, 0x24, 0xf4, 0x3f, 0x54, 0x95, 0x0e, 0x3c, 0xc4, 0xd7, 0x9a, 0x2e, 0x0d, 0x74, 0x7c, 0x10, 0x55, 0xea, 0x3f, 0xd7, 0x1d, 0xa2, 0x4a, 0x03, 0x2f, 0x9c,
0x15, 0x7a, 0x3d, 0x2f, 0x79, 0xea, 0x8d, 0x56, 0xd2, 0x64, 0x33, 0xfd, 0x51, 0x7b, 0xf3, 0xe2, 0x47, 0x57, 0x25, 0xa1, 0xff, 0xa1, 0xaa, 0x74, 0xe0, 0x21, 0xbe, 0xae, 0xd0, 0xeb, 0x79, 0xc9,
0x81, 0xda, 0x44, 0x5c, 0x7c, 0x00, 0xb5, 0xf8, 0x83, 0xe6, 0x14, 0x2e, 0x4a, 0xdf, 0x80, 0x37, 0x73, 0x70, 0xb4, 0x92, 0x26, 0x9b, 0xe9, 0x0f, 0xdf, 0x9b, 0x17, 0x0f, 0xd4, 0x26, 0xe2, 0xe2,
0xcf, 0x8f, 0x84, 0x1b, 0x0d, 0xf6, 0x3e, 0x94, 0x85, 0x2f, 0x42, 0xa2, 0xe7, 0x87, 0xc8, 0xb1, 0x03, 0xa8, 0xc5, 0x1f, 0x3d, 0xa7, 0x70, 0x51, 0xfa, 0x4e, 0xbc, 0x79, 0x7e, 0x24, 0xdc, 0x68,
0xf8, 0x79, 0xc4, 0xfd, 0x38, 0xf9, 0x1e, 0x94, 0xa2, 0x0f, 0x39, 0xa2, 0xb3, 0xa9, 0xf2, 0x7b, 0xb0, 0xf7, 0xa1, 0x2c, 0x7c, 0x35, 0x12, 0x3d, 0x3f, 0x44, 0x8e, 0xc5, 0x4f, 0x28, 0xee, 0xc7,
0x90, 0x2e, 0x37, 0x00, 0xfa, 0x5f, 0x69, 0x44, 0xcf, 0x49, 0xfb, 0x1c, 0xf8, 0x8c, 0xe3, 0x7e, 0xc9, 0xf7, 0xa0, 0x14, 0x7d, 0xec, 0x11, 0x9d, 0x4d, 0x95, 0xdf, 0x83, 0x74, 0xb9, 0x01, 0xd0,
0x9d, 0x46, 0xd3, 0x67, 0x2f, 0x3e, 0x86, 0x4d, 0x5f, 0x7c, 0xb4, 0xb4, 0x5f, 0xb7, 0xdb, 0x50, 0xff, 0x92, 0x23, 0x7a, 0x4e, 0xda, 0xe7, 0xc0, 0xa7, 0x1e, 0xf7, 0xeb, 0x34, 0x9a, 0x3e, 0x7b,
0x8d, 0x3d, 0x3e, 0x4c, 0x53, 0x61, 0xc9, 0xe3, 0xd0, 0xe6, 0xb9, 0x51, 0x50, 0xa3, 0xf5, 0xdb, 0x15, 0x32, 0x6c, 0xfa, 0xe2, 0xc3, 0xa6, 0xfd, 0xba, 0xdd, 0x86, 0x6a, 0xec, 0x81, 0x62, 0x9a,
0x86, 0x6a, 0xec, 0xe1, 0x57, 0xca, 0x48, 0xb2, 0x07, 0x6f, 0x29, 0x23, 0x49, 0xdf, 0x91, 0xa9, 0x0a, 0x4b, 0x1e, 0x90, 0x36, 0xcf, 0x8d, 0x82, 0x1a, 0xad, 0xdf, 0x36, 0x54, 0x63, 0x8f, 0xc3,
0x13, 0xe8, 0x6b, 0xc2, 0x1b, 0xb3, 0xd8, 0x83, 0x3e, 0xf4, 0xf2, 0xd0, 0x7e, 0x64, 0x0f, 0x1b, 0x52, 0x46, 0x92, 0x3d, 0x8a, 0x4b, 0x19, 0x49, 0xfa, 0xd6, 0x4c, 0x9d, 0x40, 0x5f, 0x13, 0xde,
0x9b, 0x2b, 0x07, 0x69, 0x12, 0x91, 0xc0, 0xa5, 0x8a, 0xb1, 0x34, 0x5d, 0xaa, 0x0e, 0xb2, 0x52, 0xa1, 0xc5, 0x1e, 0xfd, 0xa1, 0x97, 0x87, 0xf6, 0x23, 0x7b, 0xfc, 0xd8, 0x5c, 0x39, 0x48, 0x93,
0x1b, 0x90, 0x67, 0x2f, 0xb8, 0x90, 0x9a, 0xf2, 0x8c, 0x53, 0x78, 0xde, 0xd5, 0x7c, 0x46, 0x8a, 0x88, 0x04, 0x2e, 0x55, 0x8c, 0xa5, 0xe9, 0x52, 0x75, 0x90, 0x95, 0xda, 0x80, 0x3c, 0x7b, 0xe5,
0x13, 0x7f, 0xd3, 0xc4, 0x3a, 0x65, 0x37, 0xa5, 0x29, 0x9d, 0xc6, 0x5e, 0xed, 0x8c, 0xda, 0xa9, 0x85, 0xd4, 0x94, 0xa7, 0x9e, 0xc2, 0x13, 0xb0, 0xe6, 0x33, 0x52, 0x9c, 0xf8, 0xbb, 0x27, 0xd6,
0x06, 0x79, 0xf6, 0x8c, 0x20, 0xa5, 0xd3, 0xd8, 0x1b, 0x99, 0xe6, 0x70, 0x1c, 0x76, 0xde, 0x9d, 0x29, 0xbb, 0x29, 0x4d, 0xe9, 0x34, 0xf6, 0xb2, 0x67, 0xd4, 0x4e, 0x35, 0xc8, 0xb3, 0xa7, 0x06,
0x40, 0xeb, 0x90, 0xa3, 0xe1, 0x67, 0x74, 0x66, 0x58, 0x62, 0xfd, 0xb0, 0x1e, 0x63, 0xb9, 0xf7, 0x29, 0x9d, 0xc6, 0xde, 0xd1, 0x34, 0x87, 0xe3, 0xb0, 0xf3, 0xee, 0x04, 0x5a, 0x87, 0x1c, 0x0d,
0xea, 0x04, 0xba, 0x07, 0x39, 0x1a, 0xc0, 0x4b, 0xe9, 0x51, 0xcc, 0x8e, 0x6f, 0x0e, 0x45, 0x09, 0x3f, 0xa3, 0x33, 0xc3, 0x92, 0xef, 0x87, 0xf5, 0x18, 0xcb, 0xcf, 0x57, 0x27, 0xd0, 0x3d, 0xc8,
0x49, 0x34, 0xa1, 0x22, 0x66, 0xc2, 0xa6, 0x6c, 0x59, 0x92, 0x5c, 0xe1, 0xe6, 0x28, 0x98, 0xe1, 0xd1, 0x00, 0x5e, 0x4a, 0x8f, 0x62, 0x06, 0x7d, 0x73, 0x28, 0x4a, 0x48, 0xa2, 0x09, 0x15, 0x31,
0x28, 0x4c, 0x8d, 0xfa, 0xa1, 0xf8, 0x74, 0x35, 0x1a, 0x08, 0xf3, 0xa7, 0xab, 0xd1, 0x60, 0x64, 0x5b, 0x36, 0x65, 0xcb, 0x92, 0xe4, 0x13, 0x37, 0x47, 0xc1, 0x0c, 0x47, 0x61, 0x6a, 0xd4, 0x0f,
0x5f, 0x9d, 0x40, 0xbf, 0xa4, 0x40, 0x23, 0x2d, 0x3d, 0x13, 0xa5, 0x7a, 0x40, 0xc3, 0x72, 0x4c, 0xc5, 0xa7, 0xab, 0xd1, 0x40, 0x98, 0x3f, 0x5d, 0x8d, 0x06, 0x23, 0xfb, 0xea, 0x04, 0xfa, 0x45,
0x9b, 0xaf, 0x1e, 0xb0, 0x55, 0x44, 0xcb, 0x27, 0x34, 0xee, 0x37, 0x90, 0x90, 0x79, 0x21, 0xad, 0x05, 0x1a, 0x69, 0x29, 0x9c, 0x28, 0xd5, 0x03, 0x1a, 0x96, 0x87, 0xda, 0x7c, 0xf5, 0x80, 0xad,
0xbf, 0x94, 0x24, 0xc3, 0xe6, 0x4b, 0xa3, 0x37, 0x88, 0xc6, 0xde, 0x84, 0xb2, 0x10, 0x73, 0x4c, 0x22, 0x5a, 0x3e, 0xa1, 0x71, 0xbf, 0x81, 0xa4, 0xcd, 0x0b, 0x69, 0xfd, 0xa5, 0x24, 0x22, 0x36,
0xb1, 0xbc, 0x83, 0xc1, 0xd2, 0x94, 0x55, 0x95, 0x84, 0x2f, 0x99, 0x78, 0xd3, 0x2c, 0xbe, 0x14, 0x5f, 0x1a, 0xbd, 0x41, 0x34, 0xf6, 0x26, 0x94, 0x85, 0x98, 0x63, 0x8a, 0xe5, 0x1d, 0x0c, 0x96,
0x61, 0x14, 0x93, 0x02, 0x53, 0xc4, 0x3b, 0x96, 0x04, 0xa8, 0x4e, 0x20, 0x0c, 0x15, 0x31, 0xa5, 0xa6, 0xac, 0xaa, 0x24, 0x7c, 0xc9, 0xc4, 0x9b, 0x66, 0xfa, 0xa5, 0x08, 0xa3, 0x98, 0x38, 0x98,
0x2f, 0x45, 0x1a, 0x25, 0xd9, 0x80, 0xcd, 0x17, 0x46, 0xc0, 0x8c, 0x86, 0xd1, 0x01, 0xfa, 0x29, 0x22, 0xde, 0xb1, 0x44, 0x41, 0x75, 0x02, 0x61, 0xa8, 0x88, 0x69, 0x7f, 0x29, 0xd2, 0x28, 0xc9,
0x75, 0x29, 0x7b, 0xdd, 0x40, 0x56, 0x5f, 0xf3, 0xf9, 0x7d, 0xf1, 0xc4, 0x6d, 0x5f, 0x48, 0x92, 0x18, 0x6c, 0xbe, 0x30, 0x02, 0x66, 0x34, 0x8c, 0x0e, 0xd0, 0x4f, 0xbb, 0x4b, 0xd9, 0xeb, 0x06,
0x4b, 0xe1, 0xfe, 0x60, 0x1a, 0xdd, 0x08, 0x67, 0x91, 0xc1, 0xb4, 0xab, 0x94, 0xb3, 0x48, 0x6a, 0x32, 0xff, 0x9a, 0xcf, 0xef, 0x8b, 0x27, 0x6e, 0xfb, 0x42, 0x22, 0x5d, 0x0a, 0xf7, 0x07, 0x53,
0x86, 0x57, 0xf3, 0xc2, 0xc8, 0xf8, 0xd1, 0x7c, 0x1e, 0x42, 0x3d, 0x99, 0xa6, 0x96, 0x72, 0xc6, 0xed, 0x46, 0x38, 0x8b, 0x0c, 0xa6, 0x66, 0xa5, 0x9c, 0x45, 0x52, 0xb3, 0xc0, 0x9a, 0x17, 0x46,
0x4d, 0xc9, 0x9a, 0x6b, 0xbe, 0x38, 0x22, 0xb6, 0xb8, 0x1f, 0x9e, 0x18, 0xa4, 0xe9, 0x67, 0xac, 0xc6, 0x8f, 0xe6, 0xf3, 0x10, 0xea, 0xc9, 0x54, 0xb6, 0x94, 0x33, 0x6e, 0x4a, 0x66, 0x5d, 0xf3,
0x60, 0x9b, 0x66, 0x48, 0x8d, 0x32, 0x6b, 0x31, 0x19, 0x6b, 0x94, 0x59, 0xc7, 0x52, 0xaf, 0xf8, 0xc5, 0x11, 0xb1, 0xc5, 0xfd, 0xf0, 0xc4, 0x20, 0x4d, 0x3f, 0x63, 0x05, 0xdb, 0x34, 0x43, 0x6a,
0xe6, 0x45, 0xb3, 0x0d, 0xd2, 0x36, 0x2f, 0x31, 0xe9, 0x27, 0x65, 0x9f, 0x89, 0x67, 0xc6, 0x30, 0x94, 0x59, 0x8b, 0xc9, 0x58, 0xa3, 0xcc, 0x3a, 0x96, 0x7a, 0xc5, 0x37, 0x2f, 0x9a, 0x6d, 0x90,
0xf7, 0x33, 0x9e, 0xc5, 0x80, 0xce, 0x8d, 0x94, 0xea, 0x30, 0xcc, 0xfd, 0x94, 0xa7, 0x45, 0xb0, 0xb6, 0x79, 0x89, 0x49, 0x3f, 0x29, 0xfb, 0x4c, 0x3c, 0x33, 0x86, 0xb9, 0x9f, 0xf1, 0x2c, 0x06,
0xa3, 0x5b, 0x22, 0x49, 0x23, 0xe5, 0x28, 0x25, 0xcf, 0xf2, 0x48, 0x39, 0xba, 0xa5, 0xe4, 0x7d, 0x74, 0x6e, 0xa4, 0x54, 0x87, 0x61, 0xee, 0xa7, 0x3c, 0x2d, 0x82, 0x1d, 0xdd, 0x12, 0x49, 0x1a,
0x50, 0xc5, 0xaa, 0x27, 0x23, 0xde, 0xc3, 0xef, 0x42, 0x92, 0xa1, 0xce, 0xfd, 0xaf, 0x2b, 0xea, 0x29, 0x47, 0x29, 0x79, 0x96, 0x47, 0xca, 0xd1, 0x2d, 0x25, 0xef, 0x83, 0x2a, 0x56, 0x3d, 0x19,
0xc9, 0x50, 0x72, 0xca, 0x00, 0x29, 0x11, 0xe7, 0x11, 0x06, 0x48, 0x46, 0x61, 0x53, 0x06, 0x48, 0xf1, 0x1e, 0x7e, 0x17, 0x92, 0x0c, 0x75, 0xee, 0x7f, 0x5d, 0x51, 0x4f, 0x86, 0x92, 0x53, 0x06,
0x09, 0xd6, 0x8e, 0xe0, 0xbb, 0xc6, 0xa2, 0x9f, 0x29, 0x5b, 0xa1, 0x2c, 0x42, 0x9a, 0xb2, 0x15, 0x48, 0x89, 0x38, 0x8f, 0x30, 0x40, 0x32, 0x0a, 0x9b, 0x32, 0x40, 0x4a, 0xb0, 0x76, 0x04, 0xdf,
0x4a, 0x03, 0xb7, 0xcc, 0xa3, 0xef, 0x07, 0x31, 0x53, 0xac, 0xdc, 0x40, 0x94, 0x73, 0x3f, 0xf2, 0x35, 0x16, 0xfd, 0x4c, 0xd9, 0x0a, 0x65, 0x11, 0xd2, 0x94, 0xad, 0x50, 0x1a, 0xb8, 0x65, 0x1e,
0xef, 0x41, 0x31, 0x8c, 0x42, 0xa2, 0x67, 0x53, 0x5d, 0xc4, 0x03, 0x74, 0xf8, 0x11, 0x4c, 0x25, 0x7d, 0x3f, 0x88, 0x99, 0x62, 0xe5, 0x06, 0xa2, 0x9c, 0xfb, 0x91, 0x7f, 0x0f, 0x8a, 0x61, 0x14,
0x6e, 0xf0, 0x52, 0x44, 0x54, 0x1e, 0x85, 0xdc, 0x7f, 0x3d, 0xa1, 0x1f, 0xaf, 0x4a, 0x61, 0xc2, 0x12, 0x3d, 0x9b, 0xea, 0x22, 0x1e, 0xa0, 0xc3, 0x8f, 0x60, 0x2a, 0x71, 0x83, 0x97, 0x22, 0xa2,
0x40, 0x1c, 0x30, 0xc5, 0xd4, 0x0f, 0x06, 0xbe, 0xc4, 0x01, 0x08, 0x61, 0x43, 0x07, 0x10, 0x42, 0xf2, 0x28, 0xe4, 0xfe, 0xeb, 0x09, 0xfd, 0x78, 0x55, 0x0a, 0x13, 0x06, 0xe2, 0x80, 0x29, 0xa6,
0x55, 0x43, 0x07, 0x10, 0x83, 0x34, 0x4c, 0x22, 0x93, 0x17, 0x94, 0x29, 0x12, 0x99, 0x72, 0x5b, 0x7e, 0x30, 0xf0, 0x25, 0x0e, 0x40, 0x08, 0x1b, 0x3a, 0x80, 0x10, 0xaa, 0x1a, 0x3a, 0x80, 0x18,
0xbc, 0x1f, 0x8b, 0x36, 0xa1, 0x2c, 0x5c, 0x79, 0xa3, 0x61, 0xa4, 0x89, 0x77, 0xf5, 0x29, 0xae, 0xa4, 0x61, 0x12, 0x99, 0xbc, 0xa0, 0x4c, 0x91, 0xc8, 0x94, 0xdb, 0xe2, 0xfd, 0x58, 0xb4, 0x09,
0x82, 0xe4, 0xf6, 0x5c, 0x9d, 0x58, 0xe9, 0x41, 0x65, 0xdd, 0x73, 0x1f, 0x85, 0x1f, 0x9c, 0xfc, 0x65, 0xe1, 0xca, 0x1b, 0x0d, 0x23, 0x4d, 0xbc, 0xab, 0x4f, 0x71, 0x15, 0x24, 0xb7, 0xe7, 0xea,
0x9c, 0x36, 0xfa, 0xcb, 0x2d, 0xa8, 0x31, 0x04, 0x1d, 0x3f, 0x0a, 0x74, 0x77, 0xf3, 0x63, 0x74, 0xc4, 0x4a, 0x0f, 0x2a, 0xeb, 0x9e, 0xfb, 0x28, 0xfc, 0x28, 0xe5, 0xe7, 0xb4, 0xd1, 0x5f, 0x6e,
0x72, 0x99, 0xfd, 0x1b, 0x87, 0xe5, 0xf0, 0xdf, 0x38, 0x2c, 0xdf, 0xb0, 0x6c, 0x7c, 0x8f, 0x67, 0x41, 0x8d, 0x21, 0xe8, 0xf8, 0x51, 0xa0, 0xbb, 0x9b, 0x1f, 0xa3, 0x93, 0xcb, 0xec, 0x5f, 0x3d,
0x41, 0xfe, 0x47, 0x61, 0xc8, 0xab, 0xbc, 0xe8, 0xca, 0x5a, 0xe3, 0xff, 0x49, 0xe2, 0x9d, 0x47, 0x2c, 0x87, 0xff, 0xea, 0x61, 0xf9, 0x86, 0x65, 0xe3, 0x7b, 0x3c, 0x0b, 0xf2, 0x3f, 0x0a, 0x43,
0xc1, 0xbd, 0xcd, 0x8f, 0xaf, 0x19, 0x9f, 0x5d, 0x29, 0x40, 0x6e, 0x65, 0xf9, 0xe5, 0xe5, 0x97, 0x5e, 0xee, 0x45, 0x57, 0xd6, 0x1a, 0xff, 0x6f, 0x13, 0xef, 0x3c, 0x0a, 0xee, 0x6d, 0x7e, 0x7c,
0xa0, 0x66, 0x45, 0xe8, 0x6d, 0xaf, 0xdb, 0xba, 0x56, 0x66, 0x8d, 0xd6, 0x49, 0x3f, 0xeb, 0xca, 0xcd, 0xf8, 0xec, 0x4a, 0x01, 0x72, 0x2b, 0xcb, 0x2f, 0x2f, 0xbf, 0x04, 0x35, 0x2b, 0x42, 0x6f,
0xcf, 0x5e, 0x6c, 0x5b, 0xc1, 0x76, 0x6f, 0x93, 0x2c, 0xc1, 0x05, 0x86, 0xf6, 0xa2, 0xe5, 0xf2, 0x7b, 0xdd, 0xd6, 0xb5, 0x32, 0x6b, 0xb4, 0x4e, 0xfa, 0x59, 0x57, 0x7e, 0xf6, 0x62, 0xdb, 0x0a,
0x5f, 0x17, 0x2c, 0x27, 0xc0, 0x9e, 0x63, 0xd8, 0xec, 0x3f, 0x4c, 0x70, 0x68, 0x77, 0xf3, 0xf7, 0xb6, 0x7b, 0x9b, 0x64, 0x09, 0x2e, 0x30, 0xb4, 0x17, 0x2d, 0x97, 0xff, 0xba, 0x60, 0x39, 0x01,
0x14, 0x65, 0x33, 0x4f, 0x41, 0x17, 0xff, 0x3f, 0x00, 0x00, 0xff, 0xff, 0xff, 0xb6, 0x47, 0x70, 0xf6, 0x1c, 0xc3, 0x66, 0xff, 0x85, 0x82, 0x43, 0xbb, 0x9b, 0xbf, 0xab, 0x28, 0x9b, 0x79, 0x0a,
0xc3, 0x62, 0x00, 0x00, 0xba, 0xf8, 0xff, 0x01, 0x00, 0x00, 0xff, 0xff, 0xd9, 0xc9, 0x1c, 0xd6, 0xe7, 0x62, 0x00, 0x00,
} }
// Reference imports to suppress errors if they are not otherwise used. // Reference imports to suppress errors if they are not otherwise used.

View File

@ -64,12 +64,15 @@ type DataNode interface {
// Log an info log if a segment is under flushing // Log an info log if a segment is under flushing
FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error)
// ShowConfigurations gets specified configurations para of DataNode // ShowConfigurations gets specified configurations param of DataNode
ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)
// GetMetrics gets the metrics about DataNode. // GetMetrics gets the metrics about DataNode.
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
// Compaction will add a compaction task according to the request plan // Compaction will add a compaction task according to the request plan
Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error)
// GetCompactionState get states of all compation tasks
GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error)
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments // Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
// //
@ -249,8 +252,6 @@ type DataCoord interface {
ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)
// GetMetrics gets the metrics about DataCoord. // GetMetrics gets the metrics about DataCoord.
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
// CompleteCompaction completes a compaction with the result
CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error)
// ManualCompaction triggers a compaction for a collection // ManualCompaction triggers a compaction for a collection
ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error)
// GetCompactionState gets the state of a compaction // GetCompactionState gets the state of a compaction

View File

@ -61,6 +61,10 @@ func (m *GrpcDataNodeClient) Compaction(ctx context.Context, req *datapb.Compact
return &commonpb.Status{}, m.Err return &commonpb.Status{}, m.Err
} }
func (m *GrpcDataNodeClient) GetCompactionState(ctx context.Context, in *datapb.CompactionStateRequest, opts ...grpc.CallOption) (*datapb.CompactionStateResponse, error) {
return &datapb.CompactionStateResponse{}, m.Err
}
func (m *GrpcDataNodeClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { func (m *GrpcDataNodeClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err return &commonpb.Status{}, m.Err
} }

View File

@ -928,6 +928,7 @@ type dataCoordConfig struct {
MaxSegmentToMerge int MaxSegmentToMerge int
SegmentSmallProportion float64 SegmentSmallProportion float64
CompactionTimeoutInSeconds int32 CompactionTimeoutInSeconds int32
CompactionCheckIntervalInSeconds int64
SingleCompactionRatioThreshold float32 SingleCompactionRatioThreshold float32
SingleCompactionDeltaLogMaxSize int64 SingleCompactionDeltaLogMaxSize int64
SingleCompactionExpiredLogMaxSize int64 SingleCompactionExpiredLogMaxSize int64
@ -957,6 +958,7 @@ func (p *dataCoordConfig) init(base *BaseTable) {
p.initCompactionMaxSegment() p.initCompactionMaxSegment()
p.initSegmentSmallProportion() p.initSegmentSmallProportion()
p.initCompactionTimeoutInSeconds() p.initCompactionTimeoutInSeconds()
p.initCompactionCheckIntervalInSeconds()
p.initSingleCompactionRatioThreshold() p.initSingleCompactionRatioThreshold()
p.initSingleCompactionDeltaLogMaxSize() p.initSingleCompactionDeltaLogMaxSize()
p.initSingleCompactionExpiredLogMaxSize() p.initSingleCompactionExpiredLogMaxSize()
@ -1016,6 +1018,10 @@ func (p *dataCoordConfig) initCompactionTimeoutInSeconds() {
p.CompactionTimeoutInSeconds = p.Base.ParseInt32WithDefault("dataCoord.compaction.timeout", 60*3) p.CompactionTimeoutInSeconds = p.Base.ParseInt32WithDefault("dataCoord.compaction.timeout", 60*3)
} }
func (p *dataCoordConfig) initCompactionCheckIntervalInSeconds() {
p.CompactionCheckIntervalInSeconds = p.Base.ParseInt64WithDefault("dataCoord.compaction.check.interval", 10)
}
// if total delete entities is large than a ratio of total entities, trigger single compaction. // if total delete entities is large than a ratio of total entities, trigger single compaction.
func (p *dataCoordConfig) initSingleCompactionRatioThreshold() { func (p *dataCoordConfig) initSingleCompactionRatioThreshold() {
p.SingleCompactionRatioThreshold = float32(p.Base.ParseFloatWithDefault("dataCoord.compaction.single.ratio.threshold", 0.2)) p.SingleCompactionRatioThreshold = float32(p.Base.ParseFloatWithDefault("dataCoord.compaction.single.ratio.threshold", 0.2))

View File

@ -838,6 +838,10 @@ class TestCompactionOperation(TestcaseBase):
# create collection shard_num=1, insert 11 segments, each with one entity # create collection shard_num=1, insert 11 segments, each with one entity
collection_w = self.collection_insert_multi_segments_one_shard(prefix, num_of_segment=num_of_segment) collection_w = self.collection_insert_multi_segments_one_shard(prefix, num_of_segment=num_of_segment)
# waiting for auto compaction finished
collection_w.collection.compaction_id = 0
collection_w.wait_for_compaction_completed()
collection_w.compact() collection_w.compact()
collection_w.wait_for_compaction_completed() collection_w.wait_for_compaction_completed()
c_plans, _ = collection_w.get_compaction_plans(check_task=CheckTasks.check_merge_compact, check_items={"segment_num": 2}) c_plans, _ = collection_w.get_compaction_plans(check_task=CheckTasks.check_merge_compact, check_items={"segment_num": 2})