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