Move segment statistics update to datanode tt loop (#13233)

Signed-off-by: sunby <bingyi.sun@zilliz.com>

Co-authored-by: sunby <bingyi.sun@zilliz.com>
This commit is contained in:
Bingyi Sun 2021-12-15 10:53:16 +08:00 committed by GitHub
parent 4023447506
commit 0c39e0370c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 558 additions and 901 deletions

View File

@ -69,7 +69,6 @@ type ParamTable struct {
// --- Channels --- // --- Channels ---
ClusterChannelPrefix string ClusterChannelPrefix string
InsertChannelPrefixName string InsertChannelPrefixName string
StatisticsChannelName string
TimeTickChannelName string TimeTickChannelName string
SegmentInfoChannelName string SegmentInfoChannelName string
DataCoordSubscriptionName string DataCoordSubscriptionName string
@ -117,7 +116,6 @@ func (p *ParamTable) Init() {
// Has to init global msgchannel prefix before other channel names // Has to init global msgchannel prefix before other channel names
p.initClusterMsgChannelPrefix() p.initClusterMsgChannelPrefix()
p.initInsertChannelPrefixName() p.initInsertChannelPrefixName()
p.initStatisticsChannelName()
p.initTimeTickChannelName() p.initTimeTickChannelName()
p.initSegmentInfoChannelName() p.initSegmentInfoChannelName()
p.initDataCoordSubscriptionName() p.initDataCoordSubscriptionName()
@ -244,15 +242,6 @@ func (p *ParamTable) initInsertChannelPrefixName() {
p.InsertChannelPrefixName = strings.Join(s, "-") 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() { func (p *ParamTable) initTimeTickChannelName() {
config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick") config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick")
if err != nil { if err != nil {

View File

@ -29,9 +29,6 @@ func TestParamTable(t *testing.T) {
assert.Equal(t, Params.InsertChannelPrefixName, "by-dev-insert-channel-") assert.Equal(t, Params.InsertChannelPrefixName, "by-dev-insert-channel-")
t.Logf("data coord insert channel = %s", Params.InsertChannelPrefixName) 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") assert.Equal(t, Params.TimeTickChannelName, "by-dev-datacoord-timetick-channel")
t.Logf("data coord timetick channel = %s", Params.TimeTickChannelName) t.Logf("data coord timetick channel = %s", Params.TimeTickChannelName)

View File

@ -142,5 +142,6 @@ const flushInterval = 2 * time.Second
func flushPolicyV1(segment *SegmentInfo, t Timestamp) bool { func flushPolicyV1(segment *SegmentInfo, t Timestamp) bool {
return segment.GetState() == commonpb.SegmentState_Sealed && return segment.GetState() == commonpb.SegmentState_Sealed &&
segment.GetLastExpireTime() <= t && segment.GetLastExpireTime() <= t &&
time.Since(segment.lastFlushTime) >= flushInterval time.Since(segment.lastFlushTime) >= flushInterval &&
segment.currRows != 0
} }

View File

@ -302,6 +302,7 @@ func TestGetFlushableSegments(t *testing.T) {
assert.EqualValues(t, 1, len(ids)) assert.EqualValues(t, 1, len(ids))
assert.EqualValues(t, allocations[0].SegmentID, ids[0]) assert.EqualValues(t, allocations[0].SegmentID, ids[0])
meta.SetCurrentRows(allocations[0].SegmentID, 1)
ids, err = segmentManager.GetFlushableSegments(context.TODO(), "c1", allocations[0].ExpireTime) ids, err = segmentManager.GetFlushableSegments(context.TODO(), "c1", allocations[0].ExpireTime)
assert.Nil(t, err) assert.Nil(t, err)
assert.EqualValues(t, 1, len(ids)) assert.EqualValues(t, 1, len(ids))

View File

@ -399,8 +399,7 @@ func (s *Server) initMeta() error {
func (s *Server) startServerLoop() { func (s *Server) startServerLoop() {
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx) s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx)
s.serverLoopWg.Add(4) s.serverLoopWg.Add(3)
s.startStatsChannel(s.serverLoopCtx)
s.startDataNodeTtLoop(s.serverLoopCtx) s.startDataNodeTtLoop(s.serverLoopCtx)
s.startWatchService(s.serverLoopCtx) s.startWatchService(s.serverLoopCtx)
s.startFlushLoop(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 // startDataNodeTtLoop start a goroutine to recv data node tt msg from msgstream
// tt msg stands for the currently consumed timestamp for each channel // tt msg stands for the currently consumed timestamp for each channel
func (s *Server) startDataNodeTtLoop(ctx context.Context) { func (s *Server) startDataNodeTtLoop(ctx context.Context) {
@ -475,6 +436,7 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
checker.Start() checker.Start()
defer checker.Stop() defer checker.Stop()
} }
defer logutil.LogPanic() defer logutil.LogPanic()
defer s.serverLoopWg.Done() defer s.serverLoopWg.Done()
defer ttMsgStream.Close() defer ttMsgStream.Close()
@ -491,75 +453,119 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
return return
} }
for _, msg := range msgPack.Msgs { for _, msg := range msgPack.Msgs {
if msg.Type() != commonpb.MsgType_DataNodeTt { ttMsg, ok := msg.(*msgstream.DataNodeTtMsg)
log.Warn("receive unexpected msg type from tt channel", if !ok {
zap.Stringer("msgType", msg.Type())) log.Warn("receive unexpected msg type from tt channel")
continue continue
} }
ttMsg := msg.(*msgstream.DataNodeTtMsg)
if enableTtChecker { if enableTtChecker {
checker.Check() checker.Check()
} }
ch := ttMsg.ChannelName if err := s.handleTimetickMessage(ctx, ttMsg); err != nil {
ts := ttMsg.Timestamp log.Error("failed to handle timetick message", zap.Error(err))
if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil {
log.Warn("failed to expire allocations", zap.Error(err))
continue 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() 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 // start a goroutine wto watch services
func (s *Server) startWatchService(ctx context.Context) { func (s *Server) startWatchService(ctx context.Context) {
go s.watchService(ctx) go s.watchService(ctx)

View File

@ -194,6 +194,8 @@ func TestFlush(t *testing.T) {
resp, err := svr.Flush(context.TODO(), req) resp, err := svr.Flush(context.TODO(), req)
assert.Nil(t, err) assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
svr.meta.SetCurrentRows(segID, 1)
ids, err := svr.segmentManager.GetFlushableSegments(context.TODO(), "channel-1", expireTs) ids, err := svr.segmentManager.GetFlushableSegments(context.TODO(), "channel-1", expireTs)
assert.Nil(t, err) assert.Nil(t, err)
assert.EqualValues(t, 1, len(ids)) assert.EqualValues(t, 1, len(ids))
@ -250,15 +252,6 @@ func TestGetTimeTickChannel(t *testing.T) {
assert.EqualValues(t, Params.TimeTickChannelName, resp.Value) 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) { func TestGetSegmentStates(t *testing.T) {
t.Run("normal cases", func(t *testing.T) { t.Run("normal cases", func(t *testing.T) {
svr := newTestServer(t, nil) 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 { type spySegmentManager struct {
spyCh chan struct{} spyCh chan struct{}
} }
@ -1143,6 +1087,10 @@ func TestDataNodeTtChannel(t *testing.T) {
msgPack := msgstream.MsgPack{} msgPack := msgstream.MsgPack{}
msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime) 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) msgPack.Msgs = append(msgPack.Msgs, msg)
err = ttMsgStream.Produce(&msgPack) err = ttMsgStream.Produce(&msgPack)
assert.Nil(t, err) assert.Nil(t, err)
@ -1217,6 +1165,10 @@ func TestDataNodeTtChannel(t *testing.T) {
msgPack := msgstream.MsgPack{} msgPack := msgstream.MsgPack{}
msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime) 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) msgPack.Msgs = append(msgPack.Msgs, msg)
err = ttMsgStream.Produce(&msgPack) err = ttMsgStream.Produce(&msgPack)
assert.Nil(t, err) assert.Nil(t, err)
@ -2250,7 +2202,6 @@ func TestGetFlushState(t *testing.T) {
func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Server { func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Server {
Params.Init() Params.Init()
Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int()) Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int())
Params.StatisticsChannelName = Params.StatisticsChannelName + strconv.Itoa(rand.Int())
var err error var err error
factory := msgstream.NewPmsFactory() factory := msgstream.NewPmsFactory()
m := map[string]interface{}{ m := map[string]interface{}{

View File

@ -55,9 +55,9 @@ func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringRespon
func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) { func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{ return &milvuspb.StringResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success, ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "no statistics channel",
}, },
Value: Params.StatisticsChannelName,
}, nil }, nil
} }

View File

@ -204,7 +204,6 @@ func (node *DataNode) Register() error {
// Init function does nothing now. // Init function does nothing now.
func (node *DataNode) Init() error { func (node *DataNode) Init() error {
log.Debug("DataNode Init", log.Debug("DataNode Init",
zap.String("SegmentStatisticsChannelName", Params.SegmentStatisticsChannelName),
zap.String("TimeTickChannelName", Params.TimeTickChannelName), zap.String("TimeTickChannelName", Params.TimeTickChannelName),
) )

View File

@ -55,7 +55,6 @@ func TestMain(t *testing.M) {
Params.Init() Params.Init()
// change to specific channel for test // change to specific channel for test
Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int()) Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int())
Params.SegmentStatisticsChannelName = Params.SegmentStatisticsChannelName + strconv.Itoa(rand.Int())
code := t.Run() code := t.Run()
os.Exit(code) os.Exit(code)
} }

View File

@ -63,10 +63,9 @@ type insertBufferNode struct {
flushingSegCache *Cache flushingSegCache *Cache
flushManager flushManager flushManager flushManager
timeTickStream msgstream.MsgStream timeTickStream msgstream.MsgStream
segmentStatisticsStream msgstream.MsgStream ttLogger timeTickLogger
ttLogger timeTickLogger ttMerger *mergedTimeTickerSender
ttMerger *mergedTimeTickerSender
} }
type timeTickLogger struct { type timeTickLogger struct {
@ -158,10 +157,6 @@ func (ibNode *insertBufferNode) Close() {
if ibNode.timeTickStream != nil { if ibNode.timeTickStream != nil {
ibNode.timeTickStream.Close() ibNode.timeTickStream.Close()
} }
if ibNode.segmentStatisticsStream != nil {
ibNode.segmentStatisticsStream.Close()
}
} }
func (ibNode *insertBufferNode) Operate(in []Msg) []Msg { func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
@ -211,14 +206,6 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
return []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 // insert messages -> buffer
for _, msg := range fgMsg.insertMessages { for _, msg := range fgMsg.insertMessages {
err := ibNode.bufferInsertMsg(msg, endPositions[0]) 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)) 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. // 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.ttLogger.LogTs(ts)
ibNode.ttMerger.bufferTs(ts) ibNode.ttMerger.bufferTs(ts, segmentIDs)
return nil 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) { func (ibNode *insertBufferNode) getCollectionandPartitionIDbySegID(segmentID UniqueID) (collID, partitionID UniqueID, err error) {
return ibNode.replica.getCollectionAndPartitionID(segmentID) return ibNode.replica.getCollectionAndPartitionID(segmentID)
} }
@ -782,17 +722,16 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM
var wTtMsgStream msgstream.MsgStream = wTt var wTtMsgStream msgstream.MsgStream = wTt
wTtMsgStream.Start() wTtMsgStream.Start()
// update statistics channel mt := newMergedTimeTickerSender(func(ts Timestamp, segmentIDs []int64) error {
segS, err := config.msFactory.NewMsgStream(ctx) stats := make([]*datapb.SegmentStats, 0, len(segmentIDs))
if err != nil { for _, sid := range segmentIDs {
return nil, err stat, err := config.replica.getSegmentStatisticsUpdates(sid)
} if err != nil {
segS.AsProducer([]string{Params.SegmentStatisticsChannelName}) log.Warn("failed to get segment statistics info", zap.Int64("segmentID", sid), zap.Error(err))
log.Debug("datanode AsProducer", zap.String("SegmentStatisChannelName", Params.SegmentStatisticsChannelName)) continue
var segStatisticsMsgStream msgstream.MsgStream = segS }
segStatisticsMsgStream.Start() stats = append(stats, stat)
}
mt := newMergedTimeTickerSender(func(ts Timestamp) error {
msgPack := msgstream.MsgPack{} msgPack := msgstream.MsgPack{}
timeTickMsg := msgstream.DataNodeTtMsg{ timeTickMsg := msgstream.DataNodeTtMsg{
BaseMsg: msgstream.BaseMsg{ BaseMsg: msgstream.BaseMsg{
@ -806,8 +745,9 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM
MsgID: 0, MsgID: 0,
Timestamp: ts, Timestamp: ts,
}, },
ChannelName: config.vChannelName, ChannelName: config.vChannelName,
Timestamp: ts, Timestamp: ts,
SegmentsStats: stats,
}, },
} }
msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg) msgPack.Msgs = append(msgPack.Msgs, &timeTickMsg)
@ -818,9 +758,7 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM
BaseNode: baseNode, BaseNode: baseNode,
insertBuffer: sync.Map{}, insertBuffer: sync.Map{},
timeTickStream: wTtMsgStream, timeTickStream: wTtMsgStream,
segmentStatisticsStream: segStatisticsMsgStream,
flushMap: sync.Map{}, flushMap: sync.Map{},
flushChan: flushCh, flushChan: flushCh,
flushingSegCache: flushingSegCache, flushingSegCache: flushingSegCache,

View File

@ -110,13 +110,6 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
_, err = newInsertBufferNode(ctx, flushChan, fm, newCache(), c) _, err = newInsertBufferNode(ctx, flushChan, fm, newCache(), c)
assert.Error(t, err) 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{} type mockMsg struct{}
@ -142,7 +135,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
for _, test := range invalidInTests { for _, test := range invalidInTests {
te.Run(test.description, func(t0 *testing.T) { te.Run(test.description, func(t0 *testing.T) {
ibn := &insertBufferNode{ ibn := &insertBufferNode{
ttMerger: newMergedTimeTickerSender(func(Timestamp) error { return nil }), ttMerger: newMergedTimeTickerSender(func(Timestamp, []int64) error { return nil }),
} }
rt := ibn.Operate(test.in) rt := ibn.Operate(test.in)
assert.Empty(t0, rt) assert.Empty(t0, rt)

View File

@ -19,22 +19,22 @@ package datanode
import ( import (
"sync" "sync"
"time" "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` // 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) // it makes sure time ticker send at most 10 times a second (1tick/100millisecond)
// and the last time tick is always sent // and the last time tick is always sent
type mergedTimeTickerSender struct { type mergedTimeTickerSender struct {
ts atomic.Uint64 // current ts value ts uint64
cond *sync.Cond // condition to send timeticker segmentIDs map[int64]struct{}
send sendTimeTick // actual sender logic 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 wg sync.WaitGroup
closeCh chan struct{} closeCh chan struct{}
closeOnce sync.Once closeOnce sync.Once
@ -42,12 +42,12 @@ type mergedTimeTickerSender struct {
func newMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { func newMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender {
mt := &mergedTimeTickerSender{ mt := &mergedTimeTickerSender{
cond: sync.NewCond(&sync.Mutex{}), ts: 0, // 0 for not tt send
send: send, segmentIDs: make(map[int64]struct{}),
closeCh: make(chan 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) mt.wg.Add(2)
go mt.tick() go mt.tick()
go mt.work() go mt.work()
@ -55,10 +55,13 @@ func newMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender {
return mt return mt
} }
func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp) { func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) {
mt.ts.Store(ts) mt.mu.Lock()
mt.lastMut.RLock() defer mt.mu.Unlock()
defer mt.lastMut.RUnlock() mt.ts = ts
for _, sid := range segmentIDs {
mt.segmentIDs[sid] = struct{}{}
}
if !mt.lastSent.IsZero() && time.Since(mt.lastSent) > time.Millisecond*100 { if !mt.lastSent.IsZero() && time.Since(mt.lastSent) > time.Millisecond*100 {
mt.cond.L.Lock() mt.cond.L.Lock()
@ -94,7 +97,7 @@ func (mt *mergedTimeTickerSender) isClosed() bool {
func (mt *mergedTimeTickerSender) work() { func (mt *mergedTimeTickerSender) work() {
defer mt.wg.Done() defer mt.wg.Done()
ts, lastTs := uint64(0), uint64(0) lastTs := uint64(0)
for { for {
mt.cond.L.Lock() mt.cond.L.Lock()
if mt.isClosed() { if mt.isClosed() {
@ -102,15 +105,21 @@ func (mt *mergedTimeTickerSender) work() {
return return
} }
mt.cond.Wait() mt.cond.Wait()
ts = mt.ts.Load()
mt.cond.L.Unlock() mt.cond.L.Unlock()
if ts != lastTs {
mt.send(ts) mt.mu.Lock()
lastTs = ts if mt.ts != lastTs {
mt.lastMut.Lock() 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.lastSent = time.Now()
mt.lastMut.Unlock()
mt.send(mt.ts, sids)
} }
mt.mu.Unlock()
} }
} }

View File

@ -12,7 +12,7 @@ func TestMergedTimeTicker(t *testing.T) {
var ticks []uint64 var ticks []uint64
var mut sync.Mutex var mut sync.Mutex
mt := newMergedTimeTickerSender(func(ts Timestamp) error { mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error {
mut.Lock() mut.Lock()
defer mut.Unlock() defer mut.Unlock()
ticks = append(ticks, ts) ticks = append(ticks, ts)
@ -21,7 +21,7 @@ func TestMergedTimeTicker(t *testing.T) {
for i := 1; i < 100; i++ { for i := 1; i < 100; i++ {
time.Sleep(time.Millisecond * 10) time.Sleep(time.Millisecond * 10)
mt.bufferTs(uint64(i)) mt.bufferTs(uint64(i), nil)
} }
mt.close() mt.close()
mut.Lock() mut.Lock()
@ -35,7 +35,7 @@ func TestMergedTimeTicker_close10000(t *testing.T) {
batchSize := 10000 batchSize := 10000
wg.Add(batchSize) wg.Add(batchSize)
for i := 0; i < batchSize; i++ { for i := 0; i < batchSize; i++ {
mt := newMergedTimeTickerSender(func(ts Timestamp) error { mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error {
return nil return nil
}) })
go func(mt *mergedTimeTickerSender) { go func(mt *mergedTimeTickerSender) {

View File

@ -59,9 +59,6 @@ type ParamTable struct {
// Cluster channels // Cluster channels
ClusterChannelPrefix string ClusterChannelPrefix string
// Segment statistics channel
SegmentStatisticsChannelName string
// Timetick channel // Timetick channel
TimeTickChannelName string TimeTickChannelName string
@ -116,7 +113,6 @@ func (p *ParamTable) Init() {
// Must init global msgchannel prefix before other channel names // Must init global msgchannel prefix before other channel names
p.initClusterMsgChannelPrefix() p.initClusterMsgChannelPrefix()
p.initSegmentStatisticsChannelName()
p.initTimeTickChannelName() p.initTimeTickChannelName()
p.initEtcdEndpoints() p.initEtcdEndpoints()
@ -196,15 +192,6 @@ func (p *ParamTable) initClusterMsgChannelPrefix() {
p.ClusterChannelPrefix = name 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() { func (p *ParamTable) initTimeTickChannelName() {
config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick") config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick")
if err != nil { if err != nil {

View File

@ -71,12 +71,6 @@ func TestParamTable(t *testing.T) {
log.Println("ClusterChannelPrefix:", Params.ClusterChannelPrefix) 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) { t.Run("Test TimeTickChannelName", func(t *testing.T) {
name := Params.TimeTickChannelName name := Params.TimeTickChannelName
assert.Equal(t, name, "by-dev-datacoord-timetick-channel") assert.Equal(t, name, "by-dev-datacoord-timetick-channel")

View File

@ -67,7 +67,7 @@ type Replica interface {
updateStatistics(segID UniqueID, numRows int64) updateStatistics(segID UniqueID, numRows int64)
refreshFlushedSegStatistics(segID UniqueID, numRows int64) refreshFlushedSegStatistics(segID UniqueID, numRows int64)
getSegmentStatisticsUpdates(segID UniqueID) (*internalpb.SegmentStatisticsUpdates, error) getSegmentStatisticsUpdates(segID UniqueID) (*datapb.SegmentStats, error)
segmentFlushed(segID UniqueID) segmentFlushed(segID UniqueID)
} }
@ -580,12 +580,10 @@ func (replica *SegmentReplica) updateStatistics(segID UniqueID, numRows int64) {
} }
// getSegmentStatisticsUpdates gives current segment's statistics updates. // 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() replica.segMu.Lock()
defer replica.segMu.Unlock() defer replica.segMu.Unlock()
updates := &internalpb.SegmentStatisticsUpdates{ updates := &datapb.SegmentStats{SegmentID: segID}
SegmentID: segID,
}
if seg, ok := replica.newSegments[segID]; ok { if seg, ok := replica.newSegments[segID]; ok {
updates.NumRows = seg.numRows updates.NumRows = seg.numRows

View File

@ -609,69 +609,6 @@ func (qs *QueryNodeStatsMsg) Unmarshal(input MarshalType) (TsMsg, error) {
return queryNodeSegStatsMsg, nil 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////////////////////////////////////////// /////////////////////////////////////////CreateCollection//////////////////////////////////////////
// CreateCollectionMsg is a message pack that contains create collection request // CreateCollectionMsg is a message pack that contains create collection request

View File

@ -510,50 +510,6 @@ func TestQueryNodeStatsMsg_Unmarshal_IllegalParameter(t *testing.T) {
assert.Nil(t, tsMsg) 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) { func TestCreateCollectionMsg(t *testing.T) {
createCollectionMsg := &CreateCollectionMsg{ createCollectionMsg := &CreateCollectionMsg{
BaseMsg: generateBaseMsg(), BaseMsg: generateBaseMsg(),

View File

@ -66,7 +66,6 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher {
createPartitionMsg := CreatePartitionMsg{} createPartitionMsg := CreatePartitionMsg{}
dropPartitionMsg := DropPartitionMsg{} dropPartitionMsg := DropPartitionMsg{}
queryNodeSegStatsMsg := QueryNodeStatsMsg{} queryNodeSegStatsMsg := QueryNodeStatsMsg{}
segmentStatisticsMsg := SegmentStatisticsMsg{}
dataNodeTtMsg := DataNodeTtMsg{} dataNodeTtMsg := DataNodeTtMsg{}
sealedSegmentsChangeInfoMsg := SealedSegmentsChangeInfoMsg{} sealedSegmentsChangeInfoMsg := SealedSegmentsChangeInfoMsg{}
@ -84,7 +83,6 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher {
p.TempMap[commonpb.MsgType_DropCollection] = dropCollectionMsg.Unmarshal p.TempMap[commonpb.MsgType_DropCollection] = dropCollectionMsg.Unmarshal
p.TempMap[commonpb.MsgType_CreatePartition] = createPartitionMsg.Unmarshal p.TempMap[commonpb.MsgType_CreatePartition] = createPartitionMsg.Unmarshal
p.TempMap[commonpb.MsgType_DropPartition] = dropPartitionMsg.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_DataNodeTt] = dataNodeTtMsg.Unmarshal
p.TempMap[commonpb.MsgType_SealedSegmentsChangeInfo] = sealedSegmentsChangeInfoMsg.Unmarshal p.TempMap[commonpb.MsgType_SealedSegmentsChangeInfo] = sealedSegmentsChangeInfoMsg.Unmarshal

View File

@ -253,6 +253,12 @@ message DataNodeTtMsg {
common.MsgBase base =1; common.MsgBase base =1;
string channel_name = 2; string channel_name = 2;
uint64 timestamp = 3; uint64 timestamp = 3;
repeated SegmentStats segments_stats = 4;
}
message SegmentStats {
int64 SegmentID = 1;
int64 NumRows = 2;
} }
enum ChannelWatchState { enum ChannelWatchState {

View File

@ -1810,6 +1810,7 @@ type DataNodeTtMsg struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` 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"` 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"` 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_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -1861,6 +1862,60 @@ func (m *DataNodeTtMsg) GetTimestamp() uint64 {
return 0 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 { type ChannelStatus struct {
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` 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"` 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 (m *ChannelStatus) String() string { return proto.CompactTextString(m) }
func (*ChannelStatus) ProtoMessage() {} func (*ChannelStatus) ProtoMessage() {}
func (*ChannelStatus) Descriptor() ([]byte, []int) { func (*ChannelStatus) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{29} return fileDescriptor_82cd95f524594f49, []int{30}
} }
func (m *ChannelStatus) XXX_Unmarshal(b []byte) error { 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 (m *DataNodeInfo) String() string { return proto.CompactTextString(m) }
func (*DataNodeInfo) ProtoMessage() {} func (*DataNodeInfo) ProtoMessage() {}
func (*DataNodeInfo) Descriptor() ([]byte, []int) { func (*DataNodeInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{30} return fileDescriptor_82cd95f524594f49, []int{31}
} }
func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error { 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 (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) }
func (*SegmentBinlogs) ProtoMessage() {} func (*SegmentBinlogs) ProtoMessage() {}
func (*SegmentBinlogs) Descriptor() ([]byte, []int) { func (*SegmentBinlogs) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{31} return fileDescriptor_82cd95f524594f49, []int{32}
} }
func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error { 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 (m *FieldBinlog) String() string { return proto.CompactTextString(m) }
func (*FieldBinlog) ProtoMessage() {} func (*FieldBinlog) ProtoMessage() {}
func (*FieldBinlog) Descriptor() ([]byte, []int) { func (*FieldBinlog) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{32} return fileDescriptor_82cd95f524594f49, []int{33}
} }
func (m *FieldBinlog) XXX_Unmarshal(b []byte) error { 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 (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoResponse) ProtoMessage() {} func (*GetRecoveryInfoResponse) ProtoMessage() {}
func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) { func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{33} return fileDescriptor_82cd95f524594f49, []int{34}
} }
func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error { 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 (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) }
func (*GetRecoveryInfoRequest) ProtoMessage() {} func (*GetRecoveryInfoRequest) ProtoMessage() {}
func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) { func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{34} return fileDescriptor_82cd95f524594f49, []int{35}
} }
func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error { 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 (m *GetFlushedSegmentsRequest) String() string { return proto.CompactTextString(m) }
func (*GetFlushedSegmentsRequest) ProtoMessage() {} func (*GetFlushedSegmentsRequest) ProtoMessage() {}
func (*GetFlushedSegmentsRequest) Descriptor() ([]byte, []int) { func (*GetFlushedSegmentsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{35} return fileDescriptor_82cd95f524594f49, []int{36}
} }
func (m *GetFlushedSegmentsRequest) XXX_Unmarshal(b []byte) error { 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 (m *GetFlushedSegmentsResponse) String() string { return proto.CompactTextString(m) }
func (*GetFlushedSegmentsResponse) ProtoMessage() {} func (*GetFlushedSegmentsResponse) ProtoMessage() {}
func (*GetFlushedSegmentsResponse) Descriptor() ([]byte, []int) { func (*GetFlushedSegmentsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{36} return fileDescriptor_82cd95f524594f49, []int{37}
} }
func (m *GetFlushedSegmentsResponse) XXX_Unmarshal(b []byte) error { 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 (m *SegmentFlushCompletedMsg) String() string { return proto.CompactTextString(m) }
func (*SegmentFlushCompletedMsg) ProtoMessage() {} func (*SegmentFlushCompletedMsg) ProtoMessage() {}
func (*SegmentFlushCompletedMsg) Descriptor() ([]byte, []int) { func (*SegmentFlushCompletedMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{37} return fileDescriptor_82cd95f524594f49, []int{38}
} }
func (m *SegmentFlushCompletedMsg) XXX_Unmarshal(b []byte) error { 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 (m *ChannelWatchInfo) String() string { return proto.CompactTextString(m) }
func (*ChannelWatchInfo) ProtoMessage() {} func (*ChannelWatchInfo) ProtoMessage() {}
func (*ChannelWatchInfo) Descriptor() ([]byte, []int) { func (*ChannelWatchInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{38} return fileDescriptor_82cd95f524594f49, []int{39}
} }
func (m *ChannelWatchInfo) XXX_Unmarshal(b []byte) error { 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 (m *CompactionSegmentBinlogs) String() string { return proto.CompactTextString(m) }
func (*CompactionSegmentBinlogs) ProtoMessage() {} func (*CompactionSegmentBinlogs) ProtoMessage() {}
func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) { func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{39} return fileDescriptor_82cd95f524594f49, []int{40}
} }
func (m *CompactionSegmentBinlogs) XXX_Unmarshal(b []byte) error { 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 (m *CompactionPlan) String() string { return proto.CompactTextString(m) }
func (*CompactionPlan) ProtoMessage() {} func (*CompactionPlan) ProtoMessage() {}
func (*CompactionPlan) Descriptor() ([]byte, []int) { func (*CompactionPlan) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{40} return fileDescriptor_82cd95f524594f49, []int{41}
} }
func (m *CompactionPlan) XXX_Unmarshal(b []byte) error { 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 (m *CompactionResult) String() string { return proto.CompactTextString(m) }
func (*CompactionResult) ProtoMessage() {} func (*CompactionResult) ProtoMessage() {}
func (*CompactionResult) Descriptor() ([]byte, []int) { func (*CompactionResult) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{41} return fileDescriptor_82cd95f524594f49, []int{42}
} }
func (m *CompactionResult) XXX_Unmarshal(b []byte) error { 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 (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) ProtoMessage() {}
func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{42} return fileDescriptor_82cd95f524594f49, []int{43}
} }
func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { 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 (m *WatchChannelsRequest) String() string { return proto.CompactTextString(m) }
func (*WatchChannelsRequest) ProtoMessage() {} func (*WatchChannelsRequest) ProtoMessage() {}
func (*WatchChannelsRequest) Descriptor() ([]byte, []int) { func (*WatchChannelsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{43} return fileDescriptor_82cd95f524594f49, []int{44}
} }
func (m *WatchChannelsRequest) XXX_Unmarshal(b []byte) error { 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 (m *WatchChannelsResponse) String() string { return proto.CompactTextString(m) }
func (*WatchChannelsResponse) ProtoMessage() {} func (*WatchChannelsResponse) ProtoMessage() {}
func (*WatchChannelsResponse) Descriptor() ([]byte, []int) { func (*WatchChannelsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{44} return fileDescriptor_82cd95f524594f49, []int{45}
} }
func (m *WatchChannelsResponse) XXX_Unmarshal(b []byte) error { 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 (m *DropVirtualChannelRequest) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelRequest) ProtoMessage() {} func (*DropVirtualChannelRequest) ProtoMessage() {}
func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) { func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{45} return fileDescriptor_82cd95f524594f49, []int{46}
} }
func (m *DropVirtualChannelRequest) XXX_Unmarshal(b []byte) error { 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 (m *DropVirtualChannelSegment) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelSegment) ProtoMessage() {} func (*DropVirtualChannelSegment) ProtoMessage() {}
func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) { func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{46} return fileDescriptor_82cd95f524594f49, []int{47}
} }
func (m *DropVirtualChannelSegment) XXX_Unmarshal(b []byte) error { 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 (m *DropVirtualChannelResponse) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelResponse) ProtoMessage() {} func (*DropVirtualChannelResponse) ProtoMessage() {}
func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) { func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{47} return fileDescriptor_82cd95f524594f49, []int{48}
} }
func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error { func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error {
@ -2987,6 +3042,7 @@ func init() {
proto.RegisterType((*CheckPoint)(nil), "milvus.proto.data.CheckPoint") proto.RegisterType((*CheckPoint)(nil), "milvus.proto.data.CheckPoint")
proto.RegisterType((*DeltaLogInfo)(nil), "milvus.proto.data.DeltaLogInfo") proto.RegisterType((*DeltaLogInfo)(nil), "milvus.proto.data.DeltaLogInfo")
proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg") 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((*ChannelStatus)(nil), "milvus.proto.data.ChannelStatus")
proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo") proto.RegisterType((*DataNodeInfo)(nil), "milvus.proto.data.DataNodeInfo")
proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs") proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs")
@ -3011,180 +3067,182 @@ func init() {
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{ var fileDescriptor_82cd95f524594f49 = []byte{
// 2758 bytes of a gzipped FileDescriptorProto // 2800 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xdf, 0x6f, 0x1b, 0xc7, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x5b, 0x6f, 0xdc, 0xc6,
0xf1, 0xf7, 0xf1, 0x87, 0x44, 0x0e, 0x29, 0x8a, 0x5e, 0x39, 0x32, 0xc3, 0x38, 0xb2, 0x7c, 0x49, 0xf5, 0x37, 0xf7, 0x22, 0xed, 0x9e, 0xbd, 0x68, 0x3d, 0x72, 0xe4, 0xcd, 0xc6, 0x91, 0x65, 0x26,
0x1c, 0xc5, 0x71, 0x24, 0x5b, 0xfe, 0x06, 0xdf, 0xa0, 0x4e, 0x1a, 0x58, 0x96, 0x2d, 0x13, 0x95, 0x71, 0x14, 0xc7, 0x91, 0x62, 0xe5, 0x1f, 0xfc, 0x83, 0x26, 0x69, 0x10, 0x59, 0x91, 0xb2, 0xa8,
0x5c, 0xe5, 0xa8, 0xd8, 0x45, 0x03, 0x94, 0x38, 0xf1, 0x56, 0xd4, 0xd5, 0xbc, 0x3b, 0xfa, 0x6e, 0xe4, 0x2a, 0x5c, 0xc5, 0x2e, 0x1a, 0xa0, 0x0b, 0x6a, 0x39, 0x5a, 0xb1, 0x5e, 0x92, 0x6b, 0x72,
0x29, 0x5b, 0x79, 0x89, 0x91, 0x02, 0x05, 0x5a, 0xb4, 0x4d, 0x8a, 0xbe, 0x16, 0x68, 0xd1, 0xa7, 0x56, 0xb6, 0xf2, 0x12, 0xa3, 0x01, 0x0a, 0xb4, 0x68, 0x9b, 0x14, 0x7d, 0x2d, 0xd0, 0xa2, 0x4f,
0x02, 0x7d, 0x29, 0xfa, 0x98, 0xfe, 0x03, 0x45, 0xfb, 0xde, 0xbf, 0xa1, 0x6f, 0xfd, 0x17, 0x8a, 0x05, 0xfa, 0x52, 0xf4, 0x31, 0xfd, 0x02, 0x45, 0xfb, 0xde, 0xcf, 0xd0, 0xb7, 0x7e, 0x85, 0x62,
0xfd, 0x71, 0x7b, 0x3f, 0x78, 0x47, 0x1e, 0x25, 0xff, 0x78, 0xe3, 0xee, 0xcd, 0xce, 0xcc, 0xce, 0x2e, 0x1c, 0x5e, 0x96, 0xdc, 0xe5, 0x4a, 0xbe, 0xbc, 0x71, 0x86, 0xe7, 0xcc, 0x39, 0x73, 0xe6,
0xce, 0x7c, 0x66, 0x66, 0x97, 0x50, 0x37, 0x74, 0xa2, 0x77, 0xba, 0x8e, 0xe3, 0x1a, 0xab, 0x03, 0x9c, 0xdf, 0x39, 0x67, 0x48, 0x68, 0x18, 0x3a, 0xd1, 0xbb, 0x3d, 0xc7, 0x71, 0x8d, 0xb5, 0xa1,
0xd7, 0x21, 0x0e, 0x3a, 0x6b, 0x99, 0xfd, 0xa3, 0xa1, 0xc7, 0x47, 0xab, 0xf4, 0x73, 0xb3, 0xda, 0xeb, 0x10, 0x07, 0x5d, 0xb4, 0xcc, 0xc1, 0xc9, 0xc8, 0xe3, 0xa3, 0x35, 0xfa, 0xba, 0x55, 0xed,
0x75, 0x2c, 0xcb, 0xb1, 0xf9, 0x54, 0xb3, 0x66, 0xda, 0x04, 0xbb, 0xb6, 0xde, 0x17, 0xe3, 0x6a, 0x39, 0x96, 0xe5, 0xd8, 0x7c, 0xaa, 0x55, 0x37, 0x6d, 0x82, 0x5d, 0x5b, 0x1f, 0x88, 0x71, 0x35,
0x78, 0x41, 0xb3, 0xea, 0x75, 0x0f, 0xb1, 0xa5, 0xf3, 0x91, 0xfa, 0x14, 0xaa, 0x77, 0xfb, 0x43, 0xcc, 0xd0, 0xaa, 0x7a, 0xbd, 0x63, 0x6c, 0xe9, 0x7c, 0xa4, 0x3e, 0x82, 0xea, 0xf6, 0x60, 0xe4,
0xef, 0x50, 0xc3, 0x8f, 0x87, 0xd8, 0x23, 0xe8, 0x1a, 0x14, 0xf6, 0x75, 0x0f, 0x37, 0x94, 0x65, 0x1d, 0x6b, 0xf8, 0xc1, 0x08, 0x7b, 0x04, 0xbd, 0x0d, 0x85, 0x43, 0xdd, 0xc3, 0x4d, 0x65, 0x45,
0x65, 0xa5, 0xb2, 0x7e, 0x61, 0x35, 0x22, 0x4b, 0x48, 0xd9, 0xf1, 0x7a, 0x1b, 0xba, 0x87, 0x35, 0x59, 0xad, 0x6c, 0x5c, 0x59, 0x8b, 0xc8, 0x12, 0x52, 0xf6, 0xbc, 0xfe, 0xa6, 0xee, 0x61, 0x8d,
0x46, 0x89, 0x10, 0x14, 0x8c, 0xfd, 0xd6, 0x66, 0x23, 0xb7, 0xac, 0xac, 0xe4, 0x35, 0xf6, 0x1b, 0x51, 0x22, 0x04, 0x05, 0xe3, 0xb0, 0xbd, 0xd5, 0xcc, 0xad, 0x28, 0xab, 0x79, 0x8d, 0x3d, 0x23,
0xa9, 0x50, 0xed, 0x3a, 0xfd, 0x3e, 0xee, 0x12, 0xd3, 0xb1, 0x5b, 0x9b, 0x8d, 0x02, 0xfb, 0x16, 0x15, 0xaa, 0x3d, 0x67, 0x30, 0xc0, 0x3d, 0x62, 0x3a, 0x76, 0x7b, 0xab, 0x59, 0x60, 0xef, 0x22,
0x99, 0x53, 0x7f, 0xaf, 0xc0, 0x9c, 0x10, 0xed, 0x0d, 0x1c, 0xdb, 0xc3, 0xe8, 0x06, 0xcc, 0x78, 0x73, 0xea, 0xef, 0x15, 0xa8, 0x09, 0xd1, 0xde, 0xd0, 0xb1, 0x3d, 0x8c, 0xde, 0x81, 0x39, 0x8f,
0x44, 0x27, 0x43, 0x4f, 0x48, 0x7f, 0x23, 0x51, 0x7a, 0x9b, 0x91, 0x68, 0x82, 0x34, 0x93, 0xf8, 0xe8, 0x64, 0xe4, 0x09, 0xe9, 0x2f, 0x25, 0x4a, 0xef, 0x30, 0x12, 0x4d, 0x90, 0x66, 0x12, 0x9f,
0xfc, 0xa8, 0x78, 0xb4, 0x04, 0xe0, 0xe1, 0x9e, 0x85, 0x6d, 0xd2, 0xda, 0xf4, 0x1a, 0x85, 0xe5, 0x1f, 0x17, 0x8f, 0x96, 0x01, 0x3c, 0xdc, 0xb7, 0xb0, 0x4d, 0xda, 0x5b, 0x5e, 0xb3, 0xb0, 0x92,
0xfc, 0x4a, 0x5e, 0x0b, 0xcd, 0xa8, 0xbf, 0x55, 0xa0, 0xde, 0xf6, 0x87, 0xbe, 0x75, 0xce, 0x41, 0x5f, 0xcd, 0x6b, 0xa1, 0x19, 0xf5, 0xb7, 0x0a, 0x34, 0x3a, 0xfe, 0xd0, 0xb7, 0xce, 0x25, 0x28,
0xb1, 0xeb, 0x0c, 0x6d, 0xc2, 0x14, 0x9c, 0xd3, 0xf8, 0x00, 0x5d, 0x82, 0x6a, 0xf7, 0x50, 0xb7, 0xf6, 0x9c, 0x91, 0x4d, 0x98, 0x82, 0x35, 0x8d, 0x0f, 0xd0, 0x35, 0xa8, 0xf6, 0x8e, 0x75, 0xdb,
0x6d, 0xdc, 0xef, 0xd8, 0xba, 0x85, 0x99, 0x2a, 0x65, 0xad, 0x22, 0xe6, 0xee, 0xeb, 0x16, 0xce, 0xc6, 0x83, 0xae, 0xad, 0x5b, 0x98, 0xa9, 0x52, 0xd6, 0x2a, 0x62, 0xee, 0x8e, 0x6e, 0xe1, 0x4c,
0xa4, 0xd1, 0x32, 0x54, 0x06, 0xba, 0x4b, 0xcc, 0x88, 0xcd, 0xc2, 0x53, 0xea, 0x1f, 0x15, 0x58, 0x1a, 0xad, 0x40, 0x65, 0xa8, 0xbb, 0xc4, 0x8c, 0xd8, 0x2c, 0x3c, 0xa5, 0xfe, 0x51, 0x81, 0xa5,
0xbc, 0xe5, 0x79, 0x66, 0xcf, 0x1e, 0xd1, 0x6c, 0x11, 0x66, 0x6c, 0xc7, 0xc0, 0xad, 0x4d, 0xa6, 0x8f, 0x3d, 0xcf, 0xec, 0xdb, 0x63, 0x9a, 0x2d, 0xc1, 0x9c, 0xed, 0x18, 0xb8, 0xbd, 0xc5, 0x54,
0x5a, 0x5e, 0x13, 0x23, 0xf4, 0x06, 0x94, 0x07, 0x18, 0xbb, 0x1d, 0xd7, 0xe9, 0xfb, 0x8a, 0x95, 0xcb, 0x6b, 0x62, 0x84, 0x5e, 0x82, 0xf2, 0x10, 0x63, 0xb7, 0xeb, 0x3a, 0x03, 0x5f, 0xb1, 0x12,
0xe8, 0x84, 0xe6, 0xf4, 0x31, 0xfa, 0x0c, 0xce, 0x7a, 0x31, 0x46, 0x5e, 0x23, 0xbf, 0x9c, 0x5f, 0x9d, 0xd0, 0x9c, 0x01, 0x46, 0x9f, 0xc1, 0x45, 0x2f, 0xb6, 0x90, 0xd7, 0xcc, 0xaf, 0xe4, 0x57,
0xa9, 0xac, 0xbf, 0xb5, 0x3a, 0xe2, 0x65, 0xab, 0x71, 0xa1, 0xda, 0xe8, 0x6a, 0xf5, 0x59, 0x0e, 0x2b, 0x1b, 0xaf, 0xac, 0x8d, 0x79, 0xd9, 0x5a, 0x5c, 0xa8, 0x36, 0xce, 0xad, 0x3e, 0xce, 0xc1,
0x16, 0x24, 0x1d, 0xd7, 0x95, 0xfe, 0xa6, 0x96, 0xf3, 0x70, 0x4f, 0xaa, 0xc7, 0x07, 0x59, 0x2c, 0xa2, 0xa4, 0xe3, 0xba, 0xd2, 0x67, 0x6a, 0x39, 0x0f, 0xf7, 0xa5, 0x7a, 0x7c, 0x90, 0xc5, 0x72,
0x27, 0x4d, 0x9e, 0x0f, 0x9b, 0x3c, 0x83, 0x83, 0xc5, 0xed, 0x59, 0x1c, 0xb1, 0x27, 0xba, 0x08, 0xd2, 0xe4, 0xf9, 0xb0, 0xc9, 0x33, 0x38, 0x58, 0xdc, 0x9e, 0xc5, 0x31, 0x7b, 0xa2, 0xab, 0x50,
0x15, 0xfc, 0x74, 0x60, 0xba, 0xb8, 0x43, 0x4c, 0x0b, 0x37, 0x66, 0x96, 0x95, 0x95, 0x82, 0x06, 0xc1, 0x8f, 0x86, 0xa6, 0x8b, 0xbb, 0xc4, 0xb4, 0x70, 0x73, 0x6e, 0x45, 0x59, 0x2d, 0x68, 0xc0,
0x7c, 0x6a, 0xcf, 0xb4, 0xc2, 0x1e, 0x39, 0x9b, 0xd9, 0x23, 0xd5, 0x3f, 0x29, 0x70, 0x7e, 0xe4, 0xa7, 0x0e, 0x4c, 0x2b, 0xec, 0x91, 0xf3, 0x99, 0x3d, 0x52, 0xfd, 0x93, 0x02, 0x97, 0xc7, 0x4e,
0x94, 0x84, 0x8b, 0x6b, 0x50, 0x67, 0x3b, 0x0f, 0x2c, 0x43, 0x9d, 0x9d, 0x1a, 0xfc, 0xf2, 0x38, 0x49, 0xb8, 0xb8, 0x06, 0x0d, 0xb6, 0xf3, 0xc0, 0x32, 0xd4, 0xd9, 0xa9, 0xc1, 0xaf, 0x4f, 0x32,
0x83, 0x07, 0xe4, 0xda, 0xc8, 0xfa, 0x90, 0x92, 0xb9, 0xec, 0x4a, 0x3e, 0x82, 0xf3, 0x5b, 0x98, 0x78, 0x40, 0xae, 0x8d, 0xf1, 0x87, 0x94, 0xcc, 0x65, 0x57, 0xf2, 0x3e, 0x5c, 0xde, 0xc1, 0x44,
0x08, 0x01, 0xf4, 0x1b, 0xf6, 0x4e, 0x0e, 0x01, 0xd1, 0x58, 0xca, 0x8d, 0xc4, 0xd2, 0x5f, 0x73, 0x08, 0xa0, 0xef, 0xb0, 0x77, 0x76, 0x08, 0x88, 0xc6, 0x52, 0x6e, 0x2c, 0x96, 0xfe, 0x9a, 0x93,
0x32, 0x96, 0x98, 0xa8, 0x96, 0x7d, 0xe0, 0xa0, 0x0b, 0x50, 0x96, 0x24, 0xc2, 0x2b, 0x82, 0x09, 0xb1, 0xc4, 0x44, 0xb5, 0xed, 0x23, 0x07, 0x5d, 0x81, 0xb2, 0x24, 0x11, 0x5e, 0x11, 0x4c, 0xa0,
0xf4, 0xff, 0x50, 0xa4, 0x9a, 0x72, 0x97, 0xa8, 0xad, 0x5f, 0x4a, 0xde, 0x53, 0x88, 0xa7, 0xc6, 0xff, 0x87, 0x22, 0xd5, 0x94, 0xbb, 0x44, 0x7d, 0xe3, 0x5a, 0xf2, 0x9e, 0x42, 0x6b, 0x6a, 0x9c,
0xe9, 0x51, 0x0b, 0x6a, 0x1e, 0xd1, 0x5d, 0xd2, 0x19, 0x38, 0x1e, 0x3b, 0x67, 0xe6, 0x38, 0x95, 0x1e, 0xb5, 0xa1, 0xee, 0x11, 0xdd, 0x25, 0xdd, 0xa1, 0xe3, 0xb1, 0x73, 0x66, 0x8e, 0x53, 0xd9,
0x75, 0x35, 0xca, 0x41, 0x42, 0xe4, 0x8e, 0xd7, 0xdb, 0x15, 0x94, 0xda, 0x1c, 0x5b, 0xe9, 0x0f, 0x50, 0xa3, 0x2b, 0x48, 0x88, 0xdc, 0xf3, 0xfa, 0xfb, 0x82, 0x52, 0xab, 0x31, 0x4e, 0x7f, 0x88,
0xd1, 0x1d, 0xa8, 0x62, 0xdb, 0x08, 0x18, 0x15, 0x32, 0x33, 0xaa, 0x60, 0xdb, 0x90, 0x6c, 0x82, 0x3e, 0x81, 0x2a, 0xb6, 0x8d, 0x60, 0xa1, 0x42, 0xe6, 0x85, 0x2a, 0xd8, 0x36, 0xe4, 0x32, 0xc1,
0xf3, 0x29, 0x66, 0x3f, 0x9f, 0x5f, 0x29, 0xd0, 0x18, 0x3d, 0xa0, 0xd3, 0x00, 0xe5, 0x4d, 0xbe, 0xf9, 0x14, 0xb3, 0x9f, 0xcf, 0xaf, 0x14, 0x68, 0x8e, 0x1f, 0xd0, 0x79, 0x80, 0xf2, 0x7d, 0xce,
0x08, 0xf3, 0x03, 0x1a, 0x1b, 0xe1, 0xf2, 0x90, 0x34, 0xb1, 0x44, 0x35, 0xe1, 0xb5, 0x40, 0x1b, 0x84, 0xf9, 0x01, 0x4d, 0x8c, 0x70, 0x79, 0x48, 0x9a, 0x60, 0x51, 0x4d, 0x78, 0x21, 0xd0, 0x86,
0xf6, 0xe5, 0x85, 0x39, 0xcb, 0xcf, 0x14, 0x58, 0x8c, 0xcb, 0x3a, 0xcd, 0xbe, 0xff, 0x0f, 0x8a, 0xbd, 0x79, 0x6a, 0xce, 0xf2, 0xb5, 0x02, 0x4b, 0x71, 0x59, 0xe7, 0xd9, 0xf7, 0xff, 0x41, 0xd1,
0xa6, 0x7d, 0xe0, 0xf8, 0xdb, 0x5e, 0x1a, 0x13, 0x67, 0x54, 0x16, 0x27, 0x56, 0x2d, 0x78, 0x63, 0xb4, 0x8f, 0x1c, 0x7f, 0xdb, 0xcb, 0x13, 0xe2, 0x8c, 0xca, 0xe2, 0xc4, 0xaa, 0x05, 0x2f, 0xed,
0x0b, 0x93, 0x96, 0xed, 0x61, 0x97, 0x6c, 0x98, 0x76, 0xdf, 0xe9, 0xed, 0xea, 0xe4, 0xf0, 0x14, 0x60, 0xd2, 0xb6, 0x3d, 0xec, 0x92, 0x4d, 0xd3, 0x1e, 0x38, 0xfd, 0x7d, 0x9d, 0x1c, 0x9f, 0x23,
0x31, 0x12, 0x71, 0xf7, 0x5c, 0xcc, 0xdd, 0xd5, 0x3f, 0x2b, 0x70, 0x21, 0x59, 0x9e, 0xd8, 0x7a, 0x46, 0x22, 0xee, 0x9e, 0x8b, 0xb9, 0xbb, 0xfa, 0x67, 0x05, 0xae, 0x24, 0xcb, 0x13, 0x5b, 0x6f,
0x13, 0x4a, 0x07, 0x26, 0xee, 0x1b, 0xd4, 0x66, 0x0a, 0xb3, 0x99, 0x1c, 0xd3, 0x58, 0x19, 0x50, 0x41, 0xe9, 0xc8, 0xc4, 0x03, 0x83, 0xda, 0x4c, 0x61, 0x36, 0x93, 0x63, 0x1a, 0x2b, 0x43, 0x4a,
0x62, 0xb1, 0xc3, 0x4b, 0x29, 0x0e, 0xda, 0x26, 0xae, 0x69, 0xf7, 0xb6, 0x4d, 0x8f, 0x68, 0x9c, 0x2c, 0x76, 0x78, 0x2d, 0xc5, 0x41, 0x3b, 0xc4, 0x35, 0xed, 0xfe, 0xae, 0xe9, 0x11, 0x8d, 0xd3,
0x3e, 0x64, 0xcf, 0x7c, 0x76, 0xcf, 0xfc, 0xa5, 0x02, 0x4b, 0x5b, 0x98, 0xdc, 0x96, 0x50, 0x4b, 0x87, 0xec, 0x99, 0xcf, 0xee, 0x99, 0xbf, 0x54, 0x60, 0x79, 0x07, 0x93, 0xdb, 0x12, 0x6a, 0xe9,
0xbf, 0x9b, 0x1e, 0x31, 0xbb, 0xde, 0x8b, 0x2d, 0x22, 0x12, 0x72, 0xa6, 0xfa, 0x8d, 0x02, 0x17, 0x7b, 0xd3, 0x23, 0x66, 0xcf, 0x7b, 0xba, 0x45, 0x44, 0x42, 0xce, 0x54, 0xbf, 0x51, 0xe0, 0x6a,
0x53, 0x95, 0x11, 0xa6, 0x13, 0x50, 0xe2, 0x03, 0x6d, 0x32, 0x94, 0xfc, 0x00, 0x1f, 0x3f, 0xd0, 0xaa, 0x32, 0xc2, 0x74, 0x02, 0x4a, 0x7c, 0xa0, 0x4d, 0x86, 0x92, 0x1f, 0xe0, 0xd3, 0xbb, 0xfa,
0xfb, 0x43, 0xbc, 0xab, 0x9b, 0x2e, 0x87, 0x92, 0x13, 0x02, 0xeb, 0x5f, 0x14, 0x78, 0x73, 0x0b, 0x60, 0x84, 0xf7, 0x75, 0xd3, 0xe5, 0x50, 0x72, 0x46, 0x60, 0xfd, 0x8b, 0x02, 0x2f, 0xef, 0x60,
0x93, 0x5d, 0x3f, 0xcd, 0xbc, 0x42, 0xeb, 0x64, 0xa8, 0x28, 0x7e, 0xc3, 0x0f, 0x33, 0x51, 0xdb, 0xb2, 0xef, 0xa7, 0x99, 0xe7, 0x68, 0x9d, 0x0c, 0x15, 0xc5, 0x6f, 0xf8, 0x61, 0x26, 0x6a, 0xfb,
0x57, 0x62, 0xbe, 0x25, 0x16, 0x07, 0xa1, 0x80, 0xbc, 0xcd, 0x6b, 0x01, 0x61, 0x3c, 0xf5, 0x59, 0x5c, 0xcc, 0xb7, 0xcc, 0xe2, 0x20, 0x14, 0x90, 0xb7, 0x79, 0x2d, 0x20, 0x8c, 0xa7, 0x3e, 0xce,
0x1e, 0xaa, 0x0f, 0x44, 0x7d, 0xc0, 0xd2, 0x48, 0xdc, 0x0e, 0x4a, 0xb2, 0x1d, 0x42, 0x25, 0x45, 0x43, 0xf5, 0xae, 0xa8, 0x0f, 0x58, 0x1a, 0x89, 0xdb, 0x41, 0x49, 0xb6, 0x43, 0xa8, 0xa4, 0x48,
0x52, 0x95, 0xb1, 0x05, 0x73, 0x1e, 0xc6, 0x8f, 0x4e, 0x92, 0x34, 0xaa, 0x74, 0xa1, 0x04, 0xfb, 0xaa, 0x32, 0x76, 0xa0, 0xe6, 0x61, 0x7c, 0xff, 0x2c, 0x49, 0xa3, 0x4a, 0x19, 0x25, 0xd8, 0xef,
0x6d, 0x38, 0x3b, 0xb4, 0x0f, 0x68, 0x59, 0x8b, 0x0d, 0xb1, 0x0b, 0x5e, 0x5d, 0x4e, 0x46, 0x9e, 0xc2, 0xc5, 0x91, 0x7d, 0x44, 0xcb, 0x5a, 0x6c, 0x88, 0x5d, 0xf0, 0xea, 0x72, 0x3a, 0xf2, 0x8c,
0xd1, 0x85, 0xe8, 0x1e, 0xcc, 0xc7, 0x79, 0x15, 0x33, 0xf1, 0x8a, 0x2f, 0x43, 0x2d, 0xa8, 0x1b, 0x33, 0xa2, 0x4f, 0x61, 0x21, 0xbe, 0x56, 0x31, 0xd3, 0x5a, 0x71, 0x36, 0xd4, 0x86, 0x86, 0xe1,
0xae, 0x33, 0x18, 0x60, 0xa3, 0xe3, 0xf9, 0xac, 0x66, 0xb2, 0xb1, 0x12, 0xeb, 0x7c, 0x56, 0xea, 0x3a, 0xc3, 0x21, 0x36, 0xba, 0x9e, 0xbf, 0xd4, 0x5c, 0xb6, 0xa5, 0x04, 0x9f, 0xbf, 0x94, 0xfa,
0x2f, 0x14, 0x58, 0x7c, 0xa8, 0x93, 0xee, 0xe1, 0xa6, 0x25, 0x0e, 0xe7, 0x14, 0xae, 0xfd, 0x09, 0x0b, 0x05, 0x96, 0xee, 0xe9, 0xa4, 0x77, 0xbc, 0x65, 0x89, 0xc3, 0x39, 0x87, 0x6b, 0x7f, 0x08,
0x94, 0x8f, 0xc4, 0x41, 0xf8, 0xf8, 0x75, 0x31, 0x41, 0xa1, 0xf0, 0x91, 0x6b, 0xc1, 0x0a, 0xf5, 0xe5, 0x13, 0x71, 0x10, 0x3e, 0x7e, 0x5d, 0x4d, 0x50, 0x28, 0x7c, 0xe4, 0x5a, 0xc0, 0xa1, 0xfe,
0x1f, 0x0a, 0x9c, 0x63, 0x4d, 0x84, 0xaf, 0xdd, 0xcb, 0x0f, 0xb2, 0x09, 0x8d, 0x04, 0xba, 0x0c, 0x43, 0x81, 0x4b, 0xac, 0x89, 0xf0, 0xb5, 0x7b, 0xf6, 0x41, 0x36, 0xa5, 0x91, 0x40, 0xd7, 0xa1,
0x35, 0x4b, 0x77, 0x1f, 0xb5, 0x03, 0x9a, 0x22, 0xa3, 0x89, 0xcd, 0xaa, 0x4f, 0x01, 0xc4, 0x68, 0x6e, 0xe9, 0xee, 0xfd, 0x4e, 0x40, 0x53, 0x64, 0x34, 0xb1, 0x59, 0xf5, 0x11, 0x80, 0x18, 0xed,
0xc7, 0xeb, 0x9d, 0x40, 0xff, 0x8f, 0x60, 0x56, 0x48, 0x15, 0xf1, 0x36, 0xe9, 0x60, 0x7d, 0x72, 0x79, 0xfd, 0x33, 0xe8, 0xff, 0x1e, 0xcc, 0x0b, 0xa9, 0x22, 0xde, 0xa6, 0x1d, 0xac, 0x4f, 0xae,
0xf5, 0x9f, 0x0a, 0xd4, 0x02, 0x04, 0x65, 0x51, 0x55, 0x83, 0x9c, 0x8c, 0xa5, 0x5c, 0x6b, 0x13, 0xfe, 0x53, 0x81, 0x7a, 0x80, 0xa0, 0x2c, 0xaa, 0xea, 0x90, 0x93, 0xb1, 0x94, 0x6b, 0x6f, 0xa1,
0x7d, 0x02, 0x33, 0xbc, 0x6d, 0x14, 0xbc, 0xdf, 0x89, 0xf2, 0x16, 0x2d, 0x65, 0x08, 0x86, 0xd9, 0x0f, 0x61, 0x8e, 0xb7, 0x8d, 0x62, 0xed, 0xd7, 0xa2, 0x6b, 0x8b, 0x96, 0x32, 0x04, 0xc3, 0x6c,
0x84, 0x26, 0x16, 0x51, 0x1b, 0x49, 0xd4, 0xe1, 0x1d, 0x46, 0x5e, 0x0b, 0xcd, 0xa0, 0x16, 0xcc, 0x42, 0x13, 0x4c, 0xd4, 0x46, 0x12, 0x75, 0x78, 0x87, 0x91, 0xd7, 0x42, 0x33, 0xa8, 0x0d, 0x0b,
0x47, 0x8b, 0x36, 0x3f, 0x66, 0x96, 0xd3, 0xd0, 0x66, 0x53, 0x27, 0x3a, 0x03, 0x9b, 0x5a, 0xa4, 0xd1, 0xa2, 0xcd, 0x8f, 0x99, 0x95, 0x34, 0xb4, 0xd9, 0xd2, 0x89, 0xce, 0xc0, 0xa6, 0x1e, 0xa9,
0x66, 0xf3, 0xd4, 0xff, 0x16, 0xa1, 0x12, 0xda, 0xe5, 0xc8, 0x4e, 0xe2, 0x47, 0x9a, 0x9b, 0x8c, 0xd9, 0x3c, 0xf5, 0xbf, 0x45, 0xa8, 0x84, 0x76, 0x39, 0xb6, 0x93, 0xf8, 0x91, 0xe6, 0xa6, 0xe3,
0x9b, 0xf9, 0xd1, 0xce, 0xe1, 0x1d, 0xa8, 0x99, 0x2c, 0x57, 0x77, 0x84, 0x2b, 0x32, 0x70, 0x2d, 0x66, 0x7e, 0xbc, 0x73, 0x78, 0x0d, 0xea, 0x26, 0xcb, 0xd5, 0x5d, 0xe1, 0x8a, 0x0c, 0x5c, 0xcb,
0x6b, 0x73, 0x7c, 0x56, 0xc4, 0x05, 0x5a, 0x82, 0x8a, 0x3d, 0xb4, 0x3a, 0xce, 0x41, 0xc7, 0x75, 0x5a, 0x8d, 0xcf, 0x8a, 0xb8, 0x40, 0xcb, 0x50, 0xb1, 0x47, 0x56, 0xd7, 0x39, 0xea, 0xba, 0xce,
0x9e, 0x78, 0xa2, 0x05, 0x29, 0xdb, 0x43, 0xeb, 0x87, 0x07, 0x9a, 0xf3, 0xc4, 0x0b, 0xaa, 0xdc, 0x43, 0x4f, 0xb4, 0x20, 0x65, 0x7b, 0x64, 0xfd, 0xf0, 0x48, 0x73, 0x1e, 0x7a, 0x41, 0x95, 0x3b,
0x99, 0x29, 0xab, 0xdc, 0x25, 0xa8, 0x58, 0xfa, 0x53, 0xca, 0xb5, 0x63, 0x0f, 0x2d, 0xd6, 0x9d, 0x37, 0x63, 0x95, 0xbb, 0x0c, 0x15, 0x4b, 0x7f, 0x44, 0x57, 0xed, 0xda, 0x23, 0x8b, 0x75, 0x27,
0xe4, 0xb5, 0xb2, 0xa5, 0x3f, 0xd5, 0x9c, 0x27, 0xf7, 0x87, 0x16, 0x5a, 0x81, 0x7a, 0x5f, 0xf7, 0x79, 0xad, 0x6c, 0xe9, 0x8f, 0x34, 0xe7, 0xe1, 0x9d, 0x91, 0x85, 0x56, 0xa1, 0x31, 0xd0, 0x3d,
0x48, 0x27, 0xdc, 0xde, 0x94, 0x58, 0x7b, 0x53, 0xa3, 0xf3, 0x77, 0x82, 0x16, 0x67, 0xb4, 0x5e, 0xd2, 0x0d, 0xb7, 0x37, 0x25, 0xd6, 0xde, 0xd4, 0xe9, 0xfc, 0x27, 0x41, 0x8b, 0x33, 0x5e, 0x2f,
0x2e, 0x9f, 0xa2, 0x5e, 0x36, 0xac, 0x7e, 0xc0, 0x08, 0xb2, 0xd7, 0xcb, 0x86, 0xd5, 0x97, 0x6c, 0x97, 0xcf, 0x51, 0x2f, 0x1b, 0xd6, 0x20, 0x58, 0x08, 0xb2, 0xd7, 0xcb, 0x86, 0x35, 0x90, 0xcb,
0x3e, 0x82, 0xd9, 0x7d, 0x56, 0x01, 0x79, 0x8d, 0x4a, 0x2a, 0x42, 0xdd, 0xa5, 0xc5, 0x0f, 0x2f, 0xbc, 0x07, 0xf3, 0x87, 0xac, 0x02, 0xf2, 0x9a, 0x95, 0x54, 0x84, 0xda, 0xa6, 0xc5, 0x0f, 0x2f,
0x94, 0x34, 0x9f, 0x1c, 0x7d, 0x0c, 0x65, 0x96, 0x7a, 0xd8, 0xda, 0x6a, 0xa6, 0xb5, 0xc1, 0x02, 0x94, 0x34, 0x9f, 0x1c, 0x7d, 0x00, 0x65, 0x96, 0x7a, 0x18, 0x6f, 0x35, 0x13, 0x6f, 0xc0, 0x40,
0x0a, 0x45, 0x06, 0xee, 0x13, 0x9d, 0xad, 0x9e, 0x4b, 0x85, 0xa2, 0x4d, 0x4a, 0xb3, 0xed, 0xf4, 0xa1, 0xc8, 0xc0, 0x03, 0xa2, 0x33, 0xee, 0x5a, 0x2a, 0x14, 0x6d, 0x51, 0x9a, 0x5d, 0xa7, 0xcf,
0x38, 0x14, 0xc9, 0x15, 0xe8, 0x1a, 0x2c, 0x74, 0x5d, 0xac, 0x13, 0x6c, 0x6c, 0x1c, 0xdf, 0x76, 0xa1, 0x48, 0x72, 0xa0, 0xb7, 0x61, 0xb1, 0xe7, 0x62, 0x9d, 0x60, 0x63, 0xf3, 0xf4, 0xb6, 0x63,
0xac, 0x81, 0xce, 0xbc, 0xa9, 0x51, 0x5b, 0x56, 0x56, 0x4a, 0x5a, 0xd2, 0x27, 0x8a, 0x0c, 0x5d, 0x0d, 0x75, 0xe6, 0x4d, 0xcd, 0xfa, 0x8a, 0xb2, 0x5a, 0xd2, 0x92, 0x5e, 0x51, 0x64, 0xe8, 0xc9,
0x39, 0xba, 0xeb, 0x3a, 0x56, 0x63, 0x9e, 0x23, 0x43, 0x74, 0x16, 0xbd, 0x09, 0xe0, 0x63, 0xb7, 0xd1, 0xb6, 0xeb, 0x58, 0xcd, 0x05, 0x8e, 0x0c, 0xd1, 0x59, 0xf4, 0x32, 0x80, 0x8f, 0xdd, 0x3a,
0x4e, 0x1a, 0x75, 0x76, 0x8c, 0x65, 0x31, 0x73, 0x8b, 0xa8, 0x5f, 0xc1, 0xb9, 0xc0, 0x45, 0x42, 0x69, 0x36, 0xd8, 0x31, 0x96, 0xc5, 0xcc, 0xc7, 0x44, 0xfd, 0x0a, 0x2e, 0x05, 0x2e, 0x12, 0x3a,
0xc7, 0x31, 0x7a, 0xb2, 0xca, 0x49, 0x4f, 0x76, 0x7c, 0xf1, 0xfa, 0xb7, 0x02, 0x2c, 0xb6, 0xf5, 0x8e, 0xf1, 0x93, 0x55, 0xce, 0x7a, 0xb2, 0x93, 0x8b, 0xd7, 0xbf, 0x15, 0x60, 0xa9, 0xa3, 0x9f,
0x23, 0xfc, 0xe2, 0xeb, 0xe4, 0x4c, 0x80, 0xbc, 0x0d, 0x67, 0x59, 0x69, 0xbc, 0x1e, 0xd2, 0x67, 0xe0, 0xa7, 0x5f, 0x27, 0x67, 0x02, 0xe4, 0x5d, 0xb8, 0xc8, 0x4a, 0xe3, 0x8d, 0x90, 0x3e, 0x13,
0x4c, 0x0a, 0x0e, 0x7b, 0xc3, 0xe8, 0x42, 0xf4, 0x29, 0xad, 0x1d, 0x70, 0xf7, 0xd1, 0xae, 0x63, 0x52, 0x70, 0xd8, 0x1b, 0xc6, 0x19, 0xd1, 0x47, 0xb4, 0x76, 0xc0, 0xbd, 0xfb, 0xfb, 0x8e, 0x19,
0x06, 0xe9, 0xf7, 0xcd, 0x04, 0x3e, 0xb7, 0x25, 0x95, 0x16, 0x5e, 0x81, 0x76, 0x47, 0xb1, 0x8d, 0xa4, 0xdf, 0x97, 0x13, 0xd6, 0xb9, 0x2d, 0xa9, 0xb4, 0x30, 0x07, 0xda, 0x1f, 0xc7, 0x36, 0x9e,
0x27, 0xde, 0x77, 0xc7, 0x36, 0x60, 0x81, 0xf5, 0xe3, 0x10, 0x87, 0x1a, 0x30, 0x2b, 0xd2, 0x3b, 0x78, 0x5f, 0x9f, 0xd8, 0x80, 0x05, 0xd6, 0x8f, 0x43, 0x1c, 0x6a, 0xc2, 0xbc, 0x48, 0xef, 0x2c,
0x0b, 0xfc, 0x92, 0xe6, 0x0f, 0xd1, 0x2e, 0x2c, 0xf0, 0x1d, 0xb4, 0x85, 0x57, 0xf3, 0xcd, 0x97, 0xf0, 0x4b, 0x9a, 0x3f, 0x44, 0xfb, 0xb0, 0xc8, 0x77, 0xd0, 0x11, 0x5e, 0xcd, 0x37, 0x5f, 0xca,
0x32, 0x6d, 0x3e, 0x69, 0x69, 0x34, 0x28, 0xca, 0x53, 0x07, 0x45, 0x03, 0x66, 0x85, 0xa3, 0x32, 0xb4, 0xf9, 0x24, 0xd6, 0x68, 0x50, 0x94, 0x67, 0x0e, 0x8a, 0x26, 0xcc, 0x0b, 0x47, 0x65, 0x68,
0x34, 0x28, 0x69, 0xfe, 0x90, 0xb6, 0x11, 0x10, 0x98, 0x6c, 0xc2, 0x6d, 0xc0, 0xf7, 0xa1, 0x24, 0x50, 0xd2, 0xfc, 0x21, 0x6d, 0x23, 0x20, 0x30, 0xd9, 0x94, 0xdb, 0x80, 0xef, 0x43, 0x49, 0x3a,
0x9d, 0x38, 0x97, 0xd9, 0x89, 0xe5, 0x9a, 0x38, 0x0e, 0xe7, 0x63, 0x38, 0xac, 0xfe, 0x4b, 0x81, 0x71, 0x2e, 0xb3, 0x13, 0x4b, 0x9e, 0x38, 0x0e, 0xe7, 0x63, 0x38, 0xac, 0xfe, 0x4b, 0x81, 0x6a,
0x6a, 0x78, 0x0b, 0x14, 0xdf, 0x5d, 0xdc, 0x75, 0x5c, 0xa3, 0x83, 0x6d, 0xe2, 0x9a, 0x98, 0x77, 0x78, 0x0b, 0x14, 0xdf, 0x5d, 0xdc, 0x73, 0x5c, 0xa3, 0x8b, 0x6d, 0xe2, 0x9a, 0x98, 0x77, 0x9c,
0x9c, 0x05, 0x6d, 0x8e, 0xcf, 0xde, 0xe1, 0x93, 0x94, 0x8c, 0x42, 0xab, 0x47, 0x74, 0x6b, 0xd0, 0x05, 0xad, 0xc6, 0x67, 0x3f, 0xe1, 0x93, 0x94, 0x8c, 0x42, 0xab, 0x47, 0x74, 0x6b, 0xd8, 0x3d,
0x39, 0xa0, 0x11, 0x9c, 0xe3, 0x64, 0x72, 0x96, 0x05, 0xf0, 0x25, 0xa8, 0x06, 0x64, 0xc4, 0x61, 0xa2, 0x11, 0x9c, 0xe3, 0x64, 0x72, 0x96, 0x05, 0xf0, 0x35, 0xa8, 0x06, 0x64, 0xc4, 0x61, 0xf2,
0xf2, 0x0b, 0x5a, 0x45, 0xce, 0xed, 0x39, 0xe8, 0x6d, 0xa8, 0x31, 0xab, 0x75, 0xfa, 0x4e, 0xaf, 0x0b, 0x5a, 0x45, 0xce, 0x1d, 0x38, 0xe8, 0x55, 0xa8, 0x33, 0xab, 0x75, 0x07, 0x4e, 0xbf, 0x4b,
0x43, 0xbb, 0x33, 0x91, 0x50, 0xaa, 0x86, 0x50, 0x8b, 0x1e, 0x47, 0x94, 0xca, 0x33, 0xbf, 0xc4, 0xbb, 0x33, 0x91, 0x50, 0xaa, 0x86, 0x50, 0x8b, 0x1e, 0x47, 0x94, 0xca, 0x33, 0xbf, 0xc4, 0x22,
0x22, 0xa5, 0x48, 0xaa, 0xb6, 0xf9, 0x25, 0x56, 0xbf, 0x56, 0x60, 0x8e, 0xe6, 0xc7, 0xfb, 0x8e, 0xa5, 0x48, 0xaa, 0x8e, 0xf9, 0x25, 0xa6, 0xf9, 0xbc, 0x46, 0xf3, 0xe3, 0x1d, 0xc7, 0xc0, 0x07,
0x81, 0xf7, 0x4e, 0x58, 0x4d, 0x64, 0xb8, 0x99, 0xbb, 0x00, 0x65, 0xb9, 0x03, 0xb1, 0xa5, 0x60, 0x67, 0xac, 0x26, 0x32, 0xdc, 0xcc, 0x5d, 0x81, 0xb2, 0xdc, 0x81, 0xd8, 0x52, 0x30, 0x81, 0xb6,
0x82, 0xb6, 0xf1, 0x73, 0x22, 0x0d, 0xb6, 0xe5, 0x4d, 0x2d, 0x63, 0xa5, 0x30, 0x56, 0xec, 0x37, 0xa1, 0xee, 0x17, 0x9a, 0x5d, 0xde, 0x3f, 0x14, 0x52, 0xbd, 0x27, 0x94, 0xe1, 0x3c, 0xad, 0xe6,
0xfa, 0x5e, 0xf4, 0x9a, 0xe7, 0xed, 0xc4, 0xb8, 0x62, 0x4c, 0x58, 0xc5, 0x19, 0xc9, 0x81, 0x59, 0xb3, 0xb1, 0xa1, 0xba, 0x0d, 0xd5, 0xf0, 0x6b, 0x2a, 0xb5, 0x13, 0x77, 0x14, 0x39, 0x41, 0xfd,
0xfa, 0xc3, 0x67, 0xf4, 0x60, 0x85, 0x29, 0xd8, 0xc1, 0x36, 0x60, 0x56, 0x37, 0x0c, 0x17, 0x7b, 0xed, 0xce, 0xc8, 0xa2, 0x67, 0x2a, 0xb0, 0xc3, 0x1f, 0xaa, 0x5f, 0x2b, 0x50, 0x13, 0x69, 0xb9,
0x9e, 0xd0, 0xc3, 0x1f, 0xd2, 0x2f, 0x47, 0xd8, 0xf5, 0x7c, 0x17, 0xcb, 0x6b, 0xfe, 0x10, 0x7d, 0x23, 0x6f, 0x8e, 0xd9, 0xd6, 0x14, 0xb6, 0x35, 0xf6, 0x8c, 0xbe, 0x17, 0xbd, 0x76, 0x7a, 0x35,
0x0c, 0x25, 0x59, 0xa2, 0xe6, 0x93, 0xca, 0x92, 0xb0, 0x9e, 0xa2, 0x9f, 0x91, 0x2b, 0xd4, 0x6f, 0x31, 0xce, 0xd9, 0x22, 0xac, 0x02, 0x8e, 0xe4, 0xe4, 0x2c, 0xfd, 0xea, 0x63, 0xea, 0x68, 0xe2,
0x72, 0x50, 0x13, 0x61, 0xbd, 0x21, 0xf2, 0xd4, 0x78, 0x67, 0xdf, 0x80, 0xea, 0x41, 0x10, 0x96, 0x68, 0x98, 0xa3, 0x35, 0x61, 0x5e, 0x37, 0x0c, 0x17, 0x7b, 0x9e, 0xd0, 0xc3, 0x1f, 0xd2, 0x37,
0xe3, 0xee, 0x2d, 0xc2, 0xd1, 0x1b, 0x59, 0x33, 0xc9, 0xe1, 0xa3, 0x99, 0xb2, 0x70, 0xaa, 0x4c, 0x27, 0xd8, 0xf5, 0x7c, 0x97, 0xcf, 0x6b, 0xfe, 0x10, 0x7d, 0x00, 0x25, 0x59, 0x32, 0xe7, 0x93,
0x59, 0x9c, 0x16, 0x14, 0xd4, 0x5b, 0x50, 0x09, 0x31, 0x66, 0x70, 0xc6, 0xaf, 0x32, 0x84, 0x2d, 0xca, 0xa4, 0xb0, 0x9e, 0xa2, 0xbf, 0x92, 0x1c, 0xea, 0x37, 0x39, 0xa8, 0x0b, 0x83, 0x6d, 0x8a,
0xfc, 0x21, 0xfd, 0xb2, 0x1f, 0x32, 0x42, 0x59, 0x66, 0x7a, 0x5a, 0xf7, 0x9f, 0xdf, 0xc2, 0x44, 0xbc, 0x39, 0x39, 0xf8, 0x36, 0xa1, 0x7a, 0x14, 0xc0, 0xc4, 0xa4, 0x7b, 0x94, 0x30, 0x9a, 0x44,
0xc3, 0x5d, 0xe7, 0x08, 0xbb, 0xc7, 0xa7, 0xbf, 0x25, 0xba, 0x19, 0x3a, 0xe3, 0x8c, 0x6d, 0x88, 0x78, 0xa6, 0x05, 0x60, 0x34, 0x73, 0x17, 0xce, 0x95, 0xb9, 0x8b, 0xb3, 0x82, 0x94, 0xfa, 0x31,
0x5c, 0x80, 0x6e, 0x06, 0x7a, 0xe6, 0x93, 0x9a, 0xe4, 0x30, 0xb4, 0x8b, 0x13, 0x0a, 0xb6, 0xf2, 0x54, 0x42, 0x0b, 0x33, 0x78, 0xe5, 0x57, 0x2b, 0xc2, 0x16, 0xfe, 0x90, 0xbe, 0x39, 0x0c, 0x19,
0x2d, 0xbf, 0xef, 0x8a, 0x6e, 0xe5, 0xa4, 0xd9, 0xf3, 0xb9, 0x54, 0xb7, 0xea, 0xef, 0x14, 0x78, 0xa1, 0x2c, 0x2b, 0x0f, 0xda, 0x87, 0x5c, 0xde, 0xc1, 0x44, 0xc3, 0x3d, 0xe7, 0x04, 0xbb, 0xa7,
0x7d, 0x0b, 0x93, 0xbb, 0xd1, 0x1e, 0xf2, 0x55, 0x6b, 0x65, 0x41, 0x33, 0x49, 0xa9, 0xd3, 0x9c, 0xe7, 0xbf, 0xb5, 0x7a, 0x3f, 0x74, 0xc6, 0x19, 0xdb, 0x22, 0xc9, 0x80, 0xde, 0x0f, 0xf4, 0xcc,
0x7a, 0x13, 0x4a, 0xb2, 0x1b, 0xe6, 0x37, 0x91, 0x72, 0xac, 0xfe, 0x5c, 0x81, 0x86, 0x90, 0xc2, 0x27, 0x35, 0xed, 0xe1, 0xa0, 0x13, 0x27, 0x14, 0x6c, 0xe5, 0x5b, 0x7e, 0xff, 0x16, 0xdd, 0xca,
0x64, 0xd2, 0xc2, 0xad, 0x8f, 0x09, 0x36, 0x5e, 0x76, 0x7b, 0xf6, 0x07, 0x05, 0xea, 0x61, 0x10, 0x59, 0xb3, 0xf9, 0x13, 0xa9, 0xb6, 0xd5, 0xdf, 0x29, 0xf0, 0xe2, 0x0e, 0x26, 0xdb, 0xd1, 0x9e,
0x64, 0x38, 0xf6, 0x21, 0x14, 0x59, 0x17, 0x2c, 0x34, 0x98, 0xe8, 0xac, 0x9c, 0x9a, 0x46, 0x14, 0xf6, 0x79, 0x6b, 0x65, 0x41, 0x2b, 0x49, 0xa9, 0xf3, 0x9c, 0x7a, 0x0b, 0x4a, 0xb2, 0x3b, 0xe7,
0x2b, 0x26, 0xf6, 0x3c, 0x1f, 0xe4, 0xc4, 0x30, 0x40, 0xe2, 0xfc, 0xd4, 0x48, 0xac, 0xfe, 0x3a, 0x37, 0xa3, 0x72, 0xac, 0xfe, 0x5c, 0x81, 0xa6, 0x90, 0xc2, 0x64, 0xd2, 0x42, 0x72, 0x80, 0x09,
0x07, 0x8d, 0xa0, 0xae, 0x7d, 0xe9, 0x60, 0x97, 0x52, 0xf5, 0xe4, 0x9f, 0x53, 0xd5, 0x53, 0x98, 0x36, 0x9e, 0x75, 0xbb, 0xf8, 0x07, 0x05, 0x1a, 0x61, 0x10, 0x64, 0x38, 0xf6, 0x2e, 0x14, 0x59,
0x1a, 0xe0, 0xfe, 0x9e, 0xa3, 0x0d, 0xb5, 0x6f, 0x8f, 0xdd, 0xbe, 0x6e, 0xa3, 0x45, 0x98, 0x19, 0x57, 0x2e, 0x34, 0x98, 0xea, 0xac, 0x9c, 0x9a, 0x46, 0x14, 0x2b, 0x6e, 0x0e, 0x24, 0x5e, 0x8b,
0xf4, 0xf5, 0xe0, 0x82, 0x4a, 0x8c, 0x50, 0x1b, 0x6a, 0x5e, 0xc4, 0x5e, 0xc2, 0x02, 0xef, 0x27, 0x61, 0x80, 0xc4, 0xf9, 0x99, 0x91, 0x58, 0xfd, 0x75, 0x0e, 0x9a, 0x41, 0x9d, 0xfd, 0xcc, 0xc1,
0xd9, 0x3f, 0xc5, 0xc4, 0x5a, 0x8c, 0x05, 0x6d, 0x18, 0x78, 0xc9, 0xc9, 0xfa, 0x3e, 0x91, 0x9a, 0x2e, 0xa5, 0x0a, 0xcb, 0x3f, 0xa1, 0x2a, 0xac, 0x30, 0x33, 0xc0, 0xfd, 0x3d, 0x47, 0x1b, 0x7c,
0xf9, 0x41, 0xd3, 0x96, 0xef, 0x2a, 0x20, 0xfa, 0xc1, 0x19, 0x92, 0x8e, 0x69, 0x77, 0x3c, 0xdc, 0xdf, 0x1e, 0xfb, 0x03, 0xdd, 0x46, 0x4b, 0x30, 0x37, 0x1c, 0xe8, 0xc1, 0x85, 0x99, 0x18, 0xa1,
0x75, 0x6c, 0xc3, 0x63, 0xf5, 0x46, 0x51, 0xab, 0x8b, 0x2f, 0x2d, 0xbb, 0xcd, 0xe7, 0xd1, 0x87, 0x8e, 0x4c, 0xdb, 0x51, 0x0b, 0xbc, 0x99, 0x64, 0xff, 0x14, 0x13, 0x6b, 0xb1, 0x25, 0x68, 0x03,
0x50, 0x20, 0xc7, 0x03, 0x5e, 0x69, 0xd4, 0x12, 0x91, 0x2d, 0xd0, 0x6b, 0xef, 0x78, 0x80, 0x35, 0xc3, 0x4b, 0x60, 0xd6, 0x87, 0x8a, 0x52, 0x81, 0x1f, 0x34, 0x6d, 0x41, 0x6f, 0x02, 0xa2, 0x2f,
0x46, 0x4e, 0x5b, 0x7e, 0xca, 0x8a, 0xb8, 0xfa, 0x11, 0xee, 0xfb, 0x4f, 0x6b, 0xc1, 0x0c, 0xf5, 0x9c, 0x11, 0xe9, 0x9a, 0x76, 0xd7, 0xc3, 0x3d, 0xc7, 0x36, 0x3c, 0x56, 0xff, 0x14, 0xb5, 0x86,
0x44, 0xbf, 0x75, 0x9e, 0xe5, 0x89, 0x58, 0x0c, 0xd5, 0xef, 0x72, 0x50, 0x0f, 0x58, 0x6a, 0xd8, 0x78, 0xd3, 0xb6, 0x3b, 0x7c, 0x1e, 0xbd, 0x0b, 0x05, 0x72, 0x3a, 0xe4, 0x95, 0x4f, 0x3d, 0x11,
0x1b, 0xf6, 0x49, 0xaa, 0xfd, 0xc6, 0xb7, 0x0b, 0x93, 0xd2, 0xe0, 0xa7, 0x50, 0x11, 0x6d, 0xfc, 0xd9, 0x02, 0xbd, 0x0e, 0x4e, 0x87, 0x58, 0x63, 0xe4, 0x68, 0x19, 0x80, 0x2e, 0x45, 0x5c, 0xfd,
0x14, 0x89, 0x10, 0xf8, 0x92, 0xed, 0x31, 0xae, 0x57, 0x7c, 0x4e, 0xae, 0x37, 0x33, 0xb5, 0xeb, 0x04, 0x0f, 0xfc, 0x4f, 0x7d, 0xc1, 0x0c, 0xf5, 0x44, 0xbf, 0x95, 0x9f, 0xe7, 0x89, 0x58, 0x0c,
0xb5, 0x61, 0xd1, 0x07, 0xad, 0x40, 0xd2, 0x0e, 0x26, 0xfa, 0x98, 0x34, 0x7b, 0x11, 0x2a, 0x3c, 0xd5, 0xef, 0x72, 0xd0, 0x08, 0x96, 0xd4, 0xb0, 0x37, 0x1a, 0x90, 0x54, 0xfb, 0x4d, 0x6e, 0x5f,
0x19, 0xf1, 0xc2, 0x93, 0x97, 0x7a, 0xb0, 0x2f, 0x9b, 0x20, 0xf5, 0x27, 0x70, 0x8e, 0x05, 0x7d, 0xa6, 0xa5, 0xc1, 0x8f, 0xa0, 0x22, 0xae, 0x15, 0x66, 0x48, 0x84, 0xc0, 0x59, 0x76, 0x27, 0xb8,
0xfc, 0xba, 0x2f, 0xcb, 0xdd, 0xab, 0x2a, 0x0b, 0x49, 0x5a, 0x34, 0xfa, 0x89, 0x3c, 0x32, 0xa7, 0x5e, 0xf1, 0x09, 0xb9, 0xde, 0xdc, 0xcc, 0xae, 0xd7, 0x81, 0x25, 0x1f, 0xb4, 0x02, 0x49, 0x7b,
0x6e, 0xc3, 0x6b, 0x31, 0xfe, 0xa7, 0x00, 0x75, 0xf5, 0x3b, 0x05, 0x5e, 0xdf, 0x74, 0x9d, 0xc1, 0x98, 0xe8, 0x13, 0xd2, 0xec, 0x55, 0xa8, 0xf0, 0x64, 0xc4, 0x0b, 0x61, 0x5e, 0x7a, 0xc2, 0xa1,
0x03, 0xd3, 0x25, 0x43, 0xbd, 0x1f, 0xbd, 0x40, 0x7e, 0x31, 0xa5, 0xf0, 0xbd, 0x50, 0x1e, 0xe1, 0x6c, 0xca, 0xd4, 0x9f, 0xc0, 0x25, 0x16, 0xf4, 0xf1, 0xeb, 0xc7, 0x2c, 0x77, 0xc1, 0xaa, 0x2c,
0xb0, 0x73, 0x35, 0xe9, 0xcc, 0x46, 0x94, 0x12, 0x47, 0x15, 0xca, 0x3a, 0xff, 0xc9, 0x27, 0x29, 0x6c, 0x69, 0x11, 0xeb, 0x27, 0xf2, 0xc8, 0x9c, 0xba, 0x0b, 0x2f, 0xc4, 0xd6, 0x3f, 0x07, 0xa8,
0x2f, 0xe8, 0x26, 0x60, 0x69, 0x96, 0x34, 0x9b, 0xd8, 0x1c, 0xe7, 0x4f, 0xda, 0x1c, 0xa7, 0xb8, 0xab, 0xdf, 0x29, 0xf0, 0xe2, 0x96, 0xeb, 0x0c, 0xef, 0x9a, 0x2e, 0x19, 0xe9, 0x83, 0xe8, 0x85,
0x7f, 0xe1, 0x39, 0xb9, 0xff, 0xd4, 0xa5, 0x25, 0xba, 0x07, 0xd1, 0x9b, 0x0b, 0x06, 0x3c, 0x27, 0xf6, 0xd3, 0x29, 0xcd, 0x3f, 0x0d, 0xe5, 0x11, 0x0e, 0x3b, 0x37, 0x93, 0xce, 0x6c, 0x4c, 0x29,
0xba, 0xf2, 0xd8, 0x00, 0x08, 0xba, 0x78, 0xf1, 0xfc, 0x9f, 0x85, 0x4d, 0x68, 0x15, 0x3d, 0x2e, 0x71, 0x54, 0xa1, 0xac, 0xf3, 0x9f, 0x7c, 0x92, 0xf2, 0x82, 0x6e, 0x0a, 0x96, 0x66, 0x49, 0xb3,
0x89, 0x35, 0xec, 0xfa, 0x2d, 0xd2, 0x74, 0x7e, 0x06, 0xcd, 0x24, 0x37, 0x3d, 0x85, 0xeb, 0x5f, 0x89, 0xcd, 0x7a, 0xfe, 0xac, 0xcd, 0x7a, 0x8a, 0xfb, 0x17, 0x9e, 0x90, 0xfb, 0xcf, 0x5c, 0x5a,
0xb9, 0x0e, 0x67, 0x47, 0x92, 0x34, 0xaa, 0x01, 0x7c, 0x6e, 0x77, 0x45, 0xf5, 0x52, 0x3f, 0x83, 0xa2, 0x4f, 0x21, 0x7a, 0x93, 0xc2, 0x80, 0xe7, 0x4c, 0x57, 0x30, 0x9b, 0x00, 0xc1, 0xad, 0x82,
0xaa, 0x50, 0xf2, 0x6b, 0x99, 0xba, 0x72, 0xa5, 0x1d, 0x4e, 0x55, 0x14, 0xbf, 0xd1, 0x79, 0x58, 0xf8, 0x1d, 0x21, 0xcb, 0x32, 0x21, 0x2e, 0x7a, 0x5c, 0x12, 0x6b, 0xd8, 0x75, 0x60, 0xa4, 0x09,
0xf8, 0xdc, 0x36, 0xf0, 0x81, 0x69, 0x63, 0x23, 0xf8, 0x54, 0x3f, 0x83, 0x16, 0x60, 0xbe, 0x65, 0xfe, 0x0c, 0x5a, 0x49, 0x6e, 0x7a, 0x0e, 0xd7, 0xbf, 0x71, 0x0b, 0x2e, 0x8e, 0x25, 0x69, 0x54,
0xdb, 0xd8, 0x0d, 0x4d, 0x2a, 0x74, 0x72, 0x07, 0xbb, 0x3d, 0x1c, 0x9a, 0xcc, 0xad, 0x7f, 0xbb, 0x07, 0xf8, 0xdc, 0xee, 0x89, 0xea, 0xa5, 0x71, 0x01, 0x55, 0xa1, 0xe4, 0xd7, 0x32, 0x0d, 0xe5,
0x00, 0x65, 0xda, 0x76, 0xdd, 0x76, 0x1c, 0xd7, 0x40, 0x03, 0x40, 0xec, 0x8d, 0xce, 0x1a, 0x38, 0x46, 0x27, 0x9c, 0xaa, 0x28, 0x7e, 0xa3, 0xcb, 0xb0, 0xf8, 0xb9, 0x6d, 0xe0, 0x23, 0xd3, 0xc6,
0xb6, 0x7c, 0xcc, 0x46, 0xd7, 0x52, 0x8c, 0x39, 0x4a, 0x2a, 0x42, 0xb7, 0x79, 0x39, 0x65, 0x45, 0x46, 0xf0, 0xaa, 0x71, 0x01, 0x2d, 0xc2, 0x42, 0xdb, 0xb6, 0xb1, 0x1b, 0x9a, 0x54, 0xe8, 0xe4,
0x8c, 0x5c, 0x3d, 0x83, 0x2c, 0x26, 0x91, 0x26, 0xbb, 0x3d, 0xb3, 0xfb, 0xc8, 0xbf, 0x8d, 0x1d, 0x1e, 0x76, 0xfb, 0x38, 0x34, 0x99, 0xdb, 0xf8, 0x76, 0x11, 0xca, 0xb4, 0xed, 0xba, 0xed, 0x38,
0x23, 0x31, 0x46, 0xea, 0x4b, 0x8c, 0xbd, 0x91, 0x8b, 0x01, 0x7f, 0x48, 0xf5, 0x4f, 0x4a, 0x3d, 0xae, 0x81, 0x86, 0x80, 0xd8, 0x37, 0x43, 0x6b, 0xe8, 0xd8, 0xf2, 0xe3, 0x3a, 0x7a, 0x3b, 0xc5,
0x83, 0x1e, 0xc3, 0xb9, 0x2d, 0x4c, 0x82, 0xb7, 0x33, 0x5f, 0xe0, 0x7a, 0xba, 0xc0, 0x11, 0xe2, 0x98, 0xe3, 0xa4, 0x22, 0x74, 0x5b, 0xd7, 0x53, 0x38, 0x62, 0xe4, 0xea, 0x05, 0x64, 0x31, 0x89,
0x29, 0x45, 0x6e, 0x43, 0x91, 0x55, 0xa5, 0x28, 0x29, 0x3a, 0xc2, 0xff, 0xe8, 0x6a, 0x2e, 0xa7, 0x34, 0xd9, 0x1d, 0x98, 0xbd, 0xfb, 0xfe, 0xed, 0xf0, 0x04, 0x89, 0x31, 0x52, 0x5f, 0x62, 0xec,
0x13, 0x48, 0x6e, 0x3f, 0x85, 0xf9, 0xd8, 0x3f, 0x56, 0xd0, 0x7b, 0x09, 0xcb, 0x92, 0xff, 0x7b, 0x9b, 0xbd, 0x18, 0xf0, 0x0f, 0xbb, 0xfe, 0x49, 0xa9, 0x17, 0xd0, 0x03, 0xb8, 0xb4, 0x83, 0x49,
0xd4, 0xbc, 0x92, 0x85, 0x54, 0xca, 0xea, 0x41, 0x2d, 0xfa, 0xc2, 0x87, 0x56, 0x12, 0xd6, 0x27, 0xf0, 0x2d, 0xcf, 0x17, 0xb8, 0x91, 0x2e, 0x70, 0x8c, 0x78, 0x46, 0x91, 0xbb, 0x50, 0x64, 0x55,
0xfe, 0xdb, 0xa0, 0xf9, 0x5e, 0x06, 0x4a, 0x29, 0xc8, 0x82, 0x7a, 0xfc, 0x1f, 0x14, 0xe8, 0xca, 0x29, 0x4a, 0x8a, 0x8e, 0xf0, 0x1f, 0x66, 0xad, 0x95, 0x74, 0x02, 0xb9, 0xda, 0x4f, 0x61, 0x21,
0x58, 0x06, 0x51, 0x77, 0x7b, 0x3f, 0x13, 0xad, 0x14, 0x77, 0xcc, 0x9c, 0x60, 0xe4, 0x05, 0x1f, 0xf6, 0x07, 0x0d, 0x7a, 0x23, 0x81, 0x2d, 0xf9, 0x5f, 0xa8, 0xd6, 0x8d, 0x2c, 0xa4, 0x52, 0x56,
0xad, 0x26, 0xb3, 0x49, 0xfb, 0x6b, 0x41, 0x73, 0x2d, 0x33, 0xbd, 0x14, 0xfd, 0x35, 0xef, 0x86, 0x1f, 0xea, 0xd1, 0x2f, 0x8e, 0x68, 0x35, 0x81, 0x3f, 0xf1, 0xef, 0x87, 0xd6, 0x1b, 0x19, 0x28,
0x93, 0x5e, 0xc1, 0xd1, 0xf5, 0x64, 0x76, 0x63, 0x9e, 0xef, 0x9b, 0xeb, 0xd3, 0x2c, 0x91, 0x4a, 0xa5, 0x20, 0x0b, 0x1a, 0xf1, 0x3f, 0x3a, 0xd0, 0x8d, 0x89, 0x0b, 0x44, 0xdd, 0xed, 0xcd, 0x4c,
0x7c, 0xc5, 0xda, 0xd8, 0x84, 0x97, 0xe4, 0x78, 0xdc, 0xf9, 0xfc, 0xd2, 0x9f, 0xc8, 0x9b, 0xd7, 0xb4, 0x52, 0xdc, 0x29, 0x73, 0x82, 0xb1, 0x3f, 0x0a, 0xd0, 0x5a, 0xf2, 0x32, 0x69, 0xbf, 0x3a,
0xa7, 0x58, 0x21, 0x15, 0x70, 0xe2, 0xff, 0x51, 0xf1, 0xc3, 0x70, 0x6d, 0xa2, 0xd7, 0x9c, 0x2c, 0xb4, 0xd6, 0x33, 0xd3, 0x4b, 0xd1, 0x3f, 0xe3, 0xdd, 0x70, 0xd2, 0x57, 0x79, 0x74, 0x2b, 0x79,
0x06, 0xbf, 0x80, 0xf9, 0xd8, 0xb5, 0x77, 0x62, 0xd4, 0x24, 0x5f, 0x8d, 0x37, 0xc7, 0x01, 0x3a, 0xb9, 0x09, 0xbf, 0x13, 0xb4, 0x36, 0x66, 0x61, 0x91, 0x4a, 0x7c, 0xc5, 0xda, 0xd8, 0x84, 0x2f,
0x0f, 0xc9, 0xd8, 0xad, 0x00, 0x4a, 0xf1, 0xfe, 0x84, 0x9b, 0x83, 0xe6, 0x95, 0x2c, 0xa4, 0x72, 0xdb, 0xf1, 0xb8, 0xf3, 0xd7, 0x4b, 0xff, 0x64, 0xdf, 0xba, 0x35, 0x03, 0x87, 0x54, 0xc0, 0x89,
0x23, 0x1e, 0x83, 0xcb, 0x58, 0x67, 0x8d, 0xae, 0x26, 0xf3, 0x48, 0xbe, 0x15, 0x68, 0x7e, 0x90, 0xff, 0x33, 0xe3, 0x87, 0xe1, 0xfa, 0x54, 0xaf, 0x39, 0x5b, 0x0c, 0x7e, 0x01, 0x0b, 0xb1, 0x6b,
0x91, 0x5a, 0x0a, 0xed, 0x00, 0x6c, 0x61, 0xb2, 0x83, 0x89, 0x4b, 0x7d, 0xe4, 0x72, 0xa2, 0xc9, 0xf8, 0xc4, 0xa8, 0x49, 0xbe, 0xaa, 0x6f, 0x4d, 0x02, 0x74, 0x1e, 0x92, 0xb1, 0x5b, 0x01, 0x94,
0x03, 0x02, 0x5f, 0xcc, 0xbb, 0x13, 0xe9, 0xa4, 0x80, 0x1f, 0x01, 0xf2, 0xf3, 0x5c, 0xe8, 0xd1, 0xe2, 0xfd, 0x09, 0x37, 0x07, 0xad, 0x1b, 0x59, 0x48, 0xe5, 0x46, 0x3c, 0x06, 0x97, 0xb1, 0xce,
0xe5, 0xad, 0xb1, 0x0d, 0x0c, 0xef, 0x36, 0x26, 0x9d, 0xcd, 0x63, 0xa8, 0xef, 0xe8, 0x36, 0x4d, 0x1a, 0xdd, 0x4c, 0x5e, 0x23, 0xf9, 0x56, 0xa0, 0xf5, 0x56, 0x46, 0x6a, 0x29, 0xb4, 0x0b, 0xb0,
0xda, 0x01, 0xdf, 0xab, 0x89, 0x8a, 0xc5, 0xc9, 0x52, 0xac, 0x95, 0x4a, 0x2d, 0x37, 0xf3, 0x44, 0x83, 0xc9, 0x1e, 0x26, 0x2e, 0xf5, 0x91, 0xeb, 0x89, 0x26, 0x0f, 0x08, 0x7c, 0x31, 0xaf, 0x4f,
0xe6, 0x50, 0x5d, 0x86, 0x20, 0x8e, 0x63, 0x4b, 0x60, 0x8d, 0x18, 0x61, 0x0a, 0xb6, 0x8c, 0xa1, 0xa5, 0x93, 0x02, 0x7e, 0x04, 0xc8, 0xcf, 0x73, 0xa1, 0x8f, 0x40, 0xaf, 0x4c, 0x6c, 0x60, 0x78,
0x97, 0x82, 0x9f, 0x29, 0xec, 0x9f, 0x50, 0x31, 0x82, 0x87, 0x26, 0x39, 0xa4, 0x7d, 0xad, 0x97, 0xb7, 0x31, 0xed, 0x6c, 0x1e, 0x40, 0x63, 0x4f, 0xb7, 0x69, 0xd2, 0x0e, 0xd6, 0xbd, 0x99, 0xa8,
0x45, 0x05, 0x46, 0x38, 0x85, 0x0a, 0x82, 0x5e, 0xaa, 0x60, 0xc0, 0x5c, 0xa4, 0x3d, 0x40, 0x49, 0x58, 0x9c, 0x2c, 0xc5, 0x5a, 0xa9, 0xd4, 0x72, 0x33, 0x0f, 0x65, 0x0e, 0xd5, 0x65, 0x08, 0xe2,
0x2f, 0x27, 0x49, 0x0d, 0x4a, 0x73, 0x65, 0x32, 0xa1, 0x94, 0x72, 0x08, 0x73, 0xbe, 0xbf, 0x72, 0x38, 0xb6, 0x04, 0xd6, 0x88, 0x11, 0xa6, 0x60, 0xcb, 0x04, 0x7a, 0x29, 0xf8, 0xb1, 0xc2, 0xfe,
0xe3, 0xbe, 0x97, 0xa6, 0x69, 0x40, 0x93, 0x12, 0x6e, 0xc9, 0xa4, 0xe1, 0x70, 0x1b, 0x2d, 0xfc, 0xcc, 0x8a, 0x11, 0xdc, 0x33, 0xc9, 0x31, 0xed, 0x6b, 0xbd, 0x2c, 0x2a, 0x30, 0xc2, 0x19, 0x54,
0x50, 0xb6, 0x8e, 0x61, 0x5c, 0xb8, 0xa5, 0x57, 0x93, 0xea, 0x99, 0xf5, 0x7f, 0x17, 0xa0, 0xe4, 0x10, 0xf4, 0x52, 0x05, 0x03, 0x6a, 0x91, 0xf6, 0x00, 0x25, 0x7d, 0xc9, 0x49, 0x6a, 0x50, 0x5a,
0xdf, 0x84, 0xbf, 0x82, 0x8a, 0xec, 0x15, 0x94, 0x48, 0x5f, 0xc0, 0x7c, 0xec, 0x6f, 0x2a, 0x89, 0xab, 0xd3, 0x09, 0xa5, 0x94, 0x63, 0xa8, 0xf9, 0xfe, 0xca, 0x8d, 0xfb, 0x46, 0x9a, 0xa6, 0x01,
0x08, 0x9a, 0xfc, 0x57, 0x96, 0x49, 0x10, 0xf0, 0x50, 0xfc, 0x79, 0x5d, 0xa2, 0xe5, 0xbb, 0x69, 0x4d, 0x4a, 0xb8, 0x25, 0x93, 0x86, 0xc3, 0x6d, 0xbc, 0xf0, 0x43, 0xd9, 0x3a, 0x86, 0x49, 0xe1,
0x65, 0x56, 0x1c, 0x28, 0x27, 0x30, 0x7e, 0xe1, 0xb0, 0x78, 0x1f, 0x20, 0x04, 0x5b, 0xe3, 0xef, 0x96, 0x5e, 0x4d, 0xaa, 0x17, 0x36, 0xfe, 0x5d, 0x80, 0x92, 0x7f, 0x13, 0xfe, 0x1c, 0x2a, 0xb2,
0x73, 0x68, 0x24, 0x4e, 0x50, 0x78, 0xe3, 0xc6, 0x8f, 0xaf, 0xf7, 0x4c, 0x72, 0x38, 0xdc, 0xa7, 0xe7, 0x50, 0x22, 0x7d, 0x01, 0x0b, 0xb1, 0xdf, 0x66, 0x12, 0x11, 0x34, 0xf9, 0xd7, 0x9a, 0x69,
0x5f, 0xd6, 0x38, 0xe9, 0x07, 0xa6, 0x23, 0x7e, 0xad, 0xf9, 0x27, 0xba, 0xc6, 0x56, 0xaf, 0x51, 0x10, 0x70, 0x4f, 0xfc, 0x4c, 0x2f, 0xd1, 0xf2, 0xf5, 0xb4, 0x32, 0x2b, 0x0e, 0x94, 0x53, 0x16,
0x01, 0x83, 0xfd, 0xfd, 0x19, 0x36, 0xba, 0xf1, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x4d, 0x7e, 0xea, 0xb0, 0x78, 0x07, 0x20, 0x04, 0x5b, 0x93, 0xef, 0x73, 0x68, 0x24, 0x4e, 0x51, 0x78,
0xf6, 0xb0, 0xde, 0x30, 0x00, 0x00, 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. // Reference imports to suppress errors if they are not otherwise used.

View File

@ -222,21 +222,6 @@ message LoadIndex {
repeated common.KeyValuePair index_params = 6; 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 { message IndexStats {
repeated common.KeyValuePair index_params = 1; repeated common.KeyValuePair index_params = 1;
int64 num_related_segments = 2; int64 num_related_segments = 2;

View File

@ -1892,140 +1892,6 @@ func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair {
return nil 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 { type IndexStats struct {
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` 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"` 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 (m *IndexStats) String() string { return proto.CompactTextString(m) }
func (*IndexStats) ProtoMessage() {} func (*IndexStats) ProtoMessage() {}
func (*IndexStats) Descriptor() ([]byte, []int) { func (*IndexStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{27} return fileDescriptor_41f4a519b878ee3b, []int{25}
} }
func (m *IndexStats) XXX_Unmarshal(b []byte) error { 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 (m *FieldStats) String() string { return proto.CompactTextString(m) }
func (*FieldStats) ProtoMessage() {} func (*FieldStats) ProtoMessage() {}
func (*FieldStats) Descriptor() ([]byte, []int) { func (*FieldStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{28} return fileDescriptor_41f4a519b878ee3b, []int{26}
} }
func (m *FieldStats) XXX_Unmarshal(b []byte) error { 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 (m *SegmentStats) String() string { return proto.CompactTextString(m) }
func (*SegmentStats) ProtoMessage() {} func (*SegmentStats) ProtoMessage() {}
func (*SegmentStats) Descriptor() ([]byte, []int) { func (*SegmentStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{29} return fileDescriptor_41f4a519b878ee3b, []int{27}
} }
func (m *SegmentStats) XXX_Unmarshal(b []byte) error { 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 (m *QueryNodeStats) String() string { return proto.CompactTextString(m) }
func (*QueryNodeStats) ProtoMessage() {} func (*QueryNodeStats) ProtoMessage() {}
func (*QueryNodeStats) Descriptor() ([]byte, []int) { func (*QueryNodeStats) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{30} return fileDescriptor_41f4a519b878ee3b, []int{28}
} }
func (m *QueryNodeStats) XXX_Unmarshal(b []byte) error { 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 (m *MsgPosition) String() string { return proto.CompactTextString(m) }
func (*MsgPosition) ProtoMessage() {} func (*MsgPosition) ProtoMessage() {}
func (*MsgPosition) Descriptor() ([]byte, []int) { func (*MsgPosition) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{31} return fileDescriptor_41f4a519b878ee3b, []int{29}
} }
func (m *MsgPosition) XXX_Unmarshal(b []byte) error { 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 (m *ChannelTimeTickMsg) String() string { return proto.CompactTextString(m) }
func (*ChannelTimeTickMsg) ProtoMessage() {} func (*ChannelTimeTickMsg) ProtoMessage() {}
func (*ChannelTimeTickMsg) Descriptor() ([]byte, []int) { func (*ChannelTimeTickMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_41f4a519b878ee3b, []int{32} return fileDescriptor_41f4a519b878ee3b, []int{30}
} }
func (m *ChannelTimeTickMsg) XXX_Unmarshal(b []byte) error { func (m *ChannelTimeTickMsg) XXX_Unmarshal(b []byte) error {
@ -2399,8 +2265,6 @@ func init() {
proto.RegisterType((*RetrieveResults)(nil), "milvus.proto.internal.RetrieveResults") proto.RegisterType((*RetrieveResults)(nil), "milvus.proto.internal.RetrieveResults")
proto.RegisterType((*DeleteRequest)(nil), "milvus.proto.internal.DeleteRequest") proto.RegisterType((*DeleteRequest)(nil), "milvus.proto.internal.DeleteRequest")
proto.RegisterType((*LoadIndex)(nil), "milvus.proto.internal.LoadIndex") 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((*IndexStats)(nil), "milvus.proto.internal.IndexStats")
proto.RegisterType((*FieldStats)(nil), "milvus.proto.internal.FieldStats") proto.RegisterType((*FieldStats)(nil), "milvus.proto.internal.FieldStats")
proto.RegisterType((*SegmentStats)(nil), "milvus.proto.internal.SegmentStats") proto.RegisterType((*SegmentStats)(nil), "milvus.proto.internal.SegmentStats")
@ -2412,132 +2276,123 @@ func init() {
func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) } func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) }
var fileDescriptor_41f4a519b878ee3b = []byte{ var fileDescriptor_41f4a519b878ee3b = []byte{
// 2017 bytes of a gzipped FileDescriptorProto // 1886 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x73, 0x1c, 0x47, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0x4f, 0x73, 0x1c, 0x47,
0xf5, 0xff, 0xce, 0xce, 0xae, 0x76, 0xf7, 0xcd, 0x4a, 0x5e, 0xb5, 0x65, 0x67, 0x64, 0x3b, 0xb1, 0x15, 0x67, 0x76, 0x56, 0xda, 0xdd, 0xb7, 0x2b, 0x79, 0xdd, 0x92, 0x9d, 0xf1, 0x9f, 0xc4, 0xca,
0x32, 0xc9, 0x17, 0x44, 0x5c, 0x58, 0x46, 0x01, 0x92, 0xa2, 0x28, 0x1c, 0x5b, 0x1b, 0xcc, 0x96, 0x24, 0x80, 0x88, 0x0b, 0xdb, 0x28, 0x40, 0x52, 0x14, 0x85, 0x63, 0x69, 0xc1, 0x6c, 0x39, 0x36,
0x23, 0x21, 0x46, 0x4e, 0xaa, 0xe0, 0x32, 0xd5, 0xbb, 0xd3, 0x5a, 0x0d, 0x9e, 0x5f, 0x99, 0xee, 0x62, 0x64, 0x52, 0x05, 0x97, 0xa9, 0xde, 0x9d, 0xd6, 0x6a, 0xf0, 0xcc, 0xf4, 0xa4, 0xbb, 0x47,
0x91, 0xbd, 0x39, 0x71, 0xe0, 0x04, 0x05, 0x55, 0x50, 0xc5, 0x11, 0x6e, 0x5c, 0xe1, 0xc8, 0x89, 0xf2, 0xfa, 0xc4, 0x81, 0x13, 0x14, 0x54, 0x71, 0xe0, 0x08, 0x37, 0xae, 0x70, 0xe4, 0x04, 0x54,
0x1f, 0xc5, 0x89, 0x7f, 0x81, 0x3f, 0x80, 0x7f, 0x82, 0x13, 0xd5, 0xaf, 0x7b, 0x7e, 0xec, 0x6a, 0x71, 0xe2, 0x2b, 0xf0, 0x01, 0xf8, 0x12, 0x9c, 0xa8, 0x7e, 0xdd, 0x33, 0xfb, 0x47, 0x2b, 0x59,
0x25, 0x4b, 0x4a, 0x85, 0x98, 0xaa, 0xdc, 0xa6, 0xdf, 0x7b, 0xfd, 0xe3, 0x7d, 0xde, 0xe7, 0xbd, 0x52, 0x2a, 0xc4, 0xa9, 0xca, 0xad, 0xfb, 0xf7, 0x5e, 0x77, 0xbf, 0xf7, 0x7b, 0xef, 0x75, 0xbf,
0x7e, 0xdb, 0x0b, 0x2b, 0x41, 0x2c, 0x58, 0x16, 0xd3, 0xf0, 0x6e, 0x9a, 0x25, 0x22, 0x21, 0xd7, 0x9d, 0x85, 0xd5, 0x38, 0x53, 0x4c, 0x64, 0x34, 0xb9, 0x93, 0x0b, 0xae, 0x38, 0xb9, 0x92, 0xc6,
0xa2, 0x20, 0x3c, 0xce, 0xb9, 0x1a, 0xdd, 0x2d, 0x94, 0x37, 0x7a, 0xe3, 0x24, 0x8a, 0x92, 0x58, 0xc9, 0x61, 0x21, 0xcd, 0xec, 0x4e, 0x29, 0xbc, 0xde, 0x19, 0xf2, 0x34, 0xe5, 0x99, 0x81, 0xaf,
0x89, 0x6f, 0xf4, 0xf8, 0xf8, 0x88, 0x45, 0x54, 0x8d, 0x9c, 0xbf, 0x18, 0xb0, 0xbc, 0x93, 0x44, 0x77, 0xe4, 0xf0, 0x80, 0xa5, 0xd4, 0xcc, 0xfc, 0xbf, 0x3b, 0xb0, 0xb2, 0xc3, 0xd3, 0x9c, 0x67,
0x69, 0x12, 0xb3, 0x58, 0x0c, 0xe3, 0xc3, 0x84, 0x5c, 0x87, 0xa5, 0x38, 0xf1, 0xd9, 0x70, 0x60, 0x2c, 0x53, 0xfd, 0x6c, 0x9f, 0x93, 0xab, 0xb0, 0x9c, 0xf1, 0x88, 0xf5, 0x7b, 0x9e, 0xb3, 0xe1,
0x1b, 0x1b, 0xc6, 0xa6, 0xe9, 0xea, 0x11, 0x21, 0xd0, 0xcc, 0x92, 0x90, 0xd9, 0x8d, 0x0d, 0x63, 0x6c, 0xba, 0x81, 0x9d, 0x11, 0x02, 0x75, 0xc1, 0x13, 0xe6, 0xd5, 0x36, 0x9c, 0xcd, 0x56, 0x80,
0xb3, 0xeb, 0xe2, 0x37, 0xb9, 0x0f, 0xc0, 0x05, 0x15, 0xcc, 0x1b, 0x27, 0x3e, 0xb3, 0xcd, 0x0d, 0x63, 0x72, 0x1f, 0x40, 0x2a, 0xaa, 0x58, 0x38, 0xe4, 0x11, 0xf3, 0xdc, 0x0d, 0x67, 0x73, 0x75,
0x63, 0x73, 0x65, 0x7b, 0xe3, 0xee, 0xc2, 0x53, 0xdc, 0x3d, 0x90, 0x86, 0x3b, 0x89, 0xcf, 0xdc, 0x6b, 0xe3, 0xce, 0x42, 0x2b, 0xee, 0xec, 0x69, 0xc5, 0x1d, 0x1e, 0xb1, 0xa0, 0x25, 0xcb, 0x21,
0x2e, 0x2f, 0x3e, 0xc9, 0x7b, 0x00, 0xec, 0xb9, 0xc8, 0xa8, 0x17, 0xc4, 0x87, 0x89, 0xdd, 0xdc, 0xf9, 0x00, 0x80, 0x3d, 0x57, 0x82, 0x86, 0x71, 0xb6, 0xcf, 0xbd, 0xfa, 0x86, 0xbb, 0xd9, 0xde,
0x30, 0x37, 0xad, 0xed, 0xd7, 0x67, 0x17, 0xd0, 0x87, 0x7f, 0xcc, 0xa6, 0x1f, 0xd1, 0x30, 0x67, 0x7a, 0x73, 0x76, 0x03, 0x6b, 0xfc, 0x23, 0x36, 0xfe, 0x88, 0x26, 0x05, 0xdb, 0xa5, 0xb1, 0x08,
0xfb, 0x34, 0xc8, 0xdc, 0x2e, 0x4e, 0x92, 0xc7, 0x75, 0xfe, 0x69, 0xc0, 0x95, 0xd2, 0x01, 0xdc, 0x5a, 0xb8, 0x48, 0x9b, 0xeb, 0xff, 0xdb, 0x81, 0x4b, 0x95, 0x03, 0x78, 0x86, 0x24, 0xdf, 0x81,
0x83, 0x93, 0x6f, 0x41, 0x0b, 0xb7, 0x40, 0x0f, 0xac, 0xed, 0x37, 0x4f, 0x39, 0xd1, 0x8c, 0xdf, 0x25, 0x3c, 0x02, 0x3d, 0x68, 0x6f, 0xbd, 0x7d, 0x82, 0x45, 0x33, 0x7e, 0x07, 0x66, 0x09, 0xf9,
0xae, 0x9a, 0x42, 0x3e, 0x84, 0xab, 0x3c, 0x1f, 0x8d, 0x0b, 0x95, 0x87, 0x52, 0x6e, 0x37, 0xf0, 0x09, 0xac, 0xc9, 0x62, 0x30, 0x2c, 0x45, 0x21, 0xa2, 0xd2, 0xab, 0xa1, 0x69, 0x67, 0xdb, 0x89,
0x68, 0xe7, 0x5b, 0x89, 0xd4, 0x17, 0xd0, 0x47, 0x7a, 0x1b, 0x96, 0xe4, 0x4a, 0x39, 0x47, 0x94, 0x4c, 0x6f, 0x60, 0x4d, 0x7a, 0x17, 0x96, 0xf5, 0x4e, 0x85, 0x44, 0x96, 0xda, 0x5b, 0x37, 0x16,
0xac, 0xed, 0x9b, 0x0b, 0x9d, 0x3c, 0x40, 0x13, 0x57, 0x9b, 0x3a, 0x37, 0x61, 0xfd, 0x11, 0x13, 0x3a, 0xb9, 0x87, 0x2a, 0x81, 0x55, 0xf5, 0x6f, 0xc0, 0xb5, 0x87, 0x4c, 0xcd, 0x79, 0x17, 0xb0,
0x73, 0xde, 0xb9, 0xec, 0xe3, 0x9c, 0x71, 0xa1, 0x95, 0x4f, 0x82, 0x88, 0x3d, 0x09, 0xc6, 0x4f, 0x8f, 0x0b, 0x26, 0x95, 0x15, 0x3e, 0x8d, 0x53, 0xf6, 0x34, 0x1e, 0x3e, 0xdb, 0x39, 0xa0, 0x59,
0x77, 0x8e, 0x68, 0x1c, 0xb3, 0xb0, 0x50, 0xbe, 0x0a, 0x37, 0x1f, 0x31, 0x9c, 0x10, 0x70, 0x11, 0xc6, 0x92, 0x52, 0xf8, 0x3a, 0xdc, 0x78, 0xc8, 0x70, 0x41, 0x2c, 0x55, 0x3c, 0x94, 0x73, 0xe2,
0x8c, 0xf9, 0x9c, 0xfa, 0x1a, 0x5c, 0x7d, 0xc4, 0xc4, 0xc0, 0x9f, 0x13, 0x7f, 0x04, 0x9d, 0x3d, 0x2b, 0xb0, 0xf6, 0x90, 0xa9, 0x5e, 0x34, 0x07, 0x7f, 0x04, 0xcd, 0x27, 0x3a, 0xd8, 0x3a, 0x0d,
0x19, 0x6c, 0x49, 0x83, 0x6f, 0x42, 0x9b, 0xfa, 0x7e, 0xc6, 0x38, 0xd7, 0x28, 0xde, 0x5a, 0x78, 0xbe, 0x0d, 0x0d, 0x1a, 0x45, 0x82, 0x49, 0x69, 0x59, 0xbc, 0xb9, 0xd0, 0xe2, 0x07, 0x46, 0x27,
0xe2, 0x07, 0xca, 0xc6, 0x2d, 0x8c, 0x17, 0xd1, 0xc4, 0xf9, 0x31, 0xc0, 0x30, 0x0e, 0xc4, 0x3e, 0x28, 0x95, 0x17, 0xa5, 0x89, 0xff, 0x73, 0x80, 0x7e, 0x16, 0xab, 0x5d, 0x2a, 0x68, 0x2a, 0x4f,
0xcd, 0x68, 0xc4, 0x4f, 0x25, 0xd8, 0x00, 0x7a, 0x5c, 0xd0, 0x4c, 0x78, 0x29, 0xda, 0x69, 0xc8, 0x4c, 0xb0, 0x1e, 0x74, 0xa4, 0xa2, 0x42, 0x85, 0x39, 0xea, 0x59, 0xca, 0xcf, 0x90, 0x0d, 0x6d,
0xcf, 0xc1, 0x06, 0x0b, 0xa7, 0xa9, 0xd5, 0x9d, 0x1f, 0x02, 0x1c, 0x88, 0x2c, 0x88, 0x27, 0x1f, 0x5c, 0x66, 0x76, 0xf7, 0x7f, 0x0a, 0xb0, 0xa7, 0x44, 0x9c, 0x8d, 0x3e, 0x8c, 0xa5, 0xd2, 0x67,
0x04, 0x5c, 0xc8, 0xbd, 0x8e, 0xa5, 0x9d, 0x74, 0xc2, 0xdc, 0xec, 0xba, 0x7a, 0x54, 0x0b, 0x47, 0x1d, 0x6a, 0x3d, 0xed, 0x84, 0xbb, 0xd9, 0x0a, 0xec, 0x6c, 0x2a, 0x1c, 0xb5, 0xb3, 0x87, 0xe3,
0xe3, 0xfc, 0xe1, 0xb8, 0x0f, 0x56, 0x01, 0xf7, 0x2e, 0x9f, 0x90, 0x7b, 0xd0, 0x1c, 0x51, 0xce, 0x3e, 0xb4, 0x4b, 0xba, 0x1f, 0xcb, 0x11, 0xb9, 0x07, 0xf5, 0x01, 0x95, 0xec, 0x54, 0x7a, 0x1e,
0xce, 0x84, 0x67, 0x97, 0x4f, 0x1e, 0x52, 0xce, 0x5c, 0xb4, 0x74, 0x7e, 0x66, 0xc2, 0x2b, 0x3b, 0xcb, 0xd1, 0x36, 0x95, 0x2c, 0x40, 0x4d, 0xff, 0x57, 0x2e, 0xbc, 0xb6, 0x23, 0x18, 0x26, 0x7f,
0x19, 0x43, 0xf2, 0x87, 0x21, 0x1b, 0x8b, 0x20, 0x89, 0x35, 0xf6, 0x17, 0x5f, 0x8d, 0xbc, 0x02, 0x92, 0xb0, 0xa1, 0x8a, 0x79, 0x66, 0xb9, 0x3f, 0xff, 0x6e, 0xe4, 0x35, 0x68, 0x44, 0x83, 0x30,
0x6d, 0x7f, 0xe4, 0xc5, 0x34, 0x2a, 0xc0, 0x5e, 0xf2, 0x47, 0x7b, 0x34, 0x62, 0xe4, 0x4b, 0xb0, 0xa3, 0x69, 0x49, 0xf6, 0x72, 0x34, 0x78, 0x42, 0x53, 0x46, 0xbe, 0x02, 0xab, 0xc3, 0x6a, 0x7f,
0x32, 0x2e, 0xd7, 0x97, 0x12, 0xe4, 0x5c, 0xd7, 0x9d, 0x93, 0x92, 0x37, 0x61, 0x39, 0xa5, 0x99, 0x8d, 0x60, 0xce, 0xb5, 0x82, 0x39, 0x94, 0xbc, 0x0d, 0x2b, 0x39, 0x15, 0x2a, 0xae, 0xd4, 0xea,
0x08, 0x4a, 0xb3, 0x26, 0x9a, 0xcd, 0x0a, 0x65, 0x40, 0xfd, 0xd1, 0x70, 0x60, 0xb7, 0x30, 0x58, 0xa8, 0x36, 0x0b, 0xea, 0x80, 0x46, 0x83, 0x7e, 0xcf, 0x5b, 0xc2, 0x60, 0xe1, 0x98, 0xf8, 0xd0,
0xf8, 0x4d, 0x1c, 0xe8, 0x55, 0x6b, 0x0d, 0x07, 0xf6, 0x12, 0xea, 0x66, 0x64, 0x64, 0x03, 0xac, 0x99, 0xec, 0xd5, 0xef, 0x79, 0xcb, 0x28, 0x9b, 0xc1, 0xc8, 0x06, 0xb4, 0xab, 0x8d, 0xfa, 0x3d,
0x72, 0xa1, 0xe1, 0xc0, 0x6e, 0xa3, 0x49, 0x5d, 0x24, 0x83, 0xa3, 0x6a, 0x91, 0xdd, 0xd9, 0x30, 0xaf, 0x81, 0x2a, 0xd3, 0x90, 0x0e, 0x8e, 0xb9, 0x8b, 0xbc, 0xe6, 0x86, 0xb3, 0xd9, 0x09, 0xec,
0x36, 0x7b, 0xae, 0x1e, 0x91, 0x7b, 0x70, 0xf5, 0x38, 0xc8, 0x44, 0x4e, 0x43, 0xcd, 0x4f, 0x79, 0x8c, 0xdc, 0x83, 0xb5, 0xc3, 0x58, 0xa8, 0x82, 0x26, 0x36, 0x3f, 0xb5, 0x1d, 0xd2, 0x6b, 0x61,
0x0e, 0x6e, 0x77, 0x31, 0x82, 0x8b, 0x54, 0x64, 0x1b, 0xd6, 0xd2, 0xa3, 0x29, 0x0f, 0xc6, 0x73, 0x04, 0x17, 0x89, 0xc8, 0x16, 0xac, 0xe7, 0x07, 0x63, 0x19, 0x0f, 0xe7, 0x96, 0x00, 0x2e, 0x59,
0x53, 0x00, 0xa7, 0x2c, 0xd4, 0x39, 0x7f, 0x37, 0xe0, 0xda, 0x20, 0x4b, 0xd2, 0x97, 0x22, 0x14, 0x28, 0xf3, 0xff, 0xe9, 0xc0, 0x95, 0x9e, 0xe0, 0xf9, 0x2b, 0x11, 0x8a, 0x92, 0xe4, 0xfa, 0x29,
0x05, 0xc8, 0xcd, 0x33, 0x40, 0x6e, 0x9d, 0x04, 0xd9, 0xf9, 0x45, 0x03, 0xae, 0x2b, 0x46, 0xed, 0x24, 0x2f, 0x1d, 0x27, 0xd9, 0xff, 0x4d, 0x0d, 0xae, 0x9a, 0x8c, 0xda, 0x2d, 0x89, 0xfd, 0x14,
0x17, 0xc0, 0x7e, 0x06, 0x5e, 0x7c, 0x19, 0xae, 0x54, 0xbb, 0x2a, 0x83, 0xc5, 0x6e, 0xfc, 0x3f, 0xbc, 0xf8, 0x2a, 0x5c, 0x9a, 0x9c, 0x6a, 0x14, 0x16, 0xbb, 0xf1, 0x65, 0x58, 0xad, 0x02, 0x6c,
0xac, 0x94, 0x01, 0x56, 0x76, 0xff, 0x5d, 0x4a, 0x39, 0x3f, 0x6f, 0xc0, 0x9a, 0x0c, 0xea, 0x17, 0xf4, 0xfe, 0xbf, 0x29, 0xe5, 0xff, 0xba, 0x06, 0xeb, 0x3a, 0xa8, 0x5f, 0xb0, 0xa1, 0xd9, 0xf8,
0x68, 0x48, 0x34, 0x7e, 0x67, 0x00, 0x51, 0xec, 0x78, 0x10, 0x06, 0x94, 0x7f, 0x9e, 0x58, 0xac, 0xa3, 0x03, 0xc4, 0x64, 0xc7, 0x83, 0x24, 0xa6, 0xf2, 0xb3, 0xe4, 0x62, 0x1d, 0x96, 0xa8, 0xb6,
0x41, 0x8b, 0xca, 0x33, 0x68, 0x08, 0xd4, 0xc0, 0xe1, 0xd0, 0x97, 0xd1, 0xfa, 0xac, 0x4e, 0x57, 0xc1, 0x52, 0x60, 0x26, 0xbe, 0x84, 0xae, 0x8e, 0xd6, 0xa7, 0x65, 0x5d, 0x75, 0xa8, 0x3b, 0x7d,
0x6e, 0x6a, 0xd6, 0x37, 0xfd, 0xad, 0x01, 0xab, 0x0f, 0x42, 0xc1, 0xb2, 0x97, 0x14, 0x94, 0xbf, 0xe8, 0x1f, 0x1c, 0xb8, 0xfc, 0x20, 0x51, 0x4c, 0xbc, 0xa2, 0xa4, 0xfc, 0xa3, 0x56, 0x46, 0xad,
0x36, 0x8a, 0xa8, 0x0d, 0x63, 0x9f, 0x3d, 0xff, 0x3c, 0x0f, 0xf8, 0x2a, 0xc0, 0x61, 0xc0, 0x42, 0x9f, 0x45, 0xec, 0xf9, 0x67, 0x69, 0xe0, 0xeb, 0x00, 0xfb, 0x31, 0x4b, 0xa2, 0xe9, 0xec, 0x6d,
0xbf, 0xce, 0xde, 0x2e, 0x4a, 0x3e, 0x15, 0x73, 0x6d, 0x68, 0xe3, 0x22, 0x25, 0x6b, 0x8b, 0xa1, 0x21, 0xf2, 0x89, 0x32, 0xd7, 0x83, 0x06, 0x6e, 0x52, 0x65, 0x6d, 0x39, 0xd5, 0x3d, 0x80, 0xe9,
0xec, 0x01, 0x54, 0x3f, 0xa8, 0x7b, 0x80, 0xce, 0xb9, 0x7b, 0x00, 0x9c, 0xa6, 0x7b, 0x80, 0x3f, 0x07, 0x6d, 0x0f, 0xd0, 0x3c, 0x73, 0x0f, 0x80, 0xcb, 0x6c, 0x0f, 0xf0, 0x67, 0x17, 0x56, 0xfa,
0x98, 0xb0, 0x3c, 0x8c, 0x39, 0xcb, 0xc4, 0xe5, 0xc1, 0xbb, 0x05, 0x5d, 0x7e, 0x44, 0x33, 0x74, 0x99, 0x64, 0x42, 0x5d, 0x9c, 0xbc, 0x9b, 0xd0, 0x92, 0x07, 0x54, 0xa0, 0xa3, 0x96, 0xbe, 0x09,
0x54, 0xc3, 0x57, 0x09, 0xea, 0xd0, 0x9a, 0x2f, 0x82, 0xb6, 0x79, 0xce, 0xe2, 0xd0, 0x3a, 0xab, 0x30, 0x4d, 0xad, 0xfb, 0x32, 0x6a, 0xeb, 0x67, 0xbc, 0x1c, 0x96, 0x4e, 0xbb, 0x1c, 0x96, 0x4f,
0x38, 0x2c, 0x9d, 0x01, 0x71, 0xfb, 0xc5, 0xc5, 0xa1, 0x73, 0xf2, 0xf6, 0x95, 0x0e, 0xb2, 0x49, 0xa1, 0xb8, 0xf1, 0xf2, 0xcb, 0xa1, 0x79, 0xfc, 0xf5, 0xd5, 0x0e, 0xb2, 0x51, 0xaa, 0x9b, 0xd6,
0x24, 0x9b, 0xd6, 0x81, 0xdd, 0x45, 0x7d, 0x25, 0x20, 0xaf, 0x01, 0x88, 0x20, 0x62, 0x5c, 0xd0, 0x9e, 0xd7, 0x42, 0xf9, 0x04, 0x20, 0x6f, 0x00, 0xa8, 0x38, 0x65, 0x52, 0xd1, 0x34, 0x37, 0xef,
0x28, 0x55, 0xf7, 0x68, 0xd3, 0xad, 0x49, 0xe4, 0xdd, 0x9d, 0x25, 0xcf, 0x86, 0x03, 0x6e, 0x5b, 0x68, 0x3d, 0x98, 0x42, 0xf4, 0xdb, 0x2d, 0xf8, 0x51, 0xbf, 0x27, 0xbd, 0xf6, 0x86, 0xab, 0x9b,
0x1b, 0xa6, 0x6c, 0xe2, 0xd4, 0x88, 0x7c, 0x1d, 0x3a, 0x59, 0xf2, 0xcc, 0xf3, 0xa9, 0xa0, 0x76, 0x38, 0x33, 0x23, 0xdf, 0x84, 0xa6, 0xe0, 0x47, 0x61, 0x44, 0x15, 0xf5, 0x3a, 0x18, 0xbc, 0x6b,
0x0f, 0x83, 0xb7, 0xbe, 0x10, 0xec, 0x87, 0x61, 0x32, 0x72, 0xdb, 0x59, 0xf2, 0x6c, 0x40, 0x05, 0x0b, 0xc9, 0xde, 0x4e, 0xf8, 0x20, 0x68, 0x08, 0x7e, 0xd4, 0xa3, 0x8a, 0xfa, 0x7f, 0xaa, 0xc3,
0x75, 0x7e, 0xdf, 0x84, 0xe5, 0x03, 0x46, 0xb3, 0xf1, 0xd1, 0xe5, 0x03, 0xf6, 0x15, 0xe8, 0x67, 0xca, 0x1e, 0xa3, 0x62, 0x78, 0x70, 0xf1, 0x80, 0x7d, 0x0d, 0xba, 0x82, 0xc9, 0x22, 0x51, 0xe1,
0x8c, 0xe7, 0xa1, 0xf0, 0xc6, 0xea, 0x9a, 0x1f, 0x0e, 0x74, 0xdc, 0xae, 0x28, 0xf9, 0x4e, 0x21, 0xd0, 0x3c, 0xf3, 0xfd, 0x9e, 0x8d, 0xdb, 0x25, 0x83, 0xef, 0x94, 0x70, 0x45, 0xaa, 0x7b, 0x0a,
0x2e, 0x41, 0x35, 0xcf, 0x00, 0xb5, 0xb9, 0x00, 0x54, 0x07, 0x7a, 0x35, 0x04, 0xb9, 0xdd, 0x42, 0xa9, 0xf5, 0x05, 0xa4, 0xfa, 0xd0, 0x99, 0x62, 0x50, 0x7a, 0x4b, 0xe8, 0xfa, 0x0c, 0x46, 0xba,
0xd7, 0x67, 0x64, 0xa4, 0x0f, 0xa6, 0xcf, 0x43, 0x8c, 0x57, 0xd7, 0x95, 0x9f, 0xe4, 0x0e, 0xac, 0xe0, 0x46, 0x32, 0xc1, 0x78, 0xb5, 0x02, 0x3d, 0x24, 0xb7, 0xe1, 0x72, 0x9e, 0xd0, 0x21, 0x3b,
0xa6, 0x21, 0x1d, 0xb3, 0xa3, 0x24, 0xf4, 0x59, 0xe6, 0x4d, 0xb2, 0x24, 0x4f, 0x31, 0x66, 0x3d, 0xe0, 0x49, 0xc4, 0x44, 0x38, 0x12, 0xbc, 0xc8, 0x31, 0x66, 0x9d, 0xa0, 0x3b, 0x25, 0x78, 0xa8,
0xb7, 0x5f, 0x53, 0x3c, 0x92, 0x72, 0xf2, 0x0e, 0x74, 0x7c, 0x1e, 0x7a, 0x62, 0x9a, 0x32, 0x0c, 0x71, 0xf2, 0x1e, 0x34, 0x23, 0x99, 0x84, 0x6a, 0x9c, 0x33, 0x0c, 0xda, 0xea, 0x09, 0xbe, 0xf7,
0xda, 0xca, 0x29, 0xbe, 0x0f, 0x78, 0xf8, 0x64, 0x9a, 0x32, 0xb7, 0xed, 0xab, 0x0f, 0x72, 0x0f, 0x64, 0xf2, 0x74, 0x9c, 0xb3, 0xa0, 0x11, 0x99, 0x01, 0xb9, 0x07, 0xeb, 0x92, 0x89, 0x98, 0x26,
0xd6, 0x38, 0xcb, 0x02, 0x1a, 0x06, 0x9f, 0x30, 0xdf, 0x63, 0xcf, 0xd3, 0xcc, 0x4b, 0x43, 0x1a, 0xf1, 0x0b, 0x16, 0x85, 0xec, 0x79, 0x2e, 0xc2, 0x3c, 0xa1, 0x19, 0x46, 0xb6, 0x13, 0x90, 0x89,
0x63, 0x64, 0x7b, 0x2e, 0xa9, 0x74, 0xef, 0x3f, 0x4f, 0xb3, 0xfd, 0x90, 0xc6, 0x64, 0x13, 0xfa, 0xec, 0xfb, 0xcf, 0x73, 0xb1, 0x9b, 0xd0, 0x8c, 0x6c, 0x42, 0x97, 0x17, 0x2a, 0x2f, 0x54, 0x88,
0x49, 0x2e, 0xd2, 0x5c, 0x78, 0x98, 0x7d, 0xdc, 0x0b, 0x7c, 0x0c, 0xb4, 0xe9, 0xae, 0x28, 0xf9, 0xd5, 0x27, 0xc3, 0x38, 0xc2, 0x40, 0xbb, 0xc1, 0xaa, 0xc1, 0x7f, 0x80, 0x70, 0x3f, 0xd2, 0xd4,
0x77, 0x51, 0x3c, 0xf4, 0x25, 0xb4, 0x22, 0xa3, 0xc7, 0x2c, 0xf4, 0x4a, 0x06, 0xd8, 0xd6, 0x86, 0x2a, 0x41, 0x0f, 0x59, 0x12, 0x56, 0x19, 0xe0, 0xb5, 0x37, 0x9c, 0xcd, 0x7a, 0x70, 0xc9, 0xe0,
0xb1, 0xd9, 0x74, 0xaf, 0x28, 0xf9, 0x93, 0x42, 0x4c, 0xb6, 0xe0, 0xea, 0x24, 0xa7, 0x19, 0x8d, 0x4f, 0x4b, 0x98, 0xdc, 0x85, 0xb5, 0x51, 0x41, 0x05, 0xcd, 0x14, 0x63, 0x53, 0xda, 0x1d, 0xd4,
0x05, 0x63, 0x35, 0xeb, 0x1e, 0x5a, 0x93, 0x52, 0x55, 0x4d, 0xb8, 0x03, 0xab, 0xd2, 0x2c, 0xc9, 0x26, 0x95, 0x68, 0xb2, 0xe0, 0x36, 0x5c, 0xd6, 0x6a, 0xbc, 0x50, 0x53, 0xea, 0x2b, 0xa8, 0xde,
0x45, 0xcd, 0x7c, 0x19, 0xcd, 0xfb, 0x5a, 0x51, 0x1a, 0x3b, 0xbf, 0xaa, 0xf1, 0x44, 0x86, 0x94, 0xb5, 0x82, 0x4a, 0xd9, 0xff, 0xdd, 0x54, 0x9e, 0xe8, 0x90, 0xca, 0x0b, 0xe4, 0xc9, 0x45, 0x5a,
0x5f, 0x82, 0x27, 0x97, 0x69, 0xfd, 0x17, 0x92, 0xcb, 0x5c, 0x4c, 0xae, 0xdb, 0x60, 0x45, 0x4c, 0xff, 0x85, 0xc9, 0xe5, 0x2e, 0x4e, 0xae, 0x5b, 0xd0, 0x4e, 0x99, 0x12, 0xf1, 0xd0, 0x04, 0xd1,
0x64, 0xc1, 0x58, 0x05, 0x51, 0x65, 0x3f, 0x28, 0x11, 0x46, 0xea, 0x36, 0x58, 0x71, 0x1e, 0x79, 0x54, 0x3f, 0x18, 0x08, 0x23, 0x75, 0x0b, 0xda, 0x59, 0x91, 0x86, 0x1f, 0x17, 0x4c, 0xc4, 0x4c,
0x1f, 0xe7, 0x2c, 0x0b, 0x18, 0xd7, 0xc5, 0x13, 0xe2, 0x3c, 0xfa, 0x81, 0x92, 0x90, 0xab, 0xd0, 0xda, 0xcb, 0x13, 0xb2, 0x22, 0xfd, 0xb1, 0x41, 0xc8, 0x1a, 0x2c, 0x29, 0x9e, 0x87, 0xcf, 0xca,
0x12, 0x49, 0xea, 0x3d, 0x2d, 0x92, 0x5e, 0x24, 0xe9, 0x63, 0xf2, 0x6d, 0xb8, 0xc1, 0x19, 0x0d, 0xa2, 0x57, 0x3c, 0x7f, 0x44, 0xbe, 0x0b, 0xd7, 0x25, 0xa3, 0x09, 0x8b, 0xc2, 0xaa, 0x48, 0x65,
0x99, 0xef, 0x95, 0x49, 0xca, 0x3d, 0x8e, 0x58, 0x30, 0xdf, 0x6e, 0x63, 0xdc, 0x6c, 0x65, 0x71, 0x28, 0x91, 0x0b, 0x16, 0x79, 0x0d, 0x8c, 0x9b, 0x67, 0x34, 0xf6, 0x2a, 0x85, 0x3d, 0x2b, 0xd7,
0x50, 0x1a, 0x1c, 0x68, 0xbd, 0x0c, 0x4b, 0x79, 0xf0, 0xda, 0xb4, 0x0e, 0xf6, 0xc7, 0xa4, 0x52, 0x61, 0xa9, 0x0c, 0x9f, 0x5a, 0xd6, 0xc4, 0xfe, 0x98, 0x4c, 0x44, 0xd5, 0x82, 0xf7, 0xc1, 0x1b,
0x95, 0x13, 0xde, 0x05, 0x7b, 0x12, 0x26, 0x23, 0x1a, 0x7a, 0x27, 0x76, 0xc5, 0x46, 0xdc, 0x74, 0x25, 0x7c, 0x40, 0x93, 0xf0, 0xd8, 0xa9, 0xd8, 0x88, 0xbb, 0xc1, 0x55, 0x23, 0xdf, 0x9b, 0x3b,
0xaf, 0x2b, 0xfd, 0xc1, 0xdc, 0x96, 0xd2, 0x3d, 0x1e, 0x06, 0x63, 0xe6, 0x7b, 0xa3, 0x30, 0x19, 0x52, 0xbb, 0x27, 0x93, 0x78, 0xc8, 0xa2, 0x70, 0x90, 0xf0, 0x81, 0x07, 0x98, 0x7f, 0x60, 0x20,
0xd9, 0x80, 0xfc, 0x03, 0x25, 0x92, 0x59, 0x2f, 0x79, 0xa7, 0x0d, 0x24, 0x0c, 0xe3, 0x24, 0x8f, 0x5d, 0xf5, 0x3a, 0xef, 0xac, 0x82, 0xa6, 0x61, 0xc8, 0x8b, 0x4c, 0x61, 0x36, 0xb9, 0xc1, 0xaa,
0x05, 0xb2, 0xc9, 0x74, 0x57, 0x94, 0x7c, 0x2f, 0x8f, 0x76, 0xa4, 0x94, 0xbc, 0x01, 0xcb, 0xda, 0xc1, 0x9f, 0x14, 0xe9, 0x8e, 0x46, 0xc9, 0x5b, 0xb0, 0x62, 0x35, 0xf9, 0xfe, 0xbe, 0x64, 0x0a,
0x32, 0x39, 0x3c, 0xe4, 0x4c, 0x20, 0x8d, 0x4c, 0xb7, 0xa7, 0x84, 0xdf, 0x47, 0x99, 0xf3, 0x47, 0xd3, 0xc8, 0x0d, 0x3a, 0x06, 0xfc, 0x11, 0x62, 0xfe, 0x5f, 0x5c, 0xb8, 0x14, 0x68, 0x76, 0xd9,
0x13, 0xae, 0xb8, 0x12, 0x5d, 0x76, 0xcc, 0xfe, 0xe7, 0xab, 0xc7, 0x69, 0x59, 0xbc, 0x74, 0xa1, 0x21, 0xfb, 0xdc, 0xdf, 0x1e, 0x27, 0x55, 0xf1, 0xf2, 0xb9, 0xaa, 0xb8, 0x71, 0xe6, 0x2a, 0x6e,
0x2c, 0x6e, 0x9f, 0x3b, 0x8b, 0x3b, 0x17, 0xca, 0xe2, 0xee, 0xc5, 0xb2, 0x18, 0x4e, 0xc9, 0xe2, 0x9e, 0xab, 0x8a, 0x5b, 0xe7, 0xab, 0x62, 0x38, 0xa1, 0x8a, 0xff, 0x36, 0x13, 0xb1, 0x57, 0xb5,
0x3f, 0xcf, 0x44, 0xec, 0x65, 0xcd, 0xe3, 0xb7, 0xc0, 0x0c, 0x7c, 0xd5, 0x9a, 0x59, 0xdb, 0xf6, 0x8e, 0xdf, 0x01, 0x37, 0x8e, 0x4c, 0x6b, 0xd6, 0xde, 0xf2, 0x66, 0x37, 0xb7, 0x9f, 0xd0, 0xfa,
0xec, 0xe2, 0xfa, 0x09, 0x6d, 0x38, 0xe0, 0xae, 0x34, 0x22, 0xf7, 0xc1, 0xd2, 0xe8, 0xe3, 0xc5, 0x3d, 0x19, 0x68, 0x25, 0x72, 0x1f, 0xda, 0x96, 0x7d, 0x7c, 0xf8, 0x96, 0xf0, 0xe1, 0x7b, 0x63,
0xd7, 0xc2, 0x8b, 0xef, 0xb5, 0x85, 0x73, 0x30, 0x1c, 0xf2, 0xd2, 0x73, 0x55, 0x6b, 0xc5, 0xe5, 0xe1, 0x1a, 0x0c, 0x87, 0x7e, 0xf4, 0x02, 0xd3, 0x5a, 0x49, 0x3d, 0x26, 0xdf, 0x83, 0x1b, 0xc7,
0x37, 0xf9, 0x0e, 0xdc, 0x3c, 0x99, 0xdd, 0x99, 0xc6, 0xc8, 0xb7, 0x97, 0x30, 0xa0, 0xeb, 0xf3, 0xab, 0x5b, 0x58, 0x8e, 0x22, 0x6f, 0x19, 0x03, 0x7a, 0x6d, 0xbe, 0xbc, 0x4b, 0x12, 0x23, 0xf2,
0xe9, 0x5d, 0x80, 0xe8, 0x93, 0xaf, 0xc1, 0x5a, 0x2d, 0xbf, 0xab, 0x89, 0x6d, 0xf5, 0x9b, 0xb9, 0x0d, 0x58, 0x9f, 0xaa, 0xef, 0xc9, 0xc2, 0x86, 0xf9, 0xcd, 0x3c, 0x91, 0x4d, 0x96, 0x9c, 0x56,
0xd2, 0x55, 0x53, 0xce, 0xca, 0xf0, 0xce, 0x59, 0x19, 0xee, 0xfc, 0xab, 0x01, 0xcb, 0x03, 0x16, 0xe1, 0xcd, 0xd3, 0x2a, 0xdc, 0xff, 0x4f, 0x0d, 0x56, 0x7a, 0x2c, 0x61, 0x8a, 0x7d, 0xd1, 0x5e,
0x32, 0xc1, 0xbe, 0x68, 0xaf, 0x4e, 0x6d, 0xaf, 0x5e, 0x87, 0x5e, 0x9a, 0x05, 0x11, 0xcd, 0xa6, 0x9d, 0xd8, 0x5e, 0xbd, 0x09, 0x9d, 0x5c, 0xc4, 0x29, 0x15, 0xe3, 0xf0, 0x19, 0x1b, 0x97, 0x97,
0xde, 0x53, 0x36, 0x2d, 0x8a, 0xa6, 0xa5, 0x65, 0x8f, 0xd9, 0x94, 0xbf, 0xa8, 0xc7, 0x72, 0xfe, 0x66, 0xdb, 0x62, 0x8f, 0xd8, 0x58, 0xbe, 0xac, 0xc7, 0xf2, 0xff, 0xeb, 0x40, 0xeb, 0x43, 0x4e,
0x6d, 0x40, 0xf7, 0x83, 0x84, 0xfa, 0xf8, 0x33, 0xe0, 0x92, 0x18, 0x97, 0x1d, 0x5e, 0x63, 0xbe, 0x23, 0xfc, 0x19, 0x70, 0x41, 0x8e, 0xab, 0x0e, 0xaf, 0x36, 0xdf, 0xe1, 0xdd, 0x84, 0x49, 0x27,
0xc3, 0xbb, 0x05, 0x55, 0x27, 0xaf, 0x51, 0xae, 0xb5, 0xf6, 0xb5, 0x16, 0xbd, 0x39, 0xdb, 0xa2, 0x6f, 0x59, 0x9e, 0x6a, 0xed, 0xa7, 0x5a, 0xf4, 0xfa, 0x6c, 0x8b, 0x7e, 0x0b, 0xda, 0xb1, 0x36,
0xdf, 0x06, 0x2b, 0x90, 0x07, 0xf2, 0x52, 0x2a, 0x8e, 0x54, 0x15, 0xeb, 0xba, 0x80, 0xa2, 0x7d, 0x28, 0xcc, 0xa9, 0x3a, 0x30, 0xb7, 0x58, 0x2b, 0x00, 0x84, 0x76, 0x35, 0xa2, 0x7b, 0xf8, 0x52,
0x29, 0x91, 0x3d, 0x7c, 0x61, 0x80, 0x3d, 0xfc, 0xd2, 0xb9, 0x7b, 0x78, 0xbd, 0x08, 0xf6, 0xf0, 0x01, 0x7b, 0xf8, 0xe5, 0x33, 0xf7, 0xf0, 0x76, 0x13, 0xec, 0xe1, 0x7f, 0xe9, 0x00, 0xa0, 0xe3,
0x7f, 0x6b, 0x80, 0xad, 0x39, 0x57, 0x3d, 0x63, 0x7e, 0x98, 0xfa, 0xf8, 0x9a, 0x7a, 0x0b, 0xba, 0xba, 0x82, 0x8f, 0x6f, 0xea, 0x5c, 0x64, 0x53, 0x7d, 0xbd, 0xea, 0x37, 0x47, 0xb0, 0x84, 0xaa,
0x25, 0x1f, 0xf5, 0x2b, 0x62, 0x25, 0x90, 0xb8, 0xee, 0xb2, 0x28, 0xc9, 0xa6, 0x07, 0xc1, 0x27, 0x49, 0xc6, 0x4b, 0x4b, 0x0e, 0xc9, 0x8a, 0x34, 0x30, 0x22, 0x9b, 0xed, 0xd2, 0xff, 0xad, 0x03,
0x4c, 0x3b, 0x5e, 0x93, 0x48, 0xdf, 0xf6, 0xf2, 0xc8, 0x4d, 0x9e, 0x71, 0x5d, 0xc3, 0x8b, 0xa1, 0x80, 0x25, 0x6b, 0xcc, 0x98, 0xcf, 0x0d, 0xe7, 0xf4, 0x5f, 0x37, 0xb5, 0x59, 0xea, 0xb6, 0x4b,
0xf4, 0x6d, 0x8c, 0xbf, 0xbc, 0xb0, 0x8a, 0xa1, 0xe7, 0x4d, 0x17, 0x94, 0x48, 0xd6, 0x2f, 0xb2, 0xea, 0xf4, 0x6d, 0xa4, 0x7f, 0x95, 0x2e, 0xf0, 0xa1, 0xfa, 0xa6, 0x3c, 0x71, 0xde, 0xb2, 0x8b,
0x0e, 0x1d, 0x16, 0xfb, 0x4a, 0xdb, 0x42, 0x6d, 0x9b, 0xc5, 0x3e, 0xaa, 0x86, 0xb0, 0xa2, 0x9f, 0x63, 0xff, 0xf7, 0x0e, 0x74, 0xac, 0x75, 0xc6, 0xa4, 0x99, 0x28, 0x3b, 0xf3, 0x51, 0xc6, 0x6e,
0x2f, 0x13, 0x8e, 0x24, 0x40, 0x52, 0x59, 0xdb, 0xce, 0x29, 0x6f, 0xc6, 0xbb, 0x7c, 0xb2, 0xaf, 0x24, 0xe5, 0x62, 0x1c, 0xca, 0xf8, 0x05, 0xb3, 0x06, 0x81, 0x81, 0xf6, 0xe2, 0x17, 0x8c, 0x5c,
0x2d, 0xdd, 0x65, 0xf5, 0x82, 0xa9, 0x87, 0xe4, 0x7d, 0xe8, 0xc9, 0x5d, 0xca, 0x85, 0xda, 0xe7, 0x83, 0x26, 0x52, 0xc2, 0x8f, 0xa4, 0x7d, 0xd1, 0x1a, 0x9a, 0x06, 0x7e, 0x24, 0xf5, 0xa5, 0x2e,
0x5e, 0xc8, 0x62, 0xb1, 0x5f, 0x0c, 0x9c, 0x5f, 0x1b, 0xb0, 0x7a, 0x02, 0xc2, 0x4b, 0xf0, 0xe8, 0xd8, 0x90, 0x65, 0x2a, 0x19, 0x87, 0x29, 0x8f, 0xe2, 0xfd, 0x98, 0x45, 0x98, 0x0d, 0xcd, 0xa0,
0x31, 0x74, 0x0e, 0xd8, 0x44, 0x2e, 0x51, 0x3c, 0xca, 0x6e, 0x9d, 0xf6, 0xc6, 0x7f, 0x4a, 0xc0, 0x5b, 0x0a, 0x1e, 0x5b, 0xdc, 0xff, 0x97, 0x03, 0xab, 0xba, 0x81, 0x19, 0x3f, 0xe1, 0x11, 0x33,
0xdc, 0x72, 0x01, 0xe7, 0xa7, 0x06, 0x00, 0x12, 0x1a, 0x87, 0x27, 0xc8, 0x62, 0x5c, 0x86, 0x2c, 0x96, 0x9d, 0x3f, 0x63, 0x3f, 0x40, 0x5f, 0x2c, 0x3d, 0xe6, 0xfb, 0xef, 0x5b, 0x27, 0xfd, 0x9d,
0xf2, 0xda, 0x94, 0xbd, 0x44, 0xc6, 0x42, 0x2a, 0xaa, 0x4a, 0xc6, 0x75, 0xec, 0x49, 0x9c, 0x47, 0x30, 0xc5, 0x41, 0xd0, 0x94, 0x6c, 0x64, 0xce, 0xdc, 0xb6, 0x37, 0xf1, 0x99, 0x28, 0x9e, 0x04,
0xae, 0x52, 0xe9, 0x03, 0x72, 0xe7, 0x97, 0x06, 0x00, 0x96, 0x62, 0x75, 0x8c, 0xf9, 0x9c, 0x37, 0xd6, 0x5e, 0xc6, 0x86, 0xe2, 0x5f, 0x38, 0xd0, 0x7e, 0x2c, 0x47, 0xbb, 0x5c, 0x62, 0x31, 0xeb,
0xce, 0xfe, 0xd5, 0xda, 0x98, 0x4d, 0x89, 0x87, 0x45, 0x4a, 0x70, 0xc4, 0xc8, 0x5c, 0xe4, 0x43, 0x52, 0xb6, 0x17, 0xa8, 0xb9, 0x49, 0x1c, 0x2c, 0x96, 0xf6, 0x70, 0xf2, 0x35, 0x51, 0xff, 0x92,
0x89, 0x51, 0xe5, 0xbc, 0xce, 0x1a, 0x85, 0xcb, 0x6f, 0x0c, 0xe8, 0xd5, 0xe0, 0xe3, 0xb3, 0xd9, 0x4f, 0xe5, 0xc8, 0x46, 0xbc, 0x13, 0x98, 0x09, 0xb9, 0x0e, 0xcd, 0x54, 0x8e, 0xb0, 0xb1, 0xb7,
0x6b, 0xcc, 0x67, 0x2f, 0x76, 0x99, 0x92, 0xd1, 0x1e, 0xaf, 0x91, 0x3c, 0xaa, 0x48, 0xbe, 0x0e, 0x15, 0x56, 0xcd, 0x75, 0xd8, 0x26, 0x2f, 0x65, 0x1d, 0x5f, 0xca, 0x09, 0xe0, 0xff, 0xd5, 0x01,
0x1d, 0x84, 0xa4, 0xc6, 0xf2, 0x58, 0xb3, 0xfc, 0x0e, 0xac, 0x66, 0x6c, 0xcc, 0x62, 0x11, 0x4e, 0x62, 0x9f, 0xa2, 0x4f, 0xf4, 0xc9, 0x19, 0x13, 0x76, 0xfa, 0x8b, 0x68, 0x0d, 0xcb, 0x75, 0x06,
0xbd, 0x28, 0xf1, 0x83, 0xc3, 0x80, 0xf9, 0xc8, 0xf5, 0x8e, 0xdb, 0x2f, 0x14, 0xbb, 0x5a, 0xee, 0x9b, 0xbb, 0x87, 0xdc, 0x63, 0xbf, 0xf5, 0x6e, 0xc3, 0xe5, 0x88, 0xed, 0x53, 0xfd, 0x6a, 0xce,
0xfc, 0xc3, 0x80, 0x15, 0xd9, 0x98, 0x4e, 0xf7, 0x12, 0x9f, 0xa9, 0x93, 0x5d, 0x9c, 0x41, 0xef, 0x9b, 0xdc, 0xb5, 0x82, 0xea, 0x71, 0x7f, 0xe7, 0x7d, 0x68, 0x55, 0xff, 0xf4, 0x90, 0x2e, 0x74,
0xa1, 0x2f, 0x1a, 0x1e, 0x45, 0xa1, 0x37, 0x5e, 0x4c, 0x21, 0xee, 0x76, 0xb8, 0xa6, 0x8d, 0x84, 0xfa, 0x59, 0xac, 0xb0, 0x67, 0x89, 0xb3, 0x51, 0xf7, 0x4b, 0xa4, 0x0d, 0x8d, 0x1f, 0x32, 0x9a,
0x58, 0xbd, 0x44, 0x9c, 0x07, 0xe2, 0x2a, 0xb0, 0xfa, 0x92, 0x55, 0x10, 0xff, 0xc4, 0x00, 0xab, 0xa8, 0x83, 0x71, 0xd7, 0x21, 0x1d, 0x68, 0x3e, 0x18, 0x64, 0x5c, 0xa4, 0x34, 0xe9, 0xd6, 0xb6,
0x96, 0x2c, 0xb2, 0x44, 0xeb, 0x8b, 0x51, 0xdd, 0x10, 0x06, 0x16, 0x41, 0x6b, 0x5c, 0xbd, 0x12, 0xdf, 0xfb, 0xd9, 0xb7, 0x46, 0xb1, 0x3a, 0x28, 0x06, 0xda, 0x93, 0xbb, 0xc6, 0xb5, 0xaf, 0xc7,
0x93, 0x35, 0x68, 0x45, 0x7c, 0xa2, 0x23, 0xde, 0x73, 0xd5, 0x80, 0xdc, 0x80, 0x4e, 0xc4, 0x27, 0xdc, 0x8e, 0xee, 0x96, 0x51, 0xbb, 0x8b, 0xde, 0x56, 0xd3, 0x7c, 0x30, 0x58, 0x46, 0xe4, 0xdd,
0xf8, 0x83, 0x4d, 0x57, 0xce, 0x72, 0x2c, 0xc3, 0x56, 0x75, 0x40, 0xaa, 0x80, 0x54, 0x02, 0xe7, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x48, 0xeb, 0xcc, 0x3e, 0x0f, 0x1b, 0x00, 0x00,
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,
} }