From 1d4be0d257fa11eb956568594123031554e6aeda Mon Sep 17 00:00:00 2001 From: aoiasd <45024769+aoiasd@users.noreply.github.com> Date: Mon, 6 Nov 2023 15:26:16 +0800 Subject: [PATCH] Adjust datacoord for L0 Delta (#28021) Signed-off-by: aoiasd --- internal/datacoord/meta.go | 306 ++++---- internal/datacoord/meta_test.go | 64 +- .../datacoord/segment_allocation_policy.go | 37 +- .../segment_allocation_policy_test.go | 4 +- internal/datacoord/segment_manager.go | 31 +- internal/datacoord/segment_manager_test.go | 10 +- internal/datacoord/server_test.go | 15 + internal/datacoord/services.go | 64 +- internal/datacoord/util.go | 21 + internal/metastore/kv/datacoord/kv_catalog.go | 355 +-------- internal/metastore/kv/datacoord/util.go | 412 +++++++++++ internal/proto/data_coord.proto | 3 + internal/proto/datapb/data_coord.pb.go | 677 +++++++++--------- 13 files changed, 1102 insertions(+), 897 deletions(-) create mode 100644 internal/metastore/kv/datacoord/util.go diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 324bda0f6c..3933ac30f6 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -439,183 +439,197 @@ func (m *meta) UnsetIsImporting(segmentID UniqueID) error { return nil } -// UpdateFlushSegmentsInfo update segment partial/completed flush info -// `flushed` parameter indicating whether segment is flushed completely or partially -// `binlogs`, `checkpoints` and `statPositions` are persistence data for segment -func (m *meta) UpdateFlushSegmentsInfo( - segmentID UniqueID, - flushed bool, - dropped bool, - importing bool, - binlogs, statslogs, deltalogs []*datapb.FieldBinlog, - checkpoints []*datapb.CheckPoint, - startPositions []*datapb.SegmentStartPosition, -) error { - log.Debug("meta update: update flush segments info", - zap.Int64("segmentId", segmentID), - zap.Int("binlog", len(binlogs)), - zap.Int("stats log", len(statslogs)), - zap.Int("delta logs", len(deltalogs)), - zap.Bool("flushed", flushed), - zap.Bool("dropped", dropped), - zap.Any("check points", checkpoints), - zap.Any("start position", startPositions), - zap.Bool("importing", importing)) - m.Lock() - defer m.Unlock() +type UpdateSegmentPack struct { + meta *meta + segments map[int64]*SegmentInfo + // for update etcd binlog paths + increments map[int64]metastore.BinlogsIncrement + // for update segment metric after alter segments + metricMutation *segMetricMutation +} - segment := m.segments.GetSegment(segmentID) +func (p *UpdateSegmentPack) Get(segmentID int64) *SegmentInfo { + if segment, ok := p.segments[segmentID]; ok { + return segment + } + + segment := p.meta.segments.GetSegment(segmentID) if segment == nil || !isSegmentHealthy(segment) { - log.Warn("meta update: update flush segments info - segment not found", + log.Warn("meta update: get segment failed - segment not found", zap.Int64("segmentID", segmentID), zap.Bool("segment nil", segment == nil), zap.Bool("segment unhealthy", !isSegmentHealthy(segment))) return nil } - metricMutation := &segMetricMutation{ - stateChange: make(map[string]int), + p.segments[segmentID] = segment.Clone() + return p.segments[segmentID] +} + +type UpdateOperator func(*UpdateSegmentPack) bool + +func CreateL0Operator(collectionID, partitionID, segmentID int64, channel string) UpdateOperator { + return func(modPack *UpdateSegmentPack) bool { + segment := modPack.meta.GetSegment(segmentID) + if segment == nil { + log.Info("meta update: add new l0 segment", + zap.Int64("collectionID", collectionID), + zap.Int64("partitionID", partitionID), + zap.Int64("segmentID", segmentID)) + } + modPack.segments[segmentID] = &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: segmentID, + CollectionID: collectionID, + PartitionID: partitionID, + InsertChannel: channel, + NumOfRows: 0, + State: commonpb.SegmentState_Growing, + Level: datapb.SegmentLevel_L0, + }, + } + return true } - clonedSegment := segment.Clone() - modSegments := make(map[UniqueID]*SegmentInfo) - if flushed { - // Update segment state and prepare metrics. - updateSegStateAndPrepareMetrics(clonedSegment, commonpb.SegmentState_Flushing, metricMutation) - modSegments[segmentID] = clonedSegment +} + +// Set status of segment +// and record dropped time when change segment status to dropped +func UpdateStatusOperator(segmentID int64, status commonpb.SegmentState) UpdateOperator { + return func(modPack *UpdateSegmentPack) bool { + segment := modPack.Get(segmentID) + if segment == nil { + log.Warn("meta update: update status failed - segment not found", + zap.Int64("segmentID", segmentID), + zap.String("status", status.String())) + return false + } + + updateSegStateAndPrepareMetrics(segment, status, modPack.metricMutation) + if status == commonpb.SegmentState_Dropped { + segment.DroppedAt = uint64(time.Now().UnixNano()) + } + return true } - if dropped { - // Update segment state and prepare metrics. - updateSegStateAndPrepareMetrics(clonedSegment, commonpb.SegmentState_Dropped, metricMutation) - clonedSegment.DroppedAt = uint64(time.Now().UnixNano()) - modSegments[segmentID] = clonedSegment +} + +// update binlogs in segmentInfo +func UpdateBinlogsOperator(segmentID int64, binlogs, statslogs, deltalogs []*datapb.FieldBinlog) UpdateOperator { + return func(modPack *UpdateSegmentPack) bool { + segment := modPack.Get(segmentID) + if segment == nil { + log.Warn("meta update: update binlog failed - segment not found", + zap.Int64("segmentID", segmentID)) + return false + } + + segment.Binlogs = mergeFieldBinlogs(segment.GetBinlogs(), binlogs) + segment.Statslogs = mergeFieldBinlogs(segment.GetStatslogs(), statslogs) + segment.Deltalogs = mergeFieldBinlogs(segment.GetDeltalogs(), deltalogs) + modPack.increments[segmentID] = metastore.BinlogsIncrement{ + Segment: segment.SegmentInfo, + } + return true } - // TODO add diff encoding and compression - currBinlogs := clonedSegment.GetBinlogs() - getFieldBinlogs := func(id UniqueID, binlogs []*datapb.FieldBinlog) *datapb.FieldBinlog { - for _, binlog := range binlogs { - if id == binlog.GetFieldID() { - return binlog +} + +// update startPosition +func UpdateStartPosition(startPositions []*datapb.SegmentStartPosition) UpdateOperator { + return func(modPack *UpdateSegmentPack) bool { + for _, pos := range startPositions { + if len(pos.GetStartPosition().GetMsgID()) == 0 { + continue + } + s := modPack.Get(pos.GetSegmentID()) + if s == nil { + continue + } + + s.StartPosition = pos.GetStartPosition() + } + return true + } +} + +// update segment checkpoint and num rows +// if was importing segment +// only update rows. +func UpdateCheckPointOperator(segmentID int64, importing bool, checkpoints []*datapb.CheckPoint) UpdateOperator { + return func(modPack *UpdateSegmentPack) bool { + segment := modPack.Get(segmentID) + if segment == nil { + log.Warn("meta update: update checkpoint failed - segment not found", + zap.Int64("segmentID", segmentID)) + return false + } + + if importing { + segment.NumOfRows = segment.currRows + } else { + for _, cp := range checkpoints { + if cp.SegmentID != segmentID { + // Don't think this is gonna to happen, ignore for now. + log.Warn("checkpoint in segment is not same as flush segment to update, igreo", zap.Int64("current", segmentID), zap.Int64("checkpoint segment", cp.SegmentID)) + continue + } + + if segment.DmlPosition != nil && segment.DmlPosition.Timestamp >= cp.Position.Timestamp { + log.Warn("checkpoint in segment is larger than reported", zap.Any("current", segment.GetDmlPosition()), zap.Any("reported", cp.GetPosition())) + // segment position in etcd is larger than checkpoint, then dont change it + continue + } + + segment.NumOfRows = cp.NumOfRows + segment.DmlPosition = cp.GetPosition() } } - return nil - } - // binlogs - for _, tBinlogs := range binlogs { - fieldBinlogs := getFieldBinlogs(tBinlogs.GetFieldID(), currBinlogs) - if fieldBinlogs == nil { - currBinlogs = append(currBinlogs, tBinlogs) - } else { - fieldBinlogs.Binlogs = append(fieldBinlogs.Binlogs, tBinlogs.Binlogs...) - } - } - clonedSegment.Binlogs = currBinlogs - // statlogs - currStatsLogs := clonedSegment.GetStatslogs() - for _, tStatsLogs := range statslogs { - fieldStatsLog := getFieldBinlogs(tStatsLogs.GetFieldID(), currStatsLogs) - if fieldStatsLog == nil { - currStatsLogs = append(currStatsLogs, tStatsLogs) - } else { - fieldStatsLog.Binlogs = append(fieldStatsLog.Binlogs, tStatsLogs.Binlogs...) - } - } - clonedSegment.Statslogs = currStatsLogs - // deltalogs - currDeltaLogs := clonedSegment.GetDeltalogs() - for _, tDeltaLogs := range deltalogs { - fieldDeltaLogs := getFieldBinlogs(tDeltaLogs.GetFieldID(), currDeltaLogs) - if fieldDeltaLogs == nil { - currDeltaLogs = append(currDeltaLogs, tDeltaLogs) - } else { - fieldDeltaLogs.Binlogs = append(fieldDeltaLogs.Binlogs, tDeltaLogs.Binlogs...) - } - } - clonedSegment.Deltalogs = currDeltaLogs - modSegments[segmentID] = clonedSegment - getClonedSegment := func(segmentID UniqueID) *SegmentInfo { - if s, ok := modSegments[segmentID]; ok { - return s - } - if s := m.segments.GetSegment(segmentID); s != nil && isSegmentHealthy(s) { - return s.Clone() - } - return nil - } - for _, pos := range startPositions { - if len(pos.GetStartPosition().GetMsgID()) == 0 { - continue - } - s := getClonedSegment(pos.GetSegmentID()) - if s == nil { - continue - } - s.StartPosition = pos.GetStartPosition() - modSegments[pos.GetSegmentID()] = s - } - - if importing { - s := clonedSegment - s.NumOfRows = s.currRows - count := segmentutil.CalcRowCountFromBinLog(s.SegmentInfo) + count := segmentutil.CalcRowCountFromBinLog(segment.SegmentInfo) if count != segment.currRows && count > 0 { log.Info("check point reported inconsistent with bin log row count", - zap.Int64("segmentID", segment.GetID()), zap.Int64("current rows (wrong)", segment.currRows), zap.Int64("segment bin log row count (correct)", count)) - s.NumOfRows = count + segment.NumOfRows = count } - modSegments[segmentID] = s - } else { - for _, cp := range checkpoints { - if cp.SegmentID != segmentID { - // Don't think this is gonna to happen, ignore for now. - log.Warn("checkpoint in segment is not same as flush segment to update, igreo", zap.Int64("current", segmentID), zap.Int64("checkpoint segment", cp.SegmentID)) - continue - } - s := clonedSegment + return true + } +} - if s.DmlPosition != nil && s.DmlPosition.Timestamp >= cp.Position.Timestamp { - log.Warn("checkpoint in segment is larger than reported", zap.Any("current", s.GetDmlPosition()), zap.Any("reported", cp.GetPosition())) - // segment position in etcd is larger than checkpoint, then dont change it - continue - } +// updateSegmentsInfo update segment infos +// will exec all operators, and update all changed segments +func (m *meta) UpdateSegmentsInfo(operators ...UpdateOperator) error { + m.Lock() + defer m.Unlock() + updatePack := &UpdateSegmentPack{ + meta: m, + segments: make(map[int64]*SegmentInfo), + increments: make(map[int64]metastore.BinlogsIncrement), + metricMutation: &segMetricMutation{ + stateChange: make(map[string]int), + }, + } - s.NumOfRows = cp.NumOfRows - count := segmentutil.CalcRowCountFromBinLog(s.SegmentInfo) - // count should smaller than or equal to cp reported - if count != cp.NumOfRows && count > 0 { - log.Info("check point reported inconsistent with bin log row count", - zap.Int64("segmentID", segment.GetID()), - zap.Int64("check point (wrong)", cp.NumOfRows), - zap.Int64("segment bin log row count (correct)", count)) - s.NumOfRows = count - } - - s.DmlPosition = cp.GetPosition() - modSegments[cp.GetSegmentID()] = s + for _, operator := range operators { + ok := operator(updatePack) + if !ok { + return nil } } - segments := make([]*datapb.SegmentInfo, 0, len(modSegments)) - for _, seg := range modSegments { - segments = append(segments, seg.SegmentInfo) - } - if err := m.catalog.AlterSegments(m.ctx, segments, - metastore.BinlogsIncrement{ - Segment: clonedSegment.SegmentInfo, - }); err != nil { + + segments := lo.MapToSlice(updatePack.segments, func(_ int64, segment *SegmentInfo) *datapb.SegmentInfo { return segment.SegmentInfo }) + increments := lo.Values(updatePack.increments) + + if err := m.catalog.AlterSegments(m.ctx, segments, increments...); err != nil { log.Error("meta update: update flush segments info - failed to store flush segment info into Etcd", zap.Error(err)) return err } // Apply metric mutation after a successful meta update. - metricMutation.commit() + updatePack.metricMutation.commit() // update memory status - for id, s := range modSegments { + for id, s := range updatePack.segments { m.segments.SetSegment(id, s) } - log.Info("meta update: update flush segments info - update flush segments info successfully", - zap.Int64("segmentID", segmentID)) + log.Info("meta update: update flush segments info - update flush segments info successfully") return nil } diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index 87b122c3df..6de995530a 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -488,22 +488,29 @@ func TestGetUnFlushedSegments(t *testing.T) { assert.NotEqualValues(t, commonpb.SegmentState_Flushed, segments[0].State) } -func TestUpdateFlushSegmentsInfo(t *testing.T) { +func TestUpdateSegmentsInfo(t *testing.T) { t.Run("normal", func(t *testing.T) { meta, err := newMemoryMeta() assert.NoError(t, err) segment1 := &SegmentInfo{SegmentInfo: &datapb.SegmentInfo{ - ID: 1, State: commonpb.SegmentState_Growing, Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(1, getInsertLogPath("binlog0", 1))}, + ID: 1, State: commonpb.SegmentState_Growing, + Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(1, getInsertLogPath("binlog0", 1))}, Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(1, getStatsLogPath("statslog0", 1))}, }} err = meta.AddSegment(context.TODO(), segment1) assert.NoError(t, err) - err = meta.UpdateFlushSegmentsInfo(1, true, false, false, []*datapb.FieldBinlog{getFieldBinlogPathsWithEntry(1, 10, getInsertLogPath("binlog1", 1))}, - []*datapb.FieldBinlog{getFieldBinlogPaths(1, getStatsLogPath("statslog1", 1))}, - []*datapb.FieldBinlog{{Binlogs: []*datapb.Binlog{{EntriesNum: 1, TimestampFrom: 100, TimestampTo: 200, LogSize: 1000, LogPath: getDeltaLogPath("deltalog1", 1)}}}}, - []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}, []*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}) + err = meta.UpdateSegmentsInfo( + UpdateStatusOperator(1, commonpb.SegmentState_Flushing), + UpdateBinlogsOperator(1, + []*datapb.FieldBinlog{getFieldBinlogPathsWithEntry(1, 10, getInsertLogPath("binlog1", 1))}, + []*datapb.FieldBinlog{getFieldBinlogPaths(1, getStatsLogPath("statslog1", 1))}, + []*datapb.FieldBinlog{{Binlogs: []*datapb.Binlog{{EntriesNum: 1, TimestampFrom: 100, TimestampTo: 200, LogSize: 1000, LogPath: getDeltaLogPath("deltalog1", 1)}}}}, + ), + UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}), + UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}), + ) assert.NoError(t, err) updated := meta.GetHealthySegment(1) @@ -530,7 +537,24 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) { meta, err := newMemoryMeta() assert.NoError(t, err) - err = meta.UpdateFlushSegmentsInfo(1, false, false, false, nil, nil, nil, nil, nil) + err = meta.UpdateSegmentsInfo( + UpdateStatusOperator(1, commonpb.SegmentState_Flushing), + ) + assert.NoError(t, err) + + err = meta.UpdateSegmentsInfo( + UpdateBinlogsOperator(1, nil, nil, nil), + ) + assert.NoError(t, err) + + err = meta.UpdateSegmentsInfo( + UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}), + ) + assert.NoError(t, err) + + err = meta.UpdateSegmentsInfo( + UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}), + ) assert.NoError(t, err) }) @@ -542,9 +566,10 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) { err = meta.AddSegment(context.TODO(), segment1) assert.NoError(t, err) - err = meta.UpdateFlushSegmentsInfo(1, false, false, false, nil, nil, nil, []*datapb.CheckPoint{{SegmentID: 2, NumOfRows: 10}}, + err = meta.UpdateSegmentsInfo( + UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 2, NumOfRows: 10}}), + ) - []*datapb.SegmentStartPosition{{SegmentID: 2, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}) assert.NoError(t, err) assert.Nil(t, meta.GetHealthySegment(2)) }) @@ -568,10 +593,17 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) { } meta.segments.SetSegment(1, segmentInfo) - err = meta.UpdateFlushSegmentsInfo(1, true, false, false, []*datapb.FieldBinlog{getFieldBinlogPaths(1, getInsertLogPath("binlog", 1))}, - []*datapb.FieldBinlog{getFieldBinlogPaths(1, getInsertLogPath("statslog", 1))}, - []*datapb.FieldBinlog{{Binlogs: []*datapb.Binlog{{EntriesNum: 1, TimestampFrom: 100, TimestampTo: 200, LogSize: 1000, LogPath: getDeltaLogPath("deltalog", 1)}}}}, - []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}, []*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}) + err = meta.UpdateSegmentsInfo( + UpdateStatusOperator(1, commonpb.SegmentState_Flushing), + UpdateBinlogsOperator(1, + []*datapb.FieldBinlog{getFieldBinlogPaths(1, getInsertLogPath("binlog", 1))}, + []*datapb.FieldBinlog{getFieldBinlogPaths(1, getInsertLogPath("statslog", 1))}, + []*datapb.FieldBinlog{{Binlogs: []*datapb.Binlog{{EntriesNum: 1, TimestampFrom: 100, TimestampTo: 200, LogSize: 1000, LogPath: getDeltaLogPath("deltalog", 1)}}}}, + ), + UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}), + UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}), + ) + assert.Error(t, err) assert.Equal(t, "mocked fail", err.Error()) segmentInfo = meta.GetHealthySegment(1) @@ -599,6 +631,12 @@ func TestMeta_alterMetaStore(t *testing.T) { Binlogs: []*datapb.Binlog{}, }, }, + Statslogs: []*datapb.FieldBinlog{ + { + FieldID: 101, + Binlogs: []*datapb.Binlog{}, + }, + }, Deltalogs: []*datapb.FieldBinlog{ { FieldID: 101, diff --git a/internal/datacoord/segment_allocation_policy.go b/internal/datacoord/segment_allocation_policy.go index 0a0271c2db..9f8a8bd2e3 100644 --- a/internal/datacoord/segment_allocation_policy.go +++ b/internal/datacoord/segment_allocation_policy.go @@ -24,6 +24,7 @@ import ( "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/proto/datapb" "github.com/milvus-io/milvus/pkg/util/tsoutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -64,19 +65,19 @@ func calBySchemaPolicyWithDiskIndex(schema *schemapb.CollectionSchema) (int, err // AllocatePolicy helper function definition to allocate Segment space type AllocatePolicy func(segments []*SegmentInfo, count int64, - maxCountPerSegment int64) ([]*Allocation, []*Allocation) + maxCountPerL1Segment int64, level datapb.SegmentLevel) ([]*Allocation, []*Allocation) -// AllocatePolicyV1 v1 policy simple allocation policy using Greedy Algorithm -func AllocatePolicyV1(segments []*SegmentInfo, count int64, - maxCountPerSegment int64, +// alloca policy for L1 segment +func AllocatePolicyL1(segments []*SegmentInfo, count int64, + maxCountPerL1Segment int64, level datapb.SegmentLevel, ) ([]*Allocation, []*Allocation) { newSegmentAllocations := make([]*Allocation, 0) existedSegmentAllocations := make([]*Allocation, 0) // create new segment if count >= max num - for count >= maxCountPerSegment { - allocation := getAllocation(maxCountPerSegment) + for count >= maxCountPerL1Segment { + allocation := getAllocation(maxCountPerL1Segment) newSegmentAllocations = append(newSegmentAllocations, allocation) - count -= maxCountPerSegment + count -= maxCountPerL1Segment } // allocate space for remaining count @@ -107,19 +108,15 @@ func AllocatePolicyV1(segments []*SegmentInfo, count int64, // segmentSealPolicy seal policy applies to segment type segmentSealPolicy func(segment *SegmentInfo, ts Timestamp) bool -// getSegmentCapacityPolicy get segmentSealPolicy with segment size factor policy -func getSegmentCapacityPolicy(sizeFactor float64) segmentSealPolicy { +// sealL1SegmentByCapacity get segmentSealPolicy with segment size factor policy +func sealL1SegmentByCapacity(sizeFactor float64) segmentSealPolicy { return func(segment *SegmentInfo, ts Timestamp) bool { - var allocSize int64 - for _, allocation := range segment.allocations { - allocSize += allocation.NumOfRows - } return float64(segment.currRows) >= sizeFactor*float64(segment.GetMaxRowNum()) } } -// sealByLifetimePolicy get segmentSealPolicy with lifetime limit compares ts - segment.lastExpireTime -func sealByLifetimePolicy(lifetime time.Duration) segmentSealPolicy { +// sealL1SegmentByLifetimePolicy get segmentSealPolicy with lifetime limit compares ts - segment.lastExpireTime +func sealL1SegmentByLifetime(lifetime time.Duration) segmentSealPolicy { return func(segment *SegmentInfo, ts Timestamp) bool { pts, _ := tsoutil.ParseTS(ts) epts, _ := tsoutil.ParseTS(segment.GetLastExpireTime()) @@ -128,8 +125,8 @@ func sealByLifetimePolicy(lifetime time.Duration) segmentSealPolicy { } } -// sealByMaxBinlogFileNumberPolicy seal segment if binlog file number of segment exceed configured max number -func sealByMaxBinlogFileNumberPolicy(maxBinlogFileNumber int) segmentSealPolicy { +// sealL1SegmentByBinlogFileNumber seal L1 segment if binlog file number of segment exceed configured max number +func sealL1SegmentByBinlogFileNumber(maxBinlogFileNumber int) segmentSealPolicy { return func(segment *SegmentInfo, ts Timestamp) bool { logFileCounter := 0 for _, fieldBinlog := range segment.GetStatslogs() { @@ -148,7 +145,7 @@ func sealByMaxBinlogFileNumberPolicy(maxBinlogFileNumber int) segmentSealPolicy // into this segment anymore, so sealLongTimeIdlePolicy will seal these segments to trigger handoff of query cluster. // Q: Why we don't decrease the expiry time directly? // A: We don't want to influence segments which are accepting `frequent small` batch entities. -func sealLongTimeIdlePolicy(idleTimeTolerance time.Duration, minSizeToSealIdleSegment float64, maxSizeOfSegment float64) segmentSealPolicy { +func sealL1SegmentByIdleTime(idleTimeTolerance time.Duration, minSizeToSealIdleSegment float64, maxSizeOfSegment float64) segmentSealPolicy { return func(segment *SegmentInfo, ts Timestamp) bool { limit := (minSizeToSealIdleSegment / maxSizeOfSegment) * float64(segment.GetMaxRowNum()) return time.Since(segment.lastWrittenTime) > idleTimeTolerance && @@ -185,8 +182,8 @@ type flushPolicy func(segment *SegmentInfo, t Timestamp) bool const flushInterval = 2 * time.Second -func flushPolicyV1(segment *SegmentInfo, t Timestamp) bool { - return segment.GetState() == commonpb.SegmentState_Sealed && +func flushPolicyL1(segment *SegmentInfo, t Timestamp) bool { + return segment.GetState() == commonpb.SegmentState_Sealed && segment.Level != datapb.SegmentLevel_L0 && time.Since(segment.lastFlushTime) >= flushInterval && (segment.GetLastExpireTime() <= t && segment.currRows != 0 || (segment.IsImporting)) } diff --git a/internal/datacoord/segment_allocation_policy_test.go b/internal/datacoord/segment_allocation_policy_test.go index 250d4b55b3..e27c4a2d68 100644 --- a/internal/datacoord/segment_allocation_policy_test.go +++ b/internal/datacoord/segment_allocation_policy_test.go @@ -163,7 +163,7 @@ func TestSealSegmentPolicy(t *testing.T) { nosealTs := (now.Add(lifetime / 2)).UnixNano() / int64(time.Millisecond) sealTs := (now.Add(lifetime)).UnixNano() / int64(time.Millisecond) - p := sealByLifetimePolicy(lifetime) + p := sealL1SegmentByLifetime(lifetime) segment := &SegmentInfo{ SegmentInfo: &datapb.SegmentInfo{ @@ -184,7 +184,7 @@ func Test_sealLongTimeIdlePolicy(t *testing.T) { idleTimeTolerance := 2 * time.Second minSizeToSealIdleSegment := 16.0 maxSizeOfSegment := 512.0 - policy := sealLongTimeIdlePolicy(idleTimeTolerance, minSizeToSealIdleSegment, maxSizeOfSegment) + policy := sealL1SegmentByIdleTime(idleTimeTolerance, minSizeToSealIdleSegment, maxSizeOfSegment) seg1 := &SegmentInfo{lastWrittenTime: time.Now().Add(idleTimeTolerance * 5)} assert.False(t, policy(seg1, 100)) seg2 := &SegmentInfo{lastWrittenTime: getZeroTime(), currRows: 1, SegmentInfo: &datapb.SegmentInfo{MaxRowNum: 10000}} diff --git a/internal/datacoord/segment_manager.go b/internal/datacoord/segment_manager.go index 08b2206755..8298362892 100644 --- a/internal/datacoord/segment_manager.go +++ b/internal/datacoord/segment_manager.go @@ -66,6 +66,8 @@ func putAllocation(a *Allocation) { // Manager manages segment related operations. type Manager interface { + // CreateSegment create new segment when segment not exist + // AllocSegment allocates rows and record the allocation. AllocSegment(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64) ([]*Allocation, error) // allocSegmentForImport allocates one segment allocation for bulk insert. @@ -99,7 +101,7 @@ func (alloc *Allocation) String() string { // make sure SegmentManager implements Manager var _ Manager = (*SegmentManager)(nil) -// SegmentManager handles segment related logic +// SegmentManager handles L1 segment related logic type SegmentManager struct { meta *meta mu sync.RWMutex @@ -178,20 +180,20 @@ func defaultCalUpperLimitPolicy() calUpperLimitPolicy { } func defaultAllocatePolicy() AllocatePolicy { - return AllocatePolicyV1 + return AllocatePolicyL1 } func defaultSegmentSealPolicy() []segmentSealPolicy { return []segmentSealPolicy{ - sealByMaxBinlogFileNumberPolicy(Params.DataCoordCfg.SegmentMaxBinlogFileNumber.GetAsInt()), - sealByLifetimePolicy(Params.DataCoordCfg.SegmentMaxLifetime.GetAsDuration(time.Second)), - getSegmentCapacityPolicy(Params.DataCoordCfg.SegmentSealProportion.GetAsFloat()), - sealLongTimeIdlePolicy(Params.DataCoordCfg.SegmentMaxIdleTime.GetAsDuration(time.Second), Params.DataCoordCfg.SegmentMinSizeFromIdleToSealed.GetAsFloat(), Params.DataCoordCfg.SegmentMaxSize.GetAsFloat()), + sealL1SegmentByBinlogFileNumber(Params.DataCoordCfg.SegmentMaxBinlogFileNumber.GetAsInt()), + sealL1SegmentByLifetime(Params.DataCoordCfg.SegmentMaxLifetime.GetAsDuration(time.Second)), + sealL1SegmentByCapacity(Params.DataCoordCfg.SegmentSealProportion.GetAsFloat()), + sealL1SegmentByIdleTime(Params.DataCoordCfg.SegmentMaxIdleTime.GetAsDuration(time.Second), Params.DataCoordCfg.SegmentMinSizeFromIdleToSealed.GetAsFloat(), Params.DataCoordCfg.SegmentMaxSize.GetAsFloat()), } } func defaultFlushPolicy() flushPolicy { - return flushPolicyV1 + return flushPolicyL1 } // newSegmentManager should be the only way to retrieve SegmentManager. @@ -222,7 +224,9 @@ func (s *SegmentManager) loadSegmentsFromMeta() { segments := s.meta.GetUnFlushedSegments() segmentsID := make([]UniqueID, 0, len(segments)) for _, segment := range segments { - segmentsID = append(segmentsID, segment.GetID()) + if segment.Level != datapb.SegmentLevel_L0 { + segmentsID = append(segmentsID, segment.GetID()) + } } s.segments = segmentsID } @@ -275,7 +279,7 @@ func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID log.Warn("Failed to get segment info from meta", zap.Int64("id", segmentID)) continue } - if !satisfy(segment, collectionID, partitionID, channelName) || !isGrowing(segment) { + if !satisfy(segment, collectionID, partitionID, channelName) || !isGrowing(segment) || segment.GetLevel() == datapb.SegmentLevel_L0 { continue } segments = append(segments, segment) @@ -287,7 +291,7 @@ func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID return nil, err } newSegmentAllocations, existedSegmentAllocations := s.allocPolicy(segments, - requestRows, int64(maxCountPerSegment)) + requestRows, int64(maxCountPerSegment), datapb.SegmentLevel_L1) // create new segments and add allocations expireTs, err := s.genExpireTs(ctx, false) @@ -295,7 +299,7 @@ func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID return nil, err } for _, allocation := range newSegmentAllocations { - segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName, commonpb.SegmentState_Growing) + segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName, commonpb.SegmentState_Growing, datapb.SegmentLevel_L1) if err != nil { log.Error("Failed to open new segment for segment allocation") return nil, err @@ -337,7 +341,7 @@ func (s *SegmentManager) allocSegmentForImport(ctx context.Context, collectionID return nil, err } - segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName, commonpb.SegmentState_Importing) + segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName, commonpb.SegmentState_Importing, datapb.SegmentLevel_L1) if err != nil { return nil, err } @@ -375,7 +379,7 @@ func (s *SegmentManager) genExpireTs(ctx context.Context, isImported bool) (Time } func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID UniqueID, partitionID UniqueID, - channelName string, segmentState commonpb.SegmentState, + channelName string, segmentState commonpb.SegmentState, level datapb.SegmentLevel, ) (*SegmentInfo, error) { log := log.Ctx(ctx) ctx, sp := otel.Tracer(typeutil.DataCoordRole).Start(ctx, "open-Segment") @@ -399,6 +403,7 @@ func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID Unique NumOfRows: 0, State: segmentState, MaxRowNum: int64(maxNumOfRows), + Level: level, LastExpireTime: 0, } if segmentState == commonpb.SegmentState_Importing { diff --git a/internal/datacoord/segment_manager_test.go b/internal/datacoord/segment_manager_test.go index b99e3004d2..64d424fd68 100644 --- a/internal/datacoord/segment_manager_test.go +++ b/internal/datacoord/segment_manager_test.go @@ -548,7 +548,7 @@ func TestTryToSealSegment(t *testing.T) { collID, err := mockAllocator.allocID(context.Background()) assert.NoError(t, err) meta.AddCollection(&collectionInfo{ID: collID, Schema: schema}) - segmentManager, _ := newSegmentManager(meta, mockAllocator, withSegmentSealPolices(sealByLifetimePolicy(math.MinInt64))) // always seal + segmentManager, _ := newSegmentManager(meta, mockAllocator, withSegmentSealPolices(sealL1SegmentByLifetime(math.MinInt64))) // always seal allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2) assert.NoError(t, err) assert.EqualValues(t, 1, len(allocations)) @@ -599,7 +599,7 @@ func TestTryToSealSegment(t *testing.T) { assert.NoError(t, err) meta.AddCollection(&collectionInfo{ID: collID, Schema: schema}) segmentManager, _ := newSegmentManager(meta, mockAllocator, - withSegmentSealPolices(sealByLifetimePolicy(math.MinInt64)), + withSegmentSealPolices(sealL1SegmentByLifetime(math.MinInt64)), withChannelSealPolices(getChannelOpenSegCapacityPolicy(-1))) // always seal allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2) assert.NoError(t, err) @@ -646,7 +646,7 @@ func TestTryToSealSegment(t *testing.T) { // Not trigger seal { - segmentManager.segmentSealPolicies = []segmentSealPolicy{sealByMaxBinlogFileNumberPolicy(2)} + segmentManager.segmentSealPolicies = []segmentSealPolicy{sealL1SegmentByLifetime(2)} segments := segmentManager.meta.segments.segments assert.Equal(t, 1, len(segments)) for _, seg := range segments { @@ -671,7 +671,7 @@ func TestTryToSealSegment(t *testing.T) { // Trigger seal { - segmentManager.segmentSealPolicies = []segmentSealPolicy{sealByMaxBinlogFileNumberPolicy(2)} + segmentManager.segmentSealPolicies = []segmentSealPolicy{sealL1SegmentByBinlogFileNumber(2)} segments := segmentManager.meta.segments.segments assert.Equal(t, 1, len(segments)) for _, seg := range segments { @@ -712,7 +712,7 @@ func TestTryToSealSegment(t *testing.T) { collID, err := mockAllocator.allocID(context.Background()) assert.NoError(t, err) meta.AddCollection(&collectionInfo{ID: collID, Schema: schema}) - segmentManager, _ := newSegmentManager(meta, mockAllocator, withSegmentSealPolices(sealByLifetimePolicy(math.MinInt64))) // always seal + segmentManager, _ := newSegmentManager(meta, mockAllocator, withSegmentSealPolices(sealL1SegmentByLifetime(math.MinInt64))) // always seal allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2) assert.NoError(t, err) assert.EqualValues(t, 1, len(allocations)) diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 2a869654c6..7eed9137a7 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -1334,6 +1334,21 @@ func TestSaveBinlogPaths(t *testing.T) { }, }, }, + Field2StatslogPaths: []*datapb.FieldBinlog{ + { + FieldID: 1, + Binlogs: []*datapb.Binlog{ + { + LogPath: "/by-dev/test_stats/0/1/1/1/Allo1", + EntriesNum: 5, + }, + { + LogPath: "/by-dev/test_stats/0/1/1/1/Allo2", + EntriesNum: 5, + }, + }, + }, + }, CheckPoints: []*datapb.CheckPoint{ { SegmentID: 1, diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index e768348bc3..0c88f33ada 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -183,7 +183,9 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI zap.String("channelName", r.GetChannelName()), zap.Uint32("count", r.GetCount()), zap.Bool("isImport", r.GetIsImport()), - zap.Int64("import task ID", r.GetImportTaskID())) + zap.Int64("import task ID", r.GetImportTaskID()), + zap.String("segment level", r.GetLevel().String()), + ) // Load the collection info from Root Coordinator, if it is not found in server meta. // Note: this request wouldn't be received if collection didn't exist. @@ -449,37 +451,51 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath // validate segmentID := req.GetSegmentID() segment := s.meta.GetSegment(segmentID) - + operators := []UpdateOperator{} + // if L1 segment not exist + // return error + // but if L0 segment not exist + // will create it if segment == nil { - err := merr.WrapErrSegmentNotFound(segmentID) - log.Warn("failed to get segment", zap.Error(err)) - return merr.Status(err), nil - } + if req.SegLevel != datapb.SegmentLevel_L0 { + err := merr.WrapErrSegmentNotFound(segmentID) + log.Warn("failed to get segment", zap.Error(err)) + return merr.Status(err), nil + } - if segment.State == commonpb.SegmentState_Dropped { - log.Info("save to dropped segment, ignore this request") - return merr.Success(), nil - } else if !isSegmentHealthy(segment) { - err := merr.WrapErrSegmentNotFound(segmentID) - log.Warn("failed to get segment, the segment not healthy", zap.Error(err)) - return merr.Status(err), nil + operators = append(operators, CreateL0Operator(req.GetCollectionID(), req.GetPartitionID(), req.GetSegmentID(), req.GetChannel())) + } else { + if segment.State == commonpb.SegmentState_Dropped { + log.Info("save to dropped segment, ignore this request") + return merr.Success(), nil + } + + if !isSegmentHealthy(segment) { + err := merr.WrapErrSegmentNotFound(segmentID) + log.Warn("failed to get segment, the segment not healthy", zap.Error(err)) + return merr.Status(err), nil + } } if req.GetDropped() { s.segmentManager.DropSegment(ctx, segment.GetID()) + operators = append(operators, UpdateStatusOperator(segmentID, commonpb.SegmentState_Dropped)) + } else if req.GetFlushed() { + // set segment to SegmentState_Flushing + operators = append(operators, UpdateStatusOperator(segmentID, commonpb.SegmentState_Flushing)) } - // Set segment to SegmentState_Flushing. Also save binlogs and checkpoints. - err := s.meta.UpdateFlushSegmentsInfo( - req.GetSegmentID(), - req.GetFlushed(), - req.GetDropped(), - req.GetImporting(), - req.GetField2BinlogPaths(), - req.GetField2StatslogPaths(), - req.GetDeltalogs(), - req.GetCheckPoints(), - req.GetStartPositions()) + // save binlogs + operators = append(operators, UpdateBinlogsOperator(segmentID, req.GetField2BinlogPaths(), req.GetField2StatslogPaths(), req.GetDeltalogs())) + + // save startPositions of some other segments + operators = append(operators, UpdateStartPosition(req.GetStartPositions())) + + // save checkpoints. + operators = append(operators, UpdateCheckPointOperator(segmentID, req.GetImporting(), req.GetCheckPoints())) + + // run all operator and update new segment info + err := s.meta.UpdateSegmentsInfo(operators...) if err != nil { log.Error("save binlog and checkpoints failed", zap.Error(err)) return merr.Status(err), nil diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index 37bafea790..901d770109 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -189,3 +189,24 @@ func parseBuildIDFromFilePath(key string) (UniqueID, error) { } return strconv.ParseInt(ss[len(ss)-1], 10, 64) } + +func getFieldBinlogs(id UniqueID, binlogs []*datapb.FieldBinlog) *datapb.FieldBinlog { + for _, binlog := range binlogs { + if id == binlog.GetFieldID() { + return binlog + } + } + return nil +} + +func mergeFieldBinlogs(currentBinlogs []*datapb.FieldBinlog, newBinlogs []*datapb.FieldBinlog) []*datapb.FieldBinlog { + for _, newBinlog := range newBinlogs { + fieldBinlogs := getFieldBinlogs(newBinlog.GetFieldID(), currentBinlogs) + if fieldBinlogs == nil { + currentBinlogs = append(currentBinlogs, newBinlog) + } else { + fieldBinlogs.Binlogs = append(fieldBinlogs.Binlogs, newBinlog.Binlogs...) + } + } + return currentBinlogs +} diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index b1864d43dc..8a5a7a3ae7 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -41,7 +41,6 @@ import ( "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" - "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -289,9 +288,13 @@ func (kc *Catalog) AlterSegments(ctx context.Context, segments []*datapb.Segment for _, b := range binlogs { segment := b.Segment + + if err := ValidateSegment(segment); err != nil { + return err + } + binlogKvs, err := buildBinlogKvsWithLogID(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID(), - cloneLogs(segment.GetBinlogs()), cloneLogs(segment.GetDeltalogs()), cloneLogs(segment.GetStatslogs()), - len(segment.GetCompactionFrom()) > 0) + cloneLogs(segment.GetBinlogs()), cloneLogs(segment.GetDeltalogs()), cloneLogs(segment.GetStatslogs())) if err != nil { return err } @@ -309,7 +312,7 @@ func (kc *Catalog) handleDroppedSegment(segment *datapb.SegmentInfo) (kvs map[st } // To be compatible with previous implementation, we have to write binlogs on etcd for correct gc. if !has { - kvs, err = buildBinlogKvsWithLogID(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID(), cloneLogs(segment.GetBinlogs()), cloneLogs(segment.GetDeltalogs()), cloneLogs(segment.GetStatslogs()), true) + kvs, err = buildBinlogKvsWithLogID(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID(), cloneLogs(segment.GetBinlogs()), cloneLogs(segment.GetDeltalogs()), cloneLogs(segment.GetStatslogs())) if err != nil { return } @@ -351,20 +354,6 @@ func (kc *Catalog) SaveByBatch(kvs map[string]string) error { return nil } -func resetBinlogFields(segment *datapb.SegmentInfo) { - segment.Binlogs = nil - segment.Deltalogs = nil - segment.Statslogs = nil -} - -func cloneLogs(binlogs []*datapb.FieldBinlog) []*datapb.FieldBinlog { - var res []*datapb.FieldBinlog - for _, log := range binlogs { - res = append(res, proto.Clone(log).(*datapb.FieldBinlog)) - } - return res -} - func (kc *Catalog) collectMetrics(s *datapb.SegmentInfo) { statsFieldFn := func(fieldBinlogs []*datapb.FieldBinlog) int { cnt := 0 @@ -737,333 +726,3 @@ func fillLogPathByLogID(chunkManagerRootPath string, binlogType storage.BinlogTy binlog.LogPath = path } } - -func fillLogIDByLogPath(multiFieldBinlogs ...[]*datapb.FieldBinlog) error { - for _, fieldBinlogs := range multiFieldBinlogs { - for _, fieldBinlog := range fieldBinlogs { - for _, binlog := range fieldBinlog.Binlogs { - logPath := binlog.LogPath - idx := strings.LastIndex(logPath, "/") - if idx == -1 { - return fmt.Errorf("invailed binlog path: %s", logPath) - } - logPathStr := logPath[(idx + 1):] - logID, err := strconv.ParseInt(logPathStr, 10, 64) - if err != nil { - return err - } - - // set log path to empty and only store log id - binlog.LogPath = "" - binlog.LogID = logID - } - } - } - return nil -} - -func CompressBinLog(fieldBinLogs []*datapb.FieldBinlog) ([]*datapb.FieldBinlog, error) { - compressedFieldBinLogs := make([]*datapb.FieldBinlog, 0) - for _, fieldBinLog := range fieldBinLogs { - compressedFieldBinLog := &datapb.FieldBinlog{} - compressedFieldBinLog.FieldID = fieldBinLog.FieldID - for _, binlog := range fieldBinLog.Binlogs { - logPath := binlog.LogPath - idx := strings.LastIndex(logPath, "/") - if idx == -1 { - return nil, fmt.Errorf("invailed binlog path: %s", logPath) - } - logPathStr := logPath[(idx + 1):] - logID, err := strconv.ParseInt(logPathStr, 10, 64) - if err != nil { - return nil, err - } - binlog := &datapb.Binlog{ - EntriesNum: binlog.EntriesNum, - // remove timestamp since it's not necessary - LogSize: binlog.LogSize, - LogID: logID, - } - compressedFieldBinLog.Binlogs = append(compressedFieldBinLog.Binlogs, binlog) - } - compressedFieldBinLogs = append(compressedFieldBinLogs, compressedFieldBinLog) - } - return compressedFieldBinLogs, nil -} - -func DecompressBinLog(path string, info *datapb.SegmentInfo) error { - for _, fieldBinLogs := range info.GetBinlogs() { - fillLogPathByLogID(path, storage.InsertBinlog, info.CollectionID, info.PartitionID, info.ID, fieldBinLogs) - } - - for _, deltaLogs := range info.GetDeltalogs() { - fillLogPathByLogID(path, storage.DeleteBinlog, info.CollectionID, info.PartitionID, info.ID, deltaLogs) - } - - for _, statsLogs := range info.GetStatslogs() { - fillLogPathByLogID(path, storage.StatsBinlog, info.CollectionID, info.PartitionID, info.ID, statsLogs) - } - return nil -} - -// build a binlog path on the storage by metadata -func buildLogPath(chunkManagerRootPath string, binlogType storage.BinlogType, collectionID, partitionID, segmentID, filedID, logID typeutil.UniqueID) string { - switch binlogType { - case storage.InsertBinlog: - return metautil.BuildInsertLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, filedID, logID) - case storage.DeleteBinlog: - return metautil.BuildDeltaLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, logID) - case storage.StatsBinlog: - return metautil.BuildStatsLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, filedID, logID) - } - // should not happen - log.Panic("invalid binlog type", zap.Any("type", binlogType)) - return "" -} - -func checkBinlogs(binlogType storage.BinlogType, segmentID typeutil.UniqueID, logs []*datapb.FieldBinlog) error { - check := func(getSegmentID func(logPath string) typeutil.UniqueID) error { - for _, fieldBinlog := range logs { - for _, binlog := range fieldBinlog.Binlogs { - if segmentID != getSegmentID(binlog.LogPath) { - return fmt.Errorf("the segment path doesn't match the segmentID, segmentID %d, path %s", segmentID, binlog.LogPath) - } - } - } - return nil - } - switch binlogType { - case storage.InsertBinlog: - return check(metautil.GetSegmentIDFromInsertLogPath) - case storage.DeleteBinlog: - return check(metautil.GetSegmentIDFromDeltaLogPath) - case storage.StatsBinlog: - return check(metautil.GetSegmentIDFromStatsLogPath) - default: - return fmt.Errorf("the segment path doesn't match the segmentID, segmentID %d, type %d", segmentID, binlogType) - } -} - -func hasSepcialStatslog(logs *datapb.FieldBinlog) bool { - for _, statslog := range logs.Binlogs { - _, logidx := path.Split(statslog.LogPath) - if logidx == storage.CompoundStatsType.LogIdx() { - return true - } - } - return false -} - -func buildBinlogKvsWithLogID(collectionID, partitionID, segmentID typeutil.UniqueID, - binlogs, deltalogs, statslogs []*datapb.FieldBinlog, ignoreNumberCheck bool, -) (map[string]string, error) { - err := checkBinlogs(storage.InsertBinlog, segmentID, binlogs) - if err != nil { - return nil, err - } - checkBinlogs(storage.DeleteBinlog, segmentID, deltalogs) - if err != nil { - return nil, err - } - checkBinlogs(storage.StatsBinlog, segmentID, statslogs) - if err != nil { - return nil, err - } - // check stats log and bin log size match - // num of stats log may one more than num of binlogs if segment flushed and merged stats log - if !ignoreNumberCheck && len(binlogs) != 0 && len(statslogs) != 0 && !hasSepcialStatslog(statslogs[0]) { - if len(binlogs[0].GetBinlogs()) != len(statslogs[0].GetBinlogs()) { - log.Warn("find invalid segment while bin log size didn't match stat log size", - zap.Int64("collection", collectionID), - zap.Int64("partition", partitionID), - zap.Int64("segment", segmentID), - zap.Any("binlogs", binlogs), - zap.Any("stats", statslogs), - zap.Any("delta", deltalogs), - ) - return nil, fmt.Errorf("segment can not be saved because of binlog "+ - "file not match stat log number: collection %v, segment %v", collectionID, segmentID) - } - } - - fillLogIDByLogPath(binlogs, deltalogs, statslogs) - kvs, err := buildBinlogKvs(collectionID, partitionID, segmentID, binlogs, deltalogs, statslogs) - if err != nil { - return nil, err - } - - return kvs, nil -} - -func buildSegmentAndBinlogsKvs(segment *datapb.SegmentInfo) (map[string]string, error) { - noBinlogsSegment, binlogs, deltalogs, statslogs := CloneSegmentWithExcludeBinlogs(segment) - // `segment` is not mutated above. Also, `noBinlogsSegment` is a cloned version of `segment`. - segmentutil.ReCalcRowCount(segment, noBinlogsSegment) - - // compacted segment has only one statslog - ignore := (len(segment.GetCompactionFrom()) > 0) - // save binlogs separately - kvs, err := buildBinlogKvsWithLogID(noBinlogsSegment.CollectionID, noBinlogsSegment.PartitionID, noBinlogsSegment.ID, binlogs, deltalogs, statslogs, ignore) - if err != nil { - return nil, err - } - - // save segment info - k, v, err := buildSegmentKv(noBinlogsSegment) - if err != nil { - return nil, err - } - kvs[k] = v - - return kvs, nil -} - -func buildBinlogKeys(segment *datapb.SegmentInfo) []string { - var keys []string - // binlog - for _, binlog := range segment.Binlogs { - key := buildFieldBinlogPath(segment.CollectionID, segment.PartitionID, segment.ID, binlog.FieldID) - keys = append(keys, key) - } - - // deltalog - for _, deltalog := range segment.Deltalogs { - key := buildFieldDeltalogPath(segment.CollectionID, segment.PartitionID, segment.ID, deltalog.FieldID) - keys = append(keys, key) - } - - // statslog - for _, statslog := range segment.Statslogs { - key := buildFieldStatslogPath(segment.CollectionID, segment.PartitionID, segment.ID, statslog.FieldID) - keys = append(keys, key) - } - return keys -} - -func buildBinlogKvs(collectionID, partitionID, segmentID typeutil.UniqueID, binlogs, deltalogs, statslogs []*datapb.FieldBinlog) (map[string]string, error) { - kv := make(map[string]string) - - // binlog kv - for _, binlog := range binlogs { - binlogBytes, err := proto.Marshal(binlog) - if err != nil { - return nil, fmt.Errorf("marshal binlogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, binlog.FieldID, err) - } - key := buildFieldBinlogPath(collectionID, partitionID, segmentID, binlog.FieldID) - kv[key] = string(binlogBytes) - } - - // deltalog - for _, deltalog := range deltalogs { - binlogBytes, err := proto.Marshal(deltalog) - if err != nil { - return nil, fmt.Errorf("marshal deltalogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, deltalog.FieldID, err) - } - key := buildFieldDeltalogPath(collectionID, partitionID, segmentID, deltalog.FieldID) - kv[key] = string(binlogBytes) - } - - // statslog - for _, statslog := range statslogs { - binlogBytes, err := proto.Marshal(statslog) - if err != nil { - return nil, fmt.Errorf("marshal statslogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, statslog.FieldID, err) - } - key := buildFieldStatslogPath(collectionID, partitionID, segmentID, statslog.FieldID) - kv[key] = string(binlogBytes) - } - - return kv, nil -} - -func CloneSegmentWithExcludeBinlogs(segment *datapb.SegmentInfo) (*datapb.SegmentInfo, []*datapb.FieldBinlog, []*datapb.FieldBinlog, []*datapb.FieldBinlog) { - clonedSegment := proto.Clone(segment).(*datapb.SegmentInfo) - binlogs := clonedSegment.Binlogs - deltalogs := clonedSegment.Deltalogs - statlogs := clonedSegment.Statslogs - - clonedSegment.Binlogs = nil - clonedSegment.Deltalogs = nil - clonedSegment.Statslogs = nil - return clonedSegment, binlogs, deltalogs, statlogs -} - -func marshalSegmentInfo(segment *datapb.SegmentInfo) (string, error) { - segBytes, err := proto.Marshal(segment) - if err != nil { - return "", fmt.Errorf("failed to marshal segment: %d, err: %w", segment.ID, err) - } - - return string(segBytes), nil -} - -func buildSegmentKv(segment *datapb.SegmentInfo) (string, string, error) { - segBytes, err := marshalSegmentInfo(segment) - if err != nil { - return "", "", err - } - key := buildSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID()) - return key, segBytes, nil -} - -// buildSegmentPath common logic mapping segment info to corresponding key in kv store -func buildSegmentPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d", SegmentPrefix, collectionID, partitionID, segmentID) -} - -func buildFieldBinlogPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID, fieldID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentBinlogPathPrefix, collectionID, partitionID, segmentID, fieldID) -} - -// TODO: There's no need to include fieldID in the delta log path key. -func buildFieldDeltalogPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID, fieldID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentDeltalogPathPrefix, collectionID, partitionID, segmentID, fieldID) -} - -// TODO: There's no need to include fieldID in the stats log path key. -func buildFieldStatslogPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID, fieldID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentStatslogPathPrefix, collectionID, partitionID, segmentID, fieldID) -} - -// buildFlushedSegmentPath common logic mapping segment info to corresponding key of IndexCoord in kv store -// TODO @cai.zhang: remove this -func buildFlushedSegmentPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, collectionID, partitionID, segmentID) -} - -func buildFieldBinlogPathPrefix(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d", SegmentBinlogPathPrefix, collectionID, partitionID, segmentID) -} - -func buildFieldDeltalogPathPrefix(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d", SegmentDeltalogPathPrefix, collectionID, partitionID, segmentID) -} - -func buildFieldStatslogPathPrefix(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d/%d", SegmentStatslogPathPrefix, collectionID, partitionID, segmentID) -} - -// buildChannelRemovePath builds vchannel remove flag path -func buildChannelRemovePath(channel string) string { - return fmt.Sprintf("%s/%s", ChannelRemovePrefix, channel) -} - -func buildChannelCPKey(vChannel string) string { - return fmt.Sprintf("%s/%s", ChannelCheckpointPrefix, vChannel) -} - -func BuildIndexKey(collectionID, indexID int64) string { - return fmt.Sprintf("%s/%d/%d", util.FieldIndexPrefix, collectionID, indexID) -} - -func BuildSegmentIndexKey(collectionID, partitionID, segmentID, buildID int64) string { - return fmt.Sprintf("%s/%d/%d/%d/%d", util.SegmentIndexPrefix, collectionID, partitionID, segmentID, buildID) -} - -func buildCollectionPrefix(collectionID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d", SegmentPrefix, collectionID) -} - -func buildPartitionPrefix(collectionID, partitionID typeutil.UniqueID) string { - return fmt.Sprintf("%s/%d/%d", SegmentPrefix, collectionID, partitionID) -} diff --git a/internal/metastore/kv/datacoord/util.go b/internal/metastore/kv/datacoord/util.go new file mode 100644 index 0000000000..9cafd2e736 --- /dev/null +++ b/internal/metastore/kv/datacoord/util.go @@ -0,0 +1,412 @@ +// 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 datacoord + +import ( + "fmt" + "path" + "strconv" + "strings" + + "github.com/golang/protobuf/proto" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/segmentutil" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util" + "github.com/milvus-io/milvus/pkg/util/metautil" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +func CompressBinLog(fieldBinLogs []*datapb.FieldBinlog) ([]*datapb.FieldBinlog, error) { + compressedFieldBinLogs := make([]*datapb.FieldBinlog, 0) + for _, fieldBinLog := range fieldBinLogs { + compressedFieldBinLog := &datapb.FieldBinlog{} + compressedFieldBinLog.FieldID = fieldBinLog.FieldID + for _, binlog := range fieldBinLog.Binlogs { + logPath := binlog.LogPath + idx := strings.LastIndex(logPath, "/") + if idx == -1 { + return nil, fmt.Errorf("invailed binlog path: %s", logPath) + } + logPathStr := logPath[(idx + 1):] + logID, err := strconv.ParseInt(logPathStr, 10, 64) + if err != nil { + return nil, err + } + binlog := &datapb.Binlog{ + EntriesNum: binlog.EntriesNum, + // remove timestamp since it's not necessary + LogSize: binlog.LogSize, + LogID: logID, + } + compressedFieldBinLog.Binlogs = append(compressedFieldBinLog.Binlogs, binlog) + } + compressedFieldBinLogs = append(compressedFieldBinLogs, compressedFieldBinLog) + } + return compressedFieldBinLogs, nil +} + +func DecompressBinLog(path string, info *datapb.SegmentInfo) error { + for _, fieldBinLogs := range info.GetBinlogs() { + fillLogPathByLogID(path, storage.InsertBinlog, info.CollectionID, info.PartitionID, info.ID, fieldBinLogs) + } + + for _, deltaLogs := range info.GetDeltalogs() { + fillLogPathByLogID(path, storage.DeleteBinlog, info.CollectionID, info.PartitionID, info.ID, deltaLogs) + } + + for _, statsLogs := range info.GetStatslogs() { + fillLogPathByLogID(path, storage.StatsBinlog, info.CollectionID, info.PartitionID, info.ID, statsLogs) + } + return nil +} + +func ValidateSegment(segment *datapb.SegmentInfo) error { + log := log.With( + zap.Int64("collection", segment.GetCollectionID()), + zap.Int64("partition", segment.GetPartitionID()), + zap.Int64("segment", segment.GetID())) + err := checkBinlogs(storage.InsertBinlog, segment.GetID(), segment.GetBinlogs()) + if err != nil { + return err + } + checkBinlogs(storage.DeleteBinlog, segment.GetID(), segment.GetDeltalogs()) + if err != nil { + return err + } + checkBinlogs(storage.StatsBinlog, segment.GetID(), segment.GetStatslogs()) + if err != nil { + return err + } + // check stats log and bin log size match + + // check L0 Segment + if segment.GetLevel() == datapb.SegmentLevel_L0 { + // L0 segment should only have delta logs + if len(segment.GetBinlogs()) > 0 || len(segment.GetStatslogs()) > 0 { + log.Warn("find invalid segment while L0 segment get more than delta logs", + zap.Any("binlogs", segment.GetBinlogs()), + zap.Any("stats", segment.GetBinlogs()), + ) + return fmt.Errorf("segment can not be saved because of L0 segment get more than delta logs: collection %v, segment %v", + segment.GetCollectionID(), segment.GetID()) + } + return nil + } + // check L1 and Legacy Segment + + if len(segment.GetBinlogs()) == 0 && len(segment.GetStatslogs()) == 0 { + return nil + } + + if len(segment.GetBinlogs()) == 0 || len(segment.GetStatslogs()) == 0 { + log.Warn("find segment binlog or statslog was empty", + zap.Any("binlogs", segment.GetBinlogs()), + zap.Any("stats", segment.GetBinlogs()), + ) + return fmt.Errorf("segment can not be saved because of binlog file or stat log file lack: collection %v, segment %v", + segment.GetCollectionID(), segment.GetID()) + } + + // if segment not merge status log(growing or new flushed by old version) + // segment num of binlog should same with statslogs. + binlogNum := len(segment.GetBinlogs()[0].GetBinlogs()) + statslogNum := len(segment.GetStatslogs()[0].GetBinlogs()) + + if len(segment.GetCompactionFrom()) == 0 && statslogNum != binlogNum && !hasSpecialStatslog(segment) { + log.Warn("find invalid segment while bin log size didn't match stat log size", + zap.Any("binlogs", segment.GetBinlogs()), + zap.Any("stats", segment.GetStatslogs()), + ) + return fmt.Errorf("segment can not be saved because of binlog file not match stat log number: collection %v, segment %v", + segment.GetCollectionID(), segment.GetID()) + } + + return nil +} + +// build a binlog path on the storage by metadata +func buildLogPath(chunkManagerRootPath string, binlogType storage.BinlogType, collectionID, partitionID, segmentID, filedID, logID typeutil.UniqueID) string { + switch binlogType { + case storage.InsertBinlog: + return metautil.BuildInsertLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, filedID, logID) + case storage.DeleteBinlog: + return metautil.BuildDeltaLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, logID) + case storage.StatsBinlog: + return metautil.BuildStatsLogPath(chunkManagerRootPath, collectionID, partitionID, segmentID, filedID, logID) + } + // should not happen + log.Panic("invalid binlog type", zap.Any("type", binlogType)) + return "" +} + +func checkBinlogs(binlogType storage.BinlogType, segmentID typeutil.UniqueID, logs []*datapb.FieldBinlog) error { + check := func(getSegmentID func(logPath string) typeutil.UniqueID) error { + for _, fieldBinlog := range logs { + for _, binlog := range fieldBinlog.Binlogs { + if segmentID != getSegmentID(binlog.LogPath) { + return fmt.Errorf("the segment path doesn't match the segmentID, segmentID %d, path %s", segmentID, binlog.LogPath) + } + } + } + return nil + } + switch binlogType { + case storage.InsertBinlog: + return check(metautil.GetSegmentIDFromInsertLogPath) + case storage.DeleteBinlog: + return check(metautil.GetSegmentIDFromDeltaLogPath) + case storage.StatsBinlog: + return check(metautil.GetSegmentIDFromStatsLogPath) + default: + return fmt.Errorf("the segment path doesn't match the segmentID, segmentID %d, type %d", segmentID, binlogType) + } +} + +func hasSpecialStatslog(segment *datapb.SegmentInfo) bool { + for _, statslog := range segment.GetStatslogs()[0].GetBinlogs() { + _, logidx := path.Split(statslog.LogPath) + if logidx == storage.CompoundStatsType.LogIdx() { + return true + } + } + return false +} + +func buildBinlogKvsWithLogID(collectionID, partitionID, segmentID typeutil.UniqueID, + binlogs, deltalogs, statslogs []*datapb.FieldBinlog, +) (map[string]string, error) { + fillLogIDByLogPath(binlogs, deltalogs, statslogs) + kvs, err := buildBinlogKvs(collectionID, partitionID, segmentID, binlogs, deltalogs, statslogs) + if err != nil { + return nil, err + } + + return kvs, nil +} + +func buildSegmentAndBinlogsKvs(segment *datapb.SegmentInfo) (map[string]string, error) { + noBinlogsSegment, binlogs, deltalogs, statslogs := CloneSegmentWithExcludeBinlogs(segment) + // `segment` is not mutated above. Also, `noBinlogsSegment` is a cloned version of `segment`. + segmentutil.ReCalcRowCount(segment, noBinlogsSegment) + + // save binlogs separately + kvs, err := buildBinlogKvsWithLogID(noBinlogsSegment.CollectionID, noBinlogsSegment.PartitionID, noBinlogsSegment.ID, binlogs, deltalogs, statslogs) + if err != nil { + return nil, err + } + + // save segment info + k, v, err := buildSegmentKv(noBinlogsSegment) + if err != nil { + return nil, err + } + kvs[k] = v + + return kvs, nil +} + +func buildBinlogKeys(segment *datapb.SegmentInfo) []string { + var keys []string + // binlog + for _, binlog := range segment.Binlogs { + key := buildFieldBinlogPath(segment.CollectionID, segment.PartitionID, segment.ID, binlog.FieldID) + keys = append(keys, key) + } + + // deltalog + for _, deltalog := range segment.Deltalogs { + key := buildFieldDeltalogPath(segment.CollectionID, segment.PartitionID, segment.ID, deltalog.FieldID) + keys = append(keys, key) + } + + // statslog + for _, statslog := range segment.Statslogs { + key := buildFieldStatslogPath(segment.CollectionID, segment.PartitionID, segment.ID, statslog.FieldID) + keys = append(keys, key) + } + return keys +} + +func resetBinlogFields(segment *datapb.SegmentInfo) { + segment.Binlogs = nil + segment.Deltalogs = nil + segment.Statslogs = nil +} + +func cloneLogs(binlogs []*datapb.FieldBinlog) []*datapb.FieldBinlog { + var res []*datapb.FieldBinlog + for _, log := range binlogs { + res = append(res, proto.Clone(log).(*datapb.FieldBinlog)) + } + return res +} + +func fillLogIDByLogPath(multiFieldBinlogs ...[]*datapb.FieldBinlog) error { + for _, fieldBinlogs := range multiFieldBinlogs { + for _, fieldBinlog := range fieldBinlogs { + for _, binlog := range fieldBinlog.Binlogs { + logPath := binlog.LogPath + idx := strings.LastIndex(logPath, "/") + if idx == -1 { + return fmt.Errorf("invailed binlog path: %s", logPath) + } + logPathStr := logPath[(idx + 1):] + logID, err := strconv.ParseInt(logPathStr, 10, 64) + if err != nil { + return err + } + + // set log path to empty and only store log id + binlog.LogPath = "" + binlog.LogID = logID + } + } + } + return nil +} + +func buildBinlogKvs(collectionID, partitionID, segmentID typeutil.UniqueID, binlogs, deltalogs, statslogs []*datapb.FieldBinlog) (map[string]string, error) { + kv := make(map[string]string) + + // binlog kv + for _, binlog := range binlogs { + binlogBytes, err := proto.Marshal(binlog) + if err != nil { + return nil, fmt.Errorf("marshal binlogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, binlog.FieldID, err) + } + key := buildFieldBinlogPath(collectionID, partitionID, segmentID, binlog.FieldID) + kv[key] = string(binlogBytes) + } + + // deltalog + for _, deltalog := range deltalogs { + binlogBytes, err := proto.Marshal(deltalog) + if err != nil { + return nil, fmt.Errorf("marshal deltalogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, deltalog.FieldID, err) + } + key := buildFieldDeltalogPath(collectionID, partitionID, segmentID, deltalog.FieldID) + kv[key] = string(binlogBytes) + } + + // statslog + for _, statslog := range statslogs { + binlogBytes, err := proto.Marshal(statslog) + if err != nil { + return nil, fmt.Errorf("marshal statslogs failed, collectionID:%d, segmentID:%d, fieldID:%d, error:%w", collectionID, segmentID, statslog.FieldID, err) + } + key := buildFieldStatslogPath(collectionID, partitionID, segmentID, statslog.FieldID) + kv[key] = string(binlogBytes) + } + + return kv, nil +} + +func CloneSegmentWithExcludeBinlogs(segment *datapb.SegmentInfo) (*datapb.SegmentInfo, []*datapb.FieldBinlog, []*datapb.FieldBinlog, []*datapb.FieldBinlog) { + clonedSegment := proto.Clone(segment).(*datapb.SegmentInfo) + binlogs := clonedSegment.Binlogs + deltalogs := clonedSegment.Deltalogs + statlogs := clonedSegment.Statslogs + + clonedSegment.Binlogs = nil + clonedSegment.Deltalogs = nil + clonedSegment.Statslogs = nil + return clonedSegment, binlogs, deltalogs, statlogs +} + +func marshalSegmentInfo(segment *datapb.SegmentInfo) (string, error) { + segBytes, err := proto.Marshal(segment) + if err != nil { + return "", fmt.Errorf("failed to marshal segment: %d, err: %w", segment.ID, err) + } + + return string(segBytes), nil +} + +func buildSegmentKv(segment *datapb.SegmentInfo) (string, string, error) { + segBytes, err := marshalSegmentInfo(segment) + if err != nil { + return "", "", err + } + key := buildSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID()) + return key, segBytes, nil +} + +// buildSegmentPath common logic mapping segment info to corresponding key in kv store +func buildSegmentPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d", SegmentPrefix, collectionID, partitionID, segmentID) +} + +func buildFieldBinlogPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID, fieldID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentBinlogPathPrefix, collectionID, partitionID, segmentID, fieldID) +} + +// TODO: There's no need to include fieldID in the delta log path key. +func buildFieldDeltalogPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID, fieldID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentDeltalogPathPrefix, collectionID, partitionID, segmentID, fieldID) +} + +// TODO: There's no need to include fieldID in the stats log path key. +func buildFieldStatslogPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID, fieldID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentStatslogPathPrefix, collectionID, partitionID, segmentID, fieldID) +} + +// buildFlushedSegmentPath common logic mapping segment info to corresponding key of IndexCoord in kv store +// TODO @cai.zhang: remove this +func buildFlushedSegmentPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, collectionID, partitionID, segmentID) +} + +func buildFieldBinlogPathPrefix(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d", SegmentBinlogPathPrefix, collectionID, partitionID, segmentID) +} + +func buildFieldDeltalogPathPrefix(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d", SegmentDeltalogPathPrefix, collectionID, partitionID, segmentID) +} + +func buildFieldStatslogPathPrefix(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d", SegmentStatslogPathPrefix, collectionID, partitionID, segmentID) +} + +// buildChannelRemovePath builds vchannel remove flag path +func buildChannelRemovePath(channel string) string { + return fmt.Sprintf("%s/%s", ChannelRemovePrefix, channel) +} + +func buildChannelCPKey(vChannel string) string { + return fmt.Sprintf("%s/%s", ChannelCheckpointPrefix, vChannel) +} + +func BuildIndexKey(collectionID, indexID int64) string { + return fmt.Sprintf("%s/%d/%d", util.FieldIndexPrefix, collectionID, indexID) +} + +func BuildSegmentIndexKey(collectionID, partitionID, segmentID, buildID int64) string { + return fmt.Sprintf("%s/%d/%d/%d/%d", util.SegmentIndexPrefix, collectionID, partitionID, segmentID, buildID) +} + +func buildCollectionPrefix(collectionID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d", SegmentPrefix, collectionID) +} + +func buildPartitionPrefix(collectionID, partitionID typeutil.UniqueID) string { + return fmt.Sprintf("%s/%d/%d", SegmentPrefix, collectionID, partitionID) +} diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 7f3159e9f6..b10e68251e 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -152,6 +152,7 @@ message SegmentIDRequest { int64 partitionID = 4; bool isImport = 5; // Indicate whether this request comes from a bulk insert task. int64 importTaskID = 6; // Needed for segment lock. + SegmentLevel level = 7; } message AssignSegmentIDRequest { @@ -329,6 +330,8 @@ message SaveBinlogPathsRequest { bool dropped = 10; bool importing = 11; string channel = 12; // report channel name for verification + SegmentLevel seg_level =13; + int64 partitionID =14; // report partitionID for create L0 segment } message CheckPoint { diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index 141f3496c7..314fb3d778 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -419,15 +419,16 @@ func (m *FlushChannelsRequest) GetChannels() []string { } type SegmentIDRequest struct { - Count uint32 `protobuf:"varint,1,opt,name=count,proto3" json:"count,omitempty"` - ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` - CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` - IsImport bool `protobuf:"varint,5,opt,name=isImport,proto3" json:"isImport,omitempty"` - ImportTaskID int64 `protobuf:"varint,6,opt,name=importTaskID,proto3" json:"importTaskID,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Count uint32 `protobuf:"varint,1,opt,name=count,proto3" json:"count,omitempty"` + ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,4,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + IsImport bool `protobuf:"varint,5,opt,name=isImport,proto3" json:"isImport,omitempty"` + ImportTaskID int64 `protobuf:"varint,6,opt,name=importTaskID,proto3" json:"importTaskID,omitempty"` + Level SegmentLevel `protobuf:"varint,7,opt,name=level,proto3,enum=milvus.proto.data.SegmentLevel" json:"level,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SegmentIDRequest) Reset() { *m = SegmentIDRequest{} } @@ -497,6 +498,13 @@ func (m *SegmentIDRequest) GetImportTaskID() int64 { return 0 } +func (m *SegmentIDRequest) GetLevel() SegmentLevel { + if m != nil { + return m.Level + } + return SegmentLevel_Legacy +} + type AssignSegmentIDRequest struct { NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"` PeerRole string `protobuf:"bytes,2,opt,name=peer_role,json=peerRole,proto3" json:"peer_role,omitempty"` @@ -1853,6 +1861,8 @@ type SaveBinlogPathsRequest struct { Dropped bool `protobuf:"varint,10,opt,name=dropped,proto3" json:"dropped,omitempty"` Importing bool `protobuf:"varint,11,opt,name=importing,proto3" json:"importing,omitempty"` Channel string `protobuf:"bytes,12,opt,name=channel,proto3" json:"channel,omitempty"` + SegLevel SegmentLevel `protobuf:"varint,13,opt,name=seg_level,json=segLevel,proto3,enum=milvus.proto.data.SegmentLevel" json:"seg_level,omitempty"` + PartitionID int64 `protobuf:"varint,14,opt,name=partitionID,proto3" json:"partitionID,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1967,6 +1977,20 @@ func (m *SaveBinlogPathsRequest) GetChannel() string { return "" } +func (m *SaveBinlogPathsRequest) GetSegLevel() SegmentLevel { + if m != nil { + return m.SegLevel + } + return SegmentLevel_Legacy +} + +func (m *SaveBinlogPathsRequest) GetPartitionID() int64 { + if m != nil { + return m.PartitionID + } + return 0 +} + type CheckPoint struct { SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` Position *msgpb.MsgPosition `protobuf:"bytes,2,opt,name=position,proto3" json:"position,omitempty"` @@ -5380,327 +5404,328 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 5108 bytes of a gzipped FileDescriptorProto + // 5135 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, 0xdb, 0x92, 0x6c, 0x5a, 0xb6, 0x25, 0x99, 0xfa, 0x19, 0xeb, 0xc7, 0xda, 0xbb, 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, 0x02, 0xe4, 0x81, 0x24, 0xc8, 0xe3, 0x90, - 0x5b, 0x90, 0x43, 0xb0, 0x79, 0xec, 0x69, 0x13, 0x2c, 0x10, 0x04, 0x58, 0x20, 0xb9, 0x6c, 0x90, - 0x43, 0x10, 0x04, 0x08, 0x92, 0x43, 0xfe, 0x80, 0x9c, 0x83, 0xdc, 0x83, 0x7a, 0x74, 0xf5, 0xab, - 0x7a, 0xa6, 0xc9, 0x91, 0x6c, 0x20, 0x39, 0x91, 0x55, 0xfd, 0x55, 0xd5, 0x57, 0x5f, 0x7d, 0xef, - 0xaf, 0x6a, 0xa0, 0x65, 0xe8, 0xbe, 0xde, 0xeb, 0x3b, 0x8e, 0x6b, 0xdc, 0x1e, 0xb9, 0x8e, 0xef, - 0xa0, 0xc5, 0xa1, 0x69, 0x1d, 0x8d, 0x3d, 0xd6, 0xba, 0x4d, 0x3e, 0x77, 0x1a, 0x7d, 0x67, 0x38, - 0x74, 0x6c, 0xd6, 0xd5, 0x69, 0x9a, 0xb6, 0x8f, 0x5d, 0x5b, 0xb7, 0x78, 0xbb, 0x11, 0x1d, 0xd0, - 0x69, 0x78, 0xfd, 0x03, 0x3c, 0xd4, 0x79, 0xab, 0x36, 0xf4, 0x06, 0xfc, 0xdf, 0x45, 0xd3, 0x36, - 0xf0, 0xd3, 0xe8, 0x52, 0xea, 0x1c, 0x94, 0x3f, 0x18, 0x8e, 0xfc, 0x13, 0xf5, 0xaf, 0x14, 0x68, - 0x3c, 0xb0, 0xc6, 0xde, 0x81, 0x86, 0x3f, 0x1b, 0x63, 0xcf, 0x47, 0x77, 0xa1, 0xb4, 0xa7, 0x7b, - 0xb8, 0xad, 0x5c, 0x53, 0x56, 0xea, 0xab, 0x2f, 0xde, 0x8e, 0xe1, 0xc4, 0xb1, 0xd9, 0xf2, 0x06, - 0x6b, 0xba, 0x87, 0x35, 0x0a, 0x89, 0x10, 0x94, 0x8c, 0xbd, 0xee, 0x46, 0xbb, 0x70, 0x4d, 0x59, - 0x29, 0x6a, 0xf4, 0x7f, 0x74, 0x05, 0xc0, 0xc3, 0x83, 0x21, 0xb6, 0xfd, 0xee, 0x86, 0xd7, 0x2e, - 0x5e, 0x2b, 0xae, 0x14, 0xb5, 0x48, 0x0f, 0x52, 0xa1, 0xd1, 0x77, 0x2c, 0x0b, 0xf7, 0x7d, 0xd3, - 0xb1, 0xbb, 0x1b, 0xed, 0x12, 0x1d, 0x1b, 0xeb, 0x43, 0x1d, 0xa8, 0x9a, 0x5e, 0x77, 0x38, 0x72, - 0x5c, 0xbf, 0x5d, 0xbe, 0xa6, 0xac, 0x54, 0x35, 0xd1, 0x56, 0x7f, 0x50, 0x80, 0x79, 0x8e, 0xb6, - 0x37, 0x72, 0x6c, 0x0f, 0xa3, 0x7b, 0x50, 0xf1, 0x7c, 0xdd, 0x1f, 0x7b, 0x1c, 0xf3, 0x4b, 0x52, - 0xcc, 0x77, 0x28, 0x88, 0xc6, 0x41, 0xa5, 0xa8, 0x27, 0x51, 0x2b, 0x4a, 0x50, 0x8b, 0x6f, 0xaf, - 0x94, 0xda, 0xde, 0x0a, 0x2c, 0xec, 0x13, 0xec, 0x76, 0x42, 0xa0, 0x32, 0x05, 0x4a, 0x76, 0x93, - 0x99, 0x7c, 0x73, 0x88, 0x3f, 0xda, 0xdf, 0xc1, 0xba, 0xd5, 0xae, 0xd0, 0xb5, 0x22, 0x3d, 0xe8, - 0x22, 0x54, 0xe9, 0x90, 0x9e, 0xef, 0xb5, 0xe7, 0xae, 0x29, 0x2b, 0x25, 0x6d, 0x8e, 0xb6, 0x77, - 0x3d, 0xf5, 0xfb, 0xb0, 0x44, 0x49, 0xb0, 0x7e, 0xa0, 0xdb, 0x36, 0xb6, 0xbc, 0xb3, 0x9f, 0x60, - 0x74, 0x91, 0x42, 0x6c, 0x11, 0x72, 0x08, 0x7d, 0x3e, 0x3f, 0x3d, 0xc6, 0x9a, 0x26, 0xda, 0xea, - 0x3f, 0x2b, 0xd0, 0x12, 0x5b, 0x09, 0x56, 0x5f, 0x82, 0x72, 0xdf, 0x19, 0xdb, 0x3e, 0x5d, 0x7e, - 0x5e, 0x63, 0x0d, 0xf4, 0x0d, 0x68, 0xf0, 0x61, 0x3d, 0x5b, 0x1f, 0x62, 0xba, 0x4a, 0x4d, 0xab, - 0xf3, 0xbe, 0x47, 0xfa, 0x10, 0xe7, 0xa2, 0xfb, 0x35, 0xa8, 0x8f, 0x74, 0xd7, 0x37, 0x63, 0x5c, - 0x13, 0xed, 0x9a, 0xc4, 0x34, 0x64, 0x05, 0x93, 0xfe, 0xb7, 0xab, 0x7b, 0x87, 0xdd, 0x0d, 0x4e, - 0xed, 0x58, 0x9f, 0xfa, 0xc7, 0x0a, 0x2c, 0xdf, 0xf7, 0x3c, 0x73, 0x60, 0xa7, 0x76, 0xb6, 0x0c, - 0x15, 0xdb, 0x31, 0x70, 0x77, 0x83, 0x6e, 0xad, 0xa8, 0xf1, 0x16, 0xba, 0x04, 0xb5, 0x11, 0xc6, - 0x6e, 0xcf, 0x75, 0xac, 0x60, 0x63, 0x55, 0xd2, 0xa1, 0x39, 0x16, 0x46, 0xdf, 0x85, 0x45, 0x2f, - 0x31, 0x11, 0x23, 0x64, 0x7d, 0xf5, 0xfa, 0xed, 0x94, 0xbc, 0xdf, 0x4e, 0x2e, 0xaa, 0xa5, 0x47, - 0xab, 0x5f, 0x14, 0xe0, 0xbc, 0x80, 0x63, 0xb8, 0x92, 0xff, 0x09, 0xe5, 0x3d, 0x3c, 0x10, 0xe8, - 0xb1, 0x46, 0x1e, 0xca, 0x8b, 0x23, 0x2b, 0x46, 0x8f, 0x2c, 0x8f, 0x88, 0x26, 0xce, 0xa3, 0x9c, - 0x3e, 0x8f, 0xab, 0x50, 0xc7, 0x4f, 0x47, 0xa6, 0x8b, 0x7b, 0x84, 0xa9, 0x29, 0xc9, 0x4b, 0x1a, - 0xb0, 0xae, 0x5d, 0x73, 0x18, 0x95, 0xdb, 0xb9, 0xdc, 0x72, 0xab, 0xfe, 0x89, 0x02, 0x17, 0x52, - 0xa7, 0xc4, 0x15, 0x81, 0x06, 0x2d, 0xba, 0xf3, 0x90, 0x32, 0x44, 0x25, 0x10, 0x82, 0xbf, 0x34, - 0x89, 0xe0, 0x21, 0xb8, 0x96, 0x1a, 0x1f, 0x41, 0xb2, 0x90, 0x1f, 0xc9, 0x43, 0xb8, 0xf0, 0x10, - 0xfb, 0x7c, 0x01, 0xf2, 0x0d, 0xcf, 0x20, 0xa2, 0x71, 0x8d, 0x53, 0x48, 0x6a, 0x1c, 0xf5, 0x4f, - 0x0b, 0x42, 0x16, 0xe9, 0x52, 0x5d, 0x7b, 0xdf, 0x41, 0x2f, 0x42, 0x4d, 0x80, 0x70, 0xae, 0x08, - 0x3b, 0xd0, 0x37, 0xa1, 0x4c, 0x30, 0x65, 0x2c, 0xd1, 0x5c, 0xfd, 0x86, 0x7c, 0x4f, 0x91, 0x39, - 0x35, 0x06, 0x8f, 0x36, 0xa0, 0xe9, 0xf9, 0xba, 0xeb, 0xf7, 0x46, 0x8e, 0x47, 0xcf, 0x99, 0x32, - 0x4e, 0x7d, 0xf5, 0x72, 0x7c, 0x06, 0x62, 0x80, 0xb6, 0xbc, 0xc1, 0x36, 0x07, 0xd2, 0xe6, 0xe9, - 0xa0, 0xa0, 0x89, 0xbe, 0x03, 0x0d, 0x6c, 0x1b, 0xe1, 0x1c, 0xa5, 0x3c, 0x73, 0xd4, 0xb1, 0x6d, - 0x88, 0x19, 0xc2, 0x53, 0x29, 0xe7, 0x3f, 0x95, 0xdf, 0x52, 0xa0, 0x9d, 0x3e, 0x96, 0x59, 0x8c, - 0xc8, 0xbb, 0x6c, 0x10, 0x66, 0xc7, 0x32, 0x51, 0xae, 0xc5, 0xd1, 0x68, 0x7c, 0x88, 0xfa, 0x87, - 0x0a, 0xbc, 0x10, 0xa2, 0x43, 0x3f, 0x3d, 0x2f, 0x1e, 0x41, 0xb7, 0xa0, 0x65, 0xda, 0x7d, 0x6b, - 0x6c, 0xe0, 0xc7, 0xf6, 0x87, 0x58, 0xb7, 0xfc, 0x83, 0x13, 0x7a, 0x72, 0x55, 0x2d, 0xd5, 0xaf, - 0xfe, 0x7b, 0x01, 0x96, 0x93, 0x78, 0xcd, 0x42, 0xa4, 0xff, 0x07, 0x65, 0xd3, 0xde, 0x77, 0x02, - 0x1a, 0x5d, 0x99, 0x20, 0x8a, 0x64, 0x2d, 0x06, 0x8c, 0x1c, 0x40, 0x81, 0xf2, 0xea, 0x1f, 0xe0, - 0xfe, 0xe1, 0xc8, 0x31, 0xa9, 0x9a, 0x22, 0x53, 0x7c, 0x47, 0x32, 0x85, 0x1c, 0xe3, 0xdb, 0xdc, - 0x42, 0xae, 0x8b, 0x29, 0x3e, 0xb0, 0x7d, 0xf7, 0x44, 0x5b, 0xec, 0x27, 0xfb, 0x3b, 0x7d, 0x58, - 0x96, 0x03, 0xa3, 0x16, 0x14, 0x0f, 0xf1, 0x09, 0xdd, 0x72, 0x4d, 0x23, 0xff, 0xa2, 0x7b, 0x50, - 0x3e, 0xd2, 0xad, 0x31, 0xe6, 0x3a, 0x61, 0x0a, 0xe7, 0x32, 0xd8, 0x77, 0x0a, 0x6f, 0x29, 0xea, - 0x10, 0x2e, 0x3d, 0xc4, 0x7e, 0xd7, 0xf6, 0xb0, 0xeb, 0xaf, 0x99, 0xb6, 0xe5, 0x0c, 0xb6, 0x75, - 0xff, 0x60, 0x06, 0xe5, 0x10, 0x93, 0xf3, 0x42, 0x42, 0xce, 0xd5, 0x1f, 0x2a, 0xf0, 0xa2, 0x7c, - 0x3d, 0x7e, 0xa0, 0x1d, 0xa8, 0xee, 0x9b, 0xd8, 0x32, 0x08, 0xd7, 0x28, 0x94, 0x6b, 0x44, 0x9b, - 0x28, 0x89, 0x11, 0x01, 0xe6, 0xe7, 0x96, 0x50, 0x12, 0xc2, 0x1f, 0xdd, 0xf1, 0x5d, 0xd3, 0x1e, - 0x6c, 0x9a, 0x9e, 0xaf, 0x31, 0xf8, 0x08, 0x97, 0x14, 0xf3, 0x0b, 0xe7, 0x6f, 0x28, 0x70, 0xe5, - 0x21, 0xf6, 0xd7, 0x85, 0x8d, 0x21, 0xdf, 0x4d, 0xcf, 0x37, 0xfb, 0xde, 0xb3, 0xf5, 0x4f, 0x73, - 0x38, 0x1b, 0xea, 0xef, 0x28, 0x70, 0x35, 0x13, 0x19, 0x4e, 0x3a, 0xae, 0x43, 0x03, 0x0b, 0x23, - 0xd7, 0xa1, 0x3f, 0x87, 0x4f, 0x9e, 0x90, 0xc3, 0xdf, 0xd6, 0x4d, 0x97, 0xe9, 0xd0, 0x33, 0x5a, - 0x94, 0x1f, 0x29, 0x70, 0xf9, 0x21, 0xf6, 0xb7, 0x03, 0xfb, 0xfa, 0x35, 0x52, 0x87, 0xc0, 0x44, - 0xec, 0x7c, 0xe0, 0x04, 0xc7, 0xfa, 0xd4, 0xdf, 0x66, 0xc7, 0x29, 0xc5, 0xf7, 0x6b, 0x21, 0xe0, - 0x15, 0x2a, 0x09, 0x11, 0x15, 0xc1, 0x85, 0x9d, 0x93, 0x4f, 0xfd, 0xb2, 0x0c, 0x8d, 0x27, 0x5c, - 0x2b, 0x50, 0x0b, 0x9a, 0xa4, 0x84, 0x22, 0x77, 0x82, 0x22, 0xde, 0x94, 0xcc, 0xc1, 0x5a, 0x83, - 0x79, 0x0f, 0xe3, 0xc3, 0x53, 0xda, 0xcb, 0x06, 0x19, 0x23, 0x8c, 0xdd, 0x26, 0x2c, 0x8e, 0x6d, - 0xea, 0x95, 0x63, 0x83, 0x6f, 0x80, 0x11, 0x7d, 0xba, 0x32, 0x4d, 0x0f, 0x44, 0x1f, 0xf2, 0x00, - 0x25, 0x32, 0x57, 0x39, 0xd7, 0x5c, 0xc9, 0x61, 0xa8, 0x0b, 0x2d, 0xc3, 0x75, 0x46, 0x23, 0x6c, - 0xf4, 0xbc, 0x60, 0xaa, 0x4a, 0xbe, 0xa9, 0xf8, 0x38, 0x31, 0xd5, 0x5d, 0x38, 0x9f, 0xc4, 0xb4, - 0x6b, 0x10, 0xbf, 0x90, 0x70, 0x96, 0xec, 0x13, 0x7a, 0x0d, 0x16, 0xd3, 0xf0, 0x55, 0x0a, 0x9f, - 0xfe, 0x80, 0x5e, 0x07, 0x94, 0x40, 0x95, 0x80, 0xd7, 0x18, 0x78, 0x1c, 0x19, 0x0e, 0x4e, 0x03, - 0xe7, 0x38, 0x38, 0x30, 0x70, 0xfe, 0x25, 0x02, 0xde, 0x25, 0xd6, 0x35, 0x06, 0xee, 0xb5, 0xeb, - 0xf9, 0x08, 0x11, 0x9f, 0xcc, 0x53, 0x7f, 0x5d, 0x81, 0xe5, 0x8f, 0x75, 0xbf, 0x7f, 0xb0, 0x31, - 0x9c, 0x3d, 0xb8, 0x7b, 0x0f, 0x6a, 0x47, 0x22, 0x84, 0x63, 0x5a, 0xfc, 0xaa, 0x04, 0xa1, 0x28, - 0xdb, 0x6b, 0xe1, 0x08, 0xf5, 0xef, 0x14, 0x1e, 0x66, 0x06, 0xd8, 0x7d, 0xf5, 0xaa, 0x66, 0x5a, - 0xb4, 0x9d, 0x10, 0xc0, 0x72, 0x4a, 0x00, 0xd5, 0xa7, 0x00, 0x1c, 0xfd, 0x2d, 0x6f, 0x70, 0x06, - 0xcc, 0xdf, 0x82, 0x39, 0xbe, 0x1e, 0xd7, 0x36, 0xd3, 0x8e, 0x34, 0x00, 0x57, 0xff, 0xbb, 0x02, - 0xf5, 0xc8, 0x07, 0xd4, 0x84, 0x82, 0x50, 0x23, 0x05, 0xc9, 0xfe, 0x0b, 0xd3, 0xa3, 0xac, 0x62, - 0x3a, 0xca, 0xba, 0x09, 0x4d, 0x93, 0x9a, 0xf7, 0x1e, 0xdf, 0x35, 0xf5, 0xa6, 0x6b, 0xda, 0x3c, - 0xeb, 0xe5, 0x4c, 0x84, 0xae, 0x40, 0xdd, 0x1e, 0x0f, 0x7b, 0xce, 0x7e, 0xcf, 0x75, 0x8e, 0x3d, - 0x1e, 0xae, 0xd5, 0xec, 0xf1, 0xf0, 0xa3, 0x7d, 0xcd, 0x39, 0xf6, 0xc2, 0x88, 0xa0, 0x72, 0xca, - 0x88, 0xe0, 0x0a, 0xd4, 0x87, 0xfa, 0x53, 0x32, 0x6b, 0xcf, 0x1e, 0x0f, 0x69, 0x24, 0x57, 0xd4, - 0x6a, 0x43, 0xfd, 0xa9, 0xe6, 0x1c, 0x3f, 0x1a, 0x0f, 0xd1, 0x0a, 0xb4, 0x2c, 0xdd, 0xf3, 0x7b, - 0xd1, 0x50, 0xb0, 0x4a, 0x43, 0xc1, 0x26, 0xe9, 0xff, 0x20, 0x0c, 0x07, 0xd3, 0xb1, 0x45, 0xed, - 0x6c, 0xb1, 0x85, 0x31, 0xb4, 0xc2, 0x39, 0x20, 0x57, 0x6c, 0x61, 0x0c, 0x2d, 0x31, 0xc3, 0x5b, - 0x30, 0xb7, 0x47, 0x5d, 0xa5, 0x49, 0x42, 0xfc, 0x80, 0x78, 0x49, 0xcc, 0xa3, 0xd2, 0x02, 0x70, - 0xf4, 0x2d, 0xa8, 0x51, 0x0b, 0x45, 0xc7, 0x36, 0x72, 0x8d, 0x0d, 0x07, 0x90, 0xd1, 0x06, 0xb6, - 0x7c, 0x9d, 0x8e, 0x9e, 0xcf, 0x37, 0x5a, 0x0c, 0x20, 0x1a, 0xb4, 0xef, 0x62, 0xdd, 0xc7, 0xc6, - 0xda, 0xc9, 0xba, 0x33, 0x1c, 0xe9, 0x94, 0x85, 0xda, 0x4d, 0xea, 0xe4, 0xcb, 0x3e, 0xa1, 0x97, - 0xa0, 0xd9, 0x17, 0xad, 0x07, 0xae, 0x33, 0x6c, 0x2f, 0x50, 0xf9, 0x4a, 0xf4, 0xa2, 0xcb, 0x00, - 0x81, 0xee, 0xd4, 0xfd, 0x76, 0x8b, 0x9e, 0x5d, 0x8d, 0xf7, 0xdc, 0xa7, 0xf9, 0x1d, 0xd3, 0xeb, - 0xb1, 0x4c, 0x8a, 0x69, 0x0f, 0xda, 0x8b, 0x74, 0xc5, 0x7a, 0x90, 0x7a, 0x31, 0xed, 0x01, 0xba, - 0x00, 0x73, 0xa6, 0xd7, 0xdb, 0xd7, 0x0f, 0x71, 0x1b, 0xd1, 0xaf, 0x15, 0xd3, 0x7b, 0xa0, 0x1f, - 0x52, 0xef, 0x95, 0x2f, 0x86, 0x8d, 0xf6, 0x79, 0xfa, 0x29, 0xec, 0x40, 0x6f, 0x42, 0xd9, 0xc2, - 0x47, 0xd8, 0x6a, 0x2f, 0x51, 0x9e, 0xbc, 0x9a, 0x2d, 0x78, 0x9b, 0x04, 0x4c, 0x63, 0xd0, 0xea, - 0xe7, 0xb0, 0x14, 0x32, 0x6a, 0x84, 0x33, 0xd2, 0xfc, 0xa5, 0x9c, 0x81, 0xbf, 0x26, 0x3b, 0xdc, - 0xff, 0x59, 0x82, 0xe5, 0x1d, 0xfd, 0x08, 0x3f, 0x7f, 0xdf, 0x3e, 0x97, 0xfa, 0xdc, 0x84, 0x45, - 0xea, 0xce, 0xaf, 0x46, 0xf0, 0x99, 0xe0, 0x39, 0x44, 0x59, 0x2b, 0x3d, 0x10, 0x7d, 0x9b, 0x28, - 0x5b, 0xdc, 0x3f, 0xdc, 0x26, 0xa1, 0x51, 0xe0, 0x35, 0x5c, 0x96, 0xcc, 0xb3, 0x2e, 0xa0, 0xb4, - 0xe8, 0x08, 0xb4, 0x0d, 0x0b, 0xf1, 0x13, 0x08, 0xfc, 0x85, 0x97, 0x27, 0xc6, 0xcd, 0x21, 0xf5, - 0xb5, 0x66, 0xec, 0x30, 0x3c, 0xd4, 0x86, 0x39, 0x6e, 0xec, 0xa9, 0xe6, 0xa9, 0x6a, 0x41, 0x13, - 0x6d, 0xc3, 0x79, 0xb6, 0x83, 0x1d, 0x2e, 0x60, 0x6c, 0xf3, 0xd5, 0x5c, 0x9b, 0x97, 0x0d, 0x8d, - 0xcb, 0x67, 0xed, 0xb4, 0xf2, 0xd9, 0x86, 0x39, 0x2e, 0x33, 0x54, 0x25, 0x55, 0xb5, 0xa0, 0x49, - 0x8e, 0x39, 0x94, 0x9e, 0x3a, 0x13, 0x02, 0xd1, 0x41, 0xc6, 0x05, 0x8a, 0xbd, 0x41, 0x15, 0x7b, - 0xd0, 0x54, 0x7f, 0x55, 0x01, 0x08, 0x29, 0x3d, 0x25, 0xe3, 0xf3, 0x36, 0x54, 0x05, 0xdb, 0xe7, - 0x0a, 0x5a, 0x05, 0x78, 0xd2, 0x74, 0x14, 0x13, 0xa6, 0x43, 0xfd, 0x47, 0x05, 0x1a, 0x1b, 0x64, - 0x9f, 0x9b, 0xce, 0x80, 0x1a, 0xba, 0x9b, 0xd0, 0x74, 0x71, 0xdf, 0x71, 0x8d, 0x1e, 0xb6, 0x7d, - 0xd7, 0xc4, 0x2c, 0x5b, 0x50, 0xd2, 0xe6, 0x59, 0xef, 0x07, 0xac, 0x93, 0x80, 0x11, 0x6b, 0xe0, - 0xf9, 0xfa, 0x70, 0xd4, 0xdb, 0x27, 0xfa, 0x87, 0x25, 0xa0, 0xe7, 0x45, 0x2f, 0x55, 0x3f, 0xdf, - 0x80, 0x46, 0x08, 0xe6, 0x3b, 0x74, 0xfd, 0x92, 0x56, 0x17, 0x7d, 0xbb, 0x0e, 0xba, 0x01, 0x4d, - 0x4a, 0xe8, 0x9e, 0xe5, 0x0c, 0x7a, 0x24, 0x06, 0xe5, 0x36, 0xb0, 0x61, 0x70, 0xb4, 0xc8, 0x01, - 0xc6, 0xa1, 0x3c, 0xf3, 0x73, 0xcc, 0xad, 0xa0, 0x80, 0xda, 0x31, 0x3f, 0xc7, 0xea, 0xaf, 0x28, - 0x30, 0xcf, 0x8d, 0xe6, 0x8e, 0xa8, 0x14, 0xd0, 0xf4, 0x29, 0x8b, 0xff, 0xe9, 0xff, 0xe8, 0x9d, - 0x78, 0x02, 0xed, 0x86, 0x54, 0x08, 0xe8, 0x24, 0xd4, 0x99, 0x8b, 0x59, 0xcc, 0x3c, 0x01, 0xe8, - 0x17, 0x84, 0xa6, 0xba, 0xaf, 0x3f, 0x72, 0x0c, 0x96, 0xcf, 0x6b, 0xc3, 0x9c, 0x6e, 0x18, 0x2e, - 0xf6, 0x3c, 0x8e, 0x47, 0xd0, 0x24, 0x5f, 0x8e, 0xb0, 0xeb, 0x05, 0x07, 0x5b, 0xd4, 0x82, 0x26, - 0xfa, 0x56, 0x22, 0x81, 0x5f, 0x5f, 0xbd, 0x96, 0x8d, 0x27, 0x0f, 0x97, 0xc2, 0x14, 0xff, 0x5f, - 0x17, 0xa0, 0xc9, 0x65, 0x70, 0x8d, 0xdb, 0xb7, 0xc9, 0x2c, 0xb6, 0x06, 0x8d, 0xfd, 0x90, 0xf7, - 0x27, 0xa5, 0x7b, 0xa2, 0x22, 0x12, 0x1b, 0x33, 0x8d, 0xd7, 0xe2, 0x16, 0xb6, 0x34, 0x93, 0x85, - 0x2d, 0x9f, 0x56, 0x82, 0xd3, 0x9e, 0x56, 0x45, 0xe2, 0x69, 0xa9, 0xbf, 0x00, 0xf5, 0xc8, 0x04, - 0x54, 0x43, 0xb1, 0x8c, 0x0a, 0xa7, 0x58, 0xd0, 0x44, 0xf7, 0x42, 0x3f, 0x83, 0x91, 0xea, 0xa2, - 0x04, 0x97, 0x84, 0x8b, 0xa1, 0xfe, 0x54, 0x81, 0x0a, 0x9f, 0xf9, 0x2a, 0xd4, 0xb9, 0x7c, 0x51, - 0xcf, 0x8b, 0xcd, 0x0e, 0xbc, 0x8b, 0xb8, 0x5e, 0xcf, 0x4e, 0xc0, 0x2e, 0x42, 0x35, 0x21, 0x5a, - 0x73, 0x5c, 0x2d, 0x06, 0x9f, 0x22, 0xf2, 0x44, 0x3e, 0x11, 0x51, 0x42, 0x4b, 0x50, 0xb6, 0x9c, - 0x81, 0xa8, 0xb6, 0xb0, 0x86, 0xfa, 0x33, 0x85, 0x26, 0xc7, 0x35, 0xdc, 0x77, 0x8e, 0xb0, 0x7b, - 0x32, 0x7b, 0x7e, 0xf1, 0xdd, 0x08, 0x9b, 0xe7, 0x0c, 0x72, 0xc4, 0x00, 0xf4, 0x6e, 0x78, 0x08, - 0x45, 0x59, 0x1a, 0x22, 0x6a, 0x8a, 0x38, 0x93, 0x86, 0x87, 0xf1, 0xbb, 0x0a, 0xcd, 0x94, 0xc6, - 0xb7, 0x72, 0x56, 0x6b, 0xff, 0x4c, 0xc2, 0x01, 0xf5, 0x1f, 0x14, 0xb8, 0x98, 0x41, 0xdd, 0x27, - 0xab, 0x5f, 0x03, 0x7d, 0xdf, 0x81, 0xaa, 0x08, 0x89, 0x8b, 0xb9, 0x42, 0x62, 0x01, 0xaf, 0xfe, - 0x01, 0xcb, 0xd7, 0x4b, 0xc8, 0xfb, 0x64, 0xf5, 0x39, 0x11, 0x38, 0x99, 0xda, 0x2a, 0x4a, 0x52, - 0x5b, 0xff, 0xa4, 0x40, 0x27, 0x4c, 0x25, 0x79, 0x6b, 0x27, 0xb3, 0x16, 0x78, 0x9e, 0x4d, 0x20, - 0xf8, 0xb6, 0xa8, 0x45, 0x10, 0xbd, 0x98, 0x2b, 0x84, 0x0b, 0x2a, 0x11, 0x36, 0xcd, 0x4a, 0xa7, - 0x37, 0x34, 0x8b, 0x54, 0x76, 0x22, 0x07, 0xcf, 0xea, 0x11, 0xe1, 0xc1, 0xfe, 0x94, 0x31, 0xe9, - 0x83, 0x78, 0x3e, 0xe9, 0xeb, 0x26, 0x60, 0xb4, 0x46, 0x72, 0xc0, 0x6b, 0x24, 0xa5, 0x44, 0x8d, - 0x84, 0xf7, 0xab, 0x43, 0xca, 0x02, 0xa9, 0x0d, 0x3c, 0x2f, 0x82, 0xfd, 0x9a, 0x02, 0x6d, 0xbe, - 0x0a, 0xab, 0xfb, 0x3b, 0xc3, 0x91, 0x85, 0x7d, 0x6c, 0x7c, 0xd5, 0x39, 0x8d, 0xbf, 0x2c, 0x40, - 0x2b, 0xea, 0xd8, 0x50, 0xdf, 0xe4, 0x4d, 0x28, 0xd3, 0xa4, 0x11, 0xc7, 0x60, 0xaa, 0x76, 0x60, - 0xd0, 0xc4, 0x32, 0x52, 0x6f, 0x7e, 0xd7, 0x0b, 0x1c, 0x17, 0xde, 0x0c, 0xbd, 0xab, 0xe2, 0xe9, - 0xbd, 0xab, 0x17, 0xa1, 0x46, 0x2c, 0x97, 0x33, 0x26, 0xf3, 0xb2, 0xc2, 0x75, 0xd8, 0x81, 0xde, - 0x83, 0x0a, 0xbb, 0x2a, 0xc3, 0xeb, 0x86, 0x37, 0xe3, 0x53, 0xf3, 0x6b, 0x34, 0x91, 0xbc, 0x3f, - 0xed, 0xd0, 0xf8, 0x20, 0x72, 0x46, 0x23, 0xd7, 0x19, 0x50, 0x37, 0x8c, 0x18, 0xb5, 0xb2, 0x26, - 0xda, 0xc4, 0x4d, 0x74, 0x46, 0xdd, 0x0d, 0x9e, 0x01, 0xa1, 0xff, 0xab, 0xff, 0x1f, 0x96, 0xc3, - 0x80, 0x9b, 0xa1, 0x79, 0x56, 0x26, 0x57, 0xff, 0x55, 0x81, 0xf3, 0x3b, 0x27, 0x76, 0x3f, 0x29, - 0x2e, 0xcb, 0x50, 0x19, 0x59, 0x7a, 0x98, 0xa1, 0xe6, 0x2d, 0x5a, 0xfd, 0x0f, 0x42, 0x69, 0x62, - 0xd6, 0x19, 0x8d, 0xeb, 0xa2, 0x6f, 0xd7, 0x99, 0xea, 0x6d, 0xdd, 0x14, 0x19, 0x02, 0x6c, 0x30, - 0x07, 0x82, 0x65, 0xe0, 0xe6, 0x45, 0x2f, 0x75, 0x20, 0xde, 0x03, 0xa0, 0x3e, 0x56, 0xef, 0x34, - 0x7e, 0x15, 0x1d, 0xb1, 0x49, 0xac, 0xe8, 0x8f, 0x0b, 0xd0, 0x8e, 0x50, 0xe9, 0xab, 0x76, 0x39, - 0x33, 0x02, 0xc5, 0xe2, 0x33, 0x0a, 0x14, 0x4b, 0xb3, 0xbb, 0x99, 0x65, 0x99, 0x9b, 0xf9, 0x83, - 0x22, 0x34, 0x43, 0xaa, 0x6d, 0x5b, 0xba, 0x9d, 0xc9, 0x09, 0x3b, 0xd0, 0xf4, 0x62, 0x54, 0xe5, - 0x74, 0x7a, 0x55, 0x26, 0x57, 0x19, 0x07, 0xa1, 0x25, 0xa6, 0x40, 0x97, 0xe9, 0xa1, 0xbb, 0x3e, - 0xcb, 0xe8, 0x31, 0x9f, 0xb1, 0xc6, 0x04, 0xd8, 0x1c, 0x62, 0xf4, 0x1a, 0x20, 0x2e, 0x75, 0x3d, - 0xd3, 0xee, 0x79, 0xb8, 0xef, 0xd8, 0x06, 0x93, 0xc7, 0xb2, 0xd6, 0xe2, 0x5f, 0xba, 0xf6, 0x0e, - 0xeb, 0x47, 0x6f, 0x42, 0xc9, 0x3f, 0x19, 0x31, 0x07, 0xb2, 0x29, 0x75, 0xc1, 0x42, 0xbc, 0x76, - 0x4f, 0x46, 0x58, 0xa3, 0xe0, 0xc1, 0x0d, 0x2a, 0xdf, 0xd5, 0x8f, 0xb8, 0x37, 0x5e, 0xd2, 0x22, - 0x3d, 0xd1, 0xd8, 0x79, 0x2e, 0x16, 0x3b, 0x33, 0xce, 0x0e, 0x84, 0xbc, 0xe7, 0xfb, 0x16, 0xcd, - 0x49, 0x52, 0xce, 0x0e, 0x7a, 0x77, 0x7d, 0x8b, 0x6c, 0xd2, 0x77, 0x7c, 0xdd, 0x62, 0xf2, 0x51, - 0xe3, 0xda, 0x84, 0xf4, 0xd0, 0xc8, 0xf7, 0x5f, 0x88, 0x36, 0x14, 0x88, 0x69, 0xd8, 0x1b, 0x5b, - 0xd9, 0xf2, 0x38, 0x39, 0x9b, 0x33, 0x4d, 0x14, 0xbf, 0x0d, 0x75, 0xce, 0x15, 0xa7, 0xe0, 0x2a, - 0x60, 0x43, 0x36, 0x27, 0xb0, 0x79, 0xf9, 0x19, 0xb1, 0x79, 0xe5, 0x0c, 0xf9, 0x10, 0xf9, 0xd9, - 0xa8, 0x3f, 0x54, 0xe0, 0x85, 0x94, 0xd6, 0x9c, 0x48, 0xda, 0xc9, 0xd1, 0x38, 0xd7, 0xa6, 0xc9, - 0x29, 0xb9, 0xbd, 0x78, 0x17, 0x2a, 0x2e, 0x9d, 0x9d, 0x57, 0xe6, 0xae, 0x4f, 0x64, 0x3e, 0x86, - 0x88, 0xc6, 0x87, 0xa8, 0xbf, 0xa7, 0xc0, 0x85, 0x34, 0xaa, 0x33, 0x38, 0x01, 0x6b, 0x30, 0xc7, - 0xa6, 0x0e, 0x64, 0x74, 0x65, 0xb2, 0x8c, 0x86, 0xc4, 0xd1, 0x82, 0x81, 0xea, 0x0e, 0x2c, 0x07, - 0xbe, 0x42, 0x48, 0xfa, 0x2d, 0xec, 0xeb, 0x13, 0x62, 0xd1, 0xab, 0x50, 0x67, 0x41, 0x0d, 0x8b, - 0xf1, 0x58, 0x21, 0x13, 0xf6, 0x44, 0xf2, 0x4f, 0xfd, 0xfd, 0x02, 0x2c, 0x51, 0x63, 0x9b, 0xac, - 0x4a, 0xe5, 0x29, 0x93, 0xaa, 0xe2, 0x22, 0xda, 0x23, 0x7d, 0xc8, 0x2f, 0xcb, 0xd4, 0xb4, 0x58, - 0x1f, 0xea, 0xa6, 0x73, 0x83, 0xd2, 0x9c, 0x45, 0x58, 0x17, 0xde, 0xd0, 0x7d, 0x9d, 0x96, 0x85, - 0x93, 0x49, 0xc1, 0xd0, 0xc8, 0x97, 0xce, 0x62, 0xe4, 0x5f, 0x81, 0x16, 0x4b, 0x97, 0xf7, 0x44, - 0x08, 0x4c, 0x15, 0x53, 0x49, 0x5b, 0x60, 0xfd, 0xbb, 0x41, 0xb7, 0xba, 0x09, 0x2f, 0x24, 0x88, - 0x32, 0xc3, 0xe1, 0xab, 0x7f, 0xa6, 0x90, 0x93, 0x8b, 0xdd, 0x4f, 0x3a, 0xbb, 0x4f, 0x7c, 0x59, - 0x54, 0xce, 0x7a, 0xa6, 0x91, 0xd4, 0x37, 0x06, 0x7a, 0x1f, 0x6a, 0x36, 0x3e, 0xee, 0x45, 0xdd, - 0xac, 0x1c, 0x01, 0x43, 0xd5, 0xc6, 0xc7, 0xf4, 0x3f, 0xf5, 0x11, 0x5c, 0x48, 0xa1, 0x3a, 0xcb, - 0xde, 0xff, 0x46, 0x81, 0x8b, 0x1b, 0xae, 0x33, 0x7a, 0x62, 0xba, 0xfe, 0x58, 0xb7, 0xe2, 0xc5, - 0xf9, 0x33, 0x6c, 0x3f, 0xc7, 0xdd, 0xc7, 0x0f, 0x53, 0xa1, 0xe9, 0x6b, 0x12, 0x61, 0x4b, 0x23, - 0xc5, 0x37, 0x1d, 0x71, 0xcf, 0xff, 0xa3, 0x28, 0x43, 0x9e, 0xc3, 0x4d, 0x71, 0x61, 0xf2, 0xc4, - 0x2e, 0xd2, 0x34, 0x7e, 0xf1, 0xac, 0x69, 0xfc, 0x0c, 0x4b, 0x50, 0x7a, 0x46, 0x96, 0xe0, 0xd4, - 0x79, 0xb5, 0x75, 0x88, 0x97, 0x58, 0xa8, 0x21, 0x3f, 0x6d, 0x59, 0xe6, 0x3d, 0x80, 0xb0, 0xd2, - 0xc0, 0xef, 0x93, 0x4e, 0x99, 0x21, 0x32, 0x80, 0x9c, 0x91, 0xb0, 0xb5, 0xdc, 0x15, 0x88, 0x64, - 0xb8, 0xbf, 0x0b, 0x1d, 0x19, 0x6f, 0xce, 0xc2, 0xef, 0xff, 0x56, 0x00, 0xe8, 0x8a, 0xdb, 0xc7, - 0x67, 0x33, 0x16, 0xd7, 0x21, 0xe2, 0xae, 0x84, 0x52, 0x1e, 0xe5, 0x1d, 0x83, 0x08, 0x82, 0x08, - 0x72, 0x09, 0x4c, 0x2a, 0xf0, 0x35, 0xe8, 0x3c, 0x11, 0x59, 0x61, 0xac, 0x90, 0xd4, 0xcf, 0x97, - 0xa0, 0xe6, 0x3a, 0xc7, 0x3d, 0x22, 0x5c, 0x46, 0x70, 0xbd, 0xda, 0x75, 0x8e, 0x89, 0xc8, 0x19, - 0xe8, 0x02, 0xcc, 0xf9, 0xba, 0x77, 0x48, 0xe6, 0x67, 0xb9, 0xbe, 0x0a, 0x69, 0x76, 0x0d, 0xb4, - 0x04, 0xe5, 0x7d, 0xd3, 0xc2, 0xec, 0x26, 0x47, 0x4d, 0x63, 0x0d, 0xf4, 0xcd, 0xe0, 0x46, 0x60, - 0x35, 0xf7, 0xcd, 0x1f, 0x76, 0x29, 0xf0, 0x3a, 0xcc, 0x13, 0x4e, 0x22, 0x48, 0x30, 0xb1, 0x6e, - 0xf1, 0x3c, 0x3f, 0xef, 0xa4, 0x15, 0xff, 0x9f, 0x29, 0xb0, 0x10, 0x92, 0x96, 0xea, 0x26, 0xa2, - 0xee, 0xa8, 0xaa, 0x5b, 0x77, 0x0c, 0xa6, 0x45, 0x9a, 0x19, 0x76, 0x85, 0x0d, 0x64, 0x0a, 0x2d, - 0x1c, 0x32, 0x29, 0x38, 0x27, 0x9b, 0x27, 0x94, 0x31, 0x8d, 0x20, 0x5d, 0x54, 0x71, 0x9d, 0xe3, - 0xae, 0x21, 0x48, 0xc6, 0x2e, 0x58, 0xb3, 0x50, 0x94, 0x90, 0x6c, 0x9d, 0xde, 0xb1, 0xbe, 0x0e, - 0xf3, 0xd8, 0x75, 0x1d, 0xb7, 0x37, 0xc4, 0x9e, 0xa7, 0x0f, 0x82, 0xbb, 0x0b, 0x0d, 0xda, 0xb9, - 0xc5, 0xfa, 0xd4, 0xbf, 0x2d, 0x41, 0x33, 0xdc, 0x4a, 0x70, 0x8b, 0xc0, 0x34, 0x82, 0x5b, 0x04, - 0x26, 0x39, 0x5f, 0x70, 0x99, 0x96, 0x14, 0x1c, 0xb0, 0x56, 0x68, 0x2b, 0x5a, 0x8d, 0xf7, 0x76, - 0x0d, 0x62, 0xdc, 0x09, 0x81, 0x6c, 0xc7, 0xc0, 0x21, 0x07, 0x40, 0xd0, 0xc5, 0x19, 0x20, 0xc6, - 0x48, 0xa5, 0x1c, 0x8c, 0x54, 0xce, 0xc1, 0x48, 0x15, 0x09, 0x23, 0x2d, 0x43, 0x65, 0x6f, 0xdc, - 0x3f, 0xc4, 0x3e, 0xf7, 0xfb, 0x78, 0x2b, 0xce, 0x60, 0xd5, 0x04, 0x83, 0x09, 0x3e, 0xaa, 0x45, - 0xf9, 0xe8, 0x12, 0xd4, 0x02, 0x4b, 0xed, 0xd1, 0xaa, 0x5a, 0x51, 0xab, 0x72, 0x13, 0xed, 0xa1, - 0xb7, 0x02, 0xa7, 0xb0, 0x4e, 0x25, 0x4a, 0x95, 0x28, 0xa4, 0x04, 0x97, 0x04, 0x2e, 0xe1, 0xcb, - 0xb0, 0x10, 0x21, 0x07, 0xe5, 0x33, 0x56, 0x7a, 0x8b, 0xc4, 0x0c, 0xd4, 0x82, 0xdc, 0x84, 0x66, - 0x48, 0x12, 0x0a, 0x37, 0xcf, 0x42, 0x35, 0xd1, 0x4b, 0xc1, 0x04, 0xbb, 0x37, 0x4f, 0xc9, 0xee, - 0x17, 0xa1, 0xca, 0x63, 0x2c, 0xaf, 0xbd, 0x10, 0x4f, 0x91, 0xe4, 0x92, 0x84, 0x4f, 0x01, 0x85, - 0x5b, 0x9c, 0xcd, 0x31, 0x4d, 0xf0, 0x50, 0x21, 0xc9, 0x43, 0xea, 0x9f, 0x2b, 0xb0, 0x18, 0x5d, - 0xec, 0xac, 0x86, 0xfb, 0x7d, 0xa8, 0xb3, 0xe2, 0x67, 0x8f, 0xa8, 0x10, 0x79, 0xad, 0x32, 0x71, - 0x78, 0x1a, 0x84, 0xef, 0x38, 0x08, 0x61, 0x8e, 0x1d, 0xf7, 0xd0, 0xb4, 0x07, 0x3d, 0x82, 0x99, - 0x48, 0xe1, 0xf2, 0xce, 0x47, 0xa4, 0x4f, 0xfd, 0x4d, 0x05, 0xae, 0x3c, 0x1e, 0x19, 0xba, 0x8f, - 0x23, 0x1e, 0xcc, 0xac, 0xd7, 0x29, 0xc5, 0x7d, 0xc6, 0xc2, 0x84, 0x63, 0x8e, 0xac, 0xe7, 0xf1, - 0xfb, 0x8c, 0xc4, 0xef, 0xe3, 0xd8, 0xa4, 0x2e, 0x20, 0x9f, 0x1d, 0x9b, 0x0e, 0x54, 0x8f, 0xf8, - 0x74, 0xc1, 0xcb, 0x94, 0xa0, 0x1d, 0x2b, 0x06, 0x17, 0x4f, 0x55, 0x0c, 0x56, 0xb7, 0xe0, 0xa2, - 0x86, 0x3d, 0x6c, 0x1b, 0xb1, 0x8d, 0x9c, 0x39, 0xa9, 0x35, 0x82, 0x8e, 0x6c, 0xba, 0x59, 0x38, - 0x95, 0x39, 0xbe, 0x3d, 0x97, 0x4c, 0xeb, 0x73, 0x65, 0x4d, 0xfc, 0x2d, 0xba, 0x8e, 0xaf, 0xfe, - 0x45, 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, 0xd9, - 0x05, 0xac, 0x77, 0x79, 0xc9, 0xb4, 0x67, 0x39, 0x03, 0x6a, 0x86, 0xa7, 0x3b, 0x67, 0xd5, 0x20, - 0x39, 0xa7, 0x8e, 0xa0, 0x9d, 0x26, 0xd6, 0x8c, 0x7a, 0x24, 0xa0, 0xc8, 0xc8, 0x61, 0x89, 0xdf, - 0x06, 0xf1, 0xc4, 0x68, 0xd7, 0xb6, 0xe3, 0xa9, 0xff, 0x55, 0x80, 0xf6, 0x8e, 0x7e, 0x84, 0xff, - 0xef, 0x1c, 0xd0, 0x27, 0xb0, 0xe4, 0xe9, 0x47, 0xb8, 0x17, 0x09, 0xc0, 0x7b, 0x2e, 0xfe, 0x8c, - 0x3b, 0xb1, 0xaf, 0xc8, 0x52, 0xf3, 0xd2, 0x1b, 0x46, 0xda, 0xa2, 0x17, 0xeb, 0xd7, 0xf0, 0x67, - 0xe8, 0x25, 0x58, 0x88, 0xde, 0x86, 0x23, 0xa8, 0x55, 0x29, 0xc9, 0xe7, 0x23, 0x37, 0xde, 0xba, - 0x86, 0xfa, 0x19, 0xbc, 0xf8, 0xd8, 0xf6, 0xb0, 0xdf, 0x0d, 0x6f, 0x6d, 0xcd, 0x18, 0x7f, 0x5e, - 0x85, 0x7a, 0x48, 0xf8, 0xd4, 0x93, 0x14, 0xc3, 0x53, 0x1d, 0xe8, 0x6c, 0xe9, 0xee, 0x61, 0x90, - 0xce, 0xde, 0x60, 0xb7, 0x61, 0x9e, 0xe3, 0x82, 0xfb, 0xe2, 0x5e, 0x98, 0x86, 0xf7, 0xb1, 0x8b, - 0xed, 0x3e, 0xde, 0x74, 0xfa, 0x87, 0xc4, 0x21, 0xf1, 0xd9, 0xab, 0x40, 0x25, 0xe2, 0xbb, 0x6e, - 0x44, 0x1e, 0xfd, 0x15, 0x62, 0x8f, 0xfe, 0xa6, 0x3c, 0x70, 0x55, 0x7f, 0x54, 0x80, 0xe5, 0xfb, - 0x96, 0x8f, 0xdd, 0x30, 0xc3, 0x70, 0x9a, 0x64, 0x49, 0x98, 0xbd, 0x28, 0x9c, 0x25, 0x7b, 0x91, - 0xa3, 0x82, 0x29, 0xcb, 0xb5, 0x94, 0xce, 0x98, 0x6b, 0xb9, 0x0f, 0x30, 0x72, 0x9d, 0x11, 0x76, - 0x7d, 0x13, 0x07, 0xb1, 0x5f, 0x0e, 0x07, 0x27, 0x32, 0x48, 0xfd, 0x04, 0x5a, 0x0f, 0xfb, 0xeb, - 0x8e, 0xbd, 0x6f, 0xba, 0xc3, 0x80, 0x50, 0x29, 0xa1, 0x53, 0x72, 0x08, 0x5d, 0x21, 0x25, 0x74, - 0xaa, 0x09, 0x8b, 0x91, 0xb9, 0x67, 0x54, 0x5c, 0x83, 0x7e, 0x6f, 0xdf, 0xb4, 0x4d, 0x7a, 0xdb, - 0xac, 0x40, 0x1d, 0x54, 0x18, 0xf4, 0x1f, 0xf0, 0x1e, 0xf5, 0x4b, 0x05, 0x2e, 0x69, 0x98, 0x08, - 0x4f, 0x70, 0x71, 0x67, 0xd7, 0xdf, 0xf2, 0x06, 0x33, 0x38, 0x14, 0xf7, 0xa0, 0x34, 0xf4, 0x06, - 0x19, 0x45, 0x77, 0x62, 0xa2, 0x63, 0x0b, 0x69, 0x14, 0x58, 0xfd, 0x89, 0x02, 0x4b, 0x41, 0x69, - 0x32, 0x26, 0xc2, 0x71, 0xb6, 0x55, 0x52, 0x57, 0xa9, 0x27, 0xbc, 0x04, 0xbe, 0x00, 0x73, 0xc6, - 0x5e, 0x54, 0x41, 0x56, 0x8c, 0x3d, 0xaa, 0x1b, 0x25, 0x9e, 0x72, 0x49, 0xea, 0x29, 0x27, 0x19, - 0xbf, 0x2c, 0xb9, 0xf3, 0xf4, 0x18, 0xda, 0xdc, 0x41, 0xf9, 0x68, 0x84, 0x5d, 0x9d, 0xf2, 0x57, - 0x80, 0xfc, 0xdb, 0x81, 0x0b, 0xad, 0x64, 0xbe, 0xb3, 0x4b, 0x96, 0x25, 0xb9, 0x13, 0xad, 0xfe, - 0xbd, 0x02, 0xd7, 0x92, 0xf3, 0x6e, 0xf3, 0xa2, 0xdd, 0xcc, 0x4f, 0xc8, 0x69, 0xc5, 0xaf, 0x10, - 0x56, 0xfc, 0x66, 0x2a, 0x5d, 0x46, 0xab, 0x8b, 0xa5, 0x78, 0x75, 0xf1, 0xd6, 0xfb, 0xe2, 0x2e, - 0xf9, 0xee, 0xc9, 0x08, 0xa3, 0x39, 0x28, 0x3e, 0xc2, 0xc7, 0xad, 0x73, 0x08, 0xa0, 0xf2, 0xc8, - 0x71, 0x87, 0xba, 0xd5, 0x52, 0x50, 0x1d, 0xe6, 0x78, 0x45, 0xba, 0x55, 0x40, 0xf3, 0x50, 0x5b, - 0x0f, 0xaa, 0x74, 0xad, 0xe2, 0xad, 0x5b, 0xd0, 0x88, 0xde, 0x95, 0x25, 0xe3, 0x36, 0xf1, 0x40, - 0xef, 0x9f, 0xb4, 0xce, 0xa1, 0x0a, 0x14, 0x36, 0xef, 0xb6, 0x14, 0xfa, 0xf7, 0x8d, 0x56, 0xe1, - 0xd6, 0x1f, 0x29, 0xb0, 0x98, 0x42, 0x12, 0x35, 0x01, 0x1e, 0xdb, 0x7d, 0x5e, 0x78, 0x6e, 0x9d, - 0x43, 0x0d, 0xa8, 0x06, 0x65, 0x68, 0xb6, 0xf6, 0xae, 0x43, 0xa1, 0x5b, 0x05, 0xd4, 0x82, 0x06, - 0x1b, 0x38, 0xee, 0xf7, 0xb1, 0xe7, 0xb5, 0x8a, 0xa2, 0xe7, 0x81, 0x6e, 0x5a, 0x63, 0x17, 0xb7, - 0x4a, 0x04, 0xbf, 0x5d, 0x47, 0xc3, 0x16, 0xd6, 0x3d, 0xdc, 0x2a, 0x23, 0x04, 0x4d, 0xde, 0x08, - 0x06, 0x55, 0x22, 0x7d, 0xc1, 0xb0, 0xb9, 0x5b, 0x3f, 0x56, 0xa2, 0x65, 0x2f, 0x4a, 0x8b, 0x0b, - 0x70, 0xfe, 0xb1, 0x6d, 0xe0, 0x7d, 0xd3, 0xc6, 0x46, 0xf8, 0xa9, 0x75, 0x0e, 0x9d, 0x87, 0x85, - 0x2d, 0xec, 0x0e, 0x70, 0xa4, 0xb3, 0x80, 0x16, 0x61, 0x7e, 0xcb, 0x7c, 0x1a, 0xe9, 0x2a, 0xa2, - 0x25, 0x68, 0xed, 0x98, 0xf6, 0xc0, 0x8a, 0x02, 0x96, 0xe8, 0x68, 0xd3, 0x76, 0xdc, 0x48, 0x67, - 0x99, 0x76, 0xea, 0x9f, 0xc6, 0x3a, 0x2b, 0xa8, 0x03, 0xcb, 0x94, 0xa8, 0x77, 0x37, 0x30, 0xa1, - 0x46, 0xe4, 0xdb, 0x9c, 0x5a, 0xaa, 0x2a, 0x2d, 0x65, 0xf5, 0x27, 0x37, 0xa1, 0x46, 0x84, 0x75, - 0xdd, 0x71, 0x5c, 0x03, 0x59, 0x80, 0xe8, 0xe3, 0xb2, 0xe1, 0xc8, 0xb1, 0xc5, 0x43, 0x54, 0x74, - 0x3b, 0x21, 0xdf, 0xac, 0x91, 0x06, 0xe4, 0x22, 0xd1, 0xb9, 0x21, 0x85, 0x4f, 0x00, 0xab, 0xe7, - 0xd0, 0x90, 0xae, 0xb6, 0x6b, 0x0e, 0xf1, 0xae, 0xd9, 0x3f, 0x0c, 0x42, 0x80, 0xbb, 0x19, 0xaf, - 0xf9, 0xd2, 0xa0, 0xc1, 0x7a, 0xd7, 0xa5, 0xeb, 0xb1, 0xd7, 0x7f, 0x81, 0x1c, 0xa9, 0xe7, 0xd0, - 0x67, 0x54, 0xfd, 0x84, 0xf1, 0x54, 0xb0, 0xe0, 0x6a, 0xf6, 0x82, 0x29, 0xe0, 0x53, 0x2e, 0xb9, - 0x09, 0x65, 0xca, 0xf7, 0x48, 0x76, 0xf3, 0x20, 0xfa, 0x0b, 0x17, 0x9d, 0x6b, 0xd9, 0x00, 0x62, - 0xb6, 0x4f, 0x61, 0x21, 0xf1, 0xbe, 0x1c, 0xc9, 0x7c, 0x30, 0xf9, 0x2f, 0x05, 0x74, 0x6e, 0xe5, - 0x01, 0x15, 0x6b, 0x0d, 0xa0, 0x19, 0x7f, 0x94, 0x86, 0x56, 0x72, 0x3c, 0x6d, 0x65, 0x2b, 0xbd, - 0x92, 0xfb, 0x11, 0x2c, 0x65, 0x82, 0x56, 0xf2, 0xe5, 0x33, 0xba, 0x35, 0x71, 0x82, 0x38, 0xb3, - 0xbd, 0x9a, 0x0b, 0x56, 0x2c, 0x77, 0x42, 0x99, 0x20, 0xf5, 0xec, 0x34, 0xc9, 0xe3, 0xc1, 0x34, - 0x59, 0xef, 0x61, 0x3b, 0x77, 0x72, 0xc3, 0x8b, 0xa5, 0x7f, 0x99, 0x5d, 0x2f, 0x94, 0x3d, 0xdd, - 0x44, 0x6f, 0xc8, 0xa7, 0x9b, 0xf0, 0xe6, 0xb4, 0xb3, 0x7a, 0x9a, 0x21, 0x02, 0x89, 0xef, 0xd3, - 0x7b, 0x81, 0x92, 0xc7, 0x8f, 0x49, 0xb9, 0x0b, 0xe6, 0xcb, 0x7e, 0xd7, 0xd9, 0x79, 0xe3, 0x14, - 0x23, 0x04, 0x02, 0x4e, 0xf2, 0x69, 0x79, 0x20, 0x86, 0x77, 0xa6, 0x72, 0xcd, 0xd9, 0x64, 0xf0, - 0x7b, 0xb0, 0x90, 0x88, 0x4a, 0x50, 0xfe, 0xc8, 0xa5, 0x33, 0xc9, 0xdc, 0x32, 0x91, 0x4c, 0xdc, - 0x03, 0x44, 0x19, 0xdc, 0x2f, 0xb9, 0x2b, 0xd8, 0xb9, 0x95, 0x07, 0x54, 0x6c, 0x64, 0x04, 0x8b, - 0x89, 0x8f, 0x4f, 0x56, 0xd1, 0xab, 0xb9, 0x57, 0x7b, 0xb2, 0xda, 0x79, 0x2d, 0xff, 0x7a, 0x4f, - 0x56, 0xd5, 0x73, 0xc8, 0xa3, 0x0a, 0x3a, 0x71, 0x97, 0x0c, 0x65, 0xcc, 0x22, 0xbf, 0x33, 0xd7, - 0x79, 0x3d, 0x27, 0xb4, 0xd8, 0xe6, 0x11, 0x9c, 0x97, 0x5c, 0xf9, 0x43, 0xaf, 0x4f, 0x64, 0x8f, - 0xe4, 0x5d, 0xc7, 0xce, 0xed, 0xbc, 0xe0, 0x11, 0xf3, 0xd0, 0x0a, 0xf0, 0xba, 0x6f, 0x59, 0xcc, - 0xb3, 0x78, 0x2d, 0xcb, 0xf2, 0xc5, 0xc0, 0x32, 0xb6, 0x9a, 0x09, 0x2d, 0x96, 0xfc, 0x45, 0x40, - 0x3b, 0x07, 0xce, 0x31, 0x8d, 0x02, 0x06, 0x63, 0xee, 0x58, 0x66, 0x1a, 0xc0, 0x34, 0x68, 0x86, - 0x20, 0x4e, 0x1c, 0x21, 0x16, 0xef, 0x01, 0x3c, 0xc4, 0xfe, 0x16, 0xf6, 0x5d, 0x22, 0xfd, 0x2f, - 0x65, 0xe1, 0xce, 0x01, 0x82, 0xa5, 0x5e, 0x9e, 0x0a, 0x17, 0x25, 0xe8, 0x96, 0x6e, 0x8f, 0x75, - 0x2b, 0xf2, 0xb2, 0x4b, 0x4e, 0xd0, 0x24, 0xd8, 0x64, 0x82, 0xa6, 0xa1, 0xc5, 0x92, 0xc7, 0xc2, - 0x7f, 0x89, 0xdc, 0x42, 0x98, 0xec, 0xbf, 0xa4, 0x6f, 0xc0, 0x25, 0x75, 0xfb, 0x04, 0x78, 0xb1, - 0xf0, 0x17, 0x0a, 0xbd, 0xa8, 0x9a, 0x00, 0xf8, 0xd8, 0xf4, 0x0f, 0xb6, 0x2d, 0xdd, 0xf6, 0xf2, - 0xa0, 0x40, 0x01, 0x4f, 0x81, 0x02, 0x87, 0x17, 0x28, 0x18, 0x30, 0x1f, 0xab, 0xf8, 0x23, 0xd9, - 0xd3, 0x25, 0xd9, 0x45, 0x89, 0xce, 0xca, 0x74, 0x40, 0xb1, 0xca, 0x3e, 0xcc, 0xc7, 0x62, 0x38, - 0xe9, 0x2a, 0xb2, 0x28, 0x2f, 0xa9, 0xec, 0x12, 0xd2, 0x91, 0x24, 0xa8, 0x07, 0x28, 0x5d, 0xd8, - 0x44, 0xf9, 0xca, 0xe0, 0x93, 0x54, 0x4f, 0x76, 0xb5, 0x94, 0x69, 0xf3, 0xc4, 0xd5, 0x01, 0xb9, - 0xa9, 0x90, 0xde, 0x84, 0x90, 0x6a, 0xf3, 0x8c, 0x9b, 0x08, 0xea, 0x39, 0xf4, 0x31, 0x54, 0xf8, - 0x2f, 0x40, 0xdd, 0x98, 0x5c, 0x42, 0xe0, 0xb3, 0xdf, 0x9c, 0x02, 0x25, 0x26, 0x3e, 0x84, 0x0b, - 0x19, 0x05, 0x04, 0xa9, 0x97, 0x31, 0xb9, 0xd8, 0x30, 0xcd, 0xfe, 0x89, 0xc5, 0x52, 0xf5, 0x81, - 0x09, 0x8b, 0x65, 0xd5, 0x12, 0xa6, 0x2d, 0xd6, 0x83, 0xc5, 0x54, 0xfe, 0x55, 0x6a, 0x00, 0xb3, - 0xb2, 0xb4, 0xd3, 0x16, 0x18, 0xc0, 0x0b, 0xd2, 0x5c, 0xa3, 0xd4, 0x37, 0x99, 0x94, 0x95, 0x9c, - 0xb6, 0x50, 0x1f, 0xce, 0x4b, 0x32, 0x8c, 0x52, 0x1b, 0x97, 0x9d, 0x89, 0x9c, 0xb6, 0xc8, 0x3e, - 0x74, 0xd6, 0x5c, 0x47, 0x37, 0xfa, 0xba, 0xe7, 0xd3, 0xac, 0x1f, 0x09, 0x42, 0x03, 0xe7, 0x50, - 0x1e, 0x39, 0x48, 0x73, 0x83, 0xd3, 0xd6, 0xd9, 0x83, 0x3a, 0x3d, 0x4a, 0xf6, 0x2b, 0x3d, 0x48, - 0x6e, 0x21, 0x22, 0x10, 0x19, 0x6a, 0x47, 0x06, 0x28, 0x98, 0x7a, 0x17, 0xea, 0xeb, 0xb4, 0x7c, - 0xda, 0xb5, 0x0d, 0xfc, 0x34, 0x69, 0xad, 0xe8, 0x4f, 0x15, 0xdc, 0x8e, 0x00, 0xe4, 0xa6, 0xd0, - 0x3c, 0xf5, 0xd9, 0x0d, 0xfc, 0x94, 0x9d, 0xf3, 0x8a, 0x6c, 0xde, 0x18, 0x48, 0x46, 0x8c, 0x23, - 0x85, 0x8c, 0xd8, 0xf9, 0xa5, 0xa8, 0x27, 0x2b, 0x96, 0xbb, 0x93, 0x31, 0x49, 0x0a, 0x32, 0x58, - 0xf5, 0x6e, 0xfe, 0x01, 0x51, 0xbb, 0x10, 0xe0, 0xd5, 0xa5, 0xb5, 0xdb, 0x97, 0x27, 0xa1, 0x1e, - 0x75, 0x4f, 0x57, 0xa6, 0x03, 0x8a, 0x55, 0xb6, 0xa1, 0x46, 0xb8, 0x93, 0x1d, 0xcf, 0x0d, 0xd9, - 0x40, 0xf1, 0x39, 0xff, 0xe1, 0x6c, 0x60, 0xaf, 0xef, 0x9a, 0x7b, 0xfc, 0xd0, 0xa5, 0xe8, 0xc4, - 0x40, 0x26, 0x1e, 0x4e, 0x02, 0x52, 0x60, 0x3e, 0xa6, 0x3e, 0x83, 0x20, 0x1d, 0x57, 0x95, 0xaf, - 0x4f, 0x3b, 0xdf, 0xb8, 0x9a, 0xbc, 0x9d, 0x17, 0x5c, 0x2c, 0xfb, 0x4b, 0x34, 0x0e, 0xa2, 0xdf, - 0xd7, 0xc6, 0xa6, 0x65, 0x04, 0x89, 0x3f, 0x74, 0x77, 0xd2, 0x54, 0x31, 0xd0, 0x4c, 0xf7, 0x6f, - 0xc2, 0x08, 0xb1, 0xfe, 0xcf, 0x43, 0x4d, 0xe4, 0x9f, 0x91, 0x2c, 0x6b, 0x99, 0xcc, 0x7c, 0x77, - 0x6e, 0x4c, 0x06, 0x12, 0x33, 0x63, 0x58, 0x92, 0x65, 0x9b, 0xa5, 0x21, 0xf6, 0x84, 0xb4, 0xf4, - 0x14, 0xfe, 0x58, 0xfd, 0xb2, 0x01, 0xd5, 0x60, 0xe0, 0x57, 0x9c, 0xb8, 0xfa, 0x1a, 0x32, 0x49, - 0xdf, 0x83, 0x85, 0xc4, 0x8f, 0xaf, 0x48, 0x35, 0xb8, 0xfc, 0x07, 0x5a, 0xa6, 0x89, 0xda, 0xc7, - 0xfc, 0x77, 0x4b, 0x45, 0x88, 0xf7, 0x72, 0x56, 0x36, 0x2a, 0x19, 0xdd, 0x4d, 0x99, 0xf8, 0x7f, - 0x77, 0x80, 0xf3, 0x08, 0x20, 0x12, 0xda, 0x4c, 0x7e, 0x1b, 0x40, 0xbc, 0xf5, 0x69, 0xd4, 0x1a, - 0x4a, 0xa3, 0x97, 0x57, 0xf2, 0xdc, 0xb3, 0xce, 0xf6, 0x40, 0xb3, 0x63, 0x96, 0xc7, 0xd0, 0x88, - 0xbe, 0xda, 0x41, 0xd2, 0x5f, 0xa2, 0x4c, 0x3f, 0xeb, 0x99, 0xb6, 0x8b, 0xad, 0x53, 0x3a, 0xb6, - 0x53, 0xa6, 0xf3, 0x00, 0xa5, 0xef, 0x61, 0x48, 0x03, 0x81, 0xcc, 0xdb, 0x1f, 0xd2, 0x40, 0x20, - 0xfb, 0x72, 0x07, 0x4b, 0x4a, 0x26, 0x2f, 0x17, 0x48, 0x93, 0x92, 0x19, 0xd7, 0x35, 0xa4, 0x49, - 0xc9, 0xac, 0xdb, 0x0a, 0x11, 0xf9, 0x9b, 0x18, 0xba, 0xc9, 0x7e, 0x56, 0x77, 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, - 0x53, 0xbc, 0x6e, 0x3a, 0xfc, 0xbf, 0x3b, 0x81, 0xaa, 0xb8, 0x43, 0x67, 0xbd, 0x43, 0x66, 0x1d, - 0xed, 0xed, 0x55, 0x68, 0xeb, 0xde, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x33, 0xb2, 0x39, 0x67, - 0xef, 0x5a, 0x00, 0x00, + 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, 0x0b, 0x24, 0x97, 0x5d, 0xe4, + 0x90, 0x43, 0x80, 0x20, 0x39, 0xe4, 0xaf, 0x08, 0x72, 0x0f, 0xea, 0xd1, 0xd5, 0xaf, 0xea, 0x99, + 0x26, 0x47, 0xb2, 0x81, 0xe4, 0x44, 0x76, 0xf5, 0x57, 0x55, 0x5f, 0x7f, 0xf5, 0xbd, 0xbf, 0xaf, + 0x06, 0x5a, 0x86, 0xee, 0xeb, 0xbd, 0xbe, 0xe3, 0xb8, 0xc6, 0xed, 0x91, 0xeb, 0xf8, 0x0e, 0x5a, + 0x1c, 0x9a, 0xd6, 0xd1, 0xd8, 0x63, 0x4f, 0xb7, 0xc9, 0xeb, 0x4e, 0xa3, 0xef, 0x0c, 0x87, 0x8e, + 0xcd, 0x86, 0x3a, 0x4d, 0xd3, 0xf6, 0xb1, 0x6b, 0xeb, 0x16, 0x7f, 0x6e, 0x44, 0x27, 0x74, 0x1a, + 0x5e, 0xff, 0x00, 0x0f, 0x75, 0xfe, 0x54, 0x1b, 0x7a, 0x03, 0xfe, 0xef, 0xa2, 0x69, 0x1b, 0xf8, + 0x69, 0x74, 0x2b, 0x75, 0x0e, 0xca, 0x1f, 0x0c, 0x47, 0xfe, 0x89, 0xfa, 0xb7, 0x0a, 0x34, 0x1e, + 0x58, 0x63, 0xef, 0x40, 0xc3, 0x9f, 0x8d, 0xb1, 0xe7, 0xa3, 0xbb, 0x50, 0xda, 0xd3, 0x3d, 0xdc, + 0x56, 0xae, 0x29, 0x2b, 0xf5, 0xd5, 0x17, 0x6f, 0xc7, 0x70, 0xe2, 0xd8, 0x6c, 0x79, 0x83, 0x35, + 0xdd, 0xc3, 0x1a, 0x85, 0x44, 0x08, 0x4a, 0xc6, 0x5e, 0x77, 0xa3, 0x5d, 0xb8, 0xa6, 0xac, 0x14, + 0x35, 0xfa, 0x3f, 0xba, 0x02, 0xe0, 0xe1, 0xc1, 0x10, 0xdb, 0x7e, 0x77, 0xc3, 0x6b, 0x17, 0xaf, + 0x15, 0x57, 0x8a, 0x5a, 0x64, 0x04, 0xa9, 0xd0, 0xe8, 0x3b, 0x96, 0x85, 0xfb, 0xbe, 0xe9, 0xd8, + 0xdd, 0x8d, 0x76, 0x89, 0xce, 0x8d, 0x8d, 0xa1, 0x0e, 0x54, 0x4d, 0xaf, 0x3b, 0x1c, 0x39, 0xae, + 0xdf, 0x2e, 0x5f, 0x53, 0x56, 0xaa, 0x9a, 0x78, 0x56, 0x7f, 0x50, 0x80, 0x79, 0x8e, 0xb6, 0x37, + 0x72, 0x6c, 0x0f, 0xa3, 0x7b, 0x50, 0xf1, 0x7c, 0xdd, 0x1f, 0x7b, 0x1c, 0xf3, 0x4b, 0x52, 0xcc, + 0x77, 0x28, 0x88, 0xc6, 0x41, 0xa5, 0xa8, 0x27, 0x51, 0x2b, 0x4a, 0x50, 0x8b, 0x7f, 0x5e, 0x29, + 0xf5, 0x79, 0x2b, 0xb0, 0xb0, 0x4f, 0xb0, 0xdb, 0x09, 0x81, 0xca, 0x14, 0x28, 0x39, 0x4c, 0x56, + 0xf2, 0xcd, 0x21, 0xfe, 0x68, 0x7f, 0x07, 0xeb, 0x56, 0xbb, 0x42, 0xf7, 0x8a, 0x8c, 0xa0, 0x8b, + 0x50, 0xa5, 0x53, 0x7a, 0xbe, 0xd7, 0x9e, 0xbb, 0xa6, 0xac, 0x94, 0xb4, 0x39, 0xfa, 0xbc, 0xeb, + 0xa9, 0xdf, 0x87, 0x25, 0x4a, 0x82, 0xf5, 0x03, 0xdd, 0xb6, 0xb1, 0xe5, 0x9d, 0xfd, 0x04, 0xa3, + 0x9b, 0x14, 0x62, 0x9b, 0x90, 0x43, 0xe8, 0xf3, 0xf5, 0xe9, 0x31, 0xd6, 0x34, 0xf1, 0xac, 0xfe, + 0x66, 0x01, 0x5a, 0xe2, 0x53, 0x82, 0xdd, 0x97, 0xa0, 0xdc, 0x77, 0xc6, 0xb6, 0x4f, 0xb7, 0x9f, + 0xd7, 0xd8, 0x03, 0xfa, 0x06, 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, 0x09, 0x65, 0x0b, 0x1f, 0x61, 0x8b, 0x12, 0xbb, 0xb9, 0x7a, 0xf5, 0x76, + 0x4a, 0x26, 0x6f, 0xf3, 0x4f, 0xde, 0x24, 0x60, 0x1a, 0x83, 0x56, 0xff, 0x4c, 0x81, 0xe5, 0xfb, + 0x9e, 0x67, 0x0e, 0xec, 0x14, 0x41, 0x96, 0xa1, 0x62, 0x3b, 0x06, 0xee, 0x6e, 0x50, 0x8a, 0x14, + 0x35, 0xfe, 0x84, 0x2e, 0x41, 0x6d, 0x84, 0xb1, 0xdb, 0x73, 0x1d, 0x2b, 0xa0, 0x47, 0x95, 0x0c, + 0x68, 0x8e, 0x85, 0xd1, 0x77, 0x61, 0xd1, 0x4b, 0x2c, 0xc4, 0xe8, 0x5f, 0x5f, 0xbd, 0x9e, 0x8d, + 0x92, 0x80, 0xd5, 0xd2, 0xb3, 0xd5, 0x2f, 0x0a, 0x70, 0x5e, 0xc0, 0x31, 0x5c, 0xc9, 0xff, 0xe4, + 0xc0, 0x3c, 0x3c, 0x10, 0xe8, 0xb1, 0x87, 0x3c, 0x07, 0x26, 0x4e, 0xba, 0x18, 0x3d, 0xe9, 0x3c, + 0x92, 0x9d, 0x38, 0xc6, 0x72, 0xfa, 0x18, 0xaf, 0x42, 0x1d, 0x3f, 0x1d, 0x99, 0x2e, 0xee, 0x11, + 0x59, 0xa0, 0x27, 0x55, 0xd2, 0x80, 0x0d, 0xed, 0x9a, 0xc3, 0xa8, 0xb8, 0xcf, 0xe5, 0x16, 0x77, + 0xf5, 0xcf, 0x15, 0xb8, 0x90, 0x3a, 0x25, 0xae, 0x3f, 0x34, 0x68, 0xd1, 0x2f, 0x0f, 0x29, 0x43, + 0x34, 0x09, 0x21, 0xf8, 0x4b, 0x93, 0x08, 0x1e, 0x82, 0x6b, 0xa9, 0xf9, 0x11, 0x24, 0x0b, 0xf9, + 0x91, 0x3c, 0x84, 0x0b, 0x0f, 0xb1, 0xcf, 0x37, 0x20, 0xef, 0xf0, 0x0c, 0x92, 0x1d, 0x57, 0x54, + 0x85, 0xa4, 0xa2, 0x52, 0xff, 0x22, 0x14, 0x61, 0xba, 0x55, 0xd7, 0xde, 0x77, 0xd0, 0x8b, 0x50, + 0x13, 0x20, 0x9c, 0x2b, 0xc2, 0x01, 0xf4, 0x4d, 0x28, 0x13, 0x4c, 0x19, 0x4b, 0x34, 0x57, 0xbf, + 0x21, 0xff, 0xa6, 0xc8, 0x9a, 0x1a, 0x83, 0x47, 0x1b, 0xd0, 0xf4, 0x7c, 0xdd, 0xf5, 0x7b, 0x23, + 0xc7, 0xa3, 0xe7, 0x4c, 0x19, 0xa7, 0xbe, 0x7a, 0x39, 0xbe, 0x02, 0xb1, 0x5b, 0x5b, 0xde, 0x60, + 0x9b, 0x03, 0x69, 0xf3, 0x74, 0x52, 0xf0, 0x88, 0xbe, 0x03, 0x0d, 0x6c, 0x1b, 0xe1, 0x1a, 0xa5, + 0x3c, 0x6b, 0xd4, 0xb1, 0x6d, 0x88, 0x15, 0xc2, 0x53, 0x29, 0xe7, 0x3f, 0x95, 0xdf, 0x55, 0xa0, + 0x9d, 0x3e, 0x96, 0x59, 0x6c, 0xcf, 0xbb, 0x6c, 0x12, 0x66, 0xc7, 0x32, 0x51, 0xae, 0xc5, 0xd1, + 0x68, 0x7c, 0x8a, 0xfa, 0x27, 0x0a, 0xbc, 0x10, 0xa2, 0x43, 0x5f, 0x3d, 0x2f, 0x1e, 0x41, 0xb7, + 0xa0, 0x65, 0xda, 0x7d, 0x6b, 0x6c, 0xe0, 0xc7, 0xf6, 0x87, 0x58, 0xb7, 0xfc, 0x83, 0x13, 0x7a, + 0x72, 0x55, 0x2d, 0x35, 0xae, 0xfe, 0x47, 0x01, 0x96, 0x93, 0x78, 0xcd, 0x42, 0xa4, 0xff, 0x07, + 0x65, 0xd3, 0xde, 0x77, 0x02, 0x1a, 0x5d, 0x99, 0x20, 0x8a, 0x64, 0x2f, 0x06, 0x8c, 0x1c, 0x40, + 0x81, 0xf2, 0xea, 0x1f, 0xe0, 0xfe, 0xe1, 0xc8, 0x31, 0xa9, 0x9a, 0x22, 0x4b, 0x7c, 0x47, 0xb2, + 0x84, 0x1c, 0xe3, 0xdb, 0xdc, 0xb0, 0xae, 0x8b, 0x25, 0x3e, 0xb0, 0x7d, 0xf7, 0x44, 0x5b, 0xec, + 0x27, 0xc7, 0x3b, 0x7d, 0x58, 0x96, 0x03, 0xa3, 0x16, 0x14, 0x0f, 0xf1, 0x09, 0xfd, 0xe4, 0x9a, + 0x46, 0xfe, 0x45, 0xf7, 0xa0, 0x7c, 0xa4, 0x5b, 0x63, 0xcc, 0x75, 0xc2, 0x14, 0xce, 0x65, 0xb0, + 0xef, 0x14, 0xde, 0x52, 0xd4, 0x21, 0x5c, 0x7a, 0x88, 0xfd, 0xae, 0xed, 0x61, 0xd7, 0x5f, 0x33, + 0x6d, 0xcb, 0x19, 0x6c, 0xeb, 0xfe, 0xc1, 0x0c, 0xca, 0x21, 0x26, 0xe7, 0x85, 0x84, 0x9c, 0xab, + 0x3f, 0x54, 0xe0, 0x45, 0xf9, 0x7e, 0xfc, 0x40, 0x3b, 0x50, 0xdd, 0x37, 0xb1, 0x65, 0x10, 0xae, + 0x51, 0x28, 0xd7, 0x88, 0x67, 0xa2, 0x24, 0x46, 0x04, 0x98, 0x9f, 0x5b, 0x42, 0x49, 0x08, 0x37, + 0x76, 0xc7, 0x77, 0x4d, 0x7b, 0xb0, 0x69, 0x7a, 0xbe, 0xc6, 0xe0, 0x23, 0x5c, 0x52, 0xcc, 0x2f, + 0x9c, 0xbf, 0xad, 0xc0, 0x95, 0x87, 0xd8, 0x5f, 0x17, 0x36, 0x86, 0xbc, 0x37, 0x3d, 0xdf, 0xec, + 0x7b, 0xcf, 0xd6, 0xad, 0xcd, 0xe1, 0xa3, 0xa8, 0xbf, 0xaf, 0xc0, 0xd5, 0x4c, 0x64, 0x38, 0xe9, + 0xb8, 0x0e, 0x0d, 0x2c, 0x8c, 0x5c, 0x87, 0xfe, 0x02, 0x3e, 0x79, 0x42, 0x0e, 0x7f, 0x5b, 0x37, + 0x5d, 0xa6, 0x43, 0xcf, 0x68, 0x51, 0x7e, 0xa4, 0xc0, 0xe5, 0x87, 0xd8, 0xdf, 0x0e, 0xec, 0xeb, + 0xd7, 0x48, 0x1d, 0x02, 0x13, 0xb1, 0xf3, 0x81, 0xef, 0x1c, 0x1b, 0x53, 0x7f, 0x8f, 0x1d, 0xa7, + 0x14, 0xdf, 0xaf, 0x85, 0x80, 0x57, 0xa8, 0x24, 0x44, 0x54, 0x04, 0x17, 0x76, 0x4e, 0x3e, 0xf5, + 0xcb, 0x32, 0x34, 0x9e, 0x70, 0xad, 0x40, 0x2d, 0x68, 0x92, 0x12, 0x8a, 0xdc, 0x09, 0x8a, 0x78, + 0x53, 0x32, 0x07, 0x6b, 0x0d, 0xe6, 0x3d, 0x8c, 0x0f, 0x4f, 0x69, 0x2f, 0x1b, 0x64, 0x8e, 0x30, + 0x76, 0x9b, 0xb0, 0x38, 0xb6, 0xa9, 0x33, 0x8f, 0x0d, 0xfe, 0x01, 0x8c, 0xe8, 0xd3, 0x95, 0x69, + 0x7a, 0x22, 0xfa, 0x90, 0xc7, 0x35, 0x91, 0xb5, 0xca, 0xb9, 0xd6, 0x4a, 0x4e, 0x43, 0x5d, 0x68, + 0x19, 0xae, 0x33, 0x1a, 0x61, 0xa3, 0xe7, 0x05, 0x4b, 0x55, 0xf2, 0x2d, 0xc5, 0xe7, 0x89, 0xa5, + 0xee, 0xc2, 0xf9, 0x24, 0xa6, 0x5d, 0x83, 0xf8, 0x85, 0x84, 0xb3, 0x64, 0xaf, 0xd0, 0x6b, 0xb0, + 0x98, 0x86, 0xaf, 0x52, 0xf8, 0xf4, 0x0b, 0xf4, 0x3a, 0xa0, 0x04, 0xaa, 0x04, 0xbc, 0xc6, 0xc0, + 0xe3, 0xc8, 0x70, 0x70, 0x1a, 0x6f, 0xc7, 0xc1, 0x81, 0x81, 0xf3, 0x37, 0x11, 0xf0, 0x2e, 0xb1, + 0xae, 0x31, 0x70, 0xaf, 0x5d, 0xcf, 0x47, 0x88, 0xf8, 0x62, 0x9e, 0xfa, 0x5b, 0x0a, 0x2c, 0x7f, + 0xac, 0xfb, 0xfd, 0x83, 0x8d, 0xe1, 0xec, 0x31, 0xe1, 0x7b, 0x50, 0x3b, 0x12, 0x91, 0x1f, 0xd3, + 0xe2, 0xb2, 0x60, 0x28, 0xca, 0xf6, 0x5a, 0x38, 0x43, 0xfd, 0x47, 0x85, 0x47, 0xa7, 0x01, 0x76, + 0x5f, 0xbd, 0xaa, 0x99, 0x16, 0xa4, 0x27, 0x04, 0xb0, 0x9c, 0x12, 0x40, 0xf5, 0x29, 0x00, 0x47, + 0x7f, 0xcb, 0x1b, 0x9c, 0x01, 0xf3, 0xb7, 0x60, 0x8e, 0xef, 0xc7, 0xb5, 0xcd, 0xb4, 0x23, 0x0d, + 0xc0, 0xd5, 0xff, 0xae, 0x40, 0x3d, 0xf2, 0x02, 0x35, 0xa1, 0x20, 0xd4, 0x48, 0x41, 0xf2, 0xfd, + 0x85, 0xe9, 0x51, 0x56, 0x31, 0x1d, 0x65, 0xdd, 0x84, 0xa6, 0x49, 0xcd, 0x7b, 0x8f, 0x7f, 0x35, + 0xf5, 0xa6, 0x6b, 0xda, 0x3c, 0x1b, 0xe5, 0x4c, 0x84, 0xae, 0x40, 0xdd, 0x1e, 0x0f, 0x7b, 0xce, + 0x7e, 0xcf, 0x75, 0x8e, 0x3d, 0x1e, 0xae, 0xd5, 0xec, 0xf1, 0xf0, 0xa3, 0x7d, 0xcd, 0x39, 0xf6, + 0xc2, 0x88, 0xa0, 0x72, 0xca, 0x88, 0xe0, 0x0a, 0xd4, 0x87, 0xfa, 0x53, 0xb2, 0x6a, 0xcf, 0x1e, + 0x0f, 0x69, 0x24, 0x57, 0xd4, 0x6a, 0x43, 0xfd, 0xa9, 0xe6, 0x1c, 0x3f, 0x1a, 0x0f, 0xd1, 0x0a, + 0xb4, 0x2c, 0xdd, 0xf3, 0x7b, 0xd1, 0x50, 0xb0, 0x4a, 0x43, 0xc1, 0x26, 0x19, 0xff, 0x20, 0x0c, + 0x07, 0xd3, 0xb1, 0x45, 0xed, 0x6c, 0xb1, 0x85, 0x31, 0xb4, 0xc2, 0x35, 0x20, 0x57, 0x6c, 0x61, + 0x0c, 0x2d, 0xb1, 0xc2, 0x5b, 0x30, 0xb7, 0x47, 0x5d, 0xa5, 0x49, 0x42, 0xfc, 0x80, 0x78, 0x49, + 0xcc, 0xa3, 0xd2, 0x02, 0x70, 0xf4, 0x2d, 0xa8, 0x51, 0x0b, 0x45, 0xe7, 0x36, 0x72, 0xcd, 0x0d, + 0x27, 0x90, 0xd9, 0x06, 0xb6, 0x7c, 0x9d, 0xce, 0x9e, 0xcf, 0x37, 0x5b, 0x4c, 0x20, 0x1a, 0xb4, + 0xef, 0x62, 0xdd, 0xc7, 0xc6, 0xda, 0xc9, 0xba, 0x33, 0x1c, 0xe9, 0x94, 0x85, 0xda, 0x4d, 0xea, + 0xe4, 0xcb, 0x5e, 0xa1, 0x97, 0xa0, 0xd9, 0x17, 0x4f, 0x0f, 0x5c, 0x67, 0xd8, 0x5e, 0xa0, 0xf2, + 0x95, 0x18, 0x45, 0x97, 0x01, 0x02, 0xdd, 0xa9, 0xfb, 0xed, 0x16, 0x3d, 0xbb, 0x1a, 0x1f, 0xb9, + 0x4f, 0xd3, 0x42, 0xa6, 0xd7, 0x63, 0x09, 0x18, 0xd3, 0x1e, 0xb4, 0x17, 0xe9, 0x8e, 0xf5, 0x20, + 0x63, 0x63, 0xda, 0x03, 0x74, 0x01, 0xe6, 0x4c, 0xaf, 0xb7, 0xaf, 0x1f, 0xe2, 0x36, 0xa2, 0x6f, + 0x2b, 0xa6, 0xf7, 0x40, 0x3f, 0xa4, 0xde, 0x2b, 0xdf, 0x0c, 0x1b, 0xed, 0xf3, 0xf4, 0x55, 0x38, + 0x10, 0xe6, 0x71, 0x96, 0x4e, 0x95, 0xc7, 0xf9, 0x1c, 0x96, 0x42, 0x46, 0x8d, 0x70, 0x46, 0x9a, + 0xbf, 0x94, 0x33, 0xf0, 0xd7, 0x64, 0x87, 0xfb, 0xe7, 0x65, 0x58, 0xde, 0xd1, 0x8f, 0xf0, 0xf3, + 0xf7, 0xed, 0x73, 0xa9, 0xcf, 0x4d, 0x58, 0xa4, 0xee, 0xfc, 0x6a, 0x04, 0x9f, 0x09, 0x9e, 0x43, + 0x94, 0xb5, 0xd2, 0x13, 0xd1, 0xb7, 0x89, 0xb2, 0xc5, 0xfd, 0xc3, 0x6d, 0x12, 0x1a, 0x05, 0x5e, + 0xc3, 0x65, 0xc9, 0x3a, 0xeb, 0x02, 0x4a, 0x8b, 0xce, 0x40, 0xdb, 0xb0, 0x10, 0x3f, 0x81, 0xc0, + 0x5f, 0x78, 0x79, 0x62, 0xdc, 0x1c, 0x52, 0x5f, 0x6b, 0xc6, 0x0e, 0xc3, 0x43, 0x6d, 0x98, 0xe3, + 0xc6, 0x9e, 0x6a, 0x9e, 0xaa, 0x16, 0x3c, 0xa2, 0x6d, 0x38, 0xcf, 0xbe, 0x60, 0x87, 0x0b, 0x18, + 0xfb, 0xf8, 0x6a, 0xae, 0x8f, 0x97, 0x4d, 0x8d, 0xcb, 0x67, 0xed, 0xb4, 0xf2, 0xd9, 0x86, 0x39, + 0x2e, 0x33, 0x54, 0x25, 0x55, 0xb5, 0xe0, 0x91, 0x1c, 0x73, 0x28, 0x3d, 0x75, 0x26, 0x04, 0x62, + 0x80, 0xcc, 0x0b, 0x14, 0x7b, 0x83, 0x2a, 0xf6, 0xe0, 0x91, 0x6a, 0x1b, 0x3c, 0xe8, 0x31, 0x11, + 0x99, 0xcf, 0x27, 0x22, 0x55, 0x0f, 0x0f, 0xe8, 0x7f, 0x49, 0xcb, 0xd2, 0x4c, 0x59, 0x16, 0xf5, + 0xd7, 0x15, 0x80, 0xf0, 0x24, 0xa7, 0x64, 0x94, 0xde, 0x86, 0xaa, 0x10, 0xab, 0x5c, 0x41, 0xb1, + 0x00, 0x4f, 0x9a, 0xa6, 0x62, 0xc2, 0x34, 0xa9, 0xff, 0xac, 0x40, 0x63, 0x83, 0xd0, 0x71, 0xd3, + 0x19, 0x50, 0x43, 0x7a, 0x13, 0x9a, 0x2e, 0xee, 0x3b, 0xae, 0xd1, 0xc3, 0xb6, 0xef, 0x9a, 0x98, + 0x65, 0x23, 0x4a, 0xda, 0x3c, 0x1b, 0xfd, 0x80, 0x0d, 0x12, 0x30, 0x62, 0x6d, 0x3c, 0x5f, 0x1f, + 0x8e, 0x7a, 0xfb, 0x44, 0xbf, 0xb1, 0xbc, 0xf8, 0xbc, 0x18, 0xa5, 0xea, 0xed, 0x1b, 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, 0x14, 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, 0x4d, 0x81, 0xf3, 0x3b, 0x27, 0x76, 0x3f, 0x29, 0x2e, + 0xcb, 0x50, 0x19, 0x59, 0x7a, 0x98, 0x01, 0xe7, 0x4f, 0xb4, 0xbb, 0x20, 0x08, 0xd5, 0x89, 0x59, + 0x67, 0x34, 0xae, 0x8b, 0xb1, 0x5d, 0x67, 0xaa, 0xb7, 0x75, 0x53, 0x64, 0x20, 0xb0, 0xc1, 0x1c, + 0x08, 0x96, 0xe1, 0x9b, 0x17, 0xa3, 0xd4, 0x81, 0x78, 0x0f, 0x80, 0xfa, 0x58, 0xbd, 0xd3, 0xf8, + 0x55, 0x74, 0xc6, 0x26, 0xb1, 0xa2, 0x3f, 0x2e, 0x40, 0x3b, 0x42, 0xa5, 0xaf, 0xda, 0xe5, 0xcc, + 0x08, 0x44, 0x8b, 0xcf, 0x28, 0x10, 0x2d, 0xcd, 0xee, 0x66, 0x96, 0x65, 0x6e, 0xe6, 0x0f, 0x8a, + 0xd0, 0x0c, 0xa9, 0xb6, 0x6d, 0xe9, 0x76, 0x26, 0x27, 0xec, 0x40, 0xd3, 0x8b, 0x51, 0x95, 0xd3, + 0xe9, 0x55, 0x99, 0x5c, 0x65, 0x1c, 0x84, 0x96, 0x58, 0x02, 0x5d, 0xa6, 0x87, 0xee, 0xfa, 0x2c, + 0x63, 0xc8, 0x7c, 0xc6, 0x1a, 0x13, 0x60, 0x73, 0x88, 0xd1, 0x6b, 0x80, 0xb8, 0xd4, 0xf5, 0x4c, + 0xbb, 0xe7, 0xe1, 0xbe, 0x63, 0x1b, 0x4c, 0x1e, 0xcb, 0x5a, 0x8b, 0xbf, 0xe9, 0xda, 0x3b, 0x6c, + 0x1c, 0xbd, 0x09, 0x25, 0xff, 0x64, 0xc4, 0x1c, 0xc8, 0xa6, 0xd4, 0x05, 0x0b, 0xf1, 0xda, 0x3d, + 0x19, 0x61, 0x8d, 0x82, 0x07, 0x8d, 0x5d, 0xbe, 0xab, 0x1f, 0x71, 0x6f, 0xbc, 0xa4, 0x45, 0x46, + 0xa2, 0xb1, 0xf9, 0x5c, 0x3c, 0x36, 0xa7, 0x9c, 0x1d, 0x08, 0x79, 0xcf, 0xf7, 0x2d, 0x9a, 0xf3, + 0xa4, 0x9c, 0x1d, 0x8c, 0xee, 0xfa, 0x16, 0xf9, 0x48, 0xdf, 0xf1, 0x75, 0x8b, 0xc9, 0x47, 0x8d, + 0x6b, 0x13, 0x32, 0x42, 0x23, 0xdf, 0x7f, 0x25, 0xda, 0x50, 0x20, 0xa6, 0x61, 0x6f, 0x6c, 0x65, + 0xcb, 0xe3, 0xe4, 0x6c, 0xd1, 0x34, 0x51, 0xfc, 0x36, 0xd4, 0x39, 0x57, 0x9c, 0x82, 0xab, 0x80, + 0x4d, 0xd9, 0x9c, 0xc0, 0xe6, 0xe5, 0x67, 0xc4, 0xe6, 0x95, 0x33, 0xe4, 0x5b, 0xe4, 0x67, 0xa3, + 0xfe, 0x50, 0x81, 0x17, 0x52, 0x5a, 0x73, 0x22, 0x69, 0x27, 0x47, 0xe3, 0x5c, 0x9b, 0x26, 0x97, + 0xe4, 0xf6, 0xe2, 0x5d, 0xa8, 0xb8, 0x74, 0x75, 0x5e, 0xf9, 0xbb, 0x3e, 0x91, 0xf9, 0x18, 0x22, + 0x1a, 0x9f, 0xa2, 0xfe, 0xa1, 0x02, 0x17, 0xd2, 0xa8, 0xce, 0xe0, 0x04, 0xac, 0xc1, 0x1c, 0x5b, + 0x3a, 0x90, 0xd1, 0x95, 0xc9, 0x32, 0x1a, 0x12, 0x47, 0x0b, 0x26, 0xaa, 0x3b, 0xb0, 0x1c, 0xf8, + 0x0a, 0x21, 0xe9, 0xb7, 0xb0, 0xaf, 0x4f, 0x88, 0x45, 0xaf, 0x42, 0x9d, 0x05, 0x35, 0x2c, 0xc6, + 0x63, 0x85, 0x52, 0xd8, 0x13, 0xc9, 0x45, 0xf5, 0x8f, 0x0a, 0xb0, 0x44, 0x8d, 0x6d, 0xb2, 0xea, + 0x95, 0xa7, 0x0c, 0xab, 0x8a, 0x46, 0xb7, 0x47, 0xfa, 0x90, 0x37, 0xe3, 0xd4, 0xb4, 0xd8, 0x18, + 0xea, 0xa6, 0x73, 0x8f, 0xd2, 0x9c, 0x45, 0x58, 0x77, 0xde, 0xd0, 0x7d, 0x9d, 0x96, 0x9d, 0x93, + 0x49, 0xc7, 0xd0, 0xc8, 0x97, 0xce, 0x62, 0xe4, 0x5f, 0x81, 0x16, 0x4b, 0xc7, 0xf7, 0x44, 0x08, + 0x4c, 0x15, 0x53, 0x49, 0x5b, 0x60, 0xe3, 0xbb, 0xc1, 0xb0, 0xba, 0x09, 0x2f, 0x24, 0x88, 0x32, + 0xc3, 0xe1, 0xab, 0x7f, 0xa9, 0x90, 0x93, 0x8b, 0xf5, 0x3f, 0x9d, 0xdd, 0x27, 0xbe, 0x2c, 0x2a, + 0x73, 0x3d, 0xd3, 0x48, 0xea, 0x1b, 0x03, 0xbd, 0x0f, 0x35, 0x1b, 0x1f, 0xf7, 0xa2, 0x6e, 0x56, + 0x8e, 0x80, 0xa1, 0x6a, 0xe3, 0x63, 0xfa, 0x9f, 0xfa, 0x08, 0x2e, 0xa4, 0x50, 0x9d, 0xe5, 0xdb, + 0xff, 0x5e, 0x81, 0x8b, 0x1b, 0xae, 0x33, 0x7a, 0x62, 0xba, 0xfe, 0x58, 0xb7, 0xe2, 0xc5, 0xff, + 0x33, 0x7c, 0x7e, 0x8e, 0xde, 0xca, 0x0f, 0x53, 0xa1, 0xe9, 0x6b, 0x12, 0x61, 0x4b, 0x23, 0xc5, + 0x3f, 0x3a, 0xe2, 0x9e, 0xff, 0x67, 0x51, 0x86, 0x3c, 0x87, 0x9b, 0xe2, 0xc2, 0xe4, 0x89, 0x5d, + 0xa4, 0x65, 0x82, 0xe2, 0x59, 0xcb, 0x04, 0x19, 0x96, 0xa0, 0xf4, 0x8c, 0x2c, 0xc1, 0xa9, 0xf3, + 0x6a, 0xeb, 0x10, 0x2f, 0xe1, 0x50, 0x43, 0x7e, 0xda, 0xb2, 0xcf, 0x7b, 0x00, 0x61, 0x25, 0x83, + 0xf7, 0xab, 0x4e, 0x59, 0x21, 0x32, 0x81, 0x9c, 0x91, 0xb0, 0xb5, 0xdc, 0x15, 0x88, 0x64, 0xb8, + 0xbf, 0x0b, 0x1d, 0x19, 0x6f, 0xce, 0xc2, 0xef, 0xff, 0x5e, 0x00, 0xe8, 0x8a, 0xa6, 0xe8, 0xb3, + 0x19, 0x8b, 0xeb, 0x10, 0x71, 0x57, 0x42, 0x29, 0x8f, 0xf2, 0x8e, 0x41, 0x04, 0x41, 0x04, 0xb9, + 0x04, 0x26, 0x15, 0xf8, 0x1a, 0x74, 0x9d, 0x88, 0xac, 0x30, 0x56, 0x48, 0xea, 0xe7, 0x4b, 0x50, + 0x73, 0x9d, 0xe3, 0x1e, 0x11, 0x2e, 0x23, 0xe8, 0xfa, 0x76, 0x9d, 0x63, 0x22, 0x72, 0x06, 0xba, + 0x00, 0x73, 0xbe, 0xee, 0x1d, 0x92, 0xf5, 0x59, 0xae, 0xaf, 0x42, 0x1e, 0xbb, 0x06, 0x5a, 0x82, + 0xf2, 0xbe, 0x69, 0x61, 0xd6, 0x29, 0x52, 0xd3, 0xd8, 0x03, 0xfa, 0x66, 0xd0, 0x71, 0x58, 0xcd, + 0xdd, 0x59, 0xc4, 0x9a, 0x0e, 0xaf, 0xc3, 0x3c, 0xe1, 0x24, 0x82, 0x04, 0x13, 0xeb, 0x16, 0xcf, + 0xf3, 0xf3, 0x41, 0xda, 0x51, 0xf0, 0x33, 0x05, 0x16, 0x42, 0xd2, 0x52, 0xdd, 0x44, 0xd4, 0x1d, + 0x55, 0x75, 0xeb, 0x8e, 0xc1, 0xb4, 0x48, 0x33, 0xc3, 0xae, 0xb0, 0x89, 0x4c, 0xa1, 0x85, 0x53, + 0x26, 0x05, 0xe7, 0xe4, 0xe3, 0x09, 0x65, 0x4c, 0x23, 0x48, 0x17, 0x55, 0x5c, 0xe7, 0xb8, 0x6b, + 0x08, 0x92, 0xb1, 0x06, 0x6e, 0x16, 0x8a, 0x12, 0x92, 0xad, 0xd3, 0x1e, 0xee, 0xeb, 0x30, 0x8f, + 0x5d, 0xd7, 0x71, 0x7b, 0x43, 0xec, 0x79, 0xfa, 0x20, 0xe8, 0x8d, 0x68, 0xd0, 0xc1, 0x2d, 0x36, + 0xa6, 0xfe, 0x43, 0x09, 0x9a, 0xe1, 0xa7, 0x04, 0x5d, 0x0a, 0xa6, 0x11, 0x74, 0x29, 0x98, 0xe4, + 0x7c, 0xc1, 0x65, 0x5a, 0x52, 0x70, 0xc0, 0x5a, 0xa1, 0xad, 0x68, 0x35, 0x3e, 0xda, 0x35, 0x88, + 0x71, 0x27, 0x04, 0xb2, 0x1d, 0x03, 0x87, 0x1c, 0x00, 0xc1, 0x10, 0x67, 0x80, 0x18, 0x23, 0x95, + 0x72, 0x30, 0x52, 0x39, 0x07, 0x23, 0x55, 0x24, 0x8c, 0xb4, 0x0c, 0x95, 0xbd, 0x71, 0xff, 0x10, + 0xfb, 0xdc, 0xef, 0xe3, 0x4f, 0x71, 0x06, 0xab, 0x26, 0x18, 0x4c, 0xf0, 0x51, 0x2d, 0xca, 0x47, + 0x97, 0xa0, 0x16, 0x58, 0x6a, 0x8f, 0x56, 0xed, 0x8a, 0x5a, 0x95, 0x9b, 0x68, 0x0f, 0xbd, 0x15, + 0x38, 0x85, 0x75, 0x2a, 0x51, 0xaa, 0x44, 0x21, 0x25, 0xb8, 0x24, 0x70, 0x09, 0x5f, 0x86, 0x85, + 0x08, 0x39, 0x28, 0x9f, 0xb1, 0xd2, 0x5e, 0x24, 0x66, 0xa0, 0x16, 0xe4, 0x26, 0x34, 0x43, 0x92, + 0x50, 0xb8, 0x79, 0x16, 0xaa, 0x89, 0x51, 0x0a, 0x26, 0xd8, 0xbd, 0x79, 0x4a, 0x76, 0xbf, 0x08, + 0x55, 0x1e, 0x63, 0x79, 0xed, 0x85, 0x78, 0x8a, 0x24, 0x97, 0x24, 0x7c, 0x0a, 0x28, 0xfc, 0xc4, + 0xd9, 0x1c, 0xd3, 0x04, 0x0f, 0x15, 0x92, 0x3c, 0xa4, 0xfe, 0x95, 0x02, 0x8b, 0xd1, 0xcd, 0xce, + 0x6a, 0xb8, 0xdf, 0x87, 0x3a, 0x2b, 0xae, 0xf6, 0x88, 0x0a, 0x91, 0xd7, 0x2a, 0x13, 0x87, 0xa7, + 0x41, 0x78, 0xbd, 0x84, 0x10, 0xe6, 0xd8, 0x71, 0x0f, 0x4d, 0x7b, 0xd0, 0x23, 0x98, 0x89, 0x14, + 0x2e, 0x1f, 0x7c, 0x44, 0xc6, 0xd4, 0xdf, 0x51, 0xe0, 0xca, 0xe3, 0x91, 0xa1, 0xfb, 0x38, 0xe2, + 0xc1, 0xcc, 0xda, 0xae, 0x29, 0xfa, 0x25, 0x0b, 0x13, 0x8e, 0x39, 0xb2, 0x9f, 0xc7, 0xfb, 0x25, + 0x89, 0xdf, 0xc7, 0xb1, 0x49, 0x35, 0x38, 0x9f, 0x1d, 0x9b, 0x0e, 0x54, 0x8f, 0xf8, 0x72, 0xc1, + 0xcd, 0x97, 0xe0, 0x39, 0x56, 0x0c, 0x2e, 0x9e, 0xaa, 0x18, 0xac, 0x6e, 0xc1, 0x45, 0x0d, 0x7b, + 0xd8, 0x36, 0x62, 0x1f, 0x72, 0xe6, 0xa4, 0xd6, 0x08, 0x3a, 0xb2, 0xe5, 0x66, 0xe1, 0x54, 0xe6, + 0xf8, 0xf6, 0x5c, 0xb2, 0xac, 0xcf, 0x95, 0x35, 0xf1, 0xb7, 0xe8, 0x3e, 0xbe, 0xfa, 0xd7, 0x05, + 0xb8, 0x70, 0xdf, 0x30, 0xb8, 0x9e, 0xe7, 0xae, 0xdc, 0xf3, 0xf2, 0xb2, 0x93, 0x5e, 0x68, 0x31, + 0xed, 0x85, 0x3e, 0x2b, 0xdd, 0xcb, 0xad, 0x90, 0x3d, 0x1e, 0x06, 0x26, 0xd8, 0x65, 0x0d, 0x5e, + 0xef, 0xf2, 0x92, 0x69, 0xcf, 0x72, 0x06, 0xd4, 0x0c, 0x4f, 0x77, 0xce, 0xaa, 0x41, 0x72, 0x4e, + 0x1d, 0x41, 0x3b, 0x4d, 0xac, 0x19, 0xf5, 0x48, 0x40, 0x91, 0x91, 0xc3, 0x12, 0xbf, 0x0d, 0xe2, + 0x89, 0xd1, 0xa1, 0x6d, 0xc7, 0x53, 0xff, 0xab, 0x00, 0xed, 0x1d, 0xfd, 0x08, 0xff, 0xdf, 0x39, + 0xa0, 0x4f, 0x60, 0xc9, 0xd3, 0x8f, 0x70, 0x2f, 0x12, 0x80, 0xf7, 0x5c, 0xfc, 0x19, 0x77, 0x62, + 0x5f, 0x91, 0xa5, 0xe6, 0xa5, 0x1d, 0x4c, 0xda, 0xa2, 0x17, 0x1b, 0xd7, 0xf0, 0x67, 0xe8, 0x25, + 0x58, 0x88, 0x76, 0xdb, 0x11, 0xd4, 0xaa, 0x94, 0xe4, 0xf3, 0x91, 0x8e, 0xba, 0xae, 0xa1, 0x7e, + 0x06, 0x2f, 0x3e, 0xb6, 0x3d, 0xec, 0x77, 0xc3, 0xae, 0xb0, 0x19, 0xe3, 0xcf, 0xab, 0x50, 0x0f, + 0x09, 0x9f, 0xba, 0xf2, 0x62, 0x78, 0xaa, 0x03, 0x9d, 0x2d, 0xdd, 0x3d, 0x0c, 0xd2, 0xd9, 0x1b, + 0xac, 0xdb, 0xe6, 0x39, 0x6e, 0xb8, 0x2f, 0xfa, 0xce, 0x34, 0xbc, 0x8f, 0x5d, 0x6c, 0xf7, 0xf1, + 0xa6, 0xd3, 0x3f, 0x24, 0x0e, 0x89, 0xcf, 0x2e, 0x2b, 0x2a, 0x11, 0xdf, 0x75, 0x23, 0x72, 0xa9, + 0xb0, 0x10, 0xbb, 0x54, 0x38, 0xe5, 0xde, 0xad, 0xfa, 0xa3, 0x02, 0x2c, 0xdf, 0xb7, 0x7c, 0xec, + 0x86, 0x19, 0x86, 0xd3, 0x24, 0x4b, 0xc2, 0xec, 0x45, 0xe1, 0x2c, 0xd9, 0x8b, 0x1c, 0x15, 0x4c, + 0x59, 0xae, 0xa5, 0x74, 0xc6, 0x5c, 0xcb, 0x7d, 0x80, 0x91, 0xeb, 0x8c, 0xb0, 0xeb, 0x9b, 0x38, + 0x88, 0xfd, 0x72, 0x38, 0x38, 0x91, 0x49, 0xea, 0x27, 0xd0, 0x7a, 0xd8, 0x5f, 0x77, 0xec, 0x7d, + 0xd3, 0x1d, 0x06, 0x84, 0x4a, 0x09, 0x9d, 0x92, 0x43, 0xe8, 0x0a, 0x29, 0xa1, 0x53, 0x4d, 0x58, + 0x8c, 0xac, 0x3d, 0xa3, 0xe2, 0x1a, 0xf4, 0x7b, 0xfb, 0xa6, 0x6d, 0xd2, 0x6e, 0xb6, 0x02, 0x75, + 0x50, 0x61, 0xd0, 0x7f, 0xc0, 0x47, 0xd4, 0x2f, 0x15, 0xb8, 0xa4, 0x61, 0x22, 0x3c, 0x41, 0xe3, + 0xce, 0xae, 0xbf, 0xe5, 0x0d, 0x66, 0x70, 0x28, 0xee, 0x41, 0x69, 0xe8, 0x0d, 0x32, 0x8a, 0xee, + 0xc4, 0x44, 0xc7, 0x36, 0xd2, 0x28, 0xb0, 0xfa, 0x13, 0x05, 0x96, 0x82, 0xd2, 0x64, 0x4c, 0x84, + 0xe3, 0x6c, 0xab, 0xa4, 0x5a, 0xb5, 0x27, 0x5c, 0x50, 0xbe, 0x00, 0x73, 0xc6, 0x5e, 0x54, 0x41, + 0x56, 0x8c, 0x3d, 0xaa, 0x1b, 0x25, 0x9e, 0x72, 0x49, 0xea, 0x29, 0x27, 0x19, 0xbf, 0x2c, 0xe9, + 0x79, 0x7a, 0x0c, 0x6d, 0xee, 0xa0, 0x7c, 0x34, 0xc2, 0xae, 0x4e, 0xf9, 0x2b, 0x40, 0xfe, 0xed, + 0xc0, 0x85, 0x56, 0x32, 0xef, 0xf1, 0x25, 0xcb, 0x92, 0xdc, 0x89, 0x56, 0xff, 0x49, 0x81, 0x6b, + 0xc9, 0x75, 0xb7, 0x79, 0xd1, 0x6e, 0xe6, 0x9b, 0xed, 0xb4, 0xe2, 0x57, 0x08, 0x2b, 0x7e, 0x33, + 0x95, 0x2e, 0xa3, 0xd5, 0xc5, 0x52, 0xbc, 0xba, 0x78, 0xeb, 0x7d, 0xd1, 0xab, 0xbe, 0x7b, 0x32, + 0xc2, 0x68, 0x0e, 0x8a, 0x8f, 0xf0, 0x71, 0xeb, 0x1c, 0x02, 0xa8, 0x3c, 0x72, 0xdc, 0xa1, 0x6e, + 0xb5, 0x14, 0x54, 0x87, 0x39, 0x5e, 0x91, 0x6e, 0x15, 0xd0, 0x3c, 0xd4, 0xd6, 0x83, 0x2a, 0x5d, + 0xab, 0x78, 0xeb, 0x16, 0x34, 0xa2, 0x8d, 0x86, 0x64, 0xde, 0x26, 0x1e, 0xe8, 0xfd, 0x93, 0xd6, + 0x39, 0x54, 0x81, 0xc2, 0xe6, 0xdd, 0x96, 0x42, 0xff, 0xbe, 0xd1, 0x2a, 0xdc, 0xfa, 0x53, 0x05, + 0x16, 0x53, 0x48, 0xa2, 0x26, 0xc0, 0x63, 0xbb, 0xcf, 0x0b, 0xcf, 0xad, 0x73, 0xa8, 0x01, 0xd5, + 0xa0, 0x0c, 0xcd, 0xf6, 0xde, 0x75, 0x28, 0x74, 0xab, 0x80, 0x5a, 0xd0, 0x60, 0x13, 0xc7, 0xfd, + 0x3e, 0xf6, 0xbc, 0x56, 0x51, 0x8c, 0x3c, 0xd0, 0x4d, 0x6b, 0xec, 0xe2, 0x56, 0x89, 0xe0, 0xb7, + 0xeb, 0x68, 0xd8, 0xc2, 0xba, 0x87, 0x5b, 0x65, 0x84, 0xa0, 0xc9, 0x1f, 0x82, 0x49, 0x95, 0xc8, + 0x58, 0x30, 0x6d, 0xee, 0xd6, 0x8f, 0x95, 0x68, 0xd9, 0x8b, 0xd2, 0xe2, 0x02, 0x9c, 0x7f, 0x6c, + 0x1b, 0x78, 0xdf, 0xb4, 0xb1, 0x11, 0xbe, 0x6a, 0x9d, 0x43, 0xe7, 0x61, 0x61, 0x0b, 0xbb, 0x03, + 0x1c, 0x19, 0x2c, 0xa0, 0x45, 0x98, 0xdf, 0x32, 0x9f, 0x46, 0x86, 0x8a, 0x68, 0x09, 0x5a, 0x3b, + 0xa6, 0x3d, 0xb0, 0xa2, 0x80, 0x25, 0x3a, 0xdb, 0xb4, 0x1d, 0x37, 0x32, 0x58, 0xa6, 0x83, 0xfa, + 0xa7, 0xb1, 0xc1, 0x0a, 0xea, 0xc0, 0x32, 0x25, 0xea, 0xdd, 0x0d, 0x4c, 0xa8, 0x11, 0x79, 0x37, + 0xa7, 0x96, 0xaa, 0x4a, 0x4b, 0x59, 0xfd, 0xc9, 0x4d, 0xa8, 0x11, 0x61, 0x5d, 0x77, 0x1c, 0xd7, + 0x40, 0x16, 0x20, 0x7a, 0x79, 0x6d, 0x38, 0x72, 0x6c, 0x71, 0xd1, 0x15, 0xdd, 0x4e, 0xc8, 0x37, + 0x7b, 0x48, 0x03, 0x72, 0x91, 0xe8, 0xdc, 0x90, 0xc2, 0x27, 0x80, 0xd5, 0x73, 0x68, 0x48, 0x77, + 0xdb, 0x35, 0x87, 0x78, 0xd7, 0xec, 0x1f, 0x06, 0x21, 0xc0, 0xdd, 0x8c, 0xdb, 0x82, 0x69, 0xd0, + 0x60, 0xbf, 0xeb, 0xd2, 0xfd, 0xd8, 0xed, 0xc2, 0x40, 0x8e, 0xd4, 0x73, 0xe8, 0x33, 0xaa, 0x7e, + 0xc2, 0x78, 0x2a, 0xd8, 0x70, 0x35, 0x7b, 0xc3, 0x14, 0xf0, 0x29, 0xb7, 0xdc, 0x84, 0x32, 0xe5, + 0x7b, 0x24, 0xeb, 0x3c, 0x88, 0xfe, 0xf0, 0x46, 0xe7, 0x5a, 0x36, 0x80, 0x58, 0xed, 0x53, 0x58, + 0x48, 0xdc, 0x5f, 0x47, 0x32, 0x1f, 0x4c, 0xfe, 0x4b, 0x04, 0x9d, 0x5b, 0x79, 0x40, 0xc5, 0x5e, + 0x03, 0x68, 0xc6, 0x2f, 0xbd, 0xa1, 0x95, 0x1c, 0x57, 0x67, 0xd9, 0x4e, 0xaf, 0xe4, 0xbe, 0x64, + 0x4b, 0x99, 0xa0, 0x95, 0xbc, 0x59, 0x8d, 0x6e, 0x4d, 0x5c, 0x20, 0xce, 0x6c, 0xaf, 0xe6, 0x82, + 0x15, 0xdb, 0x9d, 0x50, 0x26, 0x48, 0x5d, 0x6b, 0x4d, 0xf2, 0x78, 0xb0, 0x4c, 0xd6, 0x7d, 0xdb, + 0xce, 0x9d, 0xdc, 0xf0, 0x62, 0xeb, 0x5f, 0x65, 0xed, 0x85, 0xb2, 0xab, 0xa1, 0xe8, 0x0d, 0xf9, + 0x72, 0x13, 0xee, 0xb4, 0x76, 0x56, 0x4f, 0x33, 0x45, 0x20, 0xf1, 0x7d, 0xda, 0x17, 0x28, 0xb9, + 0x5c, 0x99, 0x94, 0xbb, 0x60, 0xbd, 0xec, 0x7b, 0xa3, 0x9d, 0x37, 0x4e, 0x31, 0x43, 0x20, 0xe0, + 0x24, 0xaf, 0xae, 0x07, 0x62, 0x78, 0x67, 0x2a, 0xd7, 0x9c, 0x4d, 0x06, 0xbf, 0x07, 0x0b, 0x89, + 0xa8, 0x04, 0xe5, 0x8f, 0x5c, 0x3a, 0x93, 0xcc, 0x2d, 0x13, 0xc9, 0x44, 0x1f, 0x20, 0xca, 0xe0, + 0x7e, 0x49, 0xaf, 0x60, 0xe7, 0x56, 0x1e, 0x50, 0xf1, 0x21, 0x23, 0x58, 0x4c, 0xbc, 0x7c, 0xb2, + 0x8a, 0x5e, 0xcd, 0xbd, 0xdb, 0x93, 0xd5, 0xce, 0x6b, 0xf9, 0xf7, 0x7b, 0xb2, 0xaa, 0x9e, 0x43, + 0x1e, 0x55, 0xd0, 0x89, 0x5e, 0x32, 0x94, 0xb1, 0x8a, 0xbc, 0x67, 0xae, 0xf3, 0x7a, 0x4e, 0x68, + 0xf1, 0x99, 0x47, 0x70, 0x5e, 0xd2, 0xf2, 0x87, 0x5e, 0x9f, 0xc8, 0x1e, 0xc9, 0x5e, 0xc7, 0xce, + 0xed, 0xbc, 0xe0, 0x11, 0xf3, 0xd0, 0x0a, 0xf0, 0xba, 0x6f, 0x59, 0xcc, 0xb3, 0x78, 0x2d, 0xcb, + 0xf2, 0xc5, 0xc0, 0x32, 0x3e, 0x35, 0x13, 0x5a, 0x6c, 0xf9, 0xcb, 0x80, 0x76, 0x0e, 0x9c, 0x63, + 0x1a, 0x05, 0x0c, 0xc6, 0xdc, 0xb1, 0xcc, 0x34, 0x80, 0x69, 0xd0, 0x0c, 0x41, 0x9c, 0x38, 0x43, + 0x6c, 0xde, 0x03, 0x78, 0x88, 0xfd, 0x2d, 0xec, 0xbb, 0x44, 0xfa, 0x5f, 0xca, 0xc2, 0x9d, 0x03, + 0x04, 0x5b, 0xbd, 0x3c, 0x15, 0x2e, 0x4a, 0xd0, 0x2d, 0xdd, 0x1e, 0xeb, 0x56, 0xe4, 0xe6, 0x98, + 0x9c, 0xa0, 0x49, 0xb0, 0xc9, 0x04, 0x4d, 0x43, 0x8b, 0x2d, 0x8f, 0x85, 0xff, 0x12, 0xe9, 0x42, + 0x98, 0xec, 0xbf, 0xa4, 0x3b, 0xe0, 0x92, 0xba, 0x7d, 0x02, 0xbc, 0xd8, 0xf8, 0x0b, 0x85, 0x36, + 0xaa, 0x26, 0x00, 0x3e, 0x36, 0xfd, 0x83, 0x6d, 0x4b, 0xb7, 0xbd, 0x3c, 0x28, 0x50, 0xc0, 0x53, + 0xa0, 0xc0, 0xe1, 0x05, 0x0a, 0x06, 0xcc, 0xc7, 0x2a, 0xfe, 0x48, 0x76, 0x35, 0x4a, 0xd6, 0x28, + 0xd1, 0x59, 0x99, 0x0e, 0x28, 0x76, 0xd9, 0x87, 0xf9, 0x58, 0x0c, 0x27, 0xdd, 0x45, 0x16, 0xe5, + 0x25, 0x95, 0x5d, 0x42, 0x3a, 0x92, 0x04, 0xf5, 0x00, 0xa5, 0x0b, 0x9b, 0x28, 0x5f, 0x19, 0x7c, + 0x92, 0xea, 0xc9, 0xae, 0x96, 0x32, 0x6d, 0x9e, 0x68, 0x1d, 0x90, 0x9b, 0x0a, 0x69, 0x27, 0x84, + 0x54, 0x9b, 0x67, 0x74, 0x22, 0xa8, 0xe7, 0xd0, 0xc7, 0x50, 0xe1, 0x3f, 0x4c, 0x75, 0x63, 0x72, + 0x09, 0x81, 0xaf, 0x7e, 0x73, 0x0a, 0x94, 0x58, 0xf8, 0x10, 0x2e, 0x64, 0x14, 0x10, 0xa4, 0x5e, + 0xc6, 0xe4, 0x62, 0xc3, 0x34, 0xfb, 0x27, 0x36, 0x4b, 0xd5, 0x07, 0x26, 0x6c, 0x96, 0x55, 0x4b, + 0x98, 0xb6, 0x59, 0x0f, 0x16, 0x53, 0xf9, 0x57, 0xa9, 0x01, 0xcc, 0xca, 0xd2, 0x4e, 0xdb, 0x60, + 0x00, 0x2f, 0x48, 0x73, 0x8d, 0x52, 0xdf, 0x64, 0x52, 0x56, 0x72, 0xda, 0x46, 0x7d, 0x38, 0x2f, + 0xc9, 0x30, 0x4a, 0x6d, 0x5c, 0x76, 0x26, 0x72, 0xda, 0x26, 0xfb, 0xd0, 0x59, 0x73, 0x1d, 0xdd, + 0xe8, 0xeb, 0x9e, 0x4f, 0xb3, 0x7e, 0x24, 0x08, 0x0d, 0x9c, 0x43, 0x79, 0xe4, 0x20, 0xcd, 0x0d, + 0x4e, 0xdb, 0x67, 0x0f, 0xea, 0xf4, 0x28, 0xd9, 0xaf, 0x00, 0x21, 0xb9, 0x85, 0x88, 0x40, 0x64, + 0xa8, 0x1d, 0x19, 0xa0, 0x60, 0xea, 0x5d, 0xa8, 0xaf, 0xd3, 0xf2, 0x69, 0xd7, 0x36, 0xf0, 0xd3, + 0xa4, 0xb5, 0xa2, 0x3f, 0x85, 0x70, 0x3b, 0x02, 0x90, 0x9b, 0x42, 0xf3, 0xd4, 0x67, 0x37, 0xf0, + 0x53, 0x76, 0xce, 0x2b, 0xb2, 0x75, 0x63, 0x20, 0x19, 0x31, 0x8e, 0x14, 0x32, 0x62, 0xe7, 0x97, + 0xa2, 0x9e, 0xac, 0xd8, 0xee, 0x4e, 0xc6, 0x22, 0x29, 0xc8, 0x60, 0xd7, 0xbb, 0xf9, 0x27, 0x44, + 0xed, 0x42, 0x80, 0x57, 0x97, 0xd6, 0x6e, 0x5f, 0x9e, 0x84, 0x7a, 0xd4, 0x3d, 0x5d, 0x99, 0x0e, + 0x28, 0x76, 0xd9, 0x86, 0x1a, 0xe1, 0x4e, 0x76, 0x3c, 0x37, 0x64, 0x13, 0xc5, 0xeb, 0xfc, 0x87, + 0xb3, 0x81, 0xbd, 0xbe, 0x6b, 0xee, 0xf1, 0x43, 0x97, 0xa2, 0x13, 0x03, 0x99, 0x78, 0x38, 0x09, + 0x48, 0x81, 0xf9, 0x98, 0xfa, 0x0c, 0x82, 0x74, 0x5c, 0x55, 0xbe, 0x3e, 0xed, 0x7c, 0xe3, 0x6a, + 0xf2, 0x76, 0x5e, 0x70, 0xb1, 0xed, 0xaf, 0xd0, 0x38, 0x88, 0xbe, 0x5f, 0x1b, 0x9b, 0x96, 0x11, + 0x24, 0xfe, 0xd0, 0xdd, 0x49, 0x4b, 0xc5, 0x40, 0x33, 0xdd, 0xbf, 0x09, 0x33, 0xc4, 0xfe, 0xbf, + 0x08, 0x35, 0x91, 0x7f, 0x46, 0xb2, 0xac, 0x65, 0x32, 0xf3, 0xdd, 0xb9, 0x31, 0x19, 0x48, 0xac, + 0x8c, 0x61, 0x49, 0x96, 0x6d, 0x96, 0x86, 0xd8, 0x13, 0xd2, 0xd2, 0x53, 0xf8, 0x63, 0xf5, 0xcb, + 0x06, 0x54, 0x83, 0x89, 0x5f, 0x71, 0xe2, 0xea, 0x6b, 0xc8, 0x24, 0x7d, 0x0f, 0x16, 0x12, 0x3f, + 0xee, 0x22, 0xd5, 0xe0, 0xf2, 0x1f, 0x80, 0x99, 0x26, 0x6a, 0x1f, 0xf3, 0x9f, 0x53, 0x15, 0x21, + 0xde, 0xcb, 0x59, 0xd9, 0xa8, 0x64, 0x74, 0x37, 0x65, 0xe1, 0xff, 0xdd, 0x01, 0xce, 0x23, 0x80, + 0x48, 0x68, 0x33, 0xf9, 0x6e, 0x00, 0xf1, 0xd6, 0xa7, 0x51, 0x6b, 0x28, 0x8d, 0x5e, 0x5e, 0xc9, + 0xd3, 0x67, 0x9d, 0xed, 0x81, 0x66, 0xc7, 0x2c, 0x8f, 0xa1, 0x11, 0xbd, 0xb5, 0x83, 0xa4, 0xbf, + 0x74, 0x99, 0xbe, 0xd6, 0x33, 0xed, 0x2b, 0xb6, 0x4e, 0xe9, 0xd8, 0x4e, 0x59, 0xce, 0x03, 0x94, + 0xee, 0xc3, 0x90, 0x06, 0x02, 0x99, 0xdd, 0x1f, 0xd2, 0x40, 0x20, 0xbb, 0xb9, 0x83, 0x25, 0x25, + 0x93, 0xcd, 0x05, 0xd2, 0xa4, 0x64, 0x46, 0xbb, 0x86, 0x34, 0x29, 0x99, 0xd5, 0xad, 0x10, 0x91, + 0xbf, 0x89, 0xa1, 0x9b, 0xec, 0xd7, 0x7e, 0xa7, 0x11, 0xcf, 0x80, 0xe5, 0x47, 0x8e, 0x6f, 0xee, + 0x9f, 0x24, 0xcb, 0x4c, 0x52, 0xb7, 0x39, 0xab, 0xc6, 0x35, 0x5d, 0xca, 0x2f, 0x53, 0xaf, 0x2d, + 0xab, 0x96, 0x85, 0xf2, 0x14, 0xc5, 0x3a, 0xf7, 0x72, 0x60, 0x94, 0xb6, 0x63, 0x6b, 0xf7, 0x3e, + 0x79, 0x63, 0x60, 0xfa, 0x07, 0xe3, 0x3d, 0x82, 0xd6, 0x1d, 0xb6, 0xc4, 0xeb, 0xa6, 0xc3, 0xff, + 0xbb, 0x13, 0xa8, 0x8a, 0x3b, 0x74, 0xd5, 0x3b, 0x64, 0xd5, 0xd1, 0xde, 0x5e, 0x85, 0x3e, 0xdd, + 0xfb, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x09, 0x50, 0xd1, 0x55, 0x86, 0x5b, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used.