Remove datanode timetick mq, use rpc to report instead (#23156)

Signed-off-by: wayblink <anyang.wang@zilliz.com>
This commit is contained in:
wayblink 2023-06-14 14:16:38 +08:00 committed by GitHub
parent f18443447f
commit bfae6b49af
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 1082 additions and 917 deletions

View File

@ -346,6 +346,8 @@ dataNode:
forceSyncSegmentNum: 1 # number of segments to sync, segments with top largest buffer will be synced.
watermarkStandalone: 0.2 # memory watermark for standalone, upon reaching this watermark, segments will be synced.
watermarkCluster: 0.5 # memory watermark for cluster, upon reaching this watermark, segments will be synced.
timetick:
byRPC: true
# Configures the system log output.
log:

View File

@ -28,8 +28,6 @@ import (
"github.com/blang/semver/v4"
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
@ -54,10 +52,12 @@ import (
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
@ -548,8 +548,11 @@ func (s *Server) initIndexNodeManager() {
}
func (s *Server) startServerLoop() {
s.serverLoopWg.Add(3)
s.startDataNodeTtLoop(s.serverLoopCtx)
s.serverLoopWg.Add(2)
if !Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
s.serverLoopWg.Add(1)
s.startDataNodeTtLoop(s.serverLoopCtx)
}
s.startWatchService(s.serverLoopCtx)
s.startFlushLoop(s.serverLoopCtx)
s.startIndexService(s.serverLoopCtx)

View File

@ -54,7 +54,6 @@ import (
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mq/msgstream"
"github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
@ -1766,226 +1765,6 @@ func TestDropVirtualChannel(t *testing.T) {
})
}
func TestDataNodeTtChannel(t *testing.T) {
genMsg := func(msgType commonpb.MsgType, ch string, t Timestamp) *msgstream.DataNodeTtMsg {
return &msgstream.DataNodeTtMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{0},
},
DataNodeTtMsg: msgpb.DataNodeTtMsg{
Base: &commonpb.MsgBase{
MsgType: msgType,
MsgID: 0,
Timestamp: t,
SourceID: 0,
},
ChannelName: ch,
Timestamp: t,
},
}
}
t.Run("Test segment flush after tt", func(t *testing.T) {
ch := make(chan any, 1)
svr := newTestServer(t, ch)
defer closeTestServer(t, svr)
svr.meta.AddCollection(&collectionInfo{
ID: 0,
Schema: newTestSchema(),
Partitions: []int64{0},
})
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.NoError(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close()
info := &NodeInfo{
Address: "localhost:7777",
NodeID: 0,
}
err = svr.cluster.Register(info)
assert.NoError(t, err)
resp, err := svr.AssignSegmentID(context.TODO(), &datapb.AssignSegmentIDRequest{
NodeID: 0,
PeerRole: "",
SegmentIDRequests: []*datapb.SegmentIDRequest{
{
CollectionID: 0,
PartitionID: 0,
ChannelName: "ch-1",
Count: 100,
},
},
})
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, 1, len(resp.SegIDAssignments))
assign := resp.SegIDAssignments[0]
resp2, err := svr.Flush(context.TODO(), &datapb.FlushRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Flush,
MsgID: 0,
Timestamp: 0,
SourceID: 0,
},
DbID: 0,
CollectionID: 0,
})
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp2.Status.ErrorCode)
msgPack := msgstream.MsgPack{}
msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime)
msg.SegmentsStats = append(msg.SegmentsStats, &commonpb.SegmentStats{
SegmentID: assign.GetSegID(),
NumRows: 1,
})
msgPack.Msgs = append(msgPack.Msgs, msg)
err = ttMsgStream.Produce(&msgPack)
assert.NoError(t, err)
flushMsg := <-ch
flushReq := flushMsg.(*datapb.FlushSegmentsRequest)
assert.EqualValues(t, 1, len(flushReq.SegmentIDs))
assert.EqualValues(t, assign.SegID, flushReq.SegmentIDs[0])
})
t.Run("flush segment with different channels", func(t *testing.T) {
ch := make(chan any, 1)
svr := newTestServer(t, ch)
defer closeTestServer(t, svr)
svr.meta.AddCollection(&collectionInfo{
ID: 0,
Schema: newTestSchema(),
Partitions: []int64{0},
})
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.NoError(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close()
info := &NodeInfo{
Address: "localhost:7777",
NodeID: 0,
}
err = svr.cluster.Register(info)
assert.NoError(t, err)
resp, err := svr.AssignSegmentID(context.TODO(), &datapb.AssignSegmentIDRequest{
NodeID: 0,
PeerRole: "",
SegmentIDRequests: []*datapb.SegmentIDRequest{
{
CollectionID: 0,
PartitionID: 0,
ChannelName: "ch-1",
Count: 100,
},
{
CollectionID: 0,
PartitionID: 0,
ChannelName: "ch-2",
Count: 100,
},
},
})
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, 2, len(resp.SegIDAssignments))
var assign *datapb.SegmentIDAssignment
for _, segment := range resp.SegIDAssignments {
if segment.GetChannelName() == "ch-1" {
assign = segment
break
}
}
assert.NotNil(t, assign)
resp2, err := svr.Flush(context.TODO(), &datapb.FlushRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Flush,
MsgID: 0,
Timestamp: 0,
SourceID: 0,
},
DbID: 0,
CollectionID: 0,
})
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp2.Status.ErrorCode)
msgPack := msgstream.MsgPack{}
msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", assign.ExpireTime)
msg.SegmentsStats = append(msg.SegmentsStats, &commonpb.SegmentStats{
SegmentID: assign.GetSegID(),
NumRows: 1,
})
msgPack.Msgs = append(msgPack.Msgs, msg)
err = ttMsgStream.Produce(&msgPack)
assert.NoError(t, err)
flushMsg := <-ch
flushReq := flushMsg.(*datapb.FlushSegmentsRequest)
assert.EqualValues(t, 1, len(flushReq.SegmentIDs))
assert.EqualValues(t, assign.SegID, flushReq.SegmentIDs[0])
})
t.Run("test expire allocation after receiving tt msg", func(t *testing.T) {
ch := make(chan any, 1)
helper := ServerHelper{
eventAfterHandleDataNodeTt: func() { ch <- struct{}{} },
}
svr := newTestServer(t, nil, WithServerHelper(helper))
defer closeTestServer(t, svr)
svr.meta.AddCollection(&collectionInfo{
ID: 0,
Schema: newTestSchema(),
Partitions: []int64{0},
})
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.NoError(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close()
node := &NodeInfo{
NodeID: 0,
Address: "localhost:7777",
}
err = svr.cluster.Register(node)
assert.NoError(t, err)
resp, err := svr.AssignSegmentID(context.TODO(), &datapb.AssignSegmentIDRequest{
NodeID: 0,
PeerRole: "",
SegmentIDRequests: []*datapb.SegmentIDRequest{
{
CollectionID: 0,
PartitionID: 0,
ChannelName: "ch-1",
Count: 100,
},
},
})
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, 1, len(resp.SegIDAssignments))
assignedSegmentID := resp.SegIDAssignments[0].SegID
segment := svr.meta.GetHealthySegment(assignedSegmentID)
assert.EqualValues(t, 1, len(segment.allocations))
msgPack := msgstream.MsgPack{}
msg := genMsg(commonpb.MsgType_DataNodeTt, "ch-1", resp.SegIDAssignments[0].ExpireTime)
msgPack.Msgs = append(msgPack.Msgs, msg)
err = ttMsgStream.Produce(&msgPack)
assert.NoError(t, err)
<-ch
segment = svr.meta.GetHealthySegment(assignedSegmentID)
assert.EqualValues(t, 0, len(segment.allocations))
})
}
func TestGetChannelSeekPosition(t *testing.T) {
startPos1 := []*commonpb.KeyDataPair{
{
@ -4118,48 +3897,6 @@ func TestDataCoordServer_UpdateChannelCheckpoint(t *testing.T) {
})
}
// https://github.com/milvus-io/milvus/issues/15659
func TestIssue15659(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
s := &Server{
helper: ServerHelper{
eventAfterHandleDataNodeTt: func() {},
},
}
ms := &MockClosePanicMsgstream{}
msgChan := make(chan *msgstream.MsgPack)
go func() {
msgChan <- &msgstream.MsgPack{}
}()
ms.On("Chan").Return(msgChan)
ch := make(chan struct{})
go func() {
assert.NotPanics(t, func() {
s.serverLoopWg.Add(1)
s.handleDataNodeTimetickMsgstream(ctx, ms)
close(ch)
})
}()
cancel()
<-ch
}
type MockClosePanicMsgstream struct {
mock.Mock
msgstream.MsgStream
}
func (ms *MockClosePanicMsgstream) Close() {
panic("mocked close panic")
}
func (ms *MockClosePanicMsgstream) Chan() <-chan *msgstream.MsgPack {
args := ms.Called()
return args.Get(0).(chan *msgstream.MsgPack)
}
func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
var err error
paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))

View File

@ -23,7 +23,6 @@ import (
"strconv"
"sync"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/samber/lo"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
@ -37,10 +36,13 @@ import (
"github.com/milvus-io/milvus/internal/util/segmentutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/errorutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
@ -1416,6 +1418,70 @@ func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update
}, nil
}
// ReportDataNodeTtMsgs send datenode timetick messages to dataCoord.
func (s *Server) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
if s.isClosed() {
log.Warn("failed to report dataNode ttMsgs on closed server")
return merr.Status(merr.WrapErrServiceUnavailable(msgDataCoordIsUnhealthy(s.session.ServerID))), nil
}
for _, ttMsg := range req.GetMsgs() {
sub := tsoutil.SubByNow(req.GetBase().GetTimestamp())
metrics.DataCoordConsumeDataNodeTimeTickLag.
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), ttMsg.GetChannelName()).
Set(float64(sub))
err := s.handleRPCTimetickMessage(ctx, ttMsg)
if err != nil {
log.Error("fail to handle Datanode Timetick Msg",
zap.Int64("sourceID", ttMsg.GetBase().GetSourceID()),
zap.String("channelName", ttMsg.GetChannelName()),
zap.Error(err))
return merr.Status(merr.WrapErrServiceInternal("fail to handle Datanode Timetick Msg")), nil
}
}
return merr.Status(nil), nil
}
func (s *Server) handleRPCTimetickMessage(ctx context.Context, ttMsg *msgpb.DataNodeTtMsg) error {
ch := ttMsg.GetChannelName()
ts := ttMsg.GetTimestamp()
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)
if len(flushableSegments) == 0 {
return nil
}
log.Info("start flushing segments",
zap.Int64s("segment IDs", flushableIDs))
// update segment last update triggered time
// it's ok to fail flushing, since next timetick after duration will re-trigger
s.setLastFlushTime(flushableSegments)
finfo := make([]*datapb.SegmentInfo, 0, len(flushableSegments))
for _, info := range flushableSegments {
finfo = append(finfo, info.SegmentInfo)
}
err = s.cluster.Flush(s.ctx, ttMsg.GetBase().GetSourceID(), ch, finfo)
if err != nil {
log.Warn("failed to handle flush", zap.Any("source", ttMsg.GetBase().GetSourceID()), zap.Error(err))
return err
}
return nil
}
// getDiff returns the difference of base and remove. i.e. all items that are in `base` but not in `remove`.
func getDiff(base, remove []int64) []int64 {
mb := make(map[int64]struct{}, len(remove))

View File

@ -108,6 +108,7 @@ type DataNode struct {
clearSignal chan string // vchannel name
segmentCache *Cache
compactionExecutor *compactionExecutor
timeTickSender *timeTickSender
etcdCli *clientv3.Client
address string
@ -539,6 +540,11 @@ func (node *DataNode) Start() error {
go node.compactionExecutor.start(node.ctx)
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
node.timeTickSender = newTimeTickManager(node.dataCoord, node.session.ServerID)
go node.timeTickSender.start(node.ctx)
}
node.wg.Add(1)
// Start node watch node
go node.StartWatchChannels(node.ctx)

View File

@ -400,11 +400,14 @@ func TestWatchChannel(t *testing.T) {
msFactory := node.factory
defer func() { node.factory = msFactory }()
// todo review the UT logic
// As we remove timetick channel logic, flow_graph_insert_buffer_node no longer depend on MessageStreamFactory
// so data_sync_service can be created. this assert becomes true
node.factory = &FailMessageStreamFactory{}
node.handleWatchInfo(e, ch, bs)
<-chPut
exist = node.flowgraphManager.exist(ch)
assert.False(t, exist)
assert.True(t, exist)
})
t.Run("handle watchinfo out of date", func(t *testing.T) {

View File

@ -63,9 +63,10 @@ type dataSyncService struct {
chunkManager storage.ChunkManager
compactor *compactionExecutor // reference to compaction executor
serverID int64
stopOnce sync.Once
flushListener chan *segmentFlushPack // chan to listen flush event
serverID int64
stopOnce sync.Once
flushListener chan *segmentFlushPack // chan to listen flush event
timetickSender *timeTickSender // reference to timeTickSender
}
func newDataSyncService(ctx context.Context,
@ -83,6 +84,7 @@ func newDataSyncService(ctx context.Context,
compactor *compactionExecutor,
tickler *tickler,
serverID int64,
timetickSender *timeTickSender,
) (*dataSyncService, error) {
if channel == nil {
@ -115,6 +117,7 @@ func newDataSyncService(ctx context.Context,
chunkManager: chunkManager,
compactor: compactor,
serverID: serverID,
timetickSender: timetickSender,
}
if err := service.initNodes(vchan, tickler); err != nil {
@ -326,6 +329,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo, tick
dsService.flushManager,
dsService.flushingSegCache,
c,
dsService.timetickSender,
)
if err != nil {
return err

View File

@ -140,16 +140,6 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
1, 0, "by-dev-rootcoord-dml-test_v1", 0,
1, 1, "by-dev-rootcoord-dml-test_v2", 0,
"add normal segments"},
{false, false, &mockMsgStreamFactory{true, false},
0, "by-dev-rootcoord-dml-test_v0",
0, 0, "", 0,
0, 0, "", 0,
"error when newinsertbufernode"},
{false, true, &mockMsgStreamFactory{true, false},
0, "by-dev-rootcoord-dml-test_v0",
0, 0, "", 0,
0, 0, "", 0,
"channel nil"},
{true, false, &mockMsgStreamFactory{true, true},
1, "by-dev-rootcoord-dml-test_v1",
1, 1, "by-dev-rootcoord-dml-test_v1", 0,
@ -185,6 +175,7 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
newCompactionExecutor(),
genTestTickler(),
0,
nil,
)
if !test.isValidCase {
@ -286,8 +277,9 @@ func TestDataSyncService_Start(t *testing.T) {
},
}
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, alloc, dispClient, factory, vchan, signalCh, dataCoord, newCache(), cm, newCompactionExecutor(), genTestTickler(), 0)
assert.NoError(t, err)
atimeTickSender := newTimeTickManager(dataCoord, 0)
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, alloc, dispClient, factory, vchan, signalCh, dataCoord, newCache(), cm, newCompactionExecutor(), genTestTickler(), 0, atimeTickSender)
assert.Nil(t, err)
sync.flushListener = make(chan *segmentFlushPack)
defer close(sync.flushListener)
@ -443,7 +435,8 @@ func TestDataSyncService_Close(t *testing.T) {
paramtable.Get().Reset(Params.DataNodeCfg.FlushInsertBufferSize.Key)
channel := newChannel(insertChannelName, collMeta.ID, collMeta.GetSchema(), mockRootCoord, cm)
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, alloc, dispClient, factory, vchan, signalCh, mockDataCoord, newCache(), cm, newCompactionExecutor(), genTestTickler(), 0)
atimeTickSender := newTimeTickManager(mockDataCoord, 0)
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, alloc, dispClient, factory, vchan, signalCh, mockDataCoord, newCache(), cm, newCompactionExecutor(), genTestTickler(), 0, atimeTickSender)
assert.NoError(t, err)
sync.flushListener = make(chan *segmentFlushPack, 10)

View File

@ -64,6 +64,8 @@ type insertBufferNode struct {
ttLogger *timeTickLogger
ttMerger *mergedTimeTickerSender
timeTickSender *timeTickSender
lastTimestamp Timestamp
}
@ -97,7 +99,9 @@ func (ibNode *insertBufferNode) Name() string {
}
func (ibNode *insertBufferNode) Close() {
ibNode.ttMerger.close()
if ibNode.ttMerger != nil {
ibNode.ttMerger.close()
}
if ibNode.timeTickStream != nil {
ibNode.timeTickStream.Close()
@ -649,8 +653,22 @@ func (ibNode *insertBufferNode) WriteTimeTick(ts Timestamp, segmentIDs []int64)
default:
}
ibNode.ttLogger.LogTs(ts)
ibNode.ttMerger.bufferTs(ts, segmentIDs)
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
stats := make([]*commonpb.SegmentStats, 0, len(segmentIDs))
for _, sid := range segmentIDs {
stat, err := ibNode.channel.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)
}
ibNode.timeTickSender.update(ibNode.channelName, ts, stats)
} else {
ibNode.ttLogger.LogTs(ts)
ibNode.ttMerger.bufferTs(ts, segmentIDs)
}
rateCol.updateFlowGraphTt(ibNode.channelName, ts)
}
@ -659,12 +677,31 @@ func (ibNode *insertBufferNode) getCollectionandPartitionIDbySegID(segmentID Uni
}
func newInsertBufferNode(ctx context.Context, collID UniqueID, delBufManager *DeltaBufferManager, flushCh <-chan flushMsg, resendTTCh <-chan resendTTMsg,
fm flushManager, flushingSegCache *Cache, config *nodeConfig) (*insertBufferNode, error) {
fm flushManager, flushingSegCache *Cache, config *nodeConfig, timeTickManager *timeTickSender) (*insertBufferNode, error) {
baseNode := BaseNode{}
baseNode.SetMaxQueueLength(config.maxQueueLength)
baseNode.SetMaxParallelism(config.maxParallelism)
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
return &insertBufferNode{
ctx: ctx,
BaseNode: baseNode,
flushMap: sync.Map{},
flushChan: flushCh,
resendTTChan: resendTTCh,
flushingSegCache: flushingSegCache,
flushManager: fm,
delBufferManager: delBufManager,
channel: config.channel,
idAllocator: config.allocator,
channelName: config.vChannelName,
timeTickSender: timeTickManager,
}, nil
}
//input stream, data node time tick
wTt, err := config.msFactory.NewMsgStream(ctx)
if err != nil {

View File

@ -116,22 +116,11 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
delBufHeap: &PriorityQueue{},
}
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c)
dataCoord := &DataCoordFactory{}
atimeTickSender := newTimeTickManager(dataCoord, 0)
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c, atimeTickSender)
assert.NotNil(t, iBNode)
require.NoError(t, err)
/*ctxDone, cancel := context.WithCancel(ctx)
cancel() // cancel now to make context done
_, err = newInsertBufferNode(ctxDone, flushChan, fm, newCache(), c)
assert.Error(t, err)*/
c.msFactory = &CDFMsFactory{
Factory: factory,
cd: 0,
}
_, err = newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c)
assert.Error(t, err)
}
type mockMsg struct {
@ -162,9 +151,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, []int64) error { return nil }),
}
ibn := &insertBufferNode{}
assert.False(t0, ibn.IsValidInMsg(test.in))
})
}
@ -223,12 +210,11 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
delBufHeap: &PriorityQueue{},
}
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c)
dataCoord := &DataCoordFactory{}
atimeTickSender := newTimeTickManager(dataCoord, 0)
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c, atimeTickSender)
require.NoError(t, err)
// trigger log ts
iBNode.ttLogger.counter.Store(999)
flushChan <- flushMsg{
msgID: 1,
timestamp: 2000,
@ -400,7 +386,9 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
channel: channel,
delBufHeap: &PriorityQueue{},
}
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c)
dataCoord := &DataCoordFactory{}
atimeTickSender := newTimeTickManager(dataCoord, 0)
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c, atimeTickSender)
require.NoError(t, err)
// Auto flush number of rows set to 2
@ -645,7 +633,10 @@ func TestInsertBufferNodeRollBF(t *testing.T) {
channel: channel,
delBufHeap: &PriorityQueue{},
}
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c)
dataCoord := &DataCoordFactory{}
atimeTickSender := newTimeTickManager(dataCoord, 0)
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c, atimeTickSender)
require.NoError(t, err)
// Auto flush number of rows set to 2
@ -1020,7 +1011,10 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
channel: channel,
delBufHeap: &PriorityQueue{},
}
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c)
dataCoord := &DataCoordFactory{}
atimeTickSender := newTimeTickManager(dataCoord, 0)
iBNode, err := newInsertBufferNode(ctx, collMeta.ID, delBufManager, flushChan, resendTTChan, fm, newCache(), c, atimeTickSender)
require.NoError(t, err)
inMsg := genFlowGraphInsertMsg(insertChannelName)

View File

@ -112,7 +112,7 @@ func (fm *flowgraphManager) addAndStart(dn *DataNode, vchan *datapb.VchannelInfo
channel := newChannel(vchan.GetChannelName(), vchan.GetCollectionID(), schema, dn.rootCoord, dn.chunkManager)
dataSyncService, err := newDataSyncService(dn.ctx, make(chan flushMsg, 100), make(chan resendTTMsg, 100), channel,
dn.allocator, dn.dispClient, dn.factory, vchan, dn.clearSignal, dn.dataCoord, dn.segmentCache, dn.chunkManager, dn.compactionExecutor, tickler, dn.GetSession().ServerID)
dn.allocator, dn.dispClient, dn.factory, vchan, dn.clearSignal, dn.dataCoord, dn.segmentCache, dn.chunkManager, dn.compactionExecutor, tickler, dn.GetSession().ServerID, dn.timeTickSender)
if err != nil {
log.Warn("fail to create new datasyncservice", zap.Error(err))
return err

View File

@ -1,84 +0,0 @@
package datanode
import (
"sync"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
)
func TestMergedTimeTicker(t *testing.T) {
var ticks []uint64
var mut sync.Mutex
mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error {
mut.Lock()
defer mut.Unlock()
ticks = append(ticks, ts)
return nil
})
for i := 1; i < 100; i++ {
time.Sleep(time.Millisecond * 10)
mt.bufferTs(uint64(i), nil)
}
mt.close()
mut.Lock()
assert.EqualValues(t, 99, ticks[len(ticks)-1])
assert.Less(t, len(ticks), 20)
mut.Unlock()
}
func TestMergedTimeTicker_close10000(t *testing.T) {
var wg sync.WaitGroup
batchSize := 10000
wg.Add(batchSize)
for i := 0; i < batchSize; i++ {
mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error {
return nil
})
go func(mt *mergedTimeTickerSender) {
defer wg.Done()
time.Sleep(10 * time.Millisecond)
mt.close()
}(mt)
}
done := make(chan struct{})
go func() {
wg.Wait()
close(done)
}()
select {
case <-time.After(10 * time.Second):
t.Fatal("wait all timer close, timeout")
case <-done:
}
}
func TestMergedTimeTickerSendFail(t *testing.T) {
var ticks []uint64
var mut sync.Mutex
first := true
mt := newMergedTimeTickerSender(func(ts Timestamp, _ []int64) error {
mut.Lock()
defer mut.Unlock()
if first {
first = false
return errors.New("merged time tick")
}
ticks = append(ticks, ts)
return nil
})
for i := 1; i < 100; i++ {
time.Sleep(time.Millisecond * 10)
mt.bufferTs(uint64(i), nil)
}
mt.close()
mut.Lock()
assert.EqualValues(t, 99, ticks[len(ticks)-1])
assert.Less(t, len(ticks), 20)
mut.Unlock()
}

View File

@ -231,6 +231,9 @@ type DataCoordFactory struct {
AddSegmentError bool
AddSegmentNotSuccess bool
AddSegmentEmpty bool
ReportDataNodeTtMsgsError bool
ReportDataNodeTtMsgsNotSuccess bool
}
func (ds *DataCoordFactory) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
@ -298,6 +301,20 @@ func (ds *DataCoordFactory) UpdateChannelCheckpoint(ctx context.Context, req *da
}, nil
}
func (ds *DataCoordFactory) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
if ds.ReportDataNodeTtMsgsError {
return nil, errors.New("mock ReportDataNodeTtMsgs error")
}
if ds.ReportDataNodeTtMsgsNotSuccess {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
}, nil
}
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil
}
func (ds *DataCoordFactory) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,

View File

@ -0,0 +1,188 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datanode
import (
"context"
"sync"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/retry"
)
// timeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically
// timeTickSender hold a SegmentStats time sequence cache for each channel,
// after send succeeds will clean the cache earlier than the sended timestamp
type timeTickSender struct {
nodeID int64
dataCoord types.DataCoord
mu sync.Mutex
channelStatesCaches map[string]*segmentStatesSequence // string -> *segmentStatesSequence
}
// data struct only used in timeTickSender
type segmentStatesSequence struct {
data map[uint64][]*commonpb.SegmentStats // ts -> segmentStats
}
func newTimeTickManager(dataCoord types.DataCoord, nodeID int64) *timeTickSender {
return &timeTickSender{
nodeID: nodeID,
dataCoord: dataCoord,
channelStatesCaches: make(map[string]*segmentStatesSequence, 0),
}
}
func (m *timeTickSender) start(ctx context.Context) {
ticker := time.NewTicker(Params.DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond))
defer ticker.Stop()
for {
select {
case <-ctx.Done():
log.Info("timeTickSender context done")
return
case t := <-ticker.C:
m.sendReport(ctx, uint64(t.Unix()))
}
}
}
func (m *timeTickSender) update(channelName string, timestamp uint64, segmentStats []*commonpb.SegmentStats) {
m.mu.Lock()
defer m.mu.Unlock()
channelStates, ok := m.channelStatesCaches[channelName]
if !ok {
channelStates = &segmentStatesSequence{
data: make(map[uint64][]*commonpb.SegmentStats, 0),
}
}
channelStates.data[timestamp] = segmentStats
m.channelStatesCaches[channelName] = channelStates
}
func (m *timeTickSender) mergeDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[string]uint64) {
m.mu.Lock()
defer m.mu.Unlock()
var msgs []*msgpb.DataNodeTtMsg
sendedLastTss := make(map[string]uint64, 0)
for channelName, channelSegmentStates := range m.channelStatesCaches {
var lastTs uint64
segNumRows := make(map[int64]int64, 0)
for ts, segmentStates := range channelSegmentStates.data {
if ts > lastTs {
lastTs = ts
}
// merge the same segments into one
for _, segmentStat := range segmentStates {
if v, ok := segNumRows[segmentStat.GetSegmentID()]; ok {
// numRows is supposed to keep growing
if segmentStat.GetNumRows() > v {
segNumRows[segmentStat.GetSegmentID()] = segmentStat.GetNumRows()
}
} else {
segNumRows[segmentStat.GetSegmentID()] = segmentStat.GetNumRows()
}
}
}
toSendSegmentStats := make([]*commonpb.SegmentStats, 0)
for id, numRows := range segNumRows {
toSendSegmentStats = append(toSendSegmentStats, &commonpb.SegmentStats{
SegmentID: id,
NumRows: numRows,
})
}
msgs = append(msgs, &msgpb.DataNodeTtMsg{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DataNodeTt),
commonpbutil.WithSourceID(m.nodeID),
),
ChannelName: channelName,
Timestamp: lastTs,
SegmentsStats: toSendSegmentStats,
})
sendedLastTss[channelName] = lastTs
}
return msgs, sendedLastTss
}
func (m *timeTickSender) cleanStatesCache(sendedLastTss map[string]uint64) {
m.mu.Lock()
defer m.mu.Unlock()
sizeBeforeClean := len(m.channelStatesCaches)
log.With(zap.Any("sendedLastTss", sendedLastTss), zap.Int("sizeBeforeClean", sizeBeforeClean))
for channelName, sendedLastTs := range sendedLastTss {
channelCache, ok := m.channelStatesCaches[channelName]
if ok {
for ts := range channelCache.data {
if ts <= sendedLastTs {
delete(channelCache.data, ts)
}
}
m.channelStatesCaches[channelName] = channelCache
}
if len(channelCache.data) == 0 {
delete(m.channelStatesCaches, channelName)
}
}
log.Debug("timeTickSender channelStatesCaches", zap.Int("sizeAfterClean", len(m.channelStatesCaches)))
}
func (m *timeTickSender) sendReport(ctx context.Context, submitTs Timestamp) error {
toSendMsgs, sendLastTss := m.mergeDatanodeTtMsg()
log.Debug("timeTickSender send datanode timetick message", zap.Any("toSendMsgs", toSendMsgs), zap.Any("sendLastTss", sendLastTss))
err := retry.Do(ctx, func() error {
statusResp, err := m.dataCoord.ReportDataNodeTtMsgs(ctx, &datapb.ReportDataNodeTtMsgsRequest{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DataNodeTt),
commonpbutil.WithTimeStamp(submitTs),
commonpbutil.WithSourceID(m.nodeID),
),
Msgs: toSendMsgs,
})
if err != nil {
log.Warn("error happen when ReportDataNodeTtMsgs", zap.Error(err))
return err
}
if statusResp.GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("ReportDataNodeTtMsgs resp status not succeed",
zap.String("error_code", statusResp.GetErrorCode().String()),
zap.Int32("code", statusResp.GetCode()),
zap.String("reason", statusResp.GetReason()))
return errors.New(statusResp.GetReason())
}
return nil
}, retry.Attempts(20), retry.Sleep(time.Millisecond*100))
if err != nil {
log.Error("ReportDataNodeTtMsgs fail after retry", zap.Error(err))
return err
}
m.cleanStatesCache(sendLastTss)
return nil
}

View File

@ -0,0 +1,161 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datanode
import (
"context"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
)
func TestTimetickManagerNormal(t *testing.T) {
ctx := context.Background()
manager := newTimeTickManager(&DataCoordFactory{}, 0)
channelName1 := "channel1"
ts := uint64(time.Now().Unix())
var segmentID1 int64 = 28257
var segmentID2 int64 = 28258
segmentStats := []*commonpb.SegmentStats{
{
SegmentID: segmentID1,
NumRows: 100,
},
}
// update first time
manager.update(channelName1, ts, segmentStats)
channel1SegmentStates, channelSegmentStatesExist := manager.channelStatesCaches[channelName1]
assert.Equal(t, true, channelSegmentStatesExist)
segmentState1, segmentState1Exist := channel1SegmentStates.data[ts]
assert.Equal(t, segmentStats[0], segmentState1[0])
assert.Equal(t, true, segmentState1Exist)
// update second time
segmentStats2 := []*commonpb.SegmentStats{
{
SegmentID: segmentID1,
NumRows: 10000,
},
{
SegmentID: segmentID2,
NumRows: 33333,
},
}
ts2 := uint64(time.Now().Unix())
manager.update(channelName1, ts2, segmentStats2)
channelSegmentStates, channelSegmentStatesExist := manager.channelStatesCaches[channelName1]
assert.Equal(t, true, channelSegmentStatesExist)
segmentStates, segmentStatesExist := channelSegmentStates.data[ts2]
assert.Equal(t, segmentStats2[0], segmentStates[0])
assert.Equal(t, segmentStats2[1], segmentStates[1])
assert.Equal(t, true, segmentStatesExist)
var segmentID3 int64 = 28259
var segmentID4 int64 = 28260
channelName2 := "channel2"
ts3 := uint64(time.Now().Unix())
segmentStats3 := []*commonpb.SegmentStats{
{
SegmentID: segmentID3,
NumRows: 1000000,
},
{
SegmentID: segmentID4,
NumRows: 3333300,
},
}
manager.update(channelName2, ts3, segmentStats3)
err := manager.sendReport(ctx, 100)
assert.NoError(t, err)
_, channelExistAfterSubmit := manager.channelStatesCaches[channelName1]
assert.Equal(t, false, channelExistAfterSubmit)
_, channelSegmentStatesExistAfterSubmit := manager.channelStatesCaches[channelName1]
assert.Equal(t, false, channelSegmentStatesExistAfterSubmit)
var segmentID5 int64 = 28261
var segmentID6 int64 = 28262
channelName3 := "channel3"
ts4 := uint64(time.Now().Unix())
segmentStats4 := []*commonpb.SegmentStats{
{
SegmentID: segmentID5,
NumRows: 1000000,
},
{
SegmentID: segmentID6,
NumRows: 3333300,
},
}
manager.update(channelName3, ts4, segmentStats4)
err = manager.sendReport(ctx, 100)
assert.NoError(t, err)
_, channelExistAfterSubmit2 := manager.channelStatesCaches[channelName1]
assert.Equal(t, false, channelExistAfterSubmit2)
_, channelSegmentStatesExistAfterSubmit2 := manager.channelStatesCaches[channelName1]
assert.Equal(t, false, channelSegmentStatesExistAfterSubmit2)
}
func TestTimetickManagerSendErr(t *testing.T) {
ctx := context.Background()
manager := newTimeTickManager(&DataCoordFactory{ReportDataNodeTtMsgsError: true}, 0)
channelName1 := "channel1"
ts := uint64(time.Now().Unix())
var segmentID1 int64 = 28257
segmentStats := []*commonpb.SegmentStats{
{
SegmentID: segmentID1,
NumRows: 100,
},
}
// update first time
manager.update(channelName1, ts, segmentStats)
err := manager.sendReport(ctx, 100)
assert.Error(t, err)
}
func TestTimetickManagerSendNotSuccess(t *testing.T) {
ctx := context.Background()
manager := newTimeTickManager(&DataCoordFactory{ReportDataNodeTtMsgsNotSuccess: true}, 0)
channelName1 := "channel1"
ts := uint64(time.Now().Unix())
var segmentID1 int64 = 28257
segmentStats := []*commonpb.SegmentStats{
{
SegmentID: segmentID1,
NumRows: 100,
},
}
// update first time
manager.update(channelName1, ts, segmentStats)
err := manager.sendReport(ctx, 100)
assert.Error(t, err)
}

View File

@ -637,3 +637,16 @@ func (c *Client) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (
return client.DropIndex(ctx, req)
})
}
func (c *Client) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.ReportDataNodeTtMsgs(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}

View File

@ -430,3 +430,7 @@ func (s *Server) DropIndex(ctx context.Context, request *indexpb.DropIndexReques
func (s *Server) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return s.dataCoord.GetIndexBuildProgress(ctx, req)
}
func (s *Server) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
return s.dataCoord.ReportDataNodeTtMsgs(ctx, req)
}

View File

@ -487,6 +487,10 @@ func (m *MockDataCoord) GetIndexBuildProgress(ctx context.Context, req *indexpb.
return nil, nil
}
func (m *MockDataCoord) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
return nil, nil
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockProxy struct {
MockBase

View File

@ -60,8 +60,8 @@ type DataCoord_AssignSegmentID_Call struct {
}
// AssignSegmentID is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.AssignSegmentIDRequest
// - ctx context.Context
// - req *datapb.AssignSegmentIDRequest
func (_e *DataCoord_Expecter) AssignSegmentID(ctx interface{}, req interface{}) *DataCoord_AssignSegmentID_Call {
return &DataCoord_AssignSegmentID_Call{Call: _e.mock.On("AssignSegmentID", ctx, req)}
}
@ -107,8 +107,8 @@ type DataCoord_BroadcastAlteredCollection_Call struct {
}
// BroadcastAlteredCollection is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.AlterCollectionRequest
// - ctx context.Context
// - req *datapb.AlterCollectionRequest
func (_e *DataCoord_Expecter) BroadcastAlteredCollection(ctx interface{}, req interface{}) *DataCoord_BroadcastAlteredCollection_Call {
return &DataCoord_BroadcastAlteredCollection_Call{Call: _e.mock.On("BroadcastAlteredCollection", ctx, req)}
}
@ -154,8 +154,8 @@ type DataCoord_CheckHealth_Call struct {
}
// CheckHealth is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.CheckHealthRequest
// - ctx context.Context
// - req *milvuspb.CheckHealthRequest
func (_e *DataCoord_Expecter) CheckHealth(ctx interface{}, req interface{}) *DataCoord_CheckHealth_Call {
return &DataCoord_CheckHealth_Call{Call: _e.mock.On("CheckHealth", ctx, req)}
}
@ -201,8 +201,8 @@ type DataCoord_CreateIndex_Call struct {
}
// CreateIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.CreateIndexRequest
// - ctx context.Context
// - req *indexpb.CreateIndexRequest
func (_e *DataCoord_Expecter) CreateIndex(ctx interface{}, req interface{}) *DataCoord_CreateIndex_Call {
return &DataCoord_CreateIndex_Call{Call: _e.mock.On("CreateIndex", ctx, req)}
}
@ -248,8 +248,8 @@ type DataCoord_DescribeIndex_Call struct {
}
// DescribeIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.DescribeIndexRequest
// - ctx context.Context
// - req *indexpb.DescribeIndexRequest
func (_e *DataCoord_Expecter) DescribeIndex(ctx interface{}, req interface{}) *DataCoord_DescribeIndex_Call {
return &DataCoord_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex", ctx, req)}
}
@ -295,8 +295,8 @@ type DataCoord_DropIndex_Call struct {
}
// DropIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.DropIndexRequest
// - ctx context.Context
// - req *indexpb.DropIndexRequest
func (_e *DataCoord_Expecter) DropIndex(ctx interface{}, req interface{}) *DataCoord_DropIndex_Call {
return &DataCoord_DropIndex_Call{Call: _e.mock.On("DropIndex", ctx, req)}
}
@ -342,8 +342,8 @@ type DataCoord_DropVirtualChannel_Call struct {
}
// DropVirtualChannel is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.DropVirtualChannelRequest
// - ctx context.Context
// - req *datapb.DropVirtualChannelRequest
func (_e *DataCoord_Expecter) DropVirtualChannel(ctx interface{}, req interface{}) *DataCoord_DropVirtualChannel_Call {
return &DataCoord_DropVirtualChannel_Call{Call: _e.mock.On("DropVirtualChannel", ctx, req)}
}
@ -389,8 +389,8 @@ type DataCoord_Flush_Call struct {
}
// Flush is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.FlushRequest
// - ctx context.Context
// - req *datapb.FlushRequest
func (_e *DataCoord_Expecter) Flush(ctx interface{}, req interface{}) *DataCoord_Flush_Call {
return &DataCoord_Flush_Call{Call: _e.mock.On("Flush", ctx, req)}
}
@ -436,8 +436,8 @@ type DataCoord_GcConfirm_Call struct {
}
// GcConfirm is a helper method to define mock.On call
// - ctx context.Context
// - request *datapb.GcConfirmRequest
// - ctx context.Context
// - request *datapb.GcConfirmRequest
func (_e *DataCoord_Expecter) GcConfirm(ctx interface{}, request interface{}) *DataCoord_GcConfirm_Call {
return &DataCoord_GcConfirm_Call{Call: _e.mock.On("GcConfirm", ctx, request)}
}
@ -483,8 +483,8 @@ type DataCoord_GetCollectionStatistics_Call struct {
}
// GetCollectionStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetCollectionStatisticsRequest
// - ctx context.Context
// - req *datapb.GetCollectionStatisticsRequest
func (_e *DataCoord_Expecter) GetCollectionStatistics(ctx interface{}, req interface{}) *DataCoord_GetCollectionStatistics_Call {
return &DataCoord_GetCollectionStatistics_Call{Call: _e.mock.On("GetCollectionStatistics", ctx, req)}
}
@ -530,8 +530,8 @@ type DataCoord_GetCompactionState_Call struct {
}
// GetCompactionState is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetCompactionStateRequest
// - ctx context.Context
// - req *milvuspb.GetCompactionStateRequest
func (_e *DataCoord_Expecter) GetCompactionState(ctx interface{}, req interface{}) *DataCoord_GetCompactionState_Call {
return &DataCoord_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", ctx, req)}
}
@ -577,8 +577,8 @@ type DataCoord_GetCompactionStateWithPlans_Call struct {
}
// GetCompactionStateWithPlans is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetCompactionPlansRequest
// - ctx context.Context
// - req *milvuspb.GetCompactionPlansRequest
func (_e *DataCoord_Expecter) GetCompactionStateWithPlans(ctx interface{}, req interface{}) *DataCoord_GetCompactionStateWithPlans_Call {
return &DataCoord_GetCompactionStateWithPlans_Call{Call: _e.mock.On("GetCompactionStateWithPlans", ctx, req)}
}
@ -624,7 +624,7 @@ type DataCoord_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *DataCoord_Expecter) GetComponentStates(ctx interface{}) *DataCoord_GetComponentStates_Call {
return &DataCoord_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", ctx)}
}
@ -670,8 +670,8 @@ type DataCoord_GetFlushAllState_Call struct {
}
// GetFlushAllState is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetFlushAllStateRequest
// - ctx context.Context
// - req *milvuspb.GetFlushAllStateRequest
func (_e *DataCoord_Expecter) GetFlushAllState(ctx interface{}, req interface{}) *DataCoord_GetFlushAllState_Call {
return &DataCoord_GetFlushAllState_Call{Call: _e.mock.On("GetFlushAllState", ctx, req)}
}
@ -717,8 +717,8 @@ type DataCoord_GetFlushState_Call struct {
}
// GetFlushState is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetFlushStateRequest
// - ctx context.Context
// - req *milvuspb.GetFlushStateRequest
func (_e *DataCoord_Expecter) GetFlushState(ctx interface{}, req interface{}) *DataCoord_GetFlushState_Call {
return &DataCoord_GetFlushState_Call{Call: _e.mock.On("GetFlushState", ctx, req)}
}
@ -764,8 +764,8 @@ type DataCoord_GetFlushedSegments_Call struct {
}
// GetFlushedSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetFlushedSegmentsRequest
// - ctx context.Context
// - req *datapb.GetFlushedSegmentsRequest
func (_e *DataCoord_Expecter) GetFlushedSegments(ctx interface{}, req interface{}) *DataCoord_GetFlushedSegments_Call {
return &DataCoord_GetFlushedSegments_Call{Call: _e.mock.On("GetFlushedSegments", ctx, req)}
}
@ -811,8 +811,8 @@ type DataCoord_GetIndexBuildProgress_Call struct {
}
// GetIndexBuildProgress is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexBuildProgressRequest
// - ctx context.Context
// - req *indexpb.GetIndexBuildProgressRequest
func (_e *DataCoord_Expecter) GetIndexBuildProgress(ctx interface{}, req interface{}) *DataCoord_GetIndexBuildProgress_Call {
return &DataCoord_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress", ctx, req)}
}
@ -858,8 +858,8 @@ type DataCoord_GetIndexInfos_Call struct {
}
// GetIndexInfos is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexInfoRequest
// - ctx context.Context
// - req *indexpb.GetIndexInfoRequest
func (_e *DataCoord_Expecter) GetIndexInfos(ctx interface{}, req interface{}) *DataCoord_GetIndexInfos_Call {
return &DataCoord_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos", ctx, req)}
}
@ -905,8 +905,8 @@ type DataCoord_GetIndexState_Call struct {
}
// GetIndexState is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexStateRequest
// - ctx context.Context
// - req *indexpb.GetIndexStateRequest
func (_e *DataCoord_Expecter) GetIndexState(ctx interface{}, req interface{}) *DataCoord_GetIndexState_Call {
return &DataCoord_GetIndexState_Call{Call: _e.mock.On("GetIndexState", ctx, req)}
}
@ -952,8 +952,8 @@ type DataCoord_GetIndexStatistics_Call struct {
}
// GetIndexStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexStatisticsRequest
// - ctx context.Context
// - req *indexpb.GetIndexStatisticsRequest
func (_e *DataCoord_Expecter) GetIndexStatistics(ctx interface{}, req interface{}) *DataCoord_GetIndexStatistics_Call {
return &DataCoord_GetIndexStatistics_Call{Call: _e.mock.On("GetIndexStatistics", ctx, req)}
}
@ -999,8 +999,8 @@ type DataCoord_GetInsertBinlogPaths_Call struct {
}
// GetInsertBinlogPaths is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetInsertBinlogPathsRequest
// - ctx context.Context
// - req *datapb.GetInsertBinlogPathsRequest
func (_e *DataCoord_Expecter) GetInsertBinlogPaths(ctx interface{}, req interface{}) *DataCoord_GetInsertBinlogPaths_Call {
return &DataCoord_GetInsertBinlogPaths_Call{Call: _e.mock.On("GetInsertBinlogPaths", ctx, req)}
}
@ -1046,8 +1046,8 @@ type DataCoord_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
func (_e *DataCoord_Expecter) GetMetrics(ctx interface{}, req interface{}) *DataCoord_GetMetrics_Call {
return &DataCoord_GetMetrics_Call{Call: _e.mock.On("GetMetrics", ctx, req)}
}
@ -1093,8 +1093,8 @@ type DataCoord_GetPartitionStatistics_Call struct {
}
// GetPartitionStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetPartitionStatisticsRequest
// - ctx context.Context
// - req *datapb.GetPartitionStatisticsRequest
func (_e *DataCoord_Expecter) GetPartitionStatistics(ctx interface{}, req interface{}) *DataCoord_GetPartitionStatistics_Call {
return &DataCoord_GetPartitionStatistics_Call{Call: _e.mock.On("GetPartitionStatistics", ctx, req)}
}
@ -1140,8 +1140,8 @@ type DataCoord_GetRecoveryInfo_Call struct {
}
// GetRecoveryInfo is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequest
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequest
func (_e *DataCoord_Expecter) GetRecoveryInfo(ctx interface{}, req interface{}) *DataCoord_GetRecoveryInfo_Call {
return &DataCoord_GetRecoveryInfo_Call{Call: _e.mock.On("GetRecoveryInfo", ctx, req)}
}
@ -1187,8 +1187,8 @@ type DataCoord_GetRecoveryInfoV2_Call struct {
}
// GetRecoveryInfoV2 is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequestV2
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequestV2
func (_e *DataCoord_Expecter) GetRecoveryInfoV2(ctx interface{}, req interface{}) *DataCoord_GetRecoveryInfoV2_Call {
return &DataCoord_GetRecoveryInfoV2_Call{Call: _e.mock.On("GetRecoveryInfoV2", ctx, req)}
}
@ -1234,8 +1234,8 @@ type DataCoord_GetSegmentIndexState_Call struct {
}
// GetSegmentIndexState is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetSegmentIndexStateRequest
// - ctx context.Context
// - req *indexpb.GetSegmentIndexStateRequest
func (_e *DataCoord_Expecter) GetSegmentIndexState(ctx interface{}, req interface{}) *DataCoord_GetSegmentIndexState_Call {
return &DataCoord_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState", ctx, req)}
}
@ -1281,8 +1281,8 @@ type DataCoord_GetSegmentInfo_Call struct {
}
// GetSegmentInfo is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetSegmentInfoRequest
// - ctx context.Context
// - req *datapb.GetSegmentInfoRequest
func (_e *DataCoord_Expecter) GetSegmentInfo(ctx interface{}, req interface{}) *DataCoord_GetSegmentInfo_Call {
return &DataCoord_GetSegmentInfo_Call{Call: _e.mock.On("GetSegmentInfo", ctx, req)}
}
@ -1328,7 +1328,7 @@ type DataCoord_GetSegmentInfoChannel_Call struct {
}
// GetSegmentInfoChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *DataCoord_Expecter) GetSegmentInfoChannel(ctx interface{}) *DataCoord_GetSegmentInfoChannel_Call {
return &DataCoord_GetSegmentInfoChannel_Call{Call: _e.mock.On("GetSegmentInfoChannel", ctx)}
}
@ -1374,8 +1374,8 @@ type DataCoord_GetSegmentStates_Call struct {
}
// GetSegmentStates is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetSegmentStatesRequest
// - ctx context.Context
// - req *datapb.GetSegmentStatesRequest
func (_e *DataCoord_Expecter) GetSegmentStates(ctx interface{}, req interface{}) *DataCoord_GetSegmentStates_Call {
return &DataCoord_GetSegmentStates_Call{Call: _e.mock.On("GetSegmentStates", ctx, req)}
}
@ -1421,8 +1421,8 @@ type DataCoord_GetSegmentsByStates_Call struct {
}
// GetSegmentsByStates is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetSegmentsByStatesRequest
// - ctx context.Context
// - req *datapb.GetSegmentsByStatesRequest
func (_e *DataCoord_Expecter) GetSegmentsByStates(ctx interface{}, req interface{}) *DataCoord_GetSegmentsByStates_Call {
return &DataCoord_GetSegmentsByStates_Call{Call: _e.mock.On("GetSegmentsByStates", ctx, req)}
}
@ -1468,7 +1468,7 @@ type DataCoord_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *DataCoord_Expecter) GetStatisticsChannel(ctx interface{}) *DataCoord_GetStatisticsChannel_Call {
return &DataCoord_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", ctx)}
}
@ -1514,7 +1514,7 @@ type DataCoord_GetTimeTickChannel_Call struct {
}
// GetTimeTickChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *DataCoord_Expecter) GetTimeTickChannel(ctx interface{}) *DataCoord_GetTimeTickChannel_Call {
return &DataCoord_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel", ctx)}
}
@ -1560,8 +1560,8 @@ type DataCoord_Import_Call struct {
}
// Import is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ImportTaskRequest
// - ctx context.Context
// - req *datapb.ImportTaskRequest
func (_e *DataCoord_Expecter) Import(ctx interface{}, req interface{}) *DataCoord_Import_Call {
return &DataCoord_Import_Call{Call: _e.mock.On("Import", ctx, req)}
}
@ -1643,8 +1643,8 @@ type DataCoord_ManualCompaction_Call struct {
}
// ManualCompaction is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.ManualCompactionRequest
// - ctx context.Context
// - req *milvuspb.ManualCompactionRequest
func (_e *DataCoord_Expecter) ManualCompaction(ctx interface{}, req interface{}) *DataCoord_ManualCompaction_Call {
return &DataCoord_ManualCompaction_Call{Call: _e.mock.On("ManualCompaction", ctx, req)}
}
@ -1690,8 +1690,8 @@ type DataCoord_MarkSegmentsDropped_Call struct {
}
// MarkSegmentsDropped is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.MarkSegmentsDroppedRequest
// - ctx context.Context
// - req *datapb.MarkSegmentsDroppedRequest
func (_e *DataCoord_Expecter) MarkSegmentsDropped(ctx interface{}, req interface{}) *DataCoord_MarkSegmentsDropped_Call {
return &DataCoord_MarkSegmentsDropped_Call{Call: _e.mock.On("MarkSegmentsDropped", ctx, req)}
}
@ -1744,6 +1744,53 @@ func (_c *DataCoord_Register_Call) Return(_a0 error) *DataCoord_Register_Call {
return _c
}
// ReportDataNodeTtMsgs provides a mock function with given fields: ctx, req
func (_m *DataCoord) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
var r0 *commonpb.Status
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ReportDataNodeTtMsgsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ReportDataNodeTtMsgsRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// DataCoord_ReportDataNodeTtMsgs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ReportDataNodeTtMsgs'
type DataCoord_ReportDataNodeTtMsgs_Call struct {
*mock.Call
}
// ReportDataNodeTtMsgs is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ReportDataNodeTtMsgsRequest
func (_e *DataCoord_Expecter) ReportDataNodeTtMsgs(ctx interface{}, req interface{}) *DataCoord_ReportDataNodeTtMsgs_Call {
return &DataCoord_ReportDataNodeTtMsgs_Call{Call: _e.mock.On("ReportDataNodeTtMsgs", ctx, req)}
}
func (_c *DataCoord_ReportDataNodeTtMsgs_Call) Run(run func(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest)) *DataCoord_ReportDataNodeTtMsgs_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.ReportDataNodeTtMsgsRequest))
})
return _c
}
func (_c *DataCoord_ReportDataNodeTtMsgs_Call) Return(_a0 *commonpb.Status, _a1 error) *DataCoord_ReportDataNodeTtMsgs_Call {
_c.Call.Return(_a0, _a1)
return _c
}
// SaveBinlogPaths provides a mock function with given fields: ctx, req
func (_m *DataCoord) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
@ -1773,8 +1820,8 @@ type DataCoord_SaveBinlogPaths_Call struct {
}
// SaveBinlogPaths is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SaveBinlogPathsRequest
// - ctx context.Context
// - req *datapb.SaveBinlogPathsRequest
func (_e *DataCoord_Expecter) SaveBinlogPaths(ctx interface{}, req interface{}) *DataCoord_SaveBinlogPaths_Call {
return &DataCoord_SaveBinlogPaths_Call{Call: _e.mock.On("SaveBinlogPaths", ctx, req)}
}
@ -1820,8 +1867,8 @@ type DataCoord_SaveImportSegment_Call struct {
}
// SaveImportSegment is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SaveImportSegmentRequest
// - ctx context.Context
// - req *datapb.SaveImportSegmentRequest
func (_e *DataCoord_Expecter) SaveImportSegment(ctx interface{}, req interface{}) *DataCoord_SaveImportSegment_Call {
return &DataCoord_SaveImportSegment_Call{Call: _e.mock.On("SaveImportSegment", ctx, req)}
}
@ -1867,8 +1914,8 @@ type DataCoord_SetSegmentState_Call struct {
}
// SetSegmentState is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SetSegmentStateRequest
// - ctx context.Context
// - req *datapb.SetSegmentStateRequest
func (_e *DataCoord_Expecter) SetSegmentState(ctx interface{}, req interface{}) *DataCoord_SetSegmentState_Call {
return &DataCoord_SetSegmentState_Call{Call: _e.mock.On("SetSegmentState", ctx, req)}
}
@ -1914,8 +1961,8 @@ type DataCoord_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
func (_e *DataCoord_Expecter) ShowConfigurations(ctx interface{}, req interface{}) *DataCoord_ShowConfigurations_Call {
return &DataCoord_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", ctx, req)}
}
@ -2033,8 +2080,8 @@ type DataCoord_UnsetIsImportingState_Call struct {
}
// UnsetIsImportingState is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.UnsetIsImportingStateRequest
// - ctx context.Context
// - req *datapb.UnsetIsImportingStateRequest
func (_e *DataCoord_Expecter) UnsetIsImportingState(ctx interface{}, req interface{}) *DataCoord_UnsetIsImportingState_Call {
return &DataCoord_UnsetIsImportingState_Call{Call: _e.mock.On("UnsetIsImportingState", ctx, req)}
}
@ -2080,8 +2127,8 @@ type DataCoord_UpdateChannelCheckpoint_Call struct {
}
// UpdateChannelCheckpoint is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.UpdateChannelCheckpointRequest
// - ctx context.Context
// - req *datapb.UpdateChannelCheckpointRequest
func (_e *DataCoord_Expecter) UpdateChannelCheckpoint(ctx interface{}, req interface{}) *DataCoord_UpdateChannelCheckpoint_Call {
return &DataCoord_UpdateChannelCheckpoint_Call{Call: _e.mock.On("UpdateChannelCheckpoint", ctx, req)}
}
@ -2127,8 +2174,8 @@ type DataCoord_UpdateSegmentStatistics_Call struct {
}
// UpdateSegmentStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.UpdateSegmentStatisticsRequest
// - ctx context.Context
// - req *datapb.UpdateSegmentStatisticsRequest
func (_e *DataCoord_Expecter) UpdateSegmentStatistics(ctx interface{}, req interface{}) *DataCoord_UpdateSegmentStatistics_Call {
return &DataCoord_UpdateSegmentStatistics_Call{Call: _e.mock.On("UpdateSegmentStatistics", ctx, req)}
}
@ -2174,8 +2221,8 @@ type DataCoord_WatchChannels_Call struct {
}
// WatchChannels is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.WatchChannelsRequest
// - ctx context.Context
// - req *datapb.WatchChannelsRequest
func (_e *DataCoord_Expecter) WatchChannels(ctx interface{}, req interface{}) *DataCoord_WatchChannels_Call {
return &DataCoord_WatchChannels_Call{Call: _e.mock.On("WatchChannels", ctx, req)}
}

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.14.0. DO NOT EDIT.
// Code generated by mockery v2.16.0. DO NOT EDIT.
package mocks

View File

@ -83,6 +83,8 @@ service DataCoord {
rpc GetIndexBuildProgress(index.GetIndexBuildProgressRequest) returns (index.GetIndexBuildProgressResponse) {}
rpc GcConfirm(GcConfirmRequest) returns (GcConfirmResponse) {}
rpc ReportDataNodeTtMsgs(ReportDataNodeTtMsgsRequest) returns (common.Status) {}
}
service DataNode {
@ -662,142 +664,7 @@ message GcConfirmResponse {
bool gc_finished = 2;
}
//message IndexInfo {
// int64 collectionID = 1;
// int64 fieldID = 2;
// string index_name = 3;
// int64 indexID = 4;
// repeated common.KeyValuePair type_params = 5;
// repeated common.KeyValuePair index_params = 6;
// // index build progress
// // The real-time statistics may not be expected due to the existence of the compaction mechanism.
// int64 indexed_rows = 7;
// int64 total_rows = 8;
// // index state
// common.IndexState state = 9;
// string index_state_fail_reason = 10;
// bool is_auto_index = 11;
// repeated common.KeyValuePair user_index_params = 12;
//}
//
//message FieldIndex {
// IndexInfo index_info = 1;
// bool deleted = 2;
// uint64 create_time = 3;
//}
//
//message SegmentIndex {
// int64 collectionID = 1;
// int64 partitionID = 2;
// int64 segmentID = 3;
// int64 num_rows = 4;
// int64 indexID = 5;
// int64 buildID = 6;
// int64 nodeID = 7;
// int64 index_version = 8;
// common.IndexState state = 9;
// string fail_reason = 10;
// repeated string index_file_keys = 11;
// bool deleted = 12;
// uint64 create_time = 13;
// uint64 serialize_size = 14;
// bool write_handoff = 15;
//}
//
//message GetIndexStateRequest {
// int64 collectionID = 1;
// string index_name = 2;
//}
//
//message GetIndexStateResponse {
// common.Status status = 1;
// common.IndexState state = 2;
// string fail_reason = 3;
//}
//
//message GetSegmentIndexStateRequest {
// int64 collectionID = 1;
// string index_name = 2;
// repeated int64 segmentIDs = 3;
//}
//
//message SegmentIndexState {
// int64 segmentID = 1;
// common.IndexState state = 2;
// string fail_reason = 3;
//}
//
//message GetSegmentIndexStateResponse {
// common.Status status = 1;
// repeated SegmentIndexState states = 2;
//}
//
//message CreateIndexRequest {
// int64 collectionID = 1;
// int64 fieldID = 2;
// string index_name = 3;
// repeated common.KeyValuePair type_params = 4;
// repeated common.KeyValuePair index_params = 5;
// uint64 timestamp = 6;
// bool is_auto_index = 7;
// repeated common.KeyValuePair user_index_params = 8;
//}
//
//message GetIndexInfoRequest {
// int64 collectionID = 1;
// repeated int64 segmentIDs = 2;
// string index_name = 3;
//}
//
//message IndexFilePathInfo {
// int64 segmentID = 1;
// int64 fieldID = 2;
// int64 indexID = 3;
// int64 buildID = 4;
// string index_name = 5;
// repeated common.KeyValuePair index_params = 6;
// repeated string index_file_paths = 7;
// uint64 serialized_size = 8;
// int64 index_version = 9;
// int64 num_rows = 10;
//}
//
//message SegmentIndexInfo {
// int64 collectionID = 1;
// int64 segmentID = 2;
// bool enable_index = 3;
// repeated IndexFilePathInfo index_infos = 4;
//}
//
//message GetIndexInfoResponse {
// common.Status status = 1;
// map<int64, SegmentIndexInfo> segment_info = 2;
//}
//
//message DropIndexRequest {
// int64 collectionID = 1;
// repeated int64 partitionIDs = 2;
// string index_name = 3;
// bool drop_all = 4;
//}
//
//message DescribeIndexRequest {
// int64 collectionID = 1;
// string index_name = 2;
//}
//
//message DescribeIndexResponse {
// common.Status status = 1;
// repeated IndexInfo index_infos = 2;
//}
//
//message GetIndexBuildProgressRequest {
// int64 collectionID = 1;
// string index_name = 2;
//}
//
//message GetIndexBuildProgressResponse {
// common.Status status = 1;
// int64 indexed_rows = 2;
// int64 total_rows = 3;
//}
message ReportDataNodeTtMsgsRequest {
common.MsgBase base = 1;
repeated msg.DataNodeTtMsg msgs = 2; // -1 means whole collection.
}

View File

@ -4893,6 +4893,53 @@ func (m *GcConfirmResponse) GetGcFinished() bool {
return false
}
type ReportDataNodeTtMsgsRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
Msgs []*msgpb.DataNodeTtMsg `protobuf:"bytes,2,rep,name=msgs,proto3" json:"msgs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *ReportDataNodeTtMsgsRequest) Reset() { *m = ReportDataNodeTtMsgsRequest{} }
func (m *ReportDataNodeTtMsgsRequest) String() string { return proto.CompactTextString(m) }
func (*ReportDataNodeTtMsgsRequest) ProtoMessage() {}
func (*ReportDataNodeTtMsgsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{76}
}
func (m *ReportDataNodeTtMsgsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReportDataNodeTtMsgsRequest.Unmarshal(m, b)
}
func (m *ReportDataNodeTtMsgsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ReportDataNodeTtMsgsRequest.Marshal(b, m, deterministic)
}
func (m *ReportDataNodeTtMsgsRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_ReportDataNodeTtMsgsRequest.Merge(m, src)
}
func (m *ReportDataNodeTtMsgsRequest) XXX_Size() int {
return xxx_messageInfo_ReportDataNodeTtMsgsRequest.Size(m)
}
func (m *ReportDataNodeTtMsgsRequest) XXX_DiscardUnknown() {
xxx_messageInfo_ReportDataNodeTtMsgsRequest.DiscardUnknown(m)
}
var xxx_messageInfo_ReportDataNodeTtMsgsRequest proto.InternalMessageInfo
func (m *ReportDataNodeTtMsgsRequest) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *ReportDataNodeTtMsgsRequest) GetMsgs() []*msgpb.DataNodeTtMsg {
if m != nil {
return m.Msgs
}
return nil
}
func init() {
proto.RegisterEnum("milvus.proto.data.SegmentType", SegmentType_name, SegmentType_value)
proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value)
@ -4974,307 +5021,311 @@ func init() {
proto.RegisterType((*AlterCollectionRequest)(nil), "milvus.proto.data.AlterCollectionRequest")
proto.RegisterType((*GcConfirmRequest)(nil), "milvus.proto.data.GcConfirmRequest")
proto.RegisterType((*GcConfirmResponse)(nil), "milvus.proto.data.GcConfirmResponse")
proto.RegisterType((*ReportDataNodeTtMsgsRequest)(nil), "milvus.proto.data.ReportDataNodeTtMsgsRequest")
}
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 4720 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0x4b, 0x6f, 0x1b, 0x49,
0x7a, 0x6e, 0xbe, 0x44, 0x7e, 0xa4, 0x28, 0xaa, 0xec, 0x91, 0x69, 0xfa, 0x39, 0x6d, 0x7b, 0xac,
0xf1, 0xd8, 0xb2, 0x47, 0xce, 0x62, 0x67, 0xc7, 0x3b, 0xb3, 0x6b, 0x49, 0x63, 0x0f, 0x13, 0xcb,
0xab, 0x6d, 0xc9, 0x76, 0x30, 0x1b, 0x80, 0x68, 0xb1, 0x4b, 0x54, 0xaf, 0xc8, 0x6e, 0xba, 0xbb,
0x29, 0x59, 0x13, 0x20, 0x99, 0x3c, 0x81, 0x3c, 0x90, 0x04, 0x41, 0x82, 0x24, 0xb7, 0x45, 0x0e,
0xc1, 0x26, 0xc1, 0x9e, 0x36, 0xb9, 0xe4, 0xb2, 0xd7, 0x09, 0x72, 0x18, 0x24, 0x01, 0x02, 0xe4,
0x90, 0x6b, 0x90, 0x7b, 0xfe, 0x40, 0x50, 0x8f, 0xae, 0x7e, 0x55, 0x37, 0x5b, 0x92, 0x3d, 0x06,
0xb2, 0x37, 0x56, 0xf5, 0x57, 0x5f, 0x7d, 0xf5, 0xd5, 0xf7, 0xae, 0x2a, 0x42, 0xcb, 0xd0, 0x3d,
0xbd, 0xd7, 0xb7, 0x6d, 0xc7, 0x58, 0x1a, 0x3b, 0xb6, 0x67, 0xa3, 0xf9, 0x91, 0x39, 0xdc, 0x9f,
0xb8, 0xac, 0xb5, 0x44, 0x3e, 0x77, 0x1a, 0x7d, 0x7b, 0x34, 0xb2, 0x2d, 0xd6, 0xd5, 0x69, 0x9a,
0x96, 0x87, 0x1d, 0x4b, 0x1f, 0xf2, 0x76, 0x23, 0x3c, 0xa0, 0xd3, 0x70, 0xfb, 0xbb, 0x78, 0xa4,
0xf3, 0x56, 0x6d, 0xe4, 0x0e, 0xf8, 0xcf, 0x79, 0xd3, 0x32, 0xf0, 0xcb, 0xf0, 0x54, 0xea, 0x0c,
0x94, 0x3f, 0x19, 0x8d, 0xbd, 0x43, 0xf5, 0x1f, 0x14, 0x68, 0x3c, 0x1c, 0x4e, 0xdc, 0x5d, 0x0d,
0xbf, 0x98, 0x60, 0xd7, 0x43, 0x77, 0xa1, 0xb4, 0xad, 0xbb, 0xb8, 0xad, 0x5c, 0x51, 0x16, 0xeb,
0xcb, 0x17, 0x96, 0x22, 0x34, 0x71, 0x6a, 0xd6, 0xdd, 0xc1, 0x8a, 0xee, 0x62, 0x8d, 0x42, 0x22,
0x04, 0x25, 0x63, 0xbb, 0xbb, 0xd6, 0x2e, 0x5c, 0x51, 0x16, 0x8b, 0x1a, 0xfd, 0x8d, 0x2e, 0x01,
0xb8, 0x78, 0x30, 0xc2, 0x96, 0xd7, 0x5d, 0x73, 0xdb, 0xc5, 0x2b, 0xc5, 0xc5, 0xa2, 0x16, 0xea,
0x41, 0x2a, 0x34, 0xfa, 0xf6, 0x70, 0x88, 0xfb, 0x9e, 0x69, 0x5b, 0xdd, 0xb5, 0x76, 0x89, 0x8e,
0x8d, 0xf4, 0xa1, 0x0e, 0x54, 0x4d, 0xb7, 0x3b, 0x1a, 0xdb, 0x8e, 0xd7, 0x2e, 0x5f, 0x51, 0x16,
0xab, 0x9a, 0x68, 0xab, 0xff, 0xad, 0xc0, 0x2c, 0x27, 0xdb, 0x1d, 0xdb, 0x96, 0x8b, 0xd1, 0x3d,
0xa8, 0xb8, 0x9e, 0xee, 0x4d, 0x5c, 0x4e, 0xf9, 0x79, 0x29, 0xe5, 0x9b, 0x14, 0x44, 0xe3, 0xa0,
0x52, 0xd2, 0xe3, 0xa4, 0x15, 0x25, 0xa4, 0x45, 0x97, 0x57, 0x4a, 0x2c, 0x6f, 0x11, 0xe6, 0x76,
0x08, 0x75, 0x9b, 0x01, 0x50, 0x99, 0x02, 0xc5, 0xbb, 0x09, 0x26, 0xcf, 0x1c, 0xe1, 0xef, 0xed,
0x6c, 0x62, 0x7d, 0xd8, 0xae, 0xd0, 0xb9, 0x42, 0x3d, 0xea, 0xbf, 0x2a, 0xd0, 0x12, 0xe0, 0xfe,
0x1e, 0x9d, 0x81, 0x72, 0xdf, 0x9e, 0x58, 0x1e, 0x5d, 0xea, 0xac, 0xc6, 0x1a, 0xe8, 0x6d, 0x68,
0xf4, 0x77, 0x75, 0xcb, 0xc2, 0xc3, 0x9e, 0xa5, 0x8f, 0x30, 0x5d, 0x54, 0x4d, 0xab, 0xf3, 0xbe,
0x27, 0xfa, 0x08, 0xe7, 0x5a, 0xdb, 0x15, 0xa8, 0x8f, 0x75, 0xc7, 0x33, 0x23, 0x3b, 0x13, 0xee,
0xca, 0xda, 0x18, 0x32, 0x83, 0x49, 0x7f, 0x6d, 0xe9, 0xee, 0x5e, 0x77, 0x8d, 0xaf, 0x28, 0xd2,
0xa7, 0xfe, 0x48, 0x81, 0x85, 0x07, 0xae, 0x6b, 0x0e, 0xac, 0xc4, 0xca, 0x16, 0xa0, 0x62, 0xd9,
0x06, 0xee, 0xae, 0xd1, 0xa5, 0x15, 0x35, 0xde, 0x42, 0xe7, 0xa1, 0x36, 0xc6, 0xd8, 0xe9, 0x39,
0xf6, 0xd0, 0x5f, 0x58, 0x95, 0x74, 0x68, 0xf6, 0x10, 0xa3, 0xef, 0xc3, 0xbc, 0x1b, 0x43, 0xc4,
0x64, 0xae, 0xbe, 0x7c, 0x75, 0x29, 0xa1, 0x53, 0x4b, 0xf1, 0x49, 0xb5, 0xe4, 0x68, 0xf5, 0x8b,
0x02, 0x9c, 0x16, 0x70, 0x8c, 0x56, 0xf2, 0x9b, 0x70, 0xde, 0xc5, 0x03, 0x41, 0x1e, 0x6b, 0xe4,
0xe1, 0xbc, 0xd8, 0xb2, 0x62, 0x78, 0xcb, 0xf2, 0xa8, 0x41, 0x6c, 0x3f, 0xca, 0xc9, 0xfd, 0xb8,
0x0c, 0x75, 0xfc, 0x72, 0x6c, 0x3a, 0xb8, 0x47, 0x04, 0x87, 0xb2, 0xbc, 0xa4, 0x01, 0xeb, 0xda,
0x32, 0x47, 0x61, 0xdd, 0x98, 0xc9, 0xad, 0x1b, 0xea, 0x5f, 0x2b, 0x70, 0x36, 0xb1, 0x4b, 0x5c,
0xd9, 0x34, 0x68, 0xd1, 0x95, 0x07, 0x9c, 0x21, 0x6a, 0x47, 0x18, 0xfe, 0x4e, 0x16, 0xc3, 0x03,
0x70, 0x2d, 0x31, 0x3e, 0x44, 0x64, 0x21, 0x3f, 0x91, 0x7b, 0x70, 0xf6, 0x11, 0xf6, 0xf8, 0x04,
0xe4, 0x1b, 0x76, 0x8f, 0x6f, 0xc8, 0xa2, 0x5a, 0x5d, 0x88, 0x6b, 0xb5, 0xfa, 0x37, 0x05, 0xa1,
0x8b, 0x74, 0xaa, 0xae, 0xb5, 0x63, 0xa3, 0x0b, 0x50, 0x13, 0x20, 0x5c, 0x2a, 0x82, 0x0e, 0xf4,
0x4d, 0x28, 0x13, 0x4a, 0x99, 0x48, 0x34, 0x97, 0xdf, 0x96, 0xaf, 0x29, 0x84, 0x53, 0x63, 0xf0,
0x68, 0x0d, 0x9a, 0xae, 0xa7, 0x3b, 0x5e, 0x6f, 0x6c, 0xbb, 0x74, 0x9f, 0xa9, 0xe0, 0xd4, 0x97,
0x2f, 0x46, 0x31, 0x10, 0x23, 0xbf, 0xee, 0x0e, 0x36, 0x38, 0x90, 0x36, 0x4b, 0x07, 0xf9, 0x4d,
0xf4, 0x5d, 0x68, 0x60, 0xcb, 0x08, 0x70, 0x94, 0xf2, 0xe0, 0xa8, 0x63, 0xcb, 0x10, 0x18, 0x82,
0x5d, 0x29, 0xe7, 0xdf, 0x95, 0x3f, 0x54, 0xa0, 0x9d, 0xdc, 0x96, 0x93, 0x18, 0xea, 0xfb, 0x6c,
0x10, 0x66, 0xdb, 0x92, 0xa9, 0xd7, 0x62, 0x6b, 0x34, 0x3e, 0x44, 0xfd, 0x73, 0x05, 0xde, 0x0a,
0xc8, 0xa1, 0x9f, 0x5e, 0x97, 0x8c, 0xa0, 0x9b, 0xd0, 0x32, 0xad, 0xfe, 0x70, 0x62, 0xe0, 0xa7,
0xd6, 0xa7, 0x58, 0x1f, 0x7a, 0xbb, 0x87, 0x74, 0xe7, 0xaa, 0x5a, 0xa2, 0x5f, 0xfd, 0xcf, 0x02,
0x2c, 0xc4, 0xe9, 0x3a, 0x09, 0x93, 0x7e, 0x01, 0xca, 0xa6, 0xb5, 0x63, 0xfb, 0x3c, 0xba, 0x94,
0xa1, 0x8a, 0x64, 0x2e, 0x06, 0x8c, 0x6c, 0x40, 0xbe, 0xf1, 0xea, 0xef, 0xe2, 0xfe, 0xde, 0xd8,
0x36, 0xa9, 0x99, 0x22, 0x28, 0xbe, 0x2b, 0x41, 0x21, 0xa7, 0x78, 0x69, 0x95, 0xe1, 0x58, 0x15,
0x28, 0x3e, 0xb1, 0x3c, 0xe7, 0x50, 0x9b, 0xef, 0xc7, 0xfb, 0x3b, 0x7d, 0x58, 0x90, 0x03, 0xa3,
0x16, 0x14, 0xf7, 0xf0, 0x21, 0x5d, 0x72, 0x4d, 0x23, 0x3f, 0xd1, 0x3d, 0x28, 0xef, 0xeb, 0xc3,
0x09, 0xe6, 0x36, 0x61, 0x8a, 0xe4, 0x32, 0xd8, 0x0f, 0x0b, 0x1f, 0x28, 0xea, 0x08, 0xce, 0x3f,
0xc2, 0x5e, 0xd7, 0x72, 0xb1, 0xe3, 0xad, 0x98, 0xd6, 0xd0, 0x1e, 0x6c, 0xe8, 0xde, 0xee, 0x09,
0x8c, 0x43, 0x44, 0xcf, 0x0b, 0x31, 0x3d, 0x57, 0x7f, 0xac, 0xc0, 0x05, 0xf9, 0x7c, 0x7c, 0x43,
0x3b, 0x50, 0xdd, 0x31, 0xf1, 0xd0, 0x20, 0x52, 0xa3, 0x50, 0xa9, 0x11, 0x6d, 0x62, 0x24, 0xc6,
0x04, 0x98, 0xef, 0x5b, 0xcc, 0x48, 0x88, 0x98, 0x6f, 0xd3, 0x73, 0x4c, 0x6b, 0xf0, 0xd8, 0x74,
0x3d, 0x8d, 0xc1, 0x87, 0xa4, 0xa4, 0x98, 0x5f, 0x39, 0x7f, 0x5f, 0x81, 0x4b, 0x8f, 0xb0, 0xb7,
0x2a, 0x7c, 0x0c, 0xf9, 0x6e, 0xba, 0x9e, 0xd9, 0x77, 0x5f, 0x6d, 0x0c, 0x98, 0x23, 0xd8, 0x50,
0xff, 0x58, 0x81, 0xcb, 0xa9, 0xc4, 0x70, 0xd6, 0x71, 0x1b, 0xea, 0x7b, 0x18, 0xb9, 0x0d, 0xfd,
0x25, 0x7c, 0xf8, 0x8c, 0x6c, 0xfe, 0x86, 0x6e, 0x3a, 0xcc, 0x86, 0x1e, 0xd3, 0xa3, 0xfc, 0x44,
0x81, 0x8b, 0x8f, 0xb0, 0xb7, 0xe1, 0xfb, 0xd7, 0x37, 0xc8, 0x1d, 0x02, 0x13, 0xf2, 0xf3, 0x7e,
0xa0, 0x19, 0xe9, 0x53, 0xff, 0x88, 0x6d, 0xa7, 0x94, 0xde, 0x37, 0xc2, 0xc0, 0x4b, 0x54, 0x13,
0x42, 0x26, 0x82, 0x2b, 0x3b, 0x67, 0x9f, 0xfa, 0xdb, 0x65, 0x68, 0x3c, 0xe3, 0x56, 0x81, 0x7a,
0xd0, 0x38, 0x27, 0x14, 0x79, 0x10, 0x14, 0x8a, 0xa6, 0x64, 0x01, 0xd6, 0x0a, 0xcc, 0xba, 0x18,
0xef, 0x1d, 0xd1, 0x5f, 0x36, 0xc8, 0x18, 0xe1, 0xec, 0x1e, 0xc3, 0xfc, 0xc4, 0xa2, 0x11, 0x3a,
0x36, 0xf8, 0x02, 0x18, 0xd3, 0xa7, 0x1b, 0xd3, 0xe4, 0x40, 0xf4, 0x29, 0x4f, 0x02, 0x42, 0xb8,
0xca, 0xb9, 0x70, 0xc5, 0x87, 0xa1, 0x2e, 0xb4, 0x0c, 0xc7, 0x1e, 0x8f, 0xb1, 0xd1, 0x73, 0x7d,
0x54, 0x95, 0x7c, 0xa8, 0xf8, 0x38, 0x81, 0xea, 0x2e, 0x9c, 0x8e, 0x53, 0xda, 0x35, 0x48, 0x5c,
0x48, 0x24, 0x4b, 0xf6, 0x09, 0xdd, 0x82, 0xf9, 0x24, 0x7c, 0x95, 0xc2, 0x27, 0x3f, 0xa0, 0xdb,
0x80, 0x62, 0xa4, 0x12, 0xf0, 0x1a, 0x03, 0x8f, 0x12, 0xc3, 0xc1, 0x69, 0x72, 0x1a, 0x05, 0x07,
0x06, 0xce, 0xbf, 0x84, 0xc0, 0xbb, 0xc4, 0xbb, 0x46, 0xc0, 0xdd, 0x76, 0x3d, 0x1f, 0x23, 0xa2,
0xc8, 0x5c, 0xf5, 0xf7, 0x14, 0x58, 0x78, 0xae, 0x7b, 0xfd, 0xdd, 0xb5, 0x11, 0x17, 0xd0, 0x13,
0x28, 0xf8, 0x47, 0x50, 0xdb, 0xe7, 0xc2, 0xe8, 0x5b, 0xf1, 0xcb, 0x12, 0x82, 0xc2, 0x62, 0xaf,
0x05, 0x23, 0x48, 0x42, 0x74, 0xe6, 0x61, 0x28, 0x31, 0x7c, 0x03, 0xa6, 0x66, 0x4a, 0x46, 0xab,
0xbe, 0x04, 0xe0, 0xc4, 0xad, 0xbb, 0x83, 0x63, 0xd0, 0xf5, 0x01, 0xcc, 0x70, 0x6c, 0xdc, 0x96,
0x4c, 0xdb, 0x30, 0x1f, 0x5c, 0xfd, 0x51, 0x05, 0xea, 0xa1, 0x0f, 0xa8, 0x09, 0x05, 0x61, 0x24,
0x0a, 0x92, 0xd5, 0x15, 0xa6, 0xe7, 0x50, 0xc5, 0x64, 0x0e, 0x75, 0x1d, 0x9a, 0x26, 0x75, 0xde,
0x3d, 0xbe, 0x2b, 0x34, 0x56, 0xae, 0x69, 0xb3, 0xac, 0x97, 0x8b, 0x08, 0xba, 0x04, 0x75, 0x6b,
0x32, 0xea, 0xd9, 0x3b, 0x3d, 0xc7, 0x3e, 0x70, 0x79, 0x32, 0x56, 0xb3, 0x26, 0xa3, 0xef, 0xed,
0x68, 0xf6, 0x81, 0x1b, 0xc4, 0xfb, 0x95, 0x23, 0xc6, 0xfb, 0x97, 0xa0, 0x3e, 0xd2, 0x5f, 0x12,
0xac, 0x3d, 0x6b, 0x32, 0xa2, 0x79, 0x5a, 0x51, 0xab, 0x8d, 0xf4, 0x97, 0x9a, 0x7d, 0xf0, 0x64,
0x32, 0x42, 0x8b, 0xd0, 0x1a, 0xea, 0xae, 0xd7, 0x0b, 0x27, 0x7a, 0x55, 0x9a, 0xe8, 0x35, 0x49,
0xff, 0x27, 0x41, 0xb2, 0x97, 0xcc, 0x1c, 0x6a, 0xc7, 0xcb, 0x1c, 0x8c, 0xd1, 0x30, 0xc0, 0x01,
0xb9, 0x32, 0x07, 0x63, 0x34, 0x14, 0x18, 0x3e, 0x80, 0x99, 0x6d, 0x1a, 0x08, 0x65, 0xa9, 0xe8,
0x43, 0x12, 0x03, 0xb1, 0x78, 0x49, 0xf3, 0xc1, 0xd1, 0xb7, 0xa1, 0x46, 0xfd, 0x0f, 0x1d, 0xdb,
0xc8, 0x35, 0x36, 0x18, 0x40, 0x46, 0x1b, 0x78, 0xe8, 0xe9, 0x74, 0xf4, 0x6c, 0xbe, 0xd1, 0x62,
0x00, 0xb1, 0x8f, 0x7d, 0x07, 0xeb, 0x1e, 0x36, 0x56, 0x0e, 0x57, 0xed, 0xd1, 0x58, 0xa7, 0x22,
0xd4, 0x6e, 0xd2, 0x10, 0x5e, 0xf6, 0x09, 0xbd, 0x03, 0xcd, 0xbe, 0x68, 0x3d, 0x74, 0xec, 0x51,
0x7b, 0x8e, 0x6a, 0x4f, 0xac, 0x17, 0x5d, 0x04, 0xf0, 0x2d, 0xa3, 0xee, 0xb5, 0x5b, 0x74, 0xef,
0x6a, 0xbc, 0xe7, 0x01, 0xad, 0xde, 0x98, 0x6e, 0x8f, 0xd5, 0x49, 0x4c, 0x6b, 0xd0, 0x9e, 0xa7,
0x33, 0xd6, 0xfd, 0xc2, 0x8a, 0x69, 0x0d, 0xd0, 0x59, 0x98, 0x31, 0xdd, 0xde, 0x8e, 0xbe, 0x87,
0xdb, 0x88, 0x7e, 0xad, 0x98, 0xee, 0x43, 0x7d, 0x0f, 0xab, 0x9f, 0xc3, 0x99, 0x40, 0xa6, 0x42,
0x9b, 0x98, 0x14, 0x05, 0xe5, 0x18, 0xa2, 0x90, 0x1d, 0xf9, 0x7e, 0x55, 0x82, 0x85, 0x4d, 0x7d,
0x1f, 0xbf, 0xfe, 0x20, 0x3b, 0x97, 0x1d, 0x7b, 0x0c, 0xf3, 0x34, 0xae, 0x5e, 0x0e, 0xd1, 0x93,
0xe1, 0xc2, 0xc3, 0x52, 0x90, 0x1c, 0x88, 0xbe, 0x43, 0xc2, 0x0e, 0xdc, 0xdf, 0xdb, 0x20, 0x39,
0x8a, 0xef, 0xbe, 0x2f, 0x4a, 0xf0, 0xac, 0x0a, 0x28, 0x2d, 0x3c, 0x02, 0x6d, 0xc0, 0x5c, 0x74,
0x07, 0x7c, 0xc7, 0x7d, 0x23, 0x33, 0x81, 0x0d, 0xb8, 0xaf, 0x35, 0x23, 0x9b, 0xe1, 0xa2, 0x36,
0xcc, 0x70, 0xaf, 0x4b, 0x8d, 0x44, 0x55, 0xf3, 0x9b, 0x68, 0x03, 0x4e, 0xb3, 0x15, 0x6c, 0x72,
0x5d, 0x60, 0x8b, 0xaf, 0xe6, 0x5a, 0xbc, 0x6c, 0x68, 0x54, 0x95, 0x6a, 0x47, 0x55, 0xa5, 0x36,
0xcc, 0x70, 0xf1, 0xa6, 0xd6, 0xa3, 0xaa, 0xf9, 0x4d, 0xb2, 0xcd, 0x81, 0xa0, 0xd7, 0xe9, 0xb7,
0xa0, 0x43, 0xfd, 0x1d, 0x05, 0x20, 0xe0, 0xe7, 0x94, 0x02, 0xcb, 0xb7, 0xa0, 0x2a, 0x84, 0x3b,
0x57, 0x8e, 0x28, 0xc0, 0xe3, 0xb6, 0xbc, 0x18, 0xb3, 0xe5, 0xea, 0xbf, 0x28, 0xd0, 0x58, 0x23,
0xab, 0x79, 0x6c, 0x0f, 0xa8, 0xe7, 0xb9, 0x0e, 0x4d, 0x07, 0xf7, 0x6d, 0xc7, 0xe8, 0x61, 0xcb,
0x73, 0x4c, 0xcc, 0x92, 0xf3, 0x92, 0x36, 0xcb, 0x7a, 0x3f, 0x61, 0x9d, 0x04, 0x8c, 0x98, 0x67,
0xd7, 0xd3, 0x47, 0xe3, 0xde, 0x0e, 0x31, 0x08, 0x05, 0x06, 0x26, 0x7a, 0xa9, 0x3d, 0x78, 0x1b,
0x1a, 0x01, 0x98, 0x67, 0xd3, 0xf9, 0x4b, 0x5a, 0x5d, 0xf4, 0x6d, 0xd9, 0xe8, 0x1a, 0x34, 0x29,
0x3b, 0x7b, 0x43, 0x7b, 0xd0, 0x23, 0x29, 0x1f, 0x77, 0x4a, 0x0d, 0x83, 0x93, 0x45, 0xb6, 0x29,
0x0a, 0xe5, 0x9a, 0x9f, 0x63, 0xee, 0x96, 0x04, 0xd4, 0xa6, 0xf9, 0x39, 0x56, 0x7f, 0x4b, 0x81,
0x59, 0xee, 0xc5, 0x36, 0x45, 0xf1, 0x9b, 0x56, 0x2b, 0x59, 0xba, 0x4d, 0x7f, 0xa3, 0x0f, 0xa3,
0xf5, 0xaa, 0x6b, 0x52, 0x51, 0xa7, 0x48, 0x68, 0xec, 0x14, 0x71, 0x61, 0x79, 0xf2, 0xbd, 0x2f,
0x08, 0x4f, 0x75, 0x4f, 0x7f, 0x62, 0x1b, 0xac, 0x7c, 0xd6, 0x86, 0x19, 0xdd, 0x30, 0x1c, 0xec,
0xba, 0x9c, 0x0e, 0xbf, 0x49, 0xbe, 0xec, 0x63, 0xc7, 0xf5, 0x37, 0xb6, 0xa8, 0xf9, 0x4d, 0xf4,
0x6d, 0xa8, 0x8a, 0x60, 0x8b, 0xd5, 0x29, 0xae, 0xa4, 0xd3, 0xc9, 0xb3, 0x13, 0x31, 0x42, 0xfd,
0xc7, 0x02, 0x34, 0xb9, 0xa6, 0xad, 0x70, 0x87, 0x93, 0x2d, 0x62, 0x2b, 0xd0, 0xd8, 0x09, 0x24,
0x3c, 0xab, 0xba, 0x12, 0x56, 0x84, 0xc8, 0x98, 0x69, 0xb2, 0x16, 0x75, 0x79, 0xa5, 0x13, 0xb9,
0xbc, 0xf2, 0x51, 0xf5, 0x34, 0x19, 0xfa, 0x54, 0x24, 0xa1, 0x8f, 0xfa, 0x2b, 0x50, 0x0f, 0x21,
0xa0, 0x76, 0x88, 0x15, 0x30, 0x38, 0xc7, 0xfc, 0x26, 0xba, 0x17, 0x38, 0x7e, 0xc6, 0xaa, 0x73,
0x12, 0x5a, 0x62, 0x3e, 0x5f, 0xfd, 0x99, 0x02, 0x15, 0x8e, 0xf9, 0x32, 0xd4, 0xb9, 0x7e, 0xd1,
0x50, 0x88, 0x61, 0x07, 0xde, 0x45, 0x62, 0xa1, 0x57, 0xa7, 0x60, 0xe7, 0xa0, 0x1a, 0x53, 0xad,
0x19, 0x6e, 0xfc, 0xfc, 0x4f, 0x21, 0x7d, 0x22, 0x9f, 0x88, 0x2a, 0xa1, 0x33, 0x50, 0x1e, 0xda,
0x03, 0x71, 0xb8, 0xc1, 0x1a, 0xea, 0x97, 0x0a, 0xad, 0x45, 0x6b, 0xb8, 0x6f, 0xef, 0x63, 0xe7,
0xf0, 0xe4, 0xe5, 0xbc, 0xfb, 0x21, 0x31, 0xcf, 0x99, 0x53, 0x88, 0x01, 0xe8, 0x7e, 0xb0, 0x09,
0x45, 0x59, 0xd6, 0x1f, 0x76, 0x38, 0x5c, 0x48, 0x83, 0xcd, 0xf8, 0x13, 0x85, 0x16, 0x26, 0xa3,
0x4b, 0x39, 0xae, 0x4f, 0x7f, 0x25, 0xf1, 0xb9, 0xfa, 0xcf, 0x0a, 0x9c, 0x4b, 0xe1, 0xee, 0xb3,
0xe5, 0x37, 0xc0, 0xdf, 0x0f, 0xa1, 0x2a, 0x32, 0xd0, 0x62, 0xae, 0x0c, 0x54, 0xc0, 0xab, 0x7f,
0xc6, 0xca, 0xe3, 0x12, 0xf6, 0x3e, 0x5b, 0x7e, 0x4d, 0x0c, 0x8e, 0x57, 0x92, 0x8a, 0x92, 0x4a,
0xd2, 0x57, 0x0a, 0x74, 0x82, 0xca, 0x8d, 0xbb, 0x72, 0x78, 0xd2, 0xf3, 0x94, 0x57, 0x93, 0x99,
0x7d, 0x4b, 0x94, 0xfe, 0x89, 0x5d, 0xcc, 0x95, 0x53, 0xf9, 0x85, 0x7f, 0x8b, 0x16, 0x81, 0x93,
0x0b, 0x3a, 0x89, 0x56, 0x76, 0x42, 0x1b, 0xcf, 0xca, 0xff, 0xc1, 0xc6, 0xfe, 0x8c, 0x09, 0xe9,
0xc3, 0x68, 0xf9, 0xe6, 0x4d, 0x33, 0x30, 0x7c, 0x24, 0xb1, 0xcb, 0x8f, 0x24, 0x4a, 0xb1, 0x23,
0x09, 0xde, 0xaf, 0x8e, 0xa8, 0x08, 0x24, 0x16, 0xf0, 0xba, 0x18, 0xf6, 0xbb, 0x0a, 0xb4, 0xf9,
0x2c, 0x74, 0x4e, 0x92, 0x56, 0x0d, 0xb1, 0x87, 0x8d, 0xaf, 0xbb, 0xc8, 0xf0, 0x97, 0x05, 0x68,
0x85, 0x03, 0x1b, 0x1a, 0x9b, 0x7c, 0x03, 0xca, 0xb4, 0x46, 0xc3, 0x29, 0x98, 0x6a, 0x1d, 0x18,
0x34, 0xf1, 0x8c, 0x34, 0x66, 0xdf, 0x72, 0xfd, 0xc0, 0x85, 0x37, 0x83, 0xe8, 0xaa, 0x78, 0xf4,
0xe8, 0xea, 0x02, 0xd4, 0x88, 0xe7, 0xb2, 0x27, 0x04, 0x2f, 0x3b, 0x27, 0x0e, 0x3a, 0xd0, 0x47,
0x50, 0x61, 0xb7, 0x3f, 0xf8, 0x31, 0xdd, 0xf5, 0x28, 0x6a, 0x7e, 0x33, 0x24, 0x54, 0x66, 0xa7,
0x1d, 0x1a, 0x1f, 0x44, 0xf6, 0x68, 0xec, 0xd8, 0x03, 0x1a, 0x86, 0x11, 0xa7, 0x56, 0xd6, 0x44,
0x5b, 0xfd, 0x45, 0x58, 0x08, 0xb2, 0x5d, 0x46, 0xd2, 0x71, 0x05, 0x5a, 0xfd, 0x0f, 0x05, 0x4e,
0x6f, 0x1e, 0x5a, 0xfd, 0xb8, 0x6a, 0x2c, 0x40, 0x65, 0x3c, 0xd4, 0x83, 0xe2, 0x2f, 0x6f, 0xd1,
0x83, 0x75, 0x36, 0x37, 0x36, 0x88, 0x0b, 0x67, 0xfc, 0xac, 0x8b, 0xbe, 0x2d, 0x7b, 0x6a, 0x64,
0x75, 0x5d, 0xa4, 0xe7, 0xd8, 0x60, 0xc1, 0x02, 0x2b, 0x6e, 0xcd, 0x8a, 0x5e, 0x1a, 0x2c, 0x7c,
0x04, 0x40, 0xe3, 0xa9, 0xde, 0x51, 0x62, 0x28, 0x3a, 0xe2, 0x31, 0xf1, 0x98, 0x3f, 0x2d, 0x40,
0x3b, 0xc4, 0xa5, 0xaf, 0x3b, 0xbc, 0x4c, 0x49, 0xfd, 0x8a, 0xaf, 0x28, 0xf5, 0x2b, 0x9d, 0x3c,
0xa4, 0x2c, 0xcb, 0x42, 0xca, 0xdf, 0x28, 0x42, 0x33, 0xe0, 0xda, 0xc6, 0x50, 0xb7, 0x52, 0x25,
0x61, 0x13, 0x9a, 0x6e, 0x84, 0xab, 0x9c, 0x4f, 0xef, 0xc9, 0x74, 0x28, 0x65, 0x23, 0xb4, 0x18,
0x0a, 0x74, 0x91, 0x6e, 0xba, 0xe3, 0xb1, 0x72, 0x1a, 0x8b, 0x0f, 0x6b, 0x4c, 0x59, 0xcd, 0x11,
0x46, 0xb7, 0x00, 0x71, 0x0d, 0xeb, 0x99, 0x56, 0xcf, 0xc5, 0x7d, 0xdb, 0x32, 0x98, 0xee, 0x95,
0xb5, 0x16, 0xff, 0xd2, 0xb5, 0x36, 0x59, 0x3f, 0xfa, 0x06, 0x94, 0xbc, 0xc3, 0x31, 0x0b, 0x16,
0x9b, 0xd2, 0x70, 0x2b, 0xa0, 0x6b, 0xeb, 0x70, 0x8c, 0x35, 0x0a, 0xee, 0x5f, 0x00, 0xf2, 0x1c,
0x7d, 0x9f, 0x47, 0xde, 0x25, 0x2d, 0xd4, 0x43, 0xac, 0x89, 0xcf, 0xc3, 0x19, 0x16, 0xa1, 0xf2,
0x26, 0x93, 0x6c, 0x5f, 0xa1, 0x7b, 0x9e, 0x37, 0xa4, 0x05, 0x41, 0x2a, 0xd9, 0x7e, 0xef, 0x96,
0x37, 0x24, 0x8b, 0xf4, 0x6c, 0x4f, 0x1f, 0x32, 0xfd, 0xa8, 0x71, 0xcb, 0x41, 0x7a, 0x68, 0x96,
0xfb, 0xef, 0xc4, 0xf2, 0x09, 0xc2, 0x34, 0xec, 0x4e, 0x86, 0xe9, 0xfa, 0x98, 0x5d, 0x9f, 0x99,
0xa6, 0x8a, 0xdf, 0x81, 0x3a, 0x97, 0x8a, 0x23, 0x48, 0x15, 0xb0, 0x21, 0x8f, 0x33, 0xc4, 0xbc,
0xfc, 0x8a, 0xc4, 0xbc, 0x72, 0x8c, 0x0a, 0x87, 0x7c, 0x6f, 0xd4, 0x1f, 0x2b, 0xf0, 0x56, 0xc2,
0x6a, 0x66, 0xb2, 0x36, 0x3b, 0xf3, 0xe6, 0xd6, 0x34, 0x8e, 0x92, 0xfb, 0x86, 0xfb, 0x50, 0x71,
0x28, 0x76, 0x7e, 0xe8, 0x75, 0x35, 0x53, 0xf8, 0x18, 0x21, 0x1a, 0x1f, 0xa2, 0xfe, 0xa9, 0x02,
0x67, 0x93, 0xa4, 0x9e, 0xc0, 0xe1, 0xaf, 0xc0, 0x0c, 0x43, 0xed, 0xeb, 0xe8, 0x62, 0xb6, 0x8e,
0x06, 0xcc, 0xd1, 0xfc, 0x81, 0xea, 0x26, 0x2c, 0xf8, 0x71, 0x41, 0xc0, 0xfa, 0x75, 0xec, 0xe9,
0x19, 0x79, 0xe7, 0x65, 0xa8, 0xb3, 0x04, 0x86, 0xe5, 0x73, 0xec, 0x8c, 0x10, 0xb6, 0x45, 0x39,
0x4f, 0xfd, 0x1f, 0x05, 0xce, 0x50, 0xc7, 0x1a, 0x3f, 0xf0, 0xc9, 0x73, 0x02, 0xa9, 0x8a, 0x3b,
0x5e, 0x4f, 0xf4, 0x11, 0xbf, 0x87, 0x52, 0xd3, 0x22, 0x7d, 0xa8, 0x9b, 0xac, 0xf6, 0x49, 0xeb,
0x13, 0xc1, 0x91, 0xeb, 0x9a, 0xee, 0xe9, 0xf4, 0xc4, 0x35, 0x5e, 0xe6, 0x0b, 0x1c, 0x7a, 0xe9,
0x18, 0x0e, 0x5d, 0x7d, 0x0c, 0x6f, 0xc5, 0x56, 0x7a, 0x82, 0x1d, 0x55, 0xff, 0x56, 0x21, 0xdb,
0x11, 0xb9, 0xcf, 0x73, 0xfc, 0xa0, 0xf6, 0xa2, 0x38, 0x69, 0xea, 0x99, 0x46, 0xdc, 0x88, 0x18,
0xe8, 0x63, 0xa8, 0x59, 0xf8, 0xa0, 0x17, 0x8e, 0x93, 0x72, 0x44, 0xfc, 0x55, 0x0b, 0x1f, 0xd0,
0x5f, 0xea, 0x13, 0x38, 0x9b, 0x20, 0xf5, 0x24, 0x6b, 0xff, 0x27, 0x05, 0xce, 0xad, 0x39, 0xf6,
0xf8, 0x99, 0xe9, 0x78, 0x13, 0x7d, 0x18, 0x3d, 0xcc, 0x3e, 0xc6, 0xf2, 0x73, 0xdc, 0x15, 0xfc,
0x34, 0x91, 0x5b, 0xde, 0x92, 0x68, 0x50, 0x92, 0x28, 0xbe, 0xe8, 0x50, 0x7c, 0xfd, 0x5f, 0x45,
0x19, 0xf1, 0x1c, 0x6e, 0x4a, 0x5c, 0x92, 0x27, 0xf9, 0x90, 0x56, 0xdb, 0x8b, 0xc7, 0xad, 0xb6,
0xa7, 0x98, 0xf7, 0xd2, 0x2b, 0x32, 0xef, 0x47, 0x2e, 0x8c, 0xad, 0x42, 0xf4, 0x24, 0x84, 0x7a,
0xe7, 0xa3, 0x9e, 0x9e, 0x7c, 0x04, 0x10, 0x1c, 0x08, 0xf0, 0xfb, 0x97, 0x53, 0x30, 0x84, 0x06,
0x90, 0x3d, 0x12, 0x0e, 0x94, 0xfb, 0xf7, 0x50, 0x89, 0xfa, 0xfb, 0xd0, 0x91, 0xc9, 0xe6, 0x49,
0xe4, 0xfd, 0xa7, 0x05, 0x80, 0xae, 0xb8, 0xad, 0x7b, 0x3c, 0x0f, 0x70, 0x15, 0x42, 0x31, 0x48,
0xa0, 0xe5, 0x61, 0xd9, 0x31, 0x88, 0x22, 0x88, 0x2c, 0x95, 0xc0, 0x24, 0x32, 0x57, 0x83, 0xe2,
0x09, 0xe9, 0x0a, 0x13, 0x85, 0xb8, 0xd1, 0x3d, 0x0f, 0x35, 0xc7, 0x3e, 0xe8, 0x11, 0xe5, 0x32,
0xfc, 0xeb, 0xc8, 0x8e, 0x7d, 0x40, 0x54, 0xce, 0x40, 0x67, 0x61, 0xc6, 0xd3, 0xdd, 0x3d, 0x82,
0x9f, 0x15, 0xeb, 0x2a, 0xa4, 0xd9, 0x35, 0xd0, 0x19, 0x28, 0xef, 0x98, 0x43, 0xcc, 0x6e, 0x3e,
0xd4, 0x34, 0xd6, 0x40, 0xdf, 0xf4, 0x6f, 0xd0, 0x55, 0x73, 0xdf, 0x94, 0xa1, 0xf0, 0xea, 0x97,
0x0a, 0xcc, 0x05, 0x5c, 0xa3, 0x66, 0x87, 0x58, 0x32, 0x6a, 0xc5, 0x56, 0x6d, 0x83, 0x19, 0x88,
0x66, 0x8a, 0x1f, 0x60, 0x03, 0x99, 0xad, 0x0a, 0x86, 0x64, 0x25, 0xce, 0x64, 0x5d, 0x64, 0xd1,
0xa6, 0xe1, 0x97, 0x72, 0x2a, 0x8e, 0x7d, 0xd0, 0x35, 0x04, 0x37, 0xd8, 0x5d, 0x63, 0x96, 0x26,
0x12, 0x6e, 0xac, 0xd2, 0xeb, 0xc6, 0x57, 0x61, 0x16, 0x3b, 0x8e, 0xed, 0xf4, 0x46, 0xd8, 0x75,
0xf5, 0x01, 0xe6, 0x51, 0x79, 0x83, 0x76, 0xae, 0xb3, 0x3e, 0xf5, 0x2f, 0x4a, 0xd0, 0x0c, 0x96,
0xe2, 0x1f, 0xb9, 0x9b, 0x86, 0x7f, 0xe4, 0x6e, 0x92, 0xad, 0x03, 0x87, 0x19, 0x40, 0xb1, 0xb9,
0x2b, 0x85, 0xb6, 0xa2, 0xd5, 0x78, 0x6f, 0xd7, 0x20, 0xce, 0x98, 0xa8, 0x96, 0x65, 0x1b, 0x38,
0xd8, 0x5c, 0xf0, 0xbb, 0xf8, 0xde, 0x46, 0x64, 0xa4, 0x94, 0x43, 0x46, 0xca, 0x39, 0x64, 0xa4,
0x22, 0x91, 0x91, 0x05, 0xa8, 0x6c, 0x4f, 0xfa, 0x7b, 0xd8, 0xe3, 0x71, 0x1a, 0x6f, 0x45, 0x65,
0xa7, 0x1a, 0x93, 0x1d, 0x21, 0x22, 0xb5, 0xb0, 0x88, 0x9c, 0x87, 0x1a, 0x3b, 0x05, 0xee, 0x79,
0x2e, 0x3d, 0xd7, 0x2a, 0x6a, 0x55, 0xd6, 0xb1, 0xe5, 0xa2, 0x0f, 0xfc, 0x20, 0xae, 0x4e, 0x95,
0x45, 0x95, 0xd8, 0x9a, 0x98, 0x94, 0xf8, 0x21, 0xdc, 0x0d, 0x98, 0x0b, 0xb1, 0x83, 0x7a, 0x86,
0x06, 0x25, 0x35, 0x14, 0xe3, 0x53, 0xe7, 0x70, 0x1d, 0x9a, 0x01, 0x4b, 0x28, 0xdc, 0x2c, 0x4b,
0xad, 0x44, 0x2f, 0x05, 0x13, 0x92, 0xdc, 0x3c, 0x9a, 0x24, 0xa3, 0x73, 0x50, 0xe5, 0x39, 0x91,
0xdb, 0x9e, 0x8b, 0x94, 0x2f, 0xd4, 0x1f, 0x02, 0x0a, 0xa8, 0x3f, 0x59, 0x8c, 0x18, 0x13, 0x8f,
0x42, 0x5c, 0x3c, 0xd4, 0xbf, 0x53, 0x60, 0x3e, 0x3c, 0xd9, 0x71, 0xdd, 0xed, 0xc7, 0x50, 0x67,
0x27, 0x8b, 0x3d, 0xa2, 0xf8, 0xf2, 0x23, 0xc2, 0xd8, 0xbe, 0x68, 0x10, 0xbc, 0x56, 0x20, 0xe2,
0x75, 0x60, 0x3b, 0x7b, 0xa6, 0x35, 0xe8, 0x11, 0xca, 0x44, 0xe5, 0x94, 0x77, 0x3e, 0x21, 0x7d,
0xea, 0x1f, 0x28, 0x70, 0xe9, 0xe9, 0xd8, 0xd0, 0x3d, 0x1c, 0x8a, 0x3b, 0x4e, 0x7a, 0x69, 0x50,
0xdc, 0xda, 0x2b, 0x64, 0xec, 0x60, 0x68, 0x3e, 0x97, 0xdf, 0xda, 0x23, 0xd1, 0x1a, 0xa7, 0x26,
0x71, 0xcd, 0xf6, 0xf8, 0xd4, 0x74, 0xa0, 0xba, 0xcf, 0xd1, 0xf9, 0xef, 0x2f, 0xfc, 0x76, 0xe4,
0x0c, 0xb6, 0x78, 0xa4, 0x33, 0x58, 0x75, 0x1d, 0xce, 0x69, 0xd8, 0xc5, 0x96, 0x11, 0x59, 0xc8,
0xb1, 0xeb, 0x4b, 0x63, 0xe8, 0xc8, 0xd0, 0x9d, 0x44, 0x52, 0x59, 0xb8, 0xda, 0x73, 0x08, 0x5a,
0x8f, 0xdb, 0x61, 0x12, 0x25, 0xd1, 0x79, 0x3c, 0xf5, 0xef, 0x0b, 0x70, 0xf6, 0x81, 0x61, 0x70,
0x13, 0xce, 0x03, 0xb0, 0xd7, 0x15, 0x1b, 0xc7, 0x63, 0xc7, 0x62, 0x32, 0x76, 0x7c, 0x55, 0x66,
0x95, 0x3b, 0x18, 0x6b, 0x32, 0xf2, 0x1d, 0xa7, 0xc3, 0x2e, 0x22, 0xdd, 0xe7, 0x27, 0x95, 0x24,
0x87, 0xa7, 0xce, 0x73, 0x7a, 0x48, 0x55, 0xf5, 0xeb, 0x64, 0xea, 0x18, 0xda, 0x49, 0x66, 0x9d,
0xd0, 0x8e, 0xf8, 0x1c, 0x19, 0xdb, 0xac, 0xde, 0xda, 0x20, 0xf1, 0x13, 0xed, 0xda, 0xb0, 0x5d,
0xf5, 0x7f, 0x0b, 0xd0, 0xde, 0xd4, 0xf7, 0xf1, 0xcf, 0xcf, 0x06, 0x7d, 0x06, 0x67, 0x5c, 0x7d,
0x1f, 0xf7, 0x42, 0xb9, 0x70, 0xcf, 0xc1, 0x2f, 0x78, 0xe8, 0xf9, 0xae, 0xac, 0x22, 0x2e, 0xbd,
0xbe, 0xa3, 0xcd, 0xbb, 0x91, 0x7e, 0x0d, 0xbf, 0x40, 0xef, 0xc0, 0x5c, 0xf8, 0x56, 0x18, 0x21,
0xad, 0x4a, 0x59, 0x3e, 0x1b, 0xba, 0xf9, 0xd5, 0x35, 0xd4, 0x17, 0x70, 0xe1, 0xa9, 0xe5, 0x62,
0xaf, 0x1b, 0xdc, 0x5e, 0x3a, 0x61, 0xd6, 0x78, 0x19, 0xea, 0x01, 0xe3, 0x13, 0x0f, 0x2f, 0x0c,
0x57, 0xb5, 0xa1, 0xb3, 0xae, 0x3b, 0x7b, 0x7e, 0x65, 0x79, 0x8d, 0x5d, 0x35, 0x79, 0x8d, 0x13,
0xee, 0x88, 0x4b, 0x57, 0x1a, 0xde, 0xc1, 0x0e, 0xb6, 0xfa, 0xf8, 0xb1, 0xdd, 0xdf, 0x23, 0xb1,
0x86, 0xc7, 0xde, 0xbe, 0x29, 0xa1, 0x88, 0x73, 0x2d, 0xf4, 0xb4, 0xad, 0x10, 0x79, 0xda, 0x36,
0xe5, 0xa9, 0xa4, 0xfa, 0x93, 0x02, 0x2c, 0x3c, 0x18, 0x7a, 0xd8, 0x09, 0x92, 0xfd, 0xa3, 0xd4,
0x2d, 0x82, 0x42, 0x42, 0xe1, 0x38, 0x27, 0x03, 0x39, 0x0e, 0x0e, 0x65, 0x65, 0x8f, 0xd2, 0x31,
0xcb, 0x1e, 0x0f, 0x00, 0xc6, 0x8e, 0x3d, 0xc6, 0x8e, 0x67, 0x62, 0x3f, 0x63, 0xcb, 0x11, 0xbb,
0x84, 0x06, 0xa9, 0x9f, 0x41, 0xeb, 0x51, 0x7f, 0xd5, 0xb6, 0x76, 0x4c, 0x67, 0xe4, 0x33, 0x2a,
0xa1, 0x74, 0x4a, 0x0e, 0xa5, 0x2b, 0x24, 0x94, 0x4e, 0x35, 0x61, 0x3e, 0x84, 0xfb, 0x84, 0x86,
0x6b, 0xd0, 0xef, 0xed, 0x98, 0x96, 0x49, 0xaf, 0x72, 0x15, 0x68, 0xec, 0x09, 0x83, 0xfe, 0x43,
0xde, 0x73, 0xf3, 0x63, 0x71, 0xeb, 0x75, 0xeb, 0x70, 0x8c, 0xd1, 0x0c, 0x14, 0x9f, 0xe0, 0x83,
0xd6, 0x29, 0x04, 0x50, 0x79, 0x62, 0x3b, 0x23, 0x7d, 0xd8, 0x52, 0x50, 0x1d, 0x66, 0xf8, 0x51,
0x5d, 0xab, 0x80, 0x66, 0xa1, 0xb6, 0xea, 0x1f, 0x69, 0xb4, 0x8a, 0x37, 0xff, 0x4a, 0x81, 0xf9,
0xc4, 0x61, 0x12, 0x6a, 0x02, 0x3c, 0xb5, 0xfa, 0xfc, 0x94, 0xad, 0x75, 0x0a, 0x35, 0xa0, 0xea,
0x9f, 0xb9, 0x31, 0x7c, 0x5b, 0x36, 0x85, 0x6e, 0x15, 0x50, 0x0b, 0x1a, 0x6c, 0xe0, 0xa4, 0xdf,
0xc7, 0xae, 0xdb, 0x2a, 0x8a, 0x9e, 0x87, 0xba, 0x39, 0x9c, 0x38, 0xb8, 0x55, 0x22, 0x73, 0x6e,
0xd9, 0x1a, 0x1e, 0x62, 0xdd, 0xc5, 0xad, 0x32, 0x42, 0xd0, 0xe4, 0x0d, 0x7f, 0x50, 0x25, 0xd4,
0xe7, 0x0f, 0x9b, 0xb9, 0xf9, 0x3c, 0x5c, 0xf6, 0xa7, 0xcb, 0x3b, 0x0b, 0xa7, 0x9f, 0x5a, 0x06,
0xde, 0x31, 0x2d, 0x6c, 0x04, 0x9f, 0x5a, 0xa7, 0xd0, 0x69, 0x98, 0x5b, 0xc7, 0xce, 0x00, 0x87,
0x3a, 0x0b, 0x68, 0x1e, 0x66, 0xd7, 0xcd, 0x97, 0xa1, 0xae, 0xa2, 0x5a, 0xaa, 0x2a, 0x2d, 0x65,
0xf9, 0xdf, 0xae, 0x41, 0x8d, 0xc8, 0xd6, 0xaa, 0x6d, 0x3b, 0x06, 0x1a, 0x02, 0xa2, 0x6f, 0x4b,
0x46, 0x63, 0xdb, 0x12, 0xef, 0xd0, 0xd0, 0x52, 0x2c, 0x36, 0x61, 0x8d, 0x24, 0x20, 0x97, 0x9d,
0xce, 0x35, 0x29, 0x7c, 0x0c, 0x58, 0x3d, 0x85, 0x46, 0x74, 0xb6, 0x2d, 0x73, 0x84, 0xb7, 0xcc,
0xfe, 0x9e, 0x1f, 0x1b, 0xdd, 0x4d, 0x79, 0xcc, 0x93, 0x04, 0xf5, 0xe7, 0xbb, 0x2a, 0x9d, 0x8f,
0x3d, 0xfe, 0xf1, 0x65, 0x4e, 0x3d, 0x85, 0x5e, 0xc0, 0x99, 0x47, 0x38, 0x14, 0x68, 0xfa, 0x13,
0x2e, 0xa7, 0x4f, 0x98, 0x00, 0x3e, 0xe2, 0x94, 0x8f, 0xa1, 0x4c, 0xc5, 0x0d, 0xc9, 0x4e, 0x42,
0xc3, 0x8f, 0xc8, 0x3b, 0x57, 0xd2, 0x01, 0x04, 0xb6, 0x1f, 0xc2, 0x5c, 0xec, 0x79, 0x29, 0x92,
0x39, 0x27, 0xf9, 0x43, 0xe1, 0xce, 0xcd, 0x3c, 0xa0, 0x62, 0xae, 0x01, 0x34, 0xa3, 0x6f, 0x52,
0xd0, 0x62, 0x8e, 0x97, 0x6d, 0x6c, 0xa6, 0x77, 0x73, 0xbf, 0x81, 0xa3, 0x42, 0xd0, 0x8a, 0x3f,
0x7c, 0x44, 0x37, 0x33, 0x11, 0x44, 0x85, 0xed, 0xbd, 0x5c, 0xb0, 0x62, 0xba, 0x43, 0x2a, 0x04,
0x89, 0x57, 0x67, 0x71, 0x19, 0xf7, 0xd1, 0xa4, 0x3d, 0x87, 0xeb, 0xdc, 0xc9, 0x0d, 0x2f, 0xa6,
0xfe, 0x4d, 0x76, 0xdd, 0x49, 0xf6, 0x72, 0x0b, 0xbd, 0x2f, 0x47, 0x97, 0xf1, 0xe4, 0xac, 0xb3,
0x7c, 0x94, 0x21, 0x82, 0x88, 0x5f, 0xa7, 0xf7, 0x94, 0x24, 0x6f, 0x9f, 0xe2, 0x7a, 0xe7, 0xe3,
0x4b, 0x7f, 0xd6, 0xd5, 0x79, 0xff, 0x08, 0x23, 0x04, 0x01, 0x76, 0xfc, 0x65, 0xa9, 0xaf, 0x86,
0x77, 0xa6, 0x4a, 0xcd, 0xf1, 0x74, 0xf0, 0x07, 0x30, 0x17, 0x0b, 0xd7, 0x50, 0xfe, 0x90, 0xae,
0x93, 0xe5, 0x9a, 0x98, 0x4a, 0xc6, 0xee, 0x25, 0xa1, 0x14, 0xe9, 0x97, 0xdc, 0x5d, 0xea, 0xdc,
0xcc, 0x03, 0x2a, 0x16, 0x32, 0x86, 0xf9, 0xd8, 0xc7, 0x67, 0xcb, 0xe8, 0xbd, 0xdc, 0xb3, 0x3d,
0x5b, 0xee, 0xdc, 0xca, 0x3f, 0xdf, 0xb3, 0x65, 0xf5, 0x14, 0x72, 0xa9, 0x81, 0x8e, 0xdd, 0x6d,
0x41, 0x29, 0x58, 0xe4, 0x77, 0x78, 0x3a, 0xb7, 0x73, 0x42, 0x8b, 0x65, 0xee, 0xc3, 0x69, 0xc9,
0x15, 0x24, 0x74, 0x3b, 0x53, 0x3c, 0xe2, 0x77, 0xaf, 0x3a, 0x4b, 0x79, 0xc1, 0x43, 0xee, 0xa1,
0xe5, 0xd3, 0xf5, 0x60, 0x38, 0x64, 0xce, 0xff, 0x56, 0x9a, 0xe7, 0x8b, 0x80, 0xa5, 0x2c, 0x35,
0x15, 0x5a, 0x4c, 0xf9, 0xab, 0x80, 0x36, 0x77, 0xed, 0x03, 0x1a, 0x1e, 0x0d, 0x26, 0x8e, 0xce,
0x22, 0xba, 0x34, 0x07, 0x98, 0x04, 0x4d, 0x51, 0xc4, 0xcc, 0x11, 0x62, 0xf2, 0x1e, 0xc0, 0x23,
0xec, 0xad, 0x63, 0xcf, 0x21, 0xda, 0xff, 0x4e, 0x1a, 0xed, 0x1c, 0xc0, 0x9f, 0xea, 0xc6, 0x54,
0xb8, 0x30, 0x43, 0xd7, 0x75, 0x6b, 0xa2, 0x0f, 0x43, 0x4f, 0x3f, 0xe4, 0x0c, 0x8d, 0x83, 0x65,
0x33, 0x34, 0x09, 0x2d, 0xa6, 0x3c, 0x10, 0xf1, 0x4b, 0xe8, 0xa4, 0x34, 0x3b, 0x7e, 0x49, 0xde,
0xd2, 0x89, 0xdb, 0xf6, 0x0c, 0x78, 0x31, 0xf1, 0x17, 0x0a, 0xbd, 0x38, 0x17, 0x03, 0x78, 0x6e,
0x7a, 0xbb, 0x1b, 0x43, 0xdd, 0x72, 0xf3, 0x90, 0x40, 0x01, 0x8f, 0x40, 0x02, 0x87, 0x17, 0x24,
0x18, 0x30, 0x1b, 0x39, 0xc0, 0x44, 0xb2, 0x07, 0x13, 0xb2, 0xc3, 0xdc, 0xce, 0xe2, 0x74, 0x40,
0x31, 0xcb, 0x2e, 0xcc, 0xfa, 0x02, 0xcd, 0x98, 0xfb, 0x6e, 0xa6, 0xd0, 0x47, 0xf8, 0x7a, 0x33,
0x0f, 0xa8, 0x98, 0xc9, 0x05, 0x94, 0x3c, 0xa9, 0x41, 0xf9, 0xce, 0xf5, 0xb2, 0x8c, 0x4f, 0xfa,
0xf1, 0x0f, 0xb3, 0xe7, 0xb1, 0xb3, 0x50, 0xb9, 0xb3, 0x90, 0x1e, 0xed, 0x4a, 0xed, 0x79, 0xca,
0xd1, 0xaa, 0x7a, 0x0a, 0x3d, 0x87, 0x0a, 0xff, 0x0b, 0x98, 0x6b, 0xd9, 0xd5, 0x55, 0x8e, 0xfd,
0xfa, 0x14, 0x28, 0x81, 0x78, 0x0f, 0xce, 0xa6, 0xd4, 0x56, 0xa5, 0x71, 0x46, 0x76, 0x1d, 0x76,
0x9a, 0x07, 0x14, 0x93, 0x25, 0x4a, 0xa7, 0x19, 0x93, 0xa5, 0x95, 0x59, 0xa7, 0x4d, 0xd6, 0x83,
0xf9, 0x44, 0x69, 0x4a, 0xea, 0x02, 0xd3, 0x0a, 0x58, 0xd3, 0x26, 0x18, 0xc0, 0x5b, 0xd2, 0x32,
0x8c, 0x34, 0x3a, 0xc9, 0x2a, 0xd8, 0x4c, 0x9b, 0xa8, 0x0f, 0xa7, 0x25, 0xc5, 0x17, 0xa9, 0x97,
0x4b, 0x2f, 0xd2, 0x4c, 0x9b, 0x64, 0x07, 0x3a, 0x2b, 0x8e, 0xad, 0x1b, 0x7d, 0xdd, 0xf5, 0x68,
0x41, 0x84, 0xa4, 0x8a, 0x7e, 0x78, 0x28, 0xcf, 0x1d, 0xa4, 0x65, 0x93, 0x69, 0xf3, 0x6c, 0x43,
0x9d, 0x6e, 0x25, 0xfb, 0x9b, 0x0e, 0x24, 0xf7, 0x11, 0x21, 0x88, 0x14, 0xc3, 0x23, 0x03, 0x14,
0x42, 0xbd, 0x05, 0xf5, 0x55, 0x7a, 0x68, 0xd4, 0xb5, 0x0c, 0xfc, 0x32, 0xee, 0xaf, 0xe8, 0x5b,
0xe5, 0xa5, 0x10, 0x40, 0x6e, 0x0e, 0xcd, 0xd2, 0xa8, 0xdd, 0xc0, 0x2f, 0xd9, 0x3e, 0x2f, 0xca,
0xf0, 0x46, 0x40, 0x52, 0xb2, 0x1c, 0x29, 0x64, 0xc8, 0xd3, 0x9f, 0x09, 0xc7, 0xb2, 0x62, 0xba,
0x3b, 0x29, 0x48, 0x12, 0x90, 0xfe, 0xac, 0x77, 0xf3, 0x0f, 0x08, 0x7b, 0x06, 0x9f, 0xae, 0x2e,
0x3d, 0xb1, 0xba, 0x91, 0x45, 0x7a, 0x38, 0x40, 0x5d, 0x9c, 0x0e, 0x28, 0x66, 0xd9, 0x80, 0x1a,
0x91, 0x4e, 0xb6, 0x3d, 0xd7, 0x64, 0x03, 0xc5, 0xe7, 0xfc, 0x9b, 0xb3, 0x86, 0xdd, 0xbe, 0x63,
0x6e, 0xf3, 0x4d, 0x97, 0x92, 0x13, 0x01, 0xc9, 0xdc, 0x9c, 0x18, 0xa4, 0xa0, 0x7c, 0x42, 0xa3,
0x06, 0xc1, 0x3a, 0x6e, 0x2a, 0x6f, 0x4f, 0xdb, 0xdf, 0xa8, 0x99, 0x5c, 0xca, 0x0b, 0x2e, 0xa6,
0xfd, 0x35, 0x9a, 0x09, 0xd1, 0xef, 0x2b, 0x13, 0x73, 0x68, 0x6c, 0xf0, 0xfb, 0xc3, 0xe8, 0x6e,
0x16, 0xaa, 0x08, 0x68, 0x6a, 0x00, 0x98, 0x31, 0x42, 0xcc, 0xff, 0xcb, 0x50, 0x13, 0xa5, 0x39,
0x24, 0xbb, 0x00, 0x17, 0x2f, 0x0a, 0x76, 0xae, 0x65, 0x03, 0xf9, 0x98, 0x97, 0xbf, 0xac, 0x41,
0xd5, 0x7f, 0xb3, 0xf6, 0x35, 0xd7, 0x94, 0xde, 0x40, 0x91, 0xe7, 0x07, 0x30, 0x17, 0xfb, 0x5b,
0x04, 0xa9, 0x69, 0x95, 0xff, 0x75, 0xc2, 0x34, 0x1d, 0x78, 0xce, 0xff, 0xb5, 0x4f, 0x64, 0x5f,
0x37, 0xd2, 0x0a, 0x45, 0xf1, 0xc4, 0x6b, 0x0a, 0xe2, 0xff, 0xdf, 0xb9, 0xc7, 0x13, 0x80, 0x50,
0xd6, 0x91, 0x7d, 0xb5, 0x98, 0x04, 0xd2, 0xd3, 0xb8, 0x35, 0x92, 0x26, 0x16, 0xef, 0xe6, 0xb9,
0xa6, 0x99, 0x1e, 0x1a, 0xa6, 0xa7, 0x13, 0x4f, 0xa1, 0x11, 0xbe, 0xf4, 0x8f, 0xa4, 0xff, 0x11,
0x97, 0x7c, 0x15, 0x30, 0x6d, 0x15, 0xeb, 0x47, 0x8c, 0x38, 0xa7, 0xa0, 0x73, 0x01, 0x25, 0xcf,
0x8e, 0xa5, 0x11, 0x7a, 0xea, 0x89, 0xb5, 0x34, 0x42, 0x4f, 0x3f, 0x90, 0x66, 0xf5, 0xc2, 0xf8,
0x81, 0xa8, 0xb4, 0x5e, 0x98, 0x72, 0xc4, 0x2c, 0xad, 0x17, 0xa6, 0x9d, 0xb0, 0xaa, 0xa7, 0x56,
0xee, 0x7d, 0xf6, 0xfe, 0xc0, 0xf4, 0x76, 0x27, 0xdb, 0x64, 0xf5, 0x77, 0xd8, 0xd0, 0xdb, 0xa6,
0xcd, 0x7f, 0xdd, 0xf1, 0xc5, 0xfd, 0x0e, 0xc5, 0x76, 0x87, 0x60, 0x1b, 0x6f, 0x6f, 0x57, 0x68,
0xeb, 0xde, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xec, 0x97, 0xd4, 0xae, 0xb1, 0x54, 0x00, 0x00,
// 4768 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0xdd, 0x6f, 0x1c, 0x49,
0x5e, 0xe9, 0xf9, 0xf2, 0xcc, 0x6f, 0xc6, 0xe3, 0x71, 0xc5, 0xeb, 0x4c, 0x26, 0x9b, 0x8f, 0xeb,
0x4d, 0x76, 0xbd, 0xd9, 0x5d, 0x27, 0xeb, 0x70, 0xba, 0xbd, 0xcd, 0xed, 0xde, 0xc5, 0xf6, 0x26,
0x3b, 0x10, 0xe7, 0x7c, 0x6d, 0x27, 0x41, 0x7b, 0x48, 0xa3, 0xf6, 0x74, 0x79, 0xdc, 0xe7, 0x99,
0xee, 0x49, 0x77, 0x8f, 0x1d, 0x2f, 0x12, 0x2c, 0x70, 0x20, 0xf1, 0x21, 0x40, 0x08, 0x04, 0xbc,
0x9d, 0x78, 0x40, 0x07, 0xe8, 0x9e, 0x0e, 0x1e, 0xe0, 0xe5, 0x5e, 0x17, 0xf1, 0x70, 0x42, 0x48,
0x48, 0x3c, 0xf0, 0x8a, 0x78, 0xe7, 0x1f, 0x40, 0xf5, 0xd1, 0xd5, 0x5f, 0xd5, 0x3d, 0x6d, 0x4f,
0xb2, 0x91, 0xe0, 0x6d, 0xaa, 0xfa, 0x57, 0xbf, 0xfa, 0xd5, 0xaf, 0x7e, 0xdf, 0x55, 0x35, 0xd0,
0x32, 0x74, 0x4f, 0xef, 0xf5, 0x6d, 0xdb, 0x31, 0x56, 0xc7, 0x8e, 0xed, 0xd9, 0x68, 0x71, 0x64,
0x0e, 0x8f, 0x26, 0x2e, 0x6b, 0xad, 0x92, 0xcf, 0x9d, 0x46, 0xdf, 0x1e, 0x8d, 0x6c, 0x8b, 0x75,
0x75, 0x9a, 0xa6, 0xe5, 0x61, 0xc7, 0xd2, 0x87, 0xbc, 0xdd, 0x08, 0x0f, 0xe8, 0x34, 0xdc, 0xfe,
0x01, 0x1e, 0xe9, 0xbc, 0x55, 0x1b, 0xb9, 0x03, 0xfe, 0x73, 0xd1, 0xb4, 0x0c, 0xfc, 0x3c, 0x3c,
0x95, 0x3a, 0x07, 0xe5, 0x4f, 0x46, 0x63, 0xef, 0x44, 0xfd, 0x7b, 0x05, 0x1a, 0xf7, 0x87, 0x13,
0xf7, 0x40, 0xc3, 0xcf, 0x26, 0xd8, 0xf5, 0xd0, 0x6d, 0x28, 0xed, 0xe9, 0x2e, 0x6e, 0x2b, 0xd7,
0x94, 0x95, 0xfa, 0xda, 0xeb, 0xab, 0x11, 0x9a, 0x38, 0x35, 0x5b, 0xee, 0x60, 0x5d, 0x77, 0xb1,
0x46, 0x21, 0x11, 0x82, 0x92, 0xb1, 0xd7, 0xdd, 0x6c, 0x17, 0xae, 0x29, 0x2b, 0x45, 0x8d, 0xfe,
0x46, 0x57, 0x00, 0x5c, 0x3c, 0x18, 0x61, 0xcb, 0xeb, 0x6e, 0xba, 0xed, 0xe2, 0xb5, 0xe2, 0x4a,
0x51, 0x0b, 0xf5, 0x20, 0x15, 0x1a, 0x7d, 0x7b, 0x38, 0xc4, 0x7d, 0xcf, 0xb4, 0xad, 0xee, 0x66,
0xbb, 0x44, 0xc7, 0x46, 0xfa, 0x50, 0x07, 0xaa, 0xa6, 0xdb, 0x1d, 0x8d, 0x6d, 0xc7, 0x6b, 0x97,
0xaf, 0x29, 0x2b, 0x55, 0x4d, 0xb4, 0xd5, 0xff, 0x52, 0x60, 0x9e, 0x93, 0xed, 0x8e, 0x6d, 0xcb,
0xc5, 0xe8, 0x0e, 0x54, 0x5c, 0x4f, 0xf7, 0x26, 0x2e, 0xa7, 0xfc, 0x92, 0x94, 0xf2, 0x1d, 0x0a,
0xa2, 0x71, 0x50, 0x29, 0xe9, 0x71, 0xd2, 0x8a, 0x12, 0xd2, 0xa2, 0xcb, 0x2b, 0x25, 0x96, 0xb7,
0x02, 0x0b, 0xfb, 0x84, 0xba, 0x9d, 0x00, 0xa8, 0x4c, 0x81, 0xe2, 0xdd, 0x04, 0x93, 0x67, 0x8e,
0xf0, 0x77, 0xf7, 0x77, 0xb0, 0x3e, 0x6c, 0x57, 0xe8, 0x5c, 0xa1, 0x1e, 0xf5, 0x5f, 0x15, 0x68,
0x09, 0x70, 0x7f, 0x8f, 0x96, 0xa0, 0xdc, 0xb7, 0x27, 0x96, 0x47, 0x97, 0x3a, 0xaf, 0xb1, 0x06,
0xfa, 0x1a, 0x34, 0xfa, 0x07, 0xba, 0x65, 0xe1, 0x61, 0xcf, 0xd2, 0x47, 0x98, 0x2e, 0xaa, 0xa6,
0xd5, 0x79, 0xdf, 0x23, 0x7d, 0x84, 0x73, 0xad, 0xed, 0x1a, 0xd4, 0xc7, 0xba, 0xe3, 0x99, 0x91,
0x9d, 0x09, 0x77, 0x65, 0x6d, 0x0c, 0x99, 0xc1, 0xa4, 0xbf, 0x76, 0x75, 0xf7, 0xb0, 0xbb, 0xc9,
0x57, 0x14, 0xe9, 0x53, 0x7f, 0xa4, 0xc0, 0xf2, 0x3d, 0xd7, 0x35, 0x07, 0x56, 0x62, 0x65, 0xcb,
0x50, 0xb1, 0x6c, 0x03, 0x77, 0x37, 0xe9, 0xd2, 0x8a, 0x1a, 0x6f, 0xa1, 0x4b, 0x50, 0x1b, 0x63,
0xec, 0xf4, 0x1c, 0x7b, 0xe8, 0x2f, 0xac, 0x4a, 0x3a, 0x34, 0x7b, 0x88, 0xd1, 0xf7, 0x60, 0xd1,
0x8d, 0x21, 0x62, 0x32, 0x57, 0x5f, 0x7b, 0x63, 0x35, 0xa1, 0x53, 0xab, 0xf1, 0x49, 0xb5, 0xe4,
0x68, 0xf5, 0x8b, 0x02, 0x9c, 0x17, 0x70, 0x8c, 0x56, 0xf2, 0x9b, 0x70, 0xde, 0xc5, 0x03, 0x41,
0x1e, 0x6b, 0xe4, 0xe1, 0xbc, 0xd8, 0xb2, 0x62, 0x78, 0xcb, 0xf2, 0xa8, 0x41, 0x6c, 0x3f, 0xca,
0xc9, 0xfd, 0xb8, 0x0a, 0x75, 0xfc, 0x7c, 0x6c, 0x3a, 0xb8, 0x47, 0x04, 0x87, 0xb2, 0xbc, 0xa4,
0x01, 0xeb, 0xda, 0x35, 0x47, 0x61, 0xdd, 0x98, 0xcb, 0xad, 0x1b, 0xea, 0x5f, 0x29, 0x70, 0x21,
0xb1, 0x4b, 0x5c, 0xd9, 0x34, 0x68, 0xd1, 0x95, 0x07, 0x9c, 0x21, 0x6a, 0x47, 0x18, 0xfe, 0x66,
0x16, 0xc3, 0x03, 0x70, 0x2d, 0x31, 0x3e, 0x44, 0x64, 0x21, 0x3f, 0x91, 0x87, 0x70, 0xe1, 0x01,
0xf6, 0xf8, 0x04, 0xe4, 0x1b, 0x76, 0xcf, 0x6e, 0xc8, 0xa2, 0x5a, 0x5d, 0x88, 0x6b, 0xb5, 0xfa,
0xd7, 0x05, 0xa1, 0x8b, 0x74, 0xaa, 0xae, 0xb5, 0x6f, 0xa3, 0xd7, 0xa1, 0x26, 0x40, 0xb8, 0x54,
0x04, 0x1d, 0xe8, 0x1b, 0x50, 0x26, 0x94, 0x32, 0x91, 0x68, 0xae, 0x7d, 0x4d, 0xbe, 0xa6, 0x10,
0x4e, 0x8d, 0xc1, 0xa3, 0x4d, 0x68, 0xba, 0x9e, 0xee, 0x78, 0xbd, 0xb1, 0xed, 0xd2, 0x7d, 0xa6,
0x82, 0x53, 0x5f, 0xbb, 0x1c, 0xc5, 0x40, 0x8c, 0xfc, 0x96, 0x3b, 0xd8, 0xe6, 0x40, 0xda, 0x3c,
0x1d, 0xe4, 0x37, 0xd1, 0x77, 0xa0, 0x81, 0x2d, 0x23, 0xc0, 0x51, 0xca, 0x83, 0xa3, 0x8e, 0x2d,
0x43, 0x60, 0x08, 0x76, 0xa5, 0x9c, 0x7f, 0x57, 0xfe, 0x40, 0x81, 0x76, 0x72, 0x5b, 0x66, 0x31,
0xd4, 0x77, 0xd9, 0x20, 0xcc, 0xb6, 0x25, 0x53, 0xaf, 0xc5, 0xd6, 0x68, 0x7c, 0x88, 0xfa, 0x67,
0x0a, 0xbc, 0x16, 0x90, 0x43, 0x3f, 0xbd, 0x2c, 0x19, 0x41, 0x37, 0xa1, 0x65, 0x5a, 0xfd, 0xe1,
0xc4, 0xc0, 0x8f, 0xad, 0x4f, 0xb1, 0x3e, 0xf4, 0x0e, 0x4e, 0xe8, 0xce, 0x55, 0xb5, 0x44, 0xbf,
0xfa, 0x1f, 0x05, 0x58, 0x8e, 0xd3, 0x35, 0x0b, 0x93, 0x7e, 0x01, 0xca, 0xa6, 0xb5, 0x6f, 0xfb,
0x3c, 0xba, 0x92, 0xa1, 0x8a, 0x64, 0x2e, 0x06, 0x8c, 0x6c, 0x40, 0xbe, 0xf1, 0xea, 0x1f, 0xe0,
0xfe, 0xe1, 0xd8, 0x36, 0xa9, 0x99, 0x22, 0x28, 0xbe, 0x23, 0x41, 0x21, 0xa7, 0x78, 0x75, 0x83,
0xe1, 0xd8, 0x10, 0x28, 0x3e, 0xb1, 0x3c, 0xe7, 0x44, 0x5b, 0xec, 0xc7, 0xfb, 0x3b, 0x7d, 0x58,
0x96, 0x03, 0xa3, 0x16, 0x14, 0x0f, 0xf1, 0x09, 0x5d, 0x72, 0x4d, 0x23, 0x3f, 0xd1, 0x1d, 0x28,
0x1f, 0xe9, 0xc3, 0x09, 0xe6, 0x36, 0x61, 0x8a, 0xe4, 0x32, 0xd8, 0x0f, 0x0b, 0x1f, 0x28, 0xea,
0x08, 0x2e, 0x3d, 0xc0, 0x5e, 0xd7, 0x72, 0xb1, 0xe3, 0xad, 0x9b, 0xd6, 0xd0, 0x1e, 0x6c, 0xeb,
0xde, 0xc1, 0x0c, 0xc6, 0x21, 0xa2, 0xe7, 0x85, 0x98, 0x9e, 0xab, 0x3f, 0x56, 0xe0, 0x75, 0xf9,
0x7c, 0x7c, 0x43, 0x3b, 0x50, 0xdd, 0x37, 0xf1, 0xd0, 0x20, 0x52, 0xa3, 0x50, 0xa9, 0x11, 0x6d,
0x62, 0x24, 0xc6, 0x04, 0x98, 0xef, 0x5b, 0xcc, 0x48, 0x88, 0x98, 0x6f, 0xc7, 0x73, 0x4c, 0x6b,
0xf0, 0xd0, 0x74, 0x3d, 0x8d, 0xc1, 0x87, 0xa4, 0xa4, 0x98, 0x5f, 0x39, 0x7f, 0x4f, 0x81, 0x2b,
0x0f, 0xb0, 0xb7, 0x21, 0x7c, 0x0c, 0xf9, 0x6e, 0xba, 0x9e, 0xd9, 0x77, 0x5f, 0x6c, 0x0c, 0x98,
0x23, 0xd8, 0x50, 0xff, 0x48, 0x81, 0xab, 0xa9, 0xc4, 0x70, 0xd6, 0x71, 0x1b, 0xea, 0x7b, 0x18,
0xb9, 0x0d, 0xfd, 0x25, 0x7c, 0xf2, 0x84, 0x6c, 0xfe, 0xb6, 0x6e, 0x3a, 0xcc, 0x86, 0x9e, 0xd1,
0xa3, 0xfc, 0x44, 0x81, 0xcb, 0x0f, 0xb0, 0xb7, 0xed, 0xfb, 0xd7, 0x57, 0xc8, 0x1d, 0x02, 0x13,
0xf2, 0xf3, 0x7e, 0xa0, 0x19, 0xe9, 0x53, 0xff, 0x90, 0x6d, 0xa7, 0x94, 0xde, 0x57, 0xc2, 0xc0,
0x2b, 0x54, 0x13, 0x42, 0x26, 0x82, 0x2b, 0x3b, 0x67, 0x9f, 0xfa, 0xc3, 0x32, 0x34, 0x9e, 0x70,
0xab, 0x40, 0x3d, 0x68, 0x9c, 0x13, 0x8a, 0x3c, 0x08, 0x0a, 0x45, 0x53, 0xb2, 0x00, 0x6b, 0x1d,
0xe6, 0x5d, 0x8c, 0x0f, 0x4f, 0xe9, 0x2f, 0x1b, 0x64, 0x8c, 0x70, 0x76, 0x0f, 0x61, 0x71, 0x62,
0xd1, 0x08, 0x1d, 0x1b, 0x7c, 0x01, 0x8c, 0xe9, 0xd3, 0x8d, 0x69, 0x72, 0x20, 0xfa, 0x94, 0x27,
0x01, 0x21, 0x5c, 0xe5, 0x5c, 0xb8, 0xe2, 0xc3, 0x50, 0x17, 0x5a, 0x86, 0x63, 0x8f, 0xc7, 0xd8,
0xe8, 0xb9, 0x3e, 0xaa, 0x4a, 0x3e, 0x54, 0x7c, 0x9c, 0x40, 0x75, 0x1b, 0xce, 0xc7, 0x29, 0xed,
0x1a, 0x24, 0x2e, 0x24, 0x92, 0x25, 0xfb, 0x84, 0xde, 0x85, 0xc5, 0x24, 0x7c, 0x95, 0xc2, 0x27,
0x3f, 0xa0, 0xf7, 0x00, 0xc5, 0x48, 0x25, 0xe0, 0x35, 0x06, 0x1e, 0x25, 0x86, 0x83, 0xd3, 0xe4,
0x34, 0x0a, 0x0e, 0x0c, 0x9c, 0x7f, 0x09, 0x81, 0x77, 0x89, 0x77, 0x8d, 0x80, 0xbb, 0xed, 0x7a,
0x3e, 0x46, 0x44, 0x91, 0xb9, 0xea, 0xef, 0x2a, 0xb0, 0xfc, 0x54, 0xf7, 0xfa, 0x07, 0x9b, 0x23,
0x2e, 0xa0, 0x33, 0x28, 0xf8, 0x47, 0x50, 0x3b, 0xe2, 0xc2, 0xe8, 0x5b, 0xf1, 0xab, 0x12, 0x82,
0xc2, 0x62, 0xaf, 0x05, 0x23, 0x48, 0x42, 0xb4, 0x74, 0x3f, 0x94, 0x18, 0xbe, 0x02, 0x53, 0x33,
0x25, 0xa3, 0x55, 0x9f, 0x03, 0x70, 0xe2, 0xb6, 0xdc, 0xc1, 0x19, 0xe8, 0xfa, 0x00, 0xe6, 0x38,
0x36, 0x6e, 0x4b, 0xa6, 0x6d, 0x98, 0x0f, 0xae, 0xfe, 0xa8, 0x02, 0xf5, 0xd0, 0x07, 0xd4, 0x84,
0x82, 0x30, 0x12, 0x05, 0xc9, 0xea, 0x0a, 0xd3, 0x73, 0xa8, 0x62, 0x32, 0x87, 0xba, 0x01, 0x4d,
0x93, 0x3a, 0xef, 0x1e, 0xdf, 0x15, 0x1a, 0x2b, 0xd7, 0xb4, 0x79, 0xd6, 0xcb, 0x45, 0x04, 0x5d,
0x81, 0xba, 0x35, 0x19, 0xf5, 0xec, 0xfd, 0x9e, 0x63, 0x1f, 0xbb, 0x3c, 0x19, 0xab, 0x59, 0x93,
0xd1, 0x77, 0xf7, 0x35, 0xfb, 0xd8, 0x0d, 0xe2, 0xfd, 0xca, 0x29, 0xe3, 0xfd, 0x2b, 0x50, 0x1f,
0xe9, 0xcf, 0x09, 0xd6, 0x9e, 0x35, 0x19, 0xd1, 0x3c, 0xad, 0xa8, 0xd5, 0x46, 0xfa, 0x73, 0xcd,
0x3e, 0x7e, 0x34, 0x19, 0xa1, 0x15, 0x68, 0x0d, 0x75, 0xd7, 0xeb, 0x85, 0x13, 0xbd, 0x2a, 0x4d,
0xf4, 0x9a, 0xa4, 0xff, 0x93, 0x20, 0xd9, 0x4b, 0x66, 0x0e, 0xb5, 0xb3, 0x65, 0x0e, 0xc6, 0x68,
0x18, 0xe0, 0x80, 0x5c, 0x99, 0x83, 0x31, 0x1a, 0x0a, 0x0c, 0x1f, 0xc0, 0xdc, 0x1e, 0x0d, 0x84,
0xb2, 0x54, 0xf4, 0x3e, 0x89, 0x81, 0x58, 0xbc, 0xa4, 0xf9, 0xe0, 0xe8, 0x5b, 0x50, 0xa3, 0xfe,
0x87, 0x8e, 0x6d, 0xe4, 0x1a, 0x1b, 0x0c, 0x20, 0xa3, 0x0d, 0x3c, 0xf4, 0x74, 0x3a, 0x7a, 0x3e,
0xdf, 0x68, 0x31, 0x80, 0xd8, 0xc7, 0xbe, 0x83, 0x75, 0x0f, 0x1b, 0xeb, 0x27, 0x1b, 0xf6, 0x68,
0xac, 0x53, 0x11, 0x6a, 0x37, 0x69, 0x08, 0x2f, 0xfb, 0x84, 0xde, 0x84, 0x66, 0x5f, 0xb4, 0xee,
0x3b, 0xf6, 0xa8, 0xbd, 0x40, 0xb5, 0x27, 0xd6, 0x8b, 0x2e, 0x03, 0xf8, 0x96, 0x51, 0xf7, 0xda,
0x2d, 0xba, 0x77, 0x35, 0xde, 0x73, 0x8f, 0x56, 0x6f, 0x4c, 0xb7, 0xc7, 0xea, 0x24, 0xa6, 0x35,
0x68, 0x2f, 0xd2, 0x19, 0xeb, 0x7e, 0x61, 0xc5, 0xb4, 0x06, 0xe8, 0x02, 0xcc, 0x99, 0x6e, 0x6f,
0x5f, 0x3f, 0xc4, 0x6d, 0x44, 0xbf, 0x56, 0x4c, 0xf7, 0xbe, 0x7e, 0x88, 0xd5, 0xcf, 0x61, 0x29,
0x90, 0xa9, 0xd0, 0x26, 0x26, 0x45, 0x41, 0x39, 0x83, 0x28, 0x64, 0x47, 0xbe, 0x3f, 0x2f, 0xc1,
0xf2, 0x8e, 0x7e, 0x84, 0x5f, 0x7e, 0x90, 0x9d, 0xcb, 0x8e, 0x3d, 0x84, 0x45, 0x1a, 0x57, 0xaf,
0x85, 0xe8, 0xc9, 0x70, 0xe1, 0x61, 0x29, 0x48, 0x0e, 0x44, 0xdf, 0x26, 0x61, 0x07, 0xee, 0x1f,
0x6e, 0x93, 0x1c, 0xc5, 0x77, 0xdf, 0x97, 0x25, 0x78, 0x36, 0x04, 0x94, 0x16, 0x1e, 0x81, 0xb6,
0x61, 0x21, 0xba, 0x03, 0xbe, 0xe3, 0x7e, 0x2b, 0x33, 0x81, 0x0d, 0xb8, 0xaf, 0x35, 0x23, 0x9b,
0xe1, 0xa2, 0x36, 0xcc, 0x71, 0xaf, 0x4b, 0x8d, 0x44, 0x55, 0xf3, 0x9b, 0x68, 0x1b, 0xce, 0xb3,
0x15, 0xec, 0x70, 0x5d, 0x60, 0x8b, 0xaf, 0xe6, 0x5a, 0xbc, 0x6c, 0x68, 0x54, 0x95, 0x6a, 0xa7,
0x55, 0xa5, 0x36, 0xcc, 0x71, 0xf1, 0xa6, 0xd6, 0xa3, 0xaa, 0xf9, 0x4d, 0xb2, 0xcd, 0x81, 0xa0,
0xd7, 0xe9, 0xb7, 0xa0, 0x43, 0xfd, 0x6d, 0x05, 0x20, 0xe0, 0xe7, 0x94, 0x02, 0xcb, 0x37, 0xa1,
0x2a, 0x84, 0x3b, 0x57, 0x8e, 0x28, 0xc0, 0xe3, 0xb6, 0xbc, 0x18, 0xb3, 0xe5, 0xea, 0xbf, 0x28,
0xd0, 0xd8, 0x24, 0xab, 0x79, 0x68, 0x0f, 0xa8, 0xe7, 0xb9, 0x01, 0x4d, 0x07, 0xf7, 0x6d, 0xc7,
0xe8, 0x61, 0xcb, 0x73, 0x4c, 0xcc, 0x92, 0xf3, 0x92, 0x36, 0xcf, 0x7a, 0x3f, 0x61, 0x9d, 0x04,
0x8c, 0x98, 0x67, 0xd7, 0xd3, 0x47, 0xe3, 0xde, 0x3e, 0x31, 0x08, 0x05, 0x06, 0x26, 0x7a, 0xa9,
0x3d, 0xf8, 0x1a, 0x34, 0x02, 0x30, 0xcf, 0xa6, 0xf3, 0x97, 0xb4, 0xba, 0xe8, 0xdb, 0xb5, 0xd1,
0x75, 0x68, 0x52, 0x76, 0xf6, 0x86, 0xf6, 0xa0, 0x47, 0x52, 0x3e, 0xee, 0x94, 0x1a, 0x06, 0x27,
0x8b, 0x6c, 0x53, 0x14, 0xca, 0x35, 0x3f, 0xc7, 0xdc, 0x2d, 0x09, 0xa8, 0x1d, 0xf3, 0x73, 0xac,
0xfe, 0x96, 0x02, 0xf3, 0xdc, 0x8b, 0xed, 0x88, 0xe2, 0x37, 0xad, 0x56, 0xb2, 0x74, 0x9b, 0xfe,
0x46, 0x1f, 0x46, 0xeb, 0x55, 0xd7, 0xa5, 0xa2, 0x4e, 0x91, 0xd0, 0xd8, 0x29, 0xe2, 0xc2, 0xf2,
0xe4, 0x7b, 0x5f, 0x10, 0x9e, 0xea, 0x9e, 0xfe, 0xc8, 0x36, 0x58, 0xf9, 0xac, 0x0d, 0x73, 0xba,
0x61, 0x38, 0xd8, 0x75, 0x39, 0x1d, 0x7e, 0x93, 0x7c, 0x39, 0xc2, 0x8e, 0xeb, 0x6f, 0x6c, 0x51,
0xf3, 0x9b, 0xe8, 0x5b, 0x50, 0x15, 0xc1, 0x16, 0xab, 0x53, 0x5c, 0x4b, 0xa7, 0x93, 0x67, 0x27,
0x62, 0x84, 0xfa, 0x0f, 0x05, 0x68, 0x72, 0x4d, 0x5b, 0xe7, 0x0e, 0x27, 0x5b, 0xc4, 0xd6, 0xa1,
0xb1, 0x1f, 0x48, 0x78, 0x56, 0x75, 0x25, 0xac, 0x08, 0x91, 0x31, 0xd3, 0x64, 0x2d, 0xea, 0xf2,
0x4a, 0x33, 0xb9, 0xbc, 0xf2, 0x69, 0xf5, 0x34, 0x19, 0xfa, 0x54, 0x24, 0xa1, 0x8f, 0xfa, 0x2b,
0x50, 0x0f, 0x21, 0xa0, 0x76, 0x88, 0x15, 0x30, 0x38, 0xc7, 0xfc, 0x26, 0xba, 0x13, 0x38, 0x7e,
0xc6, 0xaa, 0x8b, 0x12, 0x5a, 0x62, 0x3e, 0x5f, 0xfd, 0x99, 0x02, 0x15, 0x8e, 0xf9, 0x2a, 0xd4,
0xb9, 0x7e, 0xd1, 0x50, 0x88, 0x61, 0x07, 0xde, 0x45, 0x62, 0xa1, 0x17, 0xa7, 0x60, 0x17, 0xa1,
0x1a, 0x53, 0xad, 0x39, 0x6e, 0xfc, 0xfc, 0x4f, 0x21, 0x7d, 0x22, 0x9f, 0x88, 0x2a, 0xa1, 0x25,
0x28, 0x0f, 0xed, 0x81, 0x38, 0xdc, 0x60, 0x0d, 0xf5, 0x4b, 0x85, 0xd6, 0xa2, 0x35, 0xdc, 0xb7,
0x8f, 0xb0, 0x73, 0x32, 0x7b, 0x39, 0xef, 0x6e, 0x48, 0xcc, 0x73, 0xe6, 0x14, 0x62, 0x00, 0xba,
0x1b, 0x6c, 0x42, 0x51, 0x96, 0xf5, 0x87, 0x1d, 0x0e, 0x17, 0xd2, 0x60, 0x33, 0xfe, 0x58, 0xa1,
0x85, 0xc9, 0xe8, 0x52, 0xce, 0xea, 0xd3, 0x5f, 0x48, 0x7c, 0xae, 0xfe, 0xb3, 0x02, 0x17, 0x53,
0xb8, 0xfb, 0x64, 0xed, 0x15, 0xf0, 0xf7, 0x43, 0xa8, 0x8a, 0x0c, 0xb4, 0x98, 0x2b, 0x03, 0x15,
0xf0, 0xea, 0x9f, 0xb2, 0xf2, 0xb8, 0x84, 0xbd, 0x4f, 0xd6, 0x5e, 0x12, 0x83, 0xe3, 0x95, 0xa4,
0xa2, 0xa4, 0x92, 0xf4, 0x73, 0x05, 0x3a, 0x41, 0xe5, 0xc6, 0x5d, 0x3f, 0x99, 0xf5, 0x3c, 0xe5,
0xc5, 0x64, 0x66, 0xdf, 0x14, 0xa5, 0x7f, 0x62, 0x17, 0x73, 0xe5, 0x54, 0x7e, 0xe1, 0xdf, 0xa2,
0x45, 0xe0, 0xe4, 0x82, 0x66, 0xd1, 0xca, 0x4e, 0x68, 0xe3, 0x59, 0xf9, 0x3f, 0xd8, 0xd8, 0x9f,
0x31, 0x21, 0xbd, 0x1f, 0x2d, 0xdf, 0xbc, 0x6a, 0x06, 0x86, 0x8f, 0x24, 0x0e, 0xf8, 0x91, 0x44,
0x29, 0x76, 0x24, 0xc1, 0xfb, 0xd5, 0x11, 0x15, 0x81, 0xc4, 0x02, 0x5e, 0x16, 0xc3, 0x7e, 0x47,
0x81, 0x36, 0x9f, 0x85, 0xce, 0x49, 0xd2, 0xaa, 0x21, 0xf6, 0xb0, 0xf1, 0x55, 0x17, 0x19, 0xfe,
0xa2, 0x00, 0xad, 0x70, 0x60, 0x43, 0x63, 0x93, 0xaf, 0x43, 0x99, 0xd6, 0x68, 0x38, 0x05, 0x53,
0xad, 0x03, 0x83, 0x26, 0x9e, 0x91, 0xc6, 0xec, 0xbb, 0xae, 0x1f, 0xb8, 0xf0, 0x66, 0x10, 0x5d,
0x15, 0x4f, 0x1f, 0x5d, 0xbd, 0x0e, 0x35, 0xe2, 0xb9, 0xec, 0x09, 0xc1, 0xcb, 0xce, 0x89, 0x83,
0x0e, 0xf4, 0x11, 0x54, 0xd8, 0xed, 0x0f, 0x7e, 0x4c, 0x77, 0x23, 0x8a, 0x9a, 0xdf, 0x0c, 0x09,
0x95, 0xd9, 0x69, 0x87, 0xc6, 0x07, 0x91, 0x3d, 0x1a, 0x3b, 0xf6, 0x80, 0x86, 0x61, 0xc4, 0xa9,
0x95, 0x35, 0xd1, 0x56, 0x7f, 0x11, 0x96, 0x83, 0x6c, 0x97, 0x91, 0x74, 0x56, 0x81, 0x56, 0xff,
0x5d, 0x81, 0xf3, 0x3b, 0x27, 0x56, 0x3f, 0xae, 0x1a, 0xcb, 0x50, 0x19, 0x0f, 0xf5, 0xa0, 0xf8,
0xcb, 0x5b, 0xf4, 0x60, 0x9d, 0xcd, 0x8d, 0x0d, 0xe2, 0xc2, 0x19, 0x3f, 0xeb, 0xa2, 0x6f, 0xd7,
0x9e, 0x1a, 0x59, 0xdd, 0x10, 0xe9, 0x39, 0x36, 0x58, 0xb0, 0xc0, 0x8a, 0x5b, 0xf3, 0xa2, 0x97,
0x06, 0x0b, 0x1f, 0x01, 0xd0, 0x78, 0xaa, 0x77, 0x9a, 0x18, 0x8a, 0x8e, 0x78, 0x48, 0x3c, 0xe6,
0x4f, 0x0b, 0xd0, 0x0e, 0x71, 0xe9, 0xab, 0x0e, 0x2f, 0x53, 0x52, 0xbf, 0xe2, 0x0b, 0x4a, 0xfd,
0x4a, 0xb3, 0x87, 0x94, 0x65, 0x59, 0x48, 0xf9, 0x1b, 0x45, 0x68, 0x06, 0x5c, 0xdb, 0x1e, 0xea,
0x56, 0xaa, 0x24, 0xec, 0x40, 0xd3, 0x8d, 0x70, 0x95, 0xf3, 0xe9, 0x1d, 0x99, 0x0e, 0xa5, 0x6c,
0x84, 0x16, 0x43, 0x81, 0x2e, 0xd3, 0x4d, 0x77, 0x3c, 0x56, 0x4e, 0x63, 0xf1, 0x61, 0x8d, 0x29,
0xab, 0x39, 0xc2, 0xe8, 0x5d, 0x40, 0x5c, 0xc3, 0x7a, 0xa6, 0xd5, 0x73, 0x71, 0xdf, 0xb6, 0x0c,
0xa6, 0x7b, 0x65, 0xad, 0xc5, 0xbf, 0x74, 0xad, 0x1d, 0xd6, 0x8f, 0xbe, 0x0e, 0x25, 0xef, 0x64,
0xcc, 0x82, 0xc5, 0xa6, 0x34, 0xdc, 0x0a, 0xe8, 0xda, 0x3d, 0x19, 0x63, 0x8d, 0x82, 0xfb, 0x17,
0x80, 0x3c, 0x47, 0x3f, 0xe2, 0x91, 0x77, 0x49, 0x0b, 0xf5, 0x10, 0x6b, 0xe2, 0xf3, 0x70, 0x8e,
0x45, 0xa8, 0xbc, 0xc9, 0x24, 0xdb, 0x57, 0xe8, 0x9e, 0xe7, 0x0d, 0x69, 0x41, 0x90, 0x4a, 0xb6,
0xdf, 0xbb, 0xeb, 0x0d, 0xc9, 0x22, 0x3d, 0xdb, 0xd3, 0x87, 0x4c, 0x3f, 0x6a, 0xdc, 0x72, 0x90,
0x1e, 0x9a, 0xe5, 0xfe, 0x1b, 0xb1, 0x7c, 0x82, 0x30, 0x0d, 0xbb, 0x93, 0x61, 0xba, 0x3e, 0x66,
0xd7, 0x67, 0xa6, 0xa9, 0xe2, 0xb7, 0xa1, 0xce, 0xa5, 0xe2, 0x14, 0x52, 0x05, 0x6c, 0xc8, 0xc3,
0x0c, 0x31, 0x2f, 0xbf, 0x20, 0x31, 0xaf, 0x9c, 0xa1, 0xc2, 0x21, 0xdf, 0x1b, 0xf5, 0xc7, 0x0a,
0xbc, 0x96, 0xb0, 0x9a, 0x99, 0xac, 0xcd, 0xce, 0xbc, 0xb9, 0x35, 0x8d, 0xa3, 0xe4, 0xbe, 0xe1,
0x2e, 0x54, 0x1c, 0x8a, 0x9d, 0x1f, 0x7a, 0xbd, 0x91, 0x29, 0x7c, 0x8c, 0x10, 0x8d, 0x0f, 0x51,
0xff, 0x44, 0x81, 0x0b, 0x49, 0x52, 0x67, 0x70, 0xf8, 0xeb, 0x30, 0xc7, 0x50, 0xfb, 0x3a, 0xba,
0x92, 0xad, 0xa3, 0x01, 0x73, 0x34, 0x7f, 0xa0, 0xba, 0x03, 0xcb, 0x7e, 0x5c, 0x10, 0xb0, 0x7e,
0x0b, 0x7b, 0x7a, 0x46, 0xde, 0x79, 0x15, 0xea, 0x2c, 0x81, 0x61, 0xf9, 0x1c, 0x3b, 0x23, 0x84,
0x3d, 0x51, 0xce, 0x53, 0xff, 0x5b, 0x81, 0x25, 0xea, 0x58, 0xe3, 0x07, 0x3e, 0x79, 0x4e, 0x20,
0x55, 0x71, 0xc7, 0xeb, 0x91, 0x3e, 0xe2, 0xf7, 0x50, 0x6a, 0x5a, 0xa4, 0x0f, 0x75, 0x93, 0xd5,
0x3e, 0x69, 0x7d, 0x22, 0x38, 0x72, 0xdd, 0xd4, 0x3d, 0x9d, 0x9e, 0xb8, 0xc6, 0xcb, 0x7c, 0x81,
0x43, 0x2f, 0x9d, 0xc1, 0xa1, 0xab, 0x0f, 0xe1, 0xb5, 0xd8, 0x4a, 0x67, 0xd8, 0x51, 0xf5, 0x6f,
0x14, 0xb2, 0x1d, 0x91, 0xfb, 0x3c, 0x67, 0x0f, 0x6a, 0x2f, 0x8b, 0x93, 0xa6, 0x9e, 0x69, 0xc4,
0x8d, 0x88, 0x81, 0x3e, 0x86, 0x9a, 0x85, 0x8f, 0x7b, 0xe1, 0x38, 0x29, 0x47, 0xc4, 0x5f, 0xb5,
0xf0, 0x31, 0xfd, 0xa5, 0x3e, 0x82, 0x0b, 0x09, 0x52, 0x67, 0x59, 0xfb, 0x3f, 0x29, 0x70, 0x71,
0xd3, 0xb1, 0xc7, 0x4f, 0x4c, 0xc7, 0x9b, 0xe8, 0xc3, 0xe8, 0x61, 0xf6, 0x19, 0x96, 0x9f, 0xe3,
0xae, 0xe0, 0xa7, 0x89, 0xdc, 0xf2, 0x5d, 0x89, 0x06, 0x25, 0x89, 0xe2, 0x8b, 0x0e, 0xc5, 0xd7,
0xff, 0x59, 0x94, 0x11, 0xcf, 0xe1, 0xa6, 0xc4, 0x25, 0x79, 0x92, 0x0f, 0x69, 0xb5, 0xbd, 0x78,
0xd6, 0x6a, 0x7b, 0x8a, 0x79, 0x2f, 0xbd, 0x20, 0xf3, 0x7e, 0xea, 0xc2, 0xd8, 0x06, 0x44, 0x4f,
0x42, 0xa8, 0x77, 0x3e, 0xed, 0xe9, 0xc9, 0x47, 0x00, 0xc1, 0x81, 0x00, 0xbf, 0x7f, 0x39, 0x05,
0x43, 0x68, 0x00, 0xd9, 0x23, 0xe1, 0x40, 0xb9, 0x7f, 0x0f, 0x95, 0xa8, 0xbf, 0x07, 0x1d, 0x99,
0x6c, 0xce, 0x22, 0xef, 0x3f, 0x2d, 0x00, 0x74, 0xc5, 0x6d, 0xdd, 0xb3, 0x79, 0x80, 0x37, 0x20,
0x14, 0x83, 0x04, 0x5a, 0x1e, 0x96, 0x1d, 0x83, 0x28, 0x82, 0xc8, 0x52, 0x09, 0x4c, 0x22, 0x73,
0x35, 0x28, 0x9e, 0x90, 0xae, 0x30, 0x51, 0x88, 0x1b, 0xdd, 0x4b, 0x50, 0x73, 0xec, 0xe3, 0x1e,
0x51, 0x2e, 0xc3, 0xbf, 0x8e, 0xec, 0xd8, 0xc7, 0x44, 0xe5, 0x0c, 0x74, 0x01, 0xe6, 0x3c, 0xdd,
0x3d, 0x24, 0xf8, 0x59, 0xb1, 0xae, 0x42, 0x9a, 0x5d, 0x03, 0x2d, 0x41, 0x79, 0xdf, 0x1c, 0x62,
0x76, 0xf3, 0xa1, 0xa6, 0xb1, 0x06, 0xfa, 0x86, 0x7f, 0x83, 0xae, 0x9a, 0xfb, 0xa6, 0x0c, 0x85,
0x57, 0xbf, 0x54, 0x60, 0x21, 0xe0, 0x1a, 0x35, 0x3b, 0xc4, 0x92, 0x51, 0x2b, 0xb6, 0x61, 0x1b,
0xcc, 0x40, 0x34, 0x53, 0xfc, 0x00, 0x1b, 0xc8, 0x6c, 0x55, 0x30, 0x24, 0x2b, 0x71, 0x26, 0xeb,
0x22, 0x8b, 0x36, 0x0d, 0xbf, 0x94, 0x53, 0x71, 0xec, 0xe3, 0xae, 0x21, 0xb8, 0xc1, 0xee, 0x1a,
0xb3, 0x34, 0x91, 0x70, 0x63, 0x83, 0x5e, 0x37, 0x7e, 0x03, 0xe6, 0xb1, 0xe3, 0xd8, 0x4e, 0x6f,
0x84, 0x5d, 0x57, 0x1f, 0x60, 0x1e, 0x95, 0x37, 0x68, 0xe7, 0x16, 0xeb, 0x53, 0xff, 0xbc, 0x04,
0xcd, 0x60, 0x29, 0xfe, 0x91, 0xbb, 0x69, 0xf8, 0x47, 0xee, 0x26, 0xd9, 0x3a, 0x70, 0x98, 0x01,
0x14, 0x9b, 0xbb, 0x5e, 0x68, 0x2b, 0x5a, 0x8d, 0xf7, 0x76, 0x0d, 0xe2, 0x8c, 0x89, 0x6a, 0x59,
0xb6, 0x81, 0x83, 0xcd, 0x05, 0xbf, 0x8b, 0xef, 0x6d, 0x44, 0x46, 0x4a, 0x39, 0x64, 0xa4, 0x9c,
0x43, 0x46, 0x2a, 0x12, 0x19, 0x59, 0x86, 0xca, 0xde, 0xa4, 0x7f, 0x88, 0x3d, 0x1e, 0xa7, 0xf1,
0x56, 0x54, 0x76, 0xaa, 0x31, 0xd9, 0x11, 0x22, 0x52, 0x0b, 0x8b, 0xc8, 0x25, 0xa8, 0xb1, 0x53,
0xe0, 0x9e, 0xe7, 0xd2, 0x73, 0xad, 0xa2, 0x56, 0x65, 0x1d, 0xbb, 0x2e, 0xfa, 0xc0, 0x0f, 0xe2,
0xea, 0x54, 0x59, 0x54, 0x89, 0xad, 0x89, 0x49, 0x89, 0x1f, 0xc2, 0xbd, 0x05, 0x0b, 0x21, 0x76,
0x50, 0xcf, 0xd0, 0xa0, 0xa4, 0x86, 0x62, 0x7c, 0xea, 0x1c, 0x6e, 0x40, 0x33, 0x60, 0x09, 0x85,
0x9b, 0x67, 0xa9, 0x95, 0xe8, 0xa5, 0x60, 0x42, 0x92, 0x9b, 0xa7, 0x93, 0x64, 0x74, 0x11, 0xaa,
0x3c, 0x27, 0x72, 0xdb, 0x0b, 0x91, 0xf2, 0x85, 0xfa, 0x03, 0x40, 0x01, 0xf5, 0xb3, 0xc5, 0x88,
0x31, 0xf1, 0x28, 0xc4, 0xc5, 0x43, 0xfd, 0x5b, 0x05, 0x16, 0xc3, 0x93, 0x9d, 0xd5, 0xdd, 0x7e,
0x0c, 0x75, 0x76, 0xb2, 0xd8, 0x23, 0x8a, 0x2f, 0x3f, 0x22, 0x8c, 0xed, 0x8b, 0x06, 0xc1, 0x6b,
0x05, 0x22, 0x5e, 0xc7, 0xb6, 0x73, 0x68, 0x5a, 0x83, 0x1e, 0xa1, 0x4c, 0x54, 0x4e, 0x79, 0xe7,
0x23, 0xd2, 0xa7, 0xfe, 0xbe, 0x02, 0x57, 0x1e, 0x8f, 0x0d, 0xdd, 0xc3, 0xa1, 0xb8, 0x63, 0xd6,
0x4b, 0x83, 0xe2, 0xd6, 0x5e, 0x21, 0x63, 0x07, 0x43, 0xf3, 0xb9, 0xfc, 0xd6, 0x1e, 0x89, 0xd6,
0x38, 0x35, 0x89, 0x6b, 0xb6, 0x67, 0xa7, 0xa6, 0x03, 0xd5, 0x23, 0x8e, 0xce, 0x7f, 0x7f, 0xe1,
0xb7, 0x23, 0x67, 0xb0, 0xc5, 0x53, 0x9d, 0xc1, 0xaa, 0x5b, 0x70, 0x51, 0xc3, 0x2e, 0xb6, 0x8c,
0xc8, 0x42, 0xce, 0x5c, 0x5f, 0x1a, 0x43, 0x47, 0x86, 0x6e, 0x16, 0x49, 0x65, 0xe1, 0x6a, 0xcf,
0x21, 0x68, 0x3d, 0x6e, 0x87, 0x49, 0x94, 0x44, 0xe7, 0xf1, 0xd4, 0xbf, 0x2b, 0xc0, 0x85, 0x7b,
0x86, 0xc1, 0x4d, 0x38, 0x0f, 0xc0, 0x5e, 0x56, 0x6c, 0x1c, 0x8f, 0x1d, 0x8b, 0xc9, 0xd8, 0xf1,
0x45, 0x99, 0x55, 0xee, 0x60, 0xac, 0xc9, 0xc8, 0x77, 0x9c, 0x0e, 0xbb, 0x88, 0x74, 0x97, 0x9f,
0x54, 0x92, 0x1c, 0x9e, 0x3a, 0xcf, 0xe9, 0x21, 0x55, 0xd5, 0xaf, 0x93, 0xa9, 0x63, 0x68, 0x27,
0x99, 0x35, 0xa3, 0x1d, 0xf1, 0x39, 0x32, 0xb6, 0x59, 0xbd, 0xb5, 0x41, 0xe2, 0x27, 0xda, 0xb5,
0x6d, 0xbb, 0xea, 0xff, 0x14, 0xa0, 0xbd, 0xa3, 0x1f, 0xe1, 0xff, 0x3f, 0x1b, 0xf4, 0x19, 0x2c,
0xb9, 0xfa, 0x11, 0xee, 0x85, 0x72, 0xe1, 0x9e, 0x83, 0x9f, 0xf1, 0xd0, 0xf3, 0x6d, 0x59, 0x45,
0x5c, 0x7a, 0x7d, 0x47, 0x5b, 0x74, 0x23, 0xfd, 0x1a, 0x7e, 0x86, 0xde, 0x84, 0x85, 0xf0, 0xad,
0x30, 0x42, 0x5a, 0x95, 0xb2, 0x7c, 0x3e, 0x74, 0xf3, 0xab, 0x6b, 0xa8, 0xcf, 0xe0, 0xf5, 0xc7,
0x96, 0x8b, 0xbd, 0x6e, 0x70, 0x7b, 0x69, 0xc6, 0xac, 0xf1, 0x2a, 0xd4, 0x03, 0xc6, 0x27, 0x1e,
0x5e, 0x18, 0xae, 0x6a, 0x43, 0x67, 0x4b, 0x77, 0x0e, 0xfd, 0xca, 0xf2, 0x26, 0xbb, 0x6a, 0xf2,
0x12, 0x27, 0xdc, 0x17, 0x97, 0xae, 0x34, 0xbc, 0x8f, 0x1d, 0x6c, 0xf5, 0xf1, 0x43, 0xbb, 0x7f,
0x48, 0x62, 0x0d, 0x8f, 0xbd, 0x7d, 0x53, 0x42, 0x11, 0xe7, 0x66, 0xe8, 0x69, 0x5b, 0x21, 0xf2,
0xb4, 0x6d, 0xca, 0x53, 0x49, 0xf5, 0x27, 0x05, 0x58, 0xbe, 0x37, 0xf4, 0xb0, 0x13, 0x24, 0xfb,
0xa7, 0xa9, 0x5b, 0x04, 0x85, 0x84, 0xc2, 0x59, 0x4e, 0x06, 0x72, 0x1c, 0x1c, 0xca, 0xca, 0x1e,
0xa5, 0x33, 0x96, 0x3d, 0xee, 0x01, 0x8c, 0x1d, 0x7b, 0x8c, 0x1d, 0xcf, 0xc4, 0x7e, 0xc6, 0x96,
0x23, 0x76, 0x09, 0x0d, 0x52, 0x3f, 0x83, 0xd6, 0x83, 0xfe, 0x86, 0x6d, 0xed, 0x9b, 0xce, 0xc8,
0x67, 0x54, 0x42, 0xe9, 0x94, 0x1c, 0x4a, 0x57, 0x48, 0x28, 0x9d, 0x6a, 0xc2, 0x62, 0x08, 0xf7,
0x8c, 0x86, 0x6b, 0xd0, 0xef, 0xed, 0x9b, 0x96, 0x49, 0xaf, 0x72, 0x15, 0x68, 0xec, 0x09, 0x83,
0xfe, 0x7d, 0xde, 0xa3, 0xfe, 0x50, 0x81, 0x4b, 0x1a, 0x26, 0xca, 0xe3, 0xdf, 0x97, 0xd9, 0xf5,
0xb6, 0xdc, 0xc1, 0x0c, 0x01, 0xc5, 0x1d, 0x28, 0x8d, 0xdc, 0x41, 0xca, 0x59, 0x37, 0x71, 0xd1,
0x91, 0x89, 0x34, 0x0a, 0x7c, 0xf3, 0x63, 0x71, 0xf9, 0x76, 0xf7, 0x64, 0x8c, 0xd1, 0x1c, 0x14,
0x1f, 0xe1, 0xe3, 0xd6, 0x39, 0x04, 0x50, 0x79, 0x64, 0x3b, 0x23, 0x7d, 0xd8, 0x52, 0x50, 0x1d,
0xe6, 0xf8, 0x89, 0x61, 0xab, 0x80, 0xe6, 0xa1, 0xb6, 0xe1, 0x9f, 0xac, 0xb4, 0x8a, 0x37, 0xff,
0x52, 0x81, 0xc5, 0xc4, 0x99, 0x16, 0x6a, 0x02, 0x3c, 0xb6, 0xfa, 0xfc, 0xb0, 0xaf, 0x75, 0x0e,
0x35, 0xa0, 0xea, 0x1f, 0xfd, 0x31, 0x7c, 0xbb, 0x36, 0x85, 0x6e, 0x15, 0x50, 0x0b, 0x1a, 0x6c,
0xe0, 0xa4, 0xdf, 0xc7, 0xae, 0xdb, 0x2a, 0x8a, 0x9e, 0xfb, 0xba, 0x39, 0x9c, 0x38, 0xb8, 0x55,
0x22, 0x73, 0xee, 0xda, 0x1a, 0x1e, 0x62, 0xdd, 0xc5, 0xad, 0x32, 0x42, 0xd0, 0xe4, 0x0d, 0x7f,
0x50, 0x25, 0xd4, 0xe7, 0x0f, 0x9b, 0xbb, 0xf9, 0x34, 0x7c, 0xfa, 0x40, 0x97, 0x77, 0x01, 0xce,
0x3f, 0xb6, 0x0c, 0xbc, 0x6f, 0x5a, 0xd8, 0x08, 0x3e, 0xb5, 0xce, 0xa1, 0xf3, 0xb0, 0xb0, 0x85,
0x9d, 0x01, 0x0e, 0x75, 0x16, 0xd0, 0x22, 0xcc, 0x6f, 0x99, 0xcf, 0x43, 0x5d, 0x45, 0xb5, 0x54,
0x55, 0x5a, 0xca, 0xda, 0x3f, 0xde, 0x80, 0x1a, 0x61, 0xe6, 0x86, 0x6d, 0x3b, 0x06, 0x1a, 0x02,
0xa2, 0x4f, 0x5c, 0x46, 0x63, 0xdb, 0x12, 0xcf, 0xe1, 0xd0, 0x6a, 0x8c, 0xff, 0xac, 0x91, 0x04,
0xe4, 0xfb, 0xdd, 0xb9, 0x2e, 0x85, 0x8f, 0x01, 0xab, 0xe7, 0xd0, 0x88, 0xce, 0xb6, 0x6b, 0x8e,
0xf0, 0xae, 0xd9, 0x3f, 0xf4, 0x43, 0xb4, 0xdb, 0x29, 0x6f, 0x8a, 0x92, 0xa0, 0xfe, 0x7c, 0x6f,
0x48, 0xe7, 0x63, 0x6f, 0x90, 0x7c, 0xd1, 0x57, 0xcf, 0xa1, 0x67, 0xb0, 0xf4, 0x00, 0x87, 0xe2,
0x5d, 0x7f, 0xc2, 0xb5, 0xf4, 0x09, 0x13, 0xc0, 0xa7, 0x9c, 0xf2, 0x21, 0x94, 0xa9, 0xb8, 0x21,
0xd9, 0x81, 0x6c, 0xf8, 0x2d, 0x7b, 0xe7, 0x5a, 0x3a, 0x80, 0xc0, 0xf6, 0x03, 0x58, 0x88, 0xbd,
0x72, 0x45, 0x32, 0x1f, 0x29, 0x7f, 0xaf, 0xdc, 0xb9, 0x99, 0x07, 0x54, 0xcc, 0x35, 0x80, 0x66,
0xf4, 0x69, 0x0c, 0x5a, 0xc9, 0xf1, 0xc0, 0x8e, 0xcd, 0xf4, 0x76, 0xee, 0xa7, 0x78, 0x54, 0x08,
0x5a, 0xf1, 0xf7, 0x97, 0xe8, 0x66, 0x26, 0x82, 0xa8, 0xb0, 0xbd, 0x93, 0x0b, 0x56, 0x4c, 0x77,
0x42, 0x85, 0x20, 0xf1, 0xf8, 0x2d, 0x2e, 0xe3, 0x3e, 0x9a, 0xb4, 0x57, 0x79, 0x9d, 0x5b, 0xb9,
0xe1, 0xc5, 0xd4, 0xbf, 0xc9, 0x6e, 0x5d, 0xc9, 0x1e, 0x90, 0xa1, 0xf7, 0xe5, 0xe8, 0x32, 0x5e,
0xbe, 0x75, 0xd6, 0x4e, 0x33, 0x44, 0x10, 0xf1, 0xeb, 0xf4, 0xba, 0x94, 0xe4, 0x09, 0x56, 0x5c,
0xef, 0x7c, 0x7c, 0xe9, 0xaf, 0xcb, 0x3a, 0xef, 0x9f, 0x62, 0x84, 0x20, 0xc0, 0x8e, 0x3f, 0x70,
0xf5, 0xd5, 0xf0, 0xd6, 0x54, 0xa9, 0x39, 0x9b, 0x0e, 0x7e, 0x1f, 0x16, 0x62, 0x51, 0x23, 0xca,
0x1f, 0x59, 0x76, 0xb2, 0x3c, 0x24, 0x53, 0xc9, 0xd8, 0xf5, 0x28, 0x94, 0x22, 0xfd, 0x92, 0x2b,
0x54, 0x9d, 0x9b, 0x79, 0x40, 0xc5, 0x42, 0xc6, 0xb0, 0x18, 0xfb, 0xf8, 0x64, 0x0d, 0xbd, 0x93,
0x7b, 0xb6, 0x27, 0x6b, 0x9d, 0x77, 0xf3, 0xcf, 0xf7, 0x64, 0x4d, 0x3d, 0x87, 0x5c, 0x6a, 0xa0,
0x63, 0x57, 0x6c, 0x50, 0x0a, 0x16, 0xf9, 0x55, 0xa2, 0xce, 0x7b, 0x39, 0xa1, 0xc5, 0x32, 0x8f,
0xe0, 0xbc, 0xe4, 0x26, 0x14, 0x7a, 0x2f, 0x53, 0x3c, 0xe2, 0x57, 0xc0, 0x3a, 0xab, 0x79, 0xc1,
0x43, 0xee, 0xa1, 0xe5, 0xd3, 0x75, 0x6f, 0x38, 0x64, 0xce, 0xff, 0xdd, 0x34, 0xcf, 0x17, 0x01,
0x4b, 0x59, 0x6a, 0x2a, 0xb4, 0x98, 0xf2, 0x57, 0x01, 0xed, 0x1c, 0xd8, 0xc7, 0x34, 0x4a, 0x1b,
0x4c, 0x1c, 0x9d, 0x05, 0x96, 0x69, 0x0e, 0x30, 0x09, 0x9a, 0xa2, 0x88, 0x99, 0x23, 0xc4, 0xe4,
0x3d, 0x80, 0x07, 0xd8, 0xdb, 0xc2, 0x9e, 0x43, 0xb4, 0xff, 0xcd, 0x34, 0xda, 0x39, 0x80, 0x3f,
0xd5, 0x5b, 0x53, 0xe1, 0xc2, 0x0c, 0xdd, 0xd2, 0xad, 0x89, 0x3e, 0x0c, 0xbd, 0x40, 0x91, 0x33,
0x34, 0x0e, 0x96, 0xcd, 0xd0, 0x24, 0xb4, 0x98, 0xf2, 0x58, 0xc4, 0x2f, 0xa1, 0x03, 0xdb, 0xec,
0xf8, 0x25, 0x79, 0x59, 0x28, 0x6e, 0xdb, 0x33, 0xe0, 0xc5, 0xc4, 0x5f, 0x28, 0xf4, 0xfe, 0x5e,
0x0c, 0xe0, 0xa9, 0xe9, 0x1d, 0x6c, 0x0f, 0x75, 0xcb, 0xcd, 0x43, 0x02, 0x05, 0x3c, 0x05, 0x09,
0x1c, 0x5e, 0x90, 0x60, 0xc0, 0x7c, 0xe4, 0x1c, 0x15, 0xc9, 0xde, 0x6d, 0xc8, 0xce, 0x94, 0x3b,
0x2b, 0xd3, 0x01, 0xc5, 0x2c, 0x07, 0x30, 0xef, 0x0b, 0x34, 0x63, 0xee, 0xdb, 0x99, 0x42, 0x1f,
0xe1, 0xeb, 0xcd, 0x3c, 0xa0, 0x62, 0x26, 0x17, 0x50, 0xf2, 0xc0, 0x08, 0xe5, 0x3b, 0x5e, 0xcc,
0x32, 0x3e, 0xe9, 0xa7, 0x50, 0xcc, 0x9e, 0xc7, 0x8e, 0x64, 0xe5, 0xce, 0x42, 0x7a, 0xc2, 0x2c,
0xb5, 0xe7, 0x29, 0x27, 0xbc, 0xea, 0x39, 0xf4, 0x14, 0x2a, 0xfc, 0x9f, 0x68, 0xae, 0x67, 0x17,
0x79, 0x39, 0xf6, 0x1b, 0x53, 0xa0, 0x04, 0xe2, 0x43, 0xb8, 0x90, 0x52, 0xe2, 0x95, 0xc6, 0x19,
0xd9, 0xe5, 0xe0, 0x69, 0x1e, 0x50, 0x4c, 0x96, 0xa8, 0xe0, 0x66, 0x4c, 0x96, 0x56, 0xed, 0x9d,
0x36, 0x59, 0x0f, 0x16, 0x13, 0x15, 0x32, 0xa9, 0x0b, 0x4c, 0xab, 0xa3, 0x4d, 0x9b, 0x60, 0x00,
0xaf, 0x49, 0xab, 0x41, 0xd2, 0xe8, 0x24, 0xab, 0x6e, 0x34, 0x6d, 0xa2, 0x3e, 0x9c, 0x97, 0xd4,
0x80, 0xa4, 0x5e, 0x2e, 0xbd, 0x56, 0x34, 0x6d, 0x92, 0x7d, 0xe8, 0xac, 0x3b, 0xb6, 0x6e, 0xf4,
0x75, 0xd7, 0xa3, 0x75, 0x19, 0x92, 0x2a, 0xfa, 0xe1, 0xa1, 0x3c, 0x77, 0x90, 0x56, 0x6f, 0xa6,
0xcd, 0xb3, 0x07, 0x75, 0xba, 0x95, 0xec, 0xdf, 0x42, 0x90, 0xdc, 0x47, 0x84, 0x20, 0x52, 0x0c,
0x8f, 0x0c, 0x50, 0x08, 0xf5, 0x2e, 0xd4, 0x37, 0xe8, 0xd9, 0x55, 0xd7, 0x32, 0xf0, 0xf3, 0xb8,
0xbf, 0xa2, 0x4f, 0xa6, 0x57, 0x43, 0x00, 0xb9, 0x39, 0x34, 0x4f, 0xa3, 0x76, 0x03, 0x3f, 0x67,
0xfb, 0xbc, 0x22, 0xc3, 0x1b, 0x01, 0x49, 0xc9, 0x72, 0xa4, 0x90, 0x21, 0x4f, 0xbf, 0x14, 0x8e,
0x65, 0xc5, 0x74, 0xb7, 0x52, 0x90, 0x24, 0x20, 0xfd, 0x59, 0x6f, 0xe7, 0x1f, 0x10, 0xf6, 0x0c,
0x3e, 0x5d, 0x5d, 0x7a, 0x70, 0xf6, 0x56, 0x16, 0xe9, 0xe1, 0x00, 0x75, 0x65, 0x3a, 0xa0, 0x98,
0x65, 0x1b, 0x6a, 0x44, 0x3a, 0xd9, 0xf6, 0x5c, 0x97, 0x0d, 0x14, 0x9f, 0xf3, 0x6f, 0xce, 0x26,
0x76, 0xfb, 0x8e, 0xb9, 0xc7, 0x37, 0x5d, 0x4a, 0x4e, 0x04, 0x24, 0x73, 0x73, 0x62, 0x90, 0x82,
0xf2, 0x09, 0x8d, 0x1a, 0x04, 0xeb, 0xb8, 0xa9, 0x7c, 0x6f, 0xda, 0xfe, 0x46, 0xcd, 0xe4, 0x6a,
0x5e, 0x70, 0x31, 0xed, 0xaf, 0xd1, 0x4c, 0x88, 0x7e, 0x5f, 0x9f, 0x98, 0x43, 0x63, 0x9b, 0x5f,
0x63, 0x46, 0xb7, 0xb3, 0x50, 0x45, 0x40, 0x53, 0x03, 0xc0, 0x8c, 0x11, 0x62, 0xfe, 0x5f, 0x86,
0x9a, 0xa8, 0x10, 0x22, 0xd9, 0x3d, 0xbc, 0x78, 0x6d, 0xb2, 0x73, 0x3d, 0x1b, 0x48, 0x60, 0xc6,
0xb0, 0x24, 0xab, 0x07, 0x4a, 0x93, 0xec, 0x8c, 0xc2, 0xe1, 0x14, 0xf9, 0x58, 0xfb, 0xb2, 0x06,
0x55, 0x7f, 0xe0, 0x57, 0x5c, 0xba, 0x7a, 0x05, 0xb5, 0xa4, 0xef, 0xc3, 0x42, 0xec, 0x4f, 0x20,
0xa4, 0x16, 0x5c, 0xfe, 0x47, 0x11, 0xd3, 0x54, 0xed, 0x29, 0xff, 0x8f, 0x42, 0x91, 0xe4, 0xbd,
0x95, 0x56, 0x8f, 0x8a, 0xe7, 0x77, 0x53, 0x10, 0xff, 0xdf, 0x4e, 0x71, 0x1e, 0x01, 0x84, 0x92,
0x9b, 0xec, 0x8b, 0xd4, 0x24, 0x5e, 0x9f, 0xc6, 0xad, 0x91, 0x34, 0x7f, 0x79, 0x3b, 0xcf, 0xa5,
0xd4, 0xf4, 0x08, 0x34, 0x3d, 0x6b, 0x79, 0x0c, 0x8d, 0xf0, 0x13, 0x07, 0x24, 0xfd, 0x47, 0xbc,
0xe4, 0x1b, 0x88, 0x69, 0xab, 0xd8, 0x3a, 0x65, 0x60, 0x3b, 0x05, 0x9d, 0x0b, 0x28, 0x79, 0x52,
0x2e, 0x4d, 0x04, 0x52, 0xcf, 0xe7, 0xa5, 0x89, 0x40, 0xfa, 0xf1, 0x3b, 0x2b, 0x4b, 0xc6, 0x8f,
0x7f, 0xa5, 0x65, 0xc9, 0x94, 0x03, 0x75, 0x69, 0x59, 0x32, 0xed, 0x3c, 0x59, 0x3d, 0xb7, 0x7e,
0xe7, 0xb3, 0xf7, 0x07, 0xa6, 0x77, 0x30, 0xd9, 0x23, 0xab, 0xbf, 0xc5, 0x86, 0xbe, 0x67, 0xda,
0xfc, 0xd7, 0x2d, 0x5f, 0xdc, 0x6f, 0x51, 0x6c, 0xb7, 0x08, 0xb6, 0xf1, 0xde, 0x5e, 0x85, 0xb6,
0xee, 0xfc, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcb, 0xd3, 0x50, 0x51, 0x9f, 0x55, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -5336,6 +5387,7 @@ type DataCoordClient interface {
// Deprecated: use DescribeIndex instead
GetIndexBuildProgress(ctx context.Context, in *indexpb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*indexpb.GetIndexBuildProgressResponse, error)
GcConfirm(ctx context.Context, in *GcConfirmRequest, opts ...grpc.CallOption) (*GcConfirmResponse, error)
ReportDataNodeTtMsgs(ctx context.Context, in *ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
}
type dataCoordClient struct {
@ -5733,6 +5785,15 @@ func (c *dataCoordClient) GcConfirm(ctx context.Context, in *GcConfirmRequest, o
return out, nil
}
func (c *dataCoordClient) ReportDataNodeTtMsgs(ctx context.Context, in *ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
out := new(commonpb.Status)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/ReportDataNodeTtMsgs", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// DataCoordServer is the server API for DataCoord service.
type DataCoordServer interface {
GetComponentStates(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error)
@ -5782,6 +5843,7 @@ type DataCoordServer interface {
// Deprecated: use DescribeIndex instead
GetIndexBuildProgress(context.Context, *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error)
GcConfirm(context.Context, *GcConfirmRequest) (*GcConfirmResponse, error)
ReportDataNodeTtMsgs(context.Context, *ReportDataNodeTtMsgsRequest) (*commonpb.Status, error)
}
// UnimplementedDataCoordServer can be embedded to have forward compatible implementations.
@ -5917,6 +5979,9 @@ func (*UnimplementedDataCoordServer) GetIndexBuildProgress(ctx context.Context,
func (*UnimplementedDataCoordServer) GcConfirm(ctx context.Context, req *GcConfirmRequest) (*GcConfirmResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GcConfirm not implemented")
}
func (*UnimplementedDataCoordServer) ReportDataNodeTtMsgs(ctx context.Context, req *ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method ReportDataNodeTtMsgs not implemented")
}
func RegisterDataCoordServer(s *grpc.Server, srv DataCoordServer) {
s.RegisterService(&_DataCoord_serviceDesc, srv)
@ -6696,6 +6761,24 @@ func _DataCoord_GcConfirm_Handler(srv interface{}, ctx context.Context, dec func
return interceptor(ctx, in, info, handler)
}
func _DataCoord_ReportDataNodeTtMsgs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ReportDataNodeTtMsgsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataCoordServer).ReportDataNodeTtMsgs(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataCoord/ReportDataNodeTtMsgs",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataCoordServer).ReportDataNodeTtMsgs(ctx, req.(*ReportDataNodeTtMsgsRequest))
}
return interceptor(ctx, in, info, handler)
}
var _DataCoord_serviceDesc = grpc.ServiceDesc{
ServiceName: "milvus.proto.data.DataCoord",
HandlerType: (*DataCoordServer)(nil),
@ -6872,6 +6955,10 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{
MethodName: "GcConfirm",
Handler: _DataCoord_GcConfirm_Handler,
},
{
MethodName: "ReportDataNodeTtMsgs",
Handler: _DataCoord_ReportDataNodeTtMsgs_Handler,
},
},
Streams: []grpc.StreamDesc{},
Metadata: "data_coord.proto",

View File

@ -330,6 +330,8 @@ type DataCoord interface {
UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error)
// UpdateChannelCheckpoint updates channel checkpoint in dataCoord.
UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error)
// ReportDataNodeTtMsgs report DataNodeTtMsgs to dataCoord, called by datanode.
ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error)
// SaveImportSegment saves the import segment binlog paths data and then looks for the right DataNode to add the
// segment to that DataNode.

View File

@ -217,3 +217,7 @@ func (m *GrpcDataCoordClient) GetIndexStatistics(ctx context.Context, in *indexp
func (m *GrpcDataCoordClient) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*indexpb.GetIndexBuildProgressResponse, error) {
return &indexpb.GetIndexBuildProgressResponse{}, m.Err
}
func (m *GrpcDataCoordClient) ReportDataNodeTtMsgs(ctx context.Context, in *datapb.ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}

View File

@ -174,7 +174,8 @@ type commonConfig struct {
QueryNodeSubName ParamItem `refreshable:"false"`
// Deprecated: do not use it anymore
DataCoordStatistic ParamItem `refreshable:"true"`
DataCoordStatistic ParamItem `refreshable:"true"`
// Deprecated
DataCoordTimeTick ParamItem `refreshable:"false"`
DataCoordSegmentInfo ParamItem `refreshable:"true"`
DataCoordSubName ParamItem `refreshable:"false"`
@ -2148,6 +2149,7 @@ type dataNodeConfig struct {
MemoryForceSyncSegmentNum ParamItem `refreshable:"true"`
MemoryWatermark ParamItem `refreshable:"true"`
DataNodeTimeTickByRPC ParamItem `refreshable:"false"`
// DataNode send timetick interval per collection
DataNodeTimeTickInterval ParamItem `refreshable:"false"`
@ -2260,6 +2262,14 @@ func (p *dataNodeConfig) init(base *BaseTable) {
}
p.IOConcurrency.Init(base.mgr)
p.DataNodeTimeTickByRPC = ParamItem{
Key: "datanode.timetick.byRPC",
Version: "2.2.9",
PanicIfEmpty: false,
DefaultValue: "true",
}
p.DataNodeTimeTickByRPC.Init(base.mgr)
p.DataNodeTimeTickInterval = ParamItem{
Key: "datanode.timetick.interval",
Version: "2.2.5",