From 40d5c902b643480d94e126b658ca52fb753f1b10 Mon Sep 17 00:00:00 2001 From: XuanYang-cn Date: Tue, 14 Nov 2023 15:56:19 +0800 Subject: [PATCH] Enable getting multiple segments in plan result (#28350) Compaction plan result contained one segment for one plan. For l0 compaction would write to multiple segments, this PR expand the segments number in plan results and refactor some names for readibility. - Name refactory: - CompactionStateResult -> CompactionPlanResult - CompactionResult -> CompactionSegment See also: #27606 Signed-off-by: yangxuan --- internal/datacoord/compaction.go | 29 +- internal/datacoord/compaction_test.go | 98 +-- internal/datacoord/compaction_trigger_test.go | 2 +- internal/datacoord/meta.go | 17 +- internal/datacoord/meta_test.go | 15 +- internal/datacoord/server.go | 2 +- internal/datacoord/services.go | 5 +- internal/datacoord/session_manager.go | 8 +- internal/datacoord/util.go | 11 +- internal/datanode/compaction_executor.go | 34 +- internal/datanode/compaction_executor_test.go | 2 +- internal/datanode/compactor.go | 17 +- internal/datanode/compactor_test.go | 33 +- internal/datanode/mock_test.go | 2 +- internal/datanode/services.go | 19 +- internal/datanode/services_test.go | 9 +- .../distributed/datacoord/service_test.go | 2 +- internal/proto/data_coord.proto | 11 +- internal/proto/datapb/data_coord.pb.go | 754 +++++++++--------- internal/proxy/data_coord_mock_test.go | 2 +- internal/util/mock/datacoord_client.go | 135 ---- internal/util/mock/grpc_datacoord_client.go | 2 +- 22 files changed, 557 insertions(+), 652 deletions(-) delete mode 100644 internal/util/mock/datacoord_client.go diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index 35fc589c05..d8ce57046a 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -30,7 +30,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -77,7 +76,7 @@ type compactionTask struct { plan *datapb.CompactionPlan state compactionTaskState dataNodeID int64 - result *datapb.CompactionResult + result *datapb.CompactionPlanResult } func (t *compactionTask) shadowClone(opts ...compactionTaskOpt) *compactionTask { @@ -104,7 +103,6 @@ type compactionPlanHandler struct { allocator allocator quit chan struct{} wg sync.WaitGroup - flushCh chan UniqueID scheduler *scheduler } @@ -259,8 +257,7 @@ func (s *scheduler) getExecutingTaskNum() int { return int(s.taskNumber.Load()) } -func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta *meta, - allocator allocator, flush chan UniqueID, +func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta *meta, allocator allocator, ) *compactionPlanHandler { return &compactionPlanHandler{ plans: make(map[int64]*compactionTask), @@ -268,7 +265,6 @@ func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta meta: meta, sessions: sessions, allocator: allocator, - flushCh: flush, scheduler: newScheduler(), } } @@ -390,7 +386,7 @@ func (c *compactionPlanHandler) setSegmentsCompacting(plan *datapb.CompactionPla // 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.CompactionPlanResult) error { planID := result.PlanID if _, ok := c.plans[planID]; !ok { return fmt.Errorf("plan %d is not found", planID) @@ -412,17 +408,12 @@ func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionResu return errors.New("unknown compaction type") } c.plans[planID] = c.plans[planID].shadowClone(setState(completed), setResult(result)) - if c.plans[planID].plan.GetType() == datapb.CompactionType_MergeCompaction || - c.plans[planID].plan.GetType() == datapb.CompactionType_MixCompaction { - c.flushCh <- result.GetSegmentID() - } // TODO: when to clean task list - - metrics.DataCoordCompactedSegmentSize.WithLabelValues().Observe(float64(getCompactedSegmentSize(result))) + UpdateCompactionSegmentSizeMetrics(result.GetSegments()) return nil } -func (c *compactionPlanHandler) handleMergeCompactionResult(plan *datapb.CompactionPlan, result *datapb.CompactionResult) error { +func (c *compactionPlanHandler) handleMergeCompactionResult(plan *datapb.CompactionPlan, result *datapb.CompactionPlanResult) error { // Also prepare metric updates. _, modSegments, newSegment, metricMutation, err := c.meta.PrepareCompleteCompactionMutation(plan, result) if err != nil { @@ -474,19 +465,19 @@ func (c *compactionPlanHandler) updateCompaction(ts Timestamp) error { // for DC might add new task while GetCompactionState. executingTasks := c.getTasksByState(executing) timeoutTasks := c.getTasksByState(timeout) - planStates := c.sessions.GetCompactionState() + planStates := c.sessions.GetCompactionPlansResults() c.mu.Lock() defer c.mu.Unlock() for _, task := range executingTasks { - stateResult, ok := planStates[task.plan.PlanID] - state := stateResult.GetState() + planResult, ok := planStates[task.plan.PlanID] + state := planResult.GetState() planID := task.plan.PlanID // check whether the state of CompactionPlan is working if ok { if state == commonpb.CompactionState_Completed { log.Info("complete compaction", zap.Int64("planID", planID), zap.Int64("nodeID", task.dataNodeID)) - err := c.completeCompaction(stateResult.GetResult()) + err := c.completeCompaction(planResult) if err != nil { log.Warn("fail to complete compaction", zap.Int64("planID", planID), zap.Int64("nodeID", task.dataNodeID), zap.Error(err)) } @@ -594,7 +585,7 @@ func setStartTime(startTime uint64) compactionTaskOpt { } } -func setResult(result *datapb.CompactionResult) compactionTaskOpt { +func setResult(result *datapb.CompactionPlanResult) compactionTaskOpt { return func(task *compactionTask) { task.result = result } diff --git a/internal/datacoord/compaction_test.go b/internal/datacoord/compaction_test.go index 665b9589c5..92b9d90640 100644 --- a/internal/datacoord/compaction_test.go +++ b/internal/datacoord/compaction_test.go @@ -494,22 +494,30 @@ func TestCompactionPlanHandler_handleMergeCompactionResult(t *testing.T) { meta: errMeta, } - compactionResult := &datapb.CompactionResult{ - PlanID: 1, - SegmentID: 3, - NumOfRows: 15, - InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, - Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, - Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + compactionResult := &datapb.CompactionPlanResult{ + PlanID: 1, + Segments: []*datapb.CompactionSegment{ + { + SegmentID: 3, + NumOfRows: 15, + InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, + Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, + Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + }, + }, } - compactionResult2 := &datapb.CompactionResult{ - PlanID: 1, - SegmentID: 3, - NumOfRows: 0, - InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, - Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, - Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + compactionResult2 := &datapb.CompactionPlanResult{ + PlanID: 1, + Segments: []*datapb.CompactionSegment{ + { + SegmentID: 3, + NumOfRows: 0, + InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, + Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, + Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + }, + }, } has, err := c.meta.HasSegments([]UniqueID{1, 2}) @@ -545,14 +553,14 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) { c := &compactionPlanHandler{ plans: map[int64]*compactionTask{1: {}}, } - err := c.completeCompaction(&datapb.CompactionResult{PlanID: 2}) + err := c.completeCompaction(&datapb.CompactionPlanResult{PlanID: 2}) assert.Error(t, err) }) t.Run("test completed compaction task", func(t *testing.T) { c := &compactionPlanHandler{ plans: map[int64]*compactionTask{1: {state: completed}}, } - err := c.completeCompaction(&datapb.CompactionResult{PlanID: 1}) + err := c.completeCompaction(&datapb.CompactionPlanResult{PlanID: 1}) assert.Error(t, err) }) @@ -626,29 +634,27 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) { }, }, } - compactionResult := datapb.CompactionResult{ - PlanID: 1, - SegmentID: 3, - NumOfRows: 15, - InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, - Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, - Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + compactionResult := datapb.CompactionPlanResult{ + PlanID: 1, + Segments: []*datapb.CompactionSegment{ + { + SegmentID: 3, + NumOfRows: 15, + InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, + Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, + Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + }, + }, } - flushCh := make(chan UniqueID, 1) c := &compactionPlanHandler{ plans: plans, sessions: sessions, meta: meta, - flushCh: flushCh, scheduler: newScheduler(), } err := c.completeCompaction(&compactionResult) - - segID, ok := <-flushCh - assert.True(t, ok) - assert.Equal(t, compactionResult.GetSegmentID(), segID) assert.NoError(t, err) }) @@ -728,32 +734,29 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) { segments := meta.GetAllSegmentsUnsafe() assert.Equal(t, len(segments), 2) - compactionResult := datapb.CompactionResult{ - PlanID: 1, - SegmentID: 3, - NumOfRows: 0, - InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, - Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, - Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + compactionResult := datapb.CompactionPlanResult{ + PlanID: 1, + Segments: []*datapb.CompactionSegment{ + { + SegmentID: 3, + NumOfRows: 0, + InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))}, + Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))}, + Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))}, + }, + }, } - flushCh := make(chan UniqueID, 1) c := &compactionPlanHandler{ plans: plans, sessions: sessions, meta: meta, - flushCh: flushCh, scheduler: newScheduler(), } err := c.completeCompaction(&compactionResult) assert.NoError(t, err) - segID, ok := <-flushCh - assert.True(t, ok) - assert.Equal(t, compactionResult.GetSegmentID(), segID) - assert.NoError(t, err) - segments = meta.GetAllSegmentsUnsafe() assert.Equal(t, len(segments), 3) @@ -905,9 +908,9 @@ func Test_compactionPlanHandler_updateCompaction(t *testing.T) { data: map[int64]*Session{ 2: {client: &mockDataNodeClient{ compactionStateResp: &datapb.CompactionStateResponse{ - Results: []*datapb.CompactionStateResult{ + Results: []*datapb.CompactionPlanResult{ {PlanID: 1, State: commonpb.CompactionState_Executing}, - {PlanID: 3, State: commonpb.CompactionState_Completed, Result: &datapb.CompactionResult{PlanID: 3}}, + {PlanID: 3, State: commonpb.CompactionState_Completed, Segments: []*datapb.CompactionSegment{{PlanID: 3}}}, {PlanID: 4, State: commonpb.CompactionState_Executing}, {PlanID: 6, State: commonpb.CompactionState_Executing}, }, @@ -964,7 +967,6 @@ func Test_newCompactionPlanHandler(t *testing.T) { cm *ChannelManager meta *meta allocator allocator - flush chan UniqueID } tests := []struct { name string @@ -978,7 +980,6 @@ func Test_newCompactionPlanHandler(t *testing.T) { &ChannelManager{}, &meta{}, newMockAllocator(), - nil, }, &compactionPlanHandler{ plans: map[int64]*compactionTask{}, @@ -986,14 +987,13 @@ func Test_newCompactionPlanHandler(t *testing.T) { chManager: &ChannelManager{}, meta: &meta{}, allocator: newMockAllocator(), - flushCh: nil, scheduler: newScheduler(), }, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got := newCompactionPlanHandler(tt.args.sessions, tt.args.cm, tt.args.meta, tt.args.allocator, tt.args.flush) + got := newCompactionPlanHandler(tt.args.sessions, tt.args.cm, tt.args.meta, tt.args.allocator) assert.EqualValues(t, tt.want, got) }) } diff --git a/internal/datacoord/compaction_trigger_test.go b/internal/datacoord/compaction_trigger_test.go index 9fb8f74879..c815b76be5 100644 --- a/internal/datacoord/compaction_trigger_test.go +++ b/internal/datacoord/compaction_trigger_test.go @@ -45,7 +45,7 @@ func (h *spyCompactionHandler) execCompactionPlan(signal *compactionSignal, plan } // completeCompaction record the result of a compaction -func (h *spyCompactionHandler) completeCompaction(result *datapb.CompactionResult) error { +func (h *spyCompactionHandler) completeCompaction(result *datapb.CompactionPlanResult) error { return nil } diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 3933ac30f6..21aaf3a6fa 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -997,7 +997,7 @@ func (m *meta) SetSegmentCompacting(segmentID UniqueID, compacting bool) { // - the segment info of compactedTo segment after compaction to add // The compactedTo segment could contain 0 numRows func (m *meta) PrepareCompleteCompactionMutation(plan *datapb.CompactionPlan, - result *datapb.CompactionResult, + result *datapb.CompactionPlanResult, ) ([]*SegmentInfo, []*SegmentInfo, *SegmentInfo, *segMetricMutation, error) { log.Info("meta update: prepare for complete compaction mutation") compactionLogs := plan.GetSegmentBinlogs() @@ -1048,12 +1048,15 @@ func (m *meta) PrepareCompleteCompactionMutation(plan *datapb.CompactionPlan, deletedDeltalogs = append(deletedDeltalogs, l.GetDeltalogs()...) } + // MixCompaction / MergeCompaction will generates one and only one segment + compactToSegment := result.GetSegments()[0] + newAddedDeltalogs := m.updateDeltalogs(originDeltalogs, deletedDeltalogs, nil) - copiedDeltalogs, err := m.copyDeltaFiles(newAddedDeltalogs, modSegments[0].CollectionID, modSegments[0].PartitionID, result.GetSegmentID()) + copiedDeltalogs, err := m.copyDeltaFiles(newAddedDeltalogs, modSegments[0].CollectionID, modSegments[0].PartitionID, compactToSegment.GetSegmentID()) if err != nil { return nil, nil, nil, nil, err } - deltalogs := append(result.GetDeltalogs(), copiedDeltalogs...) + deltalogs := append(compactToSegment.GetDeltalogs(), copiedDeltalogs...) compactionFrom := make([]UniqueID, 0, len(modSegments)) for _, s := range modSegments { @@ -1061,15 +1064,15 @@ func (m *meta) PrepareCompleteCompactionMutation(plan *datapb.CompactionPlan, } segmentInfo := &datapb.SegmentInfo{ - ID: result.GetSegmentID(), + ID: compactToSegment.GetSegmentID(), CollectionID: modSegments[0].CollectionID, PartitionID: modSegments[0].PartitionID, InsertChannel: modSegments[0].InsertChannel, - NumOfRows: result.NumOfRows, + NumOfRows: compactToSegment.NumOfRows, State: commonpb.SegmentState_Flushing, MaxRowNum: modSegments[0].MaxRowNum, - Binlogs: result.GetInsertLogs(), - Statslogs: result.GetField2StatslogPaths(), + Binlogs: compactToSegment.GetInsertLogs(), + Statslogs: compactToSegment.GetField2StatslogPaths(), Deltalogs: deltalogs, StartPosition: startPosition, DmlPosition: dmlPosition, diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index 6de995530a..e2a0d30bf7 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -716,13 +716,16 @@ func TestMeta_PrepareCompleteCompactionMutation(t *testing.T) { StartTime: 15, } - inCompactionResult := &datapb.CompactionResult{ + inSegment := &datapb.CompactionSegment{ SegmentID: 3, InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(1, "log5")}, Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(1, "statlog5")}, Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(0, "deltalog5")}, NumOfRows: 2, } + inCompactionResult := &datapb.CompactionPlanResult{ + Segments: []*datapb.CompactionSegment{inSegment}, + } beforeCompact, afterCompact, newSegment, metricMutation, err := m.PrepareCompleteCompactionMutation(plan, inCompactionResult) assert.NoError(t, err) assert.NotNil(t, beforeCompact) @@ -744,15 +747,15 @@ func TestMeta_PrepareCompleteCompactionMutation(t *testing.T) { assert.NotZero(t, afterCompact[0].GetDroppedAt()) assert.NotZero(t, afterCompact[1].GetDroppedAt()) - assert.Equal(t, inCompactionResult.SegmentID, newSegment.GetID()) + assert.Equal(t, inSegment.SegmentID, newSegment.GetID()) assert.Equal(t, UniqueID(100), newSegment.GetCollectionID()) assert.Equal(t, UniqueID(10), newSegment.GetPartitionID()) - assert.Equal(t, inCompactionResult.NumOfRows, newSegment.GetNumOfRows()) + assert.Equal(t, inSegment.NumOfRows, newSegment.GetNumOfRows()) assert.Equal(t, commonpb.SegmentState_Flushing, newSegment.GetState()) - assert.EqualValues(t, inCompactionResult.GetInsertLogs(), newSegment.GetBinlogs()) - assert.EqualValues(t, inCompactionResult.GetField2StatslogPaths(), newSegment.GetStatslogs()) - assert.EqualValues(t, inCompactionResult.GetDeltalogs(), newSegment.GetDeltalogs()) + assert.EqualValues(t, inSegment.GetInsertLogs(), newSegment.GetBinlogs()) + assert.EqualValues(t, inSegment.GetField2StatslogPaths(), newSegment.GetStatslogs()) + assert.EqualValues(t, inSegment.GetDeltalogs(), newSegment.GetDeltalogs()) assert.NotZero(t, newSegment.lastFlushTime) assert.Equal(t, uint64(15), newSegment.GetLastExpireTime()) } diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index ece56ed348..24f7f19406 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -446,7 +446,7 @@ func (s *Server) SetIndexNodeCreator(f func(context.Context, string, int64) (typ } func (s *Server) createCompactionHandler() { - s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh) + s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator) } func (s *Server) stopCompactionHandler() { diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index b324376ad6..4ee5775486 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -1127,7 +1127,10 @@ func getCompactionMergeInfo(task *compactionTask) *milvuspb.CompactionMergeInfo var target int64 = -1 if task.result != nil { - target = task.result.GetSegmentID() + segments := task.result.GetSegments() + if len(segments) > 0 { + target = segments[0].GetSegmentID() + } } return &milvuspb.CompactionMergeInfo{ diff --git a/internal/datacoord/session_manager.go b/internal/datacoord/session_manager.go index 7d6485a9e7..3c307cce29 100644 --- a/internal/datacoord/session_manager.go +++ b/internal/datacoord/session_manager.go @@ -227,11 +227,11 @@ func (c *SessionManager) execImport(ctx context.Context, nodeID int64, itr *data log.Info("success to import", zap.Int64("node", nodeID), zap.Any("import task", itr)) } -func (c *SessionManager) GetCompactionState() map[int64]*datapb.CompactionStateResult { +func (c *SessionManager) GetCompactionPlansResults() map[int64]*datapb.CompactionPlanResult { wg := sync.WaitGroup{} ctx := context.Background() - plans := typeutil.NewConcurrentMap[int64, *datapb.CompactionStateResult]() + plans := typeutil.NewConcurrentMap[int64, *datapb.CompactionPlanResult]() c.sessions.RLock() for nodeID, s := range c.sessions.data { wg.Add(1) @@ -264,8 +264,8 @@ func (c *SessionManager) GetCompactionState() map[int64]*datapb.CompactionStateR c.sessions.RUnlock() wg.Wait() - rst := make(map[int64]*datapb.CompactionStateResult) - plans.Range(func(planID int64, result *datapb.CompactionStateResult) bool { + rst := make(map[int64]*datapb.CompactionPlanResult) + plans.Range(func(planID int64, result *datapb.CompactionPlanResult) bool { rst[planID] = result return true }) diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index 901d770109..5e2a4a10ca 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util/merr" ) @@ -129,9 +130,15 @@ func getCollectionTTL(properties map[string]string) (time.Duration, error) { return Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second), nil } -func getCompactedSegmentSize(s *datapb.CompactionResult) int64 { - var segmentSize int64 +func UpdateCompactionSegmentSizeMetrics(segments []*datapb.CompactionSegment) { + for _, seg := range segments { + size := getCompactedSegmentSize(seg) + metrics.DataCoordCompactedSegmentSize.WithLabelValues().Observe(float64(size)) + } +} +func getCompactedSegmentSize(s *datapb.CompactionSegment) int64 { + var segmentSize int64 if s != nil { for _, binlogs := range s.GetInsertLogs() { for _, l := range binlogs.GetBinlogs() { diff --git a/internal/datanode/compaction_executor.go b/internal/datanode/compaction_executor.go index 272edef586..ed36a5d93b 100644 --- a/internal/datanode/compaction_executor.go +++ b/internal/datanode/compaction_executor.go @@ -21,6 +21,7 @@ import ( "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -31,9 +32,9 @@ const ( ) type compactionExecutor struct { - executing *typeutil.ConcurrentMap[int64, compactor] // planID to compactor - completedCompactor *typeutil.ConcurrentMap[int64, compactor] // planID to compactor - completed *typeutil.ConcurrentMap[int64, *datapb.CompactionResult] // planID to CompactionResult + executing *typeutil.ConcurrentMap[int64, compactor] // planID to compactor + completedCompactor *typeutil.ConcurrentMap[int64, compactor] // planID to compactor + completed *typeutil.ConcurrentMap[int64, *datapb.CompactionPlanResult] // planID to CompactionPlanResult taskCh chan compactor dropped *typeutil.ConcurrentSet[string] // vchannel dropped } @@ -42,7 +43,7 @@ func newCompactionExecutor() *compactionExecutor { return &compactionExecutor{ executing: typeutil.NewConcurrentMap[int64, compactor](), completedCompactor: typeutil.NewConcurrentMap[int64, compactor](), - completed: typeutil.NewConcurrentMap[int64, *datapb.CompactionResult](), + completed: typeutil.NewConcurrentMap[int64, *datapb.CompactionPlanResult](), taskCh: make(chan compactor, maxTaskNum), dropped: typeutil.NewConcurrentSet[string](), } @@ -100,8 +101,8 @@ func (c *compactionExecutor) executeTask(task compactor) { zap.Error(err), ) } else { - c.completed.Insert(task.getPlanID(), result) - c.completedCompactor.Insert(task.getPlanID(), task) + c.completed.Insert(result.GetPlanID(), result) + c.completedCompactor.Insert(result.GetPlanID(), task) } log.Info("end to execute compaction", zap.Int64("planID", task.getPlanID())) @@ -128,8 +129,9 @@ func (c *compactionExecutor) stopExecutingtaskByVChannelName(vChannelName string } return true }) + // remove all completed plans for vChannelName - c.completed.Range(func(planID int64, result *datapb.CompactionResult) bool { + c.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool { if result.GetChannel() == vChannelName { c.injectDone(planID, true) log.Info("remove compaction results for dropped channel", @@ -139,3 +141,21 @@ func (c *compactionExecutor) stopExecutingtaskByVChannelName(vChannelName string return true }) } + +func (c *compactionExecutor) getAllCompactionPlanResult() []*datapb.CompactionPlanResult { + results := make([]*datapb.CompactionPlanResult, 0) + c.executing.Range(func(planID int64, task compactor) bool { + results = append(results, &datapb.CompactionPlanResult{ + State: commonpb.CompactionState_Executing, + PlanID: planID, + }) + return true + }) + + c.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool { + results = append(results, result) + return true + }) + + return results +} diff --git a/internal/datanode/compaction_executor_test.go b/internal/datanode/compaction_executor_test.go index 107eddcd16..cbf7b674a1 100644 --- a/internal/datanode/compaction_executor_test.go +++ b/internal/datanode/compaction_executor_test.go @@ -145,7 +145,7 @@ func (mc *mockCompactor) complete() { func (mc *mockCompactor) injectDone(success bool) { } -func (mc *mockCompactor) compact() (*datapb.CompactionResult, error) { +func (mc *mockCompactor) compact() (*datapb.CompactionPlanResult, error) { if !mc.isvalid { return nil, errStart } diff --git a/internal/datanode/compactor.go b/internal/datanode/compactor.go index 892f626682..6a029c02db 100644 --- a/internal/datanode/compactor.go +++ b/internal/datanode/compactor.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -56,7 +57,8 @@ type iterator = storage.Iterator type compactor interface { complete() - compact() (*datapb.CompactionResult, error) + // compact() (*datapb.CompactionResult, error) + compact() (*datapb.CompactionPlanResult, error) injectDone(success bool) stop() getPlanID() UniqueID @@ -477,7 +479,7 @@ func (t *compactionTask) merge( return insertPaths, statPaths, numRows, nil } -func (t *compactionTask) compact() (*datapb.CompactionResult, error) { +func (t *compactionTask) compact() (*datapb.CompactionPlanResult, error) { log := log.With(zap.Int64("planID", t.plan.GetPlanID())) compactStart := time.Now() if ok := funcutil.CheckCtxValid(t.ctx); !ok { @@ -689,8 +691,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) { return nil, err } - pack := &datapb.CompactionResult{ - PlanID: t.plan.GetPlanID(), + pack := &datapb.CompactionSegment{ SegmentID: targetSegID, InsertLogs: inPaths, Field2StatslogPaths: statsPaths, @@ -712,7 +713,13 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) { metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(t.tr.ElapseSpan().Milliseconds())) metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds())) - return pack, nil + planResult := &datapb.CompactionPlanResult{ + State: commonpb.CompactionState_Completed, + PlanID: t.getPlanID(), + Segments: []*datapb.CompactionSegment{pack}, + } + + return planResult, nil } func (t *compactionTask) injectDone(success bool) { diff --git a/internal/datanode/compactor_test.go b/internal/datanode/compactor_test.go index 06d6833801..0208679544 100644 --- a/internal/datanode/compactor_test.go +++ b/internal/datanode/compactor_test.go @@ -888,10 +888,13 @@ func TestCompactorInterfaceMethods(t *testing.T) { assert.NotNil(t, result) assert.Equal(t, plan.GetPlanID(), result.GetPlanID()) - assert.Equal(t, UniqueID(19530), result.GetSegmentID()) - assert.Equal(t, int64(2), result.GetNumOfRows()) - assert.NotEmpty(t, result.InsertLogs) - assert.NotEmpty(t, result.Field2StatslogPaths) + assert.Equal(t, 1, len(result.GetSegments())) + + segment := result.GetSegments()[0] + assert.EqualValues(t, 19530, segment.GetSegmentID()) + assert.EqualValues(t, 2, segment.GetNumOfRows()) + assert.NotEmpty(t, segment.InsertLogs) + assert.NotEmpty(t, segment.Field2StatslogPaths) // New test, remove all the binlogs in memkv err = mockKv.RemoveWithPrefix("/") @@ -910,10 +913,13 @@ func TestCompactorInterfaceMethods(t *testing.T) { assert.NotNil(t, result) assert.Equal(t, plan.GetPlanID(), result.GetPlanID()) - assert.Equal(t, UniqueID(19530), result.GetSegmentID()) - assert.Equal(t, int64(2), result.GetNumOfRows()) - assert.NotEmpty(t, result.InsertLogs) - assert.NotEmpty(t, result.Field2StatslogPaths) + assert.Equal(t, 1, len(result.GetSegments())) + + segment = result.GetSegments()[0] + assert.EqualValues(t, 19530, segment.GetSegmentID()) + assert.EqualValues(t, 2, segment.GetNumOfRows()) + assert.NotEmpty(t, segment.InsertLogs) + assert.NotEmpty(t, segment.Field2StatslogPaths) assert.Equal(t, 0, mockfm.injectCount()) task.injectDone(true) @@ -1012,10 +1018,13 @@ func TestCompactorInterfaceMethods(t *testing.T) { assert.NotNil(t, result) assert.Equal(t, plan.GetPlanID(), result.GetPlanID()) - assert.Equal(t, UniqueID(19530), result.GetSegmentID()) - assert.Equal(t, int64(2), result.GetNumOfRows()) - assert.NotEmpty(t, result.InsertLogs) - assert.NotEmpty(t, result.Field2StatslogPaths) + assert.Equal(t, 1, len(result.GetSegments())) + + segment := result.GetSegments()[0] + assert.EqualValues(t, 19530, segment.GetSegmentID()) + assert.EqualValues(t, 2, segment.GetNumOfRows()) + assert.NotEmpty(t, segment.InsertLogs) + assert.NotEmpty(t, segment.Field2StatslogPaths) assert.Equal(t, 0, mockfm.injectCount()) task.injectDone(true) diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index ce65b951cd..d678dfb262 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -219,7 +219,7 @@ func (ds *DataCoordFactory) AssignSegmentID(ctx context.Context, req *datapb.Ass return res, nil } -func (ds *DataCoordFactory) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { +func (ds *DataCoordFactory) CompleteCompaction(ctx context.Context, req *datapb.CompactionPlanResult, opts ...grpc.CallOption) (*commonpb.Status, error) { if ds.CompleteCompactionError { return nil, errors.New("Error") } diff --git a/internal/datanode/services.go b/internal/datanode/services.go index 1502fdb666..871f6dfd2e 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -300,25 +300,10 @@ func (node *DataNode) GetCompactionState(ctx context.Context, req *datapb.Compac Status: merr.Status(err), }, nil } - results := make([]*datapb.CompactionStateResult, 0) - node.compactionExecutor.executing.Range(func(planID int64, task compactor) bool { - results = append(results, &datapb.CompactionStateResult{ - State: commonpb.CompactionState_Executing, - PlanID: planID, - }) - return true - }) - node.compactionExecutor.completed.Range(func(planID int64, result *datapb.CompactionResult) bool { - results = append(results, &datapb.CompactionStateResult{ - State: commonpb.CompactionState_Completed, - PlanID: planID, - Result: result, - }) - return true - }) + results := node.compactionExecutor.getAllCompactionPlanResult() if len(results) > 0 { - planIDs := lo.Map(results, func(result *datapb.CompactionStateResult, i int) UniqueID { + planIDs := lo.Map(results, func(result *datapb.CompactionPlanResult, i int) UniqueID { return result.GetPlanID() }) log.Info("Compaction results", zap.Int64s("planIDs", planIDs)) diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index fc5b5eadfc..36d595852f 100644 --- a/internal/datanode/services_test.go +++ b/internal/datanode/services_test.go @@ -179,9 +179,12 @@ func (s *DataNodeServicesSuite) TestGetCompactionState() { s.Run("success", func() { s.node.compactionExecutor.executing.Insert(int64(3), newMockCompactor(true)) s.node.compactionExecutor.executing.Insert(int64(2), newMockCompactor(true)) - s.node.compactionExecutor.completed.Insert(int64(1), &datapb.CompactionResult{ - PlanID: 1, - SegmentID: 10, + s.node.compactionExecutor.completed.Insert(int64(1), &datapb.CompactionPlanResult{ + PlanID: 1, + State: commonpb.CompactionState_Completed, + Segments: []*datapb.CompactionSegment{ + {SegmentID: 10}, + }, }) stat, err := s.node.GetCompactionState(s.ctx, nil) s.Assert().NoError(err) diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index 2188be1685..cf8429bfd1 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -188,7 +188,7 @@ func (m *MockDataCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetrics return m.metricResp, m.err } -func (m *MockDataCoord) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) { +func (m *MockDataCoord) CompleteCompaction(ctx context.Context, req *datapb.CompactionPlanResult) (*commonpb.Status, error) { return m.status, m.err } diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index d72eaa6655..146b272b61 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -509,8 +509,8 @@ message CompactionPlan { int64 total_rows = 9; } -message CompactionResult { - int64 planID = 1; +message CompactionSegment { + int64 planID = 1; // deprecated after 2.3.4 int64 segmentID = 2; int64 num_of_rows = 3; repeated FieldBinlog insert_logs = 4; @@ -519,15 +519,16 @@ message CompactionResult { string channel = 7; } -message CompactionStateResult { +message CompactionPlanResult { int64 planID = 1; common.CompactionState state = 2; - CompactionResult result = 3; + repeated CompactionSegment segments = 3; + string channel = 4; } message CompactionStateResponse { common.Status status = 1; - repeated CompactionStateResult results = 2; + repeated CompactionPlanResult results = 2; } // Deprecated diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index c06fbe625b..7b73bb34e4 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -3319,7 +3319,7 @@ func (m *CompactionPlan) GetTotalRows() int64 { return 0 } -type CompactionResult struct { +type CompactionSegment struct { PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"` SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` @@ -3332,141 +3332,149 @@ type CompactionResult struct { XXX_sizecache int32 `json:"-"` } -func (m *CompactionResult) Reset() { *m = CompactionResult{} } -func (m *CompactionResult) String() string { return proto.CompactTextString(m) } -func (*CompactionResult) ProtoMessage() {} -func (*CompactionResult) Descriptor() ([]byte, []int) { +func (m *CompactionSegment) Reset() { *m = CompactionSegment{} } +func (m *CompactionSegment) String() string { return proto.CompactTextString(m) } +func (*CompactionSegment) ProtoMessage() {} +func (*CompactionSegment) Descriptor() ([]byte, []int) { return fileDescriptor_82cd95f524594f49, []int{48} } -func (m *CompactionResult) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CompactionResult.Unmarshal(m, b) +func (m *CompactionSegment) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CompactionSegment.Unmarshal(m, b) } -func (m *CompactionResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CompactionResult.Marshal(b, m, deterministic) +func (m *CompactionSegment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CompactionSegment.Marshal(b, m, deterministic) } -func (m *CompactionResult) XXX_Merge(src proto.Message) { - xxx_messageInfo_CompactionResult.Merge(m, src) +func (m *CompactionSegment) XXX_Merge(src proto.Message) { + xxx_messageInfo_CompactionSegment.Merge(m, src) } -func (m *CompactionResult) XXX_Size() int { - return xxx_messageInfo_CompactionResult.Size(m) +func (m *CompactionSegment) XXX_Size() int { + return xxx_messageInfo_CompactionSegment.Size(m) } -func (m *CompactionResult) XXX_DiscardUnknown() { - xxx_messageInfo_CompactionResult.DiscardUnknown(m) +func (m *CompactionSegment) XXX_DiscardUnknown() { + xxx_messageInfo_CompactionSegment.DiscardUnknown(m) } -var xxx_messageInfo_CompactionResult proto.InternalMessageInfo +var xxx_messageInfo_CompactionSegment proto.InternalMessageInfo -func (m *CompactionResult) GetPlanID() int64 { +func (m *CompactionSegment) GetPlanID() int64 { if m != nil { return m.PlanID } return 0 } -func (m *CompactionResult) GetSegmentID() int64 { +func (m *CompactionSegment) GetSegmentID() int64 { if m != nil { return m.SegmentID } return 0 } -func (m *CompactionResult) GetNumOfRows() int64 { +func (m *CompactionSegment) GetNumOfRows() int64 { if m != nil { return m.NumOfRows } return 0 } -func (m *CompactionResult) GetInsertLogs() []*FieldBinlog { +func (m *CompactionSegment) GetInsertLogs() []*FieldBinlog { if m != nil { return m.InsertLogs } return nil } -func (m *CompactionResult) GetField2StatslogPaths() []*FieldBinlog { +func (m *CompactionSegment) GetField2StatslogPaths() []*FieldBinlog { if m != nil { return m.Field2StatslogPaths } return nil } -func (m *CompactionResult) GetDeltalogs() []*FieldBinlog { +func (m *CompactionSegment) GetDeltalogs() []*FieldBinlog { if m != nil { return m.Deltalogs } return nil } -func (m *CompactionResult) GetChannel() string { +func (m *CompactionSegment) GetChannel() string { if m != nil { return m.Channel } return "" } -type CompactionStateResult struct { +type CompactionPlanResult 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"` + Segments []*CompactionSegment `protobuf:"bytes,3,rep,name=segments,proto3" json:"segments,omitempty"` + Channel string `protobuf:"bytes,4,opt,name=channel,proto3" json:"channel,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) { +func (m *CompactionPlanResult) Reset() { *m = CompactionPlanResult{} } +func (m *CompactionPlanResult) String() string { return proto.CompactTextString(m) } +func (*CompactionPlanResult) ProtoMessage() {} +func (*CompactionPlanResult) Descriptor() ([]byte, []int) { return fileDescriptor_82cd95f524594f49, []int{49} } -func (m *CompactionStateResult) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CompactionStateResult.Unmarshal(m, b) +func (m *CompactionPlanResult) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CompactionPlanResult.Unmarshal(m, b) } -func (m *CompactionStateResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CompactionStateResult.Marshal(b, m, deterministic) +func (m *CompactionPlanResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CompactionPlanResult.Marshal(b, m, deterministic) } -func (m *CompactionStateResult) XXX_Merge(src proto.Message) { - xxx_messageInfo_CompactionStateResult.Merge(m, src) +func (m *CompactionPlanResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_CompactionPlanResult.Merge(m, src) } -func (m *CompactionStateResult) XXX_Size() int { - return xxx_messageInfo_CompactionStateResult.Size(m) +func (m *CompactionPlanResult) XXX_Size() int { + return xxx_messageInfo_CompactionPlanResult.Size(m) } -func (m *CompactionStateResult) XXX_DiscardUnknown() { - xxx_messageInfo_CompactionStateResult.DiscardUnknown(m) +func (m *CompactionPlanResult) XXX_DiscardUnknown() { + xxx_messageInfo_CompactionPlanResult.DiscardUnknown(m) } -var xxx_messageInfo_CompactionStateResult proto.InternalMessageInfo +var xxx_messageInfo_CompactionPlanResult proto.InternalMessageInfo -func (m *CompactionStateResult) GetPlanID() int64 { +func (m *CompactionPlanResult) GetPlanID() int64 { if m != nil { return m.PlanID } return 0 } -func (m *CompactionStateResult) GetState() commonpb.CompactionState { +func (m *CompactionPlanResult) GetState() commonpb.CompactionState { if m != nil { return m.State } return commonpb.CompactionState_UndefiedState } -func (m *CompactionStateResult) GetResult() *CompactionResult { +func (m *CompactionPlanResult) GetSegments() []*CompactionSegment { if m != nil { - return m.Result + return m.Segments } return nil } +func (m *CompactionPlanResult) GetChannel() string { + if m != nil { + return m.Channel + } + return "" +} + 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:"-"` + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Results []*CompactionPlanResult `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{} } @@ -3501,7 +3509,7 @@ func (m *CompactionStateResponse) GetStatus() *commonpb.Status { return nil } -func (m *CompactionStateResponse) GetResults() []*CompactionStateResult { +func (m *CompactionStateResponse) GetResults() []*CompactionPlanResult { if m != nil { return m.Results } @@ -5390,8 +5398,8 @@ func init() { proto.RegisterType((*SyncSegmentsRequest)(nil), "milvus.proto.data.SyncSegmentsRequest") proto.RegisterType((*CompactionSegmentBinlogs)(nil), "milvus.proto.data.CompactionSegmentBinlogs") proto.RegisterType((*CompactionPlan)(nil), "milvus.proto.data.CompactionPlan") - proto.RegisterType((*CompactionResult)(nil), "milvus.proto.data.CompactionResult") - proto.RegisterType((*CompactionStateResult)(nil), "milvus.proto.data.CompactionStateResult") + proto.RegisterType((*CompactionSegment)(nil), "milvus.proto.data.CompactionSegment") + proto.RegisterType((*CompactionPlanResult)(nil), "milvus.proto.data.CompactionPlanResult") proto.RegisterType((*CompactionStateResponse)(nil), "milvus.proto.data.CompactionStateResponse") proto.RegisterType((*SegmentFieldBinlogMeta)(nil), "milvus.proto.data.SegmentFieldBinlogMeta") proto.RegisterType((*WatchChannelsRequest)(nil), "milvus.proto.data.WatchChannelsRequest") @@ -5428,329 +5436,329 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 5149 bytes of a gzipped FileDescriptorProto + // 5148 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7c, 0xcb, 0x6f, 0x1c, 0x47, - 0x7a, 0xb8, 0x7a, 0x5e, 0x9c, 0xf9, 0x66, 0x38, 0x1c, 0x96, 0x68, 0x6a, 0x34, 0xb2, 0x1e, 0x6e, - 0x49, 0x36, 0xad, 0xb5, 0x25, 0x99, 0xfa, 0x19, 0xeb, 0xc7, 0xda, 0xbb, 0x22, 0x69, 0xc9, 0xf3, + 0x7a, 0xb8, 0x7a, 0x5e, 0x9c, 0xf9, 0x66, 0x38, 0x1c, 0x96, 0xb8, 0xd4, 0x68, 0x64, 0x3d, 0xdc, + 0x96, 0x2c, 0x5a, 0x6b, 0x4b, 0x32, 0xf5, 0x33, 0xd6, 0x8f, 0xb5, 0xd7, 0x22, 0x69, 0xc9, 0xf3, 0x0b, 0x29, 0x73, 0x9b, 0x94, 0x1c, 0x78, 0x03, 0x0c, 0x9a, 0xd3, 0xc5, 0x61, 0x9b, 0x3d, 0xdd, - 0xe3, 0xee, 0x1e, 0x52, 0x74, 0x80, 0xac, 0x37, 0x71, 0x82, 0xbc, 0x90, 0x04, 0x79, 0x1c, 0x72, - 0x0b, 0x72, 0x08, 0x36, 0x8f, 0x3d, 0x6d, 0x82, 0x05, 0x82, 0x00, 0x8b, 0x24, 0x97, 0x5d, 0xe4, - 0x90, 0x43, 0x2e, 0xc9, 0x21, 0x7f, 0x45, 0x90, 0x7b, 0x50, 0x8f, 0xae, 0x7e, 0x55, 0xcf, 0x34, - 0x39, 0x92, 0x0d, 0x24, 0x27, 0xb2, 0xab, 0xbf, 0xaa, 0xfa, 0xfa, 0xab, 0xef, 0xfd, 0x7d, 0x35, - 0xd0, 0x32, 0x74, 0x5f, 0xef, 0xf5, 0x1d, 0xc7, 0x35, 0x6e, 0x8f, 0x5c, 0xc7, 0x77, 0xd0, 0xe2, - 0xd0, 0xb4, 0x8e, 0xc6, 0x1e, 0x7b, 0xba, 0x4d, 0x5e, 0x77, 0x1a, 0x7d, 0x67, 0x38, 0x74, 0x6c, - 0x36, 0xd4, 0x69, 0x9a, 0xb6, 0x8f, 0x5d, 0x5b, 0xb7, 0xf8, 0x73, 0x23, 0x3a, 0xa1, 0xd3, 0xf0, - 0xfa, 0x07, 0x78, 0xa8, 0xf3, 0xa7, 0xda, 0xd0, 0x1b, 0xf0, 0x7f, 0x17, 0x4d, 0xdb, 0xc0, 0x4f, - 0xa3, 0x5b, 0xa9, 0x73, 0x50, 0xfe, 0x60, 0x38, 0xf2, 0x4f, 0xd4, 0xbf, 0x55, 0xa0, 0xf1, 0xc0, - 0x1a, 0x7b, 0x07, 0x1a, 0xfe, 0x6c, 0x8c, 0x3d, 0x1f, 0xdd, 0x85, 0xd2, 0x9e, 0xee, 0xe1, 0xb6, - 0x72, 0x4d, 0x59, 0xa9, 0xaf, 0xbe, 0x78, 0x3b, 0x86, 0x13, 0xc7, 0x66, 0xcb, 0x1b, 0xac, 0xe9, - 0x1e, 0xd6, 0x28, 0x24, 0x42, 0x50, 0x32, 0xf6, 0xba, 0x1b, 0xed, 0xc2, 0x35, 0x65, 0xa5, 0xa8, - 0xd1, 0xff, 0xd1, 0x15, 0x00, 0x0f, 0x0f, 0x86, 0xd8, 0xf6, 0xbb, 0x1b, 0x5e, 0xbb, 0x78, 0xad, - 0xb8, 0x52, 0xd4, 0x22, 0x23, 0x48, 0x85, 0x46, 0xdf, 0xb1, 0x2c, 0xdc, 0xf7, 0x4d, 0xc7, 0xee, - 0x6e, 0xb4, 0x4b, 0x74, 0x6e, 0x6c, 0x0c, 0x75, 0xa0, 0x6a, 0x7a, 0xdd, 0xe1, 0xc8, 0x71, 0xfd, - 0x76, 0xf9, 0x9a, 0xb2, 0x52, 0xd5, 0xc4, 0xb3, 0xfa, 0x83, 0x02, 0xcc, 0x73, 0xb4, 0xbd, 0x91, - 0x63, 0x7b, 0x18, 0xdd, 0x83, 0x8a, 0xe7, 0xeb, 0xfe, 0xd8, 0xe3, 0x98, 0x5f, 0x92, 0x62, 0xbe, - 0x43, 0x41, 0x34, 0x0e, 0x2a, 0x45, 0x3d, 0x89, 0x5a, 0x51, 0x82, 0x5a, 0xfc, 0xf3, 0x4a, 0xa9, - 0xcf, 0x5b, 0x81, 0x85, 0x7d, 0x82, 0xdd, 0x4e, 0x08, 0x54, 0xa6, 0x40, 0xc9, 0x61, 0xb2, 0x92, - 0x6f, 0x0e, 0xf1, 0x47, 0xfb, 0x3b, 0x58, 0xb7, 0xda, 0x15, 0xba, 0x57, 0x64, 0x04, 0x5d, 0x84, - 0x2a, 0x9d, 0xd2, 0xf3, 0xbd, 0xf6, 0xdc, 0x35, 0x65, 0xa5, 0xa4, 0xcd, 0xd1, 0xe7, 0x5d, 0x4f, - 0xfd, 0x3e, 0x2c, 0x51, 0x12, 0xac, 0x1f, 0xe8, 0xb6, 0x8d, 0x2d, 0xef, 0xec, 0x27, 0x18, 0xdd, - 0xa4, 0x10, 0xdb, 0x84, 0x1c, 0x42, 0x9f, 0xaf, 0x4f, 0x8f, 0xb1, 0xa6, 0x89, 0x67, 0xf5, 0x37, - 0x0b, 0xd0, 0x12, 0x9f, 0x12, 0xec, 0xbe, 0x04, 0xe5, 0xbe, 0x33, 0xb6, 0x7d, 0xba, 0xfd, 0xbc, - 0xc6, 0x1e, 0xd0, 0x4b, 0xd0, 0xe0, 0xd3, 0x7a, 0xb6, 0x3e, 0xc4, 0x74, 0x97, 0x9a, 0x56, 0xe7, - 0x63, 0x8f, 0xf4, 0x21, 0xce, 0x45, 0xf7, 0x6b, 0x50, 0x1f, 0xe9, 0xae, 0x6f, 0xc6, 0xb8, 0x26, - 0x3a, 0x34, 0x89, 0x69, 0xc8, 0x0e, 0x26, 0xfd, 0x6f, 0x57, 0xf7, 0x0e, 0xbb, 0x1b, 0x9c, 0xda, - 0xb1, 0x31, 0xf4, 0x26, 0x94, 0x2d, 0x7c, 0x84, 0x2d, 0x4a, 0xec, 0xe6, 0xea, 0xd5, 0xdb, 0x29, - 0x99, 0xbc, 0xcd, 0x3f, 0x79, 0x93, 0x80, 0x69, 0x0c, 0x5a, 0xfd, 0x33, 0x05, 0x96, 0xef, 0x7b, - 0x9e, 0x39, 0xb0, 0x53, 0x04, 0x59, 0x86, 0x8a, 0xed, 0x18, 0xb8, 0xbb, 0x41, 0x29, 0x52, 0xd4, - 0xf8, 0x13, 0xba, 0x04, 0xb5, 0x11, 0xc6, 0x6e, 0xcf, 0x75, 0xac, 0x80, 0x1e, 0x55, 0x32, 0xa0, - 0x39, 0x16, 0x46, 0xdf, 0x85, 0x45, 0x2f, 0xb1, 0x10, 0xa3, 0x7f, 0x7d, 0xf5, 0x7a, 0x36, 0x4a, - 0x02, 0x56, 0x4b, 0xcf, 0x56, 0xbf, 0x28, 0xc0, 0x79, 0x01, 0xc7, 0x70, 0x25, 0xff, 0x93, 0x03, - 0xf3, 0xf0, 0x40, 0xa0, 0xc7, 0x1e, 0xf2, 0x1c, 0x98, 0x38, 0xe9, 0x62, 0xf4, 0xa4, 0xf3, 0x48, - 0x76, 0xe2, 0x18, 0xcb, 0xe9, 0x63, 0xbc, 0x0a, 0x75, 0xfc, 0x74, 0x64, 0xba, 0xb8, 0x47, 0x64, - 0x81, 0x9e, 0x54, 0x49, 0x03, 0x36, 0xb4, 0x6b, 0x0e, 0xa3, 0xe2, 0x3e, 0x97, 0x5b, 0xdc, 0xd5, - 0x3f, 0x57, 0xe0, 0x42, 0xea, 0x94, 0xb8, 0xfe, 0xd0, 0xa0, 0x45, 0xbf, 0x3c, 0xa4, 0x0c, 0xd1, - 0x24, 0x84, 0xe0, 0x2f, 0x4f, 0x22, 0x78, 0x08, 0xae, 0xa5, 0xe6, 0x47, 0x90, 0x2c, 0xe4, 0x47, - 0xf2, 0x10, 0x2e, 0x3c, 0xc4, 0x3e, 0xdf, 0x80, 0xbc, 0xc3, 0x33, 0x48, 0x76, 0x5c, 0x51, 0x15, - 0x92, 0x8a, 0x4a, 0xfd, 0x8b, 0x50, 0x84, 0xe9, 0x56, 0x5d, 0x7b, 0xdf, 0x41, 0x2f, 0x42, 0x4d, - 0x80, 0x70, 0xae, 0x08, 0x07, 0xd0, 0x37, 0xa1, 0x4c, 0x30, 0x65, 0x2c, 0xd1, 0x5c, 0x7d, 0x49, - 0xfe, 0x4d, 0x91, 0x35, 0x35, 0x06, 0x8f, 0x36, 0xa0, 0xe9, 0xf9, 0xba, 0xeb, 0xf7, 0x46, 0x8e, - 0x47, 0xcf, 0x99, 0x32, 0x4e, 0x7d, 0xf5, 0x72, 0x7c, 0x05, 0x62, 0xb7, 0xb6, 0xbc, 0xc1, 0x36, - 0x07, 0xd2, 0xe6, 0xe9, 0xa4, 0xe0, 0x11, 0x7d, 0x07, 0x1a, 0xd8, 0x36, 0xc2, 0x35, 0x4a, 0x79, - 0xd6, 0xa8, 0x63, 0xdb, 0x10, 0x2b, 0x84, 0xa7, 0x52, 0xce, 0x7f, 0x2a, 0xbf, 0xab, 0x40, 0x3b, - 0x7d, 0x2c, 0xb3, 0xd8, 0x9e, 0x77, 0xd9, 0x24, 0xcc, 0x8e, 0x65, 0xa2, 0x5c, 0x8b, 0xa3, 0xd1, - 0xf8, 0x14, 0xf5, 0x4f, 0x14, 0x78, 0x21, 0x44, 0x87, 0xbe, 0x7a, 0x5e, 0x3c, 0x82, 0x6e, 0x41, - 0xcb, 0xb4, 0xfb, 0xd6, 0xd8, 0xc0, 0x8f, 0xed, 0x0f, 0xb1, 0x6e, 0xf9, 0x07, 0x27, 0xf4, 0xe4, - 0xaa, 0x5a, 0x6a, 0x5c, 0xfd, 0x8f, 0x02, 0x2c, 0x27, 0xf1, 0x9a, 0x85, 0x48, 0xff, 0x0f, 0xca, - 0xa6, 0xbd, 0xef, 0x04, 0x34, 0xba, 0x32, 0x41, 0x14, 0xc9, 0x5e, 0x0c, 0x18, 0x39, 0x80, 0x02, - 0xe5, 0xd5, 0x3f, 0xc0, 0xfd, 0xc3, 0x91, 0x63, 0x52, 0x35, 0x45, 0x96, 0xf8, 0x8e, 0x64, 0x09, - 0x39, 0xc6, 0xb7, 0xb9, 0x61, 0x5d, 0x17, 0x4b, 0x7c, 0x60, 0xfb, 0xee, 0x89, 0xb6, 0xd8, 0x4f, - 0x8e, 0x77, 0xfa, 0xb0, 0x2c, 0x07, 0x46, 0x2d, 0x28, 0x1e, 0xe2, 0x13, 0xfa, 0xc9, 0x35, 0x8d, - 0xfc, 0x8b, 0xee, 0x41, 0xf9, 0x48, 0xb7, 0xc6, 0x98, 0xeb, 0x84, 0x29, 0x9c, 0xcb, 0x60, 0xdf, - 0x29, 0xbc, 0xa5, 0xa8, 0x43, 0xb8, 0xf4, 0x10, 0xfb, 0x5d, 0xdb, 0xc3, 0xae, 0xbf, 0x66, 0xda, - 0x96, 0x33, 0xd8, 0xd6, 0xfd, 0x83, 0x19, 0x94, 0x43, 0x4c, 0xce, 0x0b, 0x09, 0x39, 0x57, 0x7f, - 0xa8, 0xc0, 0x8b, 0xf2, 0xfd, 0xf8, 0x81, 0x76, 0xa0, 0xba, 0x6f, 0x62, 0xcb, 0x20, 0x5c, 0xa3, - 0x50, 0xae, 0x11, 0xcf, 0x44, 0x49, 0x8c, 0x08, 0x30, 0x3f, 0xb7, 0x84, 0x92, 0x10, 0x6e, 0xec, - 0x8e, 0xef, 0x9a, 0xf6, 0x60, 0xd3, 0xf4, 0x7c, 0x8d, 0xc1, 0x47, 0xb8, 0xa4, 0x98, 0x5f, 0x38, - 0x7f, 0x5b, 0x81, 0x2b, 0x0f, 0xb1, 0xbf, 0x2e, 0x6c, 0x0c, 0x79, 0x6f, 0x7a, 0xbe, 0xd9, 0xf7, - 0x9e, 0xad, 0x5b, 0x9b, 0xc3, 0x47, 0x51, 0x7f, 0x5f, 0x81, 0xab, 0x99, 0xc8, 0x70, 0xd2, 0x71, - 0x1d, 0x1a, 0x58, 0x18, 0xb9, 0x0e, 0xfd, 0x05, 0x7c, 0xf2, 0x84, 0x1c, 0xfe, 0xb6, 0x6e, 0xba, - 0x4c, 0x87, 0x9e, 0xd1, 0xa2, 0xfc, 0x48, 0x81, 0xcb, 0x0f, 0xb1, 0xbf, 0x1d, 0xd8, 0xd7, 0xaf, - 0x91, 0x3a, 0x04, 0x26, 0x62, 0xe7, 0x03, 0xdf, 0x39, 0x36, 0xa6, 0xfe, 0x1e, 0x3b, 0x4e, 0x29, - 0xbe, 0x5f, 0x0b, 0x01, 0xaf, 0x50, 0x49, 0x88, 0xa8, 0x08, 0x2e, 0xec, 0x9c, 0x7c, 0xea, 0x97, - 0x65, 0x68, 0x3c, 0xe1, 0x5a, 0x81, 0x5a, 0xd0, 0x24, 0x25, 0x14, 0xb9, 0x13, 0x14, 0xf1, 0xa6, - 0x64, 0x0e, 0xd6, 0x1a, 0xcc, 0x7b, 0x18, 0x1f, 0x9e, 0xd2, 0x5e, 0x36, 0xc8, 0x1c, 0x61, 0xec, - 0x36, 0x61, 0x71, 0x6c, 0x53, 0x67, 0x1e, 0x1b, 0xfc, 0x03, 0x18, 0xd1, 0xa7, 0x2b, 0xd3, 0xf4, - 0x44, 0xf4, 0x21, 0x8f, 0x6b, 0x22, 0x6b, 0x95, 0x73, 0xad, 0x95, 0x9c, 0x86, 0xba, 0xd0, 0x32, - 0x5c, 0x67, 0x34, 0xc2, 0x46, 0xcf, 0x0b, 0x96, 0xaa, 0xe4, 0x5b, 0x8a, 0xcf, 0x13, 0x4b, 0xdd, - 0x85, 0xf3, 0x49, 0x4c, 0xbb, 0x06, 0xf1, 0x0b, 0x09, 0x67, 0xc9, 0x5e, 0xa1, 0xd7, 0x60, 0x31, - 0x0d, 0x5f, 0xa5, 0xf0, 0xe9, 0x17, 0xe8, 0x75, 0x40, 0x09, 0x54, 0x09, 0x78, 0x8d, 0x81, 0xc7, - 0x91, 0xe1, 0xe0, 0x34, 0xde, 0x8e, 0x83, 0x03, 0x03, 0xe7, 0x6f, 0x22, 0xe0, 0x5d, 0x62, 0x5d, - 0x63, 0xe0, 0x5e, 0xbb, 0x9e, 0x8f, 0x10, 0xf1, 0xc5, 0x3c, 0xf5, 0xb7, 0x14, 0x58, 0xfe, 0x58, - 0xf7, 0xfb, 0x07, 0x1b, 0xc3, 0xd9, 0x63, 0xc2, 0xf7, 0xa0, 0x76, 0x24, 0x22, 0x3f, 0xa6, 0xc5, - 0x65, 0xc1, 0x50, 0x94, 0xed, 0xb5, 0x70, 0x86, 0xfa, 0x4f, 0x0a, 0x8f, 0x4e, 0x03, 0xec, 0xbe, - 0x7a, 0x55, 0x33, 0x2d, 0x48, 0x4f, 0x08, 0x60, 0x39, 0x25, 0x80, 0xea, 0x53, 0x00, 0x8e, 0xfe, - 0x96, 0x37, 0x38, 0x03, 0xe6, 0x6f, 0xc1, 0x1c, 0xdf, 0x8f, 0x6b, 0x9b, 0x69, 0x47, 0x1a, 0x80, - 0xab, 0xff, 0x5d, 0x81, 0x7a, 0xe4, 0x05, 0x6a, 0x42, 0x41, 0xa8, 0x91, 0x82, 0xe4, 0xfb, 0x0b, - 0xd3, 0xa3, 0xac, 0x62, 0x3a, 0xca, 0xba, 0x09, 0x4d, 0x93, 0x9a, 0xf7, 0x1e, 0xff, 0x6a, 0xea, - 0x4d, 0xd7, 0xb4, 0x79, 0x36, 0xca, 0x99, 0x08, 0x5d, 0x81, 0xba, 0x3d, 0x1e, 0xf6, 0x9c, 0xfd, - 0x9e, 0xeb, 0x1c, 0x7b, 0x3c, 0x5c, 0xab, 0xd9, 0xe3, 0xe1, 0x47, 0xfb, 0x9a, 0x73, 0xec, 0x85, - 0x11, 0x41, 0xe5, 0x94, 0x11, 0xc1, 0x15, 0xa8, 0x0f, 0xf5, 0xa7, 0x64, 0xd5, 0x9e, 0x3d, 0x1e, - 0xd2, 0x48, 0xae, 0xa8, 0xd5, 0x86, 0xfa, 0x53, 0xcd, 0x39, 0x7e, 0x34, 0x1e, 0xa2, 0x15, 0x68, - 0x59, 0xba, 0xe7, 0xf7, 0xa2, 0xa1, 0x60, 0x95, 0x86, 0x82, 0x4d, 0x32, 0xfe, 0x41, 0x18, 0x0e, - 0xa6, 0x63, 0x8b, 0xda, 0xd9, 0x62, 0x0b, 0x63, 0x68, 0x85, 0x6b, 0x40, 0xae, 0xd8, 0xc2, 0x18, - 0x5a, 0x62, 0x85, 0xb7, 0x60, 0x6e, 0x8f, 0xba, 0x4a, 0x93, 0x84, 0xf8, 0x01, 0xf1, 0x92, 0x98, - 0x47, 0xa5, 0x05, 0xe0, 0xe8, 0x5b, 0x50, 0xa3, 0x16, 0x8a, 0xce, 0x6d, 0xe4, 0x9a, 0x1b, 0x4e, - 0x20, 0xb3, 0x0d, 0x6c, 0xf9, 0x3a, 0x9d, 0x3d, 0x9f, 0x6f, 0xb6, 0x98, 0x40, 0x34, 0x68, 0xdf, - 0xc5, 0xba, 0x8f, 0x8d, 0xb5, 0x93, 0x75, 0x67, 0x38, 0xd2, 0x29, 0x0b, 0xb5, 0x9b, 0xd4, 0xc9, - 0x97, 0xbd, 0x42, 0x2f, 0x43, 0xb3, 0x2f, 0x9e, 0x1e, 0xb8, 0xce, 0xb0, 0xbd, 0x40, 0xe5, 0x2b, - 0x31, 0x8a, 0x2e, 0x03, 0x04, 0xba, 0x53, 0xf7, 0xdb, 0x2d, 0x7a, 0x76, 0x35, 0x3e, 0x72, 0x9f, - 0xa6, 0x85, 0x4c, 0xaf, 0xc7, 0x12, 0x30, 0xa6, 0x3d, 0x68, 0x2f, 0xd2, 0x1d, 0xeb, 0x41, 0xc6, - 0xc6, 0xb4, 0x07, 0xe8, 0x02, 0xcc, 0x99, 0x5e, 0x6f, 0x5f, 0x3f, 0xc4, 0x6d, 0x44, 0xdf, 0x56, - 0x4c, 0xef, 0x81, 0x7e, 0x48, 0xbd, 0x57, 0xbe, 0x19, 0x36, 0xda, 0xe7, 0xe9, 0xab, 0x70, 0x20, - 0xcc, 0xe3, 0x2c, 0x9d, 0x2a, 0x8f, 0xf3, 0x39, 0x2c, 0x85, 0x8c, 0x1a, 0xe1, 0x8c, 0x34, 0x7f, - 0x29, 0x67, 0xe0, 0xaf, 0xc9, 0x0e, 0xf7, 0xcf, 0xcb, 0xb0, 0xbc, 0xa3, 0x1f, 0xe1, 0xe7, 0xef, - 0xdb, 0xe7, 0x52, 0x9f, 0x9b, 0xb0, 0x48, 0xdd, 0xf9, 0xd5, 0x08, 0x3e, 0x13, 0x3c, 0x87, 0x28, - 0x6b, 0xa5, 0x27, 0xa2, 0x6f, 0x13, 0x65, 0x8b, 0xfb, 0x87, 0xdb, 0x24, 0x34, 0x0a, 0xbc, 0x86, - 0xcb, 0x92, 0x75, 0xd6, 0x05, 0x94, 0x16, 0x9d, 0x81, 0xb6, 0x61, 0x21, 0x7e, 0x02, 0x81, 0xbf, - 0xf0, 0xca, 0xc4, 0xb8, 0x39, 0xa4, 0xbe, 0xd6, 0x8c, 0x1d, 0x86, 0x87, 0xda, 0x30, 0xc7, 0x8d, - 0x3d, 0xd5, 0x3c, 0x55, 0x2d, 0x78, 0x44, 0xdb, 0x70, 0x9e, 0x7d, 0xc1, 0x0e, 0x17, 0x30, 0xf6, - 0xf1, 0xd5, 0x5c, 0x1f, 0x2f, 0x9b, 0x1a, 0x97, 0xcf, 0xda, 0x69, 0xe5, 0xb3, 0x0d, 0x73, 0x5c, - 0x66, 0xa8, 0x4a, 0xaa, 0x6a, 0xc1, 0x23, 0x39, 0xe6, 0x50, 0x7a, 0xea, 0x4c, 0x08, 0xc4, 0x00, - 0x99, 0x17, 0x28, 0xf6, 0x06, 0x55, 0xec, 0xc1, 0x23, 0xd5, 0x36, 0x78, 0xd0, 0x63, 0x22, 0x32, - 0x9f, 0x4f, 0x44, 0xaa, 0x1e, 0x1e, 0xd0, 0xff, 0x92, 0x96, 0xa5, 0x99, 0xb2, 0x2c, 0xea, 0xaf, - 0x2b, 0x00, 0xe1, 0x49, 0x4e, 0xc9, 0x28, 0xbd, 0x0d, 0x55, 0x21, 0x56, 0xb9, 0x82, 0x62, 0x01, - 0x9e, 0x34, 0x4d, 0xc5, 0x84, 0x69, 0x52, 0xff, 0x45, 0x81, 0xc6, 0x06, 0xa1, 0xe3, 0xa6, 0x33, - 0xa0, 0x86, 0xf4, 0x26, 0x34, 0x5d, 0xdc, 0x77, 0x5c, 0xa3, 0x87, 0x6d, 0xdf, 0x35, 0x31, 0xcb, - 0x46, 0x94, 0xb4, 0x79, 0x36, 0xfa, 0x01, 0x1b, 0x24, 0x60, 0xc4, 0xda, 0x78, 0xbe, 0x3e, 0x1c, - 0xf5, 0xf6, 0x89, 0x7e, 0x63, 0x79, 0xf1, 0x79, 0x31, 0x4a, 0xd5, 0xdb, 0x4b, 0xd0, 0x08, 0xc1, - 0x7c, 0x87, 0xee, 0x5f, 0xd2, 0xea, 0x62, 0x6c, 0xd7, 0x41, 0x37, 0xa0, 0x49, 0x0f, 0xb2, 0x67, - 0x39, 0x83, 0x1e, 0x89, 0x71, 0xb9, 0x8d, 0x6d, 0x18, 0x1c, 0x2d, 0xc2, 0x20, 0x71, 0x28, 0xcf, - 0xfc, 0x1c, 0x73, 0x2b, 0x2b, 0xa0, 0x76, 0xcc, 0xcf, 0xb1, 0xfa, 0x6b, 0x0a, 0xcc, 0x73, 0xa3, - 0xbc, 0x23, 0x0a, 0x18, 0x34, 0x3d, 0xcb, 0xf2, 0x0b, 0xf4, 0x7f, 0xf4, 0x4e, 0x3c, 0x41, 0x77, - 0x43, 0x2a, 0x64, 0x74, 0x11, 0xea, 0x2c, 0xc6, 0x2c, 0x72, 0x9e, 0x00, 0xf7, 0x0b, 0x42, 0x53, - 0xdd, 0xd7, 0x1f, 0x39, 0x06, 0xcb, 0x17, 0xb6, 0x61, 0x4e, 0x37, 0x0c, 0x17, 0x7b, 0x1e, 0xc7, - 0x23, 0x78, 0x24, 0x6f, 0x8e, 0xb0, 0xeb, 0x05, 0x07, 0x5b, 0xd4, 0x82, 0x47, 0xf4, 0xad, 0x44, - 0x5d, 0xa1, 0xbe, 0x7a, 0x2d, 0x1b, 0x4f, 0x1e, 0x8e, 0x85, 0x95, 0x87, 0xbf, 0x2b, 0x40, 0x93, - 0xf3, 0xe6, 0x1a, 0xb7, 0x9f, 0x93, 0x59, 0x6c, 0x0d, 0x1a, 0xfb, 0xa1, 0x6c, 0x4d, 0x4a, 0x27, - 0x45, 0x45, 0x30, 0x36, 0x67, 0x1a, 0xaf, 0xc5, 0x2d, 0x78, 0x69, 0x26, 0x0b, 0x5e, 0x3e, 0xad, - 0x86, 0x48, 0x7b, 0x72, 0x15, 0x89, 0x27, 0xa7, 0xfe, 0x12, 0xd4, 0x23, 0x0b, 0x50, 0x0d, 0xc8, - 0x32, 0x36, 0x9c, 0x62, 0xc1, 0x23, 0xba, 0x17, 0xfa, 0x31, 0x8c, 0x54, 0x17, 0x25, 0xb8, 0x24, - 0x5c, 0x18, 0xf5, 0xa7, 0x0a, 0x54, 0xf8, 0xca, 0x57, 0xa1, 0xce, 0xe5, 0x8b, 0x7a, 0x76, 0x6c, - 0x75, 0xe0, 0x43, 0xc4, 0xb5, 0x7b, 0x76, 0x02, 0x76, 0x11, 0xaa, 0x09, 0xd1, 0x9a, 0xe3, 0x6a, - 0x37, 0x78, 0x15, 0x91, 0x27, 0xf2, 0x8a, 0x88, 0x12, 0x5a, 0x82, 0xb2, 0xe5, 0x0c, 0x44, 0x11, - 0x88, 0x3d, 0xa8, 0x3f, 0x53, 0x68, 0xf2, 0x5d, 0xc3, 0x7d, 0xe7, 0x08, 0xbb, 0x27, 0xb3, 0xe7, - 0x2f, 0xdf, 0x8d, 0xb0, 0x79, 0xce, 0x20, 0x4a, 0x4c, 0x40, 0xef, 0x86, 0x87, 0x50, 0x94, 0xa5, - 0x39, 0xa2, 0x2a, 0x9a, 0x33, 0x69, 0x78, 0x18, 0x7f, 0xa0, 0xd0, 0x4c, 0x6c, 0xfc, 0x53, 0xce, - 0xea, 0x4d, 0x3c, 0x93, 0x70, 0x43, 0xfd, 0xb9, 0x02, 0x17, 0x33, 0xa8, 0xfb, 0x64, 0xf5, 0x6b, - 0xa0, 0xef, 0x3b, 0x50, 0x15, 0x21, 0x77, 0x31, 0x57, 0xc8, 0x2d, 0xe0, 0xd5, 0x3f, 0x66, 0xf5, - 0x00, 0x09, 0x79, 0x9f, 0xac, 0x3e, 0x27, 0x02, 0x27, 0x53, 0x67, 0x45, 0x49, 0xea, 0xec, 0x5f, - 0x15, 0xe8, 0x84, 0xa9, 0x2a, 0x6f, 0xed, 0x64, 0xd6, 0x02, 0xd2, 0xb3, 0x09, 0x34, 0xdf, 0x16, - 0xb5, 0x0e, 0xa2, 0x17, 0x73, 0x85, 0x88, 0x41, 0xa5, 0xc3, 0xa6, 0x59, 0xef, 0xf4, 0x07, 0xcd, - 0x22, 0x95, 0x9d, 0xc8, 0xc1, 0xb3, 0x7a, 0x47, 0x78, 0xb0, 0x3f, 0x65, 0x4c, 0xfa, 0x20, 0x9e, - 0xaf, 0xfa, 0xba, 0x09, 0x18, 0xad, 0xc1, 0x1c, 0xf0, 0x1a, 0x4c, 0x29, 0x51, 0x83, 0xe1, 0xe3, - 0xea, 0x90, 0xb2, 0x40, 0xea, 0x03, 0x9e, 0x17, 0xc1, 0x7e, 0x43, 0x81, 0x36, 0xdf, 0x85, 0xb5, - 0x23, 0x38, 0xc3, 0x91, 0x85, 0x7d, 0x6c, 0x7c, 0xd5, 0x39, 0x93, 0xbf, 0x29, 0x40, 0x2b, 0xea, - 0xd8, 0x50, 0xdf, 0xe4, 0x4d, 0x28, 0xd3, 0xa4, 0x14, 0xc7, 0x60, 0xaa, 0x76, 0x60, 0xd0, 0xc4, - 0x32, 0xd2, 0x68, 0x61, 0xd7, 0x0b, 0x1c, 0x17, 0xfe, 0x18, 0x7a, 0x57, 0xc5, 0xd3, 0x7b, 0x57, - 0x2f, 0x42, 0x8d, 0x58, 0x2e, 0x67, 0x4c, 0xd6, 0x65, 0x85, 0xf1, 0x70, 0x00, 0xbd, 0x07, 0x15, - 0xd6, 0xc1, 0xc3, 0xeb, 0x92, 0x37, 0xe3, 0x4b, 0xf3, 0xee, 0x9e, 0x48, 0x5d, 0x81, 0x0e, 0x68, - 0x7c, 0x12, 0x39, 0xa3, 0x91, 0xeb, 0x0c, 0xa8, 0x1b, 0x46, 0x8c, 0x5a, 0x59, 0x13, 0xcf, 0xc4, - 0x4d, 0x74, 0x46, 0xdd, 0x0d, 0x9e, 0x61, 0xa1, 0xff, 0xab, 0xff, 0x1f, 0x96, 0xc3, 0x80, 0x9e, - 0xa1, 0x79, 0x56, 0x26, 0x57, 0xff, 0xb1, 0x00, 0xe7, 0x77, 0x4e, 0xec, 0x7e, 0x52, 0x5c, 0x96, - 0xa1, 0x32, 0xb2, 0xf4, 0x30, 0x03, 0xce, 0x9f, 0x68, 0x77, 0x41, 0x10, 0xaa, 0x13, 0xb3, 0xce, - 0x68, 0x5c, 0x17, 0x63, 0xbb, 0xce, 0x54, 0x6f, 0xeb, 0xa6, 0xc8, 0x40, 0x60, 0x83, 0x39, 0x10, - 0x2c, 0xc3, 0x37, 0x2f, 0x46, 0xa9, 0x03, 0xf1, 0x1e, 0x00, 0xf5, 0xb1, 0x7a, 0xa7, 0xf1, 0xab, - 0xe8, 0x8c, 0x4d, 0xe2, 0x57, 0x25, 0xdb, 0x20, 0x2a, 0xe9, 0x2c, 0xfd, 0x4b, 0x11, 0xb5, 0xdc, - 0x33, 0x0d, 0x4e, 0xe3, 0x88, 0xf4, 0x1a, 0xe8, 0x3a, 0xcc, 0x87, 0xf2, 0x4e, 0x60, 0xaa, 0x29, - 0x25, 0x60, 0xa8, 0x3f, 0x2e, 0x40, 0x3b, 0x72, 0x20, 0x5f, 0xb5, 0x77, 0x9b, 0x11, 0xf3, 0x16, - 0x9f, 0x51, 0xcc, 0x5b, 0x9a, 0xdd, 0xa3, 0x2d, 0xcb, 0x3c, 0xda, 0x1f, 0x14, 0xa1, 0x19, 0x52, - 0x6d, 0xdb, 0xd2, 0xed, 0x4c, 0xa6, 0xdb, 0x81, 0xa6, 0x17, 0xa3, 0x2a, 0xa7, 0xd3, 0x37, 0x64, - 0x22, 0x9c, 0x71, 0x10, 0x5a, 0x62, 0x09, 0x74, 0x99, 0xf2, 0x97, 0xeb, 0xb3, 0xe4, 0x24, 0x73, - 0x4f, 0x6b, 0x4c, 0x57, 0x98, 0x43, 0x8c, 0x5e, 0x03, 0xc4, 0x05, 0xbc, 0x67, 0xda, 0x3d, 0x0f, - 0xf7, 0x1d, 0xdb, 0x60, 0xa2, 0x5f, 0xd6, 0x5a, 0xfc, 0x4d, 0xd7, 0xde, 0x61, 0xe3, 0xe8, 0x4d, - 0x28, 0xf9, 0x27, 0x23, 0xe6, 0xab, 0x36, 0xa5, 0xde, 0x5e, 0x88, 0xd7, 0xee, 0xc9, 0x08, 0x6b, - 0x14, 0x3c, 0xe8, 0x21, 0xf3, 0x5d, 0xfd, 0x88, 0x3b, 0xfe, 0x25, 0x2d, 0x32, 0x12, 0x4d, 0x03, - 0xcc, 0xc5, 0xd3, 0x00, 0x54, 0x88, 0x04, 0x63, 0xfa, 0xbe, 0xc5, 0x39, 0x33, 0xc2, 0xae, 0xbb, - 0xbe, 0x45, 0x3e, 0xd2, 0x77, 0x7c, 0xdd, 0x62, 0xa2, 0x58, 0xe3, 0x8a, 0x8b, 0x8c, 0xd0, 0x20, - 0xfb, 0xdf, 0x88, 0xe2, 0x15, 0x88, 0x69, 0xd8, 0x1b, 0x5b, 0xd9, 0xa2, 0x3f, 0x39, 0x31, 0x35, - 0x4d, 0xea, 0xbf, 0x0d, 0x75, 0xce, 0x15, 0xa7, 0xe0, 0x2a, 0x60, 0x53, 0x36, 0x27, 0xb0, 0x79, - 0xf9, 0x19, 0xb1, 0x79, 0xe5, 0x0c, 0xa9, 0x1d, 0xf9, 0xd9, 0xa8, 0x3f, 0x54, 0xe0, 0x85, 0x94, - 0x82, 0x9e, 0x48, 0xda, 0xc9, 0x81, 0x3f, 0x57, 0xdc, 0xc9, 0x25, 0xb9, 0x69, 0x7a, 0x17, 0x2a, - 0x2e, 0x5d, 0x9d, 0x17, 0x19, 0xaf, 0x4f, 0x64, 0x3e, 0x86, 0x88, 0xc6, 0xa7, 0xa8, 0x7f, 0xa8, - 0xc0, 0x85, 0x34, 0xaa, 0x33, 0xf8, 0x1b, 0x6b, 0x30, 0xc7, 0x96, 0x0e, 0x64, 0x74, 0x65, 0xb2, - 0x8c, 0x86, 0xc4, 0xd1, 0x82, 0x89, 0xea, 0x0e, 0x2c, 0x07, 0x6e, 0x49, 0x48, 0xfa, 0x2d, 0xec, - 0xeb, 0x13, 0xc2, 0xde, 0xab, 0x50, 0x67, 0xf1, 0x13, 0x0b, 0x27, 0x59, 0x4d, 0x16, 0xf6, 0x44, - 0x1e, 0x53, 0xfd, 0xa3, 0x02, 0x2c, 0x51, 0xbb, 0x9e, 0x2c, 0xb0, 0xe5, 0xa9, 0xf8, 0xaa, 0xc2, - 0x98, 0x10, 0xc3, 0xc1, 0x3e, 0xad, 0xa6, 0xc5, 0xc6, 0x50, 0x37, 0x9d, 0xe6, 0x94, 0xa6, 0x47, - 0xc2, 0x12, 0xf7, 0x86, 0xee, 0xeb, 0xb4, 0xc2, 0x9d, 0xcc, 0x6f, 0x86, 0xfe, 0x44, 0xe9, 0x2c, - 0xfe, 0xc4, 0xab, 0xd0, 0x62, 0x99, 0xff, 0x9e, 0x88, 0xb6, 0xa9, 0x62, 0x2a, 0x69, 0x0b, 0x6c, - 0x7c, 0x37, 0x18, 0x56, 0x37, 0xe1, 0x85, 0x04, 0x51, 0x66, 0x38, 0x7c, 0xf5, 0x2f, 0x15, 0x72, - 0x72, 0xb1, 0x56, 0xab, 0xb3, 0xbb, 0xdf, 0x97, 0x45, 0x11, 0x90, 0xd8, 0xdd, 0x84, 0xbe, 0x31, - 0xd0, 0xfb, 0x50, 0xb3, 0xf1, 0x71, 0x2f, 0xea, 0xd1, 0xe5, 0x88, 0x4d, 0xaa, 0x36, 0x3e, 0xa6, - 0xff, 0xa9, 0x8f, 0xe0, 0x42, 0x0a, 0xd5, 0x59, 0xbe, 0xfd, 0xef, 0x15, 0xb8, 0xb8, 0xe1, 0x3a, - 0xa3, 0x27, 0xa6, 0xeb, 0x8f, 0x75, 0x2b, 0xde, 0x67, 0x70, 0x86, 0xcf, 0xcf, 0xd1, 0xc6, 0xf9, - 0x61, 0x2a, 0x0a, 0x7e, 0x4d, 0x22, 0x6c, 0x69, 0xa4, 0xf8, 0x47, 0x47, 0x22, 0x81, 0xff, 0x2c, - 0xca, 0x90, 0xe7, 0x70, 0x53, 0x5c, 0x98, 0x3c, 0x61, 0x92, 0xb4, 0x22, 0x51, 0x3c, 0x6b, 0x45, - 0x22, 0xc3, 0x12, 0x94, 0x9e, 0x91, 0x25, 0x38, 0x75, 0x0a, 0x6f, 0x1d, 0xe2, 0xd5, 0x22, 0x6a, - 0xc8, 0x4f, 0x5b, 0x61, 0x7a, 0x0f, 0x20, 0x2c, 0x9a, 0xf0, 0xd6, 0xd8, 0x29, 0x2b, 0x44, 0x26, - 0x90, 0x33, 0x12, 0xb6, 0x96, 0xbb, 0x02, 0x91, 0x64, 0xfa, 0x77, 0xa1, 0x23, 0xe3, 0xcd, 0x59, - 0xf8, 0xfd, 0xdf, 0x0b, 0x00, 0x5d, 0xd1, 0x7f, 0x7d, 0x36, 0x63, 0x91, 0xf2, 0xae, 0xd3, 0xbc, - 0x63, 0xa4, 0xbc, 0xf4, 0xa2, 0xdc, 0x4b, 0x8f, 0xc8, 0x0a, 0x63, 0x85, 0xa4, 0x7e, 0xbe, 0x04, - 0x35, 0xd7, 0x39, 0xee, 0x11, 0xe1, 0x32, 0x82, 0x06, 0x73, 0xd7, 0x39, 0x26, 0x22, 0x67, 0xa0, - 0x0b, 0x30, 0xe7, 0xeb, 0xde, 0x21, 0x59, 0x9f, 0xa5, 0x15, 0x2b, 0xe4, 0xb1, 0x6b, 0xa0, 0x25, - 0x28, 0xef, 0x9b, 0x16, 0x66, 0x4d, 0x29, 0x35, 0x8d, 0x3d, 0xa0, 0x6f, 0x06, 0xcd, 0x8d, 0xd5, - 0xdc, 0x4d, 0x4c, 0xac, 0xbf, 0xf1, 0x3a, 0xcc, 0x13, 0x4e, 0x22, 0x48, 0x30, 0xb1, 0x6e, 0xf1, - 0x92, 0x02, 0x1f, 0xa4, 0xcd, 0x0b, 0x3f, 0x53, 0x60, 0x21, 0x24, 0x2d, 0xd5, 0x4d, 0x44, 0xdd, - 0x51, 0x55, 0xb7, 0xee, 0x18, 0x4c, 0x8b, 0x34, 0x33, 0xec, 0x0a, 0x9b, 0xc8, 0x14, 0x5a, 0x38, - 0x65, 0x52, 0x1e, 0x80, 0x7c, 0x3c, 0xa1, 0x8c, 0x69, 0x04, 0x99, 0xa9, 0x8a, 0xeb, 0x1c, 0x77, - 0x0d, 0x41, 0x32, 0xd6, 0x2b, 0xce, 0xa2, 0x5e, 0x42, 0xb2, 0x75, 0xda, 0x2e, 0x7e, 0x1d, 0xe6, - 0xb1, 0xeb, 0x3a, 0x6e, 0x6f, 0x88, 0x3d, 0x4f, 0x1f, 0x04, 0x6d, 0x18, 0x0d, 0x3a, 0xb8, 0xc5, - 0xc6, 0xd4, 0x7f, 0x28, 0x41, 0x33, 0xfc, 0x94, 0xa0, 0x21, 0xc2, 0x34, 0x82, 0x86, 0x08, 0x93, - 0x9c, 0x2f, 0xb8, 0x4c, 0x4b, 0x0a, 0x0e, 0x58, 0x2b, 0xb4, 0x15, 0xad, 0xc6, 0x47, 0xbb, 0x06, - 0x31, 0xee, 0x84, 0x40, 0xb6, 0x63, 0xe0, 0x90, 0x03, 0x20, 0x18, 0x92, 0x85, 0x69, 0xa5, 0x1c, - 0x8c, 0x54, 0xce, 0xc1, 0x48, 0x15, 0x09, 0x23, 0x2d, 0x43, 0x65, 0x6f, 0xdc, 0x3f, 0xc4, 0x3e, - 0xf7, 0xfb, 0xf8, 0x53, 0x9c, 0xc1, 0xaa, 0x09, 0x06, 0x13, 0x7c, 0x54, 0x8b, 0xf2, 0xd1, 0x25, - 0xa8, 0x05, 0x96, 0xda, 0xa3, 0x05, 0xc2, 0xa2, 0x56, 0xe5, 0x26, 0xda, 0x43, 0x6f, 0x05, 0x4e, - 0x61, 0x9d, 0x4a, 0x94, 0x2a, 0x51, 0x48, 0x09, 0x2e, 0x09, 0x5c, 0xc2, 0x57, 0x60, 0x21, 0x42, - 0x0e, 0xca, 0x67, 0xac, 0x8a, 0x18, 0x89, 0x19, 0xa8, 0x05, 0xb9, 0x09, 0xcd, 0x90, 0x24, 0x14, - 0x6e, 0x9e, 0x85, 0x6a, 0x62, 0x94, 0x82, 0x09, 0x76, 0x6f, 0x9e, 0x92, 0xdd, 0x2f, 0x42, 0x95, - 0xc7, 0x58, 0x5e, 0x7b, 0x21, 0x9e, 0x8d, 0xc9, 0x25, 0x09, 0x9f, 0x02, 0x0a, 0x3f, 0x71, 0x36, - 0xc7, 0x34, 0xc1, 0x43, 0x85, 0x24, 0x0f, 0xa9, 0x7f, 0xa5, 0xc0, 0x62, 0x74, 0xb3, 0xb3, 0x1a, - 0xee, 0xf7, 0xa1, 0xce, 0xea, 0xb8, 0x3d, 0xa2, 0x42, 0xe4, 0x65, 0xd1, 0xc4, 0xe1, 0x69, 0x10, - 0xde, 0x64, 0x21, 0x84, 0x39, 0x76, 0xdc, 0x43, 0xd3, 0x1e, 0xf4, 0x08, 0x66, 0x22, 0x5b, 0xcc, - 0x07, 0x1f, 0x91, 0x31, 0xf5, 0x77, 0x14, 0xb8, 0xf2, 0x78, 0x64, 0xe8, 0x3e, 0x8e, 0x78, 0x30, - 0xb3, 0x76, 0x86, 0x8a, 0xd6, 0xcc, 0xc2, 0x84, 0x63, 0x8e, 0xec, 0xe7, 0xf1, 0xd6, 0x4c, 0xe2, - 0xf7, 0x71, 0x6c, 0x52, 0xbd, 0xd4, 0x67, 0xc7, 0xa6, 0x03, 0xd5, 0x23, 0xbe, 0x5c, 0x70, 0xc9, - 0x26, 0x78, 0x8e, 0xd5, 0x9d, 0x8b, 0xa7, 0xaa, 0x3b, 0xab, 0x5b, 0x70, 0x51, 0xc3, 0x1e, 0xb6, - 0x8d, 0xd8, 0x87, 0x9c, 0x39, 0x7f, 0x36, 0x82, 0x8e, 0x6c, 0xb9, 0x59, 0x38, 0x95, 0x39, 0xbe, - 0x3d, 0x97, 0x2c, 0xeb, 0x73, 0x65, 0x4d, 0xfc, 0x2d, 0xba, 0x8f, 0xaf, 0xfe, 0x75, 0x01, 0x2e, - 0xdc, 0x37, 0x0c, 0xae, 0xe7, 0xb9, 0x2b, 0xf7, 0xbc, 0xbc, 0xec, 0xa4, 0x17, 0x5a, 0x4c, 0x7b, - 0xa1, 0xcf, 0x4a, 0xf7, 0x72, 0x2b, 0x64, 0x8f, 0x87, 0x81, 0x09, 0x76, 0x59, 0x2f, 0xd9, 0xbb, - 0xbc, 0x3a, 0xdb, 0xb3, 0x9c, 0x01, 0x35, 0xc3, 0xd3, 0x9d, 0xb3, 0x6a, 0x90, 0x07, 0x54, 0x47, - 0xd0, 0x4e, 0x13, 0x6b, 0x46, 0x3d, 0x12, 0x50, 0x64, 0xe4, 0xb0, 0x1c, 0x73, 0x83, 0x78, 0x62, - 0x74, 0x68, 0xdb, 0xf1, 0xd4, 0xff, 0x2a, 0x40, 0x7b, 0x47, 0x3f, 0xc2, 0xff, 0x77, 0x0e, 0xe8, - 0x13, 0x58, 0xf2, 0xf4, 0x23, 0xdc, 0x8b, 0x04, 0xe0, 0x3d, 0x17, 0x7f, 0xc6, 0x9d, 0xd8, 0x57, - 0x65, 0x55, 0x00, 0x69, 0xb3, 0x94, 0xb6, 0xe8, 0xc5, 0xc6, 0x35, 0xfc, 0x19, 0x7a, 0x19, 0x16, - 0xa2, 0x8d, 0x7d, 0x41, 0x0a, 0xb6, 0xa1, 0xcd, 0x47, 0x9a, 0xf7, 0xba, 0x86, 0xfa, 0x19, 0xbc, - 0xf8, 0xd8, 0xf6, 0xb0, 0xdf, 0x0d, 0x1b, 0xd0, 0x66, 0x8c, 0x3f, 0xaf, 0x42, 0x3d, 0x24, 0x7c, - 0xea, 0x76, 0x8d, 0xe1, 0xa9, 0x0e, 0x74, 0xb6, 0x74, 0xf7, 0x30, 0xc8, 0x9c, 0x6f, 0xb0, 0xc6, - 0x9e, 0xe7, 0xb8, 0xe1, 0xbe, 0x68, 0x71, 0xd3, 0xf0, 0x3e, 0x76, 0xb1, 0xdd, 0xc7, 0x9b, 0x4e, - 0xff, 0x90, 0x38, 0x24, 0x3e, 0xbb, 0x17, 0xa9, 0x44, 0x7c, 0xd7, 0x8d, 0xc8, 0xfd, 0xc5, 0x42, - 0xec, 0xfe, 0xe2, 0x94, 0x2b, 0xbe, 0xea, 0x8f, 0x0a, 0xb0, 0x7c, 0xdf, 0xf2, 0xb1, 0x1b, 0x66, - 0x18, 0x4e, 0x93, 0x2c, 0x09, 0xb3, 0x17, 0x85, 0xb3, 0x64, 0x2f, 0x72, 0x14, 0x4b, 0x65, 0xb9, - 0x96, 0xd2, 0x19, 0x73, 0x2d, 0xf7, 0x01, 0x46, 0xae, 0x33, 0xc2, 0xae, 0x6f, 0xe2, 0x20, 0xf6, - 0xcb, 0xe1, 0xe0, 0x44, 0x26, 0xa9, 0x9f, 0x40, 0xeb, 0x61, 0x7f, 0xdd, 0xb1, 0xf7, 0x4d, 0x77, - 0x18, 0x10, 0x2a, 0x25, 0x74, 0x4a, 0x0e, 0xa1, 0x2b, 0xa4, 0x84, 0x4e, 0x35, 0x61, 0x31, 0xb2, - 0xf6, 0x8c, 0x8a, 0x6b, 0xd0, 0xef, 0xed, 0x9b, 0xb6, 0x49, 0x1b, 0xe7, 0x0a, 0xd4, 0x41, 0x85, - 0x41, 0xff, 0x01, 0x1f, 0x51, 0xbf, 0x54, 0xe0, 0x92, 0x86, 0x89, 0xf0, 0x04, 0x3d, 0x42, 0xbb, - 0xfe, 0x96, 0x37, 0x98, 0xc1, 0xa1, 0xb8, 0x07, 0xa5, 0xa1, 0x37, 0xc8, 0xa8, 0xef, 0x13, 0x13, - 0x1d, 0xdb, 0x48, 0xa3, 0xc0, 0xea, 0x4f, 0x14, 0x58, 0x0a, 0xaa, 0xa0, 0x31, 0x11, 0x8e, 0xb3, - 0xad, 0x92, 0xea, 0x0a, 0x9f, 0x70, 0x17, 0xfa, 0x02, 0xcc, 0x19, 0x7b, 0x51, 0x05, 0x59, 0x31, - 0xf6, 0xa8, 0x6e, 0x94, 0x78, 0xca, 0x25, 0xa9, 0xa7, 0x9c, 0x64, 0xfc, 0xb2, 0xa4, 0xbd, 0xea, - 0x31, 0xb4, 0xb9, 0x83, 0xf2, 0xd1, 0x08, 0xbb, 0x3a, 0xe5, 0xaf, 0x00, 0xf9, 0xb7, 0x03, 0x17, - 0x5a, 0xc9, 0xbc, 0x32, 0x98, 0xac, 0x80, 0x72, 0x27, 0x5a, 0xfd, 0x67, 0x05, 0xae, 0x25, 0xd7, - 0xdd, 0xe6, 0xf5, 0xc1, 0x99, 0x2f, 0xd1, 0xd3, 0xe2, 0x62, 0x21, 0x2c, 0x2e, 0xce, 0x54, 0x25, - 0x8d, 0x16, 0x32, 0x4b, 0xf1, 0x42, 0xe6, 0xad, 0xf7, 0x45, 0x5b, 0xfc, 0xee, 0xc9, 0x08, 0xa3, - 0x39, 0x28, 0x3e, 0xc2, 0xc7, 0xad, 0x73, 0x08, 0xa0, 0xf2, 0xc8, 0x71, 0x87, 0xba, 0xd5, 0x52, - 0x50, 0x1d, 0xe6, 0x78, 0xf1, 0xbb, 0x55, 0x40, 0xf3, 0x50, 0x5b, 0x0f, 0x0a, 0x82, 0xad, 0xe2, - 0xad, 0x5b, 0xd0, 0x88, 0xf6, 0x34, 0x92, 0x79, 0x9b, 0x78, 0xa0, 0xf7, 0x4f, 0x5a, 0xe7, 0x50, - 0x05, 0x0a, 0x9b, 0x77, 0x5b, 0x0a, 0xfd, 0xfb, 0x46, 0xab, 0x70, 0xeb, 0x4f, 0x15, 0x58, 0x4c, - 0x21, 0x89, 0x9a, 0x00, 0x8f, 0xed, 0x3e, 0xaf, 0x71, 0xb7, 0xce, 0xa1, 0x06, 0x54, 0x83, 0x8a, - 0x37, 0xdb, 0x7b, 0xd7, 0xa1, 0xd0, 0xad, 0x02, 0x6a, 0x41, 0x83, 0x4d, 0x1c, 0xf7, 0xfb, 0xd8, - 0xf3, 0x5a, 0x45, 0x31, 0xf2, 0x40, 0x37, 0xad, 0xb1, 0x8b, 0x5b, 0x25, 0x82, 0xdf, 0xae, 0xa3, - 0x61, 0x0b, 0xeb, 0x1e, 0x6e, 0x95, 0x11, 0x82, 0x26, 0x7f, 0x08, 0x26, 0x55, 0x22, 0x63, 0xc1, - 0xb4, 0xb9, 0x5b, 0x3f, 0x56, 0xa2, 0x65, 0x2f, 0x4a, 0x8b, 0x0b, 0x70, 0xfe, 0xb1, 0x6d, 0xe0, - 0x7d, 0xd3, 0xc6, 0x46, 0xf8, 0xaa, 0x75, 0x0e, 0x9d, 0x87, 0x85, 0x2d, 0xec, 0x0e, 0x70, 0x64, - 0xb0, 0x80, 0x16, 0x61, 0x7e, 0xcb, 0x7c, 0x1a, 0x19, 0x2a, 0xa2, 0x25, 0x68, 0xed, 0x98, 0xf6, - 0xc0, 0x8a, 0x02, 0x96, 0xe8, 0x6c, 0xd3, 0x76, 0xdc, 0xc8, 0x60, 0x99, 0x0e, 0xea, 0x9f, 0xc6, - 0x06, 0x2b, 0xa8, 0x03, 0xcb, 0x94, 0xa8, 0x77, 0x37, 0x30, 0xa1, 0x46, 0xe4, 0xdd, 0x9c, 0x5a, - 0xaa, 0x2a, 0x2d, 0x65, 0xf5, 0x27, 0x37, 0xa1, 0x46, 0x84, 0x75, 0xdd, 0x71, 0x5c, 0x03, 0x59, - 0x80, 0xe8, 0x3d, 0xb9, 0xe1, 0xc8, 0xb1, 0xc5, 0x9d, 0x5a, 0x74, 0x3b, 0x21, 0xdf, 0xec, 0x21, - 0x0d, 0xc8, 0x45, 0xa2, 0x73, 0x43, 0x0a, 0x9f, 0x00, 0x56, 0xcf, 0xa1, 0x21, 0xdd, 0x6d, 0xd7, - 0x1c, 0xe2, 0x5d, 0xb3, 0x7f, 0x18, 0x84, 0x00, 0x77, 0x33, 0x2e, 0x26, 0xa6, 0x41, 0x83, 0xfd, - 0xae, 0x4b, 0xf7, 0x63, 0x17, 0x19, 0x03, 0x39, 0x52, 0xcf, 0xa1, 0xcf, 0xa8, 0xfa, 0x09, 0xe3, - 0xa9, 0x60, 0xc3, 0xd5, 0xec, 0x0d, 0x53, 0xc0, 0xa7, 0xdc, 0x72, 0x13, 0xca, 0x94, 0xef, 0x91, - 0xac, 0xc9, 0x21, 0xfa, 0x1b, 0x1f, 0x9d, 0x6b, 0xd9, 0x00, 0x62, 0xb5, 0x4f, 0x61, 0x21, 0x71, - 0x55, 0x1e, 0xc9, 0x7c, 0x30, 0xf9, 0x8f, 0x1e, 0x74, 0x6e, 0xe5, 0x01, 0x15, 0x7b, 0x0d, 0xa0, - 0x19, 0xbf, 0x5f, 0x87, 0x56, 0x72, 0xdc, 0xd2, 0x65, 0x3b, 0xbd, 0x9a, 0xfb, 0x3e, 0x2f, 0x65, - 0x82, 0x56, 0xf2, 0x12, 0x37, 0xba, 0x35, 0x71, 0x81, 0x38, 0xb3, 0x7d, 0x23, 0x17, 0xac, 0xd8, - 0xee, 0x84, 0x32, 0x41, 0xea, 0x06, 0x6d, 0x92, 0xc7, 0x83, 0x65, 0xb2, 0xae, 0xf6, 0x76, 0xee, - 0xe4, 0x86, 0x17, 0x5b, 0xff, 0x2a, 0xeb, 0x64, 0x94, 0xdd, 0x42, 0x45, 0x6f, 0xc8, 0x97, 0x9b, - 0x70, 0x7d, 0xb6, 0xb3, 0x7a, 0x9a, 0x29, 0x02, 0x89, 0xef, 0xd3, 0x16, 0x44, 0xc9, 0x3d, 0xce, - 0xa4, 0xdc, 0x05, 0xeb, 0x65, 0x5f, 0x51, 0xed, 0xbc, 0x71, 0x8a, 0x19, 0x02, 0x01, 0x27, 0x79, - 0x4b, 0x3e, 0x10, 0xc3, 0x3b, 0x53, 0xb9, 0xe6, 0x6c, 0x32, 0xf8, 0x3d, 0x58, 0x48, 0x44, 0x25, - 0x28, 0x7f, 0xe4, 0xd2, 0x99, 0x64, 0x6e, 0x99, 0x48, 0x26, 0x5a, 0x0e, 0x51, 0x06, 0xf7, 0x4b, - 0xda, 0x12, 0x3b, 0xb7, 0xf2, 0x80, 0x8a, 0x0f, 0x19, 0xc1, 0x62, 0xe2, 0xe5, 0x93, 0x55, 0xf4, - 0x8d, 0xdc, 0xbb, 0x3d, 0x59, 0xed, 0xbc, 0x96, 0x7f, 0xbf, 0x27, 0xab, 0xea, 0x39, 0xe4, 0x51, - 0x05, 0x9d, 0x68, 0x5b, 0x43, 0x19, 0xab, 0xc8, 0xdb, 0xf3, 0x3a, 0xaf, 0xe7, 0x84, 0x16, 0x9f, - 0x79, 0x04, 0xe7, 0x25, 0xdd, 0x85, 0xe8, 0xf5, 0x89, 0xec, 0x91, 0x6c, 0xab, 0xec, 0xdc, 0xce, - 0x0b, 0x1e, 0x31, 0x0f, 0xad, 0x00, 0xaf, 0xfb, 0x96, 0xc5, 0x3c, 0x8b, 0xd7, 0xb2, 0x2c, 0x5f, - 0x0c, 0x2c, 0xe3, 0x53, 0x33, 0xa1, 0xc5, 0x96, 0xbf, 0x0c, 0x68, 0xe7, 0xc0, 0x39, 0xa6, 0x51, - 0xc0, 0x60, 0xcc, 0x1d, 0xcb, 0x4c, 0x03, 0x98, 0x06, 0xcd, 0x10, 0xc4, 0x89, 0x33, 0xc4, 0xe6, - 0x3d, 0x80, 0x87, 0xd8, 0xdf, 0xc2, 0xbe, 0x4b, 0xa4, 0xff, 0xe5, 0x2c, 0xdc, 0x39, 0x40, 0xb0, - 0xd5, 0x2b, 0x53, 0xe1, 0xa2, 0x04, 0xdd, 0xd2, 0xed, 0xb1, 0x6e, 0x45, 0x2e, 0xa9, 0xc9, 0x09, - 0x9a, 0x04, 0x9b, 0x4c, 0xd0, 0x34, 0xb4, 0xd8, 0xf2, 0x58, 0xf8, 0x2f, 0x91, 0x2e, 0x84, 0xc9, - 0xfe, 0x4b, 0xba, 0xd9, 0x2e, 0xa9, 0xdb, 0x27, 0xc0, 0x8b, 0x8d, 0xbf, 0x50, 0x68, 0x4f, 0x6c, - 0x02, 0xe0, 0x63, 0xd3, 0x3f, 0xd8, 0xb6, 0x74, 0xdb, 0xcb, 0x83, 0x02, 0x05, 0x3c, 0x05, 0x0a, - 0x1c, 0x5e, 0xa0, 0x60, 0xc0, 0x7c, 0xac, 0xe2, 0x8f, 0x64, 0xb7, 0xb0, 0x64, 0x8d, 0x12, 0x9d, - 0x95, 0xe9, 0x80, 0x62, 0x97, 0x7d, 0x98, 0x8f, 0xc5, 0x70, 0xd2, 0x5d, 0x64, 0x51, 0x5e, 0x52, - 0xd9, 0x25, 0xa4, 0x23, 0x49, 0x50, 0x0f, 0x50, 0xba, 0xb0, 0x89, 0xf2, 0x95, 0xc1, 0x27, 0xa9, - 0x9e, 0xec, 0x6a, 0x29, 0xd3, 0xe6, 0x89, 0xd6, 0x01, 0xb9, 0xa9, 0x90, 0x76, 0x42, 0x48, 0xb5, - 0x79, 0x46, 0x27, 0x82, 0x7a, 0x0e, 0x7d, 0x0c, 0x15, 0xfe, 0x1b, 0x58, 0x37, 0x26, 0x97, 0x10, - 0xf8, 0xea, 0x37, 0xa7, 0x40, 0x89, 0x85, 0x0f, 0xe1, 0x42, 0x46, 0x01, 0x41, 0xea, 0x65, 0x4c, - 0x2e, 0x36, 0x4c, 0xb3, 0x7f, 0x62, 0xb3, 0x54, 0x7d, 0x60, 0xc2, 0x66, 0x59, 0xb5, 0x84, 0x69, - 0x9b, 0xf5, 0x60, 0x31, 0x95, 0x7f, 0x95, 0x1a, 0xc0, 0xac, 0x2c, 0xed, 0xb4, 0x0d, 0x06, 0xf0, - 0x82, 0x34, 0xd7, 0x28, 0xf5, 0x4d, 0x26, 0x65, 0x25, 0xa7, 0x6d, 0xd4, 0x87, 0xf3, 0x92, 0x0c, - 0xa3, 0xd4, 0xc6, 0x65, 0x67, 0x22, 0xa7, 0x6d, 0xb2, 0x0f, 0x9d, 0x35, 0xd7, 0xd1, 0x8d, 0xbe, - 0xee, 0xf9, 0x34, 0xeb, 0x47, 0x82, 0xd0, 0xc0, 0x39, 0x94, 0x47, 0x0e, 0xd2, 0xdc, 0xe0, 0xb4, - 0x7d, 0xf6, 0xa0, 0x4e, 0x8f, 0x92, 0xfd, 0xe0, 0x10, 0x92, 0x5b, 0x88, 0x08, 0x44, 0x86, 0xda, - 0x91, 0x01, 0x0a, 0xa6, 0xde, 0x85, 0xfa, 0x3a, 0x2d, 0x9f, 0x76, 0x6d, 0x03, 0x3f, 0x4d, 0x5a, - 0x2b, 0xfa, 0xab, 0x0b, 0xb7, 0x23, 0x00, 0xb9, 0x29, 0x34, 0x4f, 0x7d, 0x76, 0x03, 0x3f, 0x65, - 0xe7, 0xbc, 0x22, 0x5b, 0x37, 0x06, 0x92, 0x11, 0xe3, 0x48, 0x21, 0x23, 0x76, 0x7e, 0x29, 0xea, - 0xc9, 0x8a, 0xed, 0xee, 0x64, 0x2c, 0x92, 0x82, 0x0c, 0x76, 0xbd, 0x9b, 0x7f, 0x42, 0xd4, 0x2e, - 0x04, 0x78, 0x75, 0x69, 0xed, 0xf6, 0x95, 0x49, 0xa8, 0x47, 0xdd, 0xd3, 0x95, 0xe9, 0x80, 0x62, - 0x97, 0x6d, 0xa8, 0x11, 0xee, 0x64, 0xc7, 0x73, 0x43, 0x36, 0x51, 0xbc, 0xce, 0x7f, 0x38, 0x1b, - 0xd8, 0xeb, 0xbb, 0xe6, 0x1e, 0x3f, 0x74, 0x29, 0x3a, 0x31, 0x90, 0x89, 0x87, 0x93, 0x80, 0x14, - 0x98, 0x8f, 0xa9, 0xcf, 0x20, 0x48, 0xc7, 0x55, 0xe5, 0xeb, 0xd3, 0xce, 0x37, 0xae, 0x26, 0x6f, - 0xe7, 0x05, 0x17, 0xdb, 0xfe, 0x0a, 0x8d, 0x83, 0xe8, 0xfb, 0xb5, 0xb1, 0x69, 0x19, 0x41, 0xe2, - 0x0f, 0xdd, 0x9d, 0xb4, 0x54, 0x0c, 0x34, 0xd3, 0xfd, 0x9b, 0x30, 0x43, 0xec, 0xff, 0x8b, 0x50, - 0x13, 0xf9, 0x67, 0x24, 0xcb, 0x5a, 0x26, 0x33, 0xdf, 0x9d, 0x1b, 0x93, 0x81, 0xc4, 0xca, 0x18, - 0x96, 0x64, 0xd9, 0x66, 0x69, 0x88, 0x3d, 0x21, 0x2d, 0x3d, 0x85, 0x3f, 0x56, 0xbf, 0x6c, 0x40, - 0x35, 0x98, 0xf8, 0x15, 0x27, 0xae, 0xbe, 0x86, 0x4c, 0xd2, 0xf7, 0x60, 0x21, 0xf1, 0x3b, 0x32, - 0x52, 0x0d, 0x2e, 0xff, 0xad, 0x99, 0x69, 0xa2, 0xf6, 0x31, 0xff, 0xe5, 0x56, 0x11, 0xe2, 0xbd, - 0x92, 0x95, 0x8d, 0x4a, 0x46, 0x77, 0x53, 0x16, 0xfe, 0xdf, 0x1d, 0xe0, 0x3c, 0x02, 0x88, 0x84, - 0x36, 0x93, 0xef, 0x06, 0x10, 0x6f, 0x7d, 0x1a, 0xb5, 0x86, 0xd2, 0xe8, 0xe5, 0xd5, 0x3c, 0x7d, - 0xd6, 0xd9, 0x1e, 0x68, 0x76, 0xcc, 0xf2, 0x18, 0x1a, 0xd1, 0x0b, 0x42, 0x48, 0xfa, 0xa3, 0x9a, - 0xe9, 0x1b, 0x44, 0xd3, 0xbe, 0x62, 0xeb, 0x94, 0x8e, 0xed, 0x94, 0xe5, 0x3c, 0x40, 0xe9, 0x3e, - 0x0c, 0x69, 0x20, 0x90, 0xd9, 0xfd, 0x21, 0x0d, 0x04, 0xb2, 0x9b, 0x3b, 0x58, 0x52, 0x32, 0xd9, - 0x5c, 0x20, 0x4d, 0x4a, 0x66, 0xb4, 0x6b, 0x48, 0x93, 0x92, 0x59, 0xdd, 0x0a, 0x11, 0xf9, 0x9b, - 0x18, 0xba, 0xc9, 0x7e, 0x58, 0x78, 0x1a, 0xf1, 0x0c, 0x58, 0x7e, 0xe4, 0xf8, 0xe6, 0xfe, 0x49, - 0xb2, 0xcc, 0x24, 0x75, 0x9b, 0xb3, 0x6a, 0x5c, 0xd3, 0xa5, 0xfc, 0x32, 0xf5, 0xda, 0xb2, 0x6a, - 0x59, 0x28, 0x4f, 0x51, 0xac, 0x73, 0x2f, 0x07, 0x46, 0x69, 0x3b, 0xb6, 0x76, 0xef, 0x93, 0x37, - 0x06, 0xa6, 0x7f, 0x30, 0xde, 0x23, 0x68, 0xdd, 0x61, 0x4b, 0xbc, 0x6e, 0x3a, 0xfc, 0xbf, 0x3b, - 0x81, 0xaa, 0xb8, 0x43, 0x57, 0xbd, 0x43, 0x56, 0x1d, 0xed, 0xed, 0x55, 0xe8, 0xd3, 0xbd, 0xff, - 0x09, 0x00, 0x00, 0xff, 0xff, 0x39, 0x92, 0xbc, 0xad, 0xf1, 0x5b, 0x00, 0x00, + 0xe3, 0xee, 0x1e, 0x52, 0x74, 0x80, 0xac, 0x37, 0x71, 0x82, 0xbc, 0x90, 0x07, 0x92, 0x1c, 0x72, + 0x0b, 0x72, 0x08, 0xf2, 0xda, 0xd3, 0x26, 0x58, 0x20, 0x08, 0xb0, 0x48, 0x02, 0x04, 0xbb, 0xc8, + 0x21, 0x97, 0x1c, 0x92, 0x43, 0xfe, 0x8a, 0x20, 0xf7, 0xa0, 0x1e, 0x5d, 0xfd, 0xaa, 0x9e, 0x69, + 0x72, 0x24, 0x1b, 0x48, 0x4e, 0x64, 0x57, 0x7f, 0x55, 0xf5, 0xd5, 0x57, 0xdf, 0xfb, 0xfb, 0x7a, + 0xa0, 0x65, 0xe8, 0xbe, 0xde, 0xeb, 0x3b, 0x8e, 0x6b, 0xdc, 0x1e, 0xb9, 0x8e, 0xef, 0xa0, 0xc5, + 0xa1, 0x69, 0x1d, 0x8d, 0x3d, 0xf6, 0x74, 0x9b, 0xbc, 0xee, 0x34, 0xfa, 0xce, 0x70, 0xe8, 0xd8, + 0x6c, 0xa8, 0xd3, 0x34, 0x6d, 0x1f, 0xbb, 0xb6, 0x6e, 0xf1, 0xe7, 0x46, 0x74, 0x42, 0xa7, 0xe1, + 0xf5, 0x0f, 0xf0, 0x50, 0xe7, 0x4f, 0xb5, 0xa1, 0x37, 0xe0, 0xff, 0x2e, 0x9a, 0xb6, 0x81, 0x9f, + 0x46, 0xb7, 0x52, 0xe7, 0xa0, 0xfc, 0xc1, 0x70, 0xe4, 0x9f, 0xa8, 0x7f, 0xa3, 0x40, 0xe3, 0x81, + 0x35, 0xf6, 0x0e, 0x34, 0xfc, 0xd9, 0x18, 0x7b, 0x3e, 0xba, 0x0b, 0xa5, 0x3d, 0xdd, 0xc3, 0x6d, + 0xe5, 0x9a, 0xb2, 0x52, 0x5f, 0x7d, 0xe1, 0x76, 0x0c, 0x27, 0x8e, 0xcd, 0x96, 0x37, 0x58, 0xd3, + 0x3d, 0xac, 0x51, 0x48, 0x84, 0xa0, 0x64, 0xec, 0x75, 0x37, 0xda, 0x85, 0x6b, 0xca, 0x4a, 0x51, + 0xa3, 0xff, 0xa3, 0x2b, 0x00, 0x1e, 0x1e, 0x0c, 0xb1, 0xed, 0x77, 0x37, 0xbc, 0x76, 0xf1, 0x5a, + 0x71, 0xa5, 0xa8, 0x45, 0x46, 0x90, 0x0a, 0x8d, 0xbe, 0x63, 0x59, 0xb8, 0xef, 0x9b, 0x8e, 0xdd, + 0xdd, 0x68, 0x97, 0xe8, 0xdc, 0xd8, 0x18, 0xea, 0x40, 0xd5, 0xf4, 0xba, 0xc3, 0x91, 0xe3, 0xfa, + 0xed, 0xf2, 0x35, 0x65, 0xa5, 0xaa, 0x89, 0x67, 0xf5, 0x07, 0x05, 0x98, 0xe7, 0x68, 0x7b, 0x23, + 0xc7, 0xf6, 0x30, 0xba, 0x07, 0x15, 0xcf, 0xd7, 0xfd, 0xb1, 0xc7, 0x31, 0xbf, 0x24, 0xc5, 0x7c, + 0x87, 0x82, 0x68, 0x1c, 0x54, 0x8a, 0x7a, 0x12, 0xb5, 0xa2, 0x04, 0xb5, 0xf8, 0xf1, 0x4a, 0xa9, + 0xe3, 0xad, 0xc0, 0xc2, 0x3e, 0xc1, 0x6e, 0x27, 0x04, 0x2a, 0x53, 0xa0, 0xe4, 0x30, 0x59, 0xc9, + 0x37, 0x87, 0xf8, 0xa3, 0xfd, 0x1d, 0xac, 0x5b, 0xed, 0x0a, 0xdd, 0x2b, 0x32, 0x82, 0x2e, 0x42, + 0x95, 0x4e, 0xe9, 0xf9, 0x5e, 0x7b, 0xee, 0x9a, 0xb2, 0x52, 0xd2, 0xe6, 0xe8, 0xf3, 0xae, 0xa7, + 0x7e, 0x1f, 0x96, 0x28, 0x09, 0xd6, 0x0f, 0x74, 0xdb, 0xc6, 0x96, 0x77, 0xf6, 0x1b, 0x8c, 0x6e, + 0x52, 0x88, 0x6d, 0x42, 0x2e, 0xa1, 0xcf, 0xd7, 0xa7, 0xd7, 0x58, 0xd3, 0xc4, 0xb3, 0xfa, 0xeb, + 0x05, 0x68, 0x89, 0xa3, 0x04, 0xbb, 0x2f, 0x41, 0xb9, 0xef, 0x8c, 0x6d, 0x9f, 0x6e, 0x3f, 0xaf, + 0xb1, 0x07, 0xf4, 0x22, 0x34, 0xf8, 0xb4, 0x9e, 0xad, 0x0f, 0x31, 0xdd, 0xa5, 0xa6, 0xd5, 0xf9, + 0xd8, 0x23, 0x7d, 0x88, 0x73, 0xd1, 0xfd, 0x1a, 0xd4, 0x47, 0xba, 0xeb, 0x9b, 0x31, 0xae, 0x89, + 0x0e, 0x4d, 0x62, 0x1a, 0xb2, 0x83, 0x49, 0xff, 0xdb, 0xd5, 0xbd, 0xc3, 0xee, 0x06, 0xa7, 0x76, + 0x6c, 0x0c, 0xbd, 0x01, 0x65, 0x0b, 0x1f, 0x61, 0x8b, 0x12, 0xbb, 0xb9, 0x7a, 0xf5, 0x76, 0x4a, + 0x26, 0x6f, 0xf3, 0x23, 0x6f, 0x12, 0x30, 0x8d, 0x41, 0xab, 0x7f, 0xa2, 0xc0, 0xf2, 0x7d, 0xcf, + 0x33, 0x07, 0x76, 0x8a, 0x20, 0xcb, 0x50, 0xb1, 0x1d, 0x03, 0x77, 0x37, 0x28, 0x45, 0x8a, 0x1a, + 0x7f, 0x42, 0x97, 0xa0, 0x36, 0xc2, 0xd8, 0xed, 0xb9, 0x8e, 0x15, 0xd0, 0xa3, 0x4a, 0x06, 0x34, + 0xc7, 0xc2, 0xe8, 0xbb, 0xb0, 0xe8, 0x25, 0x16, 0x62, 0xf4, 0xaf, 0xaf, 0xbe, 0x94, 0x8d, 0x92, + 0x80, 0xd5, 0xd2, 0xb3, 0xd5, 0x2f, 0x0a, 0x70, 0x5e, 0xc0, 0x31, 0x5c, 0xc9, 0xff, 0xe4, 0xc2, + 0x3c, 0x3c, 0x10, 0xe8, 0xb1, 0x87, 0x3c, 0x17, 0x26, 0x6e, 0xba, 0x18, 0xbd, 0xe9, 0x3c, 0x92, + 0x9d, 0xb8, 0xc6, 0x72, 0xfa, 0x1a, 0xaf, 0x42, 0x1d, 0x3f, 0x1d, 0x99, 0x2e, 0xee, 0x11, 0x59, + 0xa0, 0x37, 0x55, 0xd2, 0x80, 0x0d, 0xed, 0x9a, 0xc3, 0xa8, 0xb8, 0xcf, 0xe5, 0x16, 0x77, 0xf5, + 0x4f, 0x15, 0xb8, 0x90, 0xba, 0x25, 0xae, 0x3f, 0x34, 0x68, 0xd1, 0x93, 0x87, 0x94, 0x21, 0x9a, + 0x84, 0x10, 0xfc, 0xe5, 0x49, 0x04, 0x0f, 0xc1, 0xb5, 0xd4, 0xfc, 0x08, 0x92, 0x85, 0xfc, 0x48, + 0x1e, 0xc2, 0x85, 0x87, 0xd8, 0xe7, 0x1b, 0x90, 0x77, 0x78, 0x06, 0xc9, 0x8e, 0x2b, 0xaa, 0x42, + 0x52, 0x51, 0xa9, 0x7f, 0x16, 0x8a, 0x30, 0xdd, 0xaa, 0x6b, 0xef, 0x3b, 0xe8, 0x05, 0xa8, 0x09, + 0x10, 0xce, 0x15, 0xe1, 0x00, 0xfa, 0x16, 0x94, 0x09, 0xa6, 0x8c, 0x25, 0x9a, 0xab, 0x2f, 0xca, + 0xcf, 0x14, 0x59, 0x53, 0x63, 0xf0, 0x68, 0x03, 0x9a, 0x9e, 0xaf, 0xbb, 0x7e, 0x6f, 0xe4, 0x78, + 0xf4, 0x9e, 0x29, 0xe3, 0xd4, 0x57, 0x2f, 0xc7, 0x57, 0x20, 0x76, 0x6b, 0xcb, 0x1b, 0x6c, 0x73, + 0x20, 0x6d, 0x9e, 0x4e, 0x0a, 0x1e, 0xd1, 0xfb, 0xd0, 0xc0, 0xb6, 0x11, 0xae, 0x51, 0xca, 0xb3, + 0x46, 0x1d, 0xdb, 0x86, 0x58, 0x21, 0xbc, 0x95, 0x72, 0xfe, 0x5b, 0xf9, 0x6d, 0x05, 0xda, 0xe9, + 0x6b, 0x99, 0xc5, 0xf6, 0xbc, 0xc3, 0x26, 0x61, 0x76, 0x2d, 0x13, 0xe5, 0x5a, 0x5c, 0x8d, 0xc6, + 0xa7, 0xa8, 0x7f, 0xa4, 0xc0, 0x37, 0x42, 0x74, 0xe8, 0xab, 0xe7, 0xc5, 0x23, 0xe8, 0x16, 0xb4, + 0x4c, 0xbb, 0x6f, 0x8d, 0x0d, 0xfc, 0xd8, 0xfe, 0x10, 0xeb, 0x96, 0x7f, 0x70, 0x42, 0x6f, 0xae, + 0xaa, 0xa5, 0xc6, 0xd5, 0xff, 0x28, 0xc0, 0x72, 0x12, 0xaf, 0x59, 0x88, 0xf4, 0xff, 0xa0, 0x6c, + 0xda, 0xfb, 0x4e, 0x40, 0xa3, 0x2b, 0x13, 0x44, 0x91, 0xec, 0xc5, 0x80, 0x91, 0x03, 0x28, 0x50, + 0x5e, 0xfd, 0x03, 0xdc, 0x3f, 0x1c, 0x39, 0x26, 0x55, 0x53, 0x64, 0x89, 0xf7, 0x25, 0x4b, 0xc8, + 0x31, 0xbe, 0xcd, 0x0d, 0xeb, 0xba, 0x58, 0xe2, 0x03, 0xdb, 0x77, 0x4f, 0xb4, 0xc5, 0x7e, 0x72, + 0xbc, 0xd3, 0x87, 0x65, 0x39, 0x30, 0x6a, 0x41, 0xf1, 0x10, 0x9f, 0xd0, 0x23, 0xd7, 0x34, 0xf2, + 0x2f, 0xba, 0x07, 0xe5, 0x23, 0xdd, 0x1a, 0x63, 0xae, 0x13, 0xa6, 0x70, 0x2e, 0x83, 0x7d, 0xbb, + 0xf0, 0xa6, 0xa2, 0x0e, 0xe1, 0xd2, 0x43, 0xec, 0x77, 0x6d, 0x0f, 0xbb, 0xfe, 0x9a, 0x69, 0x5b, + 0xce, 0x60, 0x5b, 0xf7, 0x0f, 0x66, 0x50, 0x0e, 0x31, 0x39, 0x2f, 0x24, 0xe4, 0x5c, 0xfd, 0x73, + 0x05, 0x5e, 0x90, 0xef, 0xc7, 0x2f, 0xb4, 0x03, 0xd5, 0x7d, 0x13, 0x5b, 0x06, 0xe1, 0x1a, 0x85, + 0x72, 0x8d, 0x78, 0x26, 0x4a, 0x62, 0x44, 0x80, 0xf9, 0xbd, 0x25, 0x94, 0x84, 0x70, 0x63, 0x77, + 0x7c, 0xd7, 0xb4, 0x07, 0x9b, 0xa6, 0xe7, 0x6b, 0x0c, 0x3e, 0xc2, 0x25, 0xc5, 0xfc, 0xc2, 0xf9, + 0x9b, 0x0a, 0x5c, 0x79, 0x88, 0xfd, 0x75, 0x61, 0x63, 0xc8, 0x7b, 0xd3, 0xf3, 0xcd, 0xbe, 0xf7, + 0x6c, 0xdd, 0xda, 0x1c, 0x3e, 0x8a, 0xfa, 0xbb, 0x0a, 0x5c, 0xcd, 0x44, 0x86, 0x93, 0x8e, 0xeb, + 0xd0, 0xc0, 0xc2, 0xc8, 0x75, 0xe8, 0xcf, 0xe1, 0x93, 0x27, 0xe4, 0xf2, 0xb7, 0x75, 0xd3, 0x65, + 0x3a, 0xf4, 0x8c, 0x16, 0xe5, 0x87, 0x0a, 0x5c, 0x7e, 0x88, 0xfd, 0xed, 0xc0, 0xbe, 0x7e, 0x8d, + 0xd4, 0x21, 0x30, 0x11, 0x3b, 0x1f, 0xf8, 0xce, 0xb1, 0x31, 0xf5, 0x77, 0xd8, 0x75, 0x4a, 0xf1, + 0xfd, 0x5a, 0x08, 0x78, 0x85, 0x4a, 0x42, 0x44, 0x45, 0x70, 0x61, 0xe7, 0xe4, 0x53, 0xbf, 0x2c, + 0x43, 0xe3, 0x09, 0xd7, 0x0a, 0xd4, 0x82, 0x26, 0x29, 0xa1, 0xc8, 0x9d, 0xa0, 0x88, 0x37, 0x25, + 0x73, 0xb0, 0xd6, 0x60, 0xde, 0xc3, 0xf8, 0xf0, 0x94, 0xf6, 0xb2, 0x41, 0xe6, 0x08, 0x63, 0xb7, + 0x09, 0x8b, 0x63, 0x9b, 0x3a, 0xf3, 0xd8, 0xe0, 0x07, 0x60, 0x44, 0x9f, 0xae, 0x4c, 0xd3, 0x13, + 0xd1, 0x87, 0x3c, 0xae, 0x89, 0xac, 0x55, 0xce, 0xb5, 0x56, 0x72, 0x1a, 0xea, 0x42, 0xcb, 0x70, + 0x9d, 0xd1, 0x08, 0x1b, 0x3d, 0x2f, 0x58, 0xaa, 0x92, 0x6f, 0x29, 0x3e, 0x4f, 0x2c, 0x75, 0x17, + 0xce, 0x27, 0x31, 0xed, 0x1a, 0xc4, 0x2f, 0x24, 0x9c, 0x25, 0x7b, 0x85, 0x5e, 0x85, 0xc5, 0x34, + 0x7c, 0x95, 0xc2, 0xa7, 0x5f, 0xa0, 0xd7, 0x00, 0x25, 0x50, 0x25, 0xe0, 0x35, 0x06, 0x1e, 0x47, + 0x86, 0x83, 0xd3, 0x78, 0x3b, 0x0e, 0x0e, 0x0c, 0x9c, 0xbf, 0x89, 0x80, 0x77, 0x89, 0x75, 0x8d, + 0x81, 0x7b, 0xed, 0x7a, 0x3e, 0x42, 0xc4, 0x17, 0xf3, 0xd4, 0xdf, 0x50, 0x60, 0xf9, 0x63, 0xdd, + 0xef, 0x1f, 0x6c, 0x0c, 0x67, 0x8f, 0x09, 0xdf, 0x85, 0xda, 0x91, 0x88, 0xfc, 0x98, 0x16, 0x97, + 0x05, 0x43, 0x51, 0xb6, 0xd7, 0xc2, 0x19, 0xea, 0x3f, 0x2a, 0x3c, 0x3a, 0x0d, 0xb0, 0xfb, 0xea, + 0x55, 0xcd, 0xb4, 0x20, 0x3d, 0x21, 0x80, 0xe5, 0x94, 0x00, 0xaa, 0x4f, 0x01, 0x38, 0xfa, 0x5b, + 0xde, 0xe0, 0x0c, 0x98, 0xbf, 0x09, 0x73, 0x7c, 0x3f, 0xae, 0x6d, 0xa6, 0x5d, 0x69, 0x00, 0xae, + 0xfe, 0x77, 0x05, 0xea, 0x91, 0x17, 0xa8, 0x09, 0x05, 0xa1, 0x46, 0x0a, 0x92, 0xf3, 0x17, 0xa6, + 0x47, 0x59, 0xc5, 0x74, 0x94, 0x75, 0x03, 0x9a, 0x26, 0x35, 0xef, 0x3d, 0x7e, 0x6a, 0xea, 0x4d, + 0xd7, 0xb4, 0x79, 0x36, 0xca, 0x99, 0x08, 0x5d, 0x81, 0xba, 0x3d, 0x1e, 0xf6, 0x9c, 0xfd, 0x9e, + 0xeb, 0x1c, 0x7b, 0x3c, 0x5c, 0xab, 0xd9, 0xe3, 0xe1, 0x47, 0xfb, 0x9a, 0x73, 0xec, 0x85, 0x11, + 0x41, 0xe5, 0x94, 0x11, 0xc1, 0x15, 0xa8, 0x0f, 0xf5, 0xa7, 0x64, 0xd5, 0x9e, 0x3d, 0x1e, 0xd2, + 0x48, 0xae, 0xa8, 0xd5, 0x86, 0xfa, 0x53, 0xcd, 0x39, 0x7e, 0x34, 0x1e, 0xa2, 0x15, 0x68, 0x59, + 0xba, 0xe7, 0xf7, 0xa2, 0xa1, 0x60, 0x95, 0x86, 0x82, 0x4d, 0x32, 0xfe, 0x41, 0x18, 0x0e, 0xa6, + 0x63, 0x8b, 0xda, 0xd9, 0x62, 0x0b, 0x63, 0x68, 0x85, 0x6b, 0x40, 0xae, 0xd8, 0xc2, 0x18, 0x5a, + 0x62, 0x85, 0x37, 0x61, 0x6e, 0x8f, 0xba, 0x4a, 0x93, 0x84, 0xf8, 0x01, 0xf1, 0x92, 0x98, 0x47, + 0xa5, 0x05, 0xe0, 0xe8, 0xdb, 0x50, 0xa3, 0x16, 0x8a, 0xce, 0x6d, 0xe4, 0x9a, 0x1b, 0x4e, 0x20, + 0xb3, 0x0d, 0x6c, 0xf9, 0x3a, 0x9d, 0x3d, 0x9f, 0x6f, 0xb6, 0x98, 0x40, 0x34, 0x68, 0xdf, 0xc5, + 0xba, 0x8f, 0x8d, 0xb5, 0x93, 0x75, 0x67, 0x38, 0xd2, 0x29, 0x0b, 0xb5, 0x9b, 0xd4, 0xc9, 0x97, + 0xbd, 0x42, 0x2f, 0x43, 0xb3, 0x2f, 0x9e, 0x1e, 0xb8, 0xce, 0xb0, 0xbd, 0x40, 0xe5, 0x2b, 0x31, + 0x8a, 0x2e, 0x03, 0x04, 0xba, 0x53, 0xf7, 0xdb, 0x2d, 0x7a, 0x77, 0x35, 0x3e, 0x72, 0x9f, 0xa6, + 0x85, 0x4c, 0xaf, 0xc7, 0x12, 0x30, 0xa6, 0x3d, 0x68, 0x2f, 0xd2, 0x1d, 0xeb, 0x41, 0xc6, 0xc6, + 0xb4, 0x07, 0xe8, 0x02, 0xcc, 0x99, 0x5e, 0x6f, 0x5f, 0x3f, 0xc4, 0x6d, 0x44, 0xdf, 0x56, 0x4c, + 0xef, 0x81, 0x7e, 0x48, 0xbd, 0x57, 0xbe, 0x19, 0x36, 0xda, 0xe7, 0xe9, 0xab, 0x70, 0x20, 0xcc, + 0xe3, 0x2c, 0x9d, 0x2a, 0x8f, 0xf3, 0x39, 0x2c, 0x85, 0x8c, 0x1a, 0xe1, 0x8c, 0x34, 0x7f, 0x29, + 0x67, 0xe0, 0xaf, 0xc9, 0x0e, 0xf7, 0xcf, 0xca, 0xb0, 0xbc, 0xa3, 0x1f, 0xe1, 0xe7, 0xef, 0xdb, + 0xe7, 0x52, 0x9f, 0x9b, 0xb0, 0x48, 0xdd, 0xf9, 0xd5, 0x08, 0x3e, 0x13, 0x3c, 0x87, 0x28, 0x6b, + 0xa5, 0x27, 0xa2, 0xef, 0x10, 0x65, 0x8b, 0xfb, 0x87, 0xdb, 0x24, 0x34, 0x0a, 0xbc, 0x86, 0xcb, + 0x92, 0x75, 0xd6, 0x05, 0x94, 0x16, 0x9d, 0x81, 0xb6, 0x61, 0x21, 0x7e, 0x03, 0x81, 0xbf, 0x70, + 0x73, 0x62, 0xdc, 0x1c, 0x52, 0x5f, 0x6b, 0xc6, 0x2e, 0xc3, 0x43, 0x6d, 0x98, 0xe3, 0xc6, 0x9e, + 0x6a, 0x9e, 0xaa, 0x16, 0x3c, 0xa2, 0x6d, 0x38, 0xcf, 0x4e, 0xb0, 0xc3, 0x05, 0x8c, 0x1d, 0xbe, + 0x9a, 0xeb, 0xf0, 0xb2, 0xa9, 0x71, 0xf9, 0xac, 0x9d, 0x56, 0x3e, 0xdb, 0x30, 0xc7, 0x65, 0x86, + 0xaa, 0xa4, 0xaa, 0x16, 0x3c, 0x92, 0x6b, 0x0e, 0xa5, 0xa7, 0xce, 0x84, 0x40, 0x0c, 0x90, 0x79, + 0x81, 0x62, 0x6f, 0x50, 0xc5, 0x1e, 0x3c, 0x52, 0x6d, 0x83, 0x07, 0x3d, 0x26, 0x22, 0xf3, 0xf9, + 0x44, 0xa4, 0xea, 0xe1, 0x01, 0xfd, 0x2f, 0x69, 0x59, 0x9a, 0x29, 0xcb, 0xa2, 0xfe, 0xaa, 0x02, + 0x10, 0xde, 0xe4, 0x94, 0x8c, 0xd2, 0x5b, 0x50, 0x15, 0x62, 0x95, 0x2b, 0x28, 0x16, 0xe0, 0x49, + 0xd3, 0x54, 0x4c, 0x98, 0x26, 0xf5, 0x5f, 0x14, 0x68, 0x6c, 0x10, 0x3a, 0x6e, 0x3a, 0x03, 0x6a, + 0x48, 0x6f, 0x40, 0xd3, 0xc5, 0x7d, 0xc7, 0x35, 0x7a, 0xd8, 0xf6, 0x5d, 0x13, 0xb3, 0x6c, 0x44, + 0x49, 0x9b, 0x67, 0xa3, 0x1f, 0xb0, 0x41, 0x02, 0x46, 0xac, 0x8d, 0xe7, 0xeb, 0xc3, 0x51, 0x6f, + 0x9f, 0xe8, 0x37, 0x96, 0x17, 0x9f, 0x17, 0xa3, 0x54, 0xbd, 0xbd, 0x08, 0x8d, 0x10, 0xcc, 0x77, + 0xe8, 0xfe, 0x25, 0xad, 0x2e, 0xc6, 0x76, 0x1d, 0x74, 0x1d, 0x9a, 0xf4, 0x22, 0x7b, 0x96, 0x33, + 0xe8, 0x91, 0x18, 0x97, 0xdb, 0xd8, 0x86, 0xc1, 0xd1, 0x22, 0x0c, 0x12, 0x87, 0xf2, 0xcc, 0xcf, + 0x31, 0xb7, 0xb2, 0x02, 0x6a, 0xc7, 0xfc, 0x1c, 0xab, 0xbf, 0xa2, 0xc0, 0x3c, 0x37, 0xca, 0x3b, + 0xa2, 0x80, 0x41, 0xd3, 0xb3, 0x2c, 0xbf, 0x40, 0xff, 0x47, 0x6f, 0xc7, 0x13, 0x74, 0xd7, 0xa5, + 0x42, 0x46, 0x17, 0xa1, 0xce, 0x62, 0xcc, 0x22, 0xe7, 0x09, 0x70, 0xbf, 0x20, 0x34, 0xd5, 0x7d, + 0xfd, 0x91, 0x63, 0xb0, 0x7c, 0x61, 0x1b, 0xe6, 0x74, 0xc3, 0x70, 0xb1, 0xe7, 0x71, 0x3c, 0x82, + 0x47, 0xf2, 0xe6, 0x08, 0xbb, 0x5e, 0x70, 0xb1, 0x45, 0x2d, 0x78, 0x44, 0xdf, 0x4e, 0xd4, 0x15, + 0xea, 0xab, 0xd7, 0xb2, 0xf1, 0xe4, 0xe1, 0x58, 0x58, 0x79, 0xf8, 0xdb, 0x02, 0x34, 0x39, 0x6f, + 0xae, 0x71, 0xfb, 0x39, 0x99, 0xc5, 0xd6, 0xa0, 0xb1, 0x1f, 0xca, 0xd6, 0xa4, 0x74, 0x52, 0x54, + 0x04, 0x63, 0x73, 0xa6, 0xf1, 0x5a, 0xdc, 0x82, 0x97, 0x66, 0xb2, 0xe0, 0xe5, 0xd3, 0x6a, 0x88, + 0xb4, 0x27, 0x57, 0x91, 0x78, 0x72, 0xea, 0x2f, 0x40, 0x3d, 0xb2, 0x00, 0xd5, 0x80, 0x2c, 0x63, + 0xc3, 0x29, 0x16, 0x3c, 0xa2, 0x7b, 0xa1, 0x1f, 0xc3, 0x48, 0x75, 0x51, 0x82, 0x4b, 0xc2, 0x85, + 0x51, 0x7f, 0xa2, 0x40, 0x85, 0xaf, 0x7c, 0x15, 0xea, 0x5c, 0xbe, 0xa8, 0x67, 0xc7, 0x56, 0x07, + 0x3e, 0x44, 0x5c, 0xbb, 0x67, 0x27, 0x60, 0x17, 0xa1, 0x9a, 0x10, 0xad, 0x39, 0xae, 0x76, 0x83, + 0x57, 0x11, 0x79, 0x22, 0xaf, 0x88, 0x28, 0xa1, 0x25, 0x28, 0x5b, 0xce, 0x40, 0x14, 0x81, 0xd8, + 0x83, 0xfa, 0x53, 0x85, 0x26, 0xdf, 0x35, 0xdc, 0x77, 0x8e, 0xb0, 0x7b, 0x32, 0x7b, 0xfe, 0xf2, + 0x9d, 0x08, 0x9b, 0xe7, 0x0c, 0xa2, 0xc4, 0x04, 0xf4, 0x4e, 0x78, 0x09, 0x45, 0x59, 0x9a, 0x23, + 0xaa, 0xa2, 0x39, 0x93, 0x86, 0x97, 0xf1, 0x7b, 0x0a, 0xcd, 0xc4, 0xc6, 0x8f, 0x72, 0x56, 0x6f, + 0xe2, 0x99, 0x84, 0x1b, 0xea, 0xcf, 0x14, 0xb8, 0x98, 0x41, 0xdd, 0x27, 0xab, 0x5f, 0x03, 0x7d, + 0xdf, 0x86, 0xaa, 0x08, 0xb9, 0x8b, 0xb9, 0x42, 0x6e, 0x01, 0xaf, 0xfe, 0x21, 0xab, 0x07, 0x48, + 0xc8, 0xfb, 0x64, 0xf5, 0x39, 0x11, 0x38, 0x99, 0x3a, 0x2b, 0x4a, 0x52, 0x67, 0xff, 0xaa, 0x40, + 0x27, 0x4c, 0x55, 0x79, 0x6b, 0x27, 0xb3, 0x16, 0x90, 0x9e, 0x4d, 0xa0, 0xf9, 0x96, 0xa8, 0x75, + 0x10, 0xbd, 0x98, 0x2b, 0x44, 0x0c, 0x2a, 0x1d, 0x36, 0xcd, 0x7a, 0xa7, 0x0f, 0x34, 0x8b, 0x54, + 0x76, 0x22, 0x17, 0xcf, 0xea, 0x1d, 0xe1, 0xc5, 0xfe, 0x84, 0x31, 0xe9, 0x83, 0x78, 0xbe, 0xea, + 0xeb, 0x26, 0x60, 0xb4, 0x06, 0x73, 0xc0, 0x6b, 0x30, 0xa5, 0x44, 0x0d, 0x86, 0x8f, 0xab, 0x43, + 0xca, 0x02, 0xa9, 0x03, 0x3c, 0x2f, 0x82, 0xfd, 0x9a, 0x02, 0x6d, 0xbe, 0x0b, 0x6b, 0x47, 0x70, + 0x86, 0x23, 0x0b, 0xfb, 0xd8, 0xf8, 0xaa, 0x73, 0x26, 0x7f, 0x5d, 0x80, 0x56, 0xd4, 0xb1, 0xa1, + 0xbe, 0xc9, 0x1b, 0x50, 0xa6, 0x49, 0x29, 0x8e, 0xc1, 0x54, 0xed, 0xc0, 0xa0, 0x89, 0x65, 0xa4, + 0xd1, 0xc2, 0xae, 0x17, 0x38, 0x2e, 0xfc, 0x31, 0xf4, 0xae, 0x8a, 0xa7, 0xf7, 0xae, 0x5e, 0x80, + 0x1a, 0xb1, 0x5c, 0xce, 0x98, 0xac, 0xcb, 0x0a, 0xe3, 0xe1, 0x00, 0x7a, 0x17, 0x2a, 0xac, 0x83, + 0x87, 0xd7, 0x25, 0x6f, 0xc4, 0x97, 0xe6, 0xdd, 0x3d, 0x91, 0xba, 0x02, 0x1d, 0xd0, 0xf8, 0x24, + 0x72, 0x47, 0x23, 0xd7, 0x19, 0x50, 0x37, 0x8c, 0x18, 0xb5, 0xb2, 0x26, 0x9e, 0x89, 0x9b, 0xe8, + 0x8c, 0xba, 0x1b, 0x3c, 0xc3, 0x42, 0xff, 0x57, 0xff, 0x3f, 0x2c, 0x87, 0x01, 0x3d, 0x43, 0xf3, + 0xac, 0x4c, 0xae, 0xfe, 0x43, 0x01, 0xce, 0xef, 0x9c, 0xd8, 0xfd, 0xa4, 0xb8, 0x2c, 0x43, 0x65, + 0x64, 0xe9, 0x61, 0x06, 0x9c, 0x3f, 0xd1, 0xee, 0x82, 0x20, 0x54, 0x27, 0x66, 0x9d, 0xd1, 0xb8, + 0x2e, 0xc6, 0x76, 0x9d, 0xa9, 0xde, 0xd6, 0x0d, 0x91, 0x81, 0xc0, 0x06, 0x73, 0x20, 0x58, 0x86, + 0x6f, 0x5e, 0x8c, 0x52, 0x07, 0xe2, 0x5d, 0x00, 0xea, 0x63, 0xf5, 0x4e, 0xe3, 0x57, 0xd1, 0x19, + 0x9b, 0xc4, 0xaf, 0x4a, 0xb6, 0x41, 0x54, 0xd2, 0x59, 0xfa, 0x17, 0x23, 0x6a, 0xb9, 0x67, 0x1a, + 0x9c, 0xc6, 0x11, 0xe9, 0x35, 0xd0, 0x4b, 0x30, 0x1f, 0xca, 0x3b, 0x81, 0xa9, 0xa6, 0x94, 0x80, + 0xa1, 0xfe, 0xa8, 0x00, 0xed, 0xc8, 0x85, 0x7c, 0xd5, 0xde, 0x6d, 0x46, 0xcc, 0x5b, 0x7c, 0x46, + 0x31, 0x6f, 0x69, 0x76, 0x8f, 0xb6, 0x2c, 0xf3, 0x68, 0x7f, 0x50, 0x84, 0x66, 0x48, 0xb5, 0x6d, + 0x4b, 0xb7, 0x33, 0x99, 0x6e, 0x07, 0x9a, 0x5e, 0x8c, 0xaa, 0x9c, 0x4e, 0xdf, 0x94, 0x89, 0x70, + 0xc6, 0x45, 0x68, 0x89, 0x25, 0xd0, 0x65, 0xca, 0x5f, 0xae, 0xcf, 0x92, 0x93, 0xcc, 0x3d, 0xad, + 0x31, 0x5d, 0x61, 0x0e, 0x31, 0x7a, 0x15, 0x10, 0x17, 0xf0, 0x9e, 0x69, 0xf7, 0x3c, 0xdc, 0x77, + 0x6c, 0x83, 0x89, 0x7e, 0x59, 0x6b, 0xf1, 0x37, 0x5d, 0x7b, 0x87, 0x8d, 0xa3, 0x37, 0xa0, 0xe4, + 0x9f, 0x8c, 0x98, 0xaf, 0xda, 0x94, 0x7a, 0x7b, 0x21, 0x5e, 0xbb, 0x27, 0x23, 0xac, 0x51, 0xf0, + 0xa0, 0x87, 0xcc, 0x77, 0xf5, 0x23, 0xee, 0xf8, 0x97, 0xb4, 0xc8, 0x48, 0x34, 0x0d, 0x30, 0x17, + 0x4f, 0x03, 0x50, 0x21, 0x12, 0x8c, 0xe9, 0xfb, 0x16, 0xe7, 0xcc, 0x08, 0xbb, 0xee, 0xfa, 0x16, + 0x39, 0xa4, 0xef, 0xf8, 0xba, 0xc5, 0x44, 0xb1, 0xc6, 0x15, 0x17, 0x19, 0xa1, 0x41, 0xf6, 0xbf, + 0x15, 0x60, 0x31, 0x45, 0xb0, 0xcc, 0x6b, 0x98, 0x9c, 0x99, 0x9a, 0x26, 0xf6, 0xdf, 0x81, 0x3a, + 0x67, 0x8b, 0x53, 0xb0, 0x15, 0xb0, 0x29, 0x9b, 0x13, 0xf8, 0xbc, 0xfc, 0x8c, 0xf8, 0xbc, 0x72, + 0x86, 0xdc, 0x8e, 0xfc, 0x72, 0xd4, 0x7f, 0x56, 0x60, 0x29, 0xce, 0xda, 0x1a, 0xf6, 0xc6, 0x56, + 0x36, 0x65, 0x27, 0x07, 0xfe, 0x5c, 0x71, 0x27, 0x75, 0x3e, 0x37, 0x4d, 0xef, 0xa7, 0x7c, 0xe1, + 0xeb, 0x79, 0xc4, 0x22, 0xf4, 0x03, 0xa2, 0x07, 0x29, 0xc5, 0x0f, 0xf2, 0xfb, 0x0a, 0x5c, 0x48, + 0x99, 0x9a, 0x59, 0xdc, 0x91, 0xfb, 0x30, 0xe7, 0x52, 0x52, 0x04, 0x22, 0x7c, 0x73, 0x22, 0xae, + 0x21, 0xe9, 0xb4, 0x60, 0x9e, 0xba, 0x03, 0xcb, 0x81, 0xd3, 0x12, 0xde, 0xcb, 0x16, 0xf6, 0xf5, + 0x09, 0x41, 0xf1, 0x55, 0xa8, 0xb3, 0xe8, 0x8a, 0x05, 0x9b, 0xac, 0x62, 0x0b, 0x7b, 0x22, 0xcb, + 0xa9, 0xfe, 0x41, 0x01, 0x96, 0xa8, 0xd5, 0x4f, 0x96, 0xdf, 0xf2, 0xd4, 0x83, 0x55, 0x61, 0x6a, + 0x88, 0x59, 0x61, 0x27, 0xab, 0x69, 0xb1, 0x31, 0xd4, 0x4d, 0x27, 0x41, 0xa5, 0xc9, 0x93, 0xb0, + 0x00, 0xbe, 0xa1, 0xfb, 0x3a, 0xad, 0x7f, 0x27, 0xb3, 0x9f, 0xa1, 0xb7, 0x51, 0x3a, 0x8b, 0xb7, + 0xf1, 0x0a, 0xb4, 0x58, 0x5d, 0xa0, 0x27, 0x62, 0x71, 0xaa, 0xb6, 0x4a, 0xda, 0x02, 0x1b, 0xdf, + 0x0d, 0x86, 0xd5, 0x4d, 0xf8, 0x46, 0x82, 0x28, 0x33, 0xdc, 0xbd, 0xfa, 0x17, 0x0a, 0xb9, 0xb9, + 0x58, 0x23, 0xd6, 0xd9, 0x9d, 0xf3, 0xcb, 0xa2, 0x44, 0x48, 0xac, 0x72, 0x42, 0x19, 0x19, 0xe8, + 0x3d, 0xa8, 0xd9, 0xf8, 0xb8, 0x17, 0xf5, 0xf7, 0x72, 0x44, 0x2e, 0x55, 0x1b, 0x1f, 0xd3, 0xff, + 0xd4, 0x47, 0x70, 0x21, 0x85, 0xea, 0x2c, 0x67, 0xff, 0x3b, 0x05, 0x2e, 0x6e, 0xb8, 0xce, 0xe8, + 0x89, 0xe9, 0xfa, 0x63, 0xdd, 0x8a, 0x77, 0x21, 0x9c, 0xe1, 0xf8, 0x39, 0x9a, 0x3c, 0x3f, 0x4c, + 0xe9, 0x85, 0x57, 0x25, 0xb2, 0x96, 0x46, 0x2a, 0xa5, 0x1f, 0xd4, 0xff, 0x2c, 0xca, 0x90, 0x0f, + 0xac, 0xc5, 0x64, 0x07, 0x27, 0x4f, 0x10, 0x25, 0xad, 0x57, 0x14, 0xcf, 0x5a, 0xaf, 0xc8, 0x30, + 0x13, 0xa5, 0x67, 0x64, 0x26, 0x4e, 0x9d, 0xe0, 0x5b, 0x87, 0x78, 0x2d, 0x89, 0x9a, 0xf9, 0xd3, + 0xd6, 0x9f, 0xde, 0x05, 0x08, 0x4b, 0x2a, 0xbc, 0x71, 0x76, 0xca, 0x0a, 0x91, 0x09, 0xe4, 0x8e, + 0x84, 0x21, 0xe6, 0x8e, 0x42, 0x24, 0xd5, 0xfe, 0x5d, 0xe8, 0xc8, 0x78, 0x73, 0x16, 0x7e, 0xff, + 0xf7, 0x02, 0x40, 0x57, 0x74, 0x67, 0x9f, 0xcd, 0x56, 0xa4, 0x7c, 0xef, 0x34, 0xef, 0x18, 0x29, + 0x1f, 0xbe, 0x28, 0xf7, 0xe1, 0x23, 0xb2, 0xc2, 0x58, 0x21, 0xa9, 0x9f, 0x2f, 0x41, 0xcd, 0x75, + 0x8e, 0x7b, 0x44, 0xb8, 0x8c, 0xa0, 0xfd, 0xdc, 0x75, 0x8e, 0x89, 0xc8, 0x19, 0xe8, 0x02, 0xcc, + 0xf9, 0xba, 0x77, 0x48, 0xd6, 0x67, 0x49, 0xc7, 0x0a, 0x79, 0xec, 0x1a, 0x68, 0x09, 0xca, 0xfb, + 0xa6, 0x85, 0x59, 0xcb, 0x4a, 0x4d, 0x63, 0x0f, 0xe8, 0x5b, 0x41, 0xeb, 0x63, 0x35, 0x77, 0x8b, + 0x13, 0xeb, 0x7e, 0x7c, 0x09, 0xe6, 0x09, 0x27, 0x11, 0x24, 0x98, 0x58, 0xb7, 0x78, 0xc1, 0x81, + 0x0f, 0xd2, 0xd6, 0x86, 0x9f, 0x2a, 0xb0, 0x10, 0x92, 0x96, 0xea, 0x26, 0xa2, 0xee, 0xa8, 0xaa, + 0x5b, 0x77, 0x0c, 0xa6, 0x45, 0x9a, 0x19, 0x76, 0x85, 0x4d, 0x64, 0x0a, 0x2d, 0x9c, 0x32, 0x29, + 0x4b, 0x40, 0x0e, 0x4f, 0x28, 0x63, 0x1a, 0x41, 0xde, 0xaa, 0xe2, 0x3a, 0xc7, 0x5d, 0x43, 0x90, + 0x8c, 0x75, 0x92, 0xb3, 0x98, 0x98, 0x90, 0x6c, 0x9d, 0x36, 0x93, 0xbf, 0x04, 0xf3, 0xd8, 0x75, + 0x1d, 0xb7, 0x37, 0xc4, 0x9e, 0xa7, 0x0f, 0x82, 0x26, 0x8d, 0x06, 0x1d, 0xdc, 0x62, 0x63, 0xea, + 0xdf, 0x97, 0xa0, 0x19, 0x1e, 0x25, 0x68, 0x97, 0x30, 0x8d, 0xa0, 0x5d, 0xc2, 0x24, 0xf7, 0x0b, + 0x2e, 0xd3, 0x92, 0x82, 0x03, 0xd6, 0x0a, 0x6d, 0x45, 0xab, 0xf1, 0xd1, 0xae, 0x41, 0x8c, 0x3b, + 0x21, 0x90, 0xed, 0x18, 0x38, 0xe4, 0x00, 0x08, 0x86, 0x64, 0x41, 0x5c, 0x29, 0x07, 0x23, 0x95, + 0x73, 0x30, 0x52, 0x45, 0xc2, 0x48, 0xcb, 0x50, 0xd9, 0x1b, 0xf7, 0x0f, 0xb1, 0xcf, 0x9d, 0x42, + 0xfe, 0x14, 0x67, 0xb0, 0x6a, 0x82, 0xc1, 0x04, 0x1f, 0xd5, 0xa2, 0x7c, 0x74, 0x09, 0x6a, 0x81, + 0xa5, 0xf6, 0x68, 0xf9, 0xb0, 0xa8, 0x55, 0xb9, 0x89, 0xf6, 0xd0, 0x9b, 0x81, 0xcb, 0x58, 0xa7, + 0x12, 0xa5, 0x4a, 0x14, 0x52, 0x82, 0x4b, 0x02, 0x87, 0xf1, 0x26, 0x2c, 0x44, 0xc8, 0x41, 0xf9, + 0x8c, 0xd5, 0x18, 0x23, 0x11, 0x05, 0xb5, 0x20, 0x37, 0xa0, 0x19, 0x92, 0x84, 0xc2, 0xcd, 0xb3, + 0x40, 0x4e, 0x8c, 0x52, 0x30, 0xc1, 0xee, 0xcd, 0x53, 0xb2, 0xfb, 0x45, 0xa8, 0xf2, 0x08, 0xcc, + 0x6b, 0x2f, 0xc4, 0x73, 0x35, 0xb9, 0x24, 0xe1, 0x53, 0x40, 0xe1, 0x11, 0x67, 0xf3, 0x4b, 0x13, + 0x3c, 0x54, 0x48, 0xf2, 0x90, 0xfa, 0x97, 0x0a, 0x2c, 0x46, 0x37, 0x3b, 0xab, 0xe1, 0x7e, 0x0f, + 0xea, 0xac, 0xca, 0xdb, 0x23, 0x2a, 0x44, 0x5e, 0x34, 0x4d, 0x5c, 0x9e, 0x06, 0xe1, 0x77, 0x2e, + 0x84, 0x30, 0xc7, 0x8e, 0x7b, 0x68, 0xda, 0x83, 0x1e, 0xc1, 0x4c, 0xe4, 0x92, 0xf9, 0xe0, 0x23, + 0x32, 0xa6, 0xfe, 0x96, 0x02, 0x57, 0x1e, 0x8f, 0x0c, 0xdd, 0xc7, 0x11, 0x0f, 0x66, 0xd6, 0xbe, + 0x51, 0xd1, 0xb8, 0x59, 0x98, 0x70, 0xcd, 0x91, 0xfd, 0x3c, 0xde, 0xb8, 0x49, 0xfc, 0x3e, 0x8e, + 0x4d, 0xaa, 0xd3, 0xfa, 0xec, 0xd8, 0x74, 0xa0, 0x7a, 0xc4, 0x97, 0x0b, 0x3e, 0xc1, 0x09, 0x9e, + 0x63, 0x55, 0xe9, 0xe2, 0xa9, 0xaa, 0xd2, 0xea, 0x16, 0x5c, 0xd4, 0xb0, 0x87, 0x6d, 0x23, 0x76, + 0x90, 0x33, 0x67, 0xd7, 0x46, 0xd0, 0x91, 0x2d, 0x37, 0x0b, 0xa7, 0x32, 0xc7, 0xb7, 0xe7, 0x92, + 0x65, 0x7d, 0xae, 0xac, 0x89, 0xbf, 0x45, 0xf7, 0xf1, 0xd5, 0xbf, 0x2a, 0xc0, 0x85, 0xfb, 0x86, + 0xc1, 0xf5, 0x3c, 0x77, 0xe5, 0x9e, 0x97, 0x97, 0x9d, 0xf4, 0x42, 0x8b, 0x69, 0x2f, 0xf4, 0x59, + 0xe9, 0x5e, 0x6e, 0x85, 0xec, 0xf1, 0x30, 0x30, 0xc1, 0x2e, 0xeb, 0x34, 0x7b, 0x87, 0xd7, 0x6e, + 0x7b, 0x96, 0x33, 0xa0, 0x66, 0x78, 0xba, 0x73, 0x56, 0x0d, 0xb2, 0x84, 0xea, 0x08, 0xda, 0x69, + 0x62, 0xcd, 0xa8, 0x47, 0x02, 0x8a, 0x8c, 0x1c, 0x96, 0x81, 0x6e, 0x10, 0x4f, 0x8c, 0x0e, 0x6d, + 0x3b, 0x9e, 0xfa, 0x5f, 0x05, 0x68, 0xef, 0xe8, 0x47, 0xf8, 0xff, 0xce, 0x05, 0x7d, 0x02, 0x4b, + 0x9e, 0x7e, 0x84, 0x7b, 0x91, 0x00, 0xbc, 0xe7, 0xe2, 0xcf, 0xb8, 0x13, 0xfb, 0x8a, 0xac, 0x46, + 0x20, 0x6d, 0xa5, 0xd2, 0x16, 0xbd, 0xd8, 0xb8, 0x86, 0x3f, 0x43, 0x2f, 0xc3, 0x42, 0xb4, 0xed, + 0x2f, 0x48, 0xd0, 0x36, 0xb4, 0xf9, 0x48, 0x6b, 0x5f, 0xd7, 0x50, 0x3f, 0x83, 0x17, 0x1e, 0xdb, + 0x1e, 0xf6, 0xbb, 0x61, 0x7b, 0xda, 0x8c, 0xf1, 0xe7, 0x55, 0xa8, 0x87, 0x84, 0x4f, 0x7d, 0x7b, + 0x63, 0x78, 0xaa, 0x03, 0x9d, 0x2d, 0xdd, 0x3d, 0x0c, 0xf2, 0xea, 0x1b, 0xac, 0xed, 0xe7, 0x39, + 0x6e, 0xb8, 0x2f, 0x1a, 0xe0, 0x34, 0xbc, 0x8f, 0x5d, 0x6c, 0xf7, 0xf1, 0xa6, 0xd3, 0x3f, 0x24, + 0x0e, 0x89, 0xcf, 0xbe, 0x9a, 0x54, 0x22, 0xbe, 0xeb, 0x46, 0xe4, 0xeb, 0xc6, 0x42, 0xec, 0xeb, + 0xc6, 0x29, 0x1f, 0x00, 0xab, 0x3f, 0x2c, 0xc0, 0xf2, 0x7d, 0xcb, 0xc7, 0x6e, 0x98, 0x61, 0x38, + 0x4d, 0xb2, 0x24, 0xcc, 0x5e, 0x14, 0xce, 0x92, 0xbd, 0xc8, 0x51, 0x4a, 0x95, 0xe5, 0x5a, 0x4a, + 0x67, 0xcc, 0xb5, 0xdc, 0x07, 0x18, 0xb9, 0xce, 0x08, 0xbb, 0xbe, 0x89, 0x83, 0xd8, 0x2f, 0x87, + 0x83, 0x13, 0x99, 0xa4, 0x7e, 0x02, 0xad, 0x87, 0xfd, 0x75, 0xc7, 0xde, 0x37, 0xdd, 0x61, 0x40, + 0xa8, 0x94, 0xd0, 0x29, 0x39, 0x84, 0xae, 0x90, 0x12, 0x3a, 0xd5, 0x84, 0xc5, 0xc8, 0xda, 0x33, + 0x2a, 0xae, 0x41, 0xbf, 0xb7, 0x6f, 0xda, 0x26, 0x6d, 0xab, 0x2b, 0x50, 0x07, 0x15, 0x06, 0xfd, + 0x07, 0x7c, 0x44, 0xfd, 0x52, 0x81, 0x4b, 0x1a, 0x26, 0xc2, 0x13, 0x74, 0x10, 0xed, 0xfa, 0x5b, + 0xde, 0x60, 0x06, 0x87, 0xe2, 0x1e, 0x94, 0x86, 0xde, 0x20, 0xa3, 0xfa, 0x4f, 0x4c, 0x74, 0x6c, + 0x23, 0x8d, 0x02, 0xab, 0x3f, 0x56, 0x60, 0x29, 0xa8, 0x91, 0xc6, 0x44, 0x38, 0xce, 0xb6, 0x4a, + 0xaa, 0x67, 0x7c, 0xc2, 0x97, 0xd2, 0x17, 0x60, 0xce, 0xd8, 0x8b, 0x2a, 0xc8, 0x8a, 0xb1, 0x47, + 0x75, 0xa3, 0xc4, 0x53, 0x2e, 0x49, 0x3d, 0xe5, 0x24, 0xe3, 0x97, 0x25, 0xcd, 0x57, 0x8f, 0xa1, + 0xcd, 0x1d, 0x94, 0x8f, 0x46, 0xd8, 0xd5, 0x29, 0x7f, 0x05, 0xc8, 0xbf, 0x15, 0xb8, 0xd0, 0x4a, + 0xe6, 0x07, 0x85, 0xc9, 0xfa, 0x28, 0x77, 0xa2, 0xd5, 0x7f, 0x52, 0xe0, 0x5a, 0x72, 0xdd, 0x6d, + 0x5e, 0x3d, 0x9c, 0xf9, 0x13, 0x7b, 0x5a, 0x7a, 0x2c, 0x84, 0xa5, 0xc7, 0x99, 0x6a, 0xa8, 0xd1, + 0x32, 0x67, 0x29, 0x5e, 0xe6, 0xbc, 0xf5, 0x9e, 0x68, 0x9a, 0xdf, 0x3d, 0x19, 0x61, 0x34, 0x07, + 0xc5, 0x47, 0xf8, 0xb8, 0x75, 0x0e, 0x01, 0x54, 0x1e, 0x39, 0xee, 0x50, 0xb7, 0x5a, 0x0a, 0xaa, + 0xc3, 0x1c, 0x2f, 0x8d, 0xb7, 0x0a, 0x68, 0x1e, 0x6a, 0xeb, 0x41, 0xb9, 0xb0, 0x55, 0xbc, 0x75, + 0x0b, 0x1a, 0xd1, 0x8e, 0x47, 0x32, 0x6f, 0x13, 0x0f, 0xf4, 0xfe, 0x49, 0xeb, 0x1c, 0xaa, 0x40, + 0x61, 0xf3, 0x6e, 0x4b, 0xa1, 0x7f, 0x5f, 0x6f, 0x15, 0x6e, 0xfd, 0xb1, 0x02, 0x8b, 0x29, 0x24, + 0x51, 0x13, 0xe0, 0xb1, 0xdd, 0xe7, 0x15, 0xf0, 0xd6, 0x39, 0xd4, 0x80, 0x6a, 0x50, 0x0f, 0x67, + 0x7b, 0xef, 0x3a, 0x14, 0xba, 0x55, 0x40, 0x2d, 0x68, 0xb0, 0x89, 0xe3, 0x7e, 0x1f, 0x7b, 0x5e, + 0xab, 0x28, 0x46, 0x1e, 0xe8, 0xa6, 0x35, 0x76, 0x71, 0xab, 0x44, 0xf0, 0xdb, 0x75, 0x34, 0x6c, + 0x61, 0xdd, 0xc3, 0xad, 0x32, 0x42, 0xd0, 0xe4, 0x0f, 0xc1, 0xa4, 0x4a, 0x64, 0x2c, 0x98, 0x36, + 0x77, 0xeb, 0x47, 0x4a, 0xb4, 0x28, 0x46, 0x69, 0x71, 0x01, 0xce, 0x3f, 0xb6, 0x0d, 0xbc, 0x6f, + 0xda, 0xd8, 0x08, 0x5f, 0xb5, 0xce, 0xa1, 0xf3, 0xb0, 0xb0, 0x85, 0xdd, 0x01, 0x8e, 0x0c, 0x16, + 0xd0, 0x22, 0xcc, 0x6f, 0x99, 0x4f, 0x23, 0x43, 0x45, 0xb4, 0x04, 0xad, 0x1d, 0xd3, 0x1e, 0x58, + 0x51, 0xc0, 0x12, 0x9d, 0x6d, 0xda, 0x8e, 0x1b, 0x19, 0x2c, 0xd3, 0x41, 0xfd, 0xd3, 0xd8, 0x60, + 0x05, 0x75, 0x60, 0x99, 0x12, 0xf5, 0xee, 0x06, 0x26, 0xd4, 0x88, 0xbc, 0x9b, 0x53, 0x4b, 0x55, + 0xa5, 0xa5, 0xac, 0xfe, 0xf8, 0x06, 0xd4, 0x88, 0xb0, 0xae, 0x3b, 0x8e, 0x6b, 0x20, 0x0b, 0x10, + 0xfd, 0x8a, 0x6e, 0x38, 0x72, 0x6c, 0xf1, 0xc5, 0x2d, 0xba, 0x9d, 0x90, 0x6f, 0xf6, 0x90, 0x06, + 0xe4, 0x22, 0xd1, 0xb9, 0x2e, 0x85, 0x4f, 0x00, 0xab, 0xe7, 0xd0, 0x90, 0xee, 0xb6, 0x6b, 0x0e, + 0xf1, 0xae, 0xd9, 0x3f, 0x0c, 0x42, 0x80, 0xbb, 0x19, 0x9f, 0x2d, 0xa6, 0x41, 0x83, 0xfd, 0x5e, + 0x92, 0xee, 0xc7, 0x3e, 0x73, 0x0c, 0xe4, 0x48, 0x3d, 0x87, 0x3e, 0xa3, 0xea, 0x27, 0x8c, 0xa7, + 0x82, 0x0d, 0x57, 0xb3, 0x37, 0x4c, 0x01, 0x9f, 0x72, 0xcb, 0x4d, 0x28, 0x53, 0xbe, 0x47, 0xb2, + 0x16, 0x88, 0xe8, 0x2f, 0x80, 0x74, 0xae, 0x65, 0x03, 0x88, 0xd5, 0x3e, 0x85, 0x85, 0xc4, 0x87, + 0xf4, 0x48, 0xe6, 0x83, 0xc9, 0x7f, 0x12, 0xa1, 0x73, 0x2b, 0x0f, 0xa8, 0xd8, 0x6b, 0x00, 0xcd, + 0xf8, 0xd7, 0x77, 0x68, 0x25, 0xc7, 0x37, 0xbc, 0x6c, 0xa7, 0x57, 0x72, 0x7f, 0xed, 0x4b, 0x99, + 0xa0, 0x95, 0xfc, 0xc4, 0x1b, 0xdd, 0x9a, 0xb8, 0x40, 0x9c, 0xd9, 0xbe, 0x99, 0x0b, 0x56, 0x6c, + 0x77, 0x42, 0x99, 0x20, 0xf5, 0x7d, 0x6d, 0x92, 0xc7, 0x83, 0x65, 0xb2, 0x3e, 0xfc, 0xed, 0xdc, + 0xc9, 0x0d, 0x2f, 0xb6, 0xfe, 0x65, 0xd6, 0xe7, 0x28, 0xfb, 0x46, 0x15, 0xbd, 0x2e, 0x5f, 0x6e, + 0xc2, 0xc7, 0xb5, 0x9d, 0xd5, 0xd3, 0x4c, 0x11, 0x48, 0x7c, 0x9f, 0x36, 0x28, 0x4a, 0xbe, 0xf2, + 0x4c, 0xca, 0x5d, 0xb0, 0x5e, 0xf6, 0x07, 0xac, 0x9d, 0xd7, 0x4f, 0x31, 0x43, 0x20, 0xe0, 0x24, + 0xbf, 0xa1, 0x0f, 0xc4, 0xf0, 0xce, 0x54, 0xae, 0x39, 0x9b, 0x0c, 0x7e, 0x0f, 0x16, 0x12, 0x51, + 0x09, 0xca, 0x1f, 0xb9, 0x74, 0x26, 0x99, 0x5b, 0x26, 0x92, 0x89, 0x86, 0x44, 0x94, 0xc1, 0xfd, + 0x92, 0xa6, 0xc5, 0xce, 0xad, 0x3c, 0xa0, 0xe2, 0x20, 0x23, 0x58, 0x4c, 0xbc, 0x7c, 0xb2, 0x8a, + 0xbe, 0x99, 0x7b, 0xb7, 0x27, 0xab, 0x9d, 0x57, 0xf3, 0xef, 0xf7, 0x64, 0x55, 0x3d, 0x87, 0x3c, + 0xaa, 0xa0, 0x13, 0x4d, 0x6d, 0x28, 0x63, 0x15, 0x79, 0xf3, 0x5e, 0xe7, 0xb5, 0x9c, 0xd0, 0xe2, + 0x98, 0x47, 0x70, 0x5e, 0xd2, 0x7b, 0x88, 0x5e, 0x9b, 0xc8, 0x1e, 0xc9, 0xa6, 0xcb, 0xce, 0xed, + 0xbc, 0xe0, 0x11, 0xf3, 0xd0, 0x0a, 0xf0, 0xba, 0x6f, 0x59, 0xcc, 0xb3, 0x78, 0x35, 0xcb, 0xf2, + 0xc5, 0xc0, 0x32, 0x8e, 0x9a, 0x09, 0x2d, 0xb6, 0xfc, 0x45, 0x40, 0x3b, 0x07, 0xce, 0x31, 0x8d, + 0x02, 0x06, 0x63, 0xee, 0x58, 0x66, 0x1a, 0xc0, 0x34, 0x68, 0x86, 0x20, 0x4e, 0x9c, 0x21, 0x36, + 0xef, 0x01, 0x3c, 0xc4, 0xfe, 0x16, 0xf6, 0x5d, 0x22, 0xfd, 0x2f, 0x67, 0xe1, 0xce, 0x01, 0x82, + 0xad, 0x6e, 0x4e, 0x85, 0x8b, 0x12, 0x74, 0x4b, 0xb7, 0xc7, 0xba, 0x15, 0xf9, 0x84, 0x4d, 0x4e, + 0xd0, 0x24, 0xd8, 0x64, 0x82, 0xa6, 0xa1, 0xc5, 0x96, 0xc7, 0xc2, 0x7f, 0x89, 0xb4, 0x3d, 0x4c, + 0xf6, 0x5f, 0xd2, 0xad, 0x78, 0x49, 0xdd, 0x3e, 0x01, 0x5e, 0x6c, 0xfc, 0x85, 0x42, 0x3b, 0x66, + 0x13, 0x00, 0x1f, 0x9b, 0xfe, 0xc1, 0xb6, 0xa5, 0xdb, 0x5e, 0x1e, 0x14, 0x28, 0xe0, 0x29, 0x50, + 0xe0, 0xf0, 0x02, 0x05, 0x03, 0xe6, 0x63, 0x15, 0x7f, 0x24, 0xeb, 0xcf, 0x90, 0x35, 0x4a, 0x74, + 0x56, 0xa6, 0x03, 0x8a, 0x5d, 0xf6, 0x61, 0x3e, 0x16, 0xc3, 0x49, 0x77, 0x91, 0x45, 0x79, 0x49, + 0x65, 0x97, 0x90, 0x8e, 0x24, 0x41, 0x3d, 0x40, 0xe9, 0xc2, 0x26, 0xca, 0x57, 0x06, 0x9f, 0xa4, + 0x7a, 0xb2, 0xab, 0xa5, 0x4c, 0x9b, 0x27, 0x5a, 0x07, 0xe4, 0xa6, 0x42, 0xda, 0x09, 0x21, 0xd5, + 0xe6, 0x19, 0x9d, 0x08, 0xea, 0x39, 0xf4, 0x31, 0x54, 0xf8, 0x2f, 0x64, 0x5d, 0x9f, 0x5c, 0x42, + 0xe0, 0xab, 0xdf, 0x98, 0x02, 0x25, 0x16, 0x3e, 0x84, 0x0b, 0x19, 0x05, 0x04, 0xa9, 0x97, 0x31, + 0xb9, 0xd8, 0x30, 0xcd, 0xfe, 0x89, 0xcd, 0x52, 0xf5, 0x81, 0x09, 0x9b, 0x65, 0xd5, 0x12, 0xa6, + 0x6d, 0xd6, 0x83, 0xc5, 0x54, 0xfe, 0x55, 0x6a, 0x00, 0xb3, 0xb2, 0xb4, 0xd3, 0x36, 0x18, 0xc0, + 0x37, 0xa4, 0xb9, 0x46, 0xa9, 0x6f, 0x32, 0x29, 0x2b, 0x39, 0x6d, 0xa3, 0x3e, 0x9c, 0x97, 0x64, + 0x18, 0xa5, 0x36, 0x2e, 0x3b, 0x13, 0x39, 0x6d, 0x93, 0x7d, 0xe8, 0xac, 0xb9, 0x8e, 0x6e, 0xf4, + 0x75, 0xcf, 0xa7, 0x59, 0x3f, 0x12, 0x84, 0x06, 0xce, 0xa1, 0x3c, 0x72, 0x90, 0xe6, 0x06, 0xa7, + 0xed, 0xb3, 0x07, 0x75, 0x7a, 0x95, 0xec, 0xe7, 0x88, 0x90, 0xdc, 0x42, 0x44, 0x20, 0x32, 0xd4, + 0x8e, 0x0c, 0x50, 0x30, 0xf5, 0x2e, 0xd4, 0xd7, 0x69, 0xf9, 0xb4, 0x6b, 0x1b, 0xf8, 0x69, 0xd2, + 0x5a, 0xd1, 0xdf, 0x64, 0xb8, 0x1d, 0x01, 0xc8, 0x4d, 0xa1, 0x79, 0xea, 0xb3, 0x1b, 0xf8, 0x29, + 0xbb, 0xe7, 0x15, 0xd9, 0xba, 0x31, 0x90, 0x8c, 0x18, 0x47, 0x0a, 0x19, 0xb1, 0xf3, 0x4b, 0x51, + 0x4f, 0x56, 0x6c, 0x77, 0x27, 0x63, 0x91, 0x14, 0x64, 0xb0, 0xeb, 0xdd, 0xfc, 0x13, 0xa2, 0x76, + 0x21, 0xc0, 0xab, 0x4b, 0x6b, 0xb7, 0x37, 0x27, 0xa1, 0x1e, 0x75, 0x4f, 0x57, 0xa6, 0x03, 0x8a, + 0x5d, 0xb6, 0xa1, 0x46, 0xb8, 0x93, 0x5d, 0xcf, 0x75, 0xd9, 0x44, 0xf1, 0x3a, 0xff, 0xe5, 0x6c, + 0x60, 0xaf, 0xef, 0x9a, 0x7b, 0xfc, 0xd2, 0xa5, 0xe8, 0xc4, 0x40, 0x26, 0x5e, 0x4e, 0x02, 0x52, + 0x60, 0x3e, 0xa6, 0x3e, 0x83, 0x20, 0x1d, 0x57, 0x95, 0xaf, 0x4d, 0xbb, 0xdf, 0xb8, 0x9a, 0xbc, + 0x9d, 0x17, 0x5c, 0x6c, 0xfb, 0x4b, 0x34, 0x0e, 0xa2, 0xef, 0xd7, 0xc6, 0xa6, 0x65, 0x04, 0x89, + 0x3f, 0x74, 0x77, 0xd2, 0x52, 0x31, 0xd0, 0x4c, 0xf7, 0x6f, 0xc2, 0x0c, 0xb1, 0xff, 0xcf, 0x43, + 0x4d, 0xe4, 0x9f, 0x91, 0x2c, 0x6b, 0x99, 0xcc, 0x7c, 0x77, 0xae, 0x4f, 0x06, 0x12, 0x2b, 0x63, + 0x58, 0x92, 0x65, 0x9b, 0xa5, 0x21, 0xf6, 0x84, 0xb4, 0xf4, 0x14, 0xfe, 0x58, 0xfd, 0xb2, 0x01, + 0xd5, 0x60, 0xe2, 0x57, 0x9c, 0xb8, 0xfa, 0x1a, 0x32, 0x49, 0xdf, 0x83, 0x85, 0xc4, 0xaf, 0xcc, + 0x48, 0x35, 0xb8, 0xfc, 0x97, 0x68, 0xa6, 0x89, 0xda, 0xc7, 0xfc, 0x77, 0x5d, 0x45, 0x88, 0x77, + 0x33, 0x2b, 0x1b, 0x95, 0x8c, 0xee, 0xa6, 0x2c, 0xfc, 0xbf, 0x3b, 0xc0, 0x79, 0x04, 0x10, 0x09, + 0x6d, 0x5e, 0x9c, 0xda, 0x0e, 0x3d, 0x8d, 0x5a, 0x43, 0x69, 0xf4, 0xf2, 0xca, 0xe4, 0x96, 0xf0, + 0x69, 0x1e, 0x68, 0x76, 0xcc, 0xf2, 0x18, 0x1a, 0xd1, 0xcf, 0x87, 0x90, 0xf4, 0x27, 0x37, 0xd3, + 0xdf, 0x17, 0x4d, 0x3b, 0xc5, 0xd6, 0x29, 0x1d, 0xdb, 0x29, 0xcb, 0x79, 0x80, 0xd2, 0x7d, 0x18, + 0xd2, 0x40, 0x20, 0xb3, 0xfb, 0x43, 0x1a, 0x08, 0x64, 0x37, 0x77, 0xb0, 0xa4, 0x64, 0xb2, 0xb9, + 0x40, 0x9a, 0x94, 0xcc, 0x68, 0xd7, 0x90, 0x26, 0x25, 0xb3, 0xba, 0x15, 0x22, 0xf2, 0x37, 0x31, + 0x74, 0x93, 0xfd, 0xec, 0xf0, 0x34, 0xe2, 0x19, 0xb0, 0xfc, 0xc8, 0xf1, 0xcd, 0xfd, 0x93, 0x64, + 0x99, 0x49, 0xea, 0x36, 0x67, 0xd5, 0xb8, 0xa6, 0x4b, 0xf9, 0x65, 0xea, 0xb5, 0x65, 0xd5, 0xb2, + 0x50, 0x9e, 0xa2, 0x58, 0xe7, 0x5e, 0x0e, 0x8c, 0xd2, 0x76, 0x6c, 0xed, 0xde, 0x27, 0xaf, 0x0f, + 0x4c, 0xff, 0x60, 0xbc, 0x47, 0xd0, 0xba, 0xc3, 0x96, 0x78, 0xcd, 0x74, 0xf8, 0x7f, 0x77, 0x02, + 0x55, 0x71, 0x87, 0xae, 0x7a, 0x87, 0xac, 0x3a, 0xda, 0xdb, 0xab, 0xd0, 0xa7, 0x7b, 0xff, 0x13, + 0x00, 0x00, 0xff, 0xff, 0x56, 0x86, 0x63, 0xf6, 0x0f, 0x5c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/proxy/data_coord_mock_test.go b/internal/proxy/data_coord_mock_test.go index 5815437933..745c3e0732 100644 --- a/internal/proxy/data_coord_mock_test.go +++ b/internal/proxy/data_coord_mock_test.go @@ -211,7 +211,7 @@ func (coord *DataCoordMock) GetMetrics(ctx context.Context, req *milvuspb.GetMet }, nil } -func (coord *DataCoordMock) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { +func (coord *DataCoordMock) CompleteCompaction(ctx context.Context, req *datapb.CompactionPlanResult, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, nil } diff --git a/internal/util/mock/datacoord_client.go b/internal/util/mock/datacoord_client.go deleted file mode 100644 index 8276afaa33..0000000000 --- a/internal/util/mock/datacoord_client.go +++ /dev/null @@ -1,135 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package mock - -import ( - "context" - - "google.golang.org/grpc" - - "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/internalpb" - "github.com/milvus-io/milvus/internal/types" -) - -// DataCoordClient mocks of DataCoordClient -type DataCoordClient struct { - types.DataCoord - Err error -} - -func (m *DataCoordClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) { - return &milvuspb.ComponentStates{}, m.Err -} - -func (m *DataCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { - return &milvuspb.StringResponse{}, m.Err -} - -func (m *DataCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { - return &milvuspb.StringResponse{}, m.Err -} - -func (m *DataCoordClient) Flush(ctx context.Context, in *datapb.FlushRequest, opts ...grpc.CallOption) (*datapb.FlushResponse, error) { - return &datapb.FlushResponse{}, m.Err -} - -func (m *DataCoordClient) AssignSegmentID(ctx context.Context, in *datapb.AssignSegmentIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegmentIDResponse, error) { - return &datapb.AssignSegmentIDResponse{}, m.Err -} - -func (m *DataCoordClient) GetSegmentInfo(ctx context.Context, in *datapb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*datapb.GetSegmentInfoResponse, error) { - return &datapb.GetSegmentInfoResponse{}, m.Err -} - -func (m *DataCoordClient) GetSegmentStates(ctx context.Context, in *datapb.GetSegmentStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentStatesResponse, error) { - return &datapb.GetSegmentStatesResponse{}, m.Err -} - -func (m *DataCoordClient) GetInsertBinlogPaths(ctx context.Context, in *datapb.GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*datapb.GetInsertBinlogPathsResponse, error) { - return &datapb.GetInsertBinlogPathsResponse{}, m.Err -} - -func (m *DataCoordClient) GetCollectionStatistics(ctx context.Context, in *datapb.GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetCollectionStatisticsResponse, error) { - return &datapb.GetCollectionStatisticsResponse{}, m.Err -} - -func (m *DataCoordClient) GetPartitionStatistics(ctx context.Context, in *datapb.GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetPartitionStatisticsResponse, error) { - return &datapb.GetPartitionStatisticsResponse{}, m.Err -} - -func (m *DataCoordClient) GetSegmentInfoChannel(ctx context.Context, in *datapb.GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { - return &milvuspb.StringResponse{}, m.Err -} - -func (m *DataCoordClient) SaveBinlogPaths(ctx context.Context, in *datapb.SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - return &commonpb.Status{}, m.Err -} - -func (m *DataCoordClient) GetRecoveryInfo(ctx context.Context, in *datapb.GetRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetRecoveryInfoResponse, error) { - return &datapb.GetRecoveryInfoResponse{}, m.Err -} - -func (m *DataCoordClient) GetFlushedSegments(ctx context.Context, in *datapb.GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*datapb.GetFlushedSegmentsResponse, error) { - return &datapb.GetFlushedSegmentsResponse{}, m.Err -} - -func (m *DataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { - return &milvuspb.GetMetricsResponse{}, m.Err -} - -func (m *DataCoordClient) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { - return &commonpb.Status{}, m.Err -} - -func (m *DataCoordClient) ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) { - return &milvuspb.ManualCompactionResponse{}, m.Err -} - -func (m *DataCoordClient) GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) { - return &milvuspb.GetCompactionStateResponse{}, m.Err -} - -func (m *DataCoordClient) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) { - return &milvuspb.GetCompactionPlansResponse{}, m.Err -} - -func (m *DataCoordClient) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest, opts ...grpc.CallOption) (*datapb.WatchChannelsResponse, error) { - return &datapb.WatchChannelsResponse{}, m.Err -} - -func (m *DataCoordClient) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) { - return &milvuspb.GetFlushStateResponse{}, m.Err -} - -func (m *DataCoordClient) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) { - return &datapb.DropVirtualChannelResponse{}, m.Err -} - -func (m *DataCoordClient) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest, opts ...grpc.CallOption) (*datapb.SetSegmentStateResponse, error) { - return &datapb.SetSegmentStateResponse{}, m.Err -} - -func (m *DataCoordClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) { - return &datapb.ImportTaskResponse{}, m.Err -} - -func (m *DataCoordClient) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { - return &commonpb.Status{}, m.Err -} diff --git a/internal/util/mock/grpc_datacoord_client.go b/internal/util/mock/grpc_datacoord_client.go index b7dfe111cd..7afd848a39 100644 --- a/internal/util/mock/grpc_datacoord_client.go +++ b/internal/util/mock/grpc_datacoord_client.go @@ -126,7 +126,7 @@ func (m *GrpcDataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMe return &milvuspb.GetMetricsResponse{}, m.Err } -func (m *GrpcDataCoordClient) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { +func (m *GrpcDataCoordClient) CompleteCompaction(ctx context.Context, req *datapb.CompactionPlanResult, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err }