Change WatchDmChannelsRequest proto (#5577)

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
pull/5779/head
congqixia 2021-06-04 09:57:54 +08:00 committed by zhenshan.cao
parent 03a2052343
commit d0d845805b
10 changed files with 276 additions and 283 deletions

View File

@ -173,9 +173,13 @@ func (replica *CollectionSegmentReplica) addSegment(
func (replica *CollectionSegmentReplica) removeSegment(segmentID UniqueID) error { func (replica *CollectionSegmentReplica) removeSegment(segmentID UniqueID) error {
replica.mu.Lock() replica.mu.Lock()
defer replica.mu.Unlock()
delete(replica.segments, segmentID) delete(replica.segments, segmentID)
replica.mu.Unlock()
replica.posMu.Lock()
delete(replica.startPositions, segmentID)
delete(replica.endPositions, segmentID)
replica.posMu.Unlock()
return nil return nil
} }
@ -276,6 +280,15 @@ func (replica *CollectionSegmentReplica) hasCollection(collectionID UniqueID) bo
return ok return ok
} }
// getSegmentsCheckpoints get current open segments checkpoints
func (replica *CollectionSegmentReplica) getSegmentsCheckpoints() {
replica.mu.RLock()
//for segID, segment := range replica.segments {
// if segment
//}
replica.mu.RUnlock()
}
// setStartPositions set segment `Start Position` - means the `startPositions` from the MsgPack when segment is first found // setStartPositions set segment `Start Position` - means the `startPositions` from the MsgPack when segment is first found
func (replica *CollectionSegmentReplica) setStartPositions(segID UniqueID, startPositions []*internalpb.MsgPosition) error { func (replica *CollectionSegmentReplica) setStartPositions(segID UniqueID, startPositions []*internalpb.MsgPosition) error {
replica.posMu.Lock() replica.posMu.Lock()

View File

@ -188,10 +188,10 @@ func (node *DataNode) Init() error {
} }
// NewDataSyncService adds a new dataSyncService for new dmlVchannel and starts dataSyncService. // NewDataSyncService adds a new dataSyncService for new dmlVchannel and starts dataSyncService.
func (node *DataNode) NewDataSyncService(vchanPair *datapb.VchannelPair) error { func (node *DataNode) NewDataSyncService(vchan *datapb.VchannelInfo) error {
node.chanMut.Lock() node.chanMut.Lock()
defer node.chanMut.Unlock() defer node.chanMut.Unlock()
if _, ok := node.vchan2SyncService[vchanPair.GetDmlVchannelName()]; ok { if _, ok := node.vchan2SyncService[vchan.GetChannelName()]; ok {
return nil return nil
} }
@ -201,10 +201,10 @@ func (node *DataNode) NewDataSyncService(vchanPair *datapb.VchannelPair) error {
metaService := newMetaService(node.ctx, replica, node.masterService) metaService := newMetaService(node.ctx, replica, node.masterService)
flushChan := make(chan *flushMsg, 100) flushChan := make(chan *flushMsg, 100)
dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchanPair) dataSyncService := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan)
// TODO metaService using timestamp in DescribeCollection // TODO metaService using timestamp in DescribeCollection
node.vchan2SyncService[vchanPair.GetDmlVchannelName()] = dataSyncService node.vchan2SyncService[vchan.GetChannelName()] = dataSyncService
node.vchan2FlushCh[vchanPair.GetDmlVchannelName()] = flushChan node.vchan2FlushCh[vchan.GetChannelName()] = flushChan
metaService.init() metaService.init()
go dataSyncService.start() go dataSyncService.start()

View File

@ -41,17 +41,15 @@ func TestDataNode(t *testing.T) {
t.Run("Test WatchDmChannels", func(t *testing.T) { t.Run("Test WatchDmChannels", func(t *testing.T) {
node1 := newIDLEDataNodeMock() node1 := newIDLEDataNodeMock()
node1.Start() node1.Start()
vchannels := []*datapb.VchannelPair{} vchannels := []*datapb.VchannelInfo{}
for _, ch := range Params.InsertChannelNames { for _, ch := range Params.InsertChannelNames {
log.Debug("InsertChannels", zap.String("name", ch)) log.Debug("InsertChannels", zap.String("name", ch))
vpair := &datapb.VchannelPair{ vchan := &datapb.VchannelInfo{
CollectionID: 1, CollectionID: 1,
DmlVchannelName: ch, ChannelName: ch,
DdlVchannelName: Params.DDChannelNames[0], CheckPoints: []*datapb.CheckPoint{},
DdlPosition: &datapb.PositionPair{},
DmlPosition: &datapb.PositionPair{},
} }
vchannels = append(vchannels, vpair) vchannels = append(vchannels, vchan)
} }
req := &datapb.WatchDmChannelsRequest{ req := &datapb.WatchDmChannelsRequest{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
@ -92,24 +90,22 @@ func TestDataNode(t *testing.T) {
node2 := newIDLEDataNodeMock() node2 := newIDLEDataNodeMock()
node2.Start() node2.Start()
dmChannelName := "fake-dm-channel-test-NewDataSyncService" dmChannelName := "fake-dm-channel-test-NewDataSyncService"
ddChannelName := "fake-dd-channel-test-NewDataSyncService"
vpair := &datapb.VchannelPair{ vchan := &datapb.VchannelInfo{
CollectionID: 1, CollectionID: 1,
DmlVchannelName: dmChannelName, ChannelName: dmChannelName,
DdlVchannelName: ddChannelName, CheckPoints: []*datapb.CheckPoint{},
DdlPosition: &datapb.PositionPair{},
DmlPosition: &datapb.PositionPair{},
} }
require.Equal(t, 0, len(node2.vchan2FlushCh)) require.Equal(t, 0, len(node2.vchan2FlushCh))
require.Equal(t, 0, len(node2.vchan2SyncService)) require.Equal(t, 0, len(node2.vchan2SyncService))
err := node2.NewDataSyncService(vpair) err := node2.NewDataSyncService(vchan)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, 1, len(node2.vchan2FlushCh)) assert.Equal(t, 1, len(node2.vchan2FlushCh))
assert.Equal(t, 1, len(node2.vchan2SyncService)) assert.Equal(t, 1, len(node2.vchan2SyncService))
err = node2.NewDataSyncService(vpair) err = node2.NewDataSyncService(vchan)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, 1, len(node2.vchan2FlushCh)) assert.Equal(t, 1, len(node2.vchan2FlushCh))
assert.Equal(t, 1, len(node2.vchan2SyncService)) assert.Equal(t, 1, len(node2.vchan2SyncService))

View File

@ -37,7 +37,7 @@ func newDataSyncService(ctx context.Context,
replica Replica, replica Replica,
alloc allocatorInterface, alloc allocatorInterface,
factory msgstream.Factory, factory msgstream.Factory,
vchanPair *datapb.VchannelPair) *dataSyncService { vchan *datapb.VchannelInfo) *dataSyncService {
service := &dataSyncService{ service := &dataSyncService{
ctx: ctx, ctx: ctx,
@ -46,10 +46,10 @@ func newDataSyncService(ctx context.Context,
replica: replica, replica: replica,
idAllocator: alloc, idAllocator: alloc,
msFactory: factory, msFactory: factory,
collectionID: vchanPair.GetCollectionID(), collectionID: vchan.GetCollectionID(),
} }
service.initNodes(vchanPair) service.initNodes(vchan)
return service return service
} }
@ -68,7 +68,7 @@ func (dsService *dataSyncService) close() {
} }
} }
func (dsService *dataSyncService) initNodes(vchanPair *datapb.VchannelPair) { func (dsService *dataSyncService) initNodes(vchanPair *datapb.VchannelInfo) {
// TODO: add delete pipeline support // TODO: add delete pipeline support
dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx) dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx)
@ -83,7 +83,7 @@ func (dsService *dataSyncService) initNodes(vchanPair *datapb.VchannelPair) {
panic(err) panic(err)
} }
var dmStreamNode Node = newDmInputNode(dsService.ctx, dsService.msFactory, vchanPair.GetDmlVchannelName(), vchanPair.GetDmlPosition()) var dmStreamNode Node = newDmInputNode(dsService.ctx, dsService.msFactory, vchanPair.GetChannelName(), vchanPair.GetCheckPoints())
var ddNode Node = newDDNode() var ddNode Node = newDDNode()
var insertBufferNode Node = newInsertBufferNode(dsService.ctx, dsService.replica, dsService.msFactory, dsService.idAllocator, dsService.flushChan) var insertBufferNode Node = newInsertBufferNode(dsService.ctx, dsService.replica, dsService.msFactory, dsService.idAllocator, dsService.flushChan)

View File

@ -53,21 +53,14 @@ func TestDataSyncService_Start(t *testing.T) {
ddlChannelName := "data_sync_service_test_ddl" ddlChannelName := "data_sync_service_test_ddl"
Params.FlushInsertBufferSize = 1 Params.FlushInsertBufferSize = 1
vchanPair := &datapb.VchannelPair{ vchan := &datapb.VchannelInfo{
CollectionID: collMeta.GetID(), CollectionID: collMeta.GetID(),
DmlVchannelName: insertChannelName, ChannelName: insertChannelName,
DdlVchannelName: ddlChannelName, CheckPoints: []*datapb.CheckPoint{},
DmlPosition: &datapb.PositionPair{ FlushedSegments: []int64{},
StartPosition: &internalpb.MsgPosition{},
EndPosition: &internalpb.MsgPosition{},
},
DdlPosition: &datapb.PositionPair{
StartPosition: &internalpb.MsgPosition{},
EndPosition: &internalpb.MsgPosition{},
},
} }
sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchanPair) sync := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan)
sync.replica.addCollection(collMeta.ID, collMeta.Schema) sync.replica.addCollection(collMeta.ID, collMeta.Schema)
go sync.start() go sync.start()

View File

@ -20,7 +20,7 @@ import (
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
) )
func newDmInputNode(ctx context.Context, factory msgstream.Factory, vchannelName string, vchannelPos *datapb.PositionPair) *flowgraph.InputNode { func newDmInputNode(ctx context.Context, factory msgstream.Factory, vchannelName string, checkPoints []*datapb.CheckPoint) *flowgraph.InputNode {
// TODO seek // TODO seek
maxQueueLength := Params.FlowGraphMaxQueueLength maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism maxParallelism := Params.FlowGraphMaxParallelism

View File

@ -103,16 +103,15 @@ func newHEALTHDataNodeMock(dmChannelName, ddChannelName string) *DataNode {
ds := &DataServiceFactory{} ds := &DataServiceFactory{}
node.SetDataServiceInterface(ds) node.SetDataServiceInterface(ds)
vpair := &datapb.VchannelPair{ vchan := &datapb.VchannelInfo{
CollectionID: 1, CollectionID: 1,
DmlVchannelName: dmChannelName, ChannelName: dmChannelName,
DdlVchannelName: ddChannelName, CheckPoints: []*datapb.CheckPoint{},
DdlPosition: &datapb.PositionPair{}, FlushedSegments: []int64{},
DmlPosition: &datapb.PositionPair{},
} }
node.Start() node.Start()
_ = node.NewDataSyncService(vpair) _ = node.NewDataSyncService(vchan)
return node return node
} }

View File

@ -117,7 +117,7 @@ func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo {
}) })
} }
} }
pairs, err := c.posProvider.GetVChanPositions(uncompletes) _, err := c.posProvider.GetVChanPositions(uncompletes)
if err != nil { if err != nil {
log.Warn("get vchannel position failed", zap.Error(err)) log.Warn("get vchannel position failed", zap.Error(err))
continue continue
@ -131,7 +131,8 @@ func (c *cluster) watch(nodes []*datapb.DataNodeInfo) []*datapb.DataNodeInfo {
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
SourceID: Params.NodeID, SourceID: Params.NodeID,
}, },
Vchannels: pairs, // TODO change to VChannel info
// Vchannels: pairs,
} }
resp, err := cli.WatchDmChannels(c.ctx, req) resp, err := cli.WatchDmChannels(c.ctx, req)
if err != nil { if err != nil {

View File

@ -177,9 +177,16 @@ message VchannelPair {
PositionPair dml_position = 5; PositionPair dml_position = 5;
} }
message VchannelInfo {
int64 collectionID = 1;
string channelName = 2;
repeated CheckPoint checkPoints = 3;
repeated int64 flushedSegments = 4;
}
message WatchDmChannelsRequest { message WatchDmChannelsRequest {
common.MsgBase base = 1; common.MsgBase base = 1;
repeated VchannelPair vchannels = 2; repeated VchannelInfo vchannels = 2;
} }
message FlushSegmentsRequest { message FlushSegmentsRequest {
@ -274,13 +281,6 @@ message DataNodeTtMsg {
} }
message VchannelInfo {
int64 collectionID = 1;
string channelName = 2;
internal.MsgPosition seek_position = 3;
repeated CheckPoint checkPoints = 4;
repeated int64 flushedSegments = 5;
}
message SegmentBinlogs { message SegmentBinlogs {
int64 segmentID = 1; int64 segmentID = 1;

View File

@ -1294,9 +1294,72 @@ func (m *VchannelPair) GetDmlPosition() *PositionPair {
return nil return nil
} }
type VchannelInfo struct {
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"`
CheckPoints []*CheckPoint `protobuf:"bytes,3,rep,name=checkPoints,proto3" json:"checkPoints,omitempty"`
FlushedSegments []int64 `protobuf:"varint,4,rep,packed,name=flushedSegments,proto3" json:"flushedSegments,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *VchannelInfo) Reset() { *m = VchannelInfo{} }
func (m *VchannelInfo) String() string { return proto.CompactTextString(m) }
func (*VchannelInfo) ProtoMessage() {}
func (*VchannelInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{23}
}
func (m *VchannelInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_VchannelInfo.Unmarshal(m, b)
}
func (m *VchannelInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_VchannelInfo.Marshal(b, m, deterministic)
}
func (m *VchannelInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_VchannelInfo.Merge(m, src)
}
func (m *VchannelInfo) XXX_Size() int {
return xxx_messageInfo_VchannelInfo.Size(m)
}
func (m *VchannelInfo) XXX_DiscardUnknown() {
xxx_messageInfo_VchannelInfo.DiscardUnknown(m)
}
var xxx_messageInfo_VchannelInfo proto.InternalMessageInfo
func (m *VchannelInfo) GetCollectionID() int64 {
if m != nil {
return m.CollectionID
}
return 0
}
func (m *VchannelInfo) GetChannelName() string {
if m != nil {
return m.ChannelName
}
return ""
}
func (m *VchannelInfo) GetCheckPoints() []*CheckPoint {
if m != nil {
return m.CheckPoints
}
return nil
}
func (m *VchannelInfo) GetFlushedSegments() []int64 {
if m != nil {
return m.FlushedSegments
}
return nil
}
type WatchDmChannelsRequest struct { type WatchDmChannelsRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
Vchannels []*VchannelPair `protobuf:"bytes,2,rep,name=vchannels,proto3" json:"vchannels,omitempty"` Vchannels []*VchannelInfo `protobuf:"bytes,2,rep,name=vchannels,proto3" json:"vchannels,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"` XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"` XXX_sizecache int32 `json:"-"`
@ -1306,7 +1369,7 @@ func (m *WatchDmChannelsRequest) Reset() { *m = WatchDmChannelsRequest{}
func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) } func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) }
func (*WatchDmChannelsRequest) ProtoMessage() {} func (*WatchDmChannelsRequest) ProtoMessage() {}
func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) { func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{23} return fileDescriptor_3385cd32ad6cfe64, []int{24}
} }
func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error { func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error {
@ -1334,7 +1397,7 @@ func (m *WatchDmChannelsRequest) GetBase() *commonpb.MsgBase {
return nil return nil
} }
func (m *WatchDmChannelsRequest) GetVchannels() []*VchannelPair { func (m *WatchDmChannelsRequest) GetVchannels() []*VchannelInfo {
if m != nil { if m != nil {
return m.Vchannels return m.Vchannels
} }
@ -1355,7 +1418,7 @@ func (m *FlushSegmentsRequest) Reset() { *m = FlushSegmentsRequest{} }
func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) } func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) }
func (*FlushSegmentsRequest) ProtoMessage() {} func (*FlushSegmentsRequest) ProtoMessage() {}
func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) { func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{24} return fileDescriptor_3385cd32ad6cfe64, []int{25}
} }
func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error { func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error {
@ -1416,7 +1479,7 @@ func (m *SegmentMsg) Reset() { *m = SegmentMsg{} }
func (m *SegmentMsg) String() string { return proto.CompactTextString(m) } func (m *SegmentMsg) String() string { return proto.CompactTextString(m) }
func (*SegmentMsg) ProtoMessage() {} func (*SegmentMsg) ProtoMessage() {}
func (*SegmentMsg) Descriptor() ([]byte, []int) { func (*SegmentMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{25} return fileDescriptor_3385cd32ad6cfe64, []int{26}
} }
func (m *SegmentMsg) XXX_Unmarshal(b []byte) error { func (m *SegmentMsg) XXX_Unmarshal(b []byte) error {
@ -1464,7 +1527,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{}
func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) ProtoMessage() {}
func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{26} return fileDescriptor_3385cd32ad6cfe64, []int{27}
} }
func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error {
@ -1512,7 +1575,7 @@ func (m *DDLBinlogMeta) Reset() { *m = DDLBinlogMeta{} }
func (m *DDLBinlogMeta) String() string { return proto.CompactTextString(m) } func (m *DDLBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*DDLBinlogMeta) ProtoMessage() {} func (*DDLBinlogMeta) ProtoMessage() {}
func (*DDLBinlogMeta) Descriptor() ([]byte, []int) { func (*DDLBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{27} return fileDescriptor_3385cd32ad6cfe64, []int{28}
} }
func (m *DDLBinlogMeta) XXX_Unmarshal(b []byte) error { func (m *DDLBinlogMeta) XXX_Unmarshal(b []byte) error {
@ -1559,7 +1622,7 @@ func (m *FieldFlushMeta) Reset() { *m = FieldFlushMeta{} }
func (m *FieldFlushMeta) String() string { return proto.CompactTextString(m) } func (m *FieldFlushMeta) String() string { return proto.CompactTextString(m) }
func (*FieldFlushMeta) ProtoMessage() {} func (*FieldFlushMeta) ProtoMessage() {}
func (*FieldFlushMeta) Descriptor() ([]byte, []int) { func (*FieldFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{28} return fileDescriptor_3385cd32ad6cfe64, []int{29}
} }
func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error { func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1607,7 +1670,7 @@ func (m *SegmentFlushMeta) Reset() { *m = SegmentFlushMeta{} }
func (m *SegmentFlushMeta) String() string { return proto.CompactTextString(m) } func (m *SegmentFlushMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFlushMeta) ProtoMessage() {} func (*SegmentFlushMeta) ProtoMessage() {}
func (*SegmentFlushMeta) Descriptor() ([]byte, []int) { func (*SegmentFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{29} return fileDescriptor_3385cd32ad6cfe64, []int{30}
} }
func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error { func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1661,7 +1724,7 @@ func (m *DDLFlushMeta) Reset() { *m = DDLFlushMeta{} }
func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) } func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) }
func (*DDLFlushMeta) ProtoMessage() {} func (*DDLFlushMeta) ProtoMessage() {}
func (*DDLFlushMeta) Descriptor() ([]byte, []int) { func (*DDLFlushMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{30} return fileDescriptor_3385cd32ad6cfe64, []int{31}
} }
func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error { func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error {
@ -1709,7 +1772,7 @@ func (m *CollectionInfo) Reset() { *m = CollectionInfo{} }
func (m *CollectionInfo) String() string { return proto.CompactTextString(m) } func (m *CollectionInfo) String() string { return proto.CompactTextString(m) }
func (*CollectionInfo) ProtoMessage() {} func (*CollectionInfo) ProtoMessage() {}
func (*CollectionInfo) Descriptor() ([]byte, []int) { func (*CollectionInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{31} return fileDescriptor_3385cd32ad6cfe64, []int{32}
} }
func (m *CollectionInfo) XXX_Unmarshal(b []byte) error { func (m *CollectionInfo) XXX_Unmarshal(b []byte) error {
@ -1771,7 +1834,7 @@ func (m *SegmentInfo) Reset() { *m = SegmentInfo{} }
func (m *SegmentInfo) String() string { return proto.CompactTextString(m) } func (m *SegmentInfo) String() string { return proto.CompactTextString(m) }
func (*SegmentInfo) ProtoMessage() {} func (*SegmentInfo) ProtoMessage() {}
func (*SegmentInfo) Descriptor() ([]byte, []int) { func (*SegmentInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{32} return fileDescriptor_3385cd32ad6cfe64, []int{33}
} }
func (m *SegmentInfo) XXX_Unmarshal(b []byte) error { func (m *SegmentInfo) XXX_Unmarshal(b []byte) error {
@ -1874,7 +1937,7 @@ func (m *ID2PathList) Reset() { *m = ID2PathList{} }
func (m *ID2PathList) String() string { return proto.CompactTextString(m) } func (m *ID2PathList) String() string { return proto.CompactTextString(m) }
func (*ID2PathList) ProtoMessage() {} func (*ID2PathList) ProtoMessage() {}
func (*ID2PathList) Descriptor() ([]byte, []int) { func (*ID2PathList) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{33} return fileDescriptor_3385cd32ad6cfe64, []int{34}
} }
func (m *ID2PathList) XXX_Unmarshal(b []byte) error { func (m *ID2PathList) XXX_Unmarshal(b []byte) error {
@ -1921,7 +1984,7 @@ func (m *PositionPair) Reset() { *m = PositionPair{} }
func (m *PositionPair) String() string { return proto.CompactTextString(m) } func (m *PositionPair) String() string { return proto.CompactTextString(m) }
func (*PositionPair) ProtoMessage() {} func (*PositionPair) ProtoMessage() {}
func (*PositionPair) Descriptor() ([]byte, []int) { func (*PositionPair) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{34} return fileDescriptor_3385cd32ad6cfe64, []int{35}
} }
func (m *PositionPair) XXX_Unmarshal(b []byte) error { func (m *PositionPair) XXX_Unmarshal(b []byte) error {
@ -1973,7 +2036,7 @@ func (m *SaveBinlogPathsRequest) Reset() { *m = SaveBinlogPathsRequest{}
func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) } func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) }
func (*SaveBinlogPathsRequest) ProtoMessage() {} func (*SaveBinlogPathsRequest) ProtoMessage() {}
func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) { func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{35} return fileDescriptor_3385cd32ad6cfe64, []int{36}
} }
func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error { func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error {
@ -2057,7 +2120,7 @@ func (m *CheckPoint) Reset() { *m = CheckPoint{} }
func (m *CheckPoint) String() string { return proto.CompactTextString(m) } func (m *CheckPoint) String() string { return proto.CompactTextString(m) }
func (*CheckPoint) ProtoMessage() {} func (*CheckPoint) ProtoMessage() {}
func (*CheckPoint) Descriptor() ([]byte, []int) { func (*CheckPoint) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{36} return fileDescriptor_3385cd32ad6cfe64, []int{37}
} }
func (m *CheckPoint) XXX_Unmarshal(b []byte) error { func (m *CheckPoint) XXX_Unmarshal(b []byte) error {
@ -2119,7 +2182,7 @@ func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} }
func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) } func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) }
func (*DataNodeTtMsg) ProtoMessage() {} func (*DataNodeTtMsg) ProtoMessage() {}
func (*DataNodeTtMsg) Descriptor() ([]byte, []int) { func (*DataNodeTtMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{37} return fileDescriptor_3385cd32ad6cfe64, []int{38}
} }
func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error { func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error {
@ -2161,77 +2224,6 @@ func (m *DataNodeTtMsg) GetTimestamp() uint64 {
return 0 return 0
} }
type VchannelInfo struct {
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"`
SeekPosition *internalpb.MsgPosition `protobuf:"bytes,3,opt,name=seek_position,json=seekPosition,proto3" json:"seek_position,omitempty"`
CheckPoints []*CheckPoint `protobuf:"bytes,4,rep,name=checkPoints,proto3" json:"checkPoints,omitempty"`
FlushedSegments []int64 `protobuf:"varint,5,rep,packed,name=flushedSegments,proto3" json:"flushedSegments,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *VchannelInfo) Reset() { *m = VchannelInfo{} }
func (m *VchannelInfo) String() string { return proto.CompactTextString(m) }
func (*VchannelInfo) ProtoMessage() {}
func (*VchannelInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3385cd32ad6cfe64, []int{38}
}
func (m *VchannelInfo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_VchannelInfo.Unmarshal(m, b)
}
func (m *VchannelInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_VchannelInfo.Marshal(b, m, deterministic)
}
func (m *VchannelInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_VchannelInfo.Merge(m, src)
}
func (m *VchannelInfo) XXX_Size() int {
return xxx_messageInfo_VchannelInfo.Size(m)
}
func (m *VchannelInfo) XXX_DiscardUnknown() {
xxx_messageInfo_VchannelInfo.DiscardUnknown(m)
}
var xxx_messageInfo_VchannelInfo proto.InternalMessageInfo
func (m *VchannelInfo) GetCollectionID() int64 {
if m != nil {
return m.CollectionID
}
return 0
}
func (m *VchannelInfo) GetChannelName() string {
if m != nil {
return m.ChannelName
}
return ""
}
func (m *VchannelInfo) GetSeekPosition() *internalpb.MsgPosition {
if m != nil {
return m.SeekPosition
}
return nil
}
func (m *VchannelInfo) GetCheckPoints() []*CheckPoint {
if m != nil {
return m.CheckPoints
}
return nil
}
func (m *VchannelInfo) GetFlushedSegments() []int64 {
if m != nil {
return m.FlushedSegments
}
return nil
}
type SegmentBinlogs struct { type SegmentBinlogs struct {
SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"` FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"`
@ -2571,6 +2563,7 @@ func init() {
proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse") proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse")
proto.RegisterType((*GetSegmentInfoChannelRequest)(nil), "milvus.proto.data.GetSegmentInfoChannelRequest") proto.RegisterType((*GetSegmentInfoChannelRequest)(nil), "milvus.proto.data.GetSegmentInfoChannelRequest")
proto.RegisterType((*VchannelPair)(nil), "milvus.proto.data.VchannelPair") proto.RegisterType((*VchannelPair)(nil), "milvus.proto.data.VchannelPair")
proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo")
proto.RegisterType((*WatchDmChannelsRequest)(nil), "milvus.proto.data.WatchDmChannelsRequest") proto.RegisterType((*WatchDmChannelsRequest)(nil), "milvus.proto.data.WatchDmChannelsRequest")
proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest") proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest")
proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg") proto.RegisterType((*SegmentMsg)(nil), "milvus.proto.data.SegmentMsg")
@ -2586,7 +2579,6 @@ func init() {
proto.RegisterType((*SaveBinlogPathsRequest)(nil), "milvus.proto.data.SaveBinlogPathsRequest") proto.RegisterType((*SaveBinlogPathsRequest)(nil), "milvus.proto.data.SaveBinlogPathsRequest")
proto.RegisterType((*CheckPoint)(nil), "milvus.proto.data.CheckPoint") proto.RegisterType((*CheckPoint)(nil), "milvus.proto.data.CheckPoint")
proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg") proto.RegisterType((*DataNodeTtMsg)(nil), "milvus.proto.data.DataNodeTtMsg")
proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo")
proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs") proto.RegisterType((*SegmentBinlogs)(nil), "milvus.proto.data.SegmentBinlogs")
proto.RegisterType((*FieldBinlog)(nil), "milvus.proto.data.FieldBinlog") proto.RegisterType((*FieldBinlog)(nil), "milvus.proto.data.FieldBinlog")
proto.RegisterType((*ChannelStatus)(nil), "milvus.proto.data.ChannelStatus") proto.RegisterType((*ChannelStatus)(nil), "milvus.proto.data.ChannelStatus")
@ -2598,142 +2590,141 @@ func init() {
func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) } func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
var fileDescriptor_3385cd32ad6cfe64 = []byte{ var fileDescriptor_3385cd32ad6cfe64 = []byte{
// 2158 bytes of a gzipped FileDescriptorProto // 2138 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xeb, 0x6e, 0x1b, 0xc7, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0xdb, 0x6e, 0x1b, 0xc7,
0xf5, 0xd7, 0x92, 0xba, 0xf1, 0x70, 0x49, 0x49, 0x63, 0xfd, 0x15, 0xfe, 0x69, 0x5b, 0x96, 0xb7, 0x55, 0x4b, 0xea, 0x42, 0x1e, 0x5e, 0x24, 0x8d, 0x55, 0x85, 0xa5, 0x6d, 0x59, 0xde, 0x26, 0x8e,
0x89, 0xa3, 0x28, 0xa8, 0x14, 0xcb, 0x45, 0x6f, 0x6e, 0x5a, 0x58, 0x66, 0xac, 0x12, 0xb5, 0x0c, 0xa2, 0xa0, 0x52, 0x4c, 0x17, 0xbd, 0xb9, 0x69, 0x61, 0x99, 0xb1, 0x4a, 0xd4, 0x32, 0xd4, 0x91,
0x75, 0xe4, 0x38, 0x40, 0x83, 0x82, 0x58, 0x71, 0x47, 0xd4, 0x56, 0xdc, 0x5d, 0x66, 0x67, 0x29, 0xe3, 0x00, 0x0d, 0x0a, 0x62, 0xc5, 0x1d, 0x51, 0x5b, 0x73, 0x77, 0x99, 0x9d, 0xa5, 0x2c, 0x3f,
0xcb, 0x9f, 0x1c, 0x24, 0xfd, 0xd0, 0x16, 0x45, 0xd3, 0xf6, 0x09, 0xda, 0x00, 0x05, 0x0a, 0x14, 0x39, 0x48, 0xfa, 0xd2, 0xa2, 0x68, 0xda, 0x7e, 0x41, 0x1b, 0xa0, 0x40, 0x81, 0x02, 0x45, 0xff,
0x28, 0xfa, 0x12, 0x7d, 0x85, 0xbe, 0x4e, 0x8b, 0xb9, 0xec, 0x7d, 0x48, 0xae, 0xa8, 0x18, 0xfa, 0xa0, 0x4f, 0xfd, 0x85, 0xfe, 0x4e, 0x83, 0xb9, 0xec, 0x7d, 0x48, 0xae, 0x28, 0x1b, 0x7a, 0xdb,
0xb6, 0x33, 0x7b, 0x6e, 0x73, 0xe6, 0xcc, 0xef, 0x9c, 0x39, 0x03, 0xc8, 0x32, 0x03, 0xb3, 0x43, 0x99, 0x3d, 0xb7, 0x39, 0xf7, 0x39, 0x03, 0xc8, 0x34, 0x7c, 0xa3, 0x4b, 0x89, 0x77, 0x66, 0xf5,
0x89, 0x7f, 0x6e, 0x77, 0xc9, 0xf6, 0xc0, 0xf7, 0x02, 0x0f, 0xad, 0x38, 0x76, 0xff, 0x7c, 0x48, 0xc8, 0xce, 0xd0, 0x73, 0x7d, 0x17, 0xad, 0xda, 0xd6, 0xe0, 0x6c, 0x44, 0xc5, 0x6a, 0x87, 0x01,
0xc5, 0x68, 0x9b, 0x11, 0x34, 0xf5, 0xae, 0xe7, 0x38, 0x9e, 0x2b, 0xa6, 0x9a, 0x75, 0xdb, 0x0d, 0x34, 0xab, 0x3d, 0xd7, 0xb6, 0x5d, 0x47, 0x6c, 0x35, 0xeb, 0x96, 0xe3, 0x13, 0xcf, 0x31, 0x06,
0x88, 0xef, 0x9a, 0x7d, 0x39, 0xd6, 0x93, 0x0c, 0x4d, 0x9d, 0x76, 0x4f, 0x89, 0x63, 0x8a, 0x91, 0x72, 0x5d, 0x8d, 0x23, 0x34, 0xab, 0xb4, 0x77, 0x4a, 0x6c, 0x43, 0xac, 0xf4, 0x57, 0x70, 0x0d,
0xf1, 0x1a, 0x6e, 0x60, 0xd2, 0xb3, 0x69, 0x40, 0xfc, 0x67, 0x9e, 0x45, 0x30, 0xf9, 0x6c, 0x48, 0x93, 0xbe, 0x45, 0x7d, 0xe2, 0x3d, 0x71, 0x4d, 0x82, 0xc9, 0x67, 0x23, 0x42, 0x7d, 0xf4, 0x01,
0x68, 0x80, 0x3e, 0x80, 0xd9, 0x63, 0x93, 0x92, 0x86, 0xb6, 0xa1, 0x6d, 0x56, 0x77, 0x6f, 0x6d, 0xcc, 0x1f, 0x1b, 0x94, 0x34, 0xb4, 0x4d, 0x6d, 0xab, 0xd2, 0xba, 0xb1, 0x93, 0x60, 0x29, 0x99,
0xa7, 0x54, 0x4a, 0x65, 0x07, 0xb4, 0xb7, 0x67, 0x52, 0x82, 0x39, 0x25, 0xfa, 0x2e, 0x2c, 0x98, 0x1d, 0xd0, 0xfe, 0x9e, 0x41, 0x09, 0xe6, 0x90, 0xe8, 0xfb, 0xb0, 0x64, 0x98, 0xa6, 0x47, 0x28,
0x96, 0xe5, 0x13, 0x4a, 0x1b, 0xa5, 0x31, 0x4c, 0x8f, 0x04, 0x0d, 0x0e, 0x89, 0x8d, 0xaf, 0x34, 0x6d, 0x14, 0x26, 0x20, 0x3d, 0x10, 0x30, 0x38, 0x00, 0xd6, 0xbf, 0xd2, 0x60, 0x2d, 0x29, 0x01,
0x58, 0x4d, 0x5b, 0x40, 0x07, 0x9e, 0x4b, 0x09, 0xda, 0x83, 0xaa, 0xed, 0xda, 0x41, 0x67, 0x60, 0x1d, 0xba, 0x0e, 0x25, 0x68, 0x0f, 0x2a, 0x96, 0x63, 0xf9, 0xdd, 0xa1, 0xe1, 0x19, 0x36, 0x95,
0xfa, 0xa6, 0x43, 0xa5, 0x25, 0x77, 0xd3, 0x42, 0xa3, 0x85, 0xb6, 0x5d, 0x3b, 0x38, 0xe4, 0x84, 0x92, 0xdc, 0x4e, 0x12, 0x0d, 0x0f, 0xda, 0x71, 0x2c, 0xff, 0x90, 0x03, 0x62, 0xb0, 0xc2, 0x6f,
0x18, 0xec, 0xe8, 0x1b, 0x3d, 0x80, 0x79, 0x1a, 0x98, 0xc1, 0x30, 0xb4, 0xe9, 0xa6, 0xd2, 0xa6, 0x74, 0x0f, 0x16, 0xa9, 0x6f, 0xf8, 0xa3, 0x40, 0xa6, 0xeb, 0x4a, 0x99, 0x8e, 0x38, 0x08, 0x96,
0x23, 0x4e, 0x82, 0x25, 0xa9, 0x71, 0x01, 0xfa, 0x93, 0xfe, 0x90, 0x9e, 0x4e, 0xef, 0x0b, 0x04, 0xa0, 0xfa, 0x39, 0x54, 0x1f, 0x0d, 0x46, 0xf4, 0x74, 0x76, 0x5d, 0x20, 0x98, 0x37, 0x8f, 0x3b,
0xb3, 0xd6, 0x71, 0xbb, 0xc5, 0x95, 0x96, 0x31, 0xff, 0x46, 0x06, 0xe8, 0x5d, 0xaf, 0xdf, 0x27, 0x6d, 0xce, 0xb4, 0x88, 0xf9, 0x37, 0xd2, 0xa1, 0xda, 0x73, 0x07, 0x03, 0xd2, 0xf3, 0x2d, 0xd7,
0xdd, 0xc0, 0xf6, 0xdc, 0x76, 0xab, 0x31, 0xcb, 0xff, 0xa5, 0xe6, 0x8c, 0x3f, 0x69, 0xb0, 0x7c, 0xe9, 0xb4, 0x1b, 0xf3, 0xfc, 0x5f, 0x62, 0x4f, 0xff, 0xb3, 0x06, 0x2b, 0x47, 0xa4, 0x6f, 0x13,
0x44, 0x7a, 0x0e, 0x71, 0x83, 0x76, 0x2b, 0x54, 0xbf, 0x0a, 0x73, 0x5d, 0x6f, 0xe8, 0x06, 0x5c, 0xc7, 0xef, 0xb4, 0x03, 0xf6, 0x6b, 0xb0, 0xd0, 0x73, 0x47, 0x8e, 0xcf, 0xf9, 0xd7, 0xb0, 0x58,
0x7f, 0x0d, 0x8b, 0x01, 0xba, 0x0b, 0x7a, 0xf7, 0xd4, 0x74, 0x5d, 0xd2, 0xef, 0xb8, 0xa6, 0x43, 0xa0, 0xdb, 0x50, 0xed, 0x9d, 0x1a, 0x8e, 0x43, 0x06, 0x5d, 0xc7, 0xb0, 0x09, 0x67, 0x55, 0xc6,
0xb8, 0xaa, 0x0a, 0xae, 0xca, 0xb9, 0x67, 0xa6, 0x43, 0x72, 0x1a, 0xcb, 0x79, 0x8d, 0x68, 0x03, 0x15, 0xb9, 0xf7, 0xc4, 0xb0, 0x49, 0x86, 0x63, 0x31, 0xcb, 0x11, 0x6d, 0x42, 0x65, 0x68, 0x78,
0xaa, 0x03, 0xd3, 0x0f, 0xec, 0x94, 0x51, 0xc9, 0x29, 0xe3, 0x2f, 0x1a, 0xac, 0x3d, 0xa2, 0xd4, 0xbe, 0x95, 0x10, 0x2a, 0xbe, 0xa5, 0xff, 0x55, 0x83, 0xf5, 0x07, 0x94, 0x5a, 0x7d, 0x27, 0x23,
0xee, 0xb9, 0x39, 0xcb, 0xd6, 0x60, 0xde, 0xf5, 0x2c, 0xd2, 0x6e, 0x71, 0xd3, 0xca, 0x58, 0x8e, 0xd9, 0x3a, 0x2c, 0x3a, 0xae, 0x49, 0x3a, 0x6d, 0x2e, 0x5a, 0x11, 0xcb, 0x15, 0xba, 0x0e, 0xe5,
0xd0, 0x4d, 0xa8, 0x0c, 0x08, 0xf1, 0x3b, 0xbe, 0xd7, 0x0f, 0x0d, 0x5b, 0x64, 0x13, 0xd8, 0xeb, 0x21, 0x21, 0x5e, 0xd7, 0x73, 0x07, 0x81, 0x60, 0x25, 0xb6, 0x81, 0xdd, 0x01, 0x41, 0xbf, 0x84,
0x13, 0xf4, 0x73, 0x58, 0xa1, 0x19, 0x41, 0xb4, 0x51, 0xde, 0x28, 0x6f, 0x56, 0x77, 0xbf, 0xb5, 0x55, 0x9a, 0x22, 0x44, 0x1b, 0xc5, 0xcd, 0xe2, 0x56, 0xa5, 0xf5, 0x9d, 0x9d, 0x8c, 0x67, 0xef,
0x9d, 0x8b, 0xec, 0xed, 0xac, 0x52, 0x9c, 0xe7, 0x36, 0x3e, 0x2f, 0xc1, 0x8d, 0x88, 0x4e, 0xd8, 0xa4, 0x99, 0xe2, 0x2c, 0xb6, 0xfe, 0x79, 0x01, 0xae, 0x85, 0x70, 0x42, 0x56, 0xf6, 0xcd, 0x34,
0xca, 0xbe, 0x99, 0xe7, 0x28, 0xe9, 0x45, 0xe6, 0x89, 0x41, 0x11, 0xcf, 0x45, 0x2e, 0x2f, 0x27, 0x47, 0x49, 0x3f, 0x14, 0x4f, 0x2c, 0xf2, 0x68, 0x2e, 0x54, 0x79, 0x31, 0xae, 0xf2, 0x1c, 0x16,
0x5d, 0x5e, 0x60, 0x07, 0xb3, 0xfe, 0x9c, 0xcb, 0xf9, 0x13, 0xdd, 0x81, 0x2a, 0xb9, 0x18, 0xd8, 0x4c, 0xeb, 0x73, 0x21, 0xa3, 0x4f, 0x74, 0x0b, 0x2a, 0xe4, 0x7c, 0x68, 0x79, 0xa4, 0xeb, 0x5b,
0x3e, 0xe9, 0x04, 0xb6, 0x43, 0x1a, 0xf3, 0x1b, 0xda, 0xe6, 0x2c, 0x06, 0x31, 0xf5, 0xdc, 0x76, 0x36, 0x69, 0x2c, 0x6e, 0x6a, 0x5b, 0xf3, 0x18, 0xc4, 0xd6, 0x53, 0xcb, 0x26, 0x31, 0x9f, 0x5d,
0x48, 0x22, 0x66, 0x17, 0x8a, 0xc7, 0xec, 0xd7, 0x1a, 0xbc, 0x95, 0xdb, 0x25, 0x79, 0x90, 0x30, 0xca, 0xef, 0xb3, 0x5f, 0x6b, 0xf0, 0x56, 0xc6, 0x4a, 0x32, 0x90, 0x30, 0xac, 0xf0, 0x93, 0x47,
0x2c, 0xf3, 0x95, 0xc7, 0x9e, 0x61, 0xa7, 0x89, 0x39, 0xfc, 0xde, 0x38, 0x87, 0xc7, 0xe4, 0x38, 0x9a, 0x61, 0xd1, 0xc4, 0x14, 0x7e, 0x67, 0x92, 0xc2, 0x23, 0x70, 0x9c, 0xc1, 0x9f, 0x2d, 0xb0,
0xc7, 0x3f, 0xdd, 0xc1, 0xfa, 0xab, 0x06, 0x37, 0x8e, 0x4e, 0xbd, 0x97, 0x52, 0x05, 0x9d, 0xfe, 0xfe, 0xa6, 0xc1, 0xb5, 0xa3, 0x53, 0xf7, 0x85, 0x64, 0x41, 0x67, 0x0f, 0xb0, 0xb4, 0x29, 0x0a,
0x80, 0x65, 0xb7, 0xa2, 0x34, 0x79, 0x2b, 0xca, 0xf9, 0xad, 0x08, 0x8f, 0xe9, 0x6c, 0x7c, 0x4c, 0xd3, 0x4d, 0x51, 0xcc, 0x9a, 0x22, 0x08, 0xd3, 0xf9, 0x28, 0x4c, 0xf5, 0xe7, 0xb0, 0x96, 0x14,
0x8d, 0x33, 0x58, 0x4d, 0x9b, 0x28, 0x9d, 0xb8, 0x0e, 0x10, 0x05, 0x9e, 0x70, 0x5f, 0x19, 0x27, 0x51, 0x2a, 0x71, 0x03, 0x20, 0x74, 0x3c, 0xa1, 0xbe, 0x22, 0x8e, 0xed, 0xcc, 0xa6, 0x90, 0xe7,
0x66, 0xa6, 0x73, 0xc8, 0x19, 0xbc, 0xb5, 0x4f, 0x02, 0xa9, 0x8b, 0xfd, 0x23, 0x57, 0xf0, 0x49, 0xf0, 0xd6, 0x3e, 0xf1, 0x25, 0x2f, 0xf6, 0x8f, 0x5c, 0x42, 0x27, 0x49, 0x09, 0x0b, 0x69, 0x09,
0xda, 0xc2, 0x52, 0xd6, 0x42, 0xe3, 0x5f, 0xa5, 0x08, 0x5c, 0xb8, 0xaa, 0xb6, 0x7b, 0xe2, 0xa1, 0xf5, 0x7f, 0x17, 0xc2, 0xe4, 0xc2, 0x59, 0x75, 0x9c, 0x13, 0x17, 0xdd, 0x80, 0x72, 0x08, 0x22,
0x5b, 0x50, 0x89, 0x48, 0xe4, 0x31, 0x89, 0x27, 0xd0, 0xf7, 0x60, 0x8e, 0x59, 0x2a, 0xce, 0x48, 0xc3, 0x24, 0xda, 0x40, 0x3f, 0x80, 0x05, 0x26, 0xa9, 0x88, 0x91, 0x7a, 0x3a, 0xf9, 0x06, 0x67,
0x3d, 0x0b, 0xbe, 0xe1, 0x9a, 0x12, 0x32, 0xb1, 0xa0, 0x47, 0x6d, 0xa8, 0xd3, 0xc0, 0xf4, 0x83, 0x8a, 0xd1, 0xc4, 0x02, 0x1e, 0x75, 0xa0, 0x4e, 0x7d, 0xc3, 0xf3, 0xbb, 0x43, 0x97, 0x72, 0x6d,
0xce, 0xc0, 0xa3, 0xdc, 0xdb, 0xdc, 0xfd, 0xd5, 0x5d, 0x63, 0x04, 0x7c, 0x1f, 0xd0, 0xde, 0xa1, 0x73, 0xf5, 0x57, 0x5a, 0xfa, 0x98, 0xf4, 0x7d, 0x40, 0xfb, 0x87, 0x12, 0x12, 0xd7, 0x38, 0x66,
0xa4, 0xc4, 0x35, 0xce, 0x19, 0x0e, 0xd1, 0x47, 0xa0, 0x13, 0xd7, 0x8a, 0x05, 0xcd, 0x16, 0x16, 0xb0, 0x44, 0x1f, 0x41, 0x95, 0x38, 0x66, 0x44, 0x68, 0x3e, 0x37, 0xa1, 0x0a, 0x71, 0xcc, 0x90,
0x54, 0x25, 0xae, 0x15, 0x89, 0x89, 0xf7, 0x67, 0xae, 0xf8, 0xfe, 0xfc, 0x5e, 0x83, 0x46, 0x7e, 0x4c, 0x64, 0x9f, 0x85, 0xfc, 0xf6, 0xf9, 0x83, 0x06, 0x8d, 0xac, 0x81, 0xa4, 0x47, 0x44, 0x14,
0x83, 0x64, 0x44, 0xc4, 0x12, 0xb5, 0xc2, 0x12, 0xd1, 0x43, 0xc1, 0x44, 0xc4, 0x06, 0x8d, 0x85, 0xb5, 0xdc, 0x14, 0xd1, 0x7d, 0x81, 0x44, 0x84, 0x81, 0x26, 0xa6, 0xbc, 0xd0, 0x48, 0x58, 0xa2,
0xbc, 0x68, 0x93, 0xb0, 0x64, 0x31, 0x6c, 0xf8, 0xbf, 0xd8, 0x1a, 0xfe, 0xe7, 0x8d, 0x05, 0xcb, 0xe8, 0x16, 0x7c, 0x2b, 0x92, 0x86, 0xff, 0x79, 0x63, 0xce, 0xf2, 0xa5, 0x06, 0xeb, 0x69, 0x5e,
0x97, 0x1a, 0xac, 0x65, 0x75, 0x5d, 0x65, 0xdd, 0xdf, 0x81, 0x39, 0xdb, 0x3d, 0xf1, 0xc2, 0x65, 0x97, 0x39, 0xf7, 0xf7, 0x60, 0xc1, 0x72, 0x4e, 0xdc, 0xe0, 0xd8, 0x1b, 0x13, 0x12, 0x0f, 0xe3,
0xaf, 0x8f, 0x01, 0x1e, 0xa6, 0x4b, 0x10, 0x1b, 0x0e, 0xdc, 0xdc, 0x27, 0x41, 0xdb, 0xa5, 0xc4, 0x25, 0x80, 0x75, 0x1b, 0xae, 0xef, 0x13, 0xbf, 0xe3, 0x50, 0xe2, 0xf9, 0x7b, 0x96, 0x33, 0x70,
0x0f, 0xf6, 0x6c, 0xb7, 0xef, 0xf5, 0x0e, 0xcd, 0xe0, 0xf4, 0x0a, 0x67, 0x24, 0x15, 0xee, 0xa5, 0xfb, 0x87, 0x86, 0x7f, 0x7a, 0x89, 0x18, 0x49, 0xb8, 0x7b, 0x21, 0xe5, 0xee, 0xfa, 0x3f, 0x34,
0x4c, 0xb8, 0x1b, 0x7f, 0xd7, 0xe0, 0x96, 0x5a, 0x9f, 0x5c, 0x7a, 0x13, 0x16, 0x4f, 0x6c, 0xd2, 0xb8, 0xa1, 0xe6, 0x27, 0x8f, 0xde, 0x84, 0xd2, 0x89, 0x45, 0x06, 0x66, 0x94, 0x02, 0xc2, 0x35,
0xb7, 0x62, 0x08, 0x88, 0xc6, 0xec, 0xac, 0x0c, 0x18, 0xb1, 0x5c, 0xe1, 0xa8, 0x42, 0xe5, 0x28, 0x8b, 0x95, 0x21, 0x03, 0x96, 0x27, 0x1c, 0xd7, 0xa8, 0x1c, 0xf9, 0x9e, 0xe5, 0xf4, 0x1f, 0x5b,
0xf0, 0x6d, 0xb7, 0xf7, 0xd4, 0xa6, 0x01, 0x16, 0xf4, 0x09, 0x7f, 0x96, 0x8b, 0x47, 0xe6, 0xaf, 0xd4, 0xc7, 0x02, 0x3e, 0xa6, 0xcf, 0x62, 0x7e, 0xcf, 0xfc, 0xad, 0xf0, 0x4c, 0x21, 0xea, 0x43,
0x45, 0x64, 0x0a, 0x53, 0x1f, 0x8b, 0xd4, 0x45, 0xdf, 0x6c, 0xc1, 0xa2, 0x28, 0x1f, 0x8c, 0xdf, 0x51, 0xba, 0xe8, 0x9b, 0x6d, 0x58, 0x14, 0xed, 0x83, 0xfe, 0x7b, 0x0d, 0x36, 0xf6, 0x89, 0xff,
0x69, 0xb0, 0xbe, 0x4f, 0x82, 0xc7, 0xd1, 0x1c, 0x33, 0xd3, 0xa6, 0x81, 0xdd, 0xbd, 0x06, 0x63, 0x30, 0xdc, 0x63, 0x62, 0x5a, 0xd4, 0xb7, 0x7a, 0x57, 0x20, 0xcc, 0x57, 0x1a, 0xdc, 0x1a, 0x2b,
0xbe, 0xd2, 0xe0, 0xce, 0x48, 0x63, 0xe4, 0x0e, 0x4a, 0x44, 0x0b, 0x13, 0xa0, 0x1a, 0xd1, 0x7e, 0x8c, 0xb4, 0xa0, 0xcc, 0x68, 0x41, 0x01, 0x54, 0x67, 0xb4, 0x5f, 0x90, 0x97, 0xcf, 0x8c, 0xc1,
0x46, 0x5e, 0xbd, 0x30, 0xfb, 0x43, 0x72, 0x68, 0xda, 0xbe, 0x40, 0xb4, 0x29, 0xf1, 0xfd, 0x1f, 0x88, 0x1c, 0x1a, 0x96, 0x27, 0x32, 0xda, 0x8c, 0xf9, 0xfd, 0x9f, 0x1a, 0xdc, 0xdc, 0x27, 0xac,
0x1a, 0xdc, 0xde, 0x27, 0xac, 0x18, 0x15, 0x39, 0xe7, 0x1a, 0xbd, 0x53, 0xa0, 0xd2, 0xfb, 0x83, 0x19, 0x15, 0x35, 0xe7, 0x0a, 0xb5, 0x93, 0xa3, 0xd3, 0xfb, 0xa3, 0x30, 0xa6, 0x52, 0xda, 0x2b,
0xd8, 0x4c, 0xa5, 0xb5, 0xd7, 0xe2, 0xbe, 0x75, 0x7e, 0x1c, 0x13, 0xb8, 0x20, 0x03, 0x5d, 0x3a, 0x51, 0xdf, 0x06, 0x0f, 0xc7, 0x58, 0x5e, 0x90, 0x8e, 0x2e, 0x95, 0xa7, 0xff, 0xa5, 0x00, 0xd5,
0xcf, 0xf8, 0x73, 0x09, 0xf4, 0x17, 0xb2, 0x6e, 0x63, 0xca, 0x72, 0x7e, 0xd0, 0x14, 0x7e, 0xd8, 0x67, 0xb2, 0x6f, 0x63, 0xcc, 0x32, 0x7a, 0xd0, 0x14, 0x7a, 0xd8, 0x86, 0x55, 0xd3, 0x1e, 0x74,
0x82, 0x15, 0xcb, 0xe9, 0x77, 0xce, 0x15, 0x35, 0xe0, 0x92, 0xe5, 0xf4, 0x5f, 0x24, 0xeb, 0x40, 0xcf, 0x14, 0x3d, 0xe0, 0xb2, 0x69, 0x0f, 0x9e, 0xc5, 0xfb, 0x40, 0x06, 0x6b, 0xa6, 0x61, 0x8b,
0x46, 0x6b, 0x65, 0x69, 0xcb, 0x92, 0xd6, 0x4a, 0xd3, 0xee, 0x81, 0xce, 0x68, 0x33, 0x79, 0xea, 0x12, 0xd6, 0x4c, 0xc2, 0xee, 0x41, 0x95, 0xc1, 0xa6, 0xea, 0xd4, 0x2d, 0x45, 0xa2, 0x0b, 0x6a,
0x8e, 0x02, 0xe8, 0xc2, 0x9c, 0xc4, 0xfd, 0x53, 0xb5, 0xac, 0x7e, 0x94, 0xa4, 0x98, 0x0c, 0x27, 0x12, 0xd7, 0x4f, 0xc5, 0x34, 0x07, 0x61, 0x91, 0x62, 0x34, 0xec, 0x18, 0x8d, 0x85, 0xbc, 0x34,
0x21, 0x63, 0xae, 0xa8, 0x0c, 0x27, 0x92, 0x61, 0xfc, 0x56, 0x83, 0xb5, 0x4f, 0xcc, 0xa0, 0x7b, 0xec, 0x90, 0x86, 0xfe, 0x1f, 0x2d, 0x52, 0x0a, 0x2f, 0xf1, 0x79, 0x94, 0xb2, 0x09, 0xf1, 0xfe,
0xda, 0x72, 0xae, 0x8e, 0x0b, 0x1f, 0x42, 0x25, 0x5c, 0x7c, 0x08, 0x6c, 0x2a, 0x6b, 0x92, 0x9b, 0x57, 0xd5, 0x12, 0xff, 0x8c, 0x41, 0x90, 0xde, 0xf3, 0x43, 0xd7, 0x72, 0xc2, 0x86, 0xfd, 0xa6,
0x80, 0x63, 0x0e, 0x76, 0x77, 0x58, 0xe5, 0x57, 0xa9, 0xab, 0x57, 0x7c, 0xd3, 0x86, 0x7d, 0x3a, 0x42, 0xb2, 0x87, 0x21, 0x14, 0x8e, 0x63, 0xa0, 0x2d, 0x58, 0x3e, 0x61, 0xb7, 0x2a, 0x62, 0x06,
0xd1, 0xcd, 0xe6, 0x12, 0xdd, 0x05, 0x80, 0x34, 0xee, 0x80, 0xf6, 0xa6, 0xb0, 0xeb, 0xfb, 0xb0, 0xbd, 0x55, 0x63, 0x9e, 0xa7, 0xd0, 0xf4, 0xb6, 0xfe, 0x3b, 0x0d, 0xd6, 0x3f, 0x31, 0xfc, 0xde,
0x20, 0xa5, 0xc9, 0xc8, 0x9e, 0x94, 0xda, 0x42, 0x72, 0xe3, 0x08, 0xd6, 0xe4, 0xfc, 0x13, 0x96, 0x69, 0xdb, 0xbe, 0x7c, 0x66, 0xfb, 0x10, 0xca, 0x81, 0xf9, 0x82, 0xd4, 0xac, 0xd2, 0x67, 0x5c,
0x43, 0x44, 0xbe, 0x39, 0x20, 0x81, 0x89, 0x1a, 0xb0, 0x20, 0xd3, 0x8a, 0x8c, 0xe0, 0x70, 0xc8, 0x63, 0x38, 0xc2, 0x60, 0xb7, 0x9f, 0x35, 0x7e, 0x19, 0xbc, 0x7c, 0xcf, 0x3a, 0x6b, 0xe0, 0x26,
0x2e, 0x0f, 0xc7, 0x9c, 0xae, 0xc3, 0x72, 0x87, 0x0c, 0x5b, 0x38, 0x8e, 0x52, 0x95, 0xf1, 0x4b, 0x4b, 0xf5, 0x7c, 0xa6, 0x54, 0x9f, 0x03, 0x48, 0xe1, 0x0e, 0x68, 0x7f, 0x06, 0xb9, 0x7e, 0x08,
0xa8, 0xb5, 0x5a, 0x4f, 0x13, 0xb2, 0xee, 0x01, 0x8b, 0xd4, 0x4e, 0x92, 0x4b, 0xe3, 0x5c, 0x35, 0x4b, 0x92, 0x9a, 0x8c, 0xcd, 0x69, 0xc5, 0x39, 0x00, 0xd7, 0x8f, 0x60, 0x5d, 0xee, 0x3f, 0x62,
0xcb, 0xea, 0xc7, 0x39, 0x0e, 0xbd, 0x0d, 0xf5, 0x80, 0x76, 0xf2, 0xc2, 0xf5, 0x80, 0xc6, 0x54, 0x55, 0x50, 0x54, 0xcc, 0x03, 0xe2, 0x1b, 0xa8, 0x01, 0x4b, 0xb2, 0x30, 0x4a, 0x77, 0x0b, 0x96,
0xc6, 0x01, 0xd4, 0xb9, 0xb1, 0x7c, 0x53, 0x27, 0xd8, 0x7a, 0x17, 0xf4, 0x84, 0x38, 0x11, 0x3e, 0xec, 0xfa, 0x73, 0xcc, 0xe1, 0xba, 0xac, 0xfa, 0x49, 0x4f, 0x83, 0xe3, 0xb0, 0xd8, 0xea, 0xbf,
0x15, 0x5c, 0x8d, 0x8d, 0xa5, 0x2c, 0x7d, 0x84, 0x25, 0x69, 0x2c, 0x71, 0x7c, 0x49, 0x7a, 0x1b, 0x86, 0x5a, 0xbb, 0xfd, 0x38, 0x46, 0xeb, 0x0e, 0xb0, 0x58, 0xeb, 0xc6, 0xb1, 0x34, 0x8e, 0x55,
0xc0, 0xa6, 0x9d, 0x13, 0x46, 0x4d, 0x2c, 0x6e, 0xe3, 0x22, 0xae, 0xd8, 0xf4, 0x89, 0x98, 0x40, 0x33, 0xcd, 0x41, 0x54, 0xa5, 0xd1, 0xdb, 0x50, 0xf7, 0x69, 0x37, 0x4b, 0xbc, 0xea, 0xd3, 0x08,
0x3f, 0x80, 0x79, 0xae, 0x9f, 0x95, 0x79, 0x0a, 0x84, 0xe2, 0xbb, 0x91, 0x5e, 0x01, 0x96, 0x0c, 0x4a, 0x3f, 0x80, 0x3a, 0x17, 0x96, 0x1b, 0x75, 0x8a, 0xac, 0xb7, 0xa1, 0x1a, 0x23, 0x27, 0xdc,
0xc6, 0xc7, 0xa0, 0xb7, 0x5a, 0x4f, 0x63, 0x3b, 0x8a, 0x80, 0x49, 0x81, 0x35, 0xbe, 0x86, 0x7a, 0xa7, 0x8c, 0x2b, 0x91, 0xb0, 0x94, 0x15, 0xc0, 0xa0, 0xa9, 0x8e, 0x28, 0x4e, 0x6e, 0xaa, 0x6f,
0x9c, 0x91, 0x78, 0xcd, 0x5d, 0x87, 0x52, 0x24, 0xae, 0xd4, 0x6e, 0xa1, 0x0f, 0x61, 0x5e, 0xb4, 0x02, 0x58, 0xb4, 0x2b, 0x9d, 0x9e, 0xcb, 0x58, 0xc2, 0x65, 0x8b, 0x3e, 0x12, 0x1b, 0xe8, 0x47,
0x64, 0x64, 0x04, 0xbd, 0x93, 0xb6, 0x59, 0xb6, 0x6b, 0x12, 0x69, 0x8d, 0x4f, 0x60, 0xc9, 0xc4, 0xb0, 0xc8, 0xf9, 0xb3, 0x46, 0x55, 0x91, 0x63, 0xb9, 0x35, 0x92, 0x27, 0xc0, 0x12, 0x41, 0xff,
0x22, 0x3c, 0x42, 0x71, 0x71, 0x93, 0x2e, 0xe3, 0xc4, 0x8c, 0xf1, 0xcf, 0x32, 0x54, 0x13, 0x01, 0x18, 0xaa, 0xed, 0xf6, 0xe3, 0x48, 0x8e, 0x3c, 0x91, 0x9f, 0xe3, 0x8c, 0xaf, 0xa0, 0x1e, 0xd5,
0x98, 0x53, 0xff, 0xcd, 0xdc, 0xa5, 0xde, 0x81, 0xba, 0xcd, 0x8b, 0x91, 0x8e, 0x3c, 0xfd, 0x1c, 0x54, 0x9e, 0x52, 0xea, 0x50, 0x08, 0xc9, 0x15, 0x3a, 0x6d, 0xf4, 0x21, 0x2c, 0x8a, 0xa1, 0x92,
0x00, 0x2b, 0xb8, 0x66, 0x27, 0x4b, 0x14, 0xf4, 0xff, 0xb0, 0xe8, 0x0e, 0x9d, 0x8e, 0xef, 0xbd, 0xf4, 0xa0, 0x77, 0x92, 0x32, 0xcb, 0x81, 0x53, 0xac, 0x30, 0xf3, 0x0d, 0x2c, 0x91, 0x98, 0x87,
0xa4, 0xf2, 0x72, 0xbc, 0xe0, 0x0e, 0x1d, 0xec, 0xbd, 0xa4, 0xf1, 0x65, 0x63, 0xfe, 0x92, 0x97, 0x87, 0x75, 0x48, 0xa4, 0x96, 0x22, 0x8e, 0xed, 0xe8, 0xff, 0x2a, 0x42, 0x25, 0xe6, 0x80, 0x19,
0x8d, 0x47, 0x90, 0x04, 0x40, 0x79, 0x6b, 0xbe, 0x14, 0x68, 0x72, 0x11, 0x31, 0x0e, 0x37, 0x16, 0xf6, 0xaf, 0xe7, 0x36, 0xf8, 0x0e, 0xd4, 0x2d, 0xde, 0x4e, 0x75, 0x65, 0xf4, 0xf3, 0x14, 0x5e,
0xa7, 0xc0, 0xee, 0x75, 0xa8, 0x3a, 0xe6, 0x05, 0x5b, 0x59, 0xc7, 0x1d, 0x3a, 0x8d, 0x8a, 0x08, 0xc6, 0x35, 0x2b, 0xde, 0x64, 0xa1, 0x6f, 0x43, 0xc9, 0x19, 0xd9, 0x5d, 0xcf, 0x7d, 0x41, 0xe5,
0x5c, 0xc7, 0xbc, 0xc0, 0xde, 0xcb, 0x67, 0x43, 0x07, 0x6d, 0xc2, 0x72, 0xdf, 0xa4, 0x41, 0x27, 0xf5, 0x7e, 0xc9, 0x19, 0xd9, 0xd8, 0x7d, 0x41, 0xa3, 0xeb, 0xd2, 0xe2, 0x05, 0xaf, 0x4b, 0x0f,
0x79, 0xf9, 0x07, 0x7e, 0xf9, 0xaf, 0xb3, 0xf9, 0x8f, 0xa2, 0x06, 0x80, 0xf1, 0x00, 0xaa, 0xed, 0x20, 0x9e, 0xc2, 0xe5, 0xbd, 0xff, 0x42, 0x69, 0x9f, 0x93, 0x88, 0x2a, 0x49, 0xa3, 0x34, 0x43,
0xd6, 0x2e, 0x8b, 0x1e, 0x56, 0x26, 0xe6, 0xf6, 0x6b, 0x15, 0xe6, 0x0e, 0x13, 0xc1, 0x26, 0x06, 0xf5, 0xd9, 0x80, 0x8a, 0x6d, 0x9c, 0xb3, 0x93, 0x75, 0x9d, 0x91, 0xdd, 0x28, 0x0b, 0xc7, 0xb5,
0x0c, 0x6a, 0xf5, 0xa4, 0x71, 0x8a, 0x2b, 0x98, 0xf6, 0x4d, 0x5d, 0xc1, 0x4a, 0x53, 0x5d, 0xc1, 0x8d, 0x73, 0xec, 0xbe, 0x78, 0x32, 0xb2, 0xd1, 0x16, 0xac, 0x0c, 0x0c, 0xea, 0x77, 0xe3, 0xe3,
0x8c, 0xff, 0x96, 0x60, 0xed, 0xc8, 0x3c, 0x27, 0x6f, 0xbe, 0x92, 0x2f, 0x94, 0x19, 0x9e, 0xc2, 0x0b, 0xe0, 0xe3, 0x8b, 0x3a, 0xdb, 0xff, 0x28, 0x1c, 0x61, 0xe8, 0xf7, 0xa0, 0xd2, 0x69, 0xb7,
0x0a, 0x3f, 0xf9, 0xbb, 0x09, 0x7b, 0x78, 0x82, 0x50, 0x63, 0x78, 0x62, 0x4b, 0x70, 0x9e, 0x11, 0x98, 0xf7, 0xb0, 0x46, 0x37, 0x63, 0xaf, 0x35, 0x58, 0x38, 0x8c, 0x39, 0x9b, 0x58, 0xb0, 0x54,
0xfd, 0x14, 0xea, 0x29, 0x40, 0x0d, 0x01, 0x68, 0x43, 0x21, 0x2a, 0x85, 0xd0, 0x38, 0xc3, 0x87, 0x5b, 0x8d, 0x0b, 0xa7, 0xb8, 0x44, 0x6a, 0xaf, 0xeb, 0x12, 0x59, 0x98, 0xe9, 0x12, 0xa9, 0xff,
0x7e, 0x02, 0xd5, 0xee, 0x29, 0xe9, 0x9e, 0x1d, 0x7a, 0xb6, 0x1b, 0xd0, 0xc6, 0x3c, 0x17, 0x73, 0xbf, 0x00, 0xeb, 0x47, 0xc6, 0x19, 0x79, 0xf3, 0x77, 0x91, 0x5c, 0x95, 0xe1, 0x31, 0xac, 0xf2,
0x5b, 0x21, 0xe6, 0x71, 0x44, 0x85, 0x93, 0x1c, 0x1c, 0x92, 0x25, 0x3e, 0x2e, 0x70, 0x7c, 0x0c, 0xc8, 0x6f, 0xc5, 0xe4, 0xe1, 0x05, 0x42, 0x9d, 0xc3, 0x63, 0x26, 0xc1, 0x59, 0x44, 0xf4, 0x73,
0x87, 0xc6, 0xd7, 0x1a, 0x40, 0xcc, 0x35, 0x01, 0x69, 0x8b, 0xe0, 0xc2, 0x8f, 0x61, 0x71, 0x8a, 0xa8, 0x27, 0x12, 0x6a, 0x90, 0x80, 0x36, 0x15, 0xa4, 0x12, 0x19, 0x1a, 0xa7, 0xf0, 0xd2, 0xbd,
0x1b, 0x7e, 0xc4, 0xc3, 0xb2, 0x3a, 0x87, 0x02, 0xd9, 0x81, 0x61, 0xdf, 0xc6, 0x17, 0x1a, 0xd4, 0xc2, 0xe2, 0x85, 0x7b, 0x05, 0x96, 0x92, 0x65, 0x7e, 0x5c, 0xe2, 0xf9, 0x31, 0x58, 0xea, 0x5f,
0x5a, 0x66, 0x60, 0x3e, 0xf3, 0x2c, 0xf2, 0x7c, 0xca, 0xac, 0x5c, 0xa0, 0xc7, 0x77, 0x0b, 0x2a, 0x6b, 0x00, 0x11, 0xd6, 0x94, 0x4c, 0x9b, 0x27, 0x2f, 0xfc, 0x14, 0x4a, 0x33, 0xcc, 0x28, 0x42,
0xec, 0x0c, 0xd2, 0xc0, 0x74, 0x06, 0xdc, 0xf6, 0x59, 0x1c, 0x4f, 0x18, 0xbf, 0x49, 0x94, 0x96, 0x1c, 0x56, 0xd5, 0x79, 0x2a, 0x90, 0x33, 0x24, 0xf6, 0xad, 0x7f, 0xa1, 0x41, 0xad, 0x6d, 0xf8,
0x1c, 0x35, 0x8b, 0x64, 0x83, 0x0d, 0x48, 0x6a, 0x50, 0x29, 0xdd, 0x87, 0x1a, 0x25, 0xe4, 0x6c, 0xc6, 0x13, 0xd7, 0x24, 0x4f, 0x67, 0xac, 0xca, 0x39, 0xa6, 0x94, 0x37, 0xa0, 0xcc, 0x62, 0x90,
0x9a, 0xb6, 0x88, 0xce, 0x18, 0xa3, 0x23, 0x99, 0x09, 0x92, 0xd9, 0x4b, 0x07, 0xc9, 0x26, 0x2c, 0xfa, 0x86, 0x3d, 0xe4, 0xb2, 0xcf, 0xe3, 0x68, 0x43, 0xf7, 0xa0, 0x2e, 0x53, 0x8d, 0x30, 0x0d,
0xc9, 0xa8, 0x08, 0x6b, 0x33, 0x1e, 0xb0, 0x65, 0x9c, 0x9d, 0x36, 0x7c, 0xa8, 0xcb, 0x6f, 0x11, 0x9d, 0xa2, 0xac, 0x3d, 0xa8, 0x9e, 0x44, 0x55, 0x7c, 0xd2, 0x45, 0x3d, 0x56, 0xec, 0x71, 0x02,
0xa5, 0x74, 0x42, 0xdc, 0xec, 0x81, 0x7e, 0x12, 0x17, 0x34, 0xe3, 0x6e, 0xfc, 0x89, 0xba, 0x07, 0x47, 0x7f, 0x00, 0x95, 0xd8, 0xcf, 0x09, 0x95, 0xb5, 0x01, 0x4b, 0xc7, 0x31, 0x3e, 0x65, 0x1c,
0xa7, 0x78, 0x8c, 0x47, 0x50, 0x4d, 0xfc, 0x1c, 0x53, 0x64, 0x34, 0x60, 0xe1, 0x38, 0xa1, 0xa7, 0x2c, 0xf5, 0x2f, 0x35, 0xa8, 0xc9, 0x54, 0x2b, 0x6e, 0x03, 0x4c, 0xc3, 0x5c, 0x03, 0xa2, 0xe8,
0x82, 0xc3, 0xa1, 0xf1, 0xa5, 0x06, 0x35, 0x99, 0x75, 0xc4, 0xb5, 0x82, 0x05, 0x1b, 0x0f, 0x06, 0xf3, 0x6f, 0xf4, 0xe3, 0xe4, 0x60, 0xea, 0x6d, 0xa5, 0x6f, 0x71, 0x22, 0xbc, 0x91, 0x4c, 0x24,
0x51, 0xff, 0xf0, 0x6f, 0xf4, 0xc3, 0x74, 0x87, 0xeb, 0x6d, 0xa5, 0x07, 0xb9, 0x10, 0x5e, 0x53, 0xdb, 0x3c, 0x57, 0xc9, 0xcf, 0x35, 0xa8, 0x06, 0x16, 0xe4, 0x25, 0xa7, 0x11, 0xbd, 0x6e, 0x08,
0xa7, 0xf2, 0x4e, 0x91, 0x3b, 0xe9, 0xe7, 0x1a, 0xe8, 0x61, 0x30, 0xf3, 0x38, 0x6a, 0xc4, 0xcf, 0x39, 0x82, 0x25, 0xfb, 0x73, 0x46, 0x3c, 0x1a, 0x64, 0x95, 0x22, 0x0e, 0x96, 0xe8, 0x27, 0x50,
0x24, 0xc2, 0x8e, 0x70, 0xc8, 0xfe, 0x9c, 0x13, 0x9f, 0x86, 0x00, 0x5b, 0xc6, 0xe1, 0x10, 0xfd, 0x0a, 0x3b, 0xcf, 0xe2, 0xd8, 0x50, 0x4a, 0x1c, 0x16, 0x87, 0x18, 0xfa, 0x7f, 0x35, 0x3e, 0x1c,
0x08, 0x16, 0xa3, 0x22, 0xbc, 0x3c, 0x12, 0x55, 0x52, 0x8b, 0xc5, 0x11, 0x87, 0xf1, 0x6f, 0x8d, 0xc4, 0xa4, 0xe7, 0x9e, 0x11, 0xef, 0x65, 0x62, 0x04, 0x73, 0x71, 0x77, 0xba, 0x1f, 0x93, 0x25,
0x77, 0x19, 0x31, 0xe9, 0x7a, 0xe7, 0xc4, 0x7f, 0x95, 0xea, 0xe5, 0x5c, 0xfe, 0x64, 0x3d, 0x4c, 0x67, 0x17, 0x1c, 0x22, 0xa0, 0xfb, 0x91, 0xb5, 0x8a, 0x63, 0x7b, 0x92, 0xa4, 0xb3, 0x45, 0x06,
0xd8, 0x32, 0xf9, 0x42, 0xc0, 0x95, 0x45, 0x0c, 0xe8, 0x61, 0xbc, 0x5b, 0xe5, 0x91, 0xe5, 0x59, 0xfd, 0x93, 0x98, 0x24, 0x25, 0xcf, 0x71, 0xa5, 0x73, 0xdf, 0xed, 0xbb, 0xb0, 0x9a, 0xf1, 0x0e,
0x3a, 0xd8, 0xe2, 0x0d, 0xfd, 0xa3, 0x68, 0x49, 0xa5, 0xd7, 0x71, 0xad, 0x0d, 0xe4, 0xad, 0xfb, 0x54, 0x07, 0xf8, 0xd8, 0xe9, 0xb9, 0xf6, 0x70, 0x40, 0x7c, 0xb2, 0x32, 0x87, 0xaa, 0x50, 0x7a,
0xb0, 0x92, 0x8b, 0x0e, 0x54, 0x07, 0xf8, 0xd8, 0xed, 0x7a, 0xce, 0xa0, 0x4f, 0x02, 0xb2, 0x3c, 0x18, 0xac, 0xb4, 0xd6, 0xff, 0x6a, 0x50, 0x61, 0x0e, 0x71, 0x24, 0x5e, 0xe2, 0xd0, 0x10, 0x10,
0x83, 0x74, 0x58, 0x7c, 0x1c, 0x8e, 0xb4, 0xdd, 0xff, 0xd4, 0xa0, 0xca, 0x02, 0xe2, 0x48, 0x3c, 0x1f, 0x35, 0xd8, 0x43, 0xd7, 0x09, 0x47, 0x83, 0xe8, 0x83, 0x31, 0xa9, 0x23, 0x0b, 0x2a, 0x55,
0xe9, 0xa1, 0x01, 0x20, 0xde, 0xb3, 0x70, 0x06, 0x9e, 0x1b, 0xf5, 0x18, 0xd1, 0x07, 0x23, 0x00, 0xd0, 0xbc, 0x33, 0x06, 0x23, 0x05, 0xae, 0xcf, 0x21, 0x9b, 0x73, 0x64, 0xe5, 0xf5, 0xa9, 0xd5,
0x21, 0x4f, 0x2a, 0x5d, 0xd0, 0xbc, 0x37, 0x82, 0x23, 0x43, 0x6e, 0xcc, 0x20, 0x87, 0x6b, 0x64, 0x7b, 0x1e, 0x74, 0x23, 0x13, 0x38, 0xa6, 0x40, 0x03, 0x8e, 0xa9, 0x89, 0xa3, 0x5c, 0x88, 0xb1,
0x95, 0xc6, 0x73, 0xbb, 0x7b, 0x16, 0x16, 0x66, 0x63, 0x34, 0x66, 0x48, 0x43, 0x8d, 0x99, 0xd6, 0x54, 0xe0, 0x60, 0xfa, 0x1c, 0xfa, 0x0c, 0xd6, 0xd8, 0xdd, 0x3b, 0x1c, 0x01, 0x04, 0x0c, 0x5b,
0xa5, 0x1c, 0x88, 0xfe, 0x56, 0x18, 0x60, 0xc6, 0x0c, 0xfa, 0x0c, 0x56, 0xd9, 0x25, 0x3e, 0xea, 0xe3, 0x19, 0x66, 0x80, 0x2f, 0xc8, 0xd2, 0x80, 0x6a, 0xfc, 0x21, 0x10, 0xa9, 0x5e, 0x27, 0x14,
0x25, 0x84, 0x0a, 0x77, 0x47, 0x2b, 0xcc, 0x11, 0x5f, 0x52, 0xa5, 0x09, 0x7a, 0xf2, 0x45, 0x11, 0x6f, 0x95, 0xcd, 0x77, 0xa7, 0xc2, 0x85, 0x2c, 0xf6, 0x61, 0x81, 0xb7, 0xc7, 0x48, 0xe5, 0xfd,
0xa9, 0x9e, 0x39, 0x14, 0x8f, 0x9e, 0xcd, 0x77, 0x27, 0xd2, 0x45, 0x2a, 0xf6, 0x61, 0x8e, 0xdf, 0xf1, 0x47, 0xbf, 0xe6, 0xa4, 0x01, 0x85, 0x3e, 0x87, 0x7e, 0x03, 0xcb, 0xa9, 0xe7, 0x16, 0xf4,
0x14, 0x90, 0x2a, 0xfa, 0x93, 0xaf, 0x87, 0xcd, 0x71, 0x9d, 0x0e, 0x63, 0x06, 0xfd, 0x0a, 0x96, 0x9e, 0x82, 0xa4, 0xfa, 0xe1, 0xac, 0xb9, 0x9d, 0x07, 0x34, 0xae, 0x97, 0xf8, 0x93, 0x84, 0x52,
0x32, 0xef, 0x36, 0xe8, 0x3d, 0x85, 0x48, 0xf5, 0x0b, 0x5c, 0x73, 0xab, 0x08, 0x69, 0xd2, 0x2f, 0x2f, 0x8a, 0x67, 0x15, 0xa5, 0x5e, 0x54, 0x6f, 0x1b, 0xfa, 0x1c, 0xea, 0x43, 0x3d, 0x39, 0x69,
0xc9, 0xb7, 0x0d, 0xa5, 0x5f, 0x14, 0xef, 0x33, 0x4a, 0xbf, 0xa8, 0x1e, 0x49, 0x8c, 0x19, 0xd4, 0x41, 0x5b, 0x0a, 0x64, 0xe5, 0xf0, 0xb9, 0xf9, 0x5e, 0x0e, 0xc8, 0x90, 0x91, 0x0d, 0x2b, 0xe9,
0x83, 0x7a, 0xba, 0x65, 0x83, 0x36, 0x15, 0xcc, 0xca, 0x2e, 0x76, 0xf3, 0xbd, 0x02, 0x94, 0x91, 0x81, 0x3a, 0xda, 0x9e, 0x48, 0x20, 0x19, 0x2f, 0xef, 0xe7, 0x82, 0x0d, 0xd9, 0xbd, 0xe4, 0x5e,
0x22, 0x07, 0x96, 0xb3, 0x9d, 0x79, 0xb4, 0x35, 0x56, 0x40, 0xfa, 0xbc, 0xbc, 0x5f, 0x88, 0x36, 0x9c, 0x19, 0xe8, 0xa2, 0x1d, 0x35, 0x99, 0x71, 0x93, 0xe6, 0xe6, 0x6e, 0x6e, 0xf8, 0x90, 0x35,
0x52, 0xf7, 0x8a, 0x47, 0x71, 0xae, 0x33, 0x8c, 0xb6, 0xd5, 0x62, 0x46, 0xb5, 0xac, 0x9b, 0x3b, 0x81, 0xd5, 0xcc, 0x80, 0x16, 0xbd, 0x3f, 0x89, 0x4e, 0x6a, 0xd8, 0xd1, 0x9c, 0x3e, 0x42, 0xd6,
0x85, 0xe9, 0x23, 0xd5, 0x04, 0x56, 0x72, 0x9d, 0x5e, 0xf4, 0xfe, 0x38, 0x39, 0x99, 0xbe, 0x4f, 0xe7, 0xd0, 0x17, 0xa2, 0x4c, 0xa8, 0x86, 0x9e, 0xe8, 0xae, 0x9a, 0xdb, 0x84, 0x69, 0x6d, 0xb3,
0x73, 0x72, 0x2f, 0xda, 0x98, 0x41, 0x5f, 0x88, 0x34, 0xa1, 0xea, 0x9e, 0xa2, 0xfb, 0x6a, 0x6d, 0x75, 0x11, 0x94, 0xf0, 0xac, 0xaf, 0x78, 0x8a, 0x57, 0x0c, 0x0e, 0xd3, 0xf9, 0x29, 0xa0, 0x37,
0x63, 0xda, 0xbe, 0xcd, 0xdd, 0xcb, 0xb0, 0x44, 0x6b, 0x7d, 0xcd, 0x21, 0x5e, 0xd1, 0x81, 0xcc, 0x7e, 0x22, 0xda, 0xbc, 0x7b, 0x01, 0x8c, 0x50, 0x00, 0x37, 0xfd, 0x32, 0x12, 0xa4, 0xab, 0xdd,
0xe2, 0x53, 0x28, 0x6f, 0x74, 0x6b, 0xb5, 0x79, 0xff, 0x12, 0x1c, 0x91, 0x01, 0x5e, 0xf6, 0x89, 0xa9, 0xce, 0x39, 0x5b, 0xae, 0xfa, 0x14, 0x96, 0x53, 0x17, 0x01, 0x65, 0xfc, 0xab, 0x2f, 0x0b,
0x25, 0x84, 0xab, 0x9d, 0x89, 0xc1, 0x39, 0x1d, 0x56, 0x7d, 0x0a, 0x4b, 0x99, 0x3b, 0x91, 0xf2, 0x39, 0x92, 0x4b, 0xaa, 0x62, 0xa2, 0x31, 0x41, 0xa6, 0xa8, 0xaa, 0xcd, 0xed, 0x3c, 0xa0, 0xc1,
0xfc, 0xab, 0xef, 0x4d, 0x05, 0xc0, 0x25, 0x93, 0x31, 0xd1, 0x88, 0x43, 0xa6, 0xc8, 0xaa, 0xcd, 0x41, 0x5a, 0x7f, 0x2f, 0x42, 0x29, 0xe8, 0x74, 0xae, 0xa0, 0xaa, 0x5d, 0x41, 0x99, 0xf9, 0x14,
0xad, 0x22, 0xa4, 0xe1, 0x42, 0x76, 0xff, 0x56, 0x86, 0xc5, 0xb0, 0xd2, 0xb9, 0x86, 0xac, 0x76, 0x96, 0x53, 0xd3, 0x45, 0xa5, 0x76, 0xd5, 0x13, 0xc8, 0x69, 0xa6, 0xfb, 0x04, 0x6a, 0x89, 0x71,
0x0d, 0x69, 0xe6, 0x53, 0x58, 0xca, 0x34, 0x5a, 0x95, 0xde, 0x55, 0x37, 0x63, 0x27, 0x6d, 0xdd, 0x21, 0x7a, 0x77, 0x5c, 0xa1, 0x49, 0x67, 0xeb, 0xc9, 0x84, 0xf7, 0xee, 0xfd, 0xea, 0x6e, 0xdf,
0x27, 0x50, 0x4b, 0x75, 0x4e, 0xd1, 0xbb, 0xa3, 0x12, 0x4d, 0x16, 0xad, 0xc7, 0x0b, 0xde, 0x7b, 0xf2, 0x4f, 0x47, 0xc7, 0xec, 0xcf, 0xae, 0x00, 0xfd, 0xae, 0xe5, 0xca, 0xaf, 0xdd, 0x40, 0x41,
0xf0, 0x8b, 0xfb, 0x3d, 0x3b, 0x38, 0x1d, 0x1e, 0xb3, 0x3f, 0x3b, 0x82, 0xf4, 0xdb, 0xb6, 0x27, 0xbb, 0x1c, 0x7b, 0x97, 0xb1, 0x19, 0x1e, 0x1f, 0x2f, 0xf2, 0xd5, 0xbd, 0x6f, 0x02, 0x00, 0x00,
0xbf, 0x76, 0x42, 0x07, 0xed, 0x70, 0xee, 0x1d, 0xa6, 0x66, 0x70, 0x7c, 0x3c, 0xcf, 0x47, 0x0f, 0xff, 0xff, 0xf1, 0x4d, 0xb3, 0x0c, 0x46, 0x24, 0x00, 0x00,
0xfe, 0x17, 0x00, 0x00, 0xff, 0xff, 0xf3, 0x24, 0xb8, 0x96, 0x8f, 0x24, 0x00, 0x00,
} }
// Reference imports to suppress errors if they are not otherwise used. // Reference imports to suppress errors if they are not otherwise used.