From 0c39e0370cc1415b95adc4f29b44c61bcfe445f4 Mon Sep 17 00:00:00 2001 From: Bingyi Sun Date: Wed, 15 Dec 2021 10:53:16 +0800 Subject: [PATCH] Move segment statistics update to datanode tt loop (#13233) Signed-off-by: sunby Co-authored-by: sunby --- internal/datacoord/param_table.go | 11 - internal/datacoord/param_table_test.go | 3 - .../datacoord/segment_allocation_policy.go | 3 +- internal/datacoord/segment_manager_test.go | 1 + internal/datacoord/server.go | 196 ++++---- internal/datacoord/server_test.go | 69 +-- internal/datacoord/services.go | 4 +- internal/datanode/data_node.go | 1 - internal/datanode/data_node_test.go | 1 - .../datanode/flow_graph_insert_buffer_node.go | 102 +--- .../flow_graph_insert_buffer_node_test.go | 9 +- internal/datanode/flow_graph_time_ticker.go | 57 ++- .../datanode/flow_graph_time_ticker_test.go | 6 +- internal/datanode/param_table.go | 13 - internal/datanode/param_table_test.go | 6 - internal/datanode/segment_replica.go | 8 +- internal/msgstream/msg.go | 63 --- internal/msgstream/msg_test.go | 44 -- internal/msgstream/unmarshal.go | 2 - internal/proto/data_coord.proto | 6 + internal/proto/datapb/data_coord.pb.go | 444 ++++++++++-------- internal/proto/internal.proto | 15 - internal/proto/internalpb/internal.pb.go | 395 +++++----------- 23 files changed, 558 insertions(+), 901 deletions(-) diff --git a/internal/datacoord/param_table.go b/internal/datacoord/param_table.go index af39206a04..f1831212d9 100644 --- a/internal/datacoord/param_table.go +++ b/internal/datacoord/param_table.go @@ -69,7 +69,6 @@ type ParamTable struct { // --- Channels --- ClusterChannelPrefix string InsertChannelPrefixName string - StatisticsChannelName string TimeTickChannelName string SegmentInfoChannelName string DataCoordSubscriptionName string @@ -117,7 +116,6 @@ func (p *ParamTable) Init() { // Has to init global msgchannel prefix before other channel names p.initClusterMsgChannelPrefix() p.initInsertChannelPrefixName() - p.initStatisticsChannelName() p.initTimeTickChannelName() p.initSegmentInfoChannelName() p.initDataCoordSubscriptionName() @@ -244,15 +242,6 @@ func (p *ParamTable) initInsertChannelPrefixName() { p.InsertChannelPrefixName = strings.Join(s, "-") } -func (p *ParamTable) initStatisticsChannelName() { - config, err := p.Load("msgChannel.chanNamePrefix.dataCoordStatistic") - if err != nil { - panic(err) - } - s := []string{p.ClusterChannelPrefix, config} - p.StatisticsChannelName = strings.Join(s, "-") -} - func (p *ParamTable) initTimeTickChannelName() { config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick") if err != nil { diff --git a/internal/datacoord/param_table_test.go b/internal/datacoord/param_table_test.go index 7d5313129a..23f00975ed 100644 --- a/internal/datacoord/param_table_test.go +++ b/internal/datacoord/param_table_test.go @@ -29,9 +29,6 @@ func TestParamTable(t *testing.T) { assert.Equal(t, Params.InsertChannelPrefixName, "by-dev-insert-channel-") t.Logf("data coord insert channel = %s", Params.InsertChannelPrefixName) - assert.Equal(t, Params.StatisticsChannelName, "by-dev-datacoord-statistics-channel") - t.Logf("data coord stats channel = %s", Params.StatisticsChannelName) - assert.Equal(t, Params.TimeTickChannelName, "by-dev-datacoord-timetick-channel") t.Logf("data coord timetick channel = %s", Params.TimeTickChannelName) diff --git a/internal/datacoord/segment_allocation_policy.go b/internal/datacoord/segment_allocation_policy.go index 66d44c1681..8ff76badd7 100644 --- a/internal/datacoord/segment_allocation_policy.go +++ b/internal/datacoord/segment_allocation_policy.go @@ -142,5 +142,6 @@ const flushInterval = 2 * time.Second func flushPolicyV1(segment *SegmentInfo, t Timestamp) bool { return segment.GetState() == commonpb.SegmentState_Sealed && segment.GetLastExpireTime() <= t && - time.Since(segment.lastFlushTime) >= flushInterval + time.Since(segment.lastFlushTime) >= flushInterval && + segment.currRows != 0 } diff --git a/internal/datacoord/segment_manager_test.go b/internal/datacoord/segment_manager_test.go index 91aaee18aa..8ef7c3e6c5 100644 --- a/internal/datacoord/segment_manager_test.go +++ b/internal/datacoord/segment_manager_test.go @@ -302,6 +302,7 @@ func TestGetFlushableSegments(t *testing.T) { assert.EqualValues(t, 1, len(ids)) assert.EqualValues(t, allocations[0].SegmentID, ids[0]) + meta.SetCurrentRows(allocations[0].SegmentID, 1) ids, err = segmentManager.GetFlushableSegments(context.TODO(), "c1", allocations[0].ExpireTime) assert.Nil(t, err) assert.EqualValues(t, 1, len(ids)) diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index db82b4ddec..09eae3a2db 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -399,8 +399,7 @@ func (s *Server) initMeta() error { func (s *Server) startServerLoop() { s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx) - s.serverLoopWg.Add(4) - s.startStatsChannel(s.serverLoopCtx) + s.serverLoopWg.Add(3) s.startDataNodeTtLoop(s.serverLoopCtx) s.startWatchService(s.serverLoopCtx) s.startFlushLoop(s.serverLoopCtx) @@ -415,44 +414,6 @@ func (s *Server) startServerLoop() { }) } -func (s *Server) startStatsChannel(ctx context.Context) { - statsStream, _ := s.msFactory.NewMsgStream(ctx) - statsStream.AsConsumer([]string{Params.StatisticsChannelName}, Params.DataCoordSubscriptionName) - log.Debug("DataCoord creates statistics channel consumer", - zap.String("channel", Params.StatisticsChannelName), - zap.String("description", Params.DataCoordSubscriptionName)) - statsStream.Start() - go func() { - defer logutil.LogPanic() - defer s.serverLoopWg.Done() - defer statsStream.Close() - for { - select { - case <-ctx.Done(): - log.Debug("statistics channel shutdown") - return - default: - } - msgPack := statsStream.Consume() - if msgPack == nil { - log.Debug("receive nil stats msg, shutdown stats channel") - return - } - for _, msg := range msgPack.Msgs { - if msg.Type() != commonpb.MsgType_SegmentStatistics { - log.Warn("receive unknown msg from segment statistics channel", - zap.Stringer("msgType", msg.Type())) - continue - } - ssMsg := msg.(*msgstream.SegmentStatisticsMsg) - for _, stat := range ssMsg.SegStats { - s.meta.SetCurrentRows(stat.GetSegmentID(), stat.GetNumRows()) - } - } - } - }() -} - // startDataNodeTtLoop start a goroutine to recv data node tt msg from msgstream // tt msg stands for the currently consumed timestamp for each channel func (s *Server) startDataNodeTtLoop(ctx context.Context) { @@ -475,6 +436,7 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) { checker.Start() defer checker.Stop() } + defer logutil.LogPanic() defer s.serverLoopWg.Done() defer ttMsgStream.Close() @@ -491,75 +453,119 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) { return } for _, msg := range msgPack.Msgs { - if msg.Type() != commonpb.MsgType_DataNodeTt { - log.Warn("receive unexpected msg type from tt channel", - zap.Stringer("msgType", msg.Type())) + ttMsg, ok := msg.(*msgstream.DataNodeTtMsg) + if !ok { + log.Warn("receive unexpected msg type from tt channel") continue } - ttMsg := msg.(*msgstream.DataNodeTtMsg) if enableTtChecker { checker.Check() } - ch := ttMsg.ChannelName - ts := ttMsg.Timestamp - if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil { - log.Warn("failed to expire allocations", zap.Error(err)) + if err := s.handleTimetickMessage(ctx, ttMsg); err != nil { + log.Error("failed to handle timetick message", zap.Error(err)) continue } - physical, _ := tsoutil.ParseTS(ts) - if time.Since(physical).Minutes() > 1 { - // if lag behind, log every 1 mins about - log.RatedWarn(60.0, "Time tick lag behind for more than 1 minutes", zap.String("channel", ch), zap.Time("tt", physical)) - } - segments, err := s.segmentManager.GetFlushableSegments(ctx, ch, ts) - if err != nil { - log.Warn("get flushable segments failed", zap.Error(err)) - continue - } - - staleSegments := s.meta.SelectSegments(func(info *SegmentInfo) bool { - return isSegmentHealthy(info) && - info.GetInsertChannel() == ch && - !info.lastFlushTime.IsZero() && - time.Since(info.lastFlushTime).Minutes() >= segmentTimedFlushDuration - }) - - if len(segments)+len(staleSegments) == 0 { - continue - } - log.Debug("flush segments", zap.Int64s("segmentIDs", segments), zap.Int("markSegments count", len(staleSegments))) - segmentInfos := make([]*datapb.SegmentInfo, 0, len(segments)) - for _, id := range segments { - sInfo := s.meta.GetSegment(id) - if sInfo == nil { - log.Error("get segment from meta error", zap.Int64("id", id), - zap.Error(err)) - continue - } - segmentInfos = append(segmentInfos, sInfo.SegmentInfo) - s.meta.SetLastFlushTime(id, time.Now()) - } - markSegments := make([]*datapb.SegmentInfo, 0, len(staleSegments)) - for _, segment := range staleSegments { - for _, fSeg := range segmentInfos { - // check segment needs flush first - if segment.GetID() == fSeg.GetID() { - continue - } - } - markSegments = append(markSegments, segment.SegmentInfo) - s.meta.SetLastFlushTime(segment.GetID(), time.Now()) - } - if len(segmentInfos)+len(markSegments) > 0 { - s.cluster.Flush(s.ctx, segmentInfos, markSegments) - } } s.helper.eventAfterHandleDataNodeTt() } }() } +func (s *Server) handleTimetickMessage(ctx context.Context, ttMsg *msgstream.DataNodeTtMsg) error { + ch := ttMsg.GetChannelName() + ts := ttMsg.GetTimestamp() + physical, _ := tsoutil.ParseTS(ts) + if time.Since(physical).Minutes() > 1 { + // if lag behind, log every 1 mins about + log.RatedWarn(60.0, "time tick lag behind for more than 1 minutes", zap.String("channel", ch), zap.Time("timetick", physical)) + } + + s.updateSegmentStatistics(ttMsg.GetSegmentsStats()) + + if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil { + return fmt.Errorf("expire allocations: %w", err) + } + + flushableIDs, err := s.segmentManager.GetFlushableSegments(ctx, ch, ts) + if err != nil { + return fmt.Errorf("get flushable segments: %w", err) + } + flushableSegments := s.getFlushableSegmentsInfo(flushableIDs) + + staleSegments := s.getStaleSegmentsInfo(ch) + staleSegments = s.filterWithFlushableSegments(staleSegments, flushableIDs) + + if len(flushableSegments)+len(staleSegments) == 0 { + return nil + } + + log.Debug("flush segments", zap.Int64s("segmentIDs", flushableIDs), zap.Int("markSegments count", len(staleSegments))) + + s.setLastFlushTime(flushableSegments) + s.setLastFlushTime(staleSegments) + + finfo, minfo := make([]*datapb.SegmentInfo, 0, len(flushableSegments)), make([]*datapb.SegmentInfo, 0, len(staleSegments)) + for _, info := range flushableSegments { + finfo = append(finfo, info.SegmentInfo) + } + for _, info := range staleSegments { + minfo = append(minfo, info.SegmentInfo) + } + s.cluster.Flush(s.ctx, finfo, minfo) + return nil +} + +func (s *Server) updateSegmentStatistics(stats []*datapb.SegmentStats) { + for _, stat := range stats { + s.meta.SetCurrentRows(stat.GetSegmentID(), stat.GetNumRows()) + } +} + +func (s *Server) getFlushableSegmentsInfo(flushableIDs []int64) []*SegmentInfo { + res := make([]*SegmentInfo, 0, len(flushableIDs)) + for _, id := range flushableIDs { + sinfo := s.meta.GetSegment(id) + if sinfo == nil { + log.Error("get segment from meta error", zap.Int64("id", id)) + continue + } + res = append(res, sinfo) + } + return res +} + +func (s *Server) getStaleSegmentsInfo(ch string) []*SegmentInfo { + return s.meta.SelectSegments(func(info *SegmentInfo) bool { + return isSegmentHealthy(info) && + info.GetInsertChannel() == ch && + !info.lastFlushTime.IsZero() && + time.Since(info.lastFlushTime).Minutes() >= segmentTimedFlushDuration + }) +} + +func (s *Server) filterWithFlushableSegments(staleSegments []*SegmentInfo, flushableIDs []int64) []*SegmentInfo { + filter := map[int64]struct{}{} + for _, sid := range flushableIDs { + filter[sid] = struct{}{} + } + + res := make([]*SegmentInfo, 0, len(staleSegments)) + for _, sinfo := range staleSegments { + if _, ok := filter[sinfo.GetID()]; ok { + continue + } + res = append(res, sinfo) + } + return res +} + +func (s *Server) setLastFlushTime(segments []*SegmentInfo) { + for _, sinfo := range segments { + s.meta.SetLastFlushTime(sinfo.GetID(), time.Now()) + } +} + // start a goroutine wto watch services func (s *Server) startWatchService(ctx context.Context) { go s.watchService(ctx) diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 6f013643ed..334c1b6034 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -194,6 +194,8 @@ func TestFlush(t *testing.T) { resp, err := svr.Flush(context.TODO(), req) assert.Nil(t, err) assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) + + svr.meta.SetCurrentRows(segID, 1) ids, err := svr.segmentManager.GetFlushableSegments(context.TODO(), "channel-1", expireTs) assert.Nil(t, err) assert.EqualValues(t, 1, len(ids)) @@ -250,15 +252,6 @@ func TestGetTimeTickChannel(t *testing.T) { assert.EqualValues(t, Params.TimeTickChannelName, resp.Value) } -func TestGetStatisticsChannel(t *testing.T) { - svr := newTestServer(t, nil) - defer closeTestServer(t, svr) - resp, err := svr.GetStatisticsChannel(context.TODO()) - assert.Nil(t, err) - assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) - assert.EqualValues(t, Params.StatisticsChannelName, resp.Value) -} - func TestGetSegmentStates(t *testing.T) { t.Run("normal cases", func(t *testing.T) { svr := newTestServer(t, nil) @@ -720,55 +713,6 @@ func TestServer_getSystemInfoMetrics(t *testing.T) { } } -func TestChannel(t *testing.T) { - svr := newTestServer(t, nil) - defer closeTestServer(t, svr) - - t.Run("Test StatsChannel", func(t *testing.T) { - const segID = 0 - const rowNum = int64(100) - - segInfo := &datapb.SegmentInfo{ - ID: segID, - } - err := svr.meta.AddSegment(NewSegmentInfo(segInfo)) - assert.Nil(t, err) - - stats := &internalpb.SegmentStatisticsUpdates{ - SegmentID: segID, - NumRows: rowNum, - } - genMsg := func(msgType commonpb.MsgType, t Timestamp) *msgstream.SegmentStatisticsMsg { - return &msgstream.SegmentStatisticsMsg{ - BaseMsg: msgstream.BaseMsg{ - HashValues: []uint32{0}, - }, - SegmentStatistics: internalpb.SegmentStatistics{ - Base: &commonpb.MsgBase{ - MsgType: msgType, - MsgID: 0, - Timestamp: t, - SourceID: 0, - }, - SegStats: []*internalpb.SegmentStatisticsUpdates{stats}, - }, - } - } - - statsStream, _ := svr.msFactory.NewMsgStream(svr.ctx) - statsStream.AsProducer([]string{Params.StatisticsChannelName}) - statsStream.Start() - defer statsStream.Close() - - msgPack := msgstream.MsgPack{} - msgPack.Msgs = append(msgPack.Msgs, genMsg(commonpb.MsgType_SegmentStatistics, 123)) - msgPack.Msgs = append(msgPack.Msgs, genMsg(commonpb.MsgType_SegmentInfo, 234)) - msgPack.Msgs = append(msgPack.Msgs, genMsg(commonpb.MsgType_SegmentStatistics, 345)) - err = statsStream.Produce(&msgPack) - assert.Nil(t, err) - }) -} - type spySegmentManager struct { spyCh chan struct{} } @@ -1143,6 +1087,10 @@ func TestDataNodeTtChannel(t *testing.T) { msgPack := msgstream.MsgPack{} msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime) + msg.SegmentsStats = append(msg.SegmentsStats, &datapb.SegmentStats{ + SegmentID: assign.GetSegID(), + NumRows: 1, + }) msgPack.Msgs = append(msgPack.Msgs, msg) err = ttMsgStream.Produce(&msgPack) assert.Nil(t, err) @@ -1217,6 +1165,10 @@ func TestDataNodeTtChannel(t *testing.T) { msgPack := msgstream.MsgPack{} msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime) + msg.SegmentsStats = append(msg.SegmentsStats, &datapb.SegmentStats{ + SegmentID: assign.GetSegID(), + NumRows: 1, + }) msgPack.Msgs = append(msgPack.Msgs, msg) err = ttMsgStream.Produce(&msgPack) assert.Nil(t, err) @@ -2250,7 +2202,6 @@ func TestGetFlushState(t *testing.T) { func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Server { Params.Init() Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int()) - Params.StatisticsChannelName = Params.StatisticsChannelName + strconv.Itoa(rand.Int()) var err error factory := msgstream.NewPmsFactory() m := map[string]interface{}{ diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index ff7cd05320..10c972f2e0 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -55,9 +55,9 @@ func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringRespon func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) { return &milvuspb.StringResponse{ Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_Success, + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "no statistics channel", }, - Value: Params.StatisticsChannelName, }, nil } diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 1eea5f983a..0e607f8142 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -204,7 +204,6 @@ func (node *DataNode) Register() error { // Init function does nothing now. func (node *DataNode) Init() error { log.Debug("DataNode Init", - zap.String("SegmentStatisticsChannelName", Params.SegmentStatisticsChannelName), zap.String("TimeTickChannelName", Params.TimeTickChannelName), ) diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 1976c47ba7..7ae32a9f51 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -55,7 +55,6 @@ func TestMain(t *testing.M) { Params.Init() // change to specific channel for test Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int()) - Params.SegmentStatisticsChannelName = Params.SegmentStatisticsChannelName + strconv.Itoa(rand.Int()) code := t.Run() os.Exit(code) } diff --git a/internal/datanode/flow_graph_insert_buffer_node.go b/internal/datanode/flow_graph_insert_buffer_node.go index 6df4a5a971..80c6100ba9 100644 --- a/internal/datanode/flow_graph_insert_buffer_node.go +++ b/internal/datanode/flow_graph_insert_buffer_node.go @@ -63,10 +63,9 @@ type insertBufferNode struct { flushingSegCache *Cache flushManager flushManager - timeTickStream msgstream.MsgStream - segmentStatisticsStream msgstream.MsgStream - ttLogger timeTickLogger - ttMerger *mergedTimeTickerSender + timeTickStream msgstream.MsgStream + ttLogger timeTickLogger + ttMerger *mergedTimeTickerSender } type timeTickLogger struct { @@ -158,10 +157,6 @@ func (ibNode *insertBufferNode) Close() { if ibNode.timeTickStream != nil { ibNode.timeTickStream.Close() } - - if ibNode.segmentStatisticsStream != nil { - ibNode.segmentStatisticsStream.Close() - } } func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { @@ -211,14 +206,6 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { return []Msg{} } - if len(seg2Upload) > 0 { - log.Debug("flowgraph insert buffer node consumed msgs with end position", zap.String("channel", ibNode.channelName), zap.Any("end position", endPositions[0])) - err := ibNode.uploadMemStates2Coord(seg2Upload) - if err != nil { - log.Error("upload segment statistics to coord error", zap.Error(err)) - } - } - // insert messages -> buffer for _, msg := range fgMsg.insertMessages { err := ibNode.bufferInsertMsg(msg, endPositions[0]) @@ -357,7 +344,7 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { } } - if err := ibNode.writeHardTimeTick(fgMsg.timeRange.timestampMax); err != nil { + if err := ibNode.writeHardTimeTick(fgMsg.timeRange.timestampMax, seg2Upload); err != nil { log.Error("send hard time tick into pulsar channel failed", zap.Error(err)) } @@ -709,58 +696,11 @@ func readBinary(reader io.Reader, receiver interface{}, dataType schemapb.DataTy } // writeHardTimeTick writes timetick once insertBufferNode operates. -func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error { +func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp, segmentIDs []int64) error { ibNode.ttLogger.LogTs(ts) - ibNode.ttMerger.bufferTs(ts) + ibNode.ttMerger.bufferTs(ts, segmentIDs) return nil } - -// uploadMemStates2Coord uploads latest changed segments statistics in DataNode memory to DataCoord -// through a msgStream channel. -// -// Currently, the statistics includes segment ID and its total number of rows in memory. -func (ibNode *insertBufferNode) uploadMemStates2Coord(segIDs []UniqueID) error { - statsUpdates := make([]*internalpb.SegmentStatisticsUpdates, 0, len(segIDs)) - for _, segID := range segIDs { - updates, err := ibNode.replica.getSegmentStatisticsUpdates(segID) - if err != nil { - log.Error("get segment statistics updates wrong", zap.Int64("segmentID", segID), zap.Error(err)) - continue - } - - log.Debug("Segment Statistics to Update", - zap.Int64("segment ID", updates.GetSegmentID()), - zap.Int64("collection ID", ibNode.replica.getCollectionID()), - zap.String("vchannel name", ibNode.channelName), - zap.Int64("numOfRows", updates.GetNumRows()), - ) - - statsUpdates = append(statsUpdates, updates) - } - - segStats := internalpb.SegmentStatistics{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_SegmentStatistics, - MsgID: UniqueID(0), // GOOSE TODO - Timestamp: Timestamp(0), // GOOSE TODO - SourceID: Params.NodeID, - }, - SegStats: statsUpdates, - } - - var msg msgstream.TsMsg = &msgstream.SegmentStatisticsMsg{ - BaseMsg: msgstream.BaseMsg{ - HashValues: []uint32{0}, // GOOSE TODO - }, - SegmentStatistics: segStats, - } - - var msgPack = msgstream.MsgPack{ - Msgs: []msgstream.TsMsg{msg}, - } - return ibNode.segmentStatisticsStream.Produce(&msgPack) -} - func (ibNode *insertBufferNode) getCollectionandPartitionIDbySegID(segmentID UniqueID) (collID, partitionID UniqueID, err error) { return ibNode.replica.getCollectionAndPartitionID(segmentID) } @@ -782,17 +722,16 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM var wTtMsgStream msgstream.MsgStream = wTt wTtMsgStream.Start() - // update statistics channel - segS, err := config.msFactory.NewMsgStream(ctx) - if err != nil { - return nil, err - } - segS.AsProducer([]string{Params.SegmentStatisticsChannelName}) - log.Debug("datanode AsProducer", zap.String("SegmentStatisChannelName", Params.SegmentStatisticsChannelName)) - var segStatisticsMsgStream msgstream.MsgStream = segS - segStatisticsMsgStream.Start() - - mt := newMergedTimeTickerSender(func(ts Timestamp) error { + mt := newMergedTimeTickerSender(func(ts Timestamp, segmentIDs []int64) error { + stats := make([]*datapb.SegmentStats, 0, len(segmentIDs)) + for _, sid := range segmentIDs { + stat, err := config.replica.getSegmentStatisticsUpdates(sid) + if err != nil { + log.Warn("failed to get segment statistics info", zap.Int64("segmentID", sid), zap.Error(err)) + continue + } + stats = append(stats, stat) + } msgPack := msgstream.MsgPack{} timeTickMsg := msgstream.DataNodeTtMsg{ BaseMsg: msgstream.BaseMsg{ @@ -806,8 +745,9 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM MsgID: 0, Timestamp: ts, }, - ChannelName: config.vChannelName, - Timestamp: ts, + ChannelName: config.vChannelName, + Timestamp: ts, + SegmentsStats: stats, }, } msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg) @@ -818,9 +758,7 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM BaseNode: baseNode, insertBuffer: sync.Map{}, - timeTickStream: wTtMsgStream, - segmentStatisticsStream: segStatisticsMsgStream, - + timeTickStream: wTtMsgStream, flushMap: sync.Map{}, flushChan: flushCh, flushingSegCache: flushingSegCache, diff --git a/internal/datanode/flow_graph_insert_buffer_node_test.go b/internal/datanode/flow_graph_insert_buffer_node_test.go index 1d5db5d774..5093951d6b 100644 --- a/internal/datanode/flow_graph_insert_buffer_node_test.go +++ b/internal/datanode/flow_graph_insert_buffer_node_test.go @@ -110,13 +110,6 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) { _, err = newInsertBufferNode(ctx, flushChan, fm, newCache(), c) assert.Error(t, err) - - c.msFactory = &CDFMsFactory{ - Factory: msFactory, - cd: 1, - } - _, err = newInsertBufferNode(ctx, flushChan, fm, newCache(), c) - assert.Error(t, err) } type mockMsg struct{} @@ -142,7 +135,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) { for _, test := range invalidInTests { te.Run(test.description, func(t0 *testing.T) { ibn := &insertBufferNode{ - ttMerger: newMergedTimeTickerSender(func(Timestamp) error { return nil }), + ttMerger: newMergedTimeTickerSender(func(Timestamp, []int64) error { return nil }), } rt := ibn.Operate(test.in) assert.Empty(t0, rt) diff --git a/internal/datanode/flow_graph_time_ticker.go b/internal/datanode/flow_graph_time_ticker.go index bf9c629676..3de47a45fe 100644 --- a/internal/datanode/flow_graph_time_ticker.go +++ b/internal/datanode/flow_graph_time_ticker.go @@ -19,22 +19,22 @@ package datanode import ( "sync" "time" - - "go.uber.org/atomic" ) -type sendTimeTick func(Timestamp) error +type sendTimeTick func(Timestamp, []int64) error // mergedTimeTickerSender reduces time ticker sending rate when datanode is doing `fast-forwarding` // it makes sure time ticker send at most 10 times a second (1tick/100millisecond) // and the last time tick is always sent type mergedTimeTickerSender struct { - ts atomic.Uint64 // current ts value - cond *sync.Cond // condition to send timeticker - send sendTimeTick // actual sender logic + ts uint64 + segmentIDs map[int64]struct{} + lastSent time.Time + mu sync.Mutex + + cond *sync.Cond // condition to send timeticker + send sendTimeTick // actual sender logic - lastSent time.Time - lastMut sync.RWMutex wg sync.WaitGroup closeCh chan struct{} closeOnce sync.Once @@ -42,12 +42,12 @@ type mergedTimeTickerSender struct { func newMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { mt := &mergedTimeTickerSender{ - cond: sync.NewCond(&sync.Mutex{}), - send: send, - closeCh: make(chan struct{}), + ts: 0, // 0 for not tt send + segmentIDs: make(map[int64]struct{}), + cond: sync.NewCond(&sync.Mutex{}), + send: send, + closeCh: make(chan struct{}), } - mt.ts.Store(0) // 0 for not tt send - mt.wg.Add(2) go mt.tick() go mt.work() @@ -55,10 +55,13 @@ func newMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { return mt } -func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp) { - mt.ts.Store(ts) - mt.lastMut.RLock() - defer mt.lastMut.RUnlock() +func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) { + mt.mu.Lock() + defer mt.mu.Unlock() + mt.ts = ts + for _, sid := range segmentIDs { + mt.segmentIDs[sid] = struct{}{} + } if !mt.lastSent.IsZero() && time.Since(mt.lastSent) > time.Millisecond*100 { mt.cond.L.Lock() @@ -94,7 +97,7 @@ func (mt *mergedTimeTickerSender) isClosed() bool { func (mt *mergedTimeTickerSender) work() { defer mt.wg.Done() - ts, lastTs := uint64(0), uint64(0) + lastTs := uint64(0) for { mt.cond.L.Lock() if mt.isClosed() { @@ -102,15 +105,21 @@ func (mt *mergedTimeTickerSender) work() { return } mt.cond.Wait() - ts = mt.ts.Load() mt.cond.L.Unlock() - if ts != lastTs { - mt.send(ts) - lastTs = ts - mt.lastMut.Lock() + + mt.mu.Lock() + if mt.ts != lastTs { + var sids []int64 + for sid := range mt.segmentIDs { + sids = append(sids, sid) + } + mt.segmentIDs = make(map[int64]struct{}) + lastTs = mt.ts mt.lastSent = time.Now() - mt.lastMut.Unlock() + + mt.send(mt.ts, sids) } + mt.mu.Unlock() } } diff --git a/internal/datanode/flow_graph_time_ticker_test.go b/internal/datanode/flow_graph_time_ticker_test.go index 07bdf82a28..4c1b9d27e5 100644 --- a/internal/datanode/flow_graph_time_ticker_test.go +++ b/internal/datanode/flow_graph_time_ticker_test.go @@ -12,7 +12,7 @@ func TestMergedTimeTicker(t *testing.T) { var ticks []uint64 var mut sync.Mutex - mt := newMergedTimeTickerSender(func(ts Timestamp) error { + mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error { mut.Lock() defer mut.Unlock() ticks = append(ticks, ts) @@ -21,7 +21,7 @@ func TestMergedTimeTicker(t *testing.T) { for i := 1; i < 100; i++ { time.Sleep(time.Millisecond * 10) - mt.bufferTs(uint64(i)) + mt.bufferTs(uint64(i), nil) } mt.close() mut.Lock() @@ -35,7 +35,7 @@ func TestMergedTimeTicker_close10000(t *testing.T) { batchSize := 10000 wg.Add(batchSize) for i := 0; i < batchSize; i++ { - mt := newMergedTimeTickerSender(func(ts Timestamp) error { + mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error { return nil }) go func(mt *mergedTimeTickerSender) { diff --git a/internal/datanode/param_table.go b/internal/datanode/param_table.go index 575239822b..a93a482d86 100644 --- a/internal/datanode/param_table.go +++ b/internal/datanode/param_table.go @@ -59,9 +59,6 @@ type ParamTable struct { // Cluster channels ClusterChannelPrefix string - // Segment statistics channel - SegmentStatisticsChannelName string - // Timetick channel TimeTickChannelName string @@ -116,7 +113,6 @@ func (p *ParamTable) Init() { // Must init global msgchannel prefix before other channel names p.initClusterMsgChannelPrefix() - p.initSegmentStatisticsChannelName() p.initTimeTickChannelName() p.initEtcdEndpoints() @@ -196,15 +192,6 @@ func (p *ParamTable) initClusterMsgChannelPrefix() { p.ClusterChannelPrefix = name } -func (p *ParamTable) initSegmentStatisticsChannelName() { - config, err := p.Load("msgChannel.chanNamePrefix.dataCoordStatistic") - if err != nil { - panic(err) - } - s := []string{p.ClusterChannelPrefix, config} - p.SegmentStatisticsChannelName = strings.Join(s, "-") -} - func (p *ParamTable) initTimeTickChannelName() { config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick") if err != nil { diff --git a/internal/datanode/param_table_test.go b/internal/datanode/param_table_test.go index fd092b9c75..b09635ec0c 100644 --- a/internal/datanode/param_table_test.go +++ b/internal/datanode/param_table_test.go @@ -71,12 +71,6 @@ func TestParamTable(t *testing.T) { log.Println("ClusterChannelPrefix:", Params.ClusterChannelPrefix) }) - t.Run("Test SegmentStatisticsChannelName", func(t *testing.T) { - path := Params.SegmentStatisticsChannelName - assert.Equal(t, path, "by-dev-datacoord-statistics-channel") - log.Println("SegmentStatisticsChannelName:", path) - }) - t.Run("Test TimeTickChannelName", func(t *testing.T) { name := Params.TimeTickChannelName assert.Equal(t, name, "by-dev-datacoord-timetick-channel") diff --git a/internal/datanode/segment_replica.go b/internal/datanode/segment_replica.go index 34dbdd974a..bd2e6bcefb 100644 --- a/internal/datanode/segment_replica.go +++ b/internal/datanode/segment_replica.go @@ -67,7 +67,7 @@ type Replica interface { updateStatistics(segID UniqueID, numRows int64) refreshFlushedSegStatistics(segID UniqueID, numRows int64) - getSegmentStatisticsUpdates(segID UniqueID) (*internalpb.SegmentStatisticsUpdates, error) + getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) segmentFlushed(segID UniqueID) } @@ -580,12 +580,10 @@ func (replica *SegmentReplica) updateStatistics(segID UniqueID, numRows int64) { } // getSegmentStatisticsUpdates gives current segment's statistics updates. -func (replica *SegmentReplica) getSegmentStatisticsUpdates(segID UniqueID) (*internalpb.SegmentStatisticsUpdates, error) { +func (replica *SegmentReplica) getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error) { replica.segMu.Lock() defer replica.segMu.Unlock() - updates := &internalpb.SegmentStatisticsUpdates{ - SegmentID: segID, - } + updates := &datapb.SegmentStats{SegmentID: segID} if seg, ok := replica.newSegments[segID]; ok { updates.NumRows = seg.numRows diff --git a/internal/msgstream/msg.go b/internal/msgstream/msg.go index 243730c157..4894a7f057 100644 --- a/internal/msgstream/msg.go +++ b/internal/msgstream/msg.go @@ -609,69 +609,6 @@ func (qs *QueryNodeStatsMsg) Unmarshal(input MarshalType) (TsMsg, error) { return queryNodeSegStatsMsg, nil } -/////////////////////////////////////////SegmentStatisticsMsg////////////////////////////////////////// - -// SegmentStatisticsMsg is a message pack that contains segment statistic -type SegmentStatisticsMsg struct { - BaseMsg - internalpb.SegmentStatistics -} - -// interface implementation validation -var _ TsMsg = &SegmentStatisticsMsg{} - -// TraceCtx returns the context of opentracing -func (ss *SegmentStatisticsMsg) TraceCtx() context.Context { - return ss.BaseMsg.Ctx -} - -// SetTraceCtx is used to set context for opentracing -func (ss *SegmentStatisticsMsg) SetTraceCtx(ctx context.Context) { - ss.BaseMsg.Ctx = ctx -} - -// ID returns the ID of this message pack -func (ss *SegmentStatisticsMsg) ID() UniqueID { - return ss.Base.MsgID -} - -// Type returns the type of this message pack -func (ss *SegmentStatisticsMsg) Type() MsgType { - return ss.Base.MsgType -} - -// SourceID indicated which component generated this message -func (ss *SegmentStatisticsMsg) SourceID() int64 { - return ss.Base.SourceID -} - -// Marshal is used to serializing a message pack to byte array -func (ss *SegmentStatisticsMsg) Marshal(input TsMsg) (MarshalType, error) { - segStatsTask := input.(*SegmentStatisticsMsg) - segStats := &segStatsTask.SegmentStatistics - mb, err := proto.Marshal(segStats) - if err != nil { - return nil, err - } - return mb, nil -} - -// Unmarshal is used to deserializing a message pack from byte array -func (ss *SegmentStatisticsMsg) Unmarshal(input MarshalType) (TsMsg, error) { - segStats := internalpb.SegmentStatistics{} - in, err := convertToByteArray(input) - if err != nil { - return nil, err - } - err = proto.Unmarshal(in, &segStats) - if err != nil { - return nil, err - } - segStatsMsg := &SegmentStatisticsMsg{SegmentStatistics: segStats} - - return segStatsMsg, nil -} - /////////////////////////////////////////CreateCollection////////////////////////////////////////// // CreateCollectionMsg is a message pack that contains create collection request diff --git a/internal/msgstream/msg_test.go b/internal/msgstream/msg_test.go index a2189cafc6..8b7ff266a8 100644 --- a/internal/msgstream/msg_test.go +++ b/internal/msgstream/msg_test.go @@ -510,50 +510,6 @@ func TestQueryNodeStatsMsg_Unmarshal_IllegalParameter(t *testing.T) { assert.Nil(t, tsMsg) } -func TestSegmentStatisticsMsg(t *testing.T) { - segmentStatisticsMsg := &SegmentStatisticsMsg{ - BaseMsg: generateBaseMsg(), - SegmentStatistics: internalpb.SegmentStatistics{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_SegmentStatistics, - MsgID: 1, - Timestamp: 2, - SourceID: 3, - }, - SegStats: []*internalpb.SegmentStatisticsUpdates{}, - }, - } - - assert.NotNil(t, segmentStatisticsMsg.TraceCtx()) - - ctx := context.Background() - segmentStatisticsMsg.SetTraceCtx(ctx) - assert.Equal(t, ctx, segmentStatisticsMsg.TraceCtx()) - - assert.Equal(t, int64(1), segmentStatisticsMsg.ID()) - assert.Equal(t, commonpb.MsgType_SegmentStatistics, segmentStatisticsMsg.Type()) - assert.Equal(t, int64(3), segmentStatisticsMsg.SourceID()) - - bytes, err := segmentStatisticsMsg.Marshal(segmentStatisticsMsg) - assert.Nil(t, err) - - tsMsg, err := segmentStatisticsMsg.Unmarshal(bytes) - assert.Nil(t, err) - - segmentStatisticsMsg2, ok := tsMsg.(*SegmentStatisticsMsg) - assert.True(t, ok) - assert.Equal(t, int64(1), segmentStatisticsMsg2.ID()) - assert.Equal(t, commonpb.MsgType_SegmentStatistics, segmentStatisticsMsg2.Type()) - assert.Equal(t, int64(3), segmentStatisticsMsg2.SourceID()) -} - -func TestSegmentStatisticsMsg_Unmarshal_IllegalParameter(t *testing.T) { - segmentStatisticsMsg := &SegmentStatisticsMsg{} - tsMsg, err := segmentStatisticsMsg.Unmarshal(10) - assert.NotNil(t, err) - assert.Nil(t, tsMsg) -} - func TestCreateCollectionMsg(t *testing.T) { createCollectionMsg := &CreateCollectionMsg{ BaseMsg: generateBaseMsg(), diff --git a/internal/msgstream/unmarshal.go b/internal/msgstream/unmarshal.go index 5d179f039c..6789be535a 100644 --- a/internal/msgstream/unmarshal.go +++ b/internal/msgstream/unmarshal.go @@ -66,7 +66,6 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher { createPartitionMsg := CreatePartitionMsg{} dropPartitionMsg := DropPartitionMsg{} queryNodeSegStatsMsg := QueryNodeStatsMsg{} - segmentStatisticsMsg := SegmentStatisticsMsg{} dataNodeTtMsg := DataNodeTtMsg{} sealedSegmentsChangeInfoMsg := SealedSegmentsChangeInfoMsg{} @@ -84,7 +83,6 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher { p.TempMap[commonpb.MsgType_DropCollection] = dropCollectionMsg.Unmarshal p.TempMap[commonpb.MsgType_CreatePartition] = createPartitionMsg.Unmarshal p.TempMap[commonpb.MsgType_DropPartition] = dropPartitionMsg.Unmarshal - p.TempMap[commonpb.MsgType_SegmentStatistics] = segmentStatisticsMsg.Unmarshal p.TempMap[commonpb.MsgType_DataNodeTt] = dataNodeTtMsg.Unmarshal p.TempMap[commonpb.MsgType_SealedSegmentsChangeInfo] = sealedSegmentsChangeInfoMsg.Unmarshal diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 4c5e4b9f97..ee9759287d 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -253,6 +253,12 @@ message DataNodeTtMsg { common.MsgBase base =1; string channel_name = 2; uint64 timestamp = 3; + repeated SegmentStats segments_stats = 4; +} + +message SegmentStats { + int64 SegmentID = 1; + int64 NumRows = 2; } enum ChannelWatchState { diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index 19768039bd..1c9f5120e8 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -1810,6 +1810,7 @@ type DataNodeTtMsg struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` ChannelName string `protobuf:"bytes,2,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + SegmentsStats []*SegmentStats `protobuf:"bytes,4,rep,name=segments_stats,json=segmentsStats,proto3" json:"segments_stats,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1861,6 +1862,60 @@ func (m *DataNodeTtMsg) GetTimestamp() uint64 { return 0 } +func (m *DataNodeTtMsg) GetSegmentsStats() []*SegmentStats { + if m != nil { + return m.SegmentsStats + } + return nil +} + +type SegmentStats struct { + SegmentID int64 `protobuf:"varint,1,opt,name=SegmentID,proto3" json:"SegmentID,omitempty"` + NumRows int64 `protobuf:"varint,2,opt,name=NumRows,proto3" json:"NumRows,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SegmentStats) Reset() { *m = SegmentStats{} } +func (m *SegmentStats) String() string { return proto.CompactTextString(m) } +func (*SegmentStats) ProtoMessage() {} +func (*SegmentStats) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{29} +} + +func (m *SegmentStats) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SegmentStats.Unmarshal(m, b) +} +func (m *SegmentStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SegmentStats.Marshal(b, m, deterministic) +} +func (m *SegmentStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_SegmentStats.Merge(m, src) +} +func (m *SegmentStats) XXX_Size() int { + return xxx_messageInfo_SegmentStats.Size(m) +} +func (m *SegmentStats) XXX_DiscardUnknown() { + xxx_messageInfo_SegmentStats.DiscardUnknown(m) +} + +var xxx_messageInfo_SegmentStats proto.InternalMessageInfo + +func (m *SegmentStats) GetSegmentID() int64 { + if m != nil { + return m.SegmentID + } + return 0 +} + +func (m *SegmentStats) GetNumRows() int64 { + if m != nil { + return m.NumRows + } + return 0 +} + type ChannelStatus struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` State ChannelWatchState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.data.ChannelWatchState" json:"state,omitempty"` @@ -1874,7 +1929,7 @@ func (m *ChannelStatus) Reset() { *m = ChannelStatus{} } func (m *ChannelStatus) String() string { return proto.CompactTextString(m) } func (*ChannelStatus) ProtoMessage() {} func (*ChannelStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{29} + return fileDescriptor_82cd95f524594f49, []int{30} } func (m *ChannelStatus) XXX_Unmarshal(b []byte) error { @@ -1929,7 +1984,7 @@ func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} } func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) } func (*DataNodeInfo) ProtoMessage() {} func (*DataNodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{30} + return fileDescriptor_82cd95f524594f49, []int{31} } func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error { @@ -1986,7 +2041,7 @@ func (m *SegmentBinlogs) Reset() { *m = SegmentBinlogs{} } func (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) } func (*SegmentBinlogs) ProtoMessage() {} func (*SegmentBinlogs) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{31} + return fileDescriptor_82cd95f524594f49, []int{32} } func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error { @@ -2054,7 +2109,7 @@ func (m *FieldBinlog) Reset() { *m = FieldBinlog{} } func (m *FieldBinlog) String() string { return proto.CompactTextString(m) } func (*FieldBinlog) ProtoMessage() {} func (*FieldBinlog) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{32} + return fileDescriptor_82cd95f524594f49, []int{33} } func (m *FieldBinlog) XXX_Unmarshal(b []byte) error { @@ -2102,7 +2157,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) } func (*GetRecoveryInfoResponse) ProtoMessage() {} func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{33} + return fileDescriptor_82cd95f524594f49, []int{34} } func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error { @@ -2157,7 +2212,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{} func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) } func (*GetRecoveryInfoRequest) ProtoMessage() {} func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{34} + return fileDescriptor_82cd95f524594f49, []int{35} } func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error { @@ -2212,7 +2267,7 @@ func (m *GetFlushedSegmentsRequest) Reset() { *m = GetFlushedSegmentsReq func (m *GetFlushedSegmentsRequest) String() string { return proto.CompactTextString(m) } func (*GetFlushedSegmentsRequest) ProtoMessage() {} func (*GetFlushedSegmentsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{35} + return fileDescriptor_82cd95f524594f49, []int{36} } func (m *GetFlushedSegmentsRequest) XXX_Unmarshal(b []byte) error { @@ -2266,7 +2321,7 @@ func (m *GetFlushedSegmentsResponse) Reset() { *m = GetFlushedSegmentsRe func (m *GetFlushedSegmentsResponse) String() string { return proto.CompactTextString(m) } func (*GetFlushedSegmentsResponse) ProtoMessage() {} func (*GetFlushedSegmentsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{36} + return fileDescriptor_82cd95f524594f49, []int{37} } func (m *GetFlushedSegmentsResponse) XXX_Unmarshal(b []byte) error { @@ -2313,7 +2368,7 @@ func (m *SegmentFlushCompletedMsg) Reset() { *m = SegmentFlushCompletedM func (m *SegmentFlushCompletedMsg) String() string { return proto.CompactTextString(m) } func (*SegmentFlushCompletedMsg) ProtoMessage() {} func (*SegmentFlushCompletedMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{37} + return fileDescriptor_82cd95f524594f49, []int{38} } func (m *SegmentFlushCompletedMsg) XXX_Unmarshal(b []byte) error { @@ -2361,7 +2416,7 @@ func (m *ChannelWatchInfo) Reset() { *m = ChannelWatchInfo{} } func (m *ChannelWatchInfo) String() string { return proto.CompactTextString(m) } func (*ChannelWatchInfo) ProtoMessage() {} func (*ChannelWatchInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{38} + return fileDescriptor_82cd95f524594f49, []int{39} } func (m *ChannelWatchInfo) XXX_Unmarshal(b []byte) error { @@ -2417,7 +2472,7 @@ func (m *CompactionSegmentBinlogs) Reset() { *m = CompactionSegmentBinlo func (m *CompactionSegmentBinlogs) String() string { return proto.CompactTextString(m) } func (*CompactionSegmentBinlogs) ProtoMessage() {} func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{39} + return fileDescriptor_82cd95f524594f49, []int{40} } func (m *CompactionSegmentBinlogs) XXX_Unmarshal(b []byte) error { @@ -2483,7 +2538,7 @@ func (m *CompactionPlan) Reset() { *m = CompactionPlan{} } func (m *CompactionPlan) String() string { return proto.CompactTextString(m) } func (*CompactionPlan) ProtoMessage() {} func (*CompactionPlan) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{40} + return fileDescriptor_82cd95f524594f49, []int{41} } func (m *CompactionPlan) XXX_Unmarshal(b []byte) error { @@ -2569,7 +2624,7 @@ func (m *CompactionResult) Reset() { *m = CompactionResult{} } func (m *CompactionResult) String() string { return proto.CompactTextString(m) } func (*CompactionResult) ProtoMessage() {} func (*CompactionResult) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{41} + return fileDescriptor_82cd95f524594f49, []int{42} } func (m *CompactionResult) XXX_Unmarshal(b []byte) error { @@ -2645,7 +2700,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{} func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{42} + return fileDescriptor_82cd95f524594f49, []int{43} } func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { @@ -2692,7 +2747,7 @@ func (m *WatchChannelsRequest) Reset() { *m = WatchChannelsRequest{} } func (m *WatchChannelsRequest) String() string { return proto.CompactTextString(m) } func (*WatchChannelsRequest) ProtoMessage() {} func (*WatchChannelsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{43} + return fileDescriptor_82cd95f524594f49, []int{44} } func (m *WatchChannelsRequest) XXX_Unmarshal(b []byte) error { @@ -2738,7 +2793,7 @@ func (m *WatchChannelsResponse) Reset() { *m = WatchChannelsResponse{} } func (m *WatchChannelsResponse) String() string { return proto.CompactTextString(m) } func (*WatchChannelsResponse) ProtoMessage() {} func (*WatchChannelsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{44} + return fileDescriptor_82cd95f524594f49, []int{45} } func (m *WatchChannelsResponse) XXX_Unmarshal(b []byte) error { @@ -2779,7 +2834,7 @@ func (m *DropVirtualChannelRequest) Reset() { *m = DropVirtualChannelReq func (m *DropVirtualChannelRequest) String() string { return proto.CompactTextString(m) } func (*DropVirtualChannelRequest) ProtoMessage() {} func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{45} + return fileDescriptor_82cd95f524594f49, []int{46} } func (m *DropVirtualChannelRequest) XXX_Unmarshal(b []byte) error { @@ -2839,7 +2894,7 @@ func (m *DropVirtualChannelSegment) Reset() { *m = DropVirtualChannelSeg func (m *DropVirtualChannelSegment) String() string { return proto.CompactTextString(m) } func (*DropVirtualChannelSegment) ProtoMessage() {} func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{46} + return fileDescriptor_82cd95f524594f49, []int{47} } func (m *DropVirtualChannelSegment) XXX_Unmarshal(b []byte) error { @@ -2927,7 +2982,7 @@ func (m *DropVirtualChannelResponse) Reset() { *m = DropVirtualChannelRe func (m *DropVirtualChannelResponse) String() string { return proto.CompactTextString(m) } func (*DropVirtualChannelResponse) ProtoMessage() {} func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{47} + return fileDescriptor_82cd95f524594f49, []int{48} } func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error { @@ -2987,6 +3042,7 @@ func init() { proto.RegisterType((*CheckPoint)(nil), "milvus.proto.data.CheckPoint") proto.RegisterType((*DeltaLogInfo)(nil), "milvus.proto.data.DeltaLogInfo") proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg") + proto.RegisterType((*SegmentStats)(nil), "milvus.proto.data.SegmentStats") proto.RegisterType((*ChannelStatus)(nil), "milvus.proto.data.ChannelStatus") proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo") proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs") @@ -3011,180 +3067,182 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 2758 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xdf, 0x6f, 0x1b, 0xc7, - 0xf1, 0xf7, 0xf1, 0x87, 0x44, 0x0e, 0x29, 0x8a, 0x5e, 0x39, 0x32, 0xc3, 0x38, 0xb2, 0x7c, 0x49, - 0x1c, 0xc5, 0x71, 0x24, 0x5b, 0xfe, 0x06, 0xdf, 0xa0, 0x4e, 0x1a, 0x58, 0x96, 0x2d, 0x13, 0x95, - 0x5c, 0xe5, 0xa8, 0xd8, 0x45, 0x03, 0x94, 0x38, 0xf1, 0x56, 0xd4, 0xd5, 0xbc, 0x3b, 0xfa, 0x6e, - 0x29, 0x5b, 0x79, 0x89, 0x91, 0x02, 0x05, 0x5a, 0xb4, 0x4d, 0x8a, 0xbe, 0x16, 0x68, 0xd1, 0xa7, - 0x02, 0x7d, 0x29, 0xfa, 0x98, 0xfe, 0x03, 0x45, 0xfb, 0xde, 0xbf, 0xa1, 0x6f, 0xfd, 0x17, 0x8a, - 0xfd, 0x71, 0x7b, 0x3f, 0x78, 0x47, 0x1e, 0x25, 0xff, 0x78, 0xe3, 0xee, 0xcd, 0xce, 0xcc, 0xce, - 0xce, 0x7c, 0x66, 0x66, 0x97, 0x50, 0x37, 0x74, 0xa2, 0x77, 0xba, 0x8e, 0xe3, 0x1a, 0xab, 0x03, - 0xd7, 0x21, 0x0e, 0x3a, 0x6b, 0x99, 0xfd, 0xa3, 0xa1, 0xc7, 0x47, 0xab, 0xf4, 0x73, 0xb3, 0xda, - 0x75, 0x2c, 0xcb, 0xb1, 0xf9, 0x54, 0xb3, 0x66, 0xda, 0x04, 0xbb, 0xb6, 0xde, 0x17, 0xe3, 0x6a, - 0x78, 0x41, 0xb3, 0xea, 0x75, 0x0f, 0xb1, 0xa5, 0xf3, 0x91, 0xfa, 0x14, 0xaa, 0x77, 0xfb, 0x43, - 0xef, 0x50, 0xc3, 0x8f, 0x87, 0xd8, 0x23, 0xe8, 0x1a, 0x14, 0xf6, 0x75, 0x0f, 0x37, 0x94, 0x65, - 0x65, 0xa5, 0xb2, 0x7e, 0x61, 0x35, 0x22, 0x4b, 0x48, 0xd9, 0xf1, 0x7a, 0x1b, 0xba, 0x87, 0x35, - 0x46, 0x89, 0x10, 0x14, 0x8c, 0xfd, 0xd6, 0x66, 0x23, 0xb7, 0xac, 0xac, 0xe4, 0x35, 0xf6, 0x1b, - 0xa9, 0x50, 0xed, 0x3a, 0xfd, 0x3e, 0xee, 0x12, 0xd3, 0xb1, 0x5b, 0x9b, 0x8d, 0x02, 0xfb, 0x16, - 0x99, 0x53, 0x7f, 0xaf, 0xc0, 0x9c, 0x10, 0xed, 0x0d, 0x1c, 0xdb, 0xc3, 0xe8, 0x06, 0xcc, 0x78, - 0x44, 0x27, 0x43, 0x4f, 0x48, 0x7f, 0x23, 0x51, 0x7a, 0x9b, 0x91, 0x68, 0x82, 0x34, 0x93, 0xf8, - 0xfc, 0xa8, 0x78, 0xb4, 0x04, 0xe0, 0xe1, 0x9e, 0x85, 0x6d, 0xd2, 0xda, 0xf4, 0x1a, 0x85, 0xe5, - 0xfc, 0x4a, 0x5e, 0x0b, 0xcd, 0xa8, 0xbf, 0x55, 0xa0, 0xde, 0xf6, 0x87, 0xbe, 0x75, 0xce, 0x41, - 0xb1, 0xeb, 0x0c, 0x6d, 0xc2, 0x14, 0x9c, 0xd3, 0xf8, 0x00, 0x5d, 0x82, 0x6a, 0xf7, 0x50, 0xb7, - 0x6d, 0xdc, 0xef, 0xd8, 0xba, 0x85, 0x99, 0x2a, 0x65, 0xad, 0x22, 0xe6, 0xee, 0xeb, 0x16, 0xce, - 0xa4, 0xd1, 0x32, 0x54, 0x06, 0xba, 0x4b, 0xcc, 0x88, 0xcd, 0xc2, 0x53, 0xea, 0x1f, 0x15, 0x58, - 0xbc, 0xe5, 0x79, 0x66, 0xcf, 0x1e, 0xd1, 0x6c, 0x11, 0x66, 0x6c, 0xc7, 0xc0, 0xad, 0x4d, 0xa6, - 0x5a, 0x5e, 0x13, 0x23, 0xf4, 0x06, 0x94, 0x07, 0x18, 0xbb, 0x1d, 0xd7, 0xe9, 0xfb, 0x8a, 0x95, - 0xe8, 0x84, 0xe6, 0xf4, 0x31, 0xfa, 0x0c, 0xce, 0x7a, 0x31, 0x46, 0x5e, 0x23, 0xbf, 0x9c, 0x5f, - 0xa9, 0xac, 0xbf, 0xb5, 0x3a, 0xe2, 0x65, 0xab, 0x71, 0xa1, 0xda, 0xe8, 0x6a, 0xf5, 0x59, 0x0e, - 0x16, 0x24, 0x1d, 0xd7, 0x95, 0xfe, 0xa6, 0x96, 0xf3, 0x70, 0x4f, 0xaa, 0xc7, 0x07, 0x59, 0x2c, - 0x27, 0x4d, 0x9e, 0x0f, 0x9b, 0x3c, 0x83, 0x83, 0xc5, 0xed, 0x59, 0x1c, 0xb1, 0x27, 0xba, 0x08, - 0x15, 0xfc, 0x74, 0x60, 0xba, 0xb8, 0x43, 0x4c, 0x0b, 0x37, 0x66, 0x96, 0x95, 0x95, 0x82, 0x06, - 0x7c, 0x6a, 0xcf, 0xb4, 0xc2, 0x1e, 0x39, 0x9b, 0xd9, 0x23, 0xd5, 0x3f, 0x29, 0x70, 0x7e, 0xe4, - 0x94, 0x84, 0x8b, 0x6b, 0x50, 0x67, 0x3b, 0x0f, 0x2c, 0x43, 0x9d, 0x9d, 0x1a, 0xfc, 0xf2, 0x38, - 0x83, 0x07, 0xe4, 0xda, 0xc8, 0xfa, 0x90, 0x92, 0xb9, 0xec, 0x4a, 0x3e, 0x82, 0xf3, 0x5b, 0x98, - 0x08, 0x01, 0xf4, 0x1b, 0xf6, 0x4e, 0x0e, 0x01, 0xd1, 0x58, 0xca, 0x8d, 0xc4, 0xd2, 0x5f, 0x73, - 0x32, 0x96, 0x98, 0xa8, 0x96, 0x7d, 0xe0, 0xa0, 0x0b, 0x50, 0x96, 0x24, 0xc2, 0x2b, 0x82, 0x09, - 0xf4, 0xff, 0x50, 0xa4, 0x9a, 0x72, 0x97, 0xa8, 0xad, 0x5f, 0x4a, 0xde, 0x53, 0x88, 0xa7, 0xc6, - 0xe9, 0x51, 0x0b, 0x6a, 0x1e, 0xd1, 0x5d, 0xd2, 0x19, 0x38, 0x1e, 0x3b, 0x67, 0xe6, 0x38, 0x95, - 0x75, 0x35, 0xca, 0x41, 0x42, 0xe4, 0x8e, 0xd7, 0xdb, 0x15, 0x94, 0xda, 0x1c, 0x5b, 0xe9, 0x0f, - 0xd1, 0x1d, 0xa8, 0x62, 0xdb, 0x08, 0x18, 0x15, 0x32, 0x33, 0xaa, 0x60, 0xdb, 0x90, 0x6c, 0x82, - 0xf3, 0x29, 0x66, 0x3f, 0x9f, 0x5f, 0x29, 0xd0, 0x18, 0x3d, 0xa0, 0xd3, 0x00, 0xe5, 0x4d, 0xbe, - 0x08, 0xf3, 0x03, 0x1a, 0x1b, 0xe1, 0xf2, 0x90, 0x34, 0xb1, 0x44, 0x35, 0xe1, 0xb5, 0x40, 0x1b, - 0xf6, 0xe5, 0x85, 0x39, 0xcb, 0xcf, 0x14, 0x58, 0x8c, 0xcb, 0x3a, 0xcd, 0xbe, 0xff, 0x0f, 0x8a, - 0xa6, 0x7d, 0xe0, 0xf8, 0xdb, 0x5e, 0x1a, 0x13, 0x67, 0x54, 0x16, 0x27, 0x56, 0x2d, 0x78, 0x63, - 0x0b, 0x93, 0x96, 0xed, 0x61, 0x97, 0x6c, 0x98, 0x76, 0xdf, 0xe9, 0xed, 0xea, 0xe4, 0xf0, 0x14, - 0x31, 0x12, 0x71, 0xf7, 0x5c, 0xcc, 0xdd, 0xd5, 0x3f, 0x2b, 0x70, 0x21, 0x59, 0x9e, 0xd8, 0x7a, - 0x13, 0x4a, 0x07, 0x26, 0xee, 0x1b, 0xd4, 0x66, 0x0a, 0xb3, 0x99, 0x1c, 0xd3, 0x58, 0x19, 0x50, - 0x62, 0xb1, 0xc3, 0x4b, 0x29, 0x0e, 0xda, 0x26, 0xae, 0x69, 0xf7, 0xb6, 0x4d, 0x8f, 0x68, 0x9c, - 0x3e, 0x64, 0xcf, 0x7c, 0x76, 0xcf, 0xfc, 0xa5, 0x02, 0x4b, 0x5b, 0x98, 0xdc, 0x96, 0x50, 0x4b, - 0xbf, 0x9b, 0x1e, 0x31, 0xbb, 0xde, 0x8b, 0x2d, 0x22, 0x12, 0x72, 0xa6, 0xfa, 0x8d, 0x02, 0x17, - 0x53, 0x95, 0x11, 0xa6, 0x13, 0x50, 0xe2, 0x03, 0x6d, 0x32, 0x94, 0xfc, 0x00, 0x1f, 0x3f, 0xd0, - 0xfb, 0x43, 0xbc, 0xab, 0x9b, 0x2e, 0x87, 0x92, 0x13, 0x02, 0xeb, 0x5f, 0x14, 0x78, 0x73, 0x0b, - 0x93, 0x5d, 0x3f, 0xcd, 0xbc, 0x42, 0xeb, 0x64, 0xa8, 0x28, 0x7e, 0xc3, 0x0f, 0x33, 0x51, 0xdb, - 0x57, 0x62, 0xbe, 0x25, 0x16, 0x07, 0xa1, 0x80, 0xbc, 0xcd, 0x6b, 0x01, 0x61, 0x3c, 0xf5, 0x59, - 0x1e, 0xaa, 0x0f, 0x44, 0x7d, 0xc0, 0xd2, 0x48, 0xdc, 0x0e, 0x4a, 0xb2, 0x1d, 0x42, 0x25, 0x45, - 0x52, 0x95, 0xb1, 0x05, 0x73, 0x1e, 0xc6, 0x8f, 0x4e, 0x92, 0x34, 0xaa, 0x74, 0xa1, 0x04, 0xfb, - 0x6d, 0x38, 0x3b, 0xb4, 0x0f, 0x68, 0x59, 0x8b, 0x0d, 0xb1, 0x0b, 0x5e, 0x5d, 0x4e, 0x46, 0x9e, - 0xd1, 0x85, 0xe8, 0x1e, 0xcc, 0xc7, 0x79, 0x15, 0x33, 0xf1, 0x8a, 0x2f, 0x43, 0x2d, 0xa8, 0x1b, - 0xae, 0x33, 0x18, 0x60, 0xa3, 0xe3, 0xf9, 0xac, 0x66, 0xb2, 0xb1, 0x12, 0xeb, 0x7c, 0x56, 0xea, - 0x2f, 0x14, 0x58, 0x7c, 0xa8, 0x93, 0xee, 0xe1, 0xa6, 0x25, 0x0e, 0xe7, 0x14, 0xae, 0xfd, 0x09, - 0x94, 0x8f, 0xc4, 0x41, 0xf8, 0xf8, 0x75, 0x31, 0x41, 0xa1, 0xf0, 0x91, 0x6b, 0xc1, 0x0a, 0xf5, - 0x1f, 0x0a, 0x9c, 0x63, 0x4d, 0x84, 0xaf, 0xdd, 0xcb, 0x0f, 0xb2, 0x09, 0x8d, 0x04, 0xba, 0x0c, - 0x35, 0x4b, 0x77, 0x1f, 0xb5, 0x03, 0x9a, 0x22, 0xa3, 0x89, 0xcd, 0xaa, 0x4f, 0x01, 0xc4, 0x68, - 0xc7, 0xeb, 0x9d, 0x40, 0xff, 0x8f, 0x60, 0x56, 0x48, 0x15, 0xf1, 0x36, 0xe9, 0x60, 0x7d, 0x72, - 0xf5, 0x9f, 0x0a, 0xd4, 0x02, 0x04, 0x65, 0x51, 0x55, 0x83, 0x9c, 0x8c, 0xa5, 0x5c, 0x6b, 0x13, - 0x7d, 0x02, 0x33, 0xbc, 0x6d, 0x14, 0xbc, 0xdf, 0x89, 0xf2, 0x16, 0x2d, 0x65, 0x08, 0x86, 0xd9, - 0x84, 0x26, 0x16, 0x51, 0x1b, 0x49, 0xd4, 0xe1, 0x1d, 0x46, 0x5e, 0x0b, 0xcd, 0xa0, 0x16, 0xcc, - 0x47, 0x8b, 0x36, 0x3f, 0x66, 0x96, 0xd3, 0xd0, 0x66, 0x53, 0x27, 0x3a, 0x03, 0x9b, 0x5a, 0xa4, - 0x66, 0xf3, 0xd4, 0xff, 0x16, 0xa1, 0x12, 0xda, 0xe5, 0xc8, 0x4e, 0xe2, 0x47, 0x9a, 0x9b, 0x8c, - 0x9b, 0xf9, 0xd1, 0xce, 0xe1, 0x1d, 0xa8, 0x99, 0x2c, 0x57, 0x77, 0x84, 0x2b, 0x32, 0x70, 0x2d, - 0x6b, 0x73, 0x7c, 0x56, 0xc4, 0x05, 0x5a, 0x82, 0x8a, 0x3d, 0xb4, 0x3a, 0xce, 0x41, 0xc7, 0x75, - 0x9e, 0x78, 0xa2, 0x05, 0x29, 0xdb, 0x43, 0xeb, 0x87, 0x07, 0x9a, 0xf3, 0xc4, 0x0b, 0xaa, 0xdc, - 0x99, 0x29, 0xab, 0xdc, 0x25, 0xa8, 0x58, 0xfa, 0x53, 0xca, 0xb5, 0x63, 0x0f, 0x2d, 0xd6, 0x9d, - 0xe4, 0xb5, 0xb2, 0xa5, 0x3f, 0xd5, 0x9c, 0x27, 0xf7, 0x87, 0x16, 0x5a, 0x81, 0x7a, 0x5f, 0xf7, - 0x48, 0x27, 0xdc, 0xde, 0x94, 0x58, 0x7b, 0x53, 0xa3, 0xf3, 0x77, 0x82, 0x16, 0x67, 0xb4, 0x5e, - 0x2e, 0x9f, 0xa2, 0x5e, 0x36, 0xac, 0x7e, 0xc0, 0x08, 0xb2, 0xd7, 0xcb, 0x86, 0xd5, 0x97, 0x6c, - 0x3e, 0x82, 0xd9, 0x7d, 0x56, 0x01, 0x79, 0x8d, 0x4a, 0x2a, 0x42, 0xdd, 0xa5, 0xc5, 0x0f, 0x2f, - 0x94, 0x34, 0x9f, 0x1c, 0x7d, 0x0c, 0x65, 0x96, 0x7a, 0xd8, 0xda, 0x6a, 0xa6, 0xb5, 0xc1, 0x02, - 0x0a, 0x45, 0x06, 0xee, 0x13, 0x9d, 0xad, 0x9e, 0x4b, 0x85, 0xa2, 0x4d, 0x4a, 0xb3, 0xed, 0xf4, - 0x38, 0x14, 0xc9, 0x15, 0xe8, 0x1a, 0x2c, 0x74, 0x5d, 0xac, 0x13, 0x6c, 0x6c, 0x1c, 0xdf, 0x76, - 0xac, 0x81, 0xce, 0xbc, 0xa9, 0x51, 0x5b, 0x56, 0x56, 0x4a, 0x5a, 0xd2, 0x27, 0x8a, 0x0c, 0x5d, - 0x39, 0xba, 0xeb, 0x3a, 0x56, 0x63, 0x9e, 0x23, 0x43, 0x74, 0x16, 0xbd, 0x09, 0xe0, 0x63, 0xb7, - 0x4e, 0x1a, 0x75, 0x76, 0x8c, 0x65, 0x31, 0x73, 0x8b, 0xa8, 0x5f, 0xc1, 0xb9, 0xc0, 0x45, 0x42, - 0xc7, 0x31, 0x7a, 0xb2, 0xca, 0x49, 0x4f, 0x76, 0x7c, 0xf1, 0xfa, 0xb7, 0x02, 0x2c, 0xb6, 0xf5, - 0x23, 0xfc, 0xe2, 0xeb, 0xe4, 0x4c, 0x80, 0xbc, 0x0d, 0x67, 0x59, 0x69, 0xbc, 0x1e, 0xd2, 0x67, - 0x4c, 0x0a, 0x0e, 0x7b, 0xc3, 0xe8, 0x42, 0xf4, 0x29, 0xad, 0x1d, 0x70, 0xf7, 0xd1, 0xae, 0x63, - 0x06, 0xe9, 0xf7, 0xcd, 0x04, 0x3e, 0xb7, 0x25, 0x95, 0x16, 0x5e, 0x81, 0x76, 0x47, 0xb1, 0x8d, - 0x27, 0xde, 0x77, 0xc7, 0x36, 0x60, 0x81, 0xf5, 0xe3, 0x10, 0x87, 0x1a, 0x30, 0x2b, 0xd2, 0x3b, - 0x0b, 0xfc, 0x92, 0xe6, 0x0f, 0xd1, 0x2e, 0x2c, 0xf0, 0x1d, 0xb4, 0x85, 0x57, 0xf3, 0xcd, 0x97, - 0x32, 0x6d, 0x3e, 0x69, 0x69, 0x34, 0x28, 0xca, 0x53, 0x07, 0x45, 0x03, 0x66, 0x85, 0xa3, 0x32, - 0x34, 0x28, 0x69, 0xfe, 0x90, 0xb6, 0x11, 0x10, 0x98, 0x6c, 0xc2, 0x6d, 0xc0, 0xf7, 0xa1, 0x24, - 0x9d, 0x38, 0x97, 0xd9, 0x89, 0xe5, 0x9a, 0x38, 0x0e, 0xe7, 0x63, 0x38, 0xac, 0xfe, 0x4b, 0x81, - 0x6a, 0x78, 0x0b, 0x14, 0xdf, 0x5d, 0xdc, 0x75, 0x5c, 0xa3, 0x83, 0x6d, 0xe2, 0x9a, 0x98, 0x77, - 0x9c, 0x05, 0x6d, 0x8e, 0xcf, 0xde, 0xe1, 0x93, 0x94, 0x8c, 0x42, 0xab, 0x47, 0x74, 0x6b, 0xd0, - 0x39, 0xa0, 0x11, 0x9c, 0xe3, 0x64, 0x72, 0x96, 0x05, 0xf0, 0x25, 0xa8, 0x06, 0x64, 0xc4, 0x61, - 0xf2, 0x0b, 0x5a, 0x45, 0xce, 0xed, 0x39, 0xe8, 0x6d, 0xa8, 0x31, 0xab, 0x75, 0xfa, 0x4e, 0xaf, - 0x43, 0xbb, 0x33, 0x91, 0x50, 0xaa, 0x86, 0x50, 0x8b, 0x1e, 0x47, 0x94, 0xca, 0x33, 0xbf, 0xc4, - 0x22, 0xa5, 0x48, 0xaa, 0xb6, 0xf9, 0x25, 0x56, 0xbf, 0x56, 0x60, 0x8e, 0xe6, 0xc7, 0xfb, 0x8e, - 0x81, 0xf7, 0x4e, 0x58, 0x4d, 0x64, 0xb8, 0x99, 0xbb, 0x00, 0x65, 0xb9, 0x03, 0xb1, 0xa5, 0x60, - 0x82, 0xb6, 0xf1, 0x73, 0x22, 0x0d, 0xb6, 0xe5, 0x4d, 0x2d, 0x63, 0xa5, 0x30, 0x56, 0xec, 0x37, - 0xfa, 0x5e, 0xf4, 0x9a, 0xe7, 0xed, 0xc4, 0xb8, 0x62, 0x4c, 0x58, 0xc5, 0x19, 0xc9, 0x81, 0x59, - 0xfa, 0xc3, 0x67, 0xf4, 0x60, 0x85, 0x29, 0xd8, 0xc1, 0x36, 0x60, 0x56, 0x37, 0x0c, 0x17, 0x7b, - 0x9e, 0xd0, 0xc3, 0x1f, 0xd2, 0x2f, 0x47, 0xd8, 0xf5, 0x7c, 0x17, 0xcb, 0x6b, 0xfe, 0x10, 0x7d, - 0x0c, 0x25, 0x59, 0xa2, 0xe6, 0x93, 0xca, 0x92, 0xb0, 0x9e, 0xa2, 0x9f, 0x91, 0x2b, 0xd4, 0x6f, - 0x72, 0x50, 0x13, 0x61, 0xbd, 0x21, 0xf2, 0xd4, 0x78, 0x67, 0xdf, 0x80, 0xea, 0x41, 0x10, 0x96, - 0xe3, 0xee, 0x2d, 0xc2, 0xd1, 0x1b, 0x59, 0x33, 0xc9, 0xe1, 0xa3, 0x99, 0xb2, 0x70, 0xaa, 0x4c, - 0x59, 0x9c, 0x16, 0x14, 0xd4, 0x5b, 0x50, 0x09, 0x31, 0x66, 0x70, 0xc6, 0xaf, 0x32, 0x84, 0x2d, - 0xfc, 0x21, 0xfd, 0xb2, 0x1f, 0x32, 0x42, 0x59, 0x66, 0x7a, 0x5a, 0xf7, 0x9f, 0xdf, 0xc2, 0x44, - 0xc3, 0x5d, 0xe7, 0x08, 0xbb, 0xc7, 0xa7, 0xbf, 0x25, 0xba, 0x19, 0x3a, 0xe3, 0x8c, 0x6d, 0x88, - 0x5c, 0x80, 0x6e, 0x06, 0x7a, 0xe6, 0x93, 0x9a, 0xe4, 0x30, 0xb4, 0x8b, 0x13, 0x0a, 0xb6, 0xf2, - 0x2d, 0xbf, 0xef, 0x8a, 0x6e, 0xe5, 0xa4, 0xd9, 0xf3, 0xb9, 0x54, 0xb7, 0xea, 0xef, 0x14, 0x78, - 0x7d, 0x0b, 0x93, 0xbb, 0xd1, 0x1e, 0xf2, 0x55, 0x6b, 0x65, 0x41, 0x33, 0x49, 0xa9, 0xd3, 0x9c, - 0x7a, 0x13, 0x4a, 0xb2, 0x1b, 0xe6, 0x37, 0x91, 0x72, 0xac, 0xfe, 0x5c, 0x81, 0x86, 0x90, 0xc2, - 0x64, 0xd2, 0xc2, 0xad, 0x8f, 0x09, 0x36, 0x5e, 0x76, 0x7b, 0xf6, 0x07, 0x05, 0xea, 0x61, 0x10, - 0x64, 0x38, 0xf6, 0x21, 0x14, 0x59, 0x17, 0x2c, 0x34, 0x98, 0xe8, 0xac, 0x9c, 0x9a, 0x46, 0x14, - 0x2b, 0x26, 0xf6, 0x3c, 0x1f, 0xe4, 0xc4, 0x30, 0x40, 0xe2, 0xfc, 0xd4, 0x48, 0xac, 0xfe, 0x3a, - 0x07, 0x8d, 0xa0, 0xae, 0x7d, 0xe9, 0x60, 0x97, 0x52, 0xf5, 0xe4, 0x9f, 0x53, 0xd5, 0x53, 0x98, - 0x1a, 0xe0, 0xfe, 0x9e, 0xa3, 0x0d, 0xb5, 0x6f, 0x8f, 0xdd, 0xbe, 0x6e, 0xa3, 0x45, 0x98, 0x19, - 0xf4, 0xf5, 0xe0, 0x82, 0x4a, 0x8c, 0x50, 0x1b, 0x6a, 0x5e, 0xc4, 0x5e, 0xc2, 0x02, 0xef, 0x27, - 0xd9, 0x3f, 0xc5, 0xc4, 0x5a, 0x8c, 0x05, 0x6d, 0x18, 0x78, 0xc9, 0xc9, 0xfa, 0x3e, 0x91, 0x9a, - 0xf9, 0x41, 0xd3, 0x96, 0xef, 0x2a, 0x20, 0xfa, 0xc1, 0x19, 0x92, 0x8e, 0x69, 0x77, 0x3c, 0xdc, - 0x75, 0x6c, 0xc3, 0x63, 0xf5, 0x46, 0x51, 0xab, 0x8b, 0x2f, 0x2d, 0xbb, 0xcd, 0xe7, 0xd1, 0x87, - 0x50, 0x20, 0xc7, 0x03, 0x5e, 0x69, 0xd4, 0x12, 0x91, 0x2d, 0xd0, 0x6b, 0xef, 0x78, 0x80, 0x35, - 0x46, 0x4e, 0x5b, 0x7e, 0xca, 0x8a, 0xb8, 0xfa, 0x11, 0xee, 0xfb, 0x4f, 0x6b, 0xc1, 0x0c, 0xf5, - 0x44, 0xbf, 0x75, 0x9e, 0xe5, 0x89, 0x58, 0x0c, 0xd5, 0xef, 0x72, 0x50, 0x0f, 0x58, 0x6a, 0xd8, - 0x1b, 0xf6, 0x49, 0xaa, 0xfd, 0xc6, 0xb7, 0x0b, 0x93, 0xd2, 0xe0, 0xa7, 0x50, 0x11, 0x6d, 0xfc, - 0x14, 0x89, 0x10, 0xf8, 0x92, 0xed, 0x31, 0xae, 0x57, 0x7c, 0x4e, 0xae, 0x37, 0x33, 0xb5, 0xeb, - 0xb5, 0x61, 0xd1, 0x07, 0xad, 0x40, 0xd2, 0x0e, 0x26, 0xfa, 0x98, 0x34, 0x7b, 0x11, 0x2a, 0x3c, - 0x19, 0xf1, 0xc2, 0x93, 0x97, 0x7a, 0xb0, 0x2f, 0x9b, 0x20, 0xf5, 0x27, 0x70, 0x8e, 0x05, 0x7d, - 0xfc, 0xba, 0x2f, 0xcb, 0xdd, 0xab, 0x2a, 0x0b, 0x49, 0x5a, 0x34, 0xfa, 0x89, 0x3c, 0x32, 0xa7, - 0x6e, 0xc3, 0x6b, 0x31, 0xfe, 0xa7, 0x00, 0x75, 0xf5, 0x3b, 0x05, 0x5e, 0xdf, 0x74, 0x9d, 0xc1, - 0x03, 0xd3, 0x25, 0x43, 0xbd, 0x1f, 0xbd, 0x40, 0x7e, 0x31, 0xa5, 0xf0, 0xbd, 0x50, 0x1e, 0xe1, - 0xb0, 0x73, 0x35, 0xe9, 0xcc, 0x46, 0x94, 0x12, 0x47, 0x15, 0xca, 0x3a, 0xff, 0xc9, 0x27, 0x29, - 0x2f, 0xe8, 0x26, 0x60, 0x69, 0x96, 0x34, 0x9b, 0xd8, 0x1c, 0xe7, 0x4f, 0xda, 0x1c, 0xa7, 0xb8, - 0x7f, 0xe1, 0x39, 0xb9, 0xff, 0xd4, 0xa5, 0x25, 0xba, 0x07, 0xd1, 0x9b, 0x0b, 0x06, 0x3c, 0x27, - 0xba, 0xf2, 0xd8, 0x00, 0x08, 0xba, 0x78, 0xf1, 0xfc, 0x9f, 0x85, 0x4d, 0x68, 0x15, 0x3d, 0x2e, - 0x89, 0x35, 0xec, 0xfa, 0x2d, 0xd2, 0x74, 0x7e, 0x06, 0xcd, 0x24, 0x37, 0x3d, 0x85, 0xeb, 0x5f, - 0xb9, 0x0e, 0x67, 0x47, 0x92, 0x34, 0xaa, 0x01, 0x7c, 0x6e, 0x77, 0x45, 0xf5, 0x52, 0x3f, 0x83, - 0xaa, 0x50, 0xf2, 0x6b, 0x99, 0xba, 0x72, 0xa5, 0x1d, 0x4e, 0x55, 0x14, 0xbf, 0xd1, 0x79, 0x58, - 0xf8, 0xdc, 0x36, 0xf0, 0x81, 0x69, 0x63, 0x23, 0xf8, 0x54, 0x3f, 0x83, 0x16, 0x60, 0xbe, 0x65, - 0xdb, 0xd8, 0x0d, 0x4d, 0x2a, 0x74, 0x72, 0x07, 0xbb, 0x3d, 0x1c, 0x9a, 0xcc, 0xad, 0x7f, 0xbb, - 0x00, 0x65, 0xda, 0x76, 0xdd, 0x76, 0x1c, 0xd7, 0x40, 0x03, 0x40, 0xec, 0x8d, 0xce, 0x1a, 0x38, - 0xb6, 0x7c, 0xcc, 0x46, 0xd7, 0x52, 0x8c, 0x39, 0x4a, 0x2a, 0x42, 0xb7, 0x79, 0x39, 0x65, 0x45, - 0x8c, 0x5c, 0x3d, 0x83, 0x2c, 0x26, 0x91, 0x26, 0xbb, 0x3d, 0xb3, 0xfb, 0xc8, 0xbf, 0x8d, 0x1d, - 0x23, 0x31, 0x46, 0xea, 0x4b, 0x8c, 0xbd, 0x91, 0x8b, 0x01, 0x7f, 0x48, 0xf5, 0x4f, 0x4a, 0x3d, - 0x83, 0x1e, 0xc3, 0xb9, 0x2d, 0x4c, 0x82, 0xb7, 0x33, 0x5f, 0xe0, 0x7a, 0xba, 0xc0, 0x11, 0xe2, - 0x29, 0x45, 0x6e, 0x43, 0x91, 0x55, 0xa5, 0x28, 0x29, 0x3a, 0xc2, 0xff, 0xe8, 0x6a, 0x2e, 0xa7, - 0x13, 0x48, 0x6e, 0x3f, 0x85, 0xf9, 0xd8, 0x3f, 0x56, 0xd0, 0x7b, 0x09, 0xcb, 0x92, 0xff, 0x7b, - 0xd4, 0xbc, 0x92, 0x85, 0x54, 0xca, 0xea, 0x41, 0x2d, 0xfa, 0xc2, 0x87, 0x56, 0x12, 0xd6, 0x27, - 0xfe, 0xdb, 0xa0, 0xf9, 0x5e, 0x06, 0x4a, 0x29, 0xc8, 0x82, 0x7a, 0xfc, 0x1f, 0x14, 0xe8, 0xca, - 0x58, 0x06, 0x51, 0x77, 0x7b, 0x3f, 0x13, 0xad, 0x14, 0x77, 0xcc, 0x9c, 0x60, 0xe4, 0x05, 0x1f, - 0xad, 0x26, 0xb3, 0x49, 0xfb, 0x6b, 0x41, 0x73, 0x2d, 0x33, 0xbd, 0x14, 0xfd, 0x35, 0xef, 0x86, - 0x93, 0x5e, 0xc1, 0xd1, 0xf5, 0x64, 0x76, 0x63, 0x9e, 0xef, 0x9b, 0xeb, 0xd3, 0x2c, 0x91, 0x4a, - 0x7c, 0xc5, 0xda, 0xd8, 0x84, 0x97, 0xe4, 0x78, 0xdc, 0xf9, 0xfc, 0xd2, 0x9f, 0xc8, 0x9b, 0xd7, - 0xa7, 0x58, 0x21, 0x15, 0x70, 0xe2, 0xff, 0x51, 0xf1, 0xc3, 0x70, 0x6d, 0xa2, 0xd7, 0x9c, 0x2c, - 0x06, 0xbf, 0x80, 0xf9, 0xd8, 0xb5, 0x77, 0x62, 0xd4, 0x24, 0x5f, 0x8d, 0x37, 0xc7, 0x01, 0x3a, - 0x0f, 0xc9, 0xd8, 0xad, 0x00, 0x4a, 0xf1, 0xfe, 0x84, 0x9b, 0x83, 0xe6, 0x95, 0x2c, 0xa4, 0x72, - 0x23, 0x1e, 0x83, 0xcb, 0x58, 0x67, 0x8d, 0xae, 0x26, 0xf3, 0x48, 0xbe, 0x15, 0x68, 0x7e, 0x90, - 0x91, 0x5a, 0x0a, 0xed, 0x00, 0x6c, 0x61, 0xb2, 0x83, 0x89, 0x4b, 0x7d, 0xe4, 0x72, 0xa2, 0xc9, - 0x03, 0x02, 0x5f, 0xcc, 0xbb, 0x13, 0xe9, 0xa4, 0x80, 0x1f, 0x01, 0xf2, 0xf3, 0x5c, 0xe8, 0xd1, - 0xe5, 0xad, 0xb1, 0x0d, 0x0c, 0xef, 0x36, 0x26, 0x9d, 0xcd, 0x63, 0xa8, 0xef, 0xe8, 0x36, 0x4d, - 0xda, 0x01, 0xdf, 0xab, 0x89, 0x8a, 0xc5, 0xc9, 0x52, 0xac, 0x95, 0x4a, 0x2d, 0x37, 0xf3, 0x44, - 0xe6, 0x50, 0x5d, 0x86, 0x20, 0x8e, 0x63, 0x4b, 0x60, 0x8d, 0x18, 0x61, 0x0a, 0xb6, 0x8c, 0xa1, - 0x97, 0x82, 0x9f, 0x29, 0xec, 0x9f, 0x50, 0x31, 0x82, 0x87, 0x26, 0x39, 0xa4, 0x7d, 0xad, 0x97, - 0x45, 0x05, 0x46, 0x38, 0x85, 0x0a, 0x82, 0x5e, 0xaa, 0x60, 0xc0, 0x5c, 0xa4, 0x3d, 0x40, 0x49, - 0x2f, 0x27, 0x49, 0x0d, 0x4a, 0x73, 0x65, 0x32, 0xa1, 0x94, 0x72, 0x08, 0x73, 0xbe, 0xbf, 0x72, - 0xe3, 0xbe, 0x97, 0xa6, 0x69, 0x40, 0x93, 0x12, 0x6e, 0xc9, 0xa4, 0xe1, 0x70, 0x1b, 0x2d, 0xfc, - 0x50, 0xb6, 0x8e, 0x61, 0x5c, 0xb8, 0xa5, 0x57, 0x93, 0xea, 0x99, 0xf5, 0x7f, 0x17, 0xa0, 0xe4, - 0xdf, 0x84, 0xbf, 0x82, 0x8a, 0xec, 0x15, 0x94, 0x48, 0x5f, 0xc0, 0x7c, 0xec, 0x6f, 0x2a, 0x89, - 0x08, 0x9a, 0xfc, 0x57, 0x96, 0x49, 0x10, 0xf0, 0x50, 0xfc, 0x79, 0x5d, 0xa2, 0xe5, 0xbb, 0x69, - 0x65, 0x56, 0x1c, 0x28, 0x27, 0x30, 0x7e, 0xe1, 0xb0, 0x78, 0x1f, 0x20, 0x04, 0x5b, 0xe3, 0xef, - 0x73, 0x68, 0x24, 0x4e, 0x50, 0x78, 0xe3, 0xc6, 0x8f, 0xaf, 0xf7, 0x4c, 0x72, 0x38, 0xdc, 0xa7, - 0x5f, 0xd6, 0x38, 0xe9, 0x07, 0xa6, 0x23, 0x7e, 0xad, 0xf9, 0x27, 0xba, 0xc6, 0x56, 0xaf, 0x51, - 0x01, 0x83, 0xfd, 0xfd, 0x19, 0x36, 0xba, 0xf1, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x4d, - 0xf6, 0xb0, 0xde, 0x30, 0x00, 0x00, + // 2800 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x5b, 0x6f, 0xdc, 0xc6, + 0xf5, 0x37, 0xf7, 0x22, 0xed, 0x9e, 0xbd, 0x68, 0x3d, 0x72, 0xe4, 0xcd, 0xc6, 0x91, 0x65, 0x26, + 0x71, 0x14, 0xc7, 0x91, 0x62, 0xe5, 0x1f, 0xfc, 0x83, 0x26, 0x69, 0x10, 0x59, 0x91, 0xb2, 0xa8, + 0xe4, 0x2a, 0x5c, 0xc5, 0x2e, 0x1a, 0xa0, 0x0b, 0x6a, 0x39, 0x5a, 0xb1, 0x5e, 0x92, 0x6b, 0x72, + 0x56, 0xb6, 0xf2, 0x12, 0xa3, 0x01, 0x0a, 0xb4, 0x68, 0x9b, 0x14, 0x7d, 0x2d, 0xd0, 0xa2, 0x4f, + 0x05, 0xfa, 0x52, 0xf4, 0x31, 0xfd, 0x02, 0x45, 0xfb, 0xde, 0xcf, 0xd0, 0xb7, 0x7e, 0x85, 0x62, + 0x2e, 0x1c, 0x5e, 0x96, 0xdc, 0xe5, 0x4a, 0xbe, 0xbc, 0x71, 0x86, 0xe7, 0xcc, 0x39, 0x73, 0xe6, + 0x9c, 0xdf, 0x39, 0x67, 0x48, 0x68, 0x18, 0x3a, 0xd1, 0xbb, 0x3d, 0xc7, 0x71, 0x8d, 0xb5, 0xa1, + 0xeb, 0x10, 0x07, 0x5d, 0xb4, 0xcc, 0xc1, 0xc9, 0xc8, 0xe3, 0xa3, 0x35, 0xfa, 0xba, 0x55, 0xed, + 0x39, 0x96, 0xe5, 0xd8, 0x7c, 0xaa, 0x55, 0x37, 0x6d, 0x82, 0x5d, 0x5b, 0x1f, 0x88, 0x71, 0x35, + 0xcc, 0xd0, 0xaa, 0x7a, 0xbd, 0x63, 0x6c, 0xe9, 0x7c, 0xa4, 0x3e, 0x82, 0xea, 0xf6, 0x60, 0xe4, + 0x1d, 0x6b, 0xf8, 0xc1, 0x08, 0x7b, 0x04, 0xbd, 0x0d, 0x85, 0x43, 0xdd, 0xc3, 0x4d, 0x65, 0x45, + 0x59, 0xad, 0x6c, 0x5c, 0x59, 0x8b, 0xc8, 0x12, 0x52, 0xf6, 0xbc, 0xfe, 0xa6, 0xee, 0x61, 0x8d, + 0x51, 0x22, 0x04, 0x05, 0xe3, 0xb0, 0xbd, 0xd5, 0xcc, 0xad, 0x28, 0xab, 0x79, 0x8d, 0x3d, 0x23, + 0x15, 0xaa, 0x3d, 0x67, 0x30, 0xc0, 0x3d, 0x62, 0x3a, 0x76, 0x7b, 0xab, 0x59, 0x60, 0xef, 0x22, + 0x73, 0xea, 0xef, 0x15, 0xa8, 0x09, 0xd1, 0xde, 0xd0, 0xb1, 0x3d, 0x8c, 0xde, 0x81, 0x39, 0x8f, + 0xe8, 0x64, 0xe4, 0x09, 0xe9, 0x2f, 0x25, 0x4a, 0xef, 0x30, 0x12, 0x4d, 0x90, 0x66, 0x12, 0x9f, + 0x1f, 0x17, 0x8f, 0x96, 0x01, 0x3c, 0xdc, 0xb7, 0xb0, 0x4d, 0xda, 0x5b, 0x5e, 0xb3, 0xb0, 0x92, + 0x5f, 0xcd, 0x6b, 0xa1, 0x19, 0xf5, 0xb7, 0x0a, 0x34, 0x3a, 0xfe, 0xd0, 0xb7, 0xce, 0x25, 0x28, + 0xf6, 0x9c, 0x91, 0x4d, 0x98, 0x82, 0x35, 0x8d, 0x0f, 0xd0, 0x35, 0xa8, 0xf6, 0x8e, 0x75, 0xdb, + 0xc6, 0x83, 0xae, 0xad, 0x5b, 0x98, 0xa9, 0x52, 0xd6, 0x2a, 0x62, 0xee, 0x8e, 0x6e, 0xe1, 0x4c, + 0x1a, 0xad, 0x40, 0x65, 0xa8, 0xbb, 0xc4, 0x8c, 0xd8, 0x2c, 0x3c, 0xa5, 0xfe, 0x51, 0x81, 0xa5, + 0x8f, 0x3d, 0xcf, 0xec, 0xdb, 0x63, 0x9a, 0x2d, 0xc1, 0x9c, 0xed, 0x18, 0xb8, 0xbd, 0xc5, 0x54, + 0xcb, 0x6b, 0x62, 0x84, 0x5e, 0x82, 0xf2, 0x10, 0x63, 0xb7, 0xeb, 0x3a, 0x03, 0x5f, 0xb1, 0x12, + 0x9d, 0xd0, 0x9c, 0x01, 0x46, 0x9f, 0xc1, 0x45, 0x2f, 0xb6, 0x90, 0xd7, 0xcc, 0xaf, 0xe4, 0x57, + 0x2b, 0x1b, 0xaf, 0xac, 0x8d, 0x79, 0xd9, 0x5a, 0x5c, 0xa8, 0x36, 0xce, 0xad, 0x3e, 0xce, 0xc1, + 0xa2, 0xa4, 0xe3, 0xba, 0xd2, 0x67, 0x6a, 0x39, 0x0f, 0xf7, 0xa5, 0x7a, 0x7c, 0x90, 0xc5, 0x72, + 0xd2, 0xe4, 0xf9, 0xb0, 0xc9, 0x33, 0x38, 0x58, 0xdc, 0x9e, 0xc5, 0x31, 0x7b, 0xa2, 0xab, 0x50, + 0xc1, 0x8f, 0x86, 0xa6, 0x8b, 0xbb, 0xc4, 0xb4, 0x70, 0x73, 0x6e, 0x45, 0x59, 0x2d, 0x68, 0xc0, + 0xa7, 0x0e, 0x4c, 0x2b, 0xec, 0x91, 0xf3, 0x99, 0x3d, 0x52, 0xfd, 0x93, 0x02, 0x97, 0xc7, 0x4e, + 0x49, 0xb8, 0xb8, 0x06, 0x0d, 0xb6, 0xf3, 0xc0, 0x32, 0xd4, 0xd9, 0xa9, 0xc1, 0xaf, 0x4f, 0x32, + 0x78, 0x40, 0xae, 0x8d, 0xf1, 0x87, 0x94, 0xcc, 0x65, 0x57, 0xf2, 0x3e, 0x5c, 0xde, 0xc1, 0x44, + 0x08, 0xa0, 0xef, 0xb0, 0x77, 0x76, 0x08, 0x88, 0xc6, 0x52, 0x6e, 0x2c, 0x96, 0xfe, 0x9a, 0x93, + 0xb1, 0xc4, 0x44, 0xb5, 0xed, 0x23, 0x07, 0x5d, 0x81, 0xb2, 0x24, 0x11, 0x5e, 0x11, 0x4c, 0xa0, + 0xff, 0x87, 0x22, 0xd5, 0x94, 0xbb, 0x44, 0x7d, 0xe3, 0x5a, 0xf2, 0x9e, 0x42, 0x6b, 0x6a, 0x9c, + 0x1e, 0xb5, 0xa1, 0xee, 0x11, 0xdd, 0x25, 0xdd, 0xa1, 0xe3, 0xb1, 0x73, 0x66, 0x8e, 0x53, 0xd9, + 0x50, 0xa3, 0x2b, 0x48, 0x88, 0xdc, 0xf3, 0xfa, 0xfb, 0x82, 0x52, 0xab, 0x31, 0x4e, 0x7f, 0x88, + 0x3e, 0x81, 0x2a, 0xb6, 0x8d, 0x60, 0xa1, 0x42, 0xe6, 0x85, 0x2a, 0xd8, 0x36, 0xe4, 0x32, 0xc1, + 0xf9, 0x14, 0xb3, 0x9f, 0xcf, 0xaf, 0x14, 0x68, 0x8e, 0x1f, 0xd0, 0x79, 0x80, 0xf2, 0x7d, 0xce, + 0x84, 0xf9, 0x01, 0x4d, 0x8c, 0x70, 0x79, 0x48, 0x9a, 0x60, 0x51, 0x4d, 0x78, 0x21, 0xd0, 0x86, + 0xbd, 0x79, 0x6a, 0xce, 0xf2, 0xb5, 0x02, 0x4b, 0x71, 0x59, 0xe7, 0xd9, 0xf7, 0xff, 0x41, 0xd1, + 0xb4, 0x8f, 0x1c, 0x7f, 0xdb, 0xcb, 0x13, 0xe2, 0x8c, 0xca, 0xe2, 0xc4, 0xaa, 0x05, 0x2f, 0xed, + 0x60, 0xd2, 0xb6, 0x3d, 0xec, 0x92, 0x4d, 0xd3, 0x1e, 0x38, 0xfd, 0x7d, 0x9d, 0x1c, 0x9f, 0x23, + 0x46, 0x22, 0xee, 0x9e, 0x8b, 0xb9, 0xbb, 0xfa, 0x67, 0x05, 0xae, 0x24, 0xcb, 0x13, 0x5b, 0x6f, + 0x41, 0xe9, 0xc8, 0xc4, 0x03, 0x83, 0xda, 0x4c, 0x61, 0x36, 0x93, 0x63, 0x1a, 0x2b, 0x43, 0x4a, + 0x2c, 0x76, 0x78, 0x2d, 0xc5, 0x41, 0x3b, 0xc4, 0x35, 0xed, 0xfe, 0xae, 0xe9, 0x11, 0x8d, 0xd3, + 0x87, 0xec, 0x99, 0xcf, 0xee, 0x99, 0xbf, 0x54, 0x60, 0x79, 0x07, 0x93, 0xdb, 0x12, 0x6a, 0xe9, + 0x7b, 0xd3, 0x23, 0x66, 0xcf, 0x7b, 0xba, 0x45, 0x44, 0x42, 0xce, 0x54, 0xbf, 0x51, 0xe0, 0x6a, + 0xaa, 0x32, 0xc2, 0x74, 0x02, 0x4a, 0x7c, 0xa0, 0x4d, 0x86, 0x92, 0x1f, 0xe0, 0xd3, 0xbb, 0xfa, + 0x60, 0x84, 0xf7, 0x75, 0xd3, 0xe5, 0x50, 0x72, 0x46, 0x60, 0xfd, 0x8b, 0x02, 0x2f, 0xef, 0x60, + 0xb2, 0xef, 0xa7, 0x99, 0xe7, 0x68, 0x9d, 0x0c, 0x15, 0xc5, 0x6f, 0xf8, 0x61, 0x26, 0x6a, 0xfb, + 0x5c, 0xcc, 0xb7, 0xcc, 0xe2, 0x20, 0x14, 0x90, 0xb7, 0x79, 0x2d, 0x20, 0x8c, 0xa7, 0x3e, 0xce, + 0x43, 0xf5, 0xae, 0xa8, 0x0f, 0x58, 0x1a, 0x89, 0xdb, 0x41, 0x49, 0xb6, 0x43, 0xa8, 0xa4, 0x48, + 0xaa, 0x32, 0x76, 0xa0, 0xe6, 0x61, 0x7c, 0xff, 0x2c, 0x49, 0xa3, 0x4a, 0x19, 0x25, 0xd8, 0xef, + 0xc2, 0xc5, 0x91, 0x7d, 0x44, 0xcb, 0x5a, 0x6c, 0x88, 0x5d, 0xf0, 0xea, 0x72, 0x3a, 0xf2, 0x8c, + 0x33, 0xa2, 0x4f, 0x61, 0x21, 0xbe, 0x56, 0x31, 0xd3, 0x5a, 0x71, 0x36, 0xd4, 0x86, 0x86, 0xe1, + 0x3a, 0xc3, 0x21, 0x36, 0xba, 0x9e, 0xbf, 0xd4, 0x5c, 0xb6, 0xa5, 0x04, 0x9f, 0xbf, 0x94, 0xfa, + 0x0b, 0x05, 0x96, 0xee, 0xe9, 0xa4, 0x77, 0xbc, 0x65, 0x89, 0xc3, 0x39, 0x87, 0x6b, 0x7f, 0x08, + 0xe5, 0x13, 0x71, 0x10, 0x3e, 0x7e, 0x5d, 0x4d, 0x50, 0x28, 0x7c, 0xe4, 0x5a, 0xc0, 0xa1, 0xfe, + 0x43, 0x81, 0x4b, 0xac, 0x89, 0xf0, 0xb5, 0x7b, 0xf6, 0x41, 0x36, 0xa5, 0x91, 0x40, 0xd7, 0xa1, + 0x6e, 0xe9, 0xee, 0xfd, 0x4e, 0x40, 0x53, 0x64, 0x34, 0xb1, 0x59, 0xf5, 0x11, 0x80, 0x18, 0xed, + 0x79, 0xfd, 0x33, 0xe8, 0xff, 0x1e, 0xcc, 0x0b, 0xa9, 0x22, 0xde, 0xa6, 0x1d, 0xac, 0x4f, 0xae, + 0xfe, 0x53, 0x81, 0x7a, 0x80, 0xa0, 0x2c, 0xaa, 0xea, 0x90, 0x93, 0xb1, 0x94, 0x6b, 0x6f, 0xa1, + 0x0f, 0x61, 0x8e, 0xb7, 0x8d, 0x62, 0xed, 0xd7, 0xa2, 0x6b, 0x8b, 0x96, 0x32, 0x04, 0xc3, 0x6c, + 0x42, 0x13, 0x4c, 0xd4, 0x46, 0x12, 0x75, 0x78, 0x87, 0x91, 0xd7, 0x42, 0x33, 0xa8, 0x0d, 0x0b, + 0xd1, 0xa2, 0xcd, 0x8f, 0x99, 0x95, 0x34, 0xb4, 0xd9, 0xd2, 0x89, 0xce, 0xc0, 0xa6, 0x1e, 0xa9, + 0xd9, 0x3c, 0xf5, 0xbf, 0x45, 0xa8, 0x84, 0x76, 0x39, 0xb6, 0x93, 0xf8, 0x91, 0xe6, 0xa6, 0xe3, + 0x66, 0x7e, 0xbc, 0x73, 0x78, 0x0d, 0xea, 0x26, 0xcb, 0xd5, 0x5d, 0xe1, 0x8a, 0x0c, 0x5c, 0xcb, + 0x5a, 0x8d, 0xcf, 0x8a, 0xb8, 0x40, 0xcb, 0x50, 0xb1, 0x47, 0x56, 0xd7, 0x39, 0xea, 0xba, 0xce, + 0x43, 0x4f, 0xb4, 0x20, 0x65, 0x7b, 0x64, 0xfd, 0xf0, 0x48, 0x73, 0x1e, 0x7a, 0x41, 0x95, 0x3b, + 0x37, 0x63, 0x95, 0xbb, 0x0c, 0x15, 0x4b, 0x7f, 0x44, 0x57, 0xed, 0xda, 0x23, 0x8b, 0x75, 0x27, + 0x79, 0xad, 0x6c, 0xe9, 0x8f, 0x34, 0xe7, 0xe1, 0x9d, 0x91, 0x85, 0x56, 0xa1, 0x31, 0xd0, 0x3d, + 0xd2, 0x0d, 0xb7, 0x37, 0x25, 0xd6, 0xde, 0xd4, 0xe9, 0xfc, 0x27, 0x41, 0x8b, 0x33, 0x5e, 0x2f, + 0x97, 0xcf, 0x51, 0x2f, 0x1b, 0xd6, 0x20, 0x58, 0x08, 0xb2, 0xd7, 0xcb, 0x86, 0x35, 0x90, 0xcb, + 0xbc, 0x07, 0xf3, 0x87, 0xac, 0x02, 0xf2, 0x9a, 0x95, 0x54, 0x84, 0xda, 0xa6, 0xc5, 0x0f, 0x2f, + 0x94, 0x34, 0x9f, 0x1c, 0x7d, 0x00, 0x65, 0x96, 0x7a, 0x18, 0x6f, 0x35, 0x13, 0x6f, 0xc0, 0x40, + 0xa1, 0xc8, 0xc0, 0x03, 0xa2, 0x33, 0xee, 0x5a, 0x2a, 0x14, 0x6d, 0x51, 0x9a, 0x5d, 0xa7, 0xcf, + 0xa1, 0x48, 0x72, 0xa0, 0xb7, 0x61, 0xb1, 0xe7, 0x62, 0x9d, 0x60, 0x63, 0xf3, 0xf4, 0xb6, 0x63, + 0x0d, 0x75, 0xe6, 0x4d, 0xcd, 0xfa, 0x8a, 0xb2, 0x5a, 0xd2, 0x92, 0x5e, 0x51, 0x64, 0xe8, 0xc9, + 0xd1, 0xb6, 0xeb, 0x58, 0xcd, 0x05, 0x8e, 0x0c, 0xd1, 0x59, 0xf4, 0x32, 0x80, 0x8f, 0xdd, 0x3a, + 0x69, 0x36, 0xd8, 0x31, 0x96, 0xc5, 0xcc, 0xc7, 0x44, 0xfd, 0x0a, 0x2e, 0x05, 0x2e, 0x12, 0x3a, + 0x8e, 0xf1, 0x93, 0x55, 0xce, 0x7a, 0xb2, 0x93, 0x8b, 0xd7, 0xbf, 0x15, 0x60, 0xa9, 0xa3, 0x9f, + 0xe0, 0xa7, 0x5f, 0x27, 0x67, 0x02, 0xe4, 0x5d, 0xb8, 0xc8, 0x4a, 0xe3, 0x8d, 0x90, 0x3e, 0x13, + 0x52, 0x70, 0xd8, 0x1b, 0xc6, 0x19, 0xd1, 0x47, 0xb4, 0x76, 0xc0, 0xbd, 0xfb, 0xfb, 0x8e, 0x19, + 0xa4, 0xdf, 0x97, 0x13, 0xd6, 0xb9, 0x2d, 0xa9, 0xb4, 0x30, 0x07, 0xda, 0x1f, 0xc7, 0x36, 0x9e, + 0x78, 0x5f, 0x9f, 0xd8, 0x80, 0x05, 0xd6, 0x8f, 0x43, 0x1c, 0x6a, 0xc2, 0xbc, 0x48, 0xef, 0x2c, + 0xf0, 0x4b, 0x9a, 0x3f, 0x44, 0xfb, 0xb0, 0xc8, 0x77, 0xd0, 0x11, 0x5e, 0xcd, 0x37, 0x5f, 0xca, + 0xb4, 0xf9, 0x24, 0xd6, 0x68, 0x50, 0x94, 0x67, 0x0e, 0x8a, 0x26, 0xcc, 0x0b, 0x47, 0x65, 0x68, + 0x50, 0xd2, 0xfc, 0x21, 0x6d, 0x23, 0x20, 0x30, 0xd9, 0x94, 0xdb, 0x80, 0xef, 0x43, 0x49, 0x3a, + 0x71, 0x2e, 0xb3, 0x13, 0x4b, 0x9e, 0x38, 0x0e, 0xe7, 0x63, 0x38, 0xac, 0xfe, 0x4b, 0x81, 0x6a, + 0x78, 0x0b, 0x14, 0xdf, 0x5d, 0xdc, 0x73, 0x5c, 0xa3, 0x8b, 0x6d, 0xe2, 0x9a, 0x98, 0x77, 0x9c, + 0x05, 0xad, 0xc6, 0x67, 0x3f, 0xe1, 0x93, 0x94, 0x8c, 0x42, 0xab, 0x47, 0x74, 0x6b, 0xd8, 0x3d, + 0xa2, 0x11, 0x9c, 0xe3, 0x64, 0x72, 0x96, 0x05, 0xf0, 0x35, 0xa8, 0x06, 0x64, 0xc4, 0x61, 0xf2, + 0x0b, 0x5a, 0x45, 0xce, 0x1d, 0x38, 0xe8, 0x55, 0xa8, 0x33, 0xab, 0x75, 0x07, 0x4e, 0xbf, 0x4b, + 0xbb, 0x33, 0x91, 0x50, 0xaa, 0x86, 0x50, 0x8b, 0x1e, 0x47, 0x94, 0xca, 0x33, 0xbf, 0xc4, 0x22, + 0xa5, 0x48, 0xaa, 0x8e, 0xf9, 0x25, 0xa6, 0xf9, 0xbc, 0x46, 0xf3, 0xe3, 0x1d, 0xc7, 0xc0, 0x07, + 0x67, 0xac, 0x26, 0x32, 0xdc, 0xcc, 0x5d, 0x81, 0xb2, 0xdc, 0x81, 0xd8, 0x52, 0x30, 0x81, 0xb6, + 0xa1, 0xee, 0x17, 0x9a, 0x5d, 0xde, 0x3f, 0x14, 0x52, 0xbd, 0x27, 0x94, 0xe1, 0x3c, 0xad, 0xe6, + 0xb3, 0xb1, 0xa1, 0xba, 0x0d, 0xd5, 0xf0, 0x6b, 0x2a, 0xb5, 0x13, 0x77, 0x14, 0x39, 0x41, 0xfd, + 0xed, 0xce, 0xc8, 0xa2, 0x67, 0x2a, 0xb0, 0xc3, 0x1f, 0xaa, 0x5f, 0x2b, 0x50, 0x13, 0x69, 0xb9, + 0x23, 0x6f, 0x8e, 0xd9, 0xd6, 0x14, 0xb6, 0x35, 0xf6, 0x8c, 0xbe, 0x17, 0xbd, 0x76, 0x7a, 0x35, + 0x31, 0xce, 0xd9, 0x22, 0xac, 0x02, 0x8e, 0xe4, 0xe4, 0x2c, 0xfd, 0xea, 0x63, 0xea, 0x68, 0xe2, + 0x68, 0x98, 0xa3, 0x35, 0x61, 0x5e, 0x37, 0x0c, 0x17, 0x7b, 0x9e, 0xd0, 0xc3, 0x1f, 0xd2, 0x37, + 0x27, 0xd8, 0xf5, 0x7c, 0x97, 0xcf, 0x6b, 0xfe, 0x10, 0x7d, 0x00, 0x25, 0x59, 0x32, 0xe7, 0x93, + 0xca, 0xa4, 0xb0, 0x9e, 0xa2, 0xbf, 0x92, 0x1c, 0xea, 0x37, 0x39, 0xa8, 0x0b, 0x83, 0x6d, 0x8a, + 0xbc, 0x39, 0x39, 0xf8, 0x36, 0xa1, 0x7a, 0x14, 0xc0, 0xc4, 0xa4, 0x7b, 0x94, 0x30, 0x9a, 0x44, + 0x78, 0xa6, 0x05, 0x60, 0x34, 0x73, 0x17, 0xce, 0x95, 0xb9, 0x8b, 0xb3, 0x82, 0x94, 0xfa, 0x31, + 0x54, 0x42, 0x0b, 0x33, 0x78, 0xe5, 0x57, 0x2b, 0xc2, 0x16, 0xfe, 0x90, 0xbe, 0x39, 0x0c, 0x19, + 0xa1, 0x2c, 0x2b, 0x0f, 0xda, 0x87, 0x5c, 0xde, 0xc1, 0x44, 0xc3, 0x3d, 0xe7, 0x04, 0xbb, 0xa7, + 0xe7, 0xbf, 0xb5, 0x7a, 0x3f, 0x74, 0xc6, 0x19, 0xdb, 0x22, 0xc9, 0x80, 0xde, 0x0f, 0xf4, 0xcc, + 0x27, 0x35, 0xed, 0xe1, 0xa0, 0x13, 0x27, 0x14, 0x6c, 0xe5, 0x5b, 0x7e, 0xff, 0x16, 0xdd, 0xca, + 0x59, 0xb3, 0xf9, 0x13, 0xa9, 0xb6, 0xd5, 0xdf, 0x29, 0xf0, 0xe2, 0x0e, 0x26, 0xdb, 0xd1, 0x9e, + 0xf6, 0x79, 0x6b, 0x65, 0x41, 0x2b, 0x49, 0xa9, 0xf3, 0x9c, 0x7a, 0x0b, 0x4a, 0xb2, 0x3b, 0xe7, + 0x37, 0xa3, 0x72, 0xac, 0xfe, 0x5c, 0x81, 0xa6, 0x90, 0xc2, 0x64, 0xd2, 0x42, 0x72, 0x80, 0x09, + 0x36, 0x9e, 0x75, 0xbb, 0xf8, 0x07, 0x05, 0x1a, 0x61, 0x10, 0x64, 0x38, 0xf6, 0x2e, 0x14, 0x59, + 0x57, 0x2e, 0x34, 0x98, 0xea, 0xac, 0x9c, 0x9a, 0x46, 0x14, 0x2b, 0x6e, 0x0e, 0x24, 0x5e, 0x8b, + 0x61, 0x80, 0xc4, 0xf9, 0x99, 0x91, 0x58, 0xfd, 0x75, 0x0e, 0x9a, 0x41, 0x9d, 0xfd, 0xcc, 0xc1, + 0x2e, 0xa5, 0x0a, 0xcb, 0x3f, 0xa1, 0x2a, 0xac, 0x30, 0x33, 0xc0, 0xfd, 0x3d, 0x47, 0x1b, 0x7c, + 0xdf, 0x1e, 0xfb, 0x03, 0xdd, 0x46, 0x4b, 0x30, 0x37, 0x1c, 0xe8, 0xc1, 0x85, 0x99, 0x18, 0xa1, + 0x8e, 0x4c, 0xdb, 0x51, 0x0b, 0xbc, 0x99, 0x64, 0xff, 0x14, 0x13, 0x6b, 0xb1, 0x25, 0x68, 0x03, + 0xc3, 0x4b, 0x60, 0xd6, 0x87, 0x8a, 0x52, 0x81, 0x1f, 0x34, 0x6d, 0x41, 0x6f, 0x02, 0xa2, 0x2f, + 0x9c, 0x11, 0xe9, 0x9a, 0x76, 0xd7, 0xc3, 0x3d, 0xc7, 0x36, 0x3c, 0x56, 0xff, 0x14, 0xb5, 0x86, + 0x78, 0xd3, 0xb6, 0x3b, 0x7c, 0x1e, 0xbd, 0x0b, 0x05, 0x72, 0x3a, 0xe4, 0x95, 0x4f, 0x3d, 0x11, + 0xd9, 0x02, 0xbd, 0x0e, 0x4e, 0x87, 0x58, 0x63, 0xe4, 0x68, 0x19, 0x80, 0x2e, 0x45, 0x5c, 0xfd, + 0x04, 0x0f, 0xfc, 0x4f, 0x7d, 0xc1, 0x0c, 0xf5, 0x44, 0xbf, 0x95, 0x9f, 0xe7, 0x89, 0x58, 0x0c, + 0xd5, 0xef, 0x72, 0xd0, 0x08, 0x96, 0xd4, 0xb0, 0x37, 0x1a, 0x90, 0x54, 0xfb, 0x4d, 0x6e, 0x5f, + 0xa6, 0xa5, 0xc1, 0x8f, 0xa0, 0x22, 0xae, 0x15, 0x66, 0x48, 0x84, 0xc0, 0x59, 0x76, 0x27, 0xb8, + 0x5e, 0xf1, 0x09, 0xb9, 0xde, 0xdc, 0xcc, 0xae, 0xd7, 0x81, 0x25, 0x1f, 0xb4, 0x02, 0x49, 0x7b, + 0x98, 0xe8, 0x13, 0xd2, 0xec, 0x55, 0xa8, 0xf0, 0x64, 0xc4, 0x0b, 0x61, 0x5e, 0x7a, 0xc2, 0xa1, + 0x6c, 0xca, 0xd4, 0x9f, 0xc0, 0x25, 0x16, 0xf4, 0xf1, 0xeb, 0xc7, 0x2c, 0x77, 0xc1, 0xaa, 0x2c, + 0x6c, 0x69, 0x11, 0xeb, 0x27, 0xf2, 0xc8, 0x9c, 0xba, 0x0b, 0x2f, 0xc4, 0xd6, 0x3f, 0x07, 0xa8, + 0xab, 0xdf, 0x29, 0xf0, 0xe2, 0x96, 0xeb, 0x0c, 0xef, 0x9a, 0x2e, 0x19, 0xe9, 0x83, 0xe8, 0x85, + 0xf6, 0xd3, 0x29, 0xcd, 0x3f, 0x0d, 0xe5, 0x11, 0x0e, 0x3b, 0x37, 0x93, 0xce, 0x6c, 0x4c, 0x29, + 0x71, 0x54, 0xa1, 0xac, 0xf3, 0x9f, 0x7c, 0x92, 0xf2, 0x82, 0x6e, 0x0a, 0x96, 0x66, 0x49, 0xb3, + 0x89, 0xcd, 0x7a, 0xfe, 0xac, 0xcd, 0x7a, 0x8a, 0xfb, 0x17, 0x9e, 0x90, 0xfb, 0xcf, 0x5c, 0x5a, + 0xa2, 0x4f, 0x21, 0x7a, 0x93, 0xc2, 0x80, 0xe7, 0x4c, 0x57, 0x30, 0x9b, 0x00, 0xc1, 0xad, 0x82, + 0xf8, 0x1d, 0x21, 0xcb, 0x32, 0x21, 0x2e, 0x7a, 0x5c, 0x12, 0x6b, 0xd8, 0x75, 0x60, 0xa4, 0x09, + 0xfe, 0x0c, 0x5a, 0x49, 0x6e, 0x7a, 0x0e, 0xd7, 0xbf, 0x71, 0x0b, 0x2e, 0x8e, 0x25, 0x69, 0x54, + 0x07, 0xf8, 0xdc, 0xee, 0x89, 0xea, 0xa5, 0x71, 0x01, 0x55, 0xa1, 0xe4, 0xd7, 0x32, 0x0d, 0xe5, + 0x46, 0x27, 0x9c, 0xaa, 0x28, 0x7e, 0xa3, 0xcb, 0xb0, 0xf8, 0xb9, 0x6d, 0xe0, 0x23, 0xd3, 0xc6, + 0x46, 0xf0, 0xaa, 0x71, 0x01, 0x2d, 0xc2, 0x42, 0xdb, 0xb6, 0xb1, 0x1b, 0x9a, 0x54, 0xe8, 0xe4, + 0x1e, 0x76, 0xfb, 0x38, 0x34, 0x99, 0xdb, 0xf8, 0x76, 0x11, 0xca, 0xb4, 0xed, 0xba, 0xed, 0x38, + 0xae, 0x81, 0x86, 0x80, 0xd8, 0x37, 0x43, 0x6b, 0xe8, 0xd8, 0xf2, 0xe3, 0x3a, 0x7a, 0x3b, 0xc5, + 0x98, 0xe3, 0xa4, 0x22, 0x74, 0x5b, 0xd7, 0x53, 0x38, 0x62, 0xe4, 0xea, 0x05, 0x64, 0x31, 0x89, + 0x34, 0xd9, 0x1d, 0x98, 0xbd, 0xfb, 0xfe, 0xed, 0xf0, 0x04, 0x89, 0x31, 0x52, 0x5f, 0x62, 0xec, + 0x9b, 0xbd, 0x18, 0xf0, 0x0f, 0xbb, 0xfe, 0x49, 0xa9, 0x17, 0xd0, 0x03, 0xb8, 0xb4, 0x83, 0x49, + 0xf0, 0x2d, 0xcf, 0x17, 0xb8, 0x91, 0x2e, 0x70, 0x8c, 0x78, 0x46, 0x91, 0xbb, 0x50, 0x64, 0x55, + 0x29, 0x4a, 0x8a, 0x8e, 0xf0, 0x1f, 0x66, 0xad, 0x95, 0x74, 0x02, 0xb9, 0xda, 0x4f, 0x61, 0x21, + 0xf6, 0x07, 0x0d, 0x7a, 0x23, 0x81, 0x2d, 0xf9, 0x5f, 0xa8, 0xd6, 0x8d, 0x2c, 0xa4, 0x52, 0x56, + 0x1f, 0xea, 0xd1, 0x2f, 0x8e, 0x68, 0x35, 0x81, 0x3f, 0xf1, 0xef, 0x87, 0xd6, 0x1b, 0x19, 0x28, + 0xa5, 0x20, 0x0b, 0x1a, 0xf1, 0x3f, 0x3a, 0xd0, 0x8d, 0x89, 0x0b, 0x44, 0xdd, 0xed, 0xcd, 0x4c, + 0xb4, 0x52, 0xdc, 0x29, 0x73, 0x82, 0xb1, 0x3f, 0x0a, 0xd0, 0x5a, 0xf2, 0x32, 0x69, 0xbf, 0x3a, + 0xb4, 0xd6, 0x33, 0xd3, 0x4b, 0xd1, 0x3f, 0xe3, 0xdd, 0x70, 0xd2, 0x57, 0x79, 0x74, 0x2b, 0x79, + 0xb9, 0x09, 0xbf, 0x13, 0xb4, 0x36, 0x66, 0x61, 0x91, 0x4a, 0x7c, 0xc5, 0xda, 0xd8, 0x84, 0x2f, + 0xdb, 0xf1, 0xb8, 0xf3, 0xd7, 0x4b, 0xff, 0x64, 0xdf, 0xba, 0x35, 0x03, 0x87, 0x54, 0xc0, 0x89, + 0xff, 0x33, 0xe3, 0x87, 0xe1, 0xfa, 0x54, 0xaf, 0x39, 0x5b, 0x0c, 0x7e, 0x01, 0x0b, 0xb1, 0x6b, + 0xf8, 0xc4, 0xa8, 0x49, 0xbe, 0xaa, 0x6f, 0x4d, 0x02, 0x74, 0x1e, 0x92, 0xb1, 0x5b, 0x01, 0x94, + 0xe2, 0xfd, 0x09, 0x37, 0x07, 0xad, 0x1b, 0x59, 0x48, 0xe5, 0x46, 0x3c, 0x06, 0x97, 0xb1, 0xce, + 0x1a, 0xdd, 0x4c, 0x5e, 0x23, 0xf9, 0x56, 0xa0, 0xf5, 0x56, 0x46, 0x6a, 0x29, 0xb4, 0x0b, 0xb0, + 0x83, 0xc9, 0x1e, 0x26, 0x2e, 0xf5, 0x91, 0xeb, 0x89, 0x26, 0x0f, 0x08, 0x7c, 0x31, 0xaf, 0x4f, + 0xa5, 0x93, 0x02, 0x7e, 0x04, 0xc8, 0xcf, 0x73, 0xa1, 0x8f, 0x40, 0xaf, 0x4c, 0x6c, 0x60, 0x78, + 0xb7, 0x31, 0xed, 0x6c, 0x1e, 0x40, 0x63, 0x4f, 0xb7, 0x69, 0xd2, 0x0e, 0xd6, 0xbd, 0x99, 0xa8, + 0x58, 0x9c, 0x2c, 0xc5, 0x5a, 0xa9, 0xd4, 0x72, 0x33, 0x0f, 0x65, 0x0e, 0xd5, 0x65, 0x08, 0xe2, + 0x38, 0xb6, 0x04, 0xd6, 0x88, 0x11, 0xa6, 0x60, 0xcb, 0x04, 0x7a, 0x29, 0xf8, 0xb1, 0xc2, 0xfe, + 0xcc, 0x8a, 0x11, 0xdc, 0x33, 0xc9, 0x31, 0xed, 0x6b, 0xbd, 0x2c, 0x2a, 0x30, 0xc2, 0x19, 0x54, + 0x10, 0xf4, 0x52, 0x05, 0x03, 0x6a, 0x91, 0xf6, 0x00, 0x25, 0x7d, 0xc9, 0x49, 0x6a, 0x50, 0x5a, + 0xab, 0xd3, 0x09, 0xa5, 0x94, 0x63, 0xa8, 0xf9, 0xfe, 0xca, 0x8d, 0xfb, 0x46, 0x9a, 0xa6, 0x01, + 0x4d, 0x4a, 0xb8, 0x25, 0x93, 0x86, 0xc3, 0x6d, 0xbc, 0xf0, 0x43, 0xd9, 0x3a, 0x86, 0x49, 0xe1, + 0x96, 0x5e, 0x4d, 0xaa, 0x17, 0x36, 0xfe, 0x5d, 0x80, 0x92, 0x7f, 0x13, 0xfe, 0x1c, 0x2a, 0xb2, + 0xe7, 0x50, 0x22, 0x7d, 0x01, 0x0b, 0xb1, 0xdf, 0x66, 0x12, 0x11, 0x34, 0xf9, 0xd7, 0x9a, 0x69, + 0x10, 0x70, 0x4f, 0xfc, 0x4c, 0x2f, 0xd1, 0xf2, 0xf5, 0xb4, 0x32, 0x2b, 0x0e, 0x94, 0x53, 0x16, + 0x7e, 0xea, 0xb0, 0x78, 0x07, 0x20, 0x04, 0x5b, 0x93, 0xef, 0x73, 0x68, 0x24, 0x4e, 0x51, 0x78, + 0xf3, 0x9d, 0x1f, 0xdf, 0xea, 0x9b, 0xe4, 0x78, 0x74, 0x48, 0xdf, 0xac, 0x73, 0xd2, 0xb7, 0x4c, + 0x47, 0x3c, 0xad, 0xfb, 0x27, 0xba, 0xce, 0xb8, 0xd7, 0xa9, 0x80, 0xe1, 0xe1, 0xe1, 0x1c, 0x1b, + 0xbd, 0xf3, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x14, 0x2c, 0xc2, 0xf3, 0x6e, 0x31, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/proto/internal.proto b/internal/proto/internal.proto index d14aa3bd77..2c0892e7a5 100644 --- a/internal/proto/internal.proto +++ b/internal/proto/internal.proto @@ -222,21 +222,6 @@ message LoadIndex { repeated common.KeyValuePair index_params = 6; } -message SegmentStatisticsUpdates { - int64 SegmentID = 1; - int64 MemorySize = 2; - int64 NumRows = 3; - uint64 create_time = 4; - uint64 end_time = 5; - internal.MsgPosition start_position = 6; - internal.MsgPosition end_position = 7; -} - -message SegmentStatistics { - common.MsgBase base = 1; - repeated SegmentStatisticsUpdates SegStats = 2; -} - message IndexStats { repeated common.KeyValuePair index_params = 1; int64 num_related_segments = 2; diff --git a/internal/proto/internalpb/internal.pb.go b/internal/proto/internalpb/internal.pb.go index c24f8610bd..db17e43fd1 100644 --- a/internal/proto/internalpb/internal.pb.go +++ b/internal/proto/internalpb/internal.pb.go @@ -1892,140 +1892,6 @@ func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair { return nil } -type SegmentStatisticsUpdates struct { - SegmentID int64 `protobuf:"varint,1,opt,name=SegmentID,proto3" json:"SegmentID,omitempty"` - MemorySize int64 `protobuf:"varint,2,opt,name=MemorySize,proto3" json:"MemorySize,omitempty"` - NumRows int64 `protobuf:"varint,3,opt,name=NumRows,proto3" json:"NumRows,omitempty"` - CreateTime uint64 `protobuf:"varint,4,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` - EndTime uint64 `protobuf:"varint,5,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` - StartPosition *MsgPosition `protobuf:"bytes,6,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` - EndPosition *MsgPosition `protobuf:"bytes,7,opt,name=end_position,json=endPosition,proto3" json:"end_position,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SegmentStatisticsUpdates) Reset() { *m = SegmentStatisticsUpdates{} } -func (m *SegmentStatisticsUpdates) String() string { return proto.CompactTextString(m) } -func (*SegmentStatisticsUpdates) ProtoMessage() {} -func (*SegmentStatisticsUpdates) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{25} -} - -func (m *SegmentStatisticsUpdates) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SegmentStatisticsUpdates.Unmarshal(m, b) -} -func (m *SegmentStatisticsUpdates) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SegmentStatisticsUpdates.Marshal(b, m, deterministic) -} -func (m *SegmentStatisticsUpdates) XXX_Merge(src proto.Message) { - xxx_messageInfo_SegmentStatisticsUpdates.Merge(m, src) -} -func (m *SegmentStatisticsUpdates) XXX_Size() int { - return xxx_messageInfo_SegmentStatisticsUpdates.Size(m) -} -func (m *SegmentStatisticsUpdates) XXX_DiscardUnknown() { - xxx_messageInfo_SegmentStatisticsUpdates.DiscardUnknown(m) -} - -var xxx_messageInfo_SegmentStatisticsUpdates proto.InternalMessageInfo - -func (m *SegmentStatisticsUpdates) GetSegmentID() int64 { - if m != nil { - return m.SegmentID - } - return 0 -} - -func (m *SegmentStatisticsUpdates) GetMemorySize() int64 { - if m != nil { - return m.MemorySize - } - return 0 -} - -func (m *SegmentStatisticsUpdates) GetNumRows() int64 { - if m != nil { - return m.NumRows - } - return 0 -} - -func (m *SegmentStatisticsUpdates) GetCreateTime() uint64 { - if m != nil { - return m.CreateTime - } - return 0 -} - -func (m *SegmentStatisticsUpdates) GetEndTime() uint64 { - if m != nil { - return m.EndTime - } - return 0 -} - -func (m *SegmentStatisticsUpdates) GetStartPosition() *MsgPosition { - if m != nil { - return m.StartPosition - } - return nil -} - -func (m *SegmentStatisticsUpdates) GetEndPosition() *MsgPosition { - if m != nil { - return m.EndPosition - } - return nil -} - -type SegmentStatistics struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - SegStats []*SegmentStatisticsUpdates `protobuf:"bytes,2,rep,name=SegStats,proto3" json:"SegStats,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SegmentStatistics) Reset() { *m = SegmentStatistics{} } -func (m *SegmentStatistics) String() string { return proto.CompactTextString(m) } -func (*SegmentStatistics) ProtoMessage() {} -func (*SegmentStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{26} -} - -func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SegmentStatistics.Unmarshal(m, b) -} -func (m *SegmentStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SegmentStatistics.Marshal(b, m, deterministic) -} -func (m *SegmentStatistics) XXX_Merge(src proto.Message) { - xxx_messageInfo_SegmentStatistics.Merge(m, src) -} -func (m *SegmentStatistics) XXX_Size() int { - return xxx_messageInfo_SegmentStatistics.Size(m) -} -func (m *SegmentStatistics) XXX_DiscardUnknown() { - xxx_messageInfo_SegmentStatistics.DiscardUnknown(m) -} - -var xxx_messageInfo_SegmentStatistics proto.InternalMessageInfo - -func (m *SegmentStatistics) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *SegmentStatistics) GetSegStats() []*SegmentStatisticsUpdates { - if m != nil { - return m.SegStats - } - return nil -} - type IndexStats struct { IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"` @@ -2038,7 +1904,7 @@ func (m *IndexStats) Reset() { *m = IndexStats{} } func (m *IndexStats) String() string { return proto.CompactTextString(m) } func (*IndexStats) ProtoMessage() {} func (*IndexStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{27} + return fileDescriptor_41f4a519b878ee3b, []int{25} } func (m *IndexStats) XXX_Unmarshal(b []byte) error { @@ -2086,7 +1952,7 @@ func (m *FieldStats) Reset() { *m = FieldStats{} } func (m *FieldStats) String() string { return proto.CompactTextString(m) } func (*FieldStats) ProtoMessage() {} func (*FieldStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{28} + return fileDescriptor_41f4a519b878ee3b, []int{26} } func (m *FieldStats) XXX_Unmarshal(b []byte) error { @@ -2142,7 +2008,7 @@ func (m *SegmentStats) Reset() { *m = SegmentStats{} } func (m *SegmentStats) String() string { return proto.CompactTextString(m) } func (*SegmentStats) ProtoMessage() {} func (*SegmentStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{29} + return fileDescriptor_41f4a519b878ee3b, []int{27} } func (m *SegmentStats) XXX_Unmarshal(b []byte) error { @@ -2204,7 +2070,7 @@ func (m *QueryNodeStats) Reset() { *m = QueryNodeStats{} } func (m *QueryNodeStats) String() string { return proto.CompactTextString(m) } func (*QueryNodeStats) ProtoMessage() {} func (*QueryNodeStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{30} + return fileDescriptor_41f4a519b878ee3b, []int{28} } func (m *QueryNodeStats) XXX_Unmarshal(b []byte) error { @@ -2260,7 +2126,7 @@ func (m *MsgPosition) Reset() { *m = MsgPosition{} } func (m *MsgPosition) String() string { return proto.CompactTextString(m) } func (*MsgPosition) ProtoMessage() {} func (*MsgPosition) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{31} + return fileDescriptor_41f4a519b878ee3b, []int{29} } func (m *MsgPosition) XXX_Unmarshal(b []byte) error { @@ -2323,7 +2189,7 @@ func (m *ChannelTimeTickMsg) Reset() { *m = ChannelTimeTickMsg{} } func (m *ChannelTimeTickMsg) String() string { return proto.CompactTextString(m) } func (*ChannelTimeTickMsg) ProtoMessage() {} func (*ChannelTimeTickMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{32} + return fileDescriptor_41f4a519b878ee3b, []int{30} } func (m *ChannelTimeTickMsg) XXX_Unmarshal(b []byte) error { @@ -2399,8 +2265,6 @@ func init() { proto.RegisterType((*RetrieveResults)(nil), "milvus.proto.internal.RetrieveResults") proto.RegisterType((*DeleteRequest)(nil), "milvus.proto.internal.DeleteRequest") proto.RegisterType((*LoadIndex)(nil), "milvus.proto.internal.LoadIndex") - proto.RegisterType((*SegmentStatisticsUpdates)(nil), "milvus.proto.internal.SegmentStatisticsUpdates") - proto.RegisterType((*SegmentStatistics)(nil), "milvus.proto.internal.SegmentStatistics") proto.RegisterType((*IndexStats)(nil), "milvus.proto.internal.IndexStats") proto.RegisterType((*FieldStats)(nil), "milvus.proto.internal.FieldStats") proto.RegisterType((*SegmentStats)(nil), "milvus.proto.internal.SegmentStats") @@ -2412,132 +2276,123 @@ func init() { func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) } var fileDescriptor_41f4a519b878ee3b = []byte{ - // 2017 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x73, 0x1c, 0x47, - 0xf5, 0xff, 0xce, 0xce, 0xae, 0x76, 0xf7, 0xcd, 0x4a, 0x5e, 0xb5, 0x65, 0x67, 0x64, 0x3b, 0xb1, - 0x32, 0xc9, 0x17, 0x44, 0x5c, 0x58, 0x46, 0x01, 0x92, 0xa2, 0x28, 0x1c, 0x5b, 0x1b, 0xcc, 0x96, - 0x23, 0x21, 0x46, 0x4e, 0xaa, 0xe0, 0x32, 0xd5, 0xbb, 0xd3, 0x5a, 0x0d, 0x9e, 0x5f, 0x99, 0xee, - 0x91, 0xbd, 0x39, 0x71, 0xe0, 0x04, 0x05, 0x55, 0x50, 0xc5, 0x11, 0x6e, 0x5c, 0xe1, 0xc8, 0x89, - 0x1f, 0xc5, 0x89, 0x7f, 0x81, 0x3f, 0x80, 0x7f, 0x82, 0x13, 0xd5, 0xaf, 0x7b, 0x7e, 0xec, 0x6a, - 0x25, 0x4b, 0x4a, 0x85, 0x98, 0xaa, 0xdc, 0xa6, 0xdf, 0x7b, 0xfd, 0xe3, 0x7d, 0xde, 0xe7, 0xbd, - 0x7e, 0xdb, 0x0b, 0x2b, 0x41, 0x2c, 0x58, 0x16, 0xd3, 0xf0, 0x6e, 0x9a, 0x25, 0x22, 0x21, 0xd7, - 0xa2, 0x20, 0x3c, 0xce, 0xb9, 0x1a, 0xdd, 0x2d, 0x94, 0x37, 0x7a, 0xe3, 0x24, 0x8a, 0x92, 0x58, - 0x89, 0x6f, 0xf4, 0xf8, 0xf8, 0x88, 0x45, 0x54, 0x8d, 0x9c, 0xbf, 0x18, 0xb0, 0xbc, 0x93, 0x44, - 0x69, 0x12, 0xb3, 0x58, 0x0c, 0xe3, 0xc3, 0x84, 0x5c, 0x87, 0xa5, 0x38, 0xf1, 0xd9, 0x70, 0x60, - 0x1b, 0x1b, 0xc6, 0xa6, 0xe9, 0xea, 0x11, 0x21, 0xd0, 0xcc, 0x92, 0x90, 0xd9, 0x8d, 0x0d, 0x63, - 0xb3, 0xeb, 0xe2, 0x37, 0xb9, 0x0f, 0xc0, 0x05, 0x15, 0xcc, 0x1b, 0x27, 0x3e, 0xb3, 0xcd, 0x0d, - 0x63, 0x73, 0x65, 0x7b, 0xe3, 0xee, 0xc2, 0x53, 0xdc, 0x3d, 0x90, 0x86, 0x3b, 0x89, 0xcf, 0xdc, - 0x2e, 0x2f, 0x3e, 0xc9, 0x7b, 0x00, 0xec, 0xb9, 0xc8, 0xa8, 0x17, 0xc4, 0x87, 0x89, 0xdd, 0xdc, - 0x30, 0x37, 0xad, 0xed, 0xd7, 0x67, 0x17, 0xd0, 0x87, 0x7f, 0xcc, 0xa6, 0x1f, 0xd1, 0x30, 0x67, - 0xfb, 0x34, 0xc8, 0xdc, 0x2e, 0x4e, 0x92, 0xc7, 0x75, 0xfe, 0x69, 0xc0, 0x95, 0xd2, 0x01, 0xdc, - 0x83, 0x93, 0x6f, 0x41, 0x0b, 0xb7, 0x40, 0x0f, 0xac, 0xed, 0x37, 0x4f, 0x39, 0xd1, 0x8c, 0xdf, - 0xae, 0x9a, 0x42, 0x3e, 0x84, 0xab, 0x3c, 0x1f, 0x8d, 0x0b, 0x95, 0x87, 0x52, 0x6e, 0x37, 0xf0, - 0x68, 0xe7, 0x5b, 0x89, 0xd4, 0x17, 0xd0, 0x47, 0x7a, 0x1b, 0x96, 0xe4, 0x4a, 0x39, 0x47, 0x94, - 0xac, 0xed, 0x9b, 0x0b, 0x9d, 0x3c, 0x40, 0x13, 0x57, 0x9b, 0x3a, 0x37, 0x61, 0xfd, 0x11, 0x13, - 0x73, 0xde, 0xb9, 0xec, 0xe3, 0x9c, 0x71, 0xa1, 0x95, 0x4f, 0x82, 0x88, 0x3d, 0x09, 0xc6, 0x4f, - 0x77, 0x8e, 0x68, 0x1c, 0xb3, 0xb0, 0x50, 0xbe, 0x0a, 0x37, 0x1f, 0x31, 0x9c, 0x10, 0x70, 0x11, - 0x8c, 0xf9, 0x9c, 0xfa, 0x1a, 0x5c, 0x7d, 0xc4, 0xc4, 0xc0, 0x9f, 0x13, 0x7f, 0x04, 0x9d, 0x3d, - 0x19, 0x6c, 0x49, 0x83, 0x6f, 0x42, 0x9b, 0xfa, 0x7e, 0xc6, 0x38, 0xd7, 0x28, 0xde, 0x5a, 0x78, - 0xe2, 0x07, 0xca, 0xc6, 0x2d, 0x8c, 0x17, 0xd1, 0xc4, 0xf9, 0x31, 0xc0, 0x30, 0x0e, 0xc4, 0x3e, - 0xcd, 0x68, 0xc4, 0x4f, 0x25, 0xd8, 0x00, 0x7a, 0x5c, 0xd0, 0x4c, 0x78, 0x29, 0xda, 0x69, 0xc8, - 0xcf, 0xc1, 0x06, 0x0b, 0xa7, 0xa9, 0xd5, 0x9d, 0x1f, 0x02, 0x1c, 0x88, 0x2c, 0x88, 0x27, 0x1f, - 0x04, 0x5c, 0xc8, 0xbd, 0x8e, 0xa5, 0x9d, 0x74, 0xc2, 0xdc, 0xec, 0xba, 0x7a, 0x54, 0x0b, 0x47, - 0xe3, 0xfc, 0xe1, 0xb8, 0x0f, 0x56, 0x01, 0xf7, 0x2e, 0x9f, 0x90, 0x7b, 0xd0, 0x1c, 0x51, 0xce, - 0xce, 0x84, 0x67, 0x97, 0x4f, 0x1e, 0x52, 0xce, 0x5c, 0xb4, 0x74, 0x7e, 0x66, 0xc2, 0x2b, 0x3b, - 0x19, 0x43, 0xf2, 0x87, 0x21, 0x1b, 0x8b, 0x20, 0x89, 0x35, 0xf6, 0x17, 0x5f, 0x8d, 0xbc, 0x02, - 0x6d, 0x7f, 0xe4, 0xc5, 0x34, 0x2a, 0xc0, 0x5e, 0xf2, 0x47, 0x7b, 0x34, 0x62, 0xe4, 0x4b, 0xb0, - 0x32, 0x2e, 0xd7, 0x97, 0x12, 0xe4, 0x5c, 0xd7, 0x9d, 0x93, 0x92, 0x37, 0x61, 0x39, 0xa5, 0x99, - 0x08, 0x4a, 0xb3, 0x26, 0x9a, 0xcd, 0x0a, 0x65, 0x40, 0xfd, 0xd1, 0x70, 0x60, 0xb7, 0x30, 0x58, - 0xf8, 0x4d, 0x1c, 0xe8, 0x55, 0x6b, 0x0d, 0x07, 0xf6, 0x12, 0xea, 0x66, 0x64, 0x64, 0x03, 0xac, - 0x72, 0xa1, 0xe1, 0xc0, 0x6e, 0xa3, 0x49, 0x5d, 0x24, 0x83, 0xa3, 0x6a, 0x91, 0xdd, 0xd9, 0x30, - 0x36, 0x7b, 0xae, 0x1e, 0x91, 0x7b, 0x70, 0xf5, 0x38, 0xc8, 0x44, 0x4e, 0x43, 0xcd, 0x4f, 0x79, - 0x0e, 0x6e, 0x77, 0x31, 0x82, 0x8b, 0x54, 0x64, 0x1b, 0xd6, 0xd2, 0xa3, 0x29, 0x0f, 0xc6, 0x73, - 0x53, 0x00, 0xa7, 0x2c, 0xd4, 0x39, 0x7f, 0x37, 0xe0, 0xda, 0x20, 0x4b, 0xd2, 0x97, 0x22, 0x14, - 0x05, 0xc8, 0xcd, 0x33, 0x40, 0x6e, 0x9d, 0x04, 0xd9, 0xf9, 0x45, 0x03, 0xae, 0x2b, 0x46, 0xed, - 0x17, 0xc0, 0x7e, 0x06, 0x5e, 0x7c, 0x19, 0xae, 0x54, 0xbb, 0x2a, 0x83, 0xc5, 0x6e, 0xfc, 0x3f, - 0xac, 0x94, 0x01, 0x56, 0x76, 0xff, 0x5d, 0x4a, 0x39, 0x3f, 0x6f, 0xc0, 0x9a, 0x0c, 0xea, 0x17, - 0x68, 0x48, 0x34, 0x7e, 0x67, 0x00, 0x51, 0xec, 0x78, 0x10, 0x06, 0x94, 0x7f, 0x9e, 0x58, 0xac, - 0x41, 0x8b, 0xca, 0x33, 0x68, 0x08, 0xd4, 0xc0, 0xe1, 0xd0, 0x97, 0xd1, 0xfa, 0xac, 0x4e, 0x57, - 0x6e, 0x6a, 0xd6, 0x37, 0xfd, 0xad, 0x01, 0xab, 0x0f, 0x42, 0xc1, 0xb2, 0x97, 0x14, 0x94, 0xbf, - 0x36, 0x8a, 0xa8, 0x0d, 0x63, 0x9f, 0x3d, 0xff, 0x3c, 0x0f, 0xf8, 0x2a, 0xc0, 0x61, 0xc0, 0x42, - 0xbf, 0xce, 0xde, 0x2e, 0x4a, 0x3e, 0x15, 0x73, 0x6d, 0x68, 0xe3, 0x22, 0x25, 0x6b, 0x8b, 0xa1, - 0xec, 0x01, 0x54, 0x3f, 0xa8, 0x7b, 0x80, 0xce, 0xb9, 0x7b, 0x00, 0x9c, 0xa6, 0x7b, 0x80, 0x3f, - 0x98, 0xb0, 0x3c, 0x8c, 0x39, 0xcb, 0xc4, 0xe5, 0xc1, 0xbb, 0x05, 0x5d, 0x7e, 0x44, 0x33, 0x74, - 0x54, 0xc3, 0x57, 0x09, 0xea, 0xd0, 0x9a, 0x2f, 0x82, 0xb6, 0x79, 0xce, 0xe2, 0xd0, 0x3a, 0xab, - 0x38, 0x2c, 0x9d, 0x01, 0x71, 0xfb, 0xc5, 0xc5, 0xa1, 0x73, 0xf2, 0xf6, 0x95, 0x0e, 0xb2, 0x49, - 0x24, 0x9b, 0xd6, 0x81, 0xdd, 0x45, 0x7d, 0x25, 0x20, 0xaf, 0x01, 0x88, 0x20, 0x62, 0x5c, 0xd0, - 0x28, 0x55, 0xf7, 0x68, 0xd3, 0xad, 0x49, 0xe4, 0xdd, 0x9d, 0x25, 0xcf, 0x86, 0x03, 0x6e, 0x5b, - 0x1b, 0xa6, 0x6c, 0xe2, 0xd4, 0x88, 0x7c, 0x1d, 0x3a, 0x59, 0xf2, 0xcc, 0xf3, 0xa9, 0xa0, 0x76, - 0x0f, 0x83, 0xb7, 0xbe, 0x10, 0xec, 0x87, 0x61, 0x32, 0x72, 0xdb, 0x59, 0xf2, 0x6c, 0x40, 0x05, - 0x75, 0x7e, 0xdf, 0x84, 0xe5, 0x03, 0x46, 0xb3, 0xf1, 0xd1, 0xe5, 0x03, 0xf6, 0x15, 0xe8, 0x67, - 0x8c, 0xe7, 0xa1, 0xf0, 0xc6, 0xea, 0x9a, 0x1f, 0x0e, 0x74, 0xdc, 0xae, 0x28, 0xf9, 0x4e, 0x21, - 0x2e, 0x41, 0x35, 0xcf, 0x00, 0xb5, 0xb9, 0x00, 0x54, 0x07, 0x7a, 0x35, 0x04, 0xb9, 0xdd, 0x42, - 0xd7, 0x67, 0x64, 0xa4, 0x0f, 0xa6, 0xcf, 0x43, 0x8c, 0x57, 0xd7, 0x95, 0x9f, 0xe4, 0x0e, 0xac, - 0xa6, 0x21, 0x1d, 0xb3, 0xa3, 0x24, 0xf4, 0x59, 0xe6, 0x4d, 0xb2, 0x24, 0x4f, 0x31, 0x66, 0x3d, - 0xb7, 0x5f, 0x53, 0x3c, 0x92, 0x72, 0xf2, 0x0e, 0x74, 0x7c, 0x1e, 0x7a, 0x62, 0x9a, 0x32, 0x0c, - 0xda, 0xca, 0x29, 0xbe, 0x0f, 0x78, 0xf8, 0x64, 0x9a, 0x32, 0xb7, 0xed, 0xab, 0x0f, 0x72, 0x0f, - 0xd6, 0x38, 0xcb, 0x02, 0x1a, 0x06, 0x9f, 0x30, 0xdf, 0x63, 0xcf, 0xd3, 0xcc, 0x4b, 0x43, 0x1a, - 0x63, 0x64, 0x7b, 0x2e, 0xa9, 0x74, 0xef, 0x3f, 0x4f, 0xb3, 0xfd, 0x90, 0xc6, 0x64, 0x13, 0xfa, - 0x49, 0x2e, 0xd2, 0x5c, 0x78, 0x98, 0x7d, 0xdc, 0x0b, 0x7c, 0x0c, 0xb4, 0xe9, 0xae, 0x28, 0xf9, - 0x77, 0x51, 0x3c, 0xf4, 0x25, 0xb4, 0x22, 0xa3, 0xc7, 0x2c, 0xf4, 0x4a, 0x06, 0xd8, 0xd6, 0x86, - 0xb1, 0xd9, 0x74, 0xaf, 0x28, 0xf9, 0x93, 0x42, 0x4c, 0xb6, 0xe0, 0xea, 0x24, 0xa7, 0x19, 0x8d, - 0x05, 0x63, 0x35, 0xeb, 0x1e, 0x5a, 0x93, 0x52, 0x55, 0x4d, 0xb8, 0x03, 0xab, 0xd2, 0x2c, 0xc9, - 0x45, 0xcd, 0x7c, 0x19, 0xcd, 0xfb, 0x5a, 0x51, 0x1a, 0x3b, 0xbf, 0xaa, 0xf1, 0x44, 0x86, 0x94, - 0x5f, 0x82, 0x27, 0x97, 0x69, 0xfd, 0x17, 0x92, 0xcb, 0x5c, 0x4c, 0xae, 0xdb, 0x60, 0x45, 0x4c, - 0x64, 0xc1, 0x58, 0x05, 0x51, 0x65, 0x3f, 0x28, 0x11, 0x46, 0xea, 0x36, 0x58, 0x71, 0x1e, 0x79, - 0x1f, 0xe7, 0x2c, 0x0b, 0x18, 0xd7, 0xc5, 0x13, 0xe2, 0x3c, 0xfa, 0x81, 0x92, 0x90, 0xab, 0xd0, - 0x12, 0x49, 0xea, 0x3d, 0x2d, 0x92, 0x5e, 0x24, 0xe9, 0x63, 0xf2, 0x6d, 0xb8, 0xc1, 0x19, 0x0d, - 0x99, 0xef, 0x95, 0x49, 0xca, 0x3d, 0x8e, 0x58, 0x30, 0xdf, 0x6e, 0x63, 0xdc, 0x6c, 0x65, 0x71, - 0x50, 0x1a, 0x1c, 0x68, 0xbd, 0x0c, 0x4b, 0x79, 0xf0, 0xda, 0xb4, 0x0e, 0xf6, 0xc7, 0xa4, 0x52, - 0x95, 0x13, 0xde, 0x05, 0x7b, 0x12, 0x26, 0x23, 0x1a, 0x7a, 0x27, 0x76, 0xc5, 0x46, 0xdc, 0x74, - 0xaf, 0x2b, 0xfd, 0xc1, 0xdc, 0x96, 0xd2, 0x3d, 0x1e, 0x06, 0x63, 0xe6, 0x7b, 0xa3, 0x30, 0x19, - 0xd9, 0x80, 0xfc, 0x03, 0x25, 0x92, 0x59, 0x2f, 0x79, 0xa7, 0x0d, 0x24, 0x0c, 0xe3, 0x24, 0x8f, - 0x05, 0xb2, 0xc9, 0x74, 0x57, 0x94, 0x7c, 0x2f, 0x8f, 0x76, 0xa4, 0x94, 0xbc, 0x01, 0xcb, 0xda, - 0x32, 0x39, 0x3c, 0xe4, 0x4c, 0x20, 0x8d, 0x4c, 0xb7, 0xa7, 0x84, 0xdf, 0x47, 0x99, 0xf3, 0x47, - 0x13, 0xae, 0xb8, 0x12, 0x5d, 0x76, 0xcc, 0xfe, 0xe7, 0xab, 0xc7, 0x69, 0x59, 0xbc, 0x74, 0xa1, - 0x2c, 0x6e, 0x9f, 0x3b, 0x8b, 0x3b, 0x17, 0xca, 0xe2, 0xee, 0xc5, 0xb2, 0x18, 0x4e, 0xc9, 0xe2, - 0x3f, 0xcf, 0x44, 0xec, 0x65, 0xcd, 0xe3, 0xb7, 0xc0, 0x0c, 0x7c, 0xd5, 0x9a, 0x59, 0xdb, 0xf6, - 0xec, 0xe2, 0xfa, 0x09, 0x6d, 0x38, 0xe0, 0xae, 0x34, 0x22, 0xf7, 0xc1, 0xd2, 0xe8, 0xe3, 0xc5, - 0xd7, 0xc2, 0x8b, 0xef, 0xb5, 0x85, 0x73, 0x30, 0x1c, 0xf2, 0xd2, 0x73, 0x55, 0x6b, 0xc5, 0xe5, - 0x37, 0xf9, 0x0e, 0xdc, 0x3c, 0x99, 0xdd, 0x99, 0xc6, 0xc8, 0xb7, 0x97, 0x30, 0xa0, 0xeb, 0xf3, - 0xe9, 0x5d, 0x80, 0xe8, 0x93, 0xaf, 0xc1, 0x5a, 0x2d, 0xbf, 0xab, 0x89, 0x6d, 0xf5, 0x9b, 0xb9, - 0xd2, 0x55, 0x53, 0xce, 0xca, 0xf0, 0xce, 0x59, 0x19, 0xee, 0xfc, 0xab, 0x01, 0xcb, 0x03, 0x16, - 0x32, 0xc1, 0xbe, 0x68, 0xaf, 0x4e, 0x6d, 0xaf, 0x5e, 0x87, 0x5e, 0x9a, 0x05, 0x11, 0xcd, 0xa6, - 0xde, 0x53, 0x36, 0x2d, 0x8a, 0xa6, 0xa5, 0x65, 0x8f, 0xd9, 0x94, 0xbf, 0xa8, 0xc7, 0x72, 0xfe, - 0x6d, 0x40, 0xf7, 0x83, 0x84, 0xfa, 0xf8, 0x33, 0xe0, 0x92, 0x18, 0x97, 0x1d, 0x5e, 0x63, 0xbe, - 0xc3, 0xbb, 0x05, 0x55, 0x27, 0xaf, 0x51, 0xae, 0xb5, 0xf6, 0xb5, 0x16, 0xbd, 0x39, 0xdb, 0xa2, - 0xdf, 0x06, 0x2b, 0x90, 0x07, 0xf2, 0x52, 0x2a, 0x8e, 0x54, 0x15, 0xeb, 0xba, 0x80, 0xa2, 0x7d, - 0x29, 0x91, 0x3d, 0x7c, 0x61, 0x80, 0x3d, 0xfc, 0xd2, 0xb9, 0x7b, 0x78, 0xbd, 0x08, 0xf6, 0xf0, - 0x7f, 0x6b, 0x80, 0xad, 0x39, 0x57, 0x3d, 0x63, 0x7e, 0x98, 0xfa, 0xf8, 0x9a, 0x7a, 0x0b, 0xba, - 0x25, 0x1f, 0xf5, 0x2b, 0x62, 0x25, 0x90, 0xb8, 0xee, 0xb2, 0x28, 0xc9, 0xa6, 0x07, 0xc1, 0x27, - 0x4c, 0x3b, 0x5e, 0x93, 0x48, 0xdf, 0xf6, 0xf2, 0xc8, 0x4d, 0x9e, 0x71, 0x5d, 0xc3, 0x8b, 0xa1, - 0xf4, 0x6d, 0x8c, 0xbf, 0xbc, 0xb0, 0x8a, 0xa1, 0xe7, 0x4d, 0x17, 0x94, 0x48, 0xd6, 0x2f, 0xb2, - 0x0e, 0x1d, 0x16, 0xfb, 0x4a, 0xdb, 0x42, 0x6d, 0x9b, 0xc5, 0x3e, 0xaa, 0x86, 0xb0, 0xa2, 0x9f, - 0x2f, 0x13, 0x8e, 0x24, 0x40, 0x52, 0x59, 0xdb, 0xce, 0x29, 0x6f, 0xc6, 0xbb, 0x7c, 0xb2, 0xaf, - 0x2d, 0xdd, 0x65, 0xf5, 0x82, 0xa9, 0x87, 0xe4, 0x7d, 0xe8, 0xc9, 0x5d, 0xca, 0x85, 0xda, 0xe7, - 0x5e, 0xc8, 0x62, 0xb1, 0x5f, 0x0c, 0x9c, 0x5f, 0x1b, 0xb0, 0x7a, 0x02, 0xc2, 0x4b, 0xf0, 0xe8, - 0x31, 0x74, 0x0e, 0xd8, 0x44, 0x2e, 0x51, 0x3c, 0xca, 0x6e, 0x9d, 0xf6, 0xc6, 0x7f, 0x4a, 0xc0, - 0xdc, 0x72, 0x01, 0xe7, 0xa7, 0x06, 0x00, 0x12, 0x1a, 0x87, 0x27, 0xc8, 0x62, 0x5c, 0x86, 0x2c, - 0xf2, 0xda, 0x94, 0xbd, 0x44, 0xc6, 0x42, 0x2a, 0xaa, 0x4a, 0xc6, 0x75, 0xec, 0x49, 0x9c, 0x47, - 0xae, 0x52, 0xe9, 0x03, 0x72, 0xe7, 0x97, 0x06, 0x00, 0x96, 0x62, 0x75, 0x8c, 0xf9, 0x9c, 0x37, - 0xce, 0xfe, 0xd5, 0xda, 0x98, 0x4d, 0x89, 0x87, 0x45, 0x4a, 0x70, 0xc4, 0xc8, 0x5c, 0xe4, 0x43, - 0x89, 0x51, 0xe5, 0xbc, 0xce, 0x1a, 0x85, 0xcb, 0x6f, 0x0c, 0xe8, 0xd5, 0xe0, 0xe3, 0xb3, 0xd9, - 0x6b, 0xcc, 0x67, 0x2f, 0x76, 0x99, 0x92, 0xd1, 0x1e, 0xaf, 0x91, 0x3c, 0xaa, 0x48, 0xbe, 0x0e, - 0x1d, 0x84, 0xa4, 0xc6, 0xf2, 0x58, 0xb3, 0xfc, 0x0e, 0xac, 0x66, 0x6c, 0xcc, 0x62, 0x11, 0x4e, - 0xbd, 0x28, 0xf1, 0x83, 0xc3, 0x80, 0xf9, 0xc8, 0xf5, 0x8e, 0xdb, 0x2f, 0x14, 0xbb, 0x5a, 0xee, - 0xfc, 0xc3, 0x80, 0x15, 0xd9, 0x98, 0x4e, 0xf7, 0x12, 0x9f, 0xa9, 0x93, 0x5d, 0x9c, 0x41, 0xef, - 0xa1, 0x2f, 0x1a, 0x1e, 0x45, 0xa1, 0x37, 0x5e, 0x4c, 0x21, 0xee, 0x76, 0xb8, 0xa6, 0x8d, 0x84, - 0x58, 0xbd, 0x44, 0x9c, 0x07, 0xe2, 0x2a, 0xb0, 0xfa, 0x92, 0x55, 0x10, 0xff, 0xc4, 0x00, 0xab, - 0x96, 0x2c, 0xb2, 0x44, 0xeb, 0x8b, 0x51, 0xdd, 0x10, 0x06, 0x16, 0x41, 0x6b, 0x5c, 0xbd, 0x12, - 0x93, 0x35, 0x68, 0x45, 0x7c, 0xa2, 0x23, 0xde, 0x73, 0xd5, 0x80, 0xdc, 0x80, 0x4e, 0xc4, 0x27, - 0xf8, 0x83, 0x4d, 0x57, 0xce, 0x72, 0x2c, 0xc3, 0x56, 0x75, 0x40, 0xaa, 0x80, 0x54, 0x02, 0xe7, - 0x4f, 0x06, 0x10, 0xdd, 0x62, 0x7c, 0xaa, 0xbf, 0x12, 0x90, 0xb0, 0xf5, 0x97, 0xee, 0x06, 0x96, - 0xe1, 0x19, 0xd9, 0xdc, 0xfd, 0x62, 0x9e, 0xf8, 0x0d, 0x7f, 0x07, 0x56, 0x7d, 0x76, 0x48, 0x65, - 0x37, 0x34, 0x7f, 0xe4, 0xbe, 0x56, 0x94, 0x4d, 0xdb, 0x5b, 0xef, 0x42, 0xb7, 0xfc, 0x07, 0x8f, - 0xf4, 0xa1, 0x37, 0x8c, 0x03, 0x81, 0xbd, 0x68, 0x10, 0x4f, 0xfa, 0xff, 0x47, 0x2c, 0x68, 0x7f, - 0x8f, 0xd1, 0x50, 0x1c, 0x4d, 0xfb, 0x06, 0xe9, 0x41, 0xe7, 0xc1, 0x28, 0x4e, 0xb2, 0x88, 0x86, - 0xfd, 0xc6, 0xc3, 0x77, 0x7e, 0xf4, 0x8d, 0x49, 0x20, 0x8e, 0xf2, 0x91, 0xf4, 0x64, 0x4b, 0xb9, - 0xf6, 0xd5, 0x20, 0xd1, 0x5f, 0x5b, 0x45, 0xd4, 0xb6, 0xd0, 0xdb, 0x72, 0x98, 0x8e, 0x46, 0x4b, - 0x28, 0x79, 0xfb, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x60, 0x9d, 0x07, 0x94, 0xe7, 0x1c, 0x00, - 0x00, + // 1886 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0x4f, 0x73, 0x1c, 0x47, + 0x15, 0x67, 0x76, 0x56, 0xda, 0xdd, 0xb7, 0x2b, 0x79, 0xdd, 0x92, 0x9d, 0xf1, 0x9f, 0xc4, 0xca, + 0x24, 0x80, 0x88, 0x0b, 0xdb, 0x28, 0x40, 0x52, 0x14, 0x85, 0x63, 0x69, 0xc1, 0x6c, 0x39, 0x36, + 0x62, 0x64, 0x52, 0x05, 0x97, 0xa9, 0xde, 0x9d, 0xd6, 0x6a, 0xf0, 0xcc, 0xf4, 0xa4, 0xbb, 0x47, + 0xf2, 0xfa, 0xc4, 0x81, 0x13, 0x14, 0x54, 0x71, 0xe0, 0x08, 0x37, 0xae, 0x70, 0xe4, 0x04, 0x54, + 0x71, 0xe2, 0x2b, 0xf0, 0x01, 0xf8, 0x12, 0x9c, 0xa8, 0x7e, 0xdd, 0x33, 0xfb, 0x47, 0x2b, 0x59, + 0x52, 0x2a, 0xc4, 0xa9, 0xca, 0xad, 0xfb, 0xf7, 0x5e, 0x77, 0xbf, 0xf7, 0x7b, 0xef, 0x75, 0xbf, + 0x9d, 0x85, 0xd5, 0x38, 0x53, 0x4c, 0x64, 0x34, 0xb9, 0x93, 0x0b, 0xae, 0x38, 0xb9, 0x92, 0xc6, + 0xc9, 0x61, 0x21, 0xcd, 0xec, 0x4e, 0x29, 0xbc, 0xde, 0x19, 0xf2, 0x34, 0xe5, 0x99, 0x81, 0xaf, + 0x77, 0xe4, 0xf0, 0x80, 0xa5, 0xd4, 0xcc, 0xfc, 0xbf, 0x3b, 0xb0, 0xb2, 0xc3, 0xd3, 0x9c, 0x67, + 0x2c, 0x53, 0xfd, 0x6c, 0x9f, 0x93, 0xab, 0xb0, 0x9c, 0xf1, 0x88, 0xf5, 0x7b, 0x9e, 0xb3, 0xe1, + 0x6c, 0xba, 0x81, 0x9d, 0x11, 0x02, 0x75, 0xc1, 0x13, 0xe6, 0xd5, 0x36, 0x9c, 0xcd, 0x56, 0x80, + 0x63, 0x72, 0x1f, 0x40, 0x2a, 0xaa, 0x58, 0x38, 0xe4, 0x11, 0xf3, 0xdc, 0x0d, 0x67, 0x73, 0x75, + 0x6b, 0xe3, 0xce, 0x42, 0x2b, 0xee, 0xec, 0x69, 0xc5, 0x1d, 0x1e, 0xb1, 0xa0, 0x25, 0xcb, 0x21, + 0xf9, 0x00, 0x80, 0x3d, 0x57, 0x82, 0x86, 0x71, 0xb6, 0xcf, 0xbd, 0xfa, 0x86, 0xbb, 0xd9, 0xde, + 0x7a, 0x73, 0x76, 0x03, 0x6b, 0xfc, 0x23, 0x36, 0xfe, 0x88, 0x26, 0x05, 0xdb, 0xa5, 0xb1, 0x08, + 0x5a, 0xb8, 0x48, 0x9b, 0xeb, 0xff, 0xdb, 0x81, 0x4b, 0x95, 0x03, 0x78, 0x86, 0x24, 0xdf, 0x81, + 0x25, 0x3c, 0x02, 0x3d, 0x68, 0x6f, 0xbd, 0x7d, 0x82, 0x45, 0x33, 0x7e, 0x07, 0x66, 0x09, 0xf9, + 0x09, 0xac, 0xc9, 0x62, 0x30, 0x2c, 0x45, 0x21, 0xa2, 0xd2, 0xab, 0xa1, 0x69, 0x67, 0xdb, 0x89, + 0x4c, 0x6f, 0x60, 0x4d, 0x7a, 0x17, 0x96, 0xf5, 0x4e, 0x85, 0x44, 0x96, 0xda, 0x5b, 0x37, 0x16, + 0x3a, 0xb9, 0x87, 0x2a, 0x81, 0x55, 0xf5, 0x6f, 0xc0, 0xb5, 0x87, 0x4c, 0xcd, 0x79, 0x17, 0xb0, + 0x8f, 0x0b, 0x26, 0x95, 0x15, 0x3e, 0x8d, 0x53, 0xf6, 0x34, 0x1e, 0x3e, 0xdb, 0x39, 0xa0, 0x59, + 0xc6, 0x92, 0x52, 0xf8, 0x3a, 0xdc, 0x78, 0xc8, 0x70, 0x41, 0x2c, 0x55, 0x3c, 0x94, 0x73, 0xe2, + 0x2b, 0xb0, 0xf6, 0x90, 0xa9, 0x5e, 0x34, 0x07, 0x7f, 0x04, 0xcd, 0x27, 0x3a, 0xd8, 0x3a, 0x0d, + 0xbe, 0x0d, 0x0d, 0x1a, 0x45, 0x82, 0x49, 0x69, 0x59, 0xbc, 0xb9, 0xd0, 0xe2, 0x07, 0x46, 0x27, + 0x28, 0x95, 0x17, 0xa5, 0x89, 0xff, 0x73, 0x80, 0x7e, 0x16, 0xab, 0x5d, 0x2a, 0x68, 0x2a, 0x4f, + 0x4c, 0xb0, 0x1e, 0x74, 0xa4, 0xa2, 0x42, 0x85, 0x39, 0xea, 0x59, 0xca, 0xcf, 0x90, 0x0d, 0x6d, + 0x5c, 0x66, 0x76, 0xf7, 0x7f, 0x0a, 0xb0, 0xa7, 0x44, 0x9c, 0x8d, 0x3e, 0x8c, 0xa5, 0xd2, 0x67, + 0x1d, 0x6a, 0x3d, 0xed, 0x84, 0xbb, 0xd9, 0x0a, 0xec, 0x6c, 0x2a, 0x1c, 0xb5, 0xb3, 0x87, 0xe3, + 0x3e, 0xb4, 0x4b, 0xba, 0x1f, 0xcb, 0x11, 0xb9, 0x07, 0xf5, 0x01, 0x95, 0xec, 0x54, 0x7a, 0x1e, + 0xcb, 0xd1, 0x36, 0x95, 0x2c, 0x40, 0x4d, 0xff, 0x57, 0x2e, 0xbc, 0xb6, 0x23, 0x18, 0x26, 0x7f, + 0x92, 0xb0, 0xa1, 0x8a, 0x79, 0x66, 0xb9, 0x3f, 0xff, 0x6e, 0xe4, 0x35, 0x68, 0x44, 0x83, 0x30, + 0xa3, 0x69, 0x49, 0xf6, 0x72, 0x34, 0x78, 0x42, 0x53, 0x46, 0xbe, 0x02, 0xab, 0xc3, 0x6a, 0x7f, + 0x8d, 0x60, 0xce, 0xb5, 0x82, 0x39, 0x94, 0xbc, 0x0d, 0x2b, 0x39, 0x15, 0x2a, 0xae, 0xd4, 0xea, + 0xa8, 0x36, 0x0b, 0xea, 0x80, 0x46, 0x83, 0x7e, 0xcf, 0x5b, 0xc2, 0x60, 0xe1, 0x98, 0xf8, 0xd0, + 0x99, 0xec, 0xd5, 0xef, 0x79, 0xcb, 0x28, 0x9b, 0xc1, 0xc8, 0x06, 0xb4, 0xab, 0x8d, 0xfa, 0x3d, + 0xaf, 0x81, 0x2a, 0xd3, 0x90, 0x0e, 0x8e, 0xb9, 0x8b, 0xbc, 0xe6, 0x86, 0xb3, 0xd9, 0x09, 0xec, + 0x8c, 0xdc, 0x83, 0xb5, 0xc3, 0x58, 0xa8, 0x82, 0x26, 0x36, 0x3f, 0xb5, 0x1d, 0xd2, 0x6b, 0x61, + 0x04, 0x17, 0x89, 0xc8, 0x16, 0xac, 0xe7, 0x07, 0x63, 0x19, 0x0f, 0xe7, 0x96, 0x00, 0x2e, 0x59, + 0x28, 0xf3, 0xff, 0xe9, 0xc0, 0x95, 0x9e, 0xe0, 0xf9, 0x2b, 0x11, 0x8a, 0x92, 0xe4, 0xfa, 0x29, + 0x24, 0x2f, 0x1d, 0x27, 0xd9, 0xff, 0x4d, 0x0d, 0xae, 0x9a, 0x8c, 0xda, 0x2d, 0x89, 0xfd, 0x14, + 0xbc, 0xf8, 0x2a, 0x5c, 0x9a, 0x9c, 0x6a, 0x14, 0x16, 0xbb, 0xf1, 0x65, 0x58, 0xad, 0x02, 0x6c, + 0xf4, 0xfe, 0xbf, 0x29, 0xe5, 0xff, 0xba, 0x06, 0xeb, 0x3a, 0xa8, 0x5f, 0xb0, 0xa1, 0xd9, 0xf8, + 0xa3, 0x03, 0xc4, 0x64, 0xc7, 0x83, 0x24, 0xa6, 0xf2, 0xb3, 0xe4, 0x62, 0x1d, 0x96, 0xa8, 0xb6, + 0xc1, 0x52, 0x60, 0x26, 0xbe, 0x84, 0xae, 0x8e, 0xd6, 0xa7, 0x65, 0x5d, 0x75, 0xa8, 0x3b, 0x7d, + 0xe8, 0x1f, 0x1c, 0xb8, 0xfc, 0x20, 0x51, 0x4c, 0xbc, 0xa2, 0xa4, 0xfc, 0xa3, 0x56, 0x46, 0xad, + 0x9f, 0x45, 0xec, 0xf9, 0x67, 0x69, 0xe0, 0xeb, 0x00, 0xfb, 0x31, 0x4b, 0xa2, 0xe9, 0xec, 0x6d, + 0x21, 0xf2, 0x89, 0x32, 0xd7, 0x83, 0x06, 0x6e, 0x52, 0x65, 0x6d, 0x39, 0xd5, 0x3d, 0x80, 0xe9, + 0x07, 0x6d, 0x0f, 0xd0, 0x3c, 0x73, 0x0f, 0x80, 0xcb, 0x6c, 0x0f, 0xf0, 0x67, 0x17, 0x56, 0xfa, + 0x99, 0x64, 0x42, 0x5d, 0x9c, 0xbc, 0x9b, 0xd0, 0x92, 0x07, 0x54, 0xa0, 0xa3, 0x96, 0xbe, 0x09, + 0x30, 0x4d, 0xad, 0xfb, 0x32, 0x6a, 0xeb, 0x67, 0xbc, 0x1c, 0x96, 0x4e, 0xbb, 0x1c, 0x96, 0x4f, + 0xa1, 0xb8, 0xf1, 0xf2, 0xcb, 0xa1, 0x79, 0xfc, 0xf5, 0xd5, 0x0e, 0xb2, 0x51, 0xaa, 0x9b, 0xd6, + 0x9e, 0xd7, 0x42, 0xf9, 0x04, 0x20, 0x6f, 0x00, 0xa8, 0x38, 0x65, 0x52, 0xd1, 0x34, 0x37, 0xef, + 0x68, 0x3d, 0x98, 0x42, 0xf4, 0xdb, 0x2d, 0xf8, 0x51, 0xbf, 0x27, 0xbd, 0xf6, 0x86, 0xab, 0x9b, + 0x38, 0x33, 0x23, 0xdf, 0x84, 0xa6, 0xe0, 0x47, 0x61, 0x44, 0x15, 0xf5, 0x3a, 0x18, 0xbc, 0x6b, + 0x0b, 0xc9, 0xde, 0x4e, 0xf8, 0x20, 0x68, 0x08, 0x7e, 0xd4, 0xa3, 0x8a, 0xfa, 0x7f, 0xaa, 0xc3, + 0xca, 0x1e, 0xa3, 0x62, 0x78, 0x70, 0xf1, 0x80, 0x7d, 0x0d, 0xba, 0x82, 0xc9, 0x22, 0x51, 0xe1, + 0xd0, 0x3c, 0xf3, 0xfd, 0x9e, 0x8d, 0xdb, 0x25, 0x83, 0xef, 0x94, 0x70, 0x45, 0xaa, 0x7b, 0x0a, + 0xa9, 0xf5, 0x05, 0xa4, 0xfa, 0xd0, 0x99, 0x62, 0x50, 0x7a, 0x4b, 0xe8, 0xfa, 0x0c, 0x46, 0xba, + 0xe0, 0x46, 0x32, 0xc1, 0x78, 0xb5, 0x02, 0x3d, 0x24, 0xb7, 0xe1, 0x72, 0x9e, 0xd0, 0x21, 0x3b, + 0xe0, 0x49, 0xc4, 0x44, 0x38, 0x12, 0xbc, 0xc8, 0x31, 0x66, 0x9d, 0xa0, 0x3b, 0x25, 0x78, 0xa8, + 0x71, 0xf2, 0x1e, 0x34, 0x23, 0x99, 0x84, 0x6a, 0x9c, 0x33, 0x0c, 0xda, 0xea, 0x09, 0xbe, 0xf7, + 0x64, 0xf2, 0x74, 0x9c, 0xb3, 0xa0, 0x11, 0x99, 0x01, 0xb9, 0x07, 0xeb, 0x92, 0x89, 0x98, 0x26, + 0xf1, 0x0b, 0x16, 0x85, 0xec, 0x79, 0x2e, 0xc2, 0x3c, 0xa1, 0x19, 0x46, 0xb6, 0x13, 0x90, 0x89, + 0xec, 0xfb, 0xcf, 0x73, 0xb1, 0x9b, 0xd0, 0x8c, 0x6c, 0x42, 0x97, 0x17, 0x2a, 0x2f, 0x54, 0x88, + 0xd5, 0x27, 0xc3, 0x38, 0xc2, 0x40, 0xbb, 0xc1, 0xaa, 0xc1, 0x7f, 0x80, 0x70, 0x3f, 0xd2, 0xd4, + 0x2a, 0x41, 0x0f, 0x59, 0x12, 0x56, 0x19, 0xe0, 0xb5, 0x37, 0x9c, 0xcd, 0x7a, 0x70, 0xc9, 0xe0, + 0x4f, 0x4b, 0x98, 0xdc, 0x85, 0xb5, 0x51, 0x41, 0x05, 0xcd, 0x14, 0x63, 0x53, 0xda, 0x1d, 0xd4, + 0x26, 0x95, 0x68, 0xb2, 0xe0, 0x36, 0x5c, 0xd6, 0x6a, 0xbc, 0x50, 0x53, 0xea, 0x2b, 0xa8, 0xde, + 0xb5, 0x82, 0x4a, 0xd9, 0xff, 0xdd, 0x54, 0x9e, 0xe8, 0x90, 0xca, 0x0b, 0xe4, 0xc9, 0x45, 0x5a, + 0xff, 0x85, 0xc9, 0xe5, 0x2e, 0x4e, 0xae, 0x5b, 0xd0, 0x4e, 0x99, 0x12, 0xf1, 0xd0, 0x04, 0xd1, + 0x54, 0x3f, 0x18, 0x08, 0x23, 0x75, 0x0b, 0xda, 0x59, 0x91, 0x86, 0x1f, 0x17, 0x4c, 0xc4, 0x4c, + 0xda, 0xcb, 0x13, 0xb2, 0x22, 0xfd, 0xb1, 0x41, 0xc8, 0x1a, 0x2c, 0x29, 0x9e, 0x87, 0xcf, 0xca, + 0xa2, 0x57, 0x3c, 0x7f, 0x44, 0xbe, 0x0b, 0xd7, 0x25, 0xa3, 0x09, 0x8b, 0xc2, 0xaa, 0x48, 0x65, + 0x28, 0x91, 0x0b, 0x16, 0x79, 0x0d, 0x8c, 0x9b, 0x67, 0x34, 0xf6, 0x2a, 0x85, 0x3d, 0x2b, 0xd7, + 0x61, 0xa9, 0x0c, 0x9f, 0x5a, 0xd6, 0xc4, 0xfe, 0x98, 0x4c, 0x44, 0xd5, 0x82, 0xf7, 0xc1, 0x1b, + 0x25, 0x7c, 0x40, 0x93, 0xf0, 0xd8, 0xa9, 0xd8, 0x88, 0xbb, 0xc1, 0x55, 0x23, 0xdf, 0x9b, 0x3b, + 0x52, 0xbb, 0x27, 0x93, 0x78, 0xc8, 0xa2, 0x70, 0x90, 0xf0, 0x81, 0x07, 0x98, 0x7f, 0x60, 0x20, + 0x5d, 0xf5, 0x3a, 0xef, 0xac, 0x82, 0xa6, 0x61, 0xc8, 0x8b, 0x4c, 0x61, 0x36, 0xb9, 0xc1, 0xaa, + 0xc1, 0x9f, 0x14, 0xe9, 0x8e, 0x46, 0xc9, 0x5b, 0xb0, 0x62, 0x35, 0xf9, 0xfe, 0xbe, 0x64, 0x0a, + 0xd3, 0xc8, 0x0d, 0x3a, 0x06, 0xfc, 0x11, 0x62, 0xfe, 0x5f, 0x5c, 0xb8, 0x14, 0x68, 0x76, 0xd9, + 0x21, 0xfb, 0xdc, 0xdf, 0x1e, 0x27, 0x55, 0xf1, 0xf2, 0xb9, 0xaa, 0xb8, 0x71, 0xe6, 0x2a, 0x6e, + 0x9e, 0xab, 0x8a, 0x5b, 0xe7, 0xab, 0x62, 0x38, 0xa1, 0x8a, 0xff, 0x36, 0x13, 0xb1, 0x57, 0xb5, + 0x8e, 0xdf, 0x01, 0x37, 0x8e, 0x4c, 0x6b, 0xd6, 0xde, 0xf2, 0x66, 0x37, 0xb7, 0x9f, 0xd0, 0xfa, + 0x3d, 0x19, 0x68, 0x25, 0x72, 0x1f, 0xda, 0x96, 0x7d, 0x7c, 0xf8, 0x96, 0xf0, 0xe1, 0x7b, 0x63, + 0xe1, 0x1a, 0x0c, 0x87, 0x7e, 0xf4, 0x02, 0xd3, 0x5a, 0x49, 0x3d, 0x26, 0xdf, 0x83, 0x1b, 0xc7, + 0xab, 0x5b, 0x58, 0x8e, 0x22, 0x6f, 0x19, 0x03, 0x7a, 0x6d, 0xbe, 0xbc, 0x4b, 0x12, 0x23, 0xf2, + 0x0d, 0x58, 0x9f, 0xaa, 0xef, 0xc9, 0xc2, 0x86, 0xf9, 0xcd, 0x3c, 0x91, 0x4d, 0x96, 0x9c, 0x56, + 0xe1, 0xcd, 0xd3, 0x2a, 0xdc, 0xff, 0x4f, 0x0d, 0x56, 0x7a, 0x2c, 0x61, 0x8a, 0x7d, 0xd1, 0x5e, + 0x9d, 0xd8, 0x5e, 0xbd, 0x09, 0x9d, 0x5c, 0xc4, 0x29, 0x15, 0xe3, 0xf0, 0x19, 0x1b, 0x97, 0x97, + 0x66, 0xdb, 0x62, 0x8f, 0xd8, 0x58, 0xbe, 0xac, 0xc7, 0xf2, 0xff, 0xeb, 0x40, 0xeb, 0x43, 0x4e, + 0x23, 0xfc, 0x19, 0x70, 0x41, 0x8e, 0xab, 0x0e, 0xaf, 0x36, 0xdf, 0xe1, 0xdd, 0x84, 0x49, 0x27, + 0x6f, 0x59, 0x9e, 0x6a, 0xed, 0xa7, 0x5a, 0xf4, 0xfa, 0x6c, 0x8b, 0x7e, 0x0b, 0xda, 0xb1, 0x36, + 0x28, 0xcc, 0xa9, 0x3a, 0x30, 0xb7, 0x58, 0x2b, 0x00, 0x84, 0x76, 0x35, 0xa2, 0x7b, 0xf8, 0x52, + 0x01, 0x7b, 0xf8, 0xe5, 0x33, 0xf7, 0xf0, 0x76, 0x13, 0xec, 0xe1, 0x7f, 0xe9, 0x00, 0xa0, 0xe3, + 0xba, 0x82, 0x8f, 0x6f, 0xea, 0x5c, 0x64, 0x53, 0x7d, 0xbd, 0xea, 0x37, 0x47, 0xb0, 0x84, 0xaa, + 0x49, 0xc6, 0x4b, 0x4b, 0x0e, 0xc9, 0x8a, 0x34, 0x30, 0x22, 0x9b, 0xed, 0xd2, 0xff, 0xad, 0x03, + 0x80, 0x25, 0x6b, 0xcc, 0x98, 0xcf, 0x0d, 0xe7, 0xf4, 0x5f, 0x37, 0xb5, 0x59, 0xea, 0xb6, 0x4b, + 0xea, 0xf4, 0x6d, 0xa4, 0x7f, 0x95, 0x2e, 0xf0, 0xa1, 0xfa, 0xa6, 0x3c, 0x71, 0xde, 0xb2, 0x8b, + 0x63, 0xff, 0xf7, 0x0e, 0x74, 0xac, 0x75, 0xc6, 0xa4, 0x99, 0x28, 0x3b, 0xf3, 0x51, 0xc6, 0x6e, + 0x24, 0xe5, 0x62, 0x1c, 0xca, 0xf8, 0x05, 0xb3, 0x06, 0x81, 0x81, 0xf6, 0xe2, 0x17, 0x8c, 0x5c, + 0x83, 0x26, 0x52, 0xc2, 0x8f, 0xa4, 0x7d, 0xd1, 0x1a, 0x9a, 0x06, 0x7e, 0x24, 0xf5, 0xa5, 0x2e, + 0xd8, 0x90, 0x65, 0x2a, 0x19, 0x87, 0x29, 0x8f, 0xe2, 0xfd, 0x98, 0x45, 0x98, 0x0d, 0xcd, 0xa0, + 0x5b, 0x0a, 0x1e, 0x5b, 0xdc, 0xff, 0x97, 0x03, 0xab, 0xba, 0x81, 0x19, 0x3f, 0xe1, 0x11, 0x33, + 0x96, 0x9d, 0x3f, 0x63, 0x3f, 0x40, 0x5f, 0x2c, 0x3d, 0xe6, 0xfb, 0xef, 0x5b, 0x27, 0xfd, 0x9d, + 0x30, 0xc5, 0x41, 0xd0, 0x94, 0x6c, 0x64, 0xce, 0xdc, 0xb6, 0x37, 0xf1, 0x99, 0x28, 0x9e, 0x04, + 0xd6, 0x5e, 0xc6, 0x86, 0xe2, 0x5f, 0x38, 0xd0, 0x7e, 0x2c, 0x47, 0xbb, 0x5c, 0x62, 0x31, 0xeb, + 0x52, 0xb6, 0x17, 0xa8, 0xb9, 0x49, 0x1c, 0x2c, 0x96, 0xf6, 0x70, 0xf2, 0x35, 0x51, 0xff, 0x92, + 0x4f, 0xe5, 0xc8, 0x46, 0xbc, 0x13, 0x98, 0x09, 0xb9, 0x0e, 0xcd, 0x54, 0x8e, 0xb0, 0xb1, 0xb7, + 0x15, 0x56, 0xcd, 0x75, 0xd8, 0x26, 0x2f, 0x65, 0x1d, 0x5f, 0xca, 0x09, 0xe0, 0xff, 0xd5, 0x01, + 0x62, 0x9f, 0xa2, 0x4f, 0xf4, 0xc9, 0x19, 0x13, 0x76, 0xfa, 0x8b, 0x68, 0x0d, 0xcb, 0x75, 0x06, + 0x9b, 0xbb, 0x87, 0xdc, 0x63, 0xbf, 0xf5, 0x6e, 0xc3, 0xe5, 0x88, 0xed, 0x53, 0xfd, 0x6a, 0xce, + 0x9b, 0xdc, 0xb5, 0x82, 0xea, 0x71, 0x7f, 0xe7, 0x7d, 0x68, 0x55, 0xff, 0xf4, 0x90, 0x2e, 0x74, + 0xfa, 0x59, 0xac, 0xb0, 0x67, 0x89, 0xb3, 0x51, 0xf7, 0x4b, 0xa4, 0x0d, 0x8d, 0x1f, 0x32, 0x9a, + 0xa8, 0x83, 0x71, 0xd7, 0x21, 0x1d, 0x68, 0x3e, 0x18, 0x64, 0x5c, 0xa4, 0x34, 0xe9, 0xd6, 0xb6, + 0xdf, 0xfb, 0xd9, 0xb7, 0x46, 0xb1, 0x3a, 0x28, 0x06, 0xda, 0x93, 0xbb, 0xc6, 0xb5, 0xaf, 0xc7, + 0xdc, 0x8e, 0xee, 0x96, 0x51, 0xbb, 0x8b, 0xde, 0x56, 0xd3, 0x7c, 0x30, 0x58, 0x46, 0xe4, 0xdd, + 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x48, 0xeb, 0xcc, 0x3e, 0x0f, 0x1b, 0x00, 0x00, }